Skip to content
Merged
Show file tree
Hide file tree
Changes from all commits
Commits
File filter

Filter by extension

Filter by extension

Conversations
Failed to load comments.
Loading
Jump to
Jump to file
Failed to load files.
Loading
Diff view
Diff view
Original file line number Diff line number Diff line change
Expand Up @@ -19,6 +19,7 @@
package org.apache.hadoop.hdds.protocol;

import com.google.common.base.Preconditions;
import com.google.common.base.Strings;
import org.apache.hadoop.classification.InterfaceAudience;
import org.apache.hadoop.classification.InterfaceStability;
import org.apache.hadoop.hdds.protocol.proto.HddsProtos;
Expand Down Expand Up @@ -193,12 +194,12 @@ public static DatanodeDetails getFromProtoBuf(
builder.addPort(newPort(
Port.Name.valueOf(port.getName().toUpperCase()), port.getValue()));
}
if (datanodeDetailsProto.hasNetworkLocation()) {
builder.setNetworkLocation(datanodeDetailsProto.getNetworkLocation());
}
if (datanodeDetailsProto.hasNetworkName()) {
builder.setNetworkName(datanodeDetailsProto.getNetworkName());
}
if (datanodeDetailsProto.hasNetworkLocation()) {
builder.setNetworkLocation(datanodeDetailsProto.getNetworkLocation());
}
return builder.build();
}

