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 @@ -35,12 +35,15 @@
* - UUID of the DataNode.
* - IP and Hostname details.
* - Port details to which the DataNode will be listening.
* and may also include some extra info like:
* - version of the DataNode
* - setup time etc.
*/
@InterfaceAudience.Private
@InterfaceStability.Evolving
public class DatanodeDetails extends NodeImpl implements
Comparable<DatanodeDetails> {
/**
/**
* DataNode's unique identifier in the cluster.
*/
private final UUID uuid;
Expand Down Expand Up @@ -225,17 +228,34 @@ public static DatanodeDetails getFromProtoBuf(
if (datanodeDetailsProto.hasNetworkLocation()) {
builder.setNetworkLocation(datanodeDetailsProto.getNetworkLocation());
}
if (datanodeDetailsProto.hasVersion()) {
builder.setVersion(datanodeDetailsProto.getVersion());
return builder.build();
}

/**
* Returns a ExtendedDatanodeDetails from the protocol buffers.
*
* @param extendedDetailsProto - protoBuf Message
* @return DatanodeDetails
*/
public static DatanodeDetails getFromProtoBuf(
HddsProtos.ExtendedDatanodeDetailsProto extendedDetailsProto) {
DatanodeDetails.Builder builder = newBuilder();
if (extendedDetailsProto.hasDatanodeDetails()) {
DatanodeDetails datanodeDetails = getFromProtoBuf(
extendedDetailsProto.getDatanodeDetails());
builder.setDatanodeDetails(datanodeDetails);
}
if (extendedDetailsProto.hasVersion()) {
builder.setVersion(extendedDetailsProto.getVersion());
}
if (datanodeDetailsProto.hasSetupTime()) {
builder.setSetupTime(datanodeDetailsProto.getSetupTime());
if (extendedDetailsProto.hasSetupTime()) {
builder.setSetupTime(extendedDetailsProto.getSetupTime());
}
if (datanodeDetailsProto.hasRevision()) {
builder.setRevision(datanodeDetailsProto.getRevision());
if (extendedDetailsProto.hasRevision()) {
builder.setRevision(extendedDetailsProto.getRevision());
}
if (datanodeDetailsProto.hasBuildDate()) {
builder.setBuildDate(datanodeDetailsProto.getBuildDate());
if (extendedDetailsProto.hasBuildDate()) {
builder.setBuildDate(extendedDetailsProto.getBuildDate());
}
return builder.build();
}
Expand Down Expand Up @@ -279,20 +299,34 @@ public HddsProtos.DatanodeDetailsProto getProtoBufMessage() {
.build());
}

return builder.build();
}

/**
* Returns a ExtendedDatanodeDetails protobuf message from a datanode ID.
* @return HddsProtos.ExtendedDatanodeDetailsProto
*/
public HddsProtos.ExtendedDatanodeDetailsProto getExtendedProtoBufMessage() {
HddsProtos.DatanodeDetailsProto datanodeDetailsProto = getProtoBufMessage();

HddsProtos.ExtendedDatanodeDetailsProto.Builder extendedBuilder =
HddsProtos.ExtendedDatanodeDetailsProto.newBuilder()
.setDatanodeDetails(datanodeDetailsProto);

if (!Strings.isNullOrEmpty(getVersion())) {
builder.setVersion(getVersion());
extendedBuilder.setVersion(getVersion());
}

builder.setSetupTime(getSetupTime());
extendedBuilder.setSetupTime(getSetupTime());

if (!Strings.isNullOrEmpty(getRevision())) {
builder.setRevision(getRevision());
extendedBuilder.setRevision(getRevision());
}
if (!Strings.isNullOrEmpty(getBuildDate())) {
builder.setBuildDate(getBuildDate());
extendedBuilder.setBuildDate(getBuildDate());
}

return builder.build();
return extendedBuilder.build();
}

@Override
Expand Down Expand Up @@ -357,6 +391,27 @@ private Builder() {
ports = new ArrayList<>();
}

/**
* Initialize with DatanodeDetails.
*
* @param details DatanodeDetails
* @return DatanodeDetails.Builder
*/
public Builder setDatanodeDetails(DatanodeDetails details) {
this.id = details.getUuid();
this.ipAddress = details.getIpAddress();
this.hostName = details.getHostName();
this.networkName = details.getNetworkName();
this.networkLocation = details.getNetworkLocation();
this.ports = details.getPorts();
this.certSerialId = details.getCertSerialId();
this.version = details.getVersion();
this.setupTime = details.getSetupTime();
this.revision = details.getRevision();
this.buildDate = details.getBuildDate();
return this;
}

/**
* Sets the DatanodeUuid.
*
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -44,7 +44,7 @@ public static DatanodeDetails randomDatanodeDetails() {
* @return DatanodeDetails
*/
public static DatanodeDetails createDatanodeDetails(String hostname,
String loc) {
String loc) {
Random random = ThreadLocalRandom.current();
String ipAddress = random.nextInt(256)
+ "." + random.nextInt(256)
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -90,7 +90,7 @@ public class DatanodeStateMachine implements Closeable {

/**
* Constructs a a datanode state machine.
* @param datanodeDetails - DatanodeDetails used to identify a datanode
* @param datanodeDetails - DatanodeDetails used to identify a datanode
* @param conf - Configuration.
* @param certClient - Datanode Certificate client, required if security is
* enabled
Expand Down Expand Up @@ -135,7 +135,7 @@ public DatanodeStateMachine(DatanodeDetails datanodeDetails,
dnConf.getReplicationMaxStreams());

// When we add new handlers just adding a new handler here should do the
// trick.
// trick.
commandDispatcher = CommandDispatcher.newBuilder()
.addHandler(new CloseContainerCommandHandler())
.addHandler(new DeleteBlocksCommandHandler(container.getContainerSet(),
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -22,7 +22,7 @@
import org.apache.hadoop.hdds.conf.ConfigurationSource;
import org.apache.hadoop.hdds.protocol.DatanodeDetails;
import org.apache.hadoop.hdds.protocol.proto
.StorageContainerDatanodeProtocolProtos.PipelineReportsProto;
.StorageContainerDatanodeProtocolProtos.PipelineReportsProto;
import org.apache.hadoop.ozone.container.common.statemachine
.EndpointStateMachine;
import org.apache.hadoop.hdds.protocol.proto
Expand Down Expand Up @@ -119,8 +119,8 @@ public EndpointStateMachine.EndPointStates call() throws Exception {
datanodeContainerManager.getPipelineReport();
// TODO : Add responses to the command Queue.
SCMRegisteredResponseProto response = rpcEndPoint.getEndPoint()
.register(datanodeDetails.getProtoBufMessage(), nodeReport,
containerReport, pipelineReportsProto);
.register(datanodeDetails.getExtendedProtoBufMessage(),
nodeReport, containerReport, pipelineReportsProto);
Preconditions.checkState(UUID.fromString(response.getDatanodeUUID())
.equals(datanodeDetails.getUuid()),
"Unexpected datanode ID in the response.");
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -17,9 +17,9 @@
package org.apache.hadoop.ozone.protocol;

import org.apache.hadoop.hdds.annotation.InterfaceAudience;
import org.apache.hadoop.hdds.protocol.proto.HddsProtos.DatanodeDetailsProto;
import org.apache.hadoop.hdds.protocol.proto.HddsProtos.ExtendedDatanodeDetailsProto;
import org.apache.hadoop.hdds.protocol.proto
.StorageContainerDatanodeProtocolProtos.PipelineReportsProto;
.StorageContainerDatanodeProtocolProtos.PipelineReportsProto;
import org.apache.hadoop.hdds.protocol.proto
.StorageContainerDatanodeProtocolProtos.SCMHeartbeatRequestProto;
import org.apache.hadoop.hdds.protocol.proto
Expand Down Expand Up @@ -73,15 +73,15 @@ SCMHeartbeatResponseProto sendHeartbeat(SCMHeartbeatRequestProto heartbeat)

/**
* Register Datanode.
* @param datanodeDetails - Datanode Details.
* @param extendedDatanodeDetailsProto - extended Datanode Details.
* @param nodeReport - Node Report.
* @param containerReportsRequestProto - Container Reports.
* @return SCM Command.
*/
SCMRegisteredResponseProto register(
DatanodeDetailsProto datanodeDetails,
NodeReportProto nodeReport,
ContainerReportsProto containerReportsRequestProto,
PipelineReportsProto pipelineReports) throws IOException;
ExtendedDatanodeDetailsProto extendedDatanodeDetailsProto,
NodeReportProto nodeReport,
ContainerReportsProto containerReportsRequestProto,
PipelineReportsProto pipelineReports) throws IOException;

}
Original file line number Diff line number Diff line change
Expand Up @@ -18,9 +18,10 @@

import com.google.protobuf.RpcController;
import com.google.protobuf.ServiceException;
import org.apache.hadoop.hdds.protocol.proto.HddsProtos.DatanodeDetailsProto;
import org.apache.hadoop.hdds.protocol.proto.HddsProtos
.ExtendedDatanodeDetailsProto;
import org.apache.hadoop.hdds.protocol.proto
.StorageContainerDatanodeProtocolProtos.PipelineReportsProto;
.StorageContainerDatanodeProtocolProtos.PipelineReportsProto;
import org.apache.hadoop.hdds.protocol.proto
.StorageContainerDatanodeProtocolProtos.ContainerReportsProto;

Expand Down Expand Up @@ -153,20 +154,21 @@ public SCMHeartbeatResponseProto sendHeartbeat(
/**
* Register Datanode.
*
* @param datanodeDetailsProto - Datanode Details
* @param extendedDatanodeDetailsProto - extended Datanode Details
* @param nodeReport - Node Report.
* @param containerReportsRequestProto - Container Reports.
* @return SCM Command.
*/
@Override
public SCMRegisteredResponseProto register(
DatanodeDetailsProto datanodeDetailsProto, NodeReportProto nodeReport,
ExtendedDatanodeDetailsProto extendedDatanodeDetailsProto,
NodeReportProto nodeReport,
ContainerReportsProto containerReportsRequestProto,
PipelineReportsProto pipelineReportsProto)
throws IOException {
SCMRegisterRequestProto.Builder req =
SCMRegisterRequestProto.newBuilder();
req.setDatanodeDetails(datanodeDetailsProto);
req.setExtendedDatanodeDetails(extendedDatanodeDetailsProto);
req.setContainerReport(containerReportsRequestProto);
req.setPipelineReports(pipelineReportsProto);
req.setNodeReport(nodeReport);
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -68,7 +68,7 @@ public SCMRegisteredResponseProto register(
.getContainerReport();
NodeReportProto dnNodeReport = request.getNodeReport();
PipelineReportsProto pipelineReport = request.getPipelineReports();
return impl.register(request.getDatanodeDetails(), dnNodeReport,
return impl.register(request.getExtendedDatanodeDetails(), dnNodeReport,
containerRequestProto, pipelineReport);

}
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -17,6 +17,8 @@
package org.apache.hadoop.ozone.container.common;

import com.google.common.base.Preconditions;
import org.apache.hadoop.hdds.protocol.proto.HddsProtos
.ExtendedDatanodeDetailsProto;
import org.apache.hadoop.hdds.protocol.proto.StorageContainerDatanodeProtocolProtos;
import org.apache.hadoop.hdds.protocol.proto
.StorageContainerDatanodeProtocolProtos.ContainerReplicaProto;
Expand Down Expand Up @@ -211,17 +213,20 @@ private void sleepIfNeeded() {
/**
* Register Datanode.
*
* @param datanodeDetailsProto DatanodDetailsProto.
* @param extendedDatanodeDetailsProto ExtendedDatanodDetailsProto.
* @return SCM Command.
*/
@Override
public StorageContainerDatanodeProtocolProtos
.SCMRegisteredResponseProto register(
DatanodeDetailsProto datanodeDetailsProto, NodeReportProto nodeReport,
ExtendedDatanodeDetailsProto extendedDatanodeDetailsProto,
NodeReportProto nodeReport,
ContainerReportsProto containerReportsRequestProto,
PipelineReportsProto pipelineReportsProto)
throws IOException {
rpcCount.incrementAndGet();
DatanodeDetailsProto datanodeDetailsProto =
extendedDatanodeDetailsProto.getDatanodeDetails();
updateNodeReport(datanodeDetailsProto, nodeReport);
updateContainerReport(containerReportsRequestProto, datanodeDetailsProto);
sleepIfNeeded();
Expand Down
17 changes: 13 additions & 4 deletions hadoop-hdds/interface-client/src/main/proto/hdds.proto
Original file line number Diff line number Diff line change
Expand Up @@ -43,14 +43,23 @@ message DatanodeDetailsProto {
// network name, can be Ip address or host name, depends
optional string networkName = 6;
optional string networkLocation = 7; // Network topology location
optional string version = 8; // Datanode version
optional int64 setupTime = 9;
optional string revision = 10;
optional string buildDate = 11;
// TODO(runzhiwang): when uuid is gone, specify 1 as the index of uuid128 and mark as required
optional UUID uuid128 = 100; // UUID with 128 bits assigned to the Datanode.
}

/**
The DatanodeDetailsProto is a basic type that will be shared by many Proto,
to reduce the number of fields transfered by ProtoBuf, only need to extend
DatanodeDetailsProto for registering DataNode with SCM and Recon.
*/
message ExtendedDatanodeDetailsProto {
required DatanodeDetailsProto datanodeDetails = 1;
optional string version = 2;
optional int64 setupTime = 3;
optional string revision = 4;
optional string buildDate = 5;
}

/**
Proto message encapsulating information required to uniquely identify a
OzoneManager.
Expand Down
40 changes: 30 additions & 10 deletions hadoop-hdds/interface-client/src/main/resources/proto.lock
Original file line number Diff line number Diff line change
Expand Up @@ -1530,23 +1530,43 @@
"name": "networkLocation",
"type": "string"
},
{
"id": 8,
"name": "version",
"type": "string"
},
{
"id": 9,
"name": "setupTime",
"type": "int64"
},
{
"id": 100,
"name": "uuid128",
"type": "UUID"
}
]
},
{
"name": "ExtendedDatanodeDetailsProto",
"fields": [
{
"id": 1,
"name": "datanodeDetails",
"type": "DatanodeDetailsProto"
},
{
"id": 2,
"name": "version",
"type": "string"
},
{
"id": 3,
"name": "setupTime",
"type": "int64"
},
{
"id": 4,
"name": "revision",
"type": "string"
},
{
"id": 5,
"name": "buildDate",
"type": "string"
}
]
},
{
"name": "OzoneManagerDetailsProto",
"fields": [
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -88,7 +88,7 @@ message SCMVersionResponseProto {
}

message SCMRegisterRequestProto {
required DatanodeDetailsProto datanodeDetails = 1;
required ExtendedDatanodeDetailsProto extendedDatanodeDetails = 1;
required NodeReportProto nodeReport = 2;
required ContainerReportsProto containerReport = 3;
required PipelineReportsProto pipelineReports = 4;
Expand Down
4 changes: 2 additions & 2 deletions hadoop-hdds/interface-server/src/main/resources/proto.lock
Original file line number Diff line number Diff line change
Expand Up @@ -310,8 +310,8 @@
"fields": [
{
"id": 1,
"name": "datanodeDetails",
"type": "DatanodeDetailsProto"
"name": "extendedDatanodeDetails",
"type": "ExtendedDatanodeDetailsProto"
},
{
"id": 2,
Expand Down
Loading