Expand All @@ -219,8 +220,12 @@ public HddsProtos.DatanodeDetailsProto getProtoBufMessage() {
if (certSerialId != null) {
builder.setCertSerialId(certSerialId);
}
builder.setNetworkLocation(getNetworkLocation());
builder.setNetworkName(getNetworkName());
if (!Strings.isNullOrEmpty(getNetworkName())) {
builder.setNetworkName(getNetworkName());
}
if (!Strings.isNullOrEmpty(getNetworkLocation())) {
builder.setNetworkLocation(getNetworkLocation());
}

for (Port port : ports) {
builder.addPorts(HddsProtos.Port.newBuilder()
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -128,6 +128,10 @@ public EndpointStateMachine.EndPointStates call() throws Exception {
datanodeDetails.setHostName(response.getHostname());
datanodeDetails.setIpAddress(response.getIpAddress());
}
if (response.hasNetworkName() && response.hasNetworkLocation()) {
datanodeDetails.setNetworkName(response.getNetworkName());
datanodeDetails.setNetworkLocation(response.getNetworkLocation());
}
EndpointStateMachine.EndPointStates nextState =
rpcEndPoint.getState().getNextState();
rpcEndPoint.setState(nextState);
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -17,7 +17,8 @@
*/
package org.apache.hadoop.ozone.protocol.commands;

import com.google.common.base.Preconditions;
import com.google.common.base.Strings;
import org.apache.hadoop.hdds.protocol.DatanodeDetails;
import org.apache.hadoop.hdds.protocol.proto
.StorageContainerDatanodeProtocolProtos.SCMRegisteredResponseProto;
import org.apache.hadoop.hdds.protocol.proto
Expand All @@ -28,23 +29,15 @@
* Response to Datanode Register call.
*/
public class RegisteredCommand {
private String datanodeUUID;
private String clusterID;
private ErrorCode error;
private String hostname;
private String ipAddress;
private DatanodeDetails datanode;

public RegisteredCommand(final ErrorCode error, final String datanodeUUID,
public RegisteredCommand(final ErrorCode error, final DatanodeDetails node,
final String clusterID) {
this(error, datanodeUUID, clusterID, null, null);
}
public RegisteredCommand(final ErrorCode error, final String datanodeUUID,
final String clusterID, final String hostname, final String ipAddress) {
this.datanodeUUID = datanodeUUID;
this.datanode = node;
this.clusterID = clusterID;
this.error = error;
this.hostname = hostname;
this.ipAddress = ipAddress;
}

/**
Expand All @@ -57,12 +50,12 @@ public static Builder newBuilder() {
}

/**
* Returns datanode UUID.
* Returns datanode.
*
* @return - Datanode ID.
* @return - Datanode.
*/
public String getDatanodeUUID() {
return datanodeUUID;
public DatanodeDetails getDatanode() {
return datanode;
}

/**
Expand All @@ -83,79 +76,54 @@ public ErrorCode getError() {
return error;
}

/**
* Returns the hostname.
*
* @return - hostname
*/
public String getHostName() {
return hostname;
}

/**
* Returns the ipAddress of the dataNode.
*/
public String getIpAddress() {
return ipAddress;
}

/**
* Gets the protobuf message of this object.
*
* @return A protobuf message.
*/
public byte[] getProtoBufMessage() {
public SCMRegisteredResponseProto getProtoBufMessage() {
SCMRegisteredResponseProto.Builder builder =
SCMRegisteredResponseProto.newBuilder()
// TODO : Fix this later when we have multiple SCM support.
// .setAddressList(addressList)
.setClusterID(this.clusterID)
.setDatanodeUUID(this.datanodeUUID)
.setDatanodeUUID(this.datanode.getUuidString())
.setErrorCode(this.error);
if (hostname != null && ipAddress != null) {
builder.setHostname(hostname).setIpAddress(ipAddress);
if (!Strings.isNullOrEmpty(datanode.getHostName())) {
builder.setHostname(datanode.getHostName());
}
if (!Strings.isNullOrEmpty(datanode.getIpAddress())) {
builder.setIpAddress(datanode.getIpAddress());
}
if (!Strings.isNullOrEmpty(datanode.getNetworkName())) {
builder.setNetworkName(datanode.getNetworkName());
}
return builder.build().toByteArray();
if (!Strings.isNullOrEmpty(datanode.getNetworkLocation())) {
builder.setNetworkLocation(datanode.getNetworkLocation());
}

return builder.build();
}

/**
* A builder class to verify all values are sane.
*/
public static class Builder {
private String datanodeUUID;
private DatanodeDetails datanode;
private String clusterID;
private ErrorCode error;
private String ipAddress;
private String hostname;

/**
* sets UUID.
* sets datanode details.
*
* @param dnUUID - datanode UUID
* @param node - datanode details
* @return Builder
*/
public Builder setDatanodeUUID(String dnUUID) {
this.datanodeUUID = dnUUID;
public Builder setDatanode(DatanodeDetails node) {
this.datanode = node;
return this;
}

/**
* Create this object from a Protobuf message.
*
* @param response - RegisteredCmdResponseProto
* @return RegisteredCommand
*/
public RegisteredCommand getFromProtobuf(SCMRegisteredResponseProto
response) {
Preconditions.checkNotNull(response);
if (response.hasHostname() && response.hasIpAddress()) {
return new RegisteredCommand(response.getErrorCode(),
response.getDatanodeUUID(), response.getClusterID(),
response.getHostname(), response.getIpAddress());
} else {
return new RegisteredCommand(response.getErrorCode(),
response.getDatanodeUUID(), response.getClusterID());
}
}

/**
* Sets cluster ID.
*
Expand All @@ -178,38 +146,19 @@ public Builder setErrorCode(ErrorCode errorCode) {
return this;
}

/**
* sets the hostname.
*/
public Builder setHostname(String host) {
this.hostname = host;
return this;
}

public Builder setIpAddress(String ipAddr) {
this.ipAddress = ipAddr;
return this;
}

/**
* Build the command object.
*
* @return RegisteredCommand
*/
public RegisteredCommand build() {
if ((this.error == ErrorCode.success) && (this.datanodeUUID == null
|| this.datanodeUUID.isEmpty()) || (this.clusterID == null
|| this.clusterID.isEmpty())) {
if ((this.error == ErrorCode.success) && (this.datanode == null
|| Strings.isNullOrEmpty(this.datanode.getUuidString())
|| Strings.isNullOrEmpty(this.clusterID))) {
throw new IllegalArgumentException("On success, RegisteredCommand "
+ "needs datanodeUUID and ClusterID.");
}
if (hostname != null && ipAddress != null) {
return new RegisteredCommand(this.error, this.datanodeUUID,
this.clusterID, this.hostname, this.ipAddress);
} else {
return new RegisteredCommand(this.error, this.datanodeUUID,
this.clusterID);
}
return new RegisteredCommand(this.error, this.datanode, this.clusterID);
}
}
}
Original file line number Diff line number Diff line change
Expand Up @@ -70,6 +70,8 @@ message SCMRegisteredResponseProto {
optional SCMNodeAddressList addressList = 4;
optional string hostname = 5;
optional string ipAddress = 6;
optional string networkName = 7;
optional string networkLocation = 8;
}

/**
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -173,11 +173,19 @@ void processNodeReport(DatanodeDetails datanodeDetails,
List<SCMCommand> getCommandQueue(UUID dnID);

/**
* Given datanode host address, returns the DatanodeDetails for the
* node.
* Given datanode uuid, returns the DatanodeDetails for the node.
*
* @param address node host address
* @param uuid datanode uuid
* @return the given datanode, or null if not found
*/
DatanodeDetails getNode(String address);
DatanodeDetails getNodeByUuid(String uuid);

/**
* Given datanode address(Ipaddress or hostname), returns the DatanodeDetails
* for the node.
*
* @param address datanode address
* @return the given datanode, or null if not found
*/
DatanodeDetails getNodeByAddress(String address);
}
Loading