HDFS-13300. Ozone: Remove DatanodeID dependency from HDSL and Ozone.

Contributed by Nanda kumar.
This commit is contained in:
Anu Engineer 2018-03-27 12:55:26 -07:00 committed by Owen O'Malley
parent 70216e7978
commit 498b84155f
104 changed files with 1758 additions and 1244 deletions

View File

@ -27,7 +27,7 @@ OZONE-SITE.XML_ozone.scm.client.address=scm
OZONE-SITE.XML_dfs.cblock.jscsi.cblock.server.address=cblock
OZONE-SITE.XML_dfs.cblock.scm.ipaddress=scm
OZONE-SITE.XML_dfs.cblock.service.leveldb.path=/tmp
HDFS-SITE.XML_dfs.datanode.plugins=org.apache.hadoop.ozone.HdslServerPlugin,org.apache.hadoop.ozone.web.ObjectStoreRestPlugin
HDFS-SITE.XML_dfs.datanode.plugins=org.apache.hadoop.ozone.web.ObjectStoreRestPlugin,org.apache.hadoop.ozone.HdslDatanodeService
HDFS-SITE.XML_dfs.namenode.rpc-address=namenode:9000
HDFS-SITE.XML_dfs.namenode.name.dir=/data/namenode
HDFS-SITE.XML_rpc.metrics.quantile.enable=true

View File

@ -27,7 +27,7 @@ HDFS-SITE.XML_dfs.namenode.rpc-address=namenode:9000
HDFS-SITE.XML_dfs.namenode.name.dir=/data/namenode
HDFS-SITE.XML_rpc.metrics.quantile.enable=true
HDFS-SITE.XML_rpc.metrics.percentiles.intervals=60,300
HDFS-SITE.XML_dfs.datanode.plugins=org.apache.hadoop.ozone.HdslServerPlugin,org.apache.hadoop.ozone.web.ObjectStoreRestPlugin
HDFS-SITE.XML_dfs.datanode.plugins=org.apache.hadoop.ozone.web.ObjectStoreRestPlugin,org.apache.hadoop.ozone.HdslDatanodeService
LOG4J.PROPERTIES_log4j.rootLogger=INFO, stdout
LOG4J.PROPERTIES_log4j.appender.stdout=org.apache.log4j.ConsoleAppender
LOG4J.PROPERTIES_log4j.appender.stdout.layout=org.apache.log4j.PatternLayout

View File

@ -28,9 +28,9 @@
import io.netty.handler.logging.LogLevel;
import io.netty.handler.logging.LoggingHandler;
import org.apache.hadoop.conf.Configuration;
import org.apache.hadoop.hdsl.protocol.DatanodeDetails;
import org.apache.hadoop.hdsl.protocol.proto.ContainerProtos;
import org.apache.hadoop.hdsl.protocol.proto.HdslProtos;
import org.apache.hadoop.hdfs.protocol.DatanodeID;
import org.apache.hadoop.ozone.OzoneConfigKeys;
import org.apache.hadoop.ozone.client.OzoneClientUtils;
import org.apache.hadoop.scm.container.common.helpers.Pipeline;
@ -84,7 +84,7 @@ public void connect() throws Exception {
.channel(NioSocketChannel.class)
.handler(new LoggingHandler(LogLevel.INFO))
.handler(new XceiverClientInitializer(this.pipeline, semaphore));
DatanodeID leader = this.pipeline.getLeader();
DatanodeDetails leader = this.pipeline.getLeader();
// read port from the data node, on failure use default configured
// port.
@ -174,7 +174,7 @@ public ContainerProtos.ContainerCommandResponseProto sendCommand(
* @param datanodes - Datanodes
*/
@Override
public void createPipeline(String pipelineID, List<DatanodeID> datanodes)
public void createPipeline(String pipelineID, List<DatanodeDetails> datanodes)
throws IOException {
// For stand alone pipeline, there is no notion called setup pipeline.
return;

View File

@ -154,7 +154,7 @@ public CompletableFuture<ContainerCommandResponseProto> sendCommandAsync(
// Setting the datanode ID in the commands, so that we can distinguish
// commands when the cluster simulator is running.
if(!request.hasDatanodeID()) {
if(!request.hasDatanodeUuid()) {
throw new IllegalArgumentException("Invalid Datanode ID");
}

View File

@ -21,10 +21,10 @@
import com.google.common.base.Preconditions;
import com.google.protobuf.InvalidProtocolBufferException;
import org.apache.hadoop.conf.Configuration;
import org.apache.hadoop.hdsl.protocol.DatanodeDetails;
import org.apache.hadoop.hdsl.protocol.proto.ContainerProtos.ContainerCommandRequestProto;
import org.apache.hadoop.hdsl.protocol.proto.ContainerProtos.ContainerCommandResponseProto;
import org.apache.hadoop.hdsl.protocol.proto.HdslProtos;
import org.apache.hadoop.hdfs.protocol.DatanodeID;
import org.apache.hadoop.ozone.client.OzoneClientUtils;
import org.apache.hadoop.scm.container.common.helpers.Pipeline;
import org.apache.ratis.RatisHelper;
@ -84,7 +84,7 @@ private XceiverClientRatis(Pipeline pipeline, RpcType rpcType,
/**
* {@inheritDoc}
*/
public void createPipeline(String clusterId, List<DatanodeID> datanodes)
public void createPipeline(String clusterId, List<DatanodeDetails> datanodes)
throws IOException {
RaftGroup group = RatisHelper.newRaftGroup(datanodes);
LOG.debug("initializing pipeline:{} with nodes:{}", clusterId,
@ -102,14 +102,14 @@ public HdslProtos.ReplicationType getPipelineType() {
return HdslProtos.ReplicationType.RATIS;
}
private void reinitialize(List<DatanodeID> datanodes, RaftGroup group)
private void reinitialize(List<DatanodeDetails> datanodes, RaftGroup group)
throws IOException {
if (datanodes.isEmpty()) {
return;
}
IOException exception = null;
for (DatanodeID d : datanodes) {
for (DatanodeDetails d : datanodes) {
try {
reinitialize(d, group);
} catch (IOException ioe) {
@ -133,7 +133,7 @@ private void reinitialize(List<DatanodeID> datanodes, RaftGroup group)
* @param group - Raft group
* @throws IOException - on Failure.
*/
private void reinitialize(DatanodeID datanode, RaftGroup group)
private void reinitialize(DatanodeDetails datanode, RaftGroup group)
throws IOException {
final RaftPeer p = RatisHelper.toRaftPeer(datanode);
try (RaftClient client = RatisHelper.newRaftClient(rpcType, p)) {

View File

@ -255,7 +255,7 @@ public static boolean isHdslEnabled(Configuration conf) {
* @param conf - Configuration
* @return the path of datanode id as string
*/
public static String getDatanodeIDPath(Configuration conf) {
public static String getDatanodeIdFilePath(Configuration conf) {
String dataNodeIDPath = conf.get(ScmConfigKeys.OZONE_SCM_DATANODE_ID);
if (dataNodeIDPath == null) {
String metaPath = conf.get(OzoneConfigKeys.OZONE_METADATA_DIRS);

View File

@ -0,0 +1,422 @@
/**
* Licensed to the Apache Software Foundation (ASF) under one
* or more contributor license agreements. See the NOTICE file
* distributed with this work for additional information
* regarding copyright ownership. The ASF licenses this file
* to you under the Apache License, Version 2.0 (the
* "License"); you may not use this file except in compliance
* with the License. You may obtain a copy of the License at
*
* http://www.apache.org/licenses/LICENSE-2.0
*
* Unless required by applicable law or agreed to in writing, software
* distributed under the License is distributed on an "AS IS" BASIS,
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
* See the License for the specific language governing permissions and
* limitations under the License.
*/
package org.apache.hadoop.hdsl.protocol;
import com.google.common.base.Preconditions;
import org.apache.hadoop.classification.InterfaceAudience;
import org.apache.hadoop.classification.InterfaceStability;
import org.apache.hadoop.hdsl.protocol.proto.HdslProtos;
import java.util.UUID;
/**
* DatanodeDetails class contains details about DataNode like:
* - UUID of the DataNode.
* - IP and Hostname details.
* - Port details to which the DataNode will be listening.
*/
@InterfaceAudience.Private
@InterfaceStability.Evolving
public final class DatanodeDetails implements Comparable<DatanodeDetails> {
/**
* DataNode's unique identifier in the cluster.
*/
private final UUID uuid;
private String ipAddress;
private String hostName;
private Integer infoPort;
private Integer infoSecurePort;
private Integer containerPort;
private Integer ratisPort;
private Integer ozoneRestPort;
/**
* Constructs DatanodeDetails instance. DatanodeDetails.Builder is used
* for instantiating DatanodeDetails.
* @param uuid DataNode's UUID
* @param ipAddress IP Address of this DataNode
* @param hostName DataNode's hostname
* @param infoPort HTTP Port
* @param infoSecurePort HTTPS Port
* @param containerPort Container Port
* @param ratisPort Ratis Port
* @param ozoneRestPort Rest Port
*/
private DatanodeDetails(
String uuid, String ipAddress, String hostName, Integer infoPort,
Integer infoSecurePort, Integer containerPort, Integer ratisPort,
Integer ozoneRestPort) {
this.uuid = UUID.fromString(uuid);
this.ipAddress = ipAddress;
this.hostName = hostName;
this.infoPort = infoPort;
this.infoSecurePort = infoSecurePort;
this.containerPort = containerPort;
this.ratisPort = ratisPort;
this.ozoneRestPort = ozoneRestPort;
}
/**
* Returns the DataNode UUID.
*
* @return UUID of DataNode
*/
public UUID getUuid() {
return uuid;
}
/**
* Returns the string representation of DataNode UUID.
*
* @return UUID of DataNode
*/
public String getUuidString() {
return uuid.toString();
}
/**
* Sets the IP address of Datanode.
*
* @param ip IP Address
*/
public void setIpAddress(String ip) {
this.ipAddress = ip;
}
/**
* Returns IP address of DataNode.
*
* @return IP address
*/
public String getIpAddress() {
return ipAddress;
}
/**
* Sets the Datanode hostname.
*
* @param host hostname
*/
public void setHostName(String host) {
this.hostName = host;
}
/**
* Returns Hostname of DataNode.
*
* @return Hostname
*/
public String getHostName() {
return hostName;
}
/**
* Sets the InfoPort.
* @param port InfoPort
*/
public void setInfoPort(int port) {
infoPort = port;
}
/**
* Returns DataNodes Info Port.
*
* @return InfoPort
*/
public int getInfoPort() {
return infoPort;
}
/**
* Sets the InfoSecurePort.
*
* @param port InfoSecurePort
*/
public void setInfoSecurePort(int port) {
infoSecurePort = port;
}
/**
* Returns DataNodes Secure Info Port.
*
* @return InfoSecurePort
*/
public int getInfoSecurePort() {
return infoSecurePort;
}
/**
* Sets the Container Port.
* @param port ContainerPort
*/
public void setContainerPort(int port) {
containerPort = port;
}
/**
* Returns standalone container Port.
*
* @return Container Port
*/
public int getContainerPort() {
return containerPort;
}
/**
* Sets Ratis Port.
* @param port RatisPort
*/
public void setRatisPort(int port) {
ratisPort = port;
}
/**
* Returns Ratis Port.
* @return Ratis Port
*/
public int getRatisPort() {
return ratisPort;
}
/**
* Sets OzoneRestPort.
* @param port OzoneRestPort
*/
public void setOzoneRestPort(int port) {
ozoneRestPort = port;
}
/**
* Returns Ozone Rest Port.
* @return OzoneRestPort
*/
public int getOzoneRestPort() {
return ozoneRestPort;
}
/**
* Returns a DatanodeDetails from the protocol buffers.
*
* @param datanodeDetailsProto - protoBuf Message
* @return DatanodeDetails
*/
public static DatanodeDetails getFromProtoBuf(
HdslProtos.DatanodeDetailsProto datanodeDetailsProto) {
DatanodeDetails.Builder builder = newBuilder();
builder.setUuid(datanodeDetailsProto.getUuid());
if (datanodeDetailsProto.hasIpAddress()) {
builder.setIpAddress(datanodeDetailsProto.getIpAddress());
}
if (datanodeDetailsProto.hasHostName()) {
builder.setHostName(datanodeDetailsProto.getHostName());
}
if (datanodeDetailsProto.hasInfoPort()) {
builder.setInfoPort(datanodeDetailsProto.getInfoPort());
}
if (datanodeDetailsProto.hasInfoSecurePort()) {
builder.setInfoSecurePort(datanodeDetailsProto.getInfoSecurePort());
}
if (datanodeDetailsProto.hasContainerPort()) {
builder.setContainerPort(datanodeDetailsProto.getContainerPort());
}
if (datanodeDetailsProto.hasRatisPort()) {
builder.setRatisPort(datanodeDetailsProto.getRatisPort());
}
if (datanodeDetailsProto.hasOzoneRestPort()) {
builder.setOzoneRestPort(datanodeDetailsProto.getOzoneRestPort());
}
return builder.build();
}
/**
* Returns a DatanodeDetails protobuf message from a datanode ID.
* @return Hdsl.DatanodeDetailsProto
*/
public HdslProtos.DatanodeDetailsProto getProtoBufMessage() {
HdslProtos.DatanodeDetailsProto.Builder builder =
HdslProtos.DatanodeDetailsProto.newBuilder()
.setUuid(getUuidString());
if (ipAddress != null) {
builder.setIpAddress(ipAddress);
}
if (hostName != null) {
builder.setHostName(hostName);
}
if (infoPort != null) {
builder.setInfoPort(infoPort);
}
if (infoSecurePort != null) {
builder.setInfoSecurePort(infoSecurePort);
}
if (containerPort != null) {
builder.setContainerPort(containerPort);
}
if (ratisPort != null) {
builder.setRatisPort(ratisPort);
}
if (ozoneRestPort != null) {
builder.setOzoneRestPort(ozoneRestPort);
}
return builder.build();
}
@Override
public String toString() {
return uuid.toString() + "{" +
"ip: " +
ipAddress +
", host: " +
hostName +
"}";
}
@Override
public int compareTo(DatanodeDetails that) {
return this.getUuid().compareTo(that.getUuid());
}
/**
* Returns DatanodeDetails.Builder instance.
*
* @return DatanodeDetails.Builder
*/
public static Builder newBuilder() {
return new Builder();
}
/**
* Builder class for building DatanodeDetails.
*/
public static class Builder {
private String id;
private String ipAddress;
private String hostName;
private Integer infoPort;
private Integer infoSecurePort;
private Integer containerPort;
private Integer ratisPort;
private Integer ozoneRestPort;
/**
* Sets the DatanodeUuid.
*
* @param uuid DatanodeUuid
* @return DatanodeDetails.Builder
*/
public Builder setUuid(String uuid) {
this.id = uuid;
return this;
}
/**
* Sets the IP address of DataNode.
*
* @param ip address
* @return DatanodeDetails.Builder
*/
public Builder setIpAddress(String ip) {
this.ipAddress = ip;
return this;
}
/**
* Sets the hostname of DataNode.
*
* @param host hostname
* @return DatanodeDetails.Builder
*/
public Builder setHostName(String host) {
this.hostName = host;
return this;
}
/**
* Sets the InfoPort.
*
* @param port InfoPort
* @return DatanodeDetails.Builder
*/
public Builder setInfoPort(Integer port) {
this.infoPort = port;
return this;
}
/**
* Sets the Secure Info Port.
*
* @param port InfoSecurePort
* @return DatanodeDetails.Builder
*/
public Builder setInfoSecurePort(Integer port) {
this.infoSecurePort = port;
return this;
}
/**
* Sets the ContainerPort.
*
* @param port ContainerPort
* @return DatanodeDetails.Builder
*/
public Builder setContainerPort(Integer port) {
this.containerPort = port;
return this;
}
/**
* Sets the RatisPort.
*
* @param port RatisPort
* @return DatanodeDetails.Builder
*/
public Builder setRatisPort(Integer port) {
this.ratisPort = port;
return this;
}
/**
* Sets the OzoneRestPort.
*
* @param port OzoneRestPort
* @return DatanodeDetails.Builder
*/
public Builder setOzoneRestPort(Integer port) {
this.ozoneRestPort = port;
return this;
}
/**
* Builds and returns DatanodeDetails instance.
*
* @return DatanodeDetails
*/
public DatanodeDetails build() {
Preconditions.checkNotNull(id);
return new DatanodeDetails(id, ipAddress, hostName,
infoPort, infoSecurePort, containerPort, ratisPort, ozoneRestPort);
}
}
}

View File

@ -0,0 +1,22 @@
/**
* Licensed to the Apache Software Foundation (ASF) under one
* or more contributor license agreements. See the NOTICE file
* distributed with this work for additional information
* regarding copyright ownership. The ASF licenses this file
* to you under the Apache License, Version 2.0 (the
* "License"); you may not use this file except in compliance
* with the License. You may obtain a copy of the License at
*
* http://www.apache.org/licenses/LICENSE-2.0
*
* Unless required by applicable law or agreed to in writing, software
* distributed under the License is distributed on an "AS IS" BASIS,
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
* See the License for the specific language governing permissions and
* limitations under the License.
*/
/**
* This package contains HDSL protocol related classes.
*/
package org.apache.hadoop.hdsl.protocol;

View File

@ -19,11 +19,11 @@
package org.apache.hadoop.scm;
import com.google.common.annotations.VisibleForTesting;
import org.apache.hadoop.hdsl.protocol.DatanodeDetails;
import org.apache.hadoop.hdsl.protocol.proto.ContainerProtos
.ContainerCommandRequestProto;
import org.apache.hadoop.hdsl.protocol.proto.ContainerProtos
.ContainerCommandResponseProto;
import org.apache.hadoop.hdfs.protocol.DatanodeID;
import org.apache.hadoop.hdsl.protocol.proto.HdslProtos;
import org.apache.hadoop.scm.container.common.helpers.Pipeline;
@ -118,7 +118,7 @@ public abstract ContainerCommandResponseProto sendCommand(
* @param datanodes - Datanodes
*/
public abstract void createPipeline(String pipelineID,
List<DatanodeID> datanodes) throws IOException;
List<DatanodeDetails> datanodes) throws IOException;
/**
* Returns pipeline Type.

View File

@ -29,8 +29,8 @@
import com.fasterxml.jackson.databind.ser.impl.SimpleFilterProvider;
import com.google.common.annotations.VisibleForTesting;
import com.google.common.base.Preconditions;
import org.apache.hadoop.hdsl.protocol.DatanodeDetails;
import org.apache.hadoop.hdsl.protocol.proto.HdslProtos;
import org.apache.hadoop.hdfs.protocol.DatanodeID;
import java.io.IOException;
import java.util.ArrayList;
@ -100,7 +100,7 @@ public HdslProtos.ReplicationFactor getFactor() {
* @return First Machine.
*/
@JsonIgnore
public DatanodeID getLeader() {
public DatanodeDetails getLeader() {
return pipelineChannel.getDatanodes().get(pipelineChannel.getLeaderID());
}
@ -120,7 +120,7 @@ public String getLeaderHost() {
* @return List of Machines.
*/
@JsonIgnore
public List<DatanodeID> getMachines() {
public List<DatanodeDetails> getMachines() {
return new ArrayList<>(pipelineChannel.getDatanodes().values());
}
@ -131,7 +131,7 @@ public List<DatanodeID> getMachines() {
*/
public List<String> getDatanodeHosts() {
List<String> dataHosts = new ArrayList<>();
for (DatanodeID id : pipelineChannel.getDatanodes().values()) {
for (DatanodeDetails id : pipelineChannel.getDatanodes().values()) {
dataHosts.add(id.getHostName());
}
return dataHosts;

View File

@ -19,8 +19,7 @@
import com.fasterxml.jackson.annotation.JsonIgnore;
import com.google.common.base.Preconditions;
import org.apache.hadoop.hdfs.protocol.DatanodeID;
import org.apache.hadoop.hdfs.protocol.proto.HdfsProtos;
import org.apache.hadoop.hdsl.protocol.DatanodeDetails;
import org.apache.hadoop.hdsl.protocol.proto.HdslProtos;
import org.apache.hadoop.hdsl.protocol.proto.HdslProtos.LifeCycleState;
import org.apache.hadoop.hdsl.protocol.proto.HdslProtos.ReplicationType;
@ -36,7 +35,7 @@ public class PipelineChannel {
@JsonIgnore
private String leaderID;
@JsonIgnore
private Map<String, DatanodeID> datanodes;
private Map<String, DatanodeDetails> datanodes;
private LifeCycleState lifeCycleState;
private ReplicationType type;
private ReplicationFactor factor;
@ -57,7 +56,7 @@ public String getLeaderID() {
return leaderID;
}
public Map<String, DatanodeID> getDatanodes() {
public Map<String, DatanodeDetails> getDatanodes() {
return datanodes;
}
@ -77,15 +76,16 @@ public String getName() {
return name;
}
public void addMember(DatanodeID dataNodeId) {
datanodes.put(dataNodeId.getDatanodeUuid(), dataNodeId);
public void addMember(DatanodeDetails datanodeDetails) {
datanodes.put(datanodeDetails.getUuid().toString(),
datanodeDetails);
}
@JsonIgnore
public HdslProtos.PipelineChannel getProtobufMessage() {
HdslProtos.PipelineChannel.Builder builder =
HdslProtos.PipelineChannel.newBuilder();
for (DatanodeID datanode : datanodes.values()) {
for (DatanodeDetails datanode : datanodes.values()) {
builder.addMembers(datanode.getProtoBufMessage());
}
builder.setLeaderID(leaderID);
@ -113,8 +113,9 @@ public static PipelineChannel getFromProtoBuf(
transportProtos.getFactor(),
transportProtos.getName());
for (HdfsProtos.DatanodeIDProto dataID : transportProtos.getMembersList()) {
pipelineChannel.addMember(DatanodeID.getFromProtoBuf(dataID));
for (HdslProtos.DatanodeDetailsProto dataID :
transportProtos.getMembersList()) {
pipelineChannel.addMember(DatanodeDetails.getFromProtoBuf(dataID));
}
return pipelineChannel;
}

View File

@ -82,12 +82,12 @@ public static GetKeyResponseProto getKey(XceiverClientSpi xceiverClient,
.newBuilder()
.setPipeline(xceiverClient.getPipeline().getProtobufMessage())
.setKeyData(containerKeyData);
String id = xceiverClient.getPipeline().getLeader().getDatanodeUuid();
String id = xceiverClient.getPipeline().getLeader().getUuidString();
ContainerCommandRequestProto request = ContainerCommandRequestProto
.newBuilder()
.setCmdType(Type.GetKey)
.setTraceID(traceID)
.setDatanodeID(id)
.setDatanodeUuid(id)
.setGetKey(readKeyRequest)
.build();
ContainerCommandResponseProto response = xceiverClient.sendCommand(request);
@ -109,12 +109,12 @@ public static void putKey(XceiverClientSpi xceiverClient,
.newBuilder()
.setPipeline(xceiverClient.getPipeline().getProtobufMessage())
.setKeyData(containerKeyData);
String id = xceiverClient.getPipeline().getLeader().getDatanodeUuid();
String id = xceiverClient.getPipeline().getLeader().getUuidString();
ContainerCommandRequestProto request = ContainerCommandRequestProto
.newBuilder()
.setCmdType(Type.PutKey)
.setTraceID(traceID)
.setDatanodeID(id)
.setDatanodeUuid(id)
.setPutKey(createKeyRequest)
.build();
ContainerCommandResponseProto response = xceiverClient.sendCommand(request);
@ -139,12 +139,12 @@ public static ReadChunkResponseProto readChunk(XceiverClientSpi xceiverClient,
.setPipeline(xceiverClient.getPipeline().getProtobufMessage())
.setKeyName(key)
.setChunkData(chunk);
String id = xceiverClient.getPipeline().getLeader().getDatanodeUuid();
String id = xceiverClient.getPipeline().getLeader().getUuidString();
ContainerCommandRequestProto request = ContainerCommandRequestProto
.newBuilder()
.setCmdType(Type.ReadChunk)
.setTraceID(traceID)
.setDatanodeID(id)
.setDatanodeUuid(id)
.setReadChunk(readChunkRequest)
.build();
ContainerCommandResponseProto response = xceiverClient.sendCommand(request);
@ -171,12 +171,12 @@ public static void writeChunk(XceiverClientSpi xceiverClient, ChunkInfo chunk,
.setKeyName(key)
.setChunkData(chunk)
.setData(data);
String id = xceiverClient.getPipeline().getLeader().getDatanodeUuid();
String id = xceiverClient.getPipeline().getLeader().getUuidString();
ContainerCommandRequestProto request = ContainerCommandRequestProto
.newBuilder()
.setCmdType(Type.WriteChunk)
.setTraceID(traceID)
.setDatanodeID(id)
.setDatanodeUuid(id)
.setWriteChunk(writeChunkRequest)
.build();
ContainerCommandResponseProto response = xceiverClient.sendCommand(request);
@ -220,12 +220,12 @@ public static void writeSmallFile(XceiverClientSpi client,
.setKey(createKeyRequest).setData(ByteString.copyFrom(data))
.build();
String id = client.getPipeline().getLeader().getDatanodeUuid();
String id = client.getPipeline().getLeader().getUuidString();
ContainerCommandRequestProto request =
ContainerCommandRequestProto.newBuilder()
.setCmdType(Type.PutSmallFile)
.setTraceID(traceID)
.setDatanodeID(id)
.setDatanodeUuid(id)
.setPutSmallFile(putSmallFileRequest)
.build();
ContainerCommandResponseProto response = client.sendCommand(request);
@ -249,12 +249,12 @@ public static void createContainer(XceiverClientSpi client, String traceID)
createRequest.setPipeline(client.getPipeline().getProtobufMessage());
createRequest.setContainerData(containerData.build());
String id = client.getPipeline().getLeader().getDatanodeUuid();
String id = client.getPipeline().getLeader().getUuidString();
ContainerCommandRequestProto.Builder request =
ContainerCommandRequestProto.newBuilder();
request.setCmdType(ContainerProtos.Type.CreateContainer);
request.setCreateContainer(createRequest);
request.setDatanodeID(id);
request.setDatanodeUuid(id);
request.setTraceID(traceID);
ContainerCommandResponseProto response = client.sendCommand(
request.build());
@ -276,13 +276,13 @@ public static void deleteContainer(XceiverClientSpi client,
deleteRequest.setName(client.getPipeline().getContainerName());
deleteRequest.setPipeline(client.getPipeline().getProtobufMessage());
deleteRequest.setForceDelete(force);
String id = client.getPipeline().getLeader().getDatanodeUuid();
String id = client.getPipeline().getLeader().getUuidString();
ContainerCommandRequestProto.Builder request =
ContainerCommandRequestProto.newBuilder();
request.setCmdType(ContainerProtos.Type.DeleteContainer);
request.setDeleteContainer(deleteRequest);
request.setTraceID(traceID);
request.setDatanodeID(id);
request.setDatanodeUuid(id);
ContainerCommandResponseProto response =
client.sendCommand(request.build());
validateContainerResponse(response);
@ -301,13 +301,13 @@ public static void closeContainer(XceiverClientSpi client, String traceID)
ContainerProtos.CloseContainerRequestProto.newBuilder();
closeRequest.setPipeline(client.getPipeline().getProtobufMessage());
String id = client.getPipeline().getLeader().getDatanodeUuid();
String id = client.getPipeline().getLeader().getUuidString();
ContainerCommandRequestProto.Builder request =
ContainerCommandRequestProto.newBuilder();
request.setCmdType(Type.CloseContainer);
request.setCloseContainer(closeRequest);
request.setTraceID(traceID);
request.setDatanodeID(id);
request.setDatanodeUuid(id);
ContainerCommandResponseProto response =
client.sendCommand(request.build());
validateContainerResponse(response);
@ -327,12 +327,12 @@ public static ReadContainerResponseProto readContainer(
ReadContainerRequestProto.newBuilder();
readRequest.setName(containerName);
readRequest.setPipeline(client.getPipeline().getProtobufMessage());
String id = client.getPipeline().getLeader().getDatanodeUuid();
String id = client.getPipeline().getLeader().getUuidString();
ContainerCommandRequestProto.Builder request =
ContainerCommandRequestProto.newBuilder();
request.setCmdType(Type.ReadContainer);
request.setReadContainer(readRequest);
request.setDatanodeID(id);
request.setDatanodeUuid(id);
request.setTraceID(traceID);
ContainerCommandResponseProto response =
client.sendCommand(request.build());
@ -365,12 +365,12 @@ public static GetSmallFileResponseProto readSmallFile(XceiverClientSpi client,
GetSmallFileRequestProto
.newBuilder().setKey(getKey)
.build();
String id = client.getPipeline().getLeader().getDatanodeUuid();
String id = client.getPipeline().getLeader().getUuidString();
ContainerCommandRequestProto request = ContainerCommandRequestProto
.newBuilder()
.setCmdType(Type.GetSmallFile)
.setTraceID(traceID)
.setDatanodeID(id)
.setDatanodeUuid(id)
.setGetSmallFile(getSmallFileRequest)
.build();
ContainerCommandResponseProto response = client.sendCommand(request);

View File

@ -18,7 +18,7 @@
package org.apache.ratis;
import org.apache.hadoop.hdfs.protocol.DatanodeID;
import org.apache.hadoop.hdsl.protocol.DatanodeDetails;
import org.apache.hadoop.ozone.OzoneConfigKeys;
import org.apache.hadoop.scm.container.common.helpers.Pipeline;
import org.apache.ratis.client.RaftClient;
@ -47,19 +47,19 @@
public interface RatisHelper {
Logger LOG = LoggerFactory.getLogger(RatisHelper.class);
static String toRaftPeerIdString(DatanodeID id) {
return id.getIpAddr() + "_" + id.getRatisPort();
static String toRaftPeerIdString(DatanodeDetails id) {
return id.getUuidString() + "_" + id.getRatisPort();
}
static String toRaftPeerAddressString(DatanodeID id) {
return id.getIpAddr() + ":" + id.getRatisPort();
static String toRaftPeerAddressString(DatanodeDetails id) {
return id.getIpAddress() + ":" + id.getRatisPort();
}
static RaftPeerId toRaftPeerId(DatanodeID id) {
static RaftPeerId toRaftPeerId(DatanodeDetails id) {
return RaftPeerId.valueOf(toRaftPeerIdString(id));
}
static RaftPeer toRaftPeer(DatanodeID id) {
static RaftPeer toRaftPeer(DatanodeDetails id) {
return new RaftPeer(toRaftPeerId(id), toRaftPeerAddressString(id));
}
@ -67,7 +67,8 @@ static List<RaftPeer> toRaftPeers(Pipeline pipeline) {
return toRaftPeers(pipeline.getMachines());
}
static <E extends DatanodeID> List<RaftPeer> toRaftPeers(List<E> datanodes) {
static <E extends DatanodeDetails> List<RaftPeer> toRaftPeers(
List<E> datanodes) {
return datanodes.stream().map(RatisHelper::toRaftPeer)
.collect(Collectors.toList());
}
@ -85,7 +86,7 @@ static RaftGroup emptyRaftGroup() {
return EMPTY_GROUP;
}
static RaftGroup newRaftGroup(List<DatanodeID> datanodes) {
static RaftGroup newRaftGroup(List<DatanodeDetails> datanodes) {
final List<RaftPeer> newPeers = datanodes.stream()
.map(RatisHelper::toRaftPeer)
.collect(Collectors.toList());

View File

@ -164,7 +164,7 @@ message ContainerCommandRequestProto {
optional PutSmallFileRequestProto putSmallFile = 16;
optional GetSmallFileRequestProto getSmallFile = 17;
optional CloseContainerRequestProto closeContainer = 18;
required string datanodeID = 19;
required string datanodeUuid = 19;
}
message ContainerCommandResponseProto {

View File

@ -28,11 +28,21 @@ option java_generic_services = true;
option java_generate_equals_and_hash = true;
package hadoop.hdsl;
import "hdfs.proto";
message DatanodeDetailsProto {
// TODO: make the port as a seperate proto message and use it here
required string uuid = 1; // UUID assigned to the Datanode.
required string ipAddress = 2; // IP address
required string hostName = 3; // hostname
optional uint32 infoPort = 4; // datanode http port
optional uint32 infoSecurePort = 5 [default = 0]; // datanode https port
optional uint32 containerPort = 6 [default = 0]; // Ozone stand_alone protocol
optional uint32 ratisPort = 7 [default = 0]; //Ozone ratis port
optional uint32 ozoneRestPort = 8 [default = 0];
}
message PipelineChannel {
required string leaderID = 1;
repeated hadoop.hdfs.DatanodeIDProto members = 2;
repeated DatanodeDetailsProto members = 2;
optional LifeCycleState state = 3 [default = OPEN];
optional ReplicationType type = 4 [default = STAND_ALONE];
optional ReplicationFactor factor = 5 [default = ONE];
@ -82,7 +92,7 @@ enum QueryScope {
}
message Node {
required hadoop.hdfs.DatanodeIDProto nodeID = 1;
required DatanodeDetailsProto nodeID = 1;
repeated NodeState nodeStates = 2;
}

View File

@ -0,0 +1,140 @@
/*
* Licensed to the Apache Software Foundation (ASF) under one
* or more contributor license agreements. See the NOTICE file
* distributed with this work for additional information
* regarding copyright ownership. The ASF licenses this file
* to you under the Apache License, Version 2.0 (the
* "License"); you may not use this file except in compliance
* with the License. You may obtain a copy of the License at
*
* http://www.apache.org/licenses/LICENSE-2.0
*
* Unless required by applicable law or agreed to in writing, software
* distributed under the License is distributed on an "AS IS" BASIS,
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
* See the License for the specific language governing permissions and
* limitations under the License.
*/
package org.apache.hadoop.ozone;
import java.io.File;
import java.io.IOException;
import java.net.InetAddress;
import java.util.UUID;
import com.google.common.base.Preconditions;
import org.apache.hadoop.classification.InterfaceAudience;
import org.apache.hadoop.conf.Configuration;
import org.apache.hadoop.hdfs.server.datanode.DataNode;
import org.apache.hadoop.hdfs.server.datanode.DataNodeServicePlugin;
import org.apache.hadoop.hdsl.HdslUtils;
import org.apache.hadoop.hdsl.conf.OzoneConfiguration;
import org.apache.hadoop.hdsl.protocol.DatanodeDetails;
import org.apache.hadoop.ozone.container.common.helpers.ContainerUtils;
import org.apache.hadoop.ozone.container.common.statemachine
.DatanodeStateMachine;
import org.apache.hadoop.scm.ScmConfigKeys;
import org.slf4j.Logger;
import org.slf4j.LoggerFactory;
/**
* Datanode service plugin to start the HDSL container services.
*/
public class HdslDatanodeService implements DataNodeServicePlugin {
private static final Logger LOG = LoggerFactory.getLogger(
HdslDatanodeService.class);
private final boolean isOzoneEnabled;
private Configuration conf;
private DatanodeDetails datanodeDetails;
private DatanodeStateMachine datanodeStateMachine;
public HdslDatanodeService() {
try {
OzoneConfiguration.activate();
this.conf = new OzoneConfiguration();
this.isOzoneEnabled = HdslUtils.isHdslEnabled(conf);
if (isOzoneEnabled) {
this.datanodeDetails = getDatanodeDetails(conf);
String hostname = DataNode.getHostName(conf);
String ip = InetAddress.getByName(hostname).getHostAddress();
this.datanodeDetails.setHostName(hostname);
this.datanodeDetails.setIpAddress(ip);
}
} catch (IOException e) {
throw new RuntimeException("Can't start the HDSL datanode plugin", e);
}
}
@Override
public void start(Object service) {
if (isOzoneEnabled) {
try {
DataNode dataNode = (DataNode) service;
datanodeDetails.setInfoPort(dataNode.getInfoPort());
datanodeDetails.setInfoSecurePort(dataNode.getInfoSecurePort());
datanodeStateMachine = new DatanodeStateMachine(datanodeDetails, conf);
datanodeStateMachine.startDaemon();
} catch (IOException e) {
throw new RuntimeException("Can't start the HDSL datanode plugin", e);
}
}
}
/**
* Returns ContainerNodeIDProto or null in case of Error.
*
* @return ContainerNodeIDProto
*/
private static DatanodeDetails getDatanodeDetails(Configuration conf)
throws IOException {
String idFilePath = HdslUtils.getDatanodeIdFilePath(conf);
if (idFilePath == null || idFilePath.isEmpty()) {
LOG.error("A valid file path is needed for config setting {}",
ScmConfigKeys.OZONE_SCM_DATANODE_ID);
throw new IllegalArgumentException(ScmConfigKeys.OZONE_SCM_DATANODE_ID +
" must be defined. See" +
" https://wiki.apache.org/hadoop/Ozone#Configuration" +
" for details on configuring Ozone.");
}
Preconditions.checkNotNull(idFilePath);
File idFile = new File(idFilePath);
if (idFile.exists()) {
return ContainerUtils.readDatanodeDetailsFrom(idFile);
} else {
// There is no datanode.id file, this might be the first time datanode
// is started.
String datanodeUuid = UUID.randomUUID().toString();
return DatanodeDetails.newBuilder().setUuid(datanodeUuid).build();
}
}
/**
*
* Return DatanodeDetails if set, return null otherwise.
*
* @return DatanodeDetails
*/
public DatanodeDetails getDatanodeDetails() {
return datanodeDetails;
}
@InterfaceAudience.Private
public DatanodeStateMachine getDatanodeStateMachine() {
return datanodeStateMachine;
}
@Override
public void stop() {
if (datanodeStateMachine != null) {
datanodeStateMachine.stopDaemon();
}
}
@Override
public void close() throws IOException {
}
}

View File

@ -1,82 +0,0 @@
/*
* Licensed to the Apache Software Foundation (ASF) under one
* or more contributor license agreements. See the NOTICE file
* distributed with this work for additional information
* regarding copyright ownership. The ASF licenses this file
* to you under the Apache License, Version 2.0 (the
* "License"); you may not use this file except in compliance
* with the License. You may obtain a copy of the License at
*
* http://www.apache.org/licenses/LICENSE-2.0
*
* Unless required by applicable law or agreed to in writing, software
* distributed under the License is distributed on an "AS IS" BASIS,
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
* See the License for the specific language governing permissions and
* limitations under the License.
*/
package org.apache.hadoop.ozone;
import java.io.IOException;
import org.apache.hadoop.classification.InterfaceAudience;
import org.apache.hadoop.hdfs.server.datanode.DataNode;
import org.apache.hadoop.hdfs.server.datanode.DataNodeServicePlugin;
import org.apache.hadoop.hdfs.server.protocol.DatanodeRegistration;
import org.apache.hadoop.hdsl.HdslUtils;
import org.apache.hadoop.hdsl.conf.OzoneConfiguration;
import org.apache.hadoop.ozone.container.common.statemachine
.DatanodeStateMachine;
/**
* Datanode service plugin to start the HDSL container services.
*/
public class HdslServerPlugin implements DataNodeServicePlugin {
private DatanodeStateMachine datanodeStateMachine;
private DataNode dataNode;
public HdslServerPlugin() {
OzoneConfiguration.activate();
}
@Override
public void start(Object service) {
dataNode = (DataNode) service;
}
@Override
public synchronized void onDatanodeSuccessfulNamenodeRegisration(
DatanodeRegistration dataNodeId) {
if (HdslUtils.isHdslEnabled(dataNode.getConf())) {
try {
if (datanodeStateMachine==null) {
datanodeStateMachine =
new DatanodeStateMachine(dataNodeId,
dataNode.getConf());
datanodeStateMachine.startDaemon();
}
} catch (IOException e) {
throw new RuntimeException("Can't start the HDSL server plugin", e);
}
}
}
@Override
public void stop() {
if (datanodeStateMachine != null) {
datanodeStateMachine.stopDaemon();
}
}
@Override
public void close() throws IOException {
}
@InterfaceAudience.Private
public DatanodeStateMachine getDatanodeStateMachine() {
return datanodeStateMachine;
}
}

View File

@ -22,9 +22,9 @@
import org.apache.commons.io.FileUtils;
import org.apache.hadoop.conf.Configuration;
import org.apache.hadoop.fs.FileAlreadyExistsException;
import org.apache.hadoop.hdsl.protocol.DatanodeDetails;
import org.apache.hadoop.hdsl.protocol.proto.ContainerProtos;
import org.apache.hadoop.hdfs.protocol.DatanodeID;
import org.apache.hadoop.hdfs.protocol.proto.HdfsProtos;
import org.apache.hadoop.hdsl.protocol.proto.HdslProtos;
import org.apache.hadoop.ozone.OzoneConsts;
import org.apache.hadoop.ozone.container.common.impl.ContainerManagerImpl;
import org.apache.hadoop.scm.container.common.helpers.StorageContainerException;
@ -39,8 +39,6 @@
import java.io.IOException;
import java.nio.file.Path;
import java.nio.file.Paths;
import java.util.ArrayList;
import java.util.List;
import static org.apache.commons.io.FilenameUtils.removeExtension;
import static org.apache.hadoop.hdsl.protocol.proto.ContainerProtos
@ -397,15 +395,12 @@ public static void removeContainer(ContainerData containerData,
}
/**
* Write datanode ID protobuf messages to an ID file.
* The old ID file will be overwritten.
* Persistent a {@link DatanodeDetails} to a local file.
*
* @param ids A set of {@link DatanodeID}
* @param path Local ID file path
* @throws IOException When read/write error occurs
* @throws IOException when read/write error occurs
*/
private synchronized static void writeDatanodeIDs(List<DatanodeID> ids,
File path) throws IOException {
public synchronized static void writeDatanodeDetailsTo(
DatanodeDetails datanodeDetails, File path) throws IOException {
if (path.exists()) {
if (!path.delete() || !path.createNewFile()) {
throw new IOException("Unable to overwrite the datanode ID file.");
@ -417,61 +412,30 @@ private synchronized static void writeDatanodeIDs(List<DatanodeID> ids,
}
}
try (FileOutputStream out = new FileOutputStream(path)) {
for (DatanodeID id : ids) {
HdfsProtos.DatanodeIDProto dnId = id.getProtoBufMessage();
dnId.writeDelimitedTo(out);
}
HdslProtos.DatanodeDetailsProto proto =
datanodeDetails.getProtoBufMessage();
proto.writeTo(out);
}
}
/**
* Persistent a {@link DatanodeID} to a local file.
* It reads the IDs first and append a new entry only if the ID is new.
* This is to avoid on some dirty environment, this file gets too big.
*
* @throws IOException when read/write error occurs
*/
public synchronized static void writeDatanodeIDTo(DatanodeID dnID,
File path) throws IOException {
List<DatanodeID> ids = ContainerUtils.readDatanodeIDsFrom(path);
// Only create or overwrite the file
// if the ID doesn't exist in the ID file
for (DatanodeID id : ids) {
if (id.getProtoBufMessage()
.equals(dnID.getProtoBufMessage())) {
return;
}
}
ids.add(dnID);
writeDatanodeIDs(ids, path);
}
/**
* Read {@link DatanodeID} from a local ID file and return a set of
* datanode IDs. If the ID file doesn't exist, an empty set is returned.
* Read {@link DatanodeDetails} from a local ID file.
*
* @param path ID file local path
* @return A set of {@link DatanodeID}
* @return {@link DatanodeDetails}
* @throws IOException If the id file is malformed or other I/O exceptions
*/
public synchronized static List<DatanodeID> readDatanodeIDsFrom(File path)
public synchronized static DatanodeDetails readDatanodeDetailsFrom(File path)
throws IOException {
List<DatanodeID> ids = new ArrayList<DatanodeID>();
if (!path.exists()) {
return ids;
throw new IOException("Datanode ID file not found.");
}
try(FileInputStream in = new FileInputStream(path)) {
while(in.available() > 0) {
try {
HdfsProtos.DatanodeIDProto id =
HdfsProtos.DatanodeIDProto.parseDelimitedFrom(in);
ids.add(DatanodeID.getFromProtoBuf(id));
return DatanodeDetails.getFromProtoBuf(
HdslProtos.DatanodeDetailsProto.parseFrom(in));
} catch (IOException e) {
throw new IOException("Failed to parse Datanode ID from "
throw new IOException("Failed to parse DatanodeDetails from "
+ path.getAbsolutePath(), e);
}
}
}
return ids;
}
}

View File

@ -21,9 +21,9 @@
import org.apache.hadoop.classification.InterfaceAudience;
import org.apache.hadoop.classification.InterfaceStability;
import org.apache.hadoop.conf.Configuration;
import org.apache.hadoop.hdfs.protocol.DatanodeID;
import org.apache.hadoop.hdfs.server.datanode.StorageLocation;
import org.apache.hadoop.hdfs.util.RwLock;
import org.apache.hadoop.hdsl.protocol.DatanodeDetails;
import org.apache.hadoop.hdsl.protocol.proto.StorageContainerDatanodeProtocolProtos.ContainerReportsRequestProto;
import org.apache.hadoop.hdsl.protocol.proto.StorageContainerDatanodeProtocolProtos.ReportState;
import org.apache.hadoop.hdsl.protocol.proto.StorageContainerDatanodeProtocolProtos.SCMNodeReport;
@ -48,11 +48,11 @@ public interface ContainerManager extends RwLock {
*
* @param config - Configuration.
* @param containerDirs - List of Metadata Container locations.
* @param datanodeID - Datanode ID
* @param datanodeDetails - DatanodeDetails
* @throws StorageContainerException
*/
void init(Configuration config, List<StorageLocation> containerDirs,
DatanodeID datanodeID) throws IOException;
DatanodeDetails datanodeDetails) throws IOException;
/**
* Creates a container with the given name.

View File

@ -19,8 +19,8 @@
import com.google.common.annotations.VisibleForTesting;
import com.google.common.util.concurrent.ThreadFactoryBuilder;
import org.apache.hadoop.conf.Configuration;
import org.apache.hadoop.hdfs.protocol.DatanodeID;
import org.apache.hadoop.hdsl.conf.OzoneConfiguration;
import org.apache.hadoop.hdsl.protocol.DatanodeDetails;
import org.apache.hadoop.ozone.container.common.statemachine.commandhandler
.CloseContainerHandler;
import org.apache.hadoop.ozone.container.common.statemachine.commandhandler.CommandDispatcher;
@ -54,7 +54,7 @@ public class DatanodeStateMachine implements Closeable {
private final long heartbeatFrequency;
private StateContext context;
private final OzoneContainer container;
private DatanodeID datanodeID = null;
private DatanodeDetails datanodeDetails;
private final CommandDispatcher commandDispatcher;
private long commandsHandled;
private AtomicLong nextHB;
@ -64,12 +64,13 @@ public class DatanodeStateMachine implements Closeable {
/**
* Constructs a a datanode state machine.
*
* @param datanodeID - DatanodeID used to identify a datanode
* @param datanodeDetails - DatanodeDetails used to identify a datanode
* @param conf - Configuration.
*/
public DatanodeStateMachine(DatanodeID datanodeID,
public DatanodeStateMachine(DatanodeDetails datanodeDetails,
Configuration conf) throws IOException {
this.conf = conf;
this.datanodeDetails = datanodeDetails;
executorService = HadoopExecutors.newCachedThreadPool(
new ThreadFactoryBuilder().setDaemon(true)
.setNameFormat("Datanode State Machine Thread - %d").build());
@ -77,8 +78,8 @@ public DatanodeStateMachine(DatanodeID datanodeID,
context = new StateContext(this.conf, DatanodeStates.getInitState(), this);
heartbeatFrequency = TimeUnit.SECONDS.toMillis(
getScmHeartbeatInterval(conf));
container = new OzoneContainer(datanodeID, new OzoneConfiguration(conf));
this.datanodeID = datanodeID;
container = new OzoneContainer(this.datanodeDetails,
new OzoneConfiguration(conf));
nextHB = new AtomicLong(Time.monotonicNow());
// When we add new handlers just adding a new handler here should do the
@ -94,20 +95,17 @@ public DatanodeStateMachine(DatanodeID datanodeID,
.build();
}
public void setDatanodeID(DatanodeID datanodeID) {
this.datanodeID = datanodeID;
}
/**
*
* Return DatanodeID if set, return null otherwise.
* Return DatanodeDetails if set, return null otherwise.
*
* @return datanodeID
* @return DatanodeDetails
*/
public DatanodeID getDatanodeID() {
return this.datanodeID;
public DatanodeDetails getDatanodeDetails() {
return datanodeDetails;
}
/**
* Returns the Connection manager for this state machine.
*

View File

@ -18,15 +18,15 @@
import com.google.common.base.Strings;
import org.apache.hadoop.conf.Configuration;
import org.apache.hadoop.hdfs.protocol.DatanodeID;
import org.apache.hadoop.hdsl.HdslUtils;
import org.apache.hadoop.hdsl.protocol.DatanodeDetails;
import org.apache.hadoop.ozone.container.common.helpers.ContainerUtils;
import org.apache.hadoop.ozone.container.common.statemachine.DatanodeStateMachine;
import org.apache.hadoop.ozone.container.common.statemachine.SCMConnectionManager;
import org.apache.hadoop.ozone.container.common.statemachine.StateContext;
import org.apache.hadoop.ozone.container.common.states.DatanodeState;
import org.apache.hadoop.scm.ScmConfigKeys;
import org.apache.hadoop.scm.ScmConfigKeys;
import org.slf4j.Logger;
import org.slf4j.LoggerFactory;
@ -97,32 +97,22 @@ public DatanodeStateMachine.DatanodeStates call() throws Exception {
}
// If datanode ID is set, persist it to the ID file.
persistContainerDatanodeID();
persistContainerDatanodeDetails();
return this.context.getState().getNextState();
}
/**
* Update Ozone container port to the datanode ID,
* and persist the ID to a local file.
* Persist DatanodeDetails to datanode.id file.
*/
private void persistContainerDatanodeID() throws IOException {
String dataNodeIDPath = HdslUtils.getDatanodeIDPath(conf);
if (Strings.isNullOrEmpty(dataNodeIDPath)) {
LOG.error("A valid file path is needed for config setting {}",
ScmConfigKeys.OZONE_SCM_DATANODE_ID);
this.context.setState(DatanodeStateMachine.DatanodeStates.SHUTDOWN);
return;
}
private void persistContainerDatanodeDetails() throws IOException {
String dataNodeIDPath = HdslUtils.getDatanodeIdFilePath(conf);
File idPath = new File(dataNodeIDPath);
int containerPort = this.context.getContainerPort();
int ratisPort = this.context.getRatisPort();
DatanodeID datanodeID = this.context.getParent().getDatanodeID();
if (datanodeID != null) {
datanodeID.setContainerPort(containerPort);
datanodeID.setRatisPort(ratisPort);
ContainerUtils.writeDatanodeIDTo(datanodeID, idPath);
LOG.info("Datanode ID is persisted to {}", dataNodeIDPath);
DatanodeDetails datanodeDetails = this.context.getParent()
.getDatanodeDetails();
if (datanodeDetails != null && !idPath.exists()) {
ContainerUtils.writeDatanodeDetailsTo(datanodeDetails, idPath);
LOG.info("DatanodeDetails is persisted to {}", dataNodeIDPath);
}
}

View File

@ -16,11 +16,7 @@
*/
package org.apache.hadoop.ozone.container.common.states.datanode;
import com.google.common.base.Preconditions;
import org.apache.hadoop.conf.Configuration;
import org.apache.hadoop.hdfs.protocol.DatanodeID;
import org.apache.hadoop.hdsl.HdslUtils;
import org.apache.hadoop.ozone.container.common.helpers.ContainerUtils;
import org.apache.hadoop.ozone.container.common.statemachine.DatanodeStateMachine;
import org.apache.hadoop.ozone.container.common.statemachine.EndpointStateMachine;
import org.apache.hadoop.ozone.container.common.statemachine.SCMConnectionManager;
@ -29,15 +25,10 @@
import org.apache.hadoop.ozone.container.common.states.endpoint.HeartbeatEndpointTask;
import org.apache.hadoop.ozone.container.common.states.endpoint.RegisterEndpointTask;
import org.apache.hadoop.ozone.container.common.states.endpoint.VersionEndpointTask;
import org.apache.hadoop.hdsl.protocol.proto.StorageContainerDatanodeProtocolProtos;
import org.apache.hadoop.scm.ScmConfigKeys;
import org.apache.hadoop.util.Time;
import org.slf4j.Logger;
import org.slf4j.LoggerFactory;
import java.io.IOException;
import java.nio.file.Path;
import java.nio.file.Paths;
import java.util.LinkedList;
import java.util.List;
import java.util.concurrent.Callable;
@ -68,75 +59,6 @@ public RunningDatanodeState(Configuration conf,
this.context = context;
}
/**
* Reads a datanode ID from the persisted information.
*
* @param idPath - Path to the ID File.
* @return DatanodeID
* @throws IOException
*/
private StorageContainerDatanodeProtocolProtos.ContainerNodeIDProto
readPersistedDatanodeID(Path idPath) throws IOException {
Preconditions.checkNotNull(idPath);
DatanodeID datanodeID = null;
List<DatanodeID> datanodeIDs =
ContainerUtils.readDatanodeIDsFrom(idPath.toFile());
int containerPort = this.context.getContainerPort();
for(DatanodeID dnId : datanodeIDs) {
if(dnId.getContainerPort() == containerPort) {
datanodeID = dnId;
break;
}
}
if (datanodeID == null) {
throw new IOException("No valid datanode ID found from "
+ idPath.toFile().getAbsolutePath()
+ " that matches container port "
+ containerPort);
} else {
StorageContainerDatanodeProtocolProtos.ContainerNodeIDProto
containerIDProto =
StorageContainerDatanodeProtocolProtos
.ContainerNodeIDProto
.newBuilder()
.setDatanodeID(datanodeID.getProtoBufMessage())
.build();
return containerIDProto;
}
}
/**
* Returns ContainerNodeIDProto or null in case of Error.
*
* @return ContainerNodeIDProto
*/
private StorageContainerDatanodeProtocolProtos.ContainerNodeIDProto
getContainerNodeID() {
String dataNodeIDPath = HdslUtils.getDatanodeIDPath(conf);
if (dataNodeIDPath == null || dataNodeIDPath.isEmpty()) {
LOG.error("A valid file path is needed for config setting {}",
ScmConfigKeys.OZONE_SCM_DATANODE_ID);
// This is an unrecoverable error.
this.context.setState(DatanodeStateMachine.DatanodeStates.SHUTDOWN);
return null;
}
StorageContainerDatanodeProtocolProtos.ContainerNodeIDProto nodeID;
// try to read an existing ContainerNode ID.
try {
nodeID = readPersistedDatanodeID(Paths.get(dataNodeIDPath));
if (nodeID != null) {
LOG.trace("Read Node ID :", nodeID.getDatanodeID().getDatanodeUuid());
return nodeID;
}
} catch (IOException ex) {
LOG.trace("Not able to find container Node ID, creating it.", ex);
}
this.context.setState(DatanodeStateMachine.DatanodeStates.SHUTDOWN);
return null;
}
/**
* Called before entering this state.
*/
@ -178,13 +100,13 @@ public void execute(ExecutorService executor) {
return RegisterEndpointTask.newBuilder()
.setConfig(conf)
.setEndpointStateMachine(endpoint)
.setNodeID(getContainerNodeID())
.setDatanodeDetails(context.getParent().getDatanodeDetails())
.build();
case HEARTBEAT:
return HeartbeatEndpointTask.newBuilder()
.setConfig(conf)
.setEndpointStateMachine(endpoint)
.setNodeID(getContainerNodeID())
.setDatanodeDetails(context.getParent().getDatanodeDetails())
.setContext(context)
.build();
case SHUTDOWN:

View File

@ -20,7 +20,8 @@
import com.google.common.base.Preconditions;
import org.apache.hadoop.conf.Configuration;
import org.apache.hadoop.hdfs.protocol.DatanodeID;
import org.apache.hadoop.hdsl.protocol.DatanodeDetails;
import org.apache.hadoop.hdsl.protocol.proto.HdslProtos.DatanodeDetailsProto;
import org.apache.hadoop.ozone.container.common.helpers
.DeletedContainerBlocksSummary;
import org.apache.hadoop.ozone.container.common.statemachine
@ -30,8 +31,6 @@
import org.apache.hadoop.ozone.container.common.statemachine.StateContext;
import org.apache.hadoop.ozone.protocol.commands.CloseContainerCommand;
import org.apache.hadoop.ozone.protocol.commands.DeleteBlocksCommand;
import org.apache.hadoop.hdsl.protocol.proto
.StorageContainerDatanodeProtocolProtos.ContainerNodeIDProto;
import org.apache.hadoop.ozone.protocol.commands.SendContainerCommand;
import org.apache.hadoop.hdsl.protocol.proto
.StorageContainerDatanodeProtocolProtos.SCMCommandResponseProto;
@ -53,7 +52,7 @@ public class HeartbeatEndpointTask
LoggerFactory.getLogger(HeartbeatEndpointTask.class);
private final EndpointStateMachine rpcEndpoint;
private final Configuration conf;
private ContainerNodeIDProto containerNodeIDProto;
private DatanodeDetailsProto datanodeDetailsProto;
private StateContext context;
/**
@ -73,18 +72,18 @@ public HeartbeatEndpointTask(EndpointStateMachine rpcEndpoint,
*
* @return ContainerNodeIDProto
*/
public ContainerNodeIDProto getContainerNodeIDProto() {
return containerNodeIDProto;
public DatanodeDetailsProto getDatanodeDetailsProto() {
return datanodeDetailsProto;
}
/**
* Set container node ID proto.
*
* @param containerNodeIDProto - the node id.
* @param datanodeDetailsProto - the node id.
*/
public void setContainerNodeIDProto(ContainerNodeIDProto
containerNodeIDProto) {
this.containerNodeIDProto = containerNodeIDProto;
public void setDatanodeDetailsProto(DatanodeDetailsProto
datanodeDetailsProto) {
this.datanodeDetailsProto = datanodeDetailsProto;
}
/**
@ -97,14 +96,12 @@ public void setContainerNodeIDProto(ContainerNodeIDProto
public EndpointStateMachine.EndPointStates call() throws Exception {
rpcEndpoint.lock();
try {
Preconditions.checkState(this.containerNodeIDProto != null);
DatanodeID datanodeID = DatanodeID.getFromProtoBuf(this
.containerNodeIDProto.getDatanodeID());
Preconditions.checkState(this.datanodeDetailsProto != null);
SCMHeartbeatResponseProto reponse = rpcEndpoint.getEndPoint()
.sendHeartbeat(datanodeID, this.context.getNodeReport(),
.sendHeartbeat(datanodeDetailsProto, this.context.getNodeReport(),
this.context.getContainerReportState());
processResponse(reponse, datanodeID);
processResponse(reponse, datanodeDetailsProto);
rpcEndpoint.setLastSuccessfulHeartbeat(ZonedDateTime.now());
rpcEndpoint.zeroMissedCount();
} catch (IOException ex) {
@ -129,12 +126,12 @@ public static Builder newBuilder() {
* @param response - SCMHeartbeat response.
*/
private void processResponse(SCMHeartbeatResponseProto response,
final DatanodeID datanodeID) {
final DatanodeDetailsProto datanodeDetails) {
for (SCMCommandResponseProto commandResponseProto : response
.getCommandsList()) {
// Verify the response is indeed for this datanode.
Preconditions.checkState(commandResponseProto.getDatanodeUUID()
.equalsIgnoreCase(datanodeID.getDatanodeUuid().toString()),
.equalsIgnoreCase(datanodeDetails.getUuid()),
"Unexpected datanode ID in the response.");
switch (commandResponseProto.getCmdType()) {
case sendContainerReport:
@ -190,7 +187,7 @@ private void processResponse(SCMHeartbeatResponseProto response,
public static class Builder {
private EndpointStateMachine endPointStateMachine;
private Configuration conf;
private ContainerNodeIDProto containerNodeIDProto;
private DatanodeDetails datanodeDetails;
private StateContext context;
/**
@ -224,11 +221,11 @@ public Builder setConfig(Configuration config) {
/**
* Sets the NodeID.
*
* @param nodeID - NodeID proto
* @param dnDetails - NodeID proto
* @return Builder
*/
public Builder setNodeID(ContainerNodeIDProto nodeID) {
this.containerNodeIDProto = nodeID;
public Builder setDatanodeDetails(DatanodeDetails dnDetails) {
this.datanodeDetails = dnDetails;
return this;
}
@ -255,15 +252,15 @@ public HeartbeatEndpointTask build() {
" construct HeartbeatEndpointTask task");
}
if (containerNodeIDProto == null) {
LOG.error("No nodeID specified.");
if (datanodeDetails == null) {
LOG.error("No datanode specified.");
throw new IllegalArgumentException("A vaild Node ID is needed to " +
"construct HeartbeatEndpointTask task");
}
HeartbeatEndpointTask task = new HeartbeatEndpointTask(this
.endPointStateMachine, this.conf, this.context);
task.setContainerNodeIDProto(containerNodeIDProto);
task.setDatanodeDetailsProto(datanodeDetails.getProtoBufMessage());
return task;
}
}

View File

@ -18,12 +18,11 @@
import com.google.common.annotations.VisibleForTesting;
import org.apache.hadoop.conf.Configuration;
import org.apache.hadoop.hdfs.protocol.DatanodeID;
import org.apache.hadoop.hdsl.protocol.DatanodeDetails;
import org.apache.hadoop.hdsl.protocol.proto.HdslProtos.DatanodeDetailsProto;
import org.apache.hadoop.ozone.container.common.statemachine
.EndpointStateMachine;
import org.apache.hadoop.hdsl.protocol.proto
.StorageContainerDatanodeProtocolProtos.ContainerNodeIDProto;
import org.apache.hadoop.scm.ScmConfigKeys;
import org.slf4j.Logger;
import org.slf4j.LoggerFactory;
@ -42,7 +41,7 @@ public final class RegisterEndpointTask implements
private final EndpointStateMachine rpcEndPoint;
private final Configuration conf;
private Future<EndpointStateMachine.EndPointStates> result;
private ContainerNodeIDProto containerNodeIDProto;
private DatanodeDetailsProto datanodeDetailsProto;
/**
* Creates a register endpoint task.
@ -59,22 +58,22 @@ public RegisterEndpointTask(EndpointStateMachine rpcEndPoint,
}
/**
* Get the ContainerNodeID Proto.
* Get the DatanodeDetailsProto Proto.
*
* @return ContainerNodeIDProto
* @return DatanodeDetailsProto
*/
public ContainerNodeIDProto getContainerNodeIDProto() {
return containerNodeIDProto;
public DatanodeDetailsProto getDatanodeDetailsProto() {
return datanodeDetailsProto;
}
/**
* Set the contiainerNodeID Proto.
*
* @param containerNodeIDProto - Container Node ID.
* @param datanodeDetailsProto - Container Node ID.
*/
public void setContainerNodeIDProto(ContainerNodeIDProto
containerNodeIDProto) {
this.containerNodeIDProto = containerNodeIDProto;
public void setDatanodeDetailsProto(
DatanodeDetailsProto datanodeDetailsProto) {
this.datanodeDetailsProto = datanodeDetailsProto;
}
/**
@ -86,7 +85,7 @@ public void setContainerNodeIDProto(ContainerNodeIDProto
@Override
public EndpointStateMachine.EndPointStates call() throws Exception {
if (getContainerNodeIDProto() == null) {
if (getDatanodeDetailsProto() == null) {
LOG.error("Container ID proto cannot be null in RegisterEndpoint task, " +
"shutting down the endpoint.");
return rpcEndPoint.setState(EndpointStateMachine.EndPointStates.SHUTDOWN);
@ -94,11 +93,9 @@ public EndpointStateMachine.EndPointStates call() throws Exception {
rpcEndPoint.lock();
try {
DatanodeID dnNodeID = DatanodeID.getFromProtoBuf(
getContainerNodeIDProto().getDatanodeID());
// TODO : Add responses to the command Queue.
rpcEndPoint.getEndPoint().register(dnNodeID,
rpcEndPoint.getEndPoint().register(datanodeDetailsProto,
conf.getStrings(ScmConfigKeys.OZONE_SCM_NAMES));
EndpointStateMachine.EndPointStates nextState =
rpcEndPoint.getState().getNextState();
@ -129,7 +126,7 @@ public static Builder newBuilder() {
public static class Builder {
private EndpointStateMachine endPointStateMachine;
private Configuration conf;
private ContainerNodeIDProto containerNodeIDProto;
private DatanodeDetails datanodeDetails;
/**
* Constructs the builder class.
@ -162,11 +159,11 @@ public Builder setConfig(Configuration config) {
/**
* Sets the NodeID.
*
* @param nodeID - NodeID proto
* @param dnDetails - NodeID proto
* @return Builder
*/
public Builder setNodeID(ContainerNodeIDProto nodeID) {
this.containerNodeIDProto = nodeID;
public Builder setDatanodeDetails(DatanodeDetails dnDetails) {
this.datanodeDetails = dnDetails;
return this;
}
@ -183,15 +180,15 @@ public RegisterEndpointTask build() {
" construct RegisterEndpoint task");
}
if (containerNodeIDProto == null) {
LOG.error("No nodeID specified.");
if (datanodeDetails == null) {
LOG.error("No datanode specified.");
throw new IllegalArgumentException("A vaild Node ID is needed to " +
"construct RegisterEndpoint task");
}
RegisterEndpointTask task = new RegisterEndpointTask(this
.endPointStateMachine, this.conf);
task.setContainerNodeIDProto(containerNodeIDProto);
task.setDatanodeDetailsProto(datanodeDetails.getProtoBufMessage());
return task;
}
}

View File

@ -27,6 +27,7 @@
import io.netty.handler.logging.LogLevel;
import io.netty.handler.logging.LoggingHandler;
import org.apache.hadoop.conf.Configuration;
import org.apache.hadoop.hdsl.protocol.DatanodeDetails;
import org.apache.hadoop.ozone.OzoneConfigKeys;
import org.apache.hadoop.ozone.container.common.interfaces.ContainerDispatcher;
import org.apache.hadoop.hdsl.protocol.proto.HdslProtos;
@ -57,7 +58,7 @@ public final class XceiverServer implements XceiverServerSpi {
*
* @param conf - Configuration
*/
public XceiverServer(Configuration conf,
public XceiverServer(DatanodeDetails datanodeDetails, Configuration conf,
ContainerDispatcher dispatcher) {
Preconditions.checkNotNull(conf);
@ -78,6 +79,7 @@ public XceiverServer(Configuration conf,
+ "fallback to use default port {}", this.port, e);
}
}
datanodeDetails.setContainerPort(port);
this.storageContainer = dispatcher;
}

View File

@ -21,7 +21,7 @@
import com.google.common.base.Preconditions;
import com.google.common.base.Strings;
import org.apache.hadoop.conf.Configuration;
import org.apache.hadoop.hdfs.protocol.DatanodeID;
import org.apache.hadoop.hdsl.protocol.DatanodeDetails;
import org.apache.hadoop.ozone.OzoneConfigKeys;
import org.apache.hadoop.ozone.container.common.interfaces.ContainerDispatcher;
import org.apache.hadoop.ozone.container.common.transport.server
@ -62,7 +62,7 @@ public final class XceiverServerRatis implements XceiverServerSpi {
private final RaftServer server;
private ThreadPoolExecutor writeChunkExecutor;
private XceiverServerRatis(DatanodeID id, int port, String storageDir,
private XceiverServerRatis(DatanodeDetails dd, int port, String storageDir,
ContainerDispatcher dispatcher, Configuration conf) throws IOException {
final String rpcType = conf.get(
@ -80,7 +80,7 @@ private XceiverServerRatis(DatanodeID id, int port, String storageDir,
OzoneConfigKeys.DFS_CONTAINER_RATIS_NUM_WRITE_CHUNK_THREADS_KEY,
OzoneConfigKeys.DFS_CONTAINER_RATIS_NUM_WRITE_CHUNK_THREADS_DEFAULT);
Objects.requireNonNull(id, "id == null");
Objects.requireNonNull(dd, "id == null");
this.port = port;
RaftProperties serverProperties = newRaftProperties(rpc, port,
storageDir, maxChunkSize, raftSegmentSize, raftSegmentPreallocatedSize);
@ -93,7 +93,7 @@ private XceiverServerRatis(DatanodeID id, int port, String storageDir,
ContainerStateMachine stateMachine =
new ContainerStateMachine(dispatcher, writeChunkExecutor);
this.server = RaftServer.newBuilder()
.setServerId(RatisHelper.toRaftPeerId(id))
.setServerId(RatisHelper.toRaftPeerId(dd))
.setGroup(RatisHelper.emptyRaftGroup())
.setProperties(serverProperties)
.setStateMachine(stateMachine)
@ -131,9 +131,9 @@ private static RaftProperties newRaftProperties(
return properties;
}
public static XceiverServerRatis newXceiverServerRatis(DatanodeID datanodeID,
Configuration ozoneConf, ContainerDispatcher dispatcher)
throws IOException {
public static XceiverServerRatis newXceiverServerRatis(
DatanodeDetails datanodeDetails, Configuration ozoneConf,
ContainerDispatcher dispatcher) throws IOException {
final String ratisDir = File.separator + "ratis";
int localPort = ozoneConf.getInt(
OzoneConfigKeys.DFS_CONTAINER_RATIS_IPC_PORT,
@ -168,14 +168,15 @@ public static XceiverServerRatis newXceiverServerRatis(DatanodeID datanodeID,
// directories, so we need to pass different local directory for each
// local instance. So we map ratis directories under datanode ID.
storageDir =
storageDir.concat(File.separator + datanodeID.getDatanodeUuid());
storageDir.concat(File.separator +
datanodeDetails.getUuidString());
} catch (IOException e) {
LOG.error("Unable find a random free port for the server, "
+ "fallback to use default port {}", localPort, e);
}
}
datanodeID.setRatisPort(localPort);
return new XceiverServerRatis(datanodeID, localPort, storageDir,
datanodeDetails.setRatisPort(localPort);
return new XceiverServerRatis(datanodeDetails, localPort, storageDir,
dispatcher, ozoneConf);
}

View File

@ -19,8 +19,8 @@
import com.google.common.annotations.VisibleForTesting;
import org.apache.hadoop.conf.Configuration;
import org.apache.hadoop.hdfs.protocol.DatanodeID;
import org.apache.hadoop.hdfs.server.datanode.StorageLocation;
import org.apache.hadoop.hdsl.protocol.DatanodeDetails;
import org.apache.hadoop.ozone.OzoneConfigKeys;
import org.apache.hadoop.ozone.container.common.helpers.ContainerData;
import org.apache.hadoop.ozone.container.common.impl.ChunkManagerImpl;
@ -81,8 +81,9 @@ public class OzoneContainer {
* @param ozoneConfig - Config
* @throws IOException
*/
public OzoneContainer(DatanodeID datanodeID, Configuration ozoneConfig) throws
IOException {
public OzoneContainer(
DatanodeDetails datanodeDetails, Configuration ozoneConfig)
throws IOException {
this.ozoneConfig = ozoneConfig;
List<StorageLocation> locations = new LinkedList<>();
String[] paths = ozoneConfig.getStrings(
@ -97,7 +98,7 @@ public OzoneContainer(DatanodeID datanodeID, Configuration ozoneConfig) throws
}
manager = new ContainerManagerImpl();
manager.init(this.ozoneConfig, locations, datanodeID);
manager.init(this.ozoneConfig, locations, datanodeDetails);
this.chunkManager = new ChunkManagerImpl(manager);
manager.setChunkManager(this.chunkManager);
@ -116,9 +117,9 @@ public OzoneContainer(DatanodeID datanodeID, Configuration ozoneConfig) throws
this.dispatcher = new Dispatcher(manager, this.ozoneConfig);
server = new XceiverServerSpi[]{
new XceiverServer(this.ozoneConfig, this.dispatcher),
new XceiverServer(datanodeDetails, this.ozoneConfig, this.dispatcher),
XceiverServerRatis
.newXceiverServerRatis(datanodeID, ozoneConfig, dispatcher)
.newXceiverServerRatis(datanodeDetails, this.ozoneConfig, dispatcher)
};
}

View File

@ -17,7 +17,7 @@
package org.apache.hadoop.ozone.protocol;
import org.apache.hadoop.classification.InterfaceAudience;
import org.apache.hadoop.hdfs.protocol.DatanodeID;
import org.apache.hadoop.hdsl.protocol.proto.HdslProtos.DatanodeDetailsProto;
import org.apache.hadoop.hdsl.protocol.proto.StorageContainerDatanodeProtocolProtos.ContainerReportsRequestProto;
import org.apache.hadoop.hdsl.protocol.proto.StorageContainerDatanodeProtocolProtos.ContainerReportsResponseProto;
import org.apache.hadoop.hdsl.protocol.proto.StorageContainerDatanodeProtocolProtos.ReportState;
@ -45,23 +45,23 @@ SCMVersionResponseProto getVersion(SCMVersionRequestProto versionRequest)
/**
* Used by data node to send a Heartbeat.
* @param datanodeID - Datanode ID.
* @param datanodeDetails - Datanode Details.
* @param nodeReport - node report state
* @param reportState - container report state.
* @return - SCMHeartbeatResponseProto
* @throws IOException
*/
SCMHeartbeatResponseProto sendHeartbeat(DatanodeID datanodeID,
SCMHeartbeatResponseProto sendHeartbeat(DatanodeDetailsProto datanodeDetails,
SCMNodeReport nodeReport, ReportState reportState) throws IOException;
/**
* Register Datanode.
* @param datanodeID - DatanodID.
* @param datanodeDetails - Datanode Details.
* @param scmAddresses - List of SCMs this datanode is configured to
* communicate.
* @return SCM Command.
*/
SCMRegisteredCmdResponseProto register(DatanodeID datanodeID,
SCMRegisteredCmdResponseProto register(DatanodeDetailsProto datanodeDetails,
String[] scmAddresses) throws IOException;
/**

View File

@ -18,7 +18,7 @@
package org.apache.hadoop.ozone.protocol;
import org.apache.hadoop.classification.InterfaceAudience;
import org.apache.hadoop.hdfs.protocol.DatanodeID;
import org.apache.hadoop.hdsl.protocol.proto.HdslProtos.DatanodeDetailsProto;
import org.apache.hadoop.ozone.protocol.commands.SCMCommand;
import org.apache.hadoop.hdsl.protocol.proto
.StorageContainerDatanodeProtocolProtos.ReportState;
@ -50,20 +50,19 @@ public interface StorageContainerNodeProtocol {
/**
* Register the node if the node finds that it is not registered with any SCM.
* @param datanodeID - Send datanodeID with Node info, but datanode UUID is
* empty. Server returns a datanodeID for the given node.
* @param datanodeDetails DatanodeDetails
* @return SCMHeartbeatResponseProto
*/
SCMCommand register(DatanodeID datanodeID);
SCMCommand register(DatanodeDetailsProto datanodeDetails);
/**
* Send heartbeat to indicate the datanode is alive and doing well.
* @param datanodeID - Datanode ID.
* @param datanodeDetails - Datanode ID.
* @param nodeReport - node report.
* @param reportState - container report.
* @return SCMheartbeat response list
*/
List<SCMCommand> sendHeartbeat(DatanodeID datanodeID,
List<SCMCommand> sendHeartbeat(DatanodeDetailsProto datanodeDetails,
SCMNodeReport nodeReport, ReportState reportState);
}

View File

@ -18,7 +18,7 @@
import com.google.protobuf.RpcController;
import com.google.protobuf.ServiceException;
import org.apache.hadoop.hdfs.protocol.DatanodeID;
import org.apache.hadoop.hdsl.protocol.proto.HdslProtos.DatanodeDetailsProto;
import org.apache.hadoop.ipc.ProtobufHelper;
import org.apache.hadoop.ipc.ProtocolTranslator;
import org.apache.hadoop.ipc.RPC;
@ -118,17 +118,18 @@ public SCMVersionResponseProto getVersion(SCMVersionRequestProto
/**
* Send by datanode to SCM.
*
* @param datanodeID - DatanodeID
* @param datanodeDetailsProto - Datanode Details
* @param nodeReport - node report
* @throws IOException
*/
@Override
public SCMHeartbeatResponseProto sendHeartbeat(DatanodeID datanodeID,
public SCMHeartbeatResponseProto sendHeartbeat(
DatanodeDetailsProto datanodeDetailsProto,
SCMNodeReport nodeReport, ReportState reportState) throws IOException {
SCMHeartbeatRequestProto.Builder req = SCMHeartbeatRequestProto
.newBuilder();
req.setDatanodeID(datanodeID.getProtoBufMessage());
req.setDatanodeDetails(datanodeDetailsProto);
req.setNodeReport(nodeReport);
req.setContainerReportState(reportState);
final SCMHeartbeatResponseProto resp;
@ -143,15 +144,16 @@ public SCMHeartbeatResponseProto sendHeartbeat(DatanodeID datanodeID,
/**
* Register Datanode.
*
* @param datanodeID - DatanodID.
* @param datanodeDetailsProto - Datanode Details
* @return SCM Command.
*/
@Override
public SCMRegisteredCmdResponseProto register(DatanodeID datanodeID,
public SCMRegisteredCmdResponseProto register(
DatanodeDetailsProto datanodeDetailsProto,
String[] scmAddresses) throws IOException {
SCMRegisterRequestProto.Builder req =
SCMRegisterRequestProto.newBuilder();
req.setDatanodeID(datanodeID.getProtoBufMessage());
req.setDatanodeDetails(datanodeDetailsProto);
final SCMRegisteredCmdResponseProto response;
try {
response = rpcProxy.register(NULL_RPC_CONTROLLER, req.build());

View File

@ -18,7 +18,6 @@
import com.google.protobuf.RpcController;
import com.google.protobuf.ServiceException;
import org.apache.hadoop.hdfs.protocol.DatanodeID;
import org.apache.hadoop.ozone.protocol.StorageContainerDatanodeProtocol;
import org.apache.hadoop.hdsl.protocol.proto.StorageContainerDatanodeProtocolProtos;
import org.apache.hadoop.hdsl.protocol.proto.StorageContainerDatanodeProtocolProtos.ContainerReportsRequestProto;
@ -69,8 +68,7 @@ public StorageContainerDatanodeProtocolServerSideTranslatorPB(
}
try {
return impl.register(DatanodeID.getFromProtoBuf(request
.getDatanodeID()), addressArray);
return impl.register(request.getDatanodeDetails(), addressArray);
} catch (IOException e) {
throw new ServiceException(e);
}
@ -81,8 +79,8 @@ public StorageContainerDatanodeProtocolServerSideTranslatorPB(
sendHeartbeat(RpcController controller,
SCMHeartbeatRequestProto request) throws ServiceException {
try {
return impl.sendHeartbeat(DatanodeID.getFromProtoBuf(request
.getDatanodeID()), request.getNodeReport(),
return impl.sendHeartbeat(request.getDatanodeDetails(),
request.getNodeReport(),
request.getContainerReportState());
} catch (IOException e) {
throw new ServiceException(e);

View File

@ -32,12 +32,6 @@ option java_generate_equals_and_hash = true;
package hadoop.hdsl;
import "hdfs.proto";
import "HdfsServer.proto";
import "DatanodeProtocol.proto";
import "hdsl.proto";
@ -46,7 +40,7 @@ import "hdsl.proto";
* registering with the node manager.
*/
message SCMHeartbeatRequestProto {
required hadoop.hdfs.DatanodeIDProto datanodeID = 1;
required DatanodeDetailsProto datanodeDetails = 1;
optional SCMNodeReport nodeReport = 2;
optional ReportState containerReportState = 3;
}
@ -125,7 +119,7 @@ message ContainerReportsRequestProto {
fullReport = 0;
deltaReport = 1;
}
required hadoop.hdfs.DatanodeIDProto datanodeID = 1;
required DatanodeDetailsProto datanodeDetails = 1;
repeated ContainerInfo reports = 2;
required reportType type = 3;
}
@ -146,11 +140,11 @@ message SCMStorageReport {
optional uint64 capacity = 2 [default = 0];
optional uint64 scmUsed = 3 [default = 0];
optional uint64 remaining = 4 [default = 0];
optional hadoop.hdfs.StorageTypeProto storageType = 5 [default = DISK];
//optional hadoop.hdfs.StorageTypeProto storageType = 5 [default = DISK];
}
message SCMRegisterRequestProto {
required hadoop.hdfs.DatanodeIDProto datanodeID = 1;
required DatanodeDetailsProto datanodeDetails = 1;
optional SCMNodeAddressList addressList = 2;
}
@ -195,17 +189,6 @@ message SCMRegisteredCmdResponseProto {
*/
message SCMReregisterCmdResponseProto {}
/**
* Container ID maintains the container's Identity along with cluster ID
* after the registration is done.
*/
message ContainerNodeIDProto {
required hadoop.hdfs.DatanodeIDProto datanodeID = 1;
optional string clusterID = 2;
}
/**
This command tells the data node to send in the container report when possible
*/
@ -326,7 +309,8 @@ message ContainerBlocksDeletionACKProto {
* it needs to do a registration.
*
* If registration is need datanode moves into REGISTER state. It will
* send a register call with datanodeID data structure and presist that info.
* send a register call with DatanodeDetailsProto data structure and presist
* that info.
*
* The response to the command contains clusterID. This information is
* also persisted by the datanode and moves into heartbeat state.

View File

@ -17,7 +17,8 @@
package org.apache.hadoop.ozone.container.common;
import com.google.common.base.Preconditions;
import org.apache.hadoop.hdfs.protocol.DatanodeID;
import org.apache.hadoop.hdsl.protocol.DatanodeDetails;
import org.apache.hadoop.hdsl.protocol.proto.HdslProtos.DatanodeDetailsProto;
import org.apache.hadoop.ozone.protocol.StorageContainerDatanodeProtocol;
import org.apache.hadoop.ozone.protocol.VersionResponse;
import org.apache.hadoop.hdsl.protocol.proto.StorageContainerDatanodeProtocolProtos;
@ -50,7 +51,7 @@ public class ScmTestMock implements StorageContainerDatanodeProtocol {
private AtomicInteger containerReportsCount = new AtomicInteger(0);
// Map of datanode to containers
private Map<DatanodeID, Map<String, ContainerInfo>> nodeContainers =
private Map<DatanodeDetails, Map<String, ContainerInfo>> nodeContainers =
new HashMap();
/**
* Returns the number of heartbeats made to this class.
@ -161,15 +162,16 @@ private void sleepIfNeeded() {
/**
* Used by data node to send a Heartbeat.
*
* @param datanodeID - Datanode ID.
* @param datanodeDetailsProto - DatanodeDetailsProto.
* @param nodeReport - node report.
* @return - SCMHeartbeatResponseProto
* @throws IOException
*/
@Override
public StorageContainerDatanodeProtocolProtos.SCMHeartbeatResponseProto
sendHeartbeat(DatanodeID datanodeID, SCMNodeReport nodeReport,
ReportState scmReportState) throws IOException {
sendHeartbeat(DatanodeDetailsProto datanodeDetailsProto,
SCMNodeReport nodeReport, ReportState scmReportState)
throws IOException {
rpcCount.incrementAndGet();
heartbeatCount.incrementAndGet();
this.reportState = scmReportState;
@ -183,21 +185,22 @@ private void sleepIfNeeded() {
/**
* Register Datanode.
*
* @param datanodeID - DatanodID.
* @param datanodeDetailsProto DatanodDetailsProto.
* @param scmAddresses - List of SCMs this datanode is configured to
* communicate.
* @return SCM Command.
*/
@Override
public StorageContainerDatanodeProtocolProtos
.SCMRegisteredCmdResponseProto register(DatanodeID datanodeID,
String[] scmAddresses) throws IOException {
.SCMRegisteredCmdResponseProto register(
DatanodeDetailsProto datanodeDetailsProto, String[] scmAddresses)
throws IOException {
rpcCount.incrementAndGet();
sleepIfNeeded();
return StorageContainerDatanodeProtocolProtos
.SCMRegisteredCmdResponseProto
.newBuilder().setClusterID(UUID.randomUUID().toString())
.setDatanodeUUID(datanodeID.getDatanodeUuid()).setErrorCode(
.setDatanodeUUID(datanodeDetailsProto.getUuid()).setErrorCode(
StorageContainerDatanodeProtocolProtos
.SCMRegisteredCmdResponseProto.ErrorCode.success).build();
}
@ -216,7 +219,8 @@ private void sleepIfNeeded() {
Preconditions.checkNotNull(reports);
containerReportsCount.incrementAndGet();
DatanodeID datanode = DatanodeID.getFromProtoBuf(reports.getDatanodeID());
DatanodeDetails datanode = DatanodeDetails.getFromProtoBuf(
reports.getDatanodeDetails());
if (reports.getReportsCount() > 0) {
Map containers = nodeContainers.get(datanode);
if (containers == null) {

View File

@ -19,8 +19,7 @@
import com.google.common.collect.Maps;
import com.google.common.util.concurrent.ThreadFactoryBuilder;
import org.apache.hadoop.conf.Configuration;
import org.apache.hadoop.hdfs.DFSTestUtil;
import org.apache.hadoop.hdfs.protocol.DatanodeID;
import org.apache.hadoop.hdsl.protocol.DatanodeDetails;
import org.apache.hadoop.ipc.RPC;
import org.apache.hadoop.ozone.OzoneConfigKeys;
import org.apache.hadoop.ozone.container.common.helpers.ContainerUtils;
@ -48,6 +47,7 @@
import java.util.LinkedList;
import java.util.List;
import java.util.Map;
import java.util.UUID;
import java.util.concurrent.ExecutionException;
import java.util.concurrent.ExecutorService;
import java.util.concurrent.TimeUnit;
@ -154,7 +154,7 @@ public void tearDown() throws Exception {
public void testStartStopDatanodeStateMachine() throws IOException,
InterruptedException, TimeoutException {
try (DatanodeStateMachine stateMachine =
new DatanodeStateMachine(DFSTestUtil.getLocalDatanodeID(), conf)) {
new DatanodeStateMachine(getNewDatanodeDetails(), conf)) {
stateMachine.startDaemon();
SCMConnectionManager connectionManager =
stateMachine.getConnectionManager();
@ -204,12 +204,13 @@ public void testDatanodeStateContext() throws IOException,
File idPath = new File(
conf.get(ScmConfigKeys.OZONE_SCM_DATANODE_ID));
idPath.delete();
DatanodeID dnID = DFSTestUtil.getLocalDatanodeID();
dnID.setContainerPort(OzoneConfigKeys.DFS_CONTAINER_IPC_PORT_DEFAULT);
ContainerUtils.writeDatanodeIDTo(dnID, idPath);
DatanodeDetails datanodeDetails = getNewDatanodeDetails();
datanodeDetails.setContainerPort(
OzoneConfigKeys.DFS_CONTAINER_IPC_PORT_DEFAULT);
ContainerUtils.writeDatanodeDetailsTo(datanodeDetails, idPath);
try (DatanodeStateMachine stateMachine = new DatanodeStateMachine(
DFSTestUtil.getLocalDatanodeID(), conf)) {
try (DatanodeStateMachine stateMachine =
new DatanodeStateMachine(datanodeDetails, conf)) {
DatanodeStateMachine.DatanodeStates currentState =
stateMachine.getContext().getState();
Assert.assertEquals(DatanodeStateMachine.DatanodeStates.INIT,
@ -341,7 +342,7 @@ public void testDatanodeStateMachineWithInvalidConfiguration()
perTestConf.setStrings(entry.getKey(), entry.getValue());
LOG.info("Test with {} = {}", entry.getKey(), entry.getValue());
try (DatanodeStateMachine stateMachine = new DatanodeStateMachine(
DFSTestUtil.getLocalDatanodeID(), perTestConf)) {
getNewDatanodeDetails(), perTestConf)) {
DatanodeStateMachine.DatanodeStates currentState =
stateMachine.getContext().getState();
Assert.assertEquals(DatanodeStateMachine.DatanodeStates.INIT,
@ -358,4 +359,17 @@ public void testDatanodeStateMachineWithInvalidConfiguration()
}
});
}
private DatanodeDetails getNewDatanodeDetails() {
return DatanodeDetails.newBuilder()
.setUuid(UUID.randomUUID().toString())
.setHostName("localhost")
.setIpAddress("127.0.0.1")
.setInfoPort(0)
.setInfoSecurePort(0)
.setContainerPort(0)
.setRatisPort(0)
.setOzoneRestPort(0)
.build();
}
}

View File

@ -28,8 +28,8 @@
import org.apache.hadoop.classification.InterfaceAudience;
import org.apache.hadoop.hdsl.conf.OzoneConfiguration;
import org.apache.hadoop.hdfs.DFSUtil;
import org.apache.hadoop.hdfs.protocol.DatanodeID;
import org.apache.hadoop.hdsl.HdslUtils;
import org.apache.hadoop.hdsl.protocol.DatanodeDetails;
import org.apache.hadoop.io.IOUtils;
import org.apache.hadoop.ipc.ProtobufRpcEngine;
import org.apache.hadoop.ipc.RPC;
@ -48,6 +48,7 @@
import org.apache.hadoop.ozone.protocol.commands.RegisteredCommand;
import org.apache.hadoop.ozone.protocol.commands.SCMCommand;
import org.apache.hadoop.hdsl.protocol.proto.HdslProtos;
import org.apache.hadoop.hdsl.protocol.proto.HdslProtos.DatanodeDetailsProto;
import org.apache.hadoop.hdsl.protocol.proto.HdslProtos.NodeState;
import org.apache.hadoop.hdsl.protocol.proto.ScmBlockLocationProtocolProtos;
import org.apache.hadoop.hdsl.protocol.proto.StorageContainerDatanodeProtocolProtos;
@ -665,11 +666,11 @@ public HdslProtos.NodePool queryNode(EnumSet<NodeState> nodeStatuses,
throw new IllegalArgumentException("Not Supported yet");
}
List<DatanodeID> datanodes = queryNode(nodeStatuses);
List<DatanodeDetails> datanodes = queryNode(nodeStatuses);
HdslProtos.NodePool.Builder poolBuilder =
HdslProtos.NodePool.newBuilder();
for (DatanodeID datanode : datanodes) {
for (DatanodeDetails datanode : datanodes) {
HdslProtos.Node node = HdslProtos.Node.newBuilder()
.setNodeID(datanode.getProtoBufMessage())
.addAllNodeStates(nodeStatuses)
@ -746,15 +747,15 @@ public Pipeline createReplicationPipeline(
* @return List of Datanodes.
*/
public List<DatanodeID> queryNode(EnumSet<NodeState> nodeStatuses) {
public List<DatanodeDetails> queryNode(EnumSet<NodeState> nodeStatuses) {
Preconditions.checkNotNull(nodeStatuses, "Node Query set cannot be null");
Preconditions.checkState(nodeStatuses.size() > 0, "No valid arguments " +
"in the query set");
List<DatanodeID> resultList = new LinkedList<>();
Set<DatanodeID> currentSet = new TreeSet<>();
List<DatanodeDetails> resultList = new LinkedList<>();
Set<DatanodeDetails> currentSet = new TreeSet<>();
for (NodeState nodeState : nodeStatuses) {
Set<DatanodeID> nextSet = queryNodeState(nodeState);
Set<DatanodeDetails> nextSet = queryNodeState(nodeState);
if ((nextSet == null) || (nextSet.size() == 0)) {
// Right now we only support AND operation. So intersect with
// any empty set is null.
@ -779,13 +780,13 @@ public List<DatanodeID> queryNode(EnumSet<NodeState> nodeStatuses) {
* @param nodeState - NodeState that we are interested in matching.
* @return Set of Datanodes that match the NodeState.
*/
private Set<DatanodeID> queryNodeState(NodeState nodeState) {
private Set<DatanodeDetails> queryNodeState(NodeState nodeState) {
if (nodeState == NodeState.RAFT_MEMBER ||
nodeState == NodeState.FREE_NODE) {
throw new IllegalStateException("Not implemented yet");
}
Set<DatanodeID> returnSet = new TreeSet<>();
List<DatanodeID> tmp = getScmNodeManager().getNodes(nodeState);
Set<DatanodeDetails> returnSet = new TreeSet<>();
List<DatanodeDetails> tmp = getScmNodeManager().getNodes(nodeState);
if ((tmp != null) && (tmp.size() > 0)) {
returnSet.addAll(tmp);
}
@ -945,20 +946,22 @@ public SCMVersionResponseProto getVersion(
/**
* Used by data node to send a Heartbeat.
*
* @param datanodeID - Datanode ID.
* @param datanodeDetails - Datanode Details.
* @param nodeReport - Node Report
* @param reportState - Container report ready info.
* @return - SCMHeartbeatResponseProto
* @throws IOException
*/
@Override
public SCMHeartbeatResponseProto sendHeartbeat(DatanodeID datanodeID,
SCMNodeReport nodeReport, ReportState reportState) throws IOException {
public SCMHeartbeatResponseProto sendHeartbeat(
DatanodeDetailsProto datanodeDetails, SCMNodeReport nodeReport,
ReportState reportState) throws IOException {
List<SCMCommand> commands =
getScmNodeManager().sendHeartbeat(datanodeID, nodeReport, reportState);
getScmNodeManager().sendHeartbeat(datanodeDetails, nodeReport,
reportState);
List<SCMCommandResponseProto> cmdResponses = new LinkedList<>();
for (SCMCommand cmd : commands) {
cmdResponses.add(getCommandResponse(cmd, datanodeID.getDatanodeUuid()
cmdResponses.add(getCommandResponse(cmd, datanodeDetails.getUuid()
.toString()));
}
return SCMHeartbeatResponseProto.newBuilder().addAllCommands(cmdResponses)
@ -968,17 +971,17 @@ public SCMHeartbeatResponseProto sendHeartbeat(DatanodeID datanodeID,
/**
* Register Datanode.
*
* @param datanodeID - DatanodID.
* @param datanodeDetails - DatanodID.
* @param scmAddresses - List of SCMs this datanode is configured to
* communicate.
* @return SCM Command.
*/
@Override
public StorageContainerDatanodeProtocolProtos.SCMRegisteredCmdResponseProto
register(DatanodeID datanodeID, String[] scmAddresses)
throws IOException {
register(DatanodeDetailsProto datanodeDetails, String[] scmAddresses) {
// TODO : Return the list of Nodes that forms the SCM HA.
return getRegisteredResponse(scmNodeManager.register(datanodeID), null);
return getRegisteredResponse(
scmNodeManager.register(datanodeDetails), null);
}
/**
@ -1020,7 +1023,7 @@ private void updateContainerReportMetrics(
// Update container stat entry, this will trigger a removal operation if it
// exists in cache.
synchronized (containerReportCache) {
String datanodeUuid = reports.getDatanodeID().getDatanodeUuid();
String datanodeUuid = reports.getDatanodeDetails().getUuid();
if (datanodeUuid != null && newStat != null) {
containerReportCache.put(datanodeUuid, newStat);
// update global view container metrics

View File

@ -20,9 +20,10 @@
import java.util.Collections;
import java.util.List;
import java.util.Set;
import java.util.UUID;
import java.util.stream.Collectors;
import org.apache.hadoop.hdfs.protocol.DatanodeID;
import org.apache.hadoop.hdsl.protocol.DatanodeDetails;
import org.apache.hadoop.hdsl.protocol.proto.StorageContainerDatanodeProtocolProtos.DeletedBlocksTransaction;
import org.apache.hadoop.ozone.scm.container.Mapping;
import org.apache.hadoop.scm.container.common.helpers.ContainerInfo;
@ -41,7 +42,7 @@ public class DatanodeDeletedBlockTransactions {
private int currentTXNum;
private Mapping mappingService;
// A list of TXs mapped to a certain datanode ID.
private final ArrayListMultimap<DatanodeID, DeletedBlocksTransaction>
private final ArrayListMultimap<UUID, DeletedBlocksTransaction>
transactions;
DatanodeDeletedBlockTransactions(Mapping mappingService,
@ -67,7 +68,8 @@ public void addTransaction(DeletedBlocksTransaction tx) throws IOException {
return;
}
for (DatanodeID dnID : info.getPipeline().getMachines()) {
for (DatanodeDetails dd : info.getPipeline().getMachines()) {
UUID dnID = dd.getUuid();
if (transactions.containsKey(dnID)) {
List<DeletedBlocksTransaction> txs = transactions.get(dnID);
if (txs != null && txs.size() < maximumAllowedTXNum) {
@ -93,7 +95,7 @@ public void addTransaction(DeletedBlocksTransaction tx) throws IOException {
}
}
Set<DatanodeID> getDatanodes() {
Set<UUID> getDatanodeIDs() {
return transactions.keySet();
}
@ -101,18 +103,18 @@ boolean isEmpty() {
return transactions.isEmpty();
}
boolean hasTransactions(DatanodeID dnID) {
return transactions.containsKey(dnID) && !transactions.get(dnID).isEmpty();
boolean hasTransactions(UUID dnId) {
return transactions.containsKey(dnId) &&
!transactions.get(dnId).isEmpty();
}
List<DeletedBlocksTransaction> getDatanodeTransactions(
DatanodeID dnID) {
return transactions.get(dnID);
List<DeletedBlocksTransaction> getDatanodeTransactions(UUID dnId) {
return transactions.get(dnId);
}
List<String> getTransactionIDList(DatanodeID dnID) {
if (hasTransactions(dnID)) {
return transactions.get(dnID).stream()
List<String> getTransactionIDList(UUID dnId) {
if (hasTransactions(dnId)) {
return transactions.get(dnId).stream()
.map(DeletedBlocksTransaction::getTxID).map(String::valueOf)
.collect(Collectors.toList());
} else {

View File

@ -20,7 +20,7 @@
import com.google.common.base.Preconditions;
import org.apache.hadoop.conf.Configuration;
import org.apache.hadoop.hdfs.protocol.DatanodeID;
import org.apache.hadoop.hdsl.protocol.DatanodeDetails;
import org.apache.hadoop.ozone.protocol.commands.DeleteBlocksCommand;
import org.apache.hadoop.hdsl.protocol.proto.HdslProtos.NodeState;
import org.apache.hadoop.hdsl.protocol.proto.StorageContainerDatanodeProtocolProtos.DeletedBlocksTransaction;
@ -39,6 +39,7 @@
import java.io.IOException;
import java.util.List;
import java.util.UUID;
import java.util.concurrent.TimeUnit;
/**
@ -114,7 +115,7 @@ public EmptyTaskResult call() throws Exception {
// to delete blocks.
LOG.debug("Running DeletedBlockTransactionScanner");
DatanodeDeletedBlockTransactions transactions = null;
List<DatanodeID> datanodes = nodeManager.getNodes(NodeState.HEALTHY);
List<DatanodeDetails> datanodes = nodeManager.getNodes(NodeState.HEALTHY);
if (datanodes != null) {
transactions = new DatanodeDeletedBlockTransactions(mappingService,
blockDeleteLimitSize, datanodes.size());
@ -133,22 +134,22 @@ public EmptyTaskResult call() throws Exception {
}
if (transactions != null && !transactions.isEmpty()) {
for (DatanodeID datanodeID : transactions.getDatanodes()) {
for (UUID dnId : transactions.getDatanodeIDs()) {
List<DeletedBlocksTransaction> dnTXs = transactions
.getDatanodeTransactions(datanodeID);
.getDatanodeTransactions(dnId);
if (dnTXs != null && !dnTXs.isEmpty()) {
dnTxCount += dnTXs.size();
// TODO commandQueue needs a cap.
// We should stop caching new commands if num of un-processed
// command is bigger than a limit, e.g 50. In case datanode goes
// offline for sometime, the cached commands be flooded.
nodeManager.addDatanodeCommand(datanodeID,
nodeManager.addDatanodeCommand(dnId,
new DeleteBlocksCommand(dnTXs));
LOG.debug(
"Added delete block command for datanode {} in the queue,"
+ " number of delete block transactions: {}, TxID list: {}",
datanodeID, dnTXs.size(), String.join(",",
transactions.getTransactionIDList(datanodeID)));
dnId, dnTXs.size(), String.join(",",
transactions.getTransactionIDList(dnId)));
}
}
}
@ -157,7 +158,7 @@ public EmptyTaskResult call() throws Exception {
LOG.info(
"Totally added {} delete blocks command for"
+ " {} datanodes, task elapsed time: {}ms",
dnTxCount, transactions.getDatanodes().size(),
dnTxCount, transactions.getDatanodeIDs().size(),
Time.monotonicNow() - startTime);
}

View File

@ -415,7 +415,7 @@ public void processContainerReports(ContainerReportsRequestProto reports)
// Container not found in our container db.
LOG.error("Error while processing container report from datanode :" +
" {}, for container: {}, reason: container doesn't exist in" +
"container database.", reports.getDatanodeID(),
"container database.", reports.getDatanodeDetails(),
datanodeState.getContainerName());
}
} finally {

View File

@ -22,8 +22,7 @@
import com.google.common.base.Preconditions;
import com.google.common.util.concurrent.ThreadFactoryBuilder;
import org.apache.hadoop.conf.Configuration;
import org.apache.hadoop.hdfs.protocol.DatanodeID;
import org.apache.hadoop.hdfs.protocol.proto.HdfsProtos;
import org.apache.hadoop.hdsl.protocol.DatanodeDetails;
import org.apache.hadoop.ozone.protocol.commands.CloseContainerCommand;
import org.apache.hadoop.hdsl.protocol.proto.HdslProtos;
import org.apache.hadoop.ozone.scm.node.NodeManager;
@ -128,9 +127,10 @@ public void close(HdslProtos.SCMContainerInfo info) {
// to SCM. In that case also, data node will ignore this command.
HdslProtos.Pipeline pipeline = info.getPipeline();
for (HdfsProtos.DatanodeIDProto datanodeID :
for (HdslProtos.DatanodeDetailsProto datanodeDetails :
pipeline.getPipelineChannel().getMembersList()) {
nodeManager.addDatanodeCommand(DatanodeID.getFromProtoBuf(datanodeID),
nodeManager.addDatanodeCommand(
DatanodeDetails.getFromProtoBuf(datanodeDetails).getUuid(),
new CloseContainerCommand(info.getContainerName()));
}
if (!commandIssued.containsKey(info.getContainerName())) {

View File

@ -17,7 +17,7 @@
package org.apache.hadoop.ozone.scm.container.placement.algorithms;
import org.apache.hadoop.hdfs.protocol.DatanodeID;
import org.apache.hadoop.hdsl.protocol.DatanodeDetails;
import java.io.IOException;
import java.util.List;
@ -36,6 +36,6 @@ public interface ContainerPlacementPolicy {
* @return list of datanodes chosen.
* @throws IOException
*/
List<DatanodeID> chooseDatanodes(int nodesRequired, long sizeRequired)
List<DatanodeDetails> chooseDatanodes(int nodesRequired, long sizeRequired)
throws IOException;
}

View File

@ -19,7 +19,7 @@
import com.google.common.annotations.VisibleForTesting;
import org.apache.hadoop.conf.Configuration;
import org.apache.hadoop.hdfs.protocol.DatanodeID;
import org.apache.hadoop.hdsl.protocol.DatanodeDetails;
import org.apache.hadoop.hdsl.protocol.proto.HdslProtos;
import org.apache.hadoop.ozone.scm.container.placement.metrics.SCMNodeMetric;
import org.apache.hadoop.ozone.scm.exceptions.SCMException;
@ -101,9 +101,9 @@ public Configuration getConf() {
* @throws SCMException SCM exception.
*/
public List<DatanodeID> chooseDatanodes(int nodesRequired, final long
public List<DatanodeDetails> chooseDatanodes(int nodesRequired, final long
sizeRequired) throws SCMException {
List<DatanodeID> healthyNodes =
List<DatanodeDetails> healthyNodes =
nodeManager.getNodes(HdslProtos.NodeState.HEALTHY);
String msg;
if (healthyNodes.size() == 0) {
@ -121,7 +121,7 @@ public List<DatanodeID> chooseDatanodes(int nodesRequired, final long
throw new SCMException(msg,
SCMException.ResultCodes.FAILED_TO_FIND_SUITABLE_NODE);
}
List<DatanodeID> healthyList = healthyNodes.stream().filter(d ->
List<DatanodeDetails> healthyList = healthyNodes.stream().filter(d ->
hasEnoughSpace(d, sizeRequired)).collect(Collectors.toList());
if (healthyList.size() < nodesRequired) {
@ -140,11 +140,12 @@ public List<DatanodeID> chooseDatanodes(int nodesRequired, final long
/**
* Returns true if this node has enough space to meet our requirement.
*
* @param datanodeID DatanodeID
* @param datanodeDetails DatanodeDetails
* @return true if we have enough space.
*/
private boolean hasEnoughSpace(DatanodeID datanodeID, long sizeRequired) {
SCMNodeMetric nodeMetric = nodeManager.getNodeStat(datanodeID);
private boolean hasEnoughSpace(DatanodeDetails datanodeDetails,
long sizeRequired) {
SCMNodeMetric nodeMetric = nodeManager.getNodeStat(datanodeDetails);
return (nodeMetric != null) && nodeMetric.get().getRemaining()
.hasResources(sizeRequired);
}
@ -159,12 +160,13 @@ private boolean hasEnoughSpace(DatanodeID datanodeID, long sizeRequired) {
* @return List of Datanodes that can be used for placement.
* @throws SCMException
*/
public List<DatanodeID> getResultSet(int nodesRequired, List<DatanodeID>
healthyNodes) throws SCMException {
List<DatanodeID> results = new LinkedList<>();
public List<DatanodeDetails> getResultSet(
int nodesRequired, List<DatanodeDetails> healthyNodes)
throws SCMException {
List<DatanodeDetails> results = new LinkedList<>();
for (int x = 0; x < nodesRequired; x++) {
// invoke the choose function defined in the derived classes.
DatanodeID nodeId = chooseNode(healthyNodes);
DatanodeDetails nodeId = chooseNode(healthyNodes);
if (nodeId != null) {
results.add(nodeId);
}
@ -186,9 +188,10 @@ public List<DatanodeID> getResultSet(int nodesRequired, List<DatanodeID>
* PlacementRandom.
*
* @param healthyNodes - Set of healthy nodes we can choose from.
* @return DatanodeID
* @return DatanodeDetails
*/
public abstract DatanodeID chooseNode(List<DatanodeID> healthyNodes);
public abstract DatanodeDetails chooseNode(
List<DatanodeDetails> healthyNodes);
}

View File

@ -19,7 +19,7 @@
import com.google.common.annotations.VisibleForTesting;
import org.apache.hadoop.conf.Configuration;
import org.apache.hadoop.hdfs.protocol.DatanodeID;
import org.apache.hadoop.hdsl.protocol.DatanodeDetails;
import org.apache.hadoop.ozone.scm.container.placement.metrics.SCMNodeMetric;
import org.apache.hadoop.ozone.scm.exceptions.SCMException;
import org.apache.hadoop.ozone.scm.node.NodeManager;
@ -89,9 +89,9 @@ public SCMContainerPlacementCapacity(final NodeManager nodeManager,
* @throws SCMException SCMException
*/
@Override
public List<DatanodeID> chooseDatanodes(final int nodesRequired,
final long sizeRequired) throws SCMException {
List<DatanodeID> healthyNodes =
public List<DatanodeDetails> chooseDatanodes(
final int nodesRequired, final long sizeRequired) throws SCMException {
List<DatanodeDetails> healthyNodes =
super.chooseDatanodes(nodesRequired, sizeRequired);
if (healthyNodes.size() == nodesRequired) {
return healthyNodes;
@ -105,29 +105,29 @@ public List<DatanodeID> chooseDatanodes(final int nodesRequired,
*
* @param healthyNodes - List of healthy nodes that meet the size
* requirement.
* @return DatanodeID that is chosen.
* @return DatanodeDetails that is chosen.
*/
@Override
public DatanodeID chooseNode(List<DatanodeID> healthyNodes) {
public DatanodeDetails chooseNode(List<DatanodeDetails> healthyNodes) {
int firstNodeNdx = getRand().nextInt(healthyNodes.size());
int secondNodeNdx = getRand().nextInt(healthyNodes.size());
DatanodeID chosenID;
DatanodeDetails datanodeDetails;
// There is a possibility that both numbers will be same.
// if that is so, we just return the node.
if (firstNodeNdx == secondNodeNdx) {
chosenID = healthyNodes.get(firstNodeNdx);
datanodeDetails = healthyNodes.get(firstNodeNdx);
} else {
DatanodeID firstNodeID = healthyNodes.get(firstNodeNdx);
DatanodeID secondNodeID = healthyNodes.get(secondNodeNdx);
DatanodeDetails firstNodeDetails = healthyNodes.get(firstNodeNdx);
DatanodeDetails secondNodeDetails = healthyNodes.get(secondNodeNdx);
SCMNodeMetric firstNodeMetric =
getNodeManager().getNodeStat(firstNodeID);
getNodeManager().getNodeStat(firstNodeDetails);
SCMNodeMetric secondNodeMetric =
getNodeManager().getNodeStat(secondNodeID);
chosenID = firstNodeMetric.isGreater(secondNodeMetric.get())
? firstNodeID : secondNodeID;
getNodeManager().getNodeStat(secondNodeDetails);
datanodeDetails = firstNodeMetric.isGreater(secondNodeMetric.get())
? firstNodeDetails : secondNodeDetails;
}
healthyNodes.remove(chosenID);
return chosenID;
healthyNodes.remove(datanodeDetails);
return datanodeDetails;
}
}

View File

@ -19,7 +19,7 @@
import com.google.common.annotations.VisibleForTesting;
import org.apache.hadoop.conf.Configuration;
import org.apache.hadoop.hdfs.protocol.DatanodeID;
import org.apache.hadoop.hdsl.protocol.DatanodeDetails;
import org.apache.hadoop.ozone.scm.exceptions.SCMException;
import org.apache.hadoop.ozone.scm.node.NodeManager;
import org.slf4j.Logger;
@ -62,9 +62,9 @@ public SCMContainerPlacementRandom(final NodeManager nodeManager,
* @throws SCMException SCMException
*/
@Override
public List<DatanodeID> chooseDatanodes(final int nodesRequired,
final long sizeRequired) throws SCMException {
List<DatanodeID> healthyNodes =
public List<DatanodeDetails> chooseDatanodes(
final int nodesRequired, final long sizeRequired) throws SCMException {
List<DatanodeDetails> healthyNodes =
super.chooseDatanodes(nodesRequired, sizeRequired);
if (healthyNodes.size() == nodesRequired) {
@ -80,8 +80,8 @@ public List<DatanodeID> chooseDatanodes(final int nodesRequired,
* @param healthyNodes - all healthy datanodes.
* @return one randomly chosen datanode that from two randomly chosen datanode
*/
public DatanodeID chooseNode(final List<DatanodeID> healthyNodes) {
DatanodeID selectedNode =
public DatanodeDetails chooseNode(final List<DatanodeDetails> healthyNodes) {
DatanodeDetails selectedNode =
healthyNodes.get(getRand().nextInt(healthyNodes.size()));
healthyNodes.remove(selectedNode);
return selectedNode;

View File

@ -20,7 +20,7 @@
import com.google.common.base.Preconditions;
import com.google.common.util.concurrent.ThreadFactoryBuilder;
import org.apache.hadoop.conf.Configuration;
import org.apache.hadoop.hdfs.protocol.DatanodeID;
import org.apache.hadoop.hdsl.protocol.DatanodeDetails;
import org.apache.hadoop.hdsl.protocol.proto
.StorageContainerDatanodeProtocolProtos.ContainerReportsRequestProto;
import org.apache.hadoop.ozone.scm.exceptions.SCMException;
@ -288,11 +288,11 @@ private void initPoolProcessThread() {
*/
public void handleContainerReport(
ContainerReportsRequestProto containerReport) {
DatanodeID datanodeID = DatanodeID.getFromProtoBuf(
containerReport.getDatanodeID());
DatanodeDetails datanodeDetails = DatanodeDetails.getFromProtoBuf(
containerReport.getDatanodeDetails());
inProgressPoolListLock.readLock().lock();
try {
String poolName = poolManager.getNodePool(datanodeID);
String poolName = poolManager.getNodePool(datanodeDetails);
for (InProgressPool ppool : inProgressPoolList) {
if (ppool.getPoolName().equalsIgnoreCase(poolName)) {
ppool.handleContainerReport(containerReport);
@ -302,11 +302,12 @@ public void handleContainerReport(
// TODO: Decide if we can do anything else with this report.
LOG.debug("Discarding the container report for pool {}. " +
"That pool is not currently in the pool reconciliation process." +
" Container Name: {}", poolName, containerReport.getDatanodeID());
" Container Name: {}", poolName,
containerReport.getDatanodeDetails());
} catch (SCMException e) {
LOG.warn("Skipping processing container report from datanode {}, "
+ "cause: failed to get the corresponding node pool",
datanodeID.toString(), e);
datanodeDetails.toString(), e);
} finally {
inProgressPoolListLock.readLock().unlock();
}

View File

@ -18,7 +18,7 @@
import com.google.common.annotations.VisibleForTesting;
import com.google.common.base.Preconditions;
import org.apache.hadoop.hdfs.protocol.DatanodeID;
import org.apache.hadoop.hdsl.protocol.DatanodeDetails;
import org.apache.hadoop.ozone.protocol.commands.SendContainerCommand;
import org.apache.hadoop.hdsl.protocol.proto.HdslProtos.NodeState;
import org.apache.hadoop.hdsl.protocol.proto
@ -33,6 +33,7 @@
import java.util.List;
import java.util.Map;
import java.util.UUID;
import java.util.concurrent.ConcurrentHashMap;
import java.util.concurrent.ExecutorService;
import java.util.concurrent.TimeUnit;
@ -62,7 +63,7 @@ public final class InProgressPool {
private final NodePoolManager poolManager;
private final ExecutorService executorService;
private final Map<String, Integer> containerCountMap;
private final Map<String, Boolean> processedNodeSet;
private final Map<UUID, Boolean> processedNodeSet;
private final long startTime;
private ProgressStatus status;
private AtomicInteger nodeCount;
@ -165,9 +166,9 @@ public ProgressStatus getStatus() {
* Starts the reconciliation process for all the nodes in the pool.
*/
public void startReconciliation() {
List<DatanodeID> datanodeIDList =
List<DatanodeDetails> datanodeDetailsList =
this.poolManager.getNodes(pool.getPoolName());
if (datanodeIDList.size() == 0) {
if (datanodeDetailsList.size() == 0) {
LOG.error("Datanode list for {} is Empty. Pool with no nodes ? ",
pool.getPoolName());
this.status = ProgressStatus.Error;
@ -181,14 +182,14 @@ public void startReconciliation() {
Ask each datanode to send us commands.
*/
SendContainerCommand cmd = SendContainerCommand.newBuilder().build();
for (DatanodeID id : datanodeIDList) {
NodeState currentState = getNodestate(id);
for (DatanodeDetails dd : datanodeDetailsList) {
NodeState currentState = getNodestate(dd);
if (currentState == HEALTHY || currentState == STALE) {
nodeCount.incrementAndGet();
// Queue commands to all datanodes in this pool to send us container
// report. Since we ignore dead nodes, it is possible that we would have
// over replicated the container if the node comes back.
nodeManager.addDatanodeCommand(id, cmd);
nodeManager.addDatanodeCommand(dd.getUuid(), cmd);
}
}
this.status = ProgressStatus.InProgress;
@ -198,10 +199,10 @@ public void startReconciliation() {
/**
* Gets the node state.
*
* @param id - datanode ID.
* @param datanode - datanode information.
* @return NodeState.
*/
private NodeState getNodestate(DatanodeID id) {
private NodeState getNodestate(DatanodeDetails datanode) {
NodeState currentState = INVALID;
int maxTry = 100;
// We need to loop to make sure that we will retry if we get
@ -212,7 +213,7 @@ private NodeState getNodestate(DatanodeID id) {
while (currentState == INVALID && currentTry < maxTry) {
// Retry to make sure that we deal with the case of node state not
// known.
currentState = nodeManager.getNodeState(id);
currentState = nodeManager.getNodeState(datanode);
currentTry++;
if (currentState == INVALID) {
// Sleep to make sure that this is not a tight loop.
@ -222,7 +223,7 @@ private NodeState getNodestate(DatanodeID id) {
if (currentState == INVALID) {
LOG.error("Not able to determine the state of Node: {}, Exceeded max " +
"try and node manager returns INVALID state. This indicates we " +
"are dealing with a node that we don't know about.", id);
"are dealing with a node that we don't know about.", datanode);
}
return currentState;
}
@ -248,13 +249,13 @@ public void handleContainerReport(
private Runnable processContainerReport(
ContainerReportsRequestProto reports) {
return () -> {
DatanodeID datanodeID =
DatanodeID.getFromProtoBuf(reports.getDatanodeID());
if (processedNodeSet.computeIfAbsent(datanodeID.getDatanodeUuid(),
DatanodeDetails datanodeDetails =
DatanodeDetails.getFromProtoBuf(reports.getDatanodeDetails());
if (processedNodeSet.computeIfAbsent(datanodeDetails.getUuid(),
(k) -> true)) {
nodeProcessed.incrementAndGet();
LOG.debug("Total Nodes processed : {} Node Name: {} ", nodeProcessed,
datanodeID.getDatanodeUuid());
datanodeDetails.getUuid());
for (ContainerInfo info : reports.getReportsList()) {
containerProcessedCount.incrementAndGet();
LOG.debug("Total Containers processed: {} Container Name: {}",

View File

@ -19,7 +19,6 @@
import com.google.common.annotations.VisibleForTesting;
import com.google.common.base.Preconditions;
import org.apache.hadoop.hdfs.protocol.DatanodeID;
import org.apache.hadoop.ozone.protocol.commands.SCMCommand;
import org.apache.hadoop.util.Time;
@ -27,6 +26,7 @@
import java.util.LinkedList;
import java.util.List;
import java.util.Map;
import java.util.UUID;
import java.util.concurrent.locks.Lock;
import java.util.concurrent.locks.ReentrantLock;
@ -40,7 +40,7 @@
public class CommandQueue {
// This list is used as default return value.
private static final List<SCMCommand> DEFAULT_LIST = new LinkedList<>();
private final Map<DatanodeID, Commands> commandMap;
private final Map<UUID, Commands> commandMap;
private final Lock lock;
private long commandsInQueue;
@ -82,14 +82,14 @@ public void clear() {
* commands returns a empty list otherwise the current set of
* commands are returned and command map set to empty list again.
*
* @param datanodeID DatanodeID
* @param datanodeUuid Datanode UUID
* @return List of SCM Commands.
*/
@SuppressWarnings("unchecked")
List<SCMCommand> getCommand(final DatanodeID datanodeID) {
List<SCMCommand> getCommand(final UUID datanodeUuid) {
lock.lock();
try {
Commands cmds = commandMap.remove(datanodeID);
Commands cmds = commandMap.remove(datanodeUuid);
List<SCMCommand> cmdList = null;
if(cmds != null) {
cmdList = cmds.getCommands();
@ -106,17 +106,17 @@ List<SCMCommand> getCommand(final DatanodeID datanodeID) {
/**
* Adds a Command to the SCM Queue to send the command to container.
*
* @param datanodeID DatanodeID
* @param datanodeUuid DatanodeDetails.Uuid
* @param command - Command
*/
public void addCommand(final DatanodeID datanodeID, final SCMCommand
public void addCommand(final UUID datanodeUuid, final SCMCommand
command) {
lock.lock();
try {
if (commandMap.containsKey(datanodeID)) {
commandMap.get(datanodeID).add(command);
if (commandMap.containsKey(datanodeUuid)) {
commandMap.get(datanodeUuid).add(command);
} else {
commandMap.put(datanodeID, new Commands(command));
commandMap.put(datanodeUuid, new Commands(command));
}
commandsInQueue++;
} finally {

View File

@ -19,7 +19,7 @@
package org.apache.hadoop.ozone.scm.node;
import com.google.common.annotations.VisibleForTesting;
import org.apache.hadoop.hdfs.protocol.DatanodeID;
import org.apache.hadoop.hdsl.protocol.DatanodeDetails;
import org.apache.hadoop.hdsl.protocol.proto.StorageContainerDatanodeProtocolProtos.ReportState;
import org.apache.hadoop.hdsl.protocol.proto
.StorageContainerDatanodeProtocolProtos.SCMNodeReport;
@ -30,21 +30,21 @@
* This class represents the item in SCM heartbeat queue.
*/
public class HeartbeatQueueItem {
private DatanodeID datanodeID;
private DatanodeDetails datanodeDetails;
private long recvTimestamp;
private SCMNodeReport nodeReport;
private ReportState containerReportState;
/**
*
* @param datanodeID - datanode ID of the heartbeat.
* @param datanodeDetails - datanode ID of the heartbeat.
* @param recvTimestamp - heartbeat receive timestamp.
* @param nodeReport - node report associated with the heartbeat if any.
* @param containerReportState - container report state.
*/
HeartbeatQueueItem(DatanodeID datanodeID, long recvTimestamp,
HeartbeatQueueItem(DatanodeDetails datanodeDetails, long recvTimestamp,
SCMNodeReport nodeReport, ReportState containerReportState) {
this.datanodeID = datanodeID;
this.datanodeDetails = datanodeDetails;
this.recvTimestamp = recvTimestamp;
this.nodeReport = nodeReport;
this.containerReportState = containerReportState;
@ -53,8 +53,8 @@ public class HeartbeatQueueItem {
/**
* @return datanode ID.
*/
public DatanodeID getDatanodeID() {
return datanodeID;
public DatanodeDetails getDatanodeDetails() {
return datanodeDetails;
}
/**
@ -82,13 +82,13 @@ public long getRecvTimestamp() {
* Builder for HeartbeatQueueItem.
*/
public static class Builder {
private DatanodeID datanodeID;
private DatanodeDetails datanodeDetails;
private SCMNodeReport nodeReport;
private ReportState containerReportState;
private long recvTimestamp = monotonicNow();
public Builder setDatanodeID(DatanodeID datanodeId) {
this.datanodeID = datanodeId;
public Builder setDatanodeDetails(DatanodeDetails dnDetails) {
this.datanodeDetails = dnDetails;
return this;
}
@ -109,7 +109,7 @@ public Builder setRecvTimestamp(long recvTime) {
}
public HeartbeatQueueItem build() {
return new HeartbeatQueueItem(datanodeID, recvTimestamp, nodeReport,
return new HeartbeatQueueItem(datanodeDetails, recvTimestamp, nodeReport,
containerReportState);
}
}

View File

@ -18,8 +18,8 @@
package org.apache.hadoop.ozone.scm.node;
import com.google.common.annotations.VisibleForTesting;
import org.apache.hadoop.hdfs.protocol.DatanodeID;
import org.apache.hadoop.hdfs.protocol.UnregisteredNodeException;
import org.apache.hadoop.hdsl.protocol.DatanodeDetails;
import org.apache.hadoop.ozone.protocol.StorageContainerNodeProtocol;
import org.apache.hadoop.ozone.protocol.commands.SCMCommand;
import org.apache.hadoop.hdsl.protocol.proto.HdslProtos.NodeState;
@ -29,6 +29,7 @@
import java.io.Closeable;
import java.util.List;
import java.util.Map;
import java.util.UUID;
/**
* A node manager supports a simple interface for managing a datanode.
@ -60,14 +61,14 @@ public interface NodeManager extends StorageContainerNodeProtocol,
* @param node - DataNode.
* @throws UnregisteredNodeException
*/
void removeNode(DatanodeID node) throws UnregisteredNodeException;
void removeNode(DatanodeDetails node) throws UnregisteredNodeException;
/**
* Gets all Live Datanodes that is currently communicating with SCM.
* @param nodeState - State of the node
* @return List of Datanodes that are Heartbeating SCM.
*/
List<DatanodeID> getNodes(NodeState nodeState);
List<DatanodeDetails> getNodes(NodeState nodeState);
/**
* Returns the Number of Datanodes that are communicating with SCM.
@ -79,9 +80,9 @@ public interface NodeManager extends StorageContainerNodeProtocol,
/**
* Get all datanodes known to SCM.
*
* @return List of DatanodeIDs known to SCM.
* @return List of DatanodeDetails known to SCM.
*/
List<DatanodeID> getAllNodes();
List<DatanodeDetails> getAllNodes();
/**
* Chill mode is the period when node manager waits for a minimum
@ -113,14 +114,14 @@ public interface NodeManager extends StorageContainerNodeProtocol,
* Return a map of node stats.
* @return a map of individual node stats (live/stale but not dead).
*/
Map<String, SCMNodeStat> getNodeStats();
Map<UUID, SCMNodeStat> getNodeStats();
/**
* Return the node stat of the specified datanode.
* @param datanodeID - datanode ID.
* @param datanodeDetails DatanodeDetails.
* @return node stat if it is live/stale, null if it is dead or does't exist.
*/
SCMNodeMetric getNodeStat(DatanodeID datanodeID);
SCMNodeMetric getNodeStat(DatanodeDetails datanodeDetails);
/**
* Returns the NodePoolManager associated with the NodeManager.
@ -137,16 +138,16 @@ public interface NodeManager extends StorageContainerNodeProtocol,
/**
* Returns the node state of a specific node.
* @param id - DatanodeID
* @param datanodeDetails DatanodeDetails
* @return Healthy/Stale/Dead.
*/
NodeState getNodeState(DatanodeID id);
NodeState getNodeState(DatanodeDetails datanodeDetails);
/**
* Add a {@link SCMCommand} to the command queue, which are
* handled by HB thread asynchronously.
* @param id
* @param dnId datanode uuid
* @param command
*/
void addDatanodeCommand(DatanodeID id, SCMCommand command);
void addDatanodeCommand(UUID dnId, SCMCommand command);
}

View File

@ -19,7 +19,7 @@
package org.apache.hadoop.ozone.scm.node;
import org.apache.hadoop.hdfs.protocol.DatanodeID;
import org.apache.hadoop.hdsl.protocol.DatanodeDetails;
import org.apache.hadoop.ozone.scm.exceptions.SCMException;
import java.io.Closeable;
@ -36,7 +36,7 @@ public interface NodePoolManager extends Closeable {
* @param pool - name of the node pool.
* @param node - data node.
*/
void addNode(String pool, DatanodeID node) throws IOException;
void addNode(String pool, DatanodeDetails node) throws IOException;
/**
* Remove a node from a node pool.
@ -44,7 +44,7 @@ public interface NodePoolManager extends Closeable {
* @param node - data node.
* @throws SCMException
*/
void removeNode(String pool, DatanodeID node)
void removeNode(String pool, DatanodeDetails node)
throws SCMException;
/**
@ -60,13 +60,13 @@ void removeNode(String pool, DatanodeID node)
* @return a list of datanode ids or an empty list if the node pool was not
* found.
*/
List<DatanodeID> getNodes(String pool);
List<DatanodeDetails> getNodes(String pool);
/**
* Get the node pool name if the node has been added to a node pool.
* @param datanodeID - datanode ID.
* @param datanodeDetails - datanode ID.
* @return node pool name if it has been assigned.
* null if the node has not been assigned to any node pool yet.
*/
String getNodePool(DatanodeID datanodeID) throws SCMException;
String getNodePool(DatanodeDetails datanodeDetails) throws SCMException;
}

View File

@ -20,8 +20,10 @@
import com.google.common.annotations.VisibleForTesting;
import com.google.common.base.Preconditions;
import com.google.common.util.concurrent.ThreadFactoryBuilder;
import org.apache.hadoop.hdfs.protocol.DatanodeID;
import org.apache.hadoop.hdfs.protocol.UnregisteredNodeException;
import org.apache.hadoop.hdsl.protocol.DatanodeDetails;
import org.apache.hadoop.hdsl.protocol.proto.HdslProtos.DatanodeDetailsProto;
import org.apache.hadoop.ipc.Server;
import org.apache.hadoop.metrics2.util.MBeans;
import org.apache.hadoop.hdsl.conf.OzoneConfiguration;
import org.apache.hadoop.ozone.protocol.StorageContainerNodeProtocol;
@ -62,11 +64,13 @@
import javax.management.ObjectName;
import java.io.IOException;
import java.net.InetAddress;
import java.util.Collections;
import java.util.HashMap;
import java.util.List;
import java.util.Map;
import java.util.Queue;
import java.util.UUID;
import java.util.concurrent.ConcurrentHashMap;
import java.util.concurrent.ConcurrentLinkedQueue;
import java.util.concurrent.ScheduledExecutorService;
@ -115,13 +119,13 @@ public class SCMNodeManager
/**
* Key = NodeID, value = timestamp.
*/
private final ConcurrentHashMap<String, Long> healthyNodes;
private final ConcurrentHashMap<String, Long> staleNodes;
private final ConcurrentHashMap<String, Long> deadNodes;
private final ConcurrentHashMap<UUID, Long> healthyNodes;
private final ConcurrentHashMap<UUID, Long> staleNodes;
private final ConcurrentHashMap<UUID, Long> deadNodes;
private final Queue<HeartbeatQueueItem> heartbeatQueue;
private final ConcurrentHashMap<String, DatanodeID> nodes;
private final ConcurrentHashMap<UUID, DatanodeDetails> nodes;
// Individual live node stats
private final ConcurrentHashMap<String, SCMNodeStat> nodeStats;
private final ConcurrentHashMap<UUID, SCMNodeStat> nodeStats;
// Aggregated node stats
private SCMNodeStat scmStat;
// TODO: expose nodeStats and scmStat as metrics
@ -170,7 +174,7 @@ public SCMNodeManager(OzoneConfiguration conf, String clusterID,
deadNodes = new ConcurrentHashMap<>();
staleNodes = new ConcurrentHashMap<>();
nodes = new ConcurrentHashMap<>();
nodeStats = new ConcurrentHashMap();
nodeStats = new ConcurrentHashMap<>();
scmStat = new SCMNodeStat();
healthyNodeCount = new AtomicInteger(0);
@ -228,7 +232,7 @@ private void unregisterMXBean() {
* @throws UnregisteredNodeException
*/
@Override
public void removeNode(DatanodeID node) throws UnregisteredNodeException {
public void removeNode(DatanodeDetails node) {
// TODO : Fix me when adding the SCM CLI.
}
@ -242,9 +246,9 @@ public void removeNode(DatanodeID node) throws UnregisteredNodeException {
* @return List of Datanodes that are known to SCM in the requested state.
*/
@Override
public List<DatanodeID> getNodes(NodeState nodestate)
public List<DatanodeDetails> getNodes(NodeState nodestate)
throws IllegalArgumentException {
Map<String, Long> set;
Map<UUID, Long> set;
switch (nodestate) {
case HEALTHY:
synchronized (this) {
@ -272,11 +276,11 @@ public List<DatanodeID> getNodes(NodeState nodestate)
/**
* Returns all datanodes that are known to SCM.
*
* @return List of DatanodeIDs
* @return List of DatanodeDetails
*/
@Override
public List<DatanodeID> getAllNodes() {
Map<String, DatanodeID> set;
public List<DatanodeDetails> getAllNodes() {
Map<UUID, DatanodeDetails> set;
synchronized (this) {
set = Collections.unmodifiableMap(new HashMap<>(nodes));
}
@ -406,11 +410,11 @@ public boolean waitForHeartbeatProcessed() {
/**
* Returns the node state of a specific node.
*
* @param id - DatanodeID
* @param datanodeDetails - Datanode Details
* @return Healthy/Stale/Dead/Unknown.
*/
@Override
public NodeState getNodeState(DatanodeID id) {
public NodeState getNodeState(DatanodeDetails datanodeDetails) {
// There is a subtle race condition here, hence we also support
// the NODEState.UNKNOWN. It is possible that just before we check the
// healthyNodes, we have removed the node from the healthy list but stil
@ -419,15 +423,16 @@ public NodeState getNodeState(DatanodeID id) {
// then the node is in 2 states to avoid this race condition. Instead we
// just deal with the possibilty of getting a state called unknown.
if(healthyNodes.containsKey(id.getDatanodeUuid())) {
UUID id = datanodeDetails.getUuid();
if(healthyNodes.containsKey(id)) {
return HEALTHY;
}
if(staleNodes.containsKey(id.getDatanodeUuid())) {
if(staleNodes.containsKey(id)) {
return STALE;
}
if(deadNodes.containsKey(id.getDatanodeUuid())) {
if(deadNodes.containsKey(id)) {
return DEAD;
}
@ -477,7 +482,7 @@ public void run() {
// Iterate over the Stale nodes and decide if we need to move any node to
// dead State.
long currentTime = monotonicNow();
for (Map.Entry<String, Long> entry : staleNodes.entrySet()) {
for (Map.Entry<UUID, Long> entry : staleNodes.entrySet()) {
if (currentTime - entry.getValue() > deadNodeIntervalMs) {
synchronized (this) {
moveStaleNodeToDead(entry);
@ -488,7 +493,7 @@ public void run() {
// Iterate over the healthy nodes and decide if we need to move any node to
// Stale State.
currentTime = monotonicNow();
for (Map.Entry<String, Long> entry : healthyNodes.entrySet()) {
for (Map.Entry<UUID, Long> entry : healthyNodes.entrySet()) {
if (currentTime - entry.getValue() > staleNodeIntervalMs) {
synchronized (this) {
moveHealthyNodeToStale(entry);
@ -555,7 +560,7 @@ private void monitorHBProcessingTime() {
*
* @param entry - Map Entry
*/
private void moveHealthyNodeToStale(Map.Entry<String, Long> entry) {
private void moveHealthyNodeToStale(Map.Entry<UUID, Long> entry) {
LOG.trace("Moving healthy node to stale: {}", entry.getKey());
healthyNodes.remove(entry.getKey());
healthyNodeCount.decrementAndGet();
@ -564,7 +569,7 @@ private void moveHealthyNodeToStale(Map.Entry<String, Long> entry) {
if (scmManager != null) {
// remove stale node's container report
scmManager.removeContainerReport(entry.getKey());
scmManager.removeContainerReport(entry.getKey().toString());
}
}
@ -573,7 +578,7 @@ private void moveHealthyNodeToStale(Map.Entry<String, Long> entry) {
*
* @param entry - Map Entry
*/
private void moveStaleNodeToDead(Map.Entry<String, Long> entry) {
private void moveStaleNodeToDead(Map.Entry<UUID, Long> entry) {
LOG.trace("Moving stale node to dead: {}", entry.getKey());
staleNodes.remove(entry.getKey());
staleNodeCount.decrementAndGet();
@ -594,8 +599,8 @@ private void moveStaleNodeToDead(Map.Entry<String, Long> entry) {
private void handleHeartbeat(HeartbeatQueueItem hbItem) {
lastHBProcessedCount++;
DatanodeID datanodeID = hbItem.getDatanodeID();
String datanodeUuid = datanodeID.getDatanodeUuid();
DatanodeDetails datanodeDetails = hbItem.getDatanodeDetails();
UUID datanodeUuid = datanodeDetails.getUuid();
SCMNodeReport nodeReport = hbItem.getNodeReport();
long recvTimestamp = hbItem.getRecvTimestamp();
long processTimestamp = Time.monotonicNow();
@ -610,7 +615,7 @@ private void handleHeartbeat(HeartbeatQueueItem hbItem) {
if (healthyNodes.containsKey(datanodeUuid)) {
healthyNodes.put(datanodeUuid, processTimestamp);
updateNodeStat(datanodeUuid, nodeReport);
updateCommandQueue(datanodeID,
updateCommandQueue(datanodeUuid,
hbItem.getContainerReportState().getState());
return;
}
@ -623,7 +628,7 @@ private void handleHeartbeat(HeartbeatQueueItem hbItem) {
healthyNodeCount.incrementAndGet();
staleNodeCount.decrementAndGet();
updateNodeStat(datanodeUuid, nodeReport);
updateCommandQueue(datanodeID,
updateCommandQueue(datanodeUuid,
hbItem.getContainerReportState().getState());
return;
}
@ -636,22 +641,22 @@ private void handleHeartbeat(HeartbeatQueueItem hbItem) {
deadNodeCount.decrementAndGet();
healthyNodeCount.incrementAndGet();
updateNodeStat(datanodeUuid, nodeReport);
updateCommandQueue(datanodeID,
updateCommandQueue(datanodeUuid,
hbItem.getContainerReportState().getState());
return;
}
LOG.warn("SCM receive heartbeat from unregistered datanode {}",
datanodeUuid);
this.commandQueue.addCommand(hbItem.getDatanodeID(),
this.commandQueue.addCommand(datanodeUuid,
new ReregisterCommand());
}
private void updateNodeStat(String datanodeUuid, SCMNodeReport nodeReport) {
SCMNodeStat stat = nodeStats.get(datanodeUuid);
private void updateNodeStat(UUID dnId, SCMNodeReport nodeReport) {
SCMNodeStat stat = nodeStats.get(dnId);
if (stat == null) {
LOG.debug("SCM updateNodeStat based on heartbeat from previous" +
"dead datanode {}", datanodeUuid);
"dead datanode {}", dnId);
stat = new SCMNodeStat();
}
@ -667,17 +672,17 @@ private void updateNodeStat(String datanodeUuid, SCMNodeReport nodeReport) {
}
scmStat.subtract(stat);
stat.set(totalCapacity, totalScmUsed, totalRemaining);
nodeStats.put(datanodeUuid, stat);
nodeStats.put(dnId, stat);
scmStat.add(stat);
}
}
private void updateCommandQueue(DatanodeID datanodeID,
private void updateCommandQueue(UUID dnId,
ReportState.states containerReportState) {
if (containerReportState != null) {
switch (containerReportState) {
case completeContinerReport:
commandQueue.addCommand(datanodeID,
commandQueue.addCommand(dnId,
SendContainerCommand.newBuilder().build());
return;
case deltaContainerReport:
@ -736,26 +741,36 @@ public VersionResponse getVersion(SCMVersionRequestProto versionRequest) {
* Register the node if the node finds that it is not registered with any
* SCM.
*
* @param datanodeID - Send datanodeID with Node info. This function
* generates and assigns new datanode ID for the datanode.
* This allows SCM to be run independent of Namenode if
* required.
* @param datanodeDetailsProto - Send datanodeDetails with Node info.
* This function generates and assigns new datanode ID
* for the datanode. This allows SCM to be run independent
* of Namenode if required.
*
* @return SCMHeartbeatResponseProto
*/
@Override
public SCMCommand register(DatanodeID datanodeID) {
public SCMCommand register(DatanodeDetailsProto datanodeDetailsProto) {
SCMCommand responseCommand = verifyDatanodeUUID(datanodeID);
DatanodeDetails datanodeDetails = DatanodeDetails.getFromProtoBuf(
datanodeDetailsProto);
InetAddress dnAddress = Server.getRemoteIp();
if (dnAddress != null) {
// Mostly called inside an RPC, update ip and peer hostname
String hostname = dnAddress.getHostName();
String ip = dnAddress.getHostAddress();
datanodeDetails.setHostName(hostname);
datanodeDetails.setIpAddress(ip);
}
SCMCommand responseCommand = verifyDatanodeUUID(datanodeDetails);
if (responseCommand != null) {
return responseCommand;
}
nodes.put(datanodeID.getDatanodeUuid(), datanodeID);
UUID dnId = datanodeDetails.getUuid();
nodes.put(dnId, datanodeDetails);
totalNodes.incrementAndGet();
healthyNodes.put(datanodeID.getDatanodeUuid(), monotonicNow());
healthyNodes.put(dnId, monotonicNow());
healthyNodeCount.incrementAndGet();
nodeStats.put(datanodeID.getDatanodeUuid(), new SCMNodeStat());
nodeStats.put(dnId, new SCMNodeStat());
if(inStartupChillMode.get() &&
totalNodes.get() >= getMinimumChillModeNodes()) {
@ -767,9 +782,9 @@ public SCMCommand register(DatanodeID datanodeID) {
// For now, all nodes are added to the "DefaultNodePool" upon registration
// if it has not been added to any node pool yet.
try {
if (nodePoolManager.getNodePool(datanodeID) == null) {
if (nodePoolManager.getNodePool(datanodeDetails) == null) {
nodePoolManager.addNode(SCMNodePoolManager.DEFAULT_NODEPOOL,
datanodeID);
datanodeDetails);
}
} catch (IOException e) {
// TODO: make sure registration failure is handled correctly.
@ -778,10 +793,10 @@ public SCMCommand register(DatanodeID datanodeID) {
.build();
}
LOG.info("Data node with ID: {} Registered.",
datanodeID.getDatanodeUuid());
datanodeDetails.getUuid());
return RegisteredCommand.newBuilder()
.setErrorCode(ErrorCode.success)
.setDatanodeUUID(datanodeID.getDatanodeUuid())
.setDatanodeUUID(datanodeDetails.getUuidString())
.setClusterID(this.clusterID)
.build();
}
@ -789,18 +804,18 @@ public SCMCommand register(DatanodeID datanodeID) {
/**
* Verifies the datanode does not have a valid UUID already.
*
* @param datanodeID - Datanode UUID.
* @param datanodeDetails - Datanode Details.
* @return SCMCommand
*/
private SCMCommand verifyDatanodeUUID(DatanodeID datanodeID) {
if (datanodeID.getDatanodeUuid() != null &&
nodes.containsKey(datanodeID.getDatanodeUuid())) {
private SCMCommand verifyDatanodeUUID(DatanodeDetails datanodeDetails) {
if (datanodeDetails.getUuid() != null &&
nodes.containsKey(datanodeDetails.getUuid())) {
LOG.trace("Datanode is already registered. Datanode: {}",
datanodeID.toString());
datanodeDetails.toString());
return RegisteredCommand.newBuilder()
.setErrorCode(ErrorCode.success)
.setClusterID(this.clusterID)
.setDatanodeUUID(datanodeID.getDatanodeUuid())
.setDatanodeUUID(datanodeDetails.getUuidString())
.build();
}
return null;
@ -809,24 +824,28 @@ private SCMCommand verifyDatanodeUUID(DatanodeID datanodeID) {
/**
* Send heartbeat to indicate the datanode is alive and doing well.
*
* @param datanodeID - Datanode ID.
* @param datanodeDetailsProto - DatanodeDetailsProto.
* @param nodeReport - node report.
* @param containerReportState - container report state.
* @return SCMheartbeat response.
* @throws IOException
*/
@Override
public List<SCMCommand> sendHeartbeat(DatanodeID datanodeID,
SCMNodeReport nodeReport, ReportState containerReportState) {
public List<SCMCommand> sendHeartbeat(
DatanodeDetailsProto datanodeDetailsProto, SCMNodeReport nodeReport,
ReportState containerReportState) {
DatanodeDetails datanodeDetails = DatanodeDetails
.getFromProtoBuf(datanodeDetailsProto);
// Checking for NULL to make sure that we don't get
// an exception from ConcurrentList.
// This could be a problem in tests, if this function is invoked via
// protobuf, transport layer will guarantee that this is not null.
if (datanodeID != null) {
if (datanodeDetails != null) {
heartbeatQueue.add(
new HeartbeatQueueItem.Builder()
.setDatanodeID(datanodeID)
.setDatanodeDetails(datanodeDetails)
.setNodeReport(nodeReport)
.setContainerReportState(containerReportState)
.build());
@ -834,7 +853,7 @@ public List<SCMCommand> sendHeartbeat(DatanodeID datanodeID,
LOG.error("Datanode ID in heartbeat is null");
}
return commandQueue.getCommand(datanodeID);
return commandQueue.getCommand(datanodeDetails.getUuid());
}
/**
@ -851,18 +870,18 @@ public SCMNodeStat getStats() {
* @return a map of individual node stats (live/stale but not dead).
*/
@Override
public Map<String, SCMNodeStat> getNodeStats() {
public Map<UUID, SCMNodeStat> getNodeStats() {
return Collections.unmodifiableMap(nodeStats);
}
/**
* Return the node stat of the specified datanode.
* @param datanodeID - datanode ID.
* @param datanodeDetails - datanode ID.
* @return node stat if it is live/stale, null if it is dead or does't exist.
*/
@Override
public SCMNodeMetric getNodeStat(DatanodeID datanodeID) {
return new SCMNodeMetric(nodeStats.get(datanodeID.getDatanodeUuid()));
public SCMNodeMetric getNodeStat(DatanodeDetails datanodeDetails) {
return new SCMNodeMetric(nodeStats.get(datanodeDetails));
}
@Override
@ -880,8 +899,8 @@ public Map<String, Integer> getNodeCount() {
}
@Override
public void addDatanodeCommand(DatanodeID id, SCMCommand command) {
this.commandQueue.addCommand(id, command);
public void addDatanodeCommand(UUID dnId, SCMCommand command) {
this.commandQueue.addCommand(dnId, command);
}
@VisibleForTesting

View File

@ -20,9 +20,9 @@
import com.google.common.base.Preconditions;
import org.apache.hadoop.hdfs.DFSUtil;
import org.apache.hadoop.hdfs.protocol.DatanodeID;
import org.apache.hadoop.hdfs.protocol.proto.HdfsProtos;
import org.apache.hadoop.hdsl.conf.OzoneConfiguration;
import org.apache.hadoop.hdsl.protocol.DatanodeDetails;
import org.apache.hadoop.hdsl.protocol.proto.HdslProtos;
import org.apache.hadoop.ozone.OzoneConsts;
import org.apache.hadoop.ozone.scm.exceptions.SCMException;
import org.apache.hadoop.utils.MetadataStore;
@ -60,7 +60,8 @@ public final class SCMNodePoolManager implements NodePoolManager {
private static final Logger LOG =
LoggerFactory.getLogger(SCMNodePoolManager.class);
private static final List<DatanodeID> EMPTY_NODE_LIST = new ArrayList<>();
private static final List<DatanodeDetails> EMPTY_NODE_LIST =
new ArrayList<>();
private static final List<String> EMPTY_NODEPOOL_LIST = new ArrayList<>();
public static final String DEFAULT_NODEPOOL = "DefaultNodePool";
@ -68,7 +69,7 @@ public final class SCMNodePoolManager implements NodePoolManager {
private MetadataStore nodePoolStore;
// In-memory node pool to nodes mapping
private HashMap<String, Set<DatanodeID>> nodePools;
private HashMap<String, Set<DatanodeDetails>> nodePools;
// Read-write lock for nodepool operations
private ReadWriteLock lock;
@ -104,11 +105,11 @@ private void init() throws SCMException {
try {
nodePoolStore.iterate(null, (key, value) -> {
try {
DatanodeID nodeId = DatanodeID.getFromProtoBuf(
HdfsProtos.DatanodeIDProto.PARSER.parseFrom(key));
DatanodeDetails nodeId = DatanodeDetails.getFromProtoBuf(
HdslProtos.DatanodeDetailsProto.PARSER.parseFrom(key));
String poolName = DFSUtil.bytes2String(value);
Set<DatanodeID> nodePool = null;
Set<DatanodeDetails> nodePool = null;
if (nodePools.containsKey(poolName)) {
nodePool = nodePools.get(poolName);
} else {
@ -138,7 +139,7 @@ private void init() throws SCMException {
* @param node - name of the datanode.
*/
@Override
public void addNode(final String pool, final DatanodeID node)
public void addNode(final String pool, final DatanodeDetails node)
throws IOException {
Preconditions.checkNotNull(pool, "pool name is null");
Preconditions.checkNotNull(node, "node is null");
@ -149,11 +150,11 @@ public void addNode(final String pool, final DatanodeID node)
DFSUtil.string2Bytes(pool));
// add to the in-memory store
Set<DatanodeID> nodePool = null;
Set<DatanodeDetails> nodePool = null;
if (nodePools.containsKey(pool)) {
nodePool = nodePools.get(pool);
} else {
nodePool = new HashSet<DatanodeID>();
nodePool = new HashSet<DatanodeDetails>();
nodePools.put(pool, nodePool);
}
nodePool.add(node);
@ -169,7 +170,7 @@ public void addNode(final String pool, final DatanodeID node)
* @throws SCMException
*/
@Override
public void removeNode(final String pool, final DatanodeID node)
public void removeNode(final String pool, final DatanodeDetails node)
throws SCMException {
Preconditions.checkNotNull(pool, "pool name is null");
Preconditions.checkNotNull(node, "node is null");
@ -187,12 +188,13 @@ public void removeNode(final String pool, final DatanodeID node)
// Remove from the in-memory store
if (nodePools.containsKey(pool)) {
Set<DatanodeID> nodePool = nodePools.get(pool);
Set<DatanodeDetails> nodePool = nodePools.get(pool);
nodePool.remove(node);
} else {
throw new SCMException(String.format("Unable to find node %s from" +
" pool %s in MAP.", DFSUtil.bytes2String(kName), pool),
FAILED_TO_FIND_NODE_IN_POOL); }
FAILED_TO_FIND_NODE_IN_POOL);
}
} catch (IOException e) {
throw new SCMException("Failed to remove node " + node.toString()
+ " from node pool " + pool, e,
@ -226,7 +228,7 @@ public List<String> getNodePools() {
* @return all datanodes of the specified node pool.
*/
@Override
public List<DatanodeID> getNodes(final String pool) {
public List<DatanodeDetails> getNodes(final String pool) {
Preconditions.checkNotNull(pool, "pool name is null");
if (nodePools.containsKey(pool)) {
return nodePools.get(pool).stream().collect(Collectors.toList());
@ -237,21 +239,22 @@ public List<DatanodeID> getNodes(final String pool) {
/**
* Get the node pool name if the node has been added to a node pool.
* @param datanodeID - datanode ID.
* @param datanodeDetails - datanode ID.
* @return node pool name if it has been assigned.
* null if the node has not been assigned to any node pool yet.
* TODO: Put this in a in-memory map if performance is an issue.
*/
@Override
public String getNodePool(final DatanodeID datanodeID) throws SCMException {
Preconditions.checkNotNull(datanodeID, "node is null");
public String getNodePool(final DatanodeDetails datanodeDetails)
throws SCMException {
Preconditions.checkNotNull(datanodeDetails, "node is null");
try {
byte[] result = nodePoolStore.get(
datanodeID.getProtoBufMessage().toByteArray());
datanodeDetails.getProtoBufMessage().toByteArray());
return result == null ? null : DFSUtil.bytes2String(result);
} catch (IOException e) {
throw new SCMException("Failed to get node pool for node "
+ datanodeID.toString(), e,
+ datanodeDetails.toString(), e,
SCMException.ResultCodes.IO_EXCEPTION);
}
}

View File

@ -17,7 +17,7 @@
package org.apache.hadoop.ozone.scm.pipelines;
import org.apache.hadoop.hdfs.protocol.DatanodeID;
import org.apache.hadoop.hdsl.protocol.DatanodeDetails;
import org.apache.hadoop.hdsl.protocol.proto.HdslProtos.ReplicationFactor;
import org.apache.hadoop.hdsl.protocol.proto.HdslProtos.ReplicationType;
import org.apache.hadoop.hdsl.protocol.proto.HdslProtos.LifeCycleState;
@ -154,7 +154,7 @@ private int getNextIndex() {
* @param datanodes - The list of datanodes that make this pipeline.
*/
public abstract void createPipeline(String pipelineID,
List<DatanodeID> datanodes) throws IOException;
List<DatanodeDetails> datanodes) throws IOException;
/**
* Close the pipeline with the given clusterId.
@ -165,12 +165,12 @@ public abstract void createPipeline(String pipelineID,
* list members in the pipeline .
* @return the datanode
*/
public abstract List<DatanodeID> getMembers(String pipelineID)
public abstract List<DatanodeDetails> getMembers(String pipelineID)
throws IOException;
/**
* Update the datanode list of the pipeline.
*/
public abstract void updatePipeline(String pipelineID,
List<DatanodeID> newDatanodes) throws IOException;
List<DatanodeDetails> newDatanodes) throws IOException;
}

View File

@ -18,7 +18,7 @@
import com.google.common.base.Preconditions;
import org.apache.hadoop.conf.Configuration;
import org.apache.hadoop.hdfs.protocol.DatanodeID;
import org.apache.hadoop.hdsl.protocol.DatanodeDetails;
import org.apache.hadoop.ozone.OzoneConsts;
import org.apache.hadoop.hdsl.protocol.proto.HdslProtos;
import org.apache.hadoop.hdsl.protocol.proto.HdslProtos.ReplicationType;
@ -83,16 +83,17 @@ public PipelineSelector(NodeManager nodeManager, Configuration conf) {
* The first of the list will be the leader node.
* @return pipeline corresponding to nodes
*/
public static PipelineChannel newPipelineFromNodes(List<DatanodeID> nodes,
LifeCycleState state, ReplicationType replicationType,
ReplicationFactor replicationFactor, String name) {
public static PipelineChannel newPipelineFromNodes(
List<DatanodeDetails> nodes, LifeCycleState state,
ReplicationType replicationType, ReplicationFactor replicationFactor,
String name) {
Preconditions.checkNotNull(nodes);
Preconditions.checkArgument(nodes.size() > 0);
String leaderId = nodes.get(0).getDatanodeUuid();
String leaderId = nodes.get(0).getUuidString();
PipelineChannel
pipelineChannel = new PipelineChannel(leaderId, state, replicationType,
replicationFactor, name);
for (DatanodeID node : nodes) {
for (DatanodeDetails node : nodes) {
pipelineChannel.addMember(node);
}
return pipelineChannel;
@ -178,11 +179,11 @@ public Pipeline getReplicationPipeline(ReplicationType replicationType,
*/
public void createPipeline(ReplicationType replicationType, String
pipelineID, List<DatanodeID> datanodes) throws IOException {
pipelineID, List<DatanodeDetails> datanodes) throws IOException {
PipelineManager manager = getPipelineManager(replicationType);
Preconditions.checkNotNull(manager, "Found invalid pipeline manager");
LOG.debug("Creating a pipeline: {} with nodes:{}", pipelineID,
datanodes.stream().map(DatanodeID::toString)
datanodes.stream().map(DatanodeDetails::toString)
.collect(Collectors.joining(",")));
manager.createPipeline(pipelineID, datanodes);
}
@ -203,7 +204,7 @@ public void closePipeline(ReplicationType replicationType, String
* list members in the pipeline .
*/
public List<DatanodeID> getDatanodes(ReplicationType replicationType,
public List<DatanodeDetails> getDatanodes(ReplicationType replicationType,
String pipelineID) throws IOException {
PipelineManager manager = getPipelineManager(replicationType);
Preconditions.checkNotNull(manager, "Found invalid pipeline manager");
@ -216,11 +217,11 @@ public List<DatanodeID> getDatanodes(ReplicationType replicationType,
*/
public void updateDatanodes(ReplicationType replicationType, String
pipelineID, List<DatanodeID> newDatanodes) throws IOException {
pipelineID, List<DatanodeDetails> newDatanodes) throws IOException {
PipelineManager manager = getPipelineManager(replicationType);
Preconditions.checkNotNull(manager, "Found invalid pipeline manager");
LOG.debug("Updating pipeline: {} with new nodes:{}", pipelineID,
newDatanodes.stream().map(DatanodeID::toString)
newDatanodes.stream().map(DatanodeDetails::toString)
.collect(Collectors.joining(",")));
manager.updatePipeline(pipelineID, newDatanodes);
}

View File

@ -18,7 +18,7 @@
import com.google.common.base.Preconditions;
import org.apache.hadoop.conf.Configuration;
import org.apache.hadoop.hdfs.protocol.DatanodeID;
import org.apache.hadoop.hdsl.protocol.DatanodeDetails;
import org.apache.hadoop.hdsl.protocol.proto.HdslProtos.LifeCycleState;
import org.apache.hadoop.hdsl.protocol.proto.HdslProtos.ReplicationFactor;
import org.apache.hadoop.hdsl.protocol.proto.HdslProtos.ReplicationType;
@ -52,7 +52,7 @@ public class RatisManagerImpl extends PipelineManager {
private static final String PREFIX = "Ratis-";
private final Configuration conf;
private final NodeManager nodeManager;
private final Set<DatanodeID> ratisMembers;
private final Set<DatanodeDetails> ratisMembers;
/**
* Constructs a Ratis Pipeline Manager.
@ -74,12 +74,12 @@ public RatisManagerImpl(NodeManager nodeManager,
* @return PipelineChannel.
*/
public PipelineChannel allocatePipelineChannel(ReplicationFactor factor) {
List<DatanodeID> newNodesList = new LinkedList<>();
List<DatanodeID> datanodes = nodeManager.getNodes(NodeState.HEALTHY);
List<DatanodeDetails> newNodesList = new LinkedList<>();
List<DatanodeDetails> datanodes = nodeManager.getNodes(NodeState.HEALTHY);
int count = getReplicationCount(factor);
//TODO: Add Raft State to the Nodes, so we can query and skip nodes from
// data from datanode instead of maintaining a set.
for (DatanodeID datanode : datanodes) {
for (DatanodeDetails datanode : datanodes) {
Preconditions.checkNotNull(datanode);
if (!ratisMembers.contains(datanode)) {
newNodesList.add(datanode);
@ -116,7 +116,8 @@ public PipelineChannel allocatePipelineChannel(ReplicationFactor factor) {
* @param datanodes - The list of datanodes that make this pipeline.
*/
@Override
public void createPipeline(String pipelineID, List<DatanodeID> datanodes) {
public void createPipeline(String pipelineID,
List<DatanodeDetails> datanodes) {
}
@ -137,7 +138,8 @@ public void closePipeline(String pipelineID) throws IOException {
* @return the datanode
*/
@Override
public List<DatanodeID> getMembers(String pipelineID) throws IOException {
public List<DatanodeDetails> getMembers(String pipelineID)
throws IOException {
return null;
}
@ -148,7 +150,8 @@ public List<DatanodeID> getMembers(String pipelineID) throws IOException {
* @param newDatanodes
*/
@Override
public void updatePipeline(String pipelineID, List<DatanodeID> newDatanodes)
public void updatePipeline(String pipelineID,
List<DatanodeDetails> newDatanodes)
throws IOException {
}

View File

@ -17,7 +17,7 @@
package org.apache.hadoop.ozone.scm.pipelines.standalone;
import com.google.common.base.Preconditions;
import org.apache.hadoop.hdfs.protocol.DatanodeID;
import org.apache.hadoop.hdsl.protocol.DatanodeDetails;
import org.apache.hadoop.hdsl.protocol.proto.HdslProtos.ReplicationFactor;
import org.apache.hadoop.hdsl.protocol.proto.HdslProtos.ReplicationType;
import org.apache.hadoop.hdsl.protocol.proto.HdslProtos.LifeCycleState;
@ -47,7 +47,7 @@ public class StandaloneManagerImpl extends PipelineManager {
private final NodeManager nodeManager;
private final ContainerPlacementPolicy placementPolicy;
private final long containerSize;
private final Set<DatanodeID> standAloneMembers;
private final Set<DatanodeDetails> standAloneMembers;
/**
* Constructor for Standalone Node Manager Impl.
@ -72,10 +72,10 @@ public StandaloneManagerImpl(NodeManager nodeManager,
* @return PipelineChannel.
*/
public PipelineChannel allocatePipelineChannel(ReplicationFactor factor) {
List<DatanodeID> newNodesList = new LinkedList<>();
List<DatanodeID> datanodes = nodeManager.getNodes(NodeState.HEALTHY);
List<DatanodeDetails> newNodesList = new LinkedList<>();
List<DatanodeDetails> datanodes = nodeManager.getNodes(NodeState.HEALTHY);
int count = getReplicationCount(factor);
for (DatanodeID datanode : datanodes) {
for (DatanodeDetails datanode : datanodes) {
Preconditions.checkNotNull(datanode);
if (!standAloneMembers.contains(datanode)) {
newNodesList.add(datanode);
@ -103,7 +103,8 @@ public PipelineChannel allocatePipelineChannel(ReplicationFactor factor) {
* @param datanodes - The list of datanodes that make this pipeline.
*/
@Override
public void createPipeline(String pipelineID, List<DatanodeID> datanodes) {
public void createPipeline(String pipelineID,
List<DatanodeDetails> datanodes) {
//return newPipelineFromNodes(datanodes, pipelineID);
}
@ -124,7 +125,8 @@ public void closePipeline(String pipelineID) throws IOException {
* @return the datanode
*/
@Override
public List<DatanodeID> getMembers(String pipelineID) throws IOException {
public List<DatanodeDetails> getMembers(String pipelineID)
throws IOException {
return null;
}
@ -135,7 +137,7 @@ public List<DatanodeID> getMembers(String pipelineID) throws IOException {
* @param newDatanodes
*/
@Override
public void updatePipeline(String pipelineID, List<DatanodeID>
public void updatePipeline(String pipelineID, List<DatanodeDetails>
newDatanodes) throws IOException {
}

View File

@ -19,8 +19,8 @@
import org.apache.commons.codec.digest.DigestUtils;
import org.apache.hadoop.conf.Configuration;
import org.apache.hadoop.fs.FileUtil;
import org.apache.hadoop.hdfs.DFSTestUtil;
import org.apache.hadoop.hdfs.protocol.DatanodeID;
import org.apache.hadoop.hdsl.protocol.DatanodeDetails;
import org.apache.hadoop.hdsl.protocol.proto.HdslProtos;
import org.apache.hadoop.ipc.RPC;
import org.apache.hadoop.ozone.OzoneConfigKeys;
import org.apache.hadoop.ozone.OzoneConsts;
@ -38,8 +38,6 @@
.VersionEndpointTask;
import org.apache.hadoop.hdsl.protocol.proto
.StorageContainerDatanodeProtocolProtos;
import org.apache.hadoop.hdsl.protocol.proto
.StorageContainerDatanodeProtocolProtos.ContainerNodeIDProto;
import org.apache.hadoop.hdsl.protocol.proto
.StorageContainerDatanodeProtocolProtos.ContainerReportsRequestProto;
import org.apache.hadoop.hdsl.protocol.proto
@ -54,13 +52,14 @@
.StorageContainerDatanodeProtocolProtos.SCMStorageReport;
import org.apache.hadoop.hdsl.protocol.proto
.StorageContainerDatanodeProtocolProtos.SCMVersionResponseProto;
import org.apache.hadoop.ozone.scm.TestUtils;
import org.apache.hadoop.ozone.scm.VersionInfo;
import org.apache.hadoop.test.PathUtils;
import org.apache.hadoop.util.Time;
import static org.apache.hadoop.ozone.container.common.ContainerTestUtils
.createEndpoint;
import static org.apache.hadoop.ozone.scm.TestUtils.getDatanodeID;
import static org.apache.hadoop.ozone.scm.TestUtils.getDatanodeDetails;
import static org.hamcrest.Matchers.lessThanOrEqualTo;
import org.junit.AfterClass;
import org.junit.Assert;
@ -208,21 +207,21 @@ public void testGetVersionAssertRpcTimeOut() throws Exception {
public void testRegister() throws Exception {
String[] scmAddressArray = new String[1];
scmAddressArray[0] = serverAddress.toString();
DatanodeID nodeToRegister = getDatanodeID();
DatanodeDetails nodeToRegister = getDatanodeDetails();
try (EndpointStateMachine rpcEndPoint =
createEndpoint(
SCMTestUtils.getConf(), serverAddress, 1000)) {
SCMRegisteredCmdResponseProto responseProto = rpcEndPoint.getEndPoint()
.register(nodeToRegister, scmAddressArray);
.register(nodeToRegister.getProtoBufMessage(), scmAddressArray);
Assert.assertNotNull(responseProto);
Assert.assertEquals(nodeToRegister.getDatanodeUuid(),
Assert.assertEquals(nodeToRegister.getUuid(),
responseProto.getDatanodeUUID());
Assert.assertNotNull(responseProto.getClusterID());
}
}
private EndpointStateMachine registerTaskHelper(InetSocketAddress scmAddress,
int rpcTimeout, boolean clearContainerID) throws Exception {
int rpcTimeout, boolean clearDatanodeDetails) throws Exception {
Configuration conf = SCMTestUtils.getConf();
EndpointStateMachine rpcEndPoint =
createEndpoint(conf,
@ -230,12 +229,12 @@ private EndpointStateMachine registerTaskHelper(InetSocketAddress scmAddress,
rpcEndPoint.setState(EndpointStateMachine.EndPointStates.REGISTER);
RegisterEndpointTask endpointTask =
new RegisterEndpointTask(rpcEndPoint, conf);
if (!clearContainerID) {
ContainerNodeIDProto containerNodeID = ContainerNodeIDProto.newBuilder()
.setClusterID(UUID.randomUUID().toString())
.setDatanodeID(getDatanodeID().getProtoBufMessage())
if (!clearDatanodeDetails) {
HdslProtos.DatanodeDetailsProto datanodeDetails =
HdslProtos.DatanodeDetailsProto.newBuilder()
.setUuid(UUID.randomUUID().toString())
.build();
endpointTask.setContainerNodeIDProto(containerNodeID);
endpointTask.setDatanodeDetailsProto(datanodeDetails);
}
endpointTask.call();
return rpcEndPoint;
@ -287,7 +286,7 @@ public void testRegisterRpcTimeout() throws Exception {
@Test
public void testHeartbeat() throws Exception {
DatanodeID dataNode = getDatanodeID();
DatanodeDetails dataNode = getDatanodeDetails();
try (EndpointStateMachine rpcEndPoint =
createEndpoint(SCMTestUtils.getConf(),
serverAddress, 1000)) {
@ -297,7 +296,8 @@ public void testHeartbeat() throws Exception {
srb.setCapacity(2000).setScmUsed(500).setRemaining(1500).build();
nrb.addStorageReport(srb);
SCMHeartbeatResponseProto responseProto = rpcEndPoint.getEndPoint()
.sendHeartbeat(dataNode, nrb.build(), defaultReportState);
.sendHeartbeat(
dataNode.getProtoBufMessage(), nrb.build(), defaultReportState);
Assert.assertNotNull(responseProto);
Assert.assertEquals(0, responseProto.getCommandsCount());
}
@ -316,12 +316,11 @@ private void heartbeatTaskHelper(InetSocketAddress scmAddress,
// Create a datanode state machine for stateConext used by endpoint task
try (DatanodeStateMachine stateMachine = new DatanodeStateMachine(
DFSTestUtil.getLocalDatanodeID(), conf);
TestUtils.getDatanodeDetails(), conf);
EndpointStateMachine rpcEndPoint =
createEndpoint(conf, scmAddress, rpcTimeout)) {
ContainerNodeIDProto containerNodeID = ContainerNodeIDProto.newBuilder()
.setClusterID(UUID.randomUUID().toString())
.setDatanodeID(getDatanodeID().getProtoBufMessage()).build();
HdslProtos.DatanodeDetailsProto datanodeDetailsProto =
getDatanodeDetails().getProtoBufMessage();
rpcEndPoint.setState(EndpointStateMachine.EndPointStates.HEARTBEAT);
final StateContext stateContext =
@ -330,9 +329,9 @@ private void heartbeatTaskHelper(InetSocketAddress scmAddress,
HeartbeatEndpointTask endpointTask =
new HeartbeatEndpointTask(rpcEndPoint, conf, stateContext);
endpointTask.setContainerNodeIDProto(containerNodeID);
endpointTask.setDatanodeDetailsProto(datanodeDetailsProto);
endpointTask.call();
Assert.assertNotNull(endpointTask.getContainerNodeIDProto());
Assert.assertNotNull(endpointTask.getDatanodeDetailsProto());
Assert.assertEquals(EndpointStateMachine.EndPointStates.HEARTBEAT,
rpcEndPoint.getState());
@ -387,7 +386,7 @@ ContainerReport getRandomContainerReport() {
reportsBuilder.addReports(getRandomContainerReport()
.getProtoBufMessage());
}
reportsBuilder.setDatanodeID(getDatanodeID()
reportsBuilder.setDatanodeDetails(getDatanodeDetails()
.getProtoBufMessage());
reportsBuilder.setType(StorageContainerDatanodeProtocolProtos
.ContainerReportsRequestProto.reportType.fullReport);
@ -456,7 +455,7 @@ private ContainerReportsRequestProto createContainerReport(int count) {
reportsBuilder.addReports(report.getProtoBufMessage());
}
reportsBuilder.setDatanodeID(getDatanodeID()
reportsBuilder.setDatanodeDetails(getDatanodeDetails()
.getProtoBufMessage());
reportsBuilder.setType(StorageContainerDatanodeProtocolProtos
.ContainerReportsRequestProto.reportType.fullReport);

View File

@ -18,7 +18,7 @@
import org.apache.commons.math3.stat.descriptive.DescriptiveStatistics;
import org.apache.hadoop.conf.Configuration;
import org.apache.hadoop.hdfs.protocol.DatanodeID;
import org.apache.hadoop.hdsl.protocol.DatanodeDetails;
import org.apache.hadoop.ozone.OzoneConsts;
import org.apache.hadoop.ozone.scm.container.MockNodeManager;
import org.apache.hadoop.ozone.scm.container.placement.algorithms.SCMContainerPlacementCapacity;
@ -41,10 +41,10 @@ public class TestContainerPlacement {
private DescriptiveStatistics computeStatistics(NodeManager nodeManager) {
DescriptiveStatistics descriptiveStatistics = new DescriptiveStatistics();
for (DatanodeID id : nodeManager.getNodes(HEALTHY)) {
for (DatanodeDetails dd : nodeManager.getNodes(HEALTHY)) {
float weightedValue =
nodeManager.getNodeStat(id).get().getScmUsed().get() / (float)
nodeManager.getNodeStat(id).get().getCapacity().get();
nodeManager.getNodeStat(dd).get().getScmUsed().get() / (float)
nodeManager.getNodeStat(dd).get().getCapacity().get();
descriptiveStatistics.addValue(weightedValue);
}
return descriptiveStatistics;
@ -82,11 +82,11 @@ public void testCapacityPlacementYieldsBetterDataDistribution() throws
for (int x = 0; x < opsCount; x++) {
long containerSize = random.nextInt(100) * OzoneConsts.GB;
List<DatanodeID> nodesCapacity =
List<DatanodeDetails> nodesCapacity =
capacityPlacer.chooseDatanodes(nodesRequired, containerSize);
assertEquals(nodesRequired, nodesCapacity.size());
List<DatanodeID> nodesRandom = randomPlacer.chooseDatanodes(nodesRequired,
List<DatanodeDetails> nodesRandom = randomPlacer.chooseDatanodes(nodesRequired,
containerSize);
// One fifth of all calls are delete
@ -116,16 +116,16 @@ public void testCapacityPlacementYieldsBetterDataDistribution() throws
}
private void deleteContainer(MockNodeManager nodeManager,
List<DatanodeID> nodes, long containerSize) {
for (DatanodeID id : nodes) {
nodeManager.delContainer(id, containerSize);
List<DatanodeDetails> nodes, long containerSize) {
for (DatanodeDetails dd : nodes) {
nodeManager.delContainer(dd, containerSize);
}
}
private void createContainer(MockNodeManager nodeManager,
List<DatanodeID> nodes, long containerSize) {
for (DatanodeID id : nodes) {
nodeManager.addContainer(id, containerSize);
List<DatanodeDetails> nodes, long containerSize) {
for (DatanodeDetails dd : nodes) {
nodeManager.addContainer(dd, containerSize);
}
}
}

View File

@ -17,8 +17,8 @@
package org.apache.hadoop.ozone.container.replication;
import org.apache.commons.logging.LogFactory;
import org.apache.hadoop.hdfs.protocol.DatanodeID;
import org.apache.hadoop.hdsl.conf.OzoneConfiguration;
import org.apache.hadoop.hdsl.protocol.DatanodeDetails;
import org.apache.hadoop.ozone.container.testutils.ReplicationDatanodeStateManager;
import org.apache.hadoop.ozone.container.testutils.ReplicationNodeManagerMock;
import org.apache.hadoop.ozone.container.testutils.ReplicationNodePoolManagerMock;
@ -66,7 +66,7 @@ public class TestContainerSupervisor {
static final int POOL_COUNT = 3;
private LogCapturer logCapturer = LogCapturer.captureLogs(
LogFactory.getLog(ContainerSupervisor.class));
private List<DatanodeID> datanodes = new LinkedList<>();
private List<DatanodeDetails> datanodes = new LinkedList<>();
private NodeManager nodeManager;
private NodePoolManager poolManager;
private CommandQueue commandQueue;
@ -82,11 +82,11 @@ public void tearDown() throws Exception {
@Before
public void setUp() throws Exception {
GenericTestUtils.setLogLevel(ContainerSupervisor.LOG, Level.DEBUG);
Map<DatanodeID, NodeState> nodeStateMap = new HashMap<>();
Map<DatanodeDetails, NodeState> nodeStateMap = new HashMap<>();
// We are setting up 3 pools with 24 nodes each in this cluster.
// First we create 72 Datanodes.
for (int x = 0; x < MAX_DATANODES; x++) {
DatanodeID datanode = TestUtils.getDatanodeID();
DatanodeDetails datanode = TestUtils.getDatanodeDetails();
datanodes.add(datanode);
nodeStateMap.put(datanode, HEALTHY);
}
@ -105,7 +105,7 @@ public void setUp() throws Exception {
for (int y = 1; y <= POOL_COUNT; y++) {
String poolName = String.format(POOL_NAME_TEMPLATE, y);
for (int z = 0; z < POOL_SIZE; z++) {
DatanodeID id = datanodes.get(y * z);
DatanodeDetails id = datanodes.get(y * z);
poolManager.addNode(poolName, id);
}
}
@ -245,7 +245,7 @@ public void testAddingNewPoolWorks()
LogFactory.getLog(InProgressPool.class));
GenericTestUtils.setLogLevel(InProgressPool.LOG, Level.DEBUG);
try {
DatanodeID id = TestUtils.getDatanodeID();
DatanodeDetails id = TestUtils.getDatanodeDetails();
((ReplicationNodeManagerMock) (nodeManager)).addNode(id, HEALTHY);
poolManager.addNode("PoolNew", id);
GenericTestUtils.waitFor(() ->
@ -260,7 +260,8 @@ public void testAddingNewPoolWorks()
containerSupervisor.handleContainerReport(clist.get(0));
GenericTestUtils.waitFor(() ->
inProgressLog.getOutput().contains("NewContainer1") && inProgressLog
.getOutput().contains(id.getDatanodeUuid()), 200, 10 * 1000);
.getOutput().contains(id.getUuidString()),
200, 10 * 1000);
} finally {
inProgressLog.stopCapturing();
}

View File

@ -17,7 +17,7 @@
package org.apache.hadoop.ozone.container.testutils;
import org.apache.commons.codec.digest.DigestUtils;
import org.apache.hadoop.hdfs.protocol.DatanodeID;
import org.apache.hadoop.hdsl.protocol.DatanodeDetails;
import org.apache.hadoop.hdsl.protocol.proto.StorageContainerDatanodeProtocolProtos.ContainerInfo;
import org.apache.hadoop.hdsl.protocol.proto.StorageContainerDatanodeProtocolProtos.ContainerReportsRequestProto;
import org.apache.hadoop.ozone.scm.node.NodeManager;
@ -61,7 +61,7 @@ public ReplicationDatanodeStateManager(NodeManager nodeManager,
public List<ContainerReportsRequestProto> getContainerReport(
String containerName, String poolName, int dataNodeCount) {
List<ContainerReportsRequestProto> containerList = new LinkedList<>();
List<DatanodeID> nodesInPool = poolManager.getNodes(poolName);
List<DatanodeDetails> nodesInPool = poolManager.getNodes(poolName);
if (nodesInPool == null) {
return containerList;
@ -74,7 +74,7 @@ public List<ContainerReportsRequestProto> getContainerReport(
int containerID = 1;
while (containerList.size() < dataNodeCount && nodesInPool.size() > 0) {
DatanodeID id = nodesInPool.get(r.nextInt(nodesInPool.size()));
DatanodeDetails id = nodesInPool.get(r.nextInt(nodesInPool.size()));
nodesInPool.remove(id);
containerID++;
// We return container reports only for nodes that are healthy.
@ -86,7 +86,7 @@ public List<ContainerReportsRequestProto> getContainerReport(
.build();
ContainerReportsRequestProto containerReport =
ContainerReportsRequestProto.newBuilder().addReports(info)
.setDatanodeID(id.getProtoBufMessage())
.setDatanodeDetails(id.getProtoBufMessage())
.setType(ContainerReportsRequestProto.reportType.fullReport)
.build();
containerList.add(containerReport);

View File

@ -17,8 +17,9 @@
package org.apache.hadoop.ozone.container.testutils;
import com.google.common.base.Preconditions;
import org.apache.hadoop.hdfs.protocol.DatanodeID;
import org.apache.hadoop.hdfs.protocol.UnregisteredNodeException;
import org.apache.hadoop.hdsl.protocol.DatanodeDetails;
import org.apache.hadoop.hdsl.protocol.proto.HdslProtos;
import org.apache.hadoop.ozone.protocol.VersionResponse;
import org.apache.hadoop.ozone.protocol.commands.SCMCommand;
import org.apache.hadoop.hdsl.protocol.proto.HdslProtos.NodeState;
@ -37,20 +38,22 @@
import java.io.IOException;
import java.util.List;
import java.util.Map;
import java.util.UUID;
import org.mockito.Mockito;
/**
* A Node Manager to test replication.
*/
public class ReplicationNodeManagerMock implements NodeManager {
private final Map<DatanodeID, NodeState> nodeStateMap;
private final Map<DatanodeDetails, NodeState> nodeStateMap;
private final CommandQueue commandQueue;
/**
* A list of Datanodes and current states.
* @param nodeState A node state map.
*/
public ReplicationNodeManagerMock(Map<DatanodeID, NodeState> nodeState,
public ReplicationNodeManagerMock(Map<DatanodeDetails, NodeState> nodeState,
CommandQueue commandQueue) {
Preconditions.checkNotNull(nodeState);
this.nodeStateMap = nodeState;
@ -94,7 +97,8 @@ public Map<String, Integer> getNodeCount() {
* @throws UnregisteredNodeException
*/
@Override
public void removeNode(DatanodeID node) throws UnregisteredNodeException {
public void removeNode(DatanodeDetails node)
throws UnregisteredNodeException {
nodeStateMap.remove(node);
}
@ -106,7 +110,7 @@ public void removeNode(DatanodeID node) throws UnregisteredNodeException {
* @return List of Datanodes that are Heartbeating SCM.
*/
@Override
public List<DatanodeID> getNodes(NodeState nodestate) {
public List<DatanodeDetails> getNodes(NodeState nodestate) {
return null;
}
@ -124,10 +128,10 @@ public int getNodeCount(NodeState nodestate) {
/**
* Get all datanodes known to SCM.
*
* @return List of DatanodeIDs known to SCM.
* @return List of DatanodeDetails known to SCM.
*/
@Override
public List<DatanodeID> getAllNodes() {
public List<DatanodeDetails> getAllNodes() {
return null;
}
@ -185,18 +189,18 @@ public SCMNodeStat getStats() {
* @return a map of individual node stats (live/stale but not dead).
*/
@Override
public Map<String, SCMNodeStat> getNodeStats() {
public Map<UUID, SCMNodeStat> getNodeStats() {
return null;
}
/**
* Return the node stat of the specified datanode.
*
* @param datanodeID - datanode ID.
* @param dd - datanode details.
* @return node stat if it is live/stale, null if it is dead or does't exist.
*/
@Override
public SCMNodeMetric getNodeStat(DatanodeID datanodeID) {
public SCMNodeMetric getNodeStat(DatanodeDetails dd) {
return null;
}
@ -218,12 +222,12 @@ public boolean waitForHeartbeatProcessed() {
/**
* Returns the node state of a specific node.
*
* @param id - DatanodeID
* @param dd - DatanodeDetails
* @return Healthy/Stale/Dead.
*/
@Override
public NodeState getNodeState(DatanodeID id) {
return nodeStateMap.get(id);
public NodeState getNodeState(DatanodeDetails dd) {
return nodeStateMap.get(dd);
}
/**
@ -275,25 +279,25 @@ public VersionResponse getVersion(SCMVersionRequestProto versionRequest) {
/**
* Register the node if the node finds that it is not registered with any SCM.
*
* @param datanodeID - Send datanodeID with Node info, but datanode UUID is
* empty. Server returns a datanodeID for the given node.
* @param dd DatanodeDetailsProto
*
* @return SCMHeartbeatResponseProto
*/
@Override
public SCMCommand register(DatanodeID datanodeID) {
public SCMCommand register(HdslProtos.DatanodeDetailsProto dd) {
return null;
}
/**
* Send heartbeat to indicate the datanode is alive and doing well.
*
* @param datanodeID - Datanode ID.
* @param dd - Datanode Details.
* @param nodeReport - node report.
* @param containerReportState - container report state.
* @return SCMheartbeat response list
*/
@Override
public List<SCMCommand> sendHeartbeat(DatanodeID datanodeID,
public List<SCMCommand> sendHeartbeat(HdslProtos.DatanodeDetailsProto dd,
SCMNodeReport nodeReport, ReportState containerReportState) {
return null;
}
@ -308,16 +312,16 @@ public void clearMap() {
/**
* Adds a node to the existing Node manager. This is used only for test
* purposes.
* @param id - DatanodeID
* @param id DatanodeDetails
* @param state State you want to put that node to.
*/
public void addNode(DatanodeID id, NodeState state) {
public void addNode(DatanodeDetails id, NodeState state) {
nodeStateMap.put(id, state);
}
@Override
public void addDatanodeCommand(DatanodeID id, SCMCommand command) {
this.commandQueue.addCommand(id, command);
public void addDatanodeCommand(UUID dnId, SCMCommand command) {
this.commandQueue.addCommand(dnId, command);
}
}

View File

@ -16,7 +16,7 @@
*/
package org.apache.hadoop.ozone.container.testutils;
import org.apache.hadoop.hdfs.protocol.DatanodeID;
import org.apache.hadoop.hdsl.protocol.DatanodeDetails;
import org.apache.hadoop.ozone.scm.exceptions.SCMException;
import org.apache.hadoop.ozone.scm.node.NodePoolManager;
@ -33,7 +33,7 @@
*/
public class ReplicationNodePoolManagerMock implements NodePoolManager {
private final Map<DatanodeID, String> nodeMemberShip;
private final Map<DatanodeDetails, String> nodeMemberShip;
/**
* A node pool manager for testing.
@ -49,7 +49,7 @@ public ReplicationNodePoolManagerMock() {
* @param node - data node.
*/
@Override
public void addNode(String pool, DatanodeID node) {
public void addNode(String pool, DatanodeDetails node) {
nodeMemberShip.put(node, pool);
}
@ -61,7 +61,8 @@ public void addNode(String pool, DatanodeID node) {
* @throws SCMException
*/
@Override
public void removeNode(String pool, DatanodeID node) throws SCMException {
public void removeNode(String pool, DatanodeDetails node)
throws SCMException {
nodeMemberShip.remove(node);
}
@ -75,7 +76,7 @@ public void removeNode(String pool, DatanodeID node) throws SCMException {
@Override
public List<String> getNodePools() {
Set<String> poolSet = new HashSet<>();
for (Map.Entry<DatanodeID, String> entry : nodeMemberShip.entrySet()) {
for (Map.Entry<DatanodeDetails, String> entry : nodeMemberShip.entrySet()) {
poolSet.add(entry.getValue());
}
return new ArrayList<>(poolSet);
@ -90,9 +91,9 @@ public List<String> getNodePools() {
* found.
*/
@Override
public List<DatanodeID> getNodes(String pool) {
Set<DatanodeID> datanodeSet = new HashSet<>();
for (Map.Entry<DatanodeID, String> entry : nodeMemberShip.entrySet()) {
public List<DatanodeDetails> getNodes(String pool) {
Set<DatanodeDetails> datanodeSet = new HashSet<>();
for (Map.Entry<DatanodeDetails, String> entry : nodeMemberShip.entrySet()) {
if (entry.getValue().equals(pool)) {
datanodeSet.add(entry.getKey());
}
@ -103,13 +104,13 @@ public List<DatanodeID> getNodes(String pool) {
/**
* Get the node pool name if the node has been added to a node pool.
*
* @param datanodeID - datanode ID.
* @param datanodeDetails DatanodeDetails.
* @return node pool name if it has been assigned. null if the node has not
* been assigned to any node pool yet.
*/
@Override
public String getNodePool(DatanodeID datanodeID) {
return nodeMemberShip.get(datanodeID);
public String getNodePool(DatanodeDetails datanodeDetails) {
return nodeMemberShip.get(datanodeDetails);
}
/**

View File

@ -21,8 +21,7 @@
import java.util.Random;
import java.util.UUID;
import org.apache.hadoop.hdfs.protocol.DatanodeID;
import org.apache.hadoop.ozone.protocol.commands.RegisteredCommand;
import org.apache.hadoop.hdsl.protocol.DatanodeDetails;
import org.apache.hadoop.ozone.scm.node.SCMNodeManager;
/**
@ -33,70 +32,79 @@ public class TestUtils {
private TestUtils() {
}
public static DatanodeID getDatanodeID(SCMNodeManager nodeManager) {
public static DatanodeDetails getDatanodeDetails(SCMNodeManager nodeManager) {
return getDatanodeID(nodeManager, UUID.randomUUID().toString());
return getDatanodeDetails(nodeManager, UUID.randomUUID().toString());
}
/**
* Create a new DatanodeID with NodeID set to the string.
* Create a new DatanodeDetails with NodeID set to the string.
*
* @param uuid - node ID, it is generally UUID.
* @return DatanodeID.
*/
public static DatanodeID getDatanodeID(SCMNodeManager nodeManager,
public static DatanodeDetails getDatanodeDetails(SCMNodeManager nodeManager,
String uuid) {
DatanodeID tempDataNode = getDatanodeID(uuid);
RegisteredCommand command =
(RegisteredCommand) nodeManager.register(tempDataNode);
return new DatanodeID(command.getDatanodeUUID(), tempDataNode);
DatanodeDetails datanodeDetails = getDatanodeDetails(uuid);
nodeManager.register(datanodeDetails.getProtoBufMessage());
return datanodeDetails;
}
/**
* Get specified number of datanode IDs and registered them with node manager.
* Get specified number of DatanodeDetails and registered them with node
* manager.
*
* @param nodeManager - node manager to register the datanode ids.
* @param count - number of datanode IDs needed.
* @param count - number of DatanodeDetails needed.
* @return
*/
public static List<DatanodeID> getRegisteredDatanodeIDs(
public static List<DatanodeDetails> getListOfRegisteredDatanodeDetails(
SCMNodeManager nodeManager, int count) {
ArrayList<DatanodeID> datanodes = new ArrayList<>();
ArrayList<DatanodeDetails> datanodes = new ArrayList<>();
for (int i = 0; i < count; i++) {
datanodes.add(getDatanodeID(nodeManager));
datanodes.add(getDatanodeDetails(nodeManager));
}
return datanodes;
}
/**
* Get a datanode ID.
* Get a datanode details.
*
* @return DatanodeID
* @return DatanodeDetails
*/
public static DatanodeID getDatanodeID() {
return getDatanodeID(UUID.randomUUID().toString());
public static DatanodeDetails getDatanodeDetails() {
return getDatanodeDetails(UUID.randomUUID().toString());
}
private static DatanodeID getDatanodeID(String uuid) {
private static DatanodeDetails getDatanodeDetails(String uuid) {
Random random = new Random();
String ipAddress =
random.nextInt(256) + "." + random.nextInt(256) + "." + random
.nextInt(256) + "." + random.nextInt(256);
String hostName = uuid;
return new DatanodeID(ipAddress, hostName, uuid, 0, 0, 0, 0);
DatanodeDetails.Builder builder = DatanodeDetails.newBuilder();
builder.setUuid(uuid)
.setHostName("localhost")
.setIpAddress(ipAddress)
.setInfoPort(0)
.setInfoSecurePort(0)
.setContainerPort(0)
.setRatisPort(0)
.setOzoneRestPort(0);
return builder.build();
}
/**
* Get specified number of datanode IDs.
* Get specified number of list of DatanodeDetails.
*
* @param count - number of datanode IDs needed.
* @return
*/
public static List<DatanodeID> getDatanodeIDs(int count) {
ArrayList<DatanodeID> datanodes = new ArrayList<>();
public static List<DatanodeDetails> getListOfDatanodeDetails(int count) {
ArrayList<DatanodeDetails> datanodes = new ArrayList<>();
for (int i = 0; i < count; i++) {
datanodes.add(getDatanodeID());
datanodes.add(getDatanodeDetails());
}
return datanodes;
}

View File

@ -112,8 +112,8 @@ public void testGetAllocatedBlock() throws IOException {
type, factor, containerOwner);
Assert.assertNotNull(block);
Pipeline pipeline = blockManager.getBlock(block.getKey());
Assert.assertEquals(pipeline.getLeader().getDatanodeUuid(),
block.getPipeline().getLeader().getDatanodeUuid());
Assert.assertEquals(pipeline.getLeader().getUuid(),
block.getPipeline().getLeader().getUuid());
}
@Test
@ -131,8 +131,8 @@ public void testDeleteBlock() throws Exception {
// cleaned yet.
String deletedKeyName = blockManager.getDeletedKeyName(block.getKey());
Pipeline pipeline = blockManager.getBlock(deletedKeyName);
Assert.assertEquals(pipeline.getLeader().getDatanodeUuid(),
block.getPipeline().getLeader().getDatanodeUuid());
Assert.assertEquals(pipeline.getLeader().getUuid(),
block.getPipeline().getLeader().getUuid());
}
@Test

View File

@ -19,8 +19,8 @@
import org.apache.commons.io.FileUtils;
import org.apache.hadoop.hdfs.DFSUtil;
import org.apache.hadoop.hdfs.protocol.DatanodeID;
import org.apache.hadoop.hdsl.conf.OzoneConfiguration;
import org.apache.hadoop.hdsl.protocol.DatanodeDetails;
import org.apache.hadoop.hdsl.protocol.proto.HdslProtos.LifeCycleState;
import org.apache.hadoop.hdsl.protocol.proto.HdslProtos.ReplicationType;
import org.apache.hadoop.hdsl.protocol.proto.HdslProtos.ReplicationFactor;
@ -260,8 +260,26 @@ public void testDeletedBlockTransactions() throws IOException {
int count = 0;
String containerName = null;
DatanodeID dnID1 = new DatanodeID(null, null, "node1", 0, 0, 0, 0);
DatanodeID dnID2 = new DatanodeID(null, null, "node2", 0, 0, 0, 0);
DatanodeDetails dnDd1 = DatanodeDetails.newBuilder()
.setUuid("node1")
.setIpAddress("127.0.0.1")
.setHostName("localhost")
.setInfoPort(0)
.setInfoSecurePort(0)
.setContainerPort(0)
.setRatisPort(0)
.setOzoneRestPort(0)
.build();
DatanodeDetails dnId2 = DatanodeDetails.newBuilder()
.setUuid("node2")
.setIpAddress("127.0.0.1")
.setHostName("localhost")
.setInfoPort(0)
.setInfoSecurePort(0)
.setContainerPort(0)
.setRatisPort(0)
.setOzoneRestPort(0)
.build();
Mapping mappingService = mock(ContainerMapping.class);
// Creates {TXNum} TX in the log.
for (Map.Entry<String, List<String>> entry : generateData(txNum)
@ -273,9 +291,9 @@ public void testDeletedBlockTransactions() throws IOException {
// make TX[1-6] for datanode1; TX[7-10] for datanode2
if (count <= (maximumAllowedTXNum + 1)) {
mockContainerInfo(mappingService, containerName, dnID1);
mockContainerInfo(mappingService, containerName, dnDd1);
} else {
mockContainerInfo(mappingService, containerName, dnID2);
mockContainerInfo(mappingService, containerName, dnId2);
}
}
@ -285,9 +303,9 @@ public void testDeletedBlockTransactions() throws IOException {
deletedBlockLog.getTransactions(transactions);
List<Long> txIDs = new LinkedList<>();
for (DatanodeID dnID : transactions.getDatanodes()) {
for (UUID id : transactions.getDatanodeIDs()) {
List<DeletedBlocksTransaction> txs = transactions
.getDatanodeTransactions(dnID);
.getDatanodeTransactions(id);
for (DeletedBlocksTransaction tx : txs) {
txIDs.add(tx.getTxID());
}
@ -303,9 +321,9 @@ public void testDeletedBlockTransactions() throws IOException {
Assert.assertFalse(transactions.isFull());
// The number of TX in dnID1 won't more than maximum value.
Assert.assertEquals(maximumAllowedTXNum,
transactions.getDatanodeTransactions(dnID1).size());
transactions.getDatanodeTransactions(dnDd1.getUuid()).size());
int size = transactions.getDatanodeTransactions(dnID2).size();
int size = transactions.getDatanodeTransactions(dnId2.getUuid()).size();
// add duplicated container in dnID2, this should be failed.
DeletedBlocksTransaction.Builder builder =
DeletedBlocksTransaction.newBuilder();
@ -316,7 +334,7 @@ public void testDeletedBlockTransactions() throws IOException {
// The number of TX in dnID2 should not be changed.
Assert.assertEquals(size,
transactions.getDatanodeTransactions(dnID2).size());
transactions.getDatanodeTransactions(dnId2.getUuid()).size());
// Add new TX in dnID2, then dnID2 will reach maximum value.
containerName = "newContainer";
@ -324,18 +342,18 @@ public void testDeletedBlockTransactions() throws IOException {
builder.setTxID(12);
builder.setContainerName(containerName);
builder.setCount(0);
mockContainerInfo(mappingService, containerName, dnID2);
mockContainerInfo(mappingService, containerName, dnId2);
transactions.addTransaction(builder.build());
// Since all node are full, then transactions is full.
Assert.assertTrue(transactions.isFull());
}
private void mockContainerInfo(Mapping mappingService, String containerName,
DatanodeID dnID) throws IOException {
DatanodeDetails dd) throws IOException {
PipelineChannel pipelineChannel =
new PipelineChannel("fake", LifeCycleState.OPEN,
ReplicationType.STAND_ALONE, ReplicationFactor.ONE, "fake");
pipelineChannel.addMember(dnID);
pipelineChannel.addMember(dd);
Pipeline pipeline = new Pipeline(containerName, pipelineChannel);
ContainerInfo.Builder builder = new ContainerInfo.Builder();

View File

@ -16,8 +16,8 @@
*/
package org.apache.hadoop.ozone.scm.container;
import org.apache.hadoop.hdfs.protocol.DatanodeID;
import org.apache.hadoop.hdfs.protocol.UnregisteredNodeException;
import org.apache.hadoop.hdsl.protocol.DatanodeDetails;
import org.apache.hadoop.ozone.OzoneConsts;
import org.apache.hadoop.ozone.protocol.VersionResponse;
import org.apache.hadoop.ozone.protocol.commands.SCMCommand;
@ -36,7 +36,7 @@
import org.apache.hadoop.ozone.scm.node.NodeManager;
import org.apache.hadoop.ozone.scm.node.NodePoolManager;
import static org.apache.hadoop.ozone.scm.TestUtils.getDatanodeID;
import static org.apache.hadoop.ozone.scm.TestUtils.getDatanodeDetails;
import org.mockito.Mockito;
import org.assertj.core.util.Preconditions;
@ -45,6 +45,7 @@
import java.util.LinkedList;
import java.util.List;
import java.util.Map;
import java.util.UUID;
import static org.apache.hadoop.hdsl.protocol.proto.HdslProtos.NodeState.DEAD;
import static org.apache.hadoop.hdsl.protocol.proto.HdslProtos.NodeState
@ -69,13 +70,13 @@ public class MockNodeManager implements NodeManager {
new NodeData(OzoneConsts.TB, 200L * OzoneConsts.GB, NodeData.STALE),
new NodeData(OzoneConsts.TB, 200L * OzoneConsts.GB, NodeData.DEAD)
};
private final List<DatanodeID> healthyNodes;
private final List<DatanodeID> staleNodes;
private final List<DatanodeID> deadNodes;
private final Map<String, SCMNodeStat> nodeMetricMap;
private final List<DatanodeDetails> healthyNodes;
private final List<DatanodeDetails> staleNodes;
private final List<DatanodeDetails> deadNodes;
private final Map<UUID, SCMNodeStat> nodeMetricMap;
private final SCMNodeStat aggregateStat;
private boolean chillmode;
private final Map<DatanodeID, List<SCMCommand>> commandMap;
private final Map<UUID, List<SCMCommand>> commandMap;
public MockNodeManager(boolean initializeFakeNodes, int nodeCount) {
this.healthyNodes = new LinkedList<>();
@ -85,8 +86,8 @@ public MockNodeManager(boolean initializeFakeNodes, int nodeCount) {
aggregateStat = new SCMNodeStat();
if (initializeFakeNodes) {
for (int x = 0; x < nodeCount; x++) {
DatanodeID id = getDatanodeID();
populateNodeMetric(id, x);
DatanodeDetails dd = getDatanodeDetails();
populateNodeMetric(dd, x);
}
}
chillmode = false;
@ -96,28 +97,28 @@ public MockNodeManager(boolean initializeFakeNodes, int nodeCount) {
/**
* Invoked from ctor to create some node Metrics.
*
* @param datanodeID - Datanode ID
* @param datanodeDetails - Datanode details
*/
private void populateNodeMetric(DatanodeID datanodeID, int x) {
private void populateNodeMetric(DatanodeDetails datanodeDetails, int x) {
SCMNodeStat newStat = new SCMNodeStat();
long remaining =
NODES[x % NODES.length].capacity - NODES[x % NODES.length].used;
newStat.set(
(NODES[x % NODES.length].capacity),
(NODES[x % NODES.length].used), remaining);
this.nodeMetricMap.put(datanodeID.toString(), newStat);
this.nodeMetricMap.put(datanodeDetails.getUuid(), newStat);
aggregateStat.add(newStat);
if (NODES[x % NODES.length].getCurrentState() == NodeData.HEALTHY) {
healthyNodes.add(datanodeID);
healthyNodes.add(datanodeDetails);
}
if (NODES[x % NODES.length].getCurrentState() == NodeData.STALE) {
staleNodes.add(datanodeID);
staleNodes.add(datanodeDetails);
}
if (NODES[x % NODES.length].getCurrentState() == NodeData.DEAD) {
deadNodes.add(datanodeID);
deadNodes.add(datanodeDetails);
}
}
@ -137,7 +138,8 @@ public void setChillmode(boolean chillmode) {
* @throws UnregisteredNodeException
*/
@Override
public void removeNode(DatanodeID node) throws UnregisteredNodeException {
public void removeNode(DatanodeDetails node)
throws UnregisteredNodeException {
}
@ -148,7 +150,7 @@ public void removeNode(DatanodeID node) throws UnregisteredNodeException {
* @return List of Datanodes that are Heartbeating SCM.
*/
@Override
public List<DatanodeID> getNodes(HdslProtos.NodeState nodestate) {
public List<DatanodeDetails> getNodes(HdslProtos.NodeState nodestate) {
if (nodestate == HEALTHY) {
return healthyNodes;
}
@ -172,7 +174,7 @@ public List<DatanodeID> getNodes(HdslProtos.NodeState nodestate) {
*/
@Override
public int getNodeCount(HdslProtos.NodeState nodestate) {
List<DatanodeID> nodes = getNodes(nodestate);
List<DatanodeDetails> nodes = getNodes(nodestate);
if (nodes != null) {
return nodes.size();
}
@ -182,10 +184,10 @@ public int getNodeCount(HdslProtos.NodeState nodestate) {
/**
* Get all datanodes known to SCM.
*
* @return List of DatanodeIDs known to SCM.
* @return List of DatanodeDetails known to SCM.
*/
@Override
public List<DatanodeID> getAllNodes() {
public List<DatanodeDetails> getAllNodes() {
return null;
}
@ -261,18 +263,18 @@ public SCMNodeStat getStats() {
* @return a list of individual node stats (live/stale but not dead).
*/
@Override
public Map<String, SCMNodeStat> getNodeStats() {
public Map<UUID, SCMNodeStat> getNodeStats() {
return nodeMetricMap;
}
/**
* Return the node stat of the specified datanode.
* @param datanodeID - datanode ID.
* @param datanodeDetails - datanode details.
* @return node stat if it is live/stale, null if it is dead or does't exist.
*/
@Override
public SCMNodeMetric getNodeStat(DatanodeID datanodeID) {
return new SCMNodeMetric(nodeMetricMap.get(datanodeID.toString()));
public SCMNodeMetric getNodeStat(DatanodeDetails datanodeDetails) {
return new SCMNodeMetric(nodeMetricMap.get(datanodeDetails.getUuid()));
}
@Override
@ -293,36 +295,36 @@ public boolean waitForHeartbeatProcessed() {
/**
* Returns the node state of a specific node.
*
* @param id - DatanodeID
* @param dd - DatanodeDetails
* @return Healthy/Stale/Dead.
*/
@Override
public HdslProtos.NodeState getNodeState(DatanodeID id) {
public HdslProtos.NodeState getNodeState(DatanodeDetails dd) {
return null;
}
@Override
public void addDatanodeCommand(DatanodeID id, SCMCommand command) {
if(commandMap.containsKey(id)) {
List<SCMCommand> commandList = commandMap.get(id);
public void addDatanodeCommand(UUID dnId, SCMCommand command) {
if(commandMap.containsKey(dnId)) {
List<SCMCommand> commandList = commandMap.get(dnId);
Preconditions.checkNotNull(commandList);
commandList.add(command);
} else {
List<SCMCommand> commandList = new LinkedList<>();
commandList.add(command);
commandMap.put(id, commandList);
commandMap.put(dnId, commandList);
}
}
// Returns the number of commands that is queued to this node manager.
public int getCommandCount(DatanodeID id) {
List<SCMCommand> list = commandMap.get(id);
public int getCommandCount(DatanodeDetails dd) {
List<SCMCommand> list = commandMap.get(dd);
return (list == null) ? 0 : list.size();
}
public void clearCommandQueue(DatanodeID id) {
if(commandMap.containsKey(id)) {
commandMap.put(id, new LinkedList<>());
public void clearCommandQueue(UUID dnId) {
if(commandMap.containsKey(dnId)) {
commandMap.put(dnId, new LinkedList<>());
}
}
@ -373,29 +375,29 @@ public VersionResponse getVersion(SCMVersionRequestProto versionRequest) {
* Register the node if the node finds that it is not registered with any
* SCM.
*
* @param datanodeID - Send datanodeID with Node info, but datanode UUID is
* empty. Server returns a datanodeID for the given node.
* @param datanodeDetails DatanodeDetailsProto
* @return SCMHeartbeatResponseProto
*/
@Override
public SCMCommand register(DatanodeID datanodeID) {
public SCMCommand register(HdslProtos.DatanodeDetailsProto datanodeDetails) {
return null;
}
/**
* Send heartbeat to indicate the datanode is alive and doing well.
*
* @param datanodeID - Datanode ID.
* @param datanodeDetails - Datanode ID.
* @param nodeReport - node report.
* @param containerReportState - container report state.
* @return SCMheartbeat response list
*/
@Override
public List<SCMCommand> sendHeartbeat(DatanodeID datanodeID,
public List<SCMCommand> sendHeartbeat(
HdslProtos.DatanodeDetailsProto datanodeDetails,
SCMNodeReport nodeReport, ReportState containerReportState) {
if ((datanodeID != null) && (nodeReport != null) && (nodeReport
if ((datanodeDetails != null) && (nodeReport != null) && (nodeReport
.getStorageReportCount() > 0)) {
SCMNodeStat stat = this.nodeMetricMap.get(datanodeID.toString());
SCMNodeStat stat = this.nodeMetricMap.get(datanodeDetails.getUuid());
long totalCapacity = 0L;
long totalRemaining = 0L;
@ -409,7 +411,8 @@ public List<SCMCommand> sendHeartbeat(DatanodeID datanodeID,
aggregateStat.subtract(stat);
stat.set(totalCapacity, totalScmUsed, totalRemaining);
aggregateStat.add(stat);
nodeMetricMap.put(datanodeID.toString(), stat);
nodeMetricMap.put(DatanodeDetails
.getFromProtoBuf(datanodeDetails).getUuid(), stat);
}
return null;
@ -427,32 +430,32 @@ public Map<String, Integer> getNodeCount() {
/**
* Makes it easy to add a container.
*
* @param datanodeID datanode ID
* @param datanodeDetails datanode details
* @param size number of bytes.
*/
public void addContainer(DatanodeID datanodeID, long size) {
SCMNodeStat stat = this.nodeMetricMap.get(datanodeID.toString());
public void addContainer(DatanodeDetails datanodeDetails, long size) {
SCMNodeStat stat = this.nodeMetricMap.get(datanodeDetails.getUuid());
if (stat != null) {
aggregateStat.subtract(stat);
stat.getCapacity().add(size);
aggregateStat.add(stat);
nodeMetricMap.put(datanodeID.toString(), stat);
nodeMetricMap.put(datanodeDetails.getUuid(), stat);
}
}
/**
* Makes it easy to simulate a delete of a container.
*
* @param datanodeID datanode ID
* @param datanodeDetails datanode Details
* @param size number of bytes.
*/
public void delContainer(DatanodeID datanodeID, long size) {
SCMNodeStat stat = this.nodeMetricMap.get(datanodeID.toString());
public void delContainer(DatanodeDetails datanodeDetails, long size) {
SCMNodeStat stat = this.nodeMetricMap.get(datanodeDetails.getUuid());
if (stat != null) {
aggregateStat.subtract(stat);
stat.getCapacity().subtract(size);
aggregateStat.add(stat);
nodeMetricMap.put(datanodeID.toString(), stat);
nodeMetricMap.put(datanodeDetails.getUuid(), stat);
}
}

View File

@ -18,7 +18,7 @@
import org.apache.hadoop.conf.Configuration;
import org.apache.hadoop.fs.FileUtil;
import org.apache.hadoop.hdfs.protocol.DatanodeID;
import org.apache.hadoop.hdsl.protocol.DatanodeDetails;
import org.apache.hadoop.ozone.OzoneConfigKeys;
import org.apache.hadoop.hdsl.protocol.proto.HdslProtos;
import org.apache.hadoop.hdsl.protocol.proto
@ -34,7 +34,6 @@
import org.apache.hadoop.scm.container.common.helpers.Pipeline;
import org.apache.hadoop.test.GenericTestUtils;
import static org.apache.hadoop.ozone.scm.TestUtils.getDatanodeID;
import org.junit.AfterClass;
import org.junit.Assert;
import org.junit.Before;
@ -118,7 +117,7 @@ public void testallocateContainerDistributesAllocation() throws Exception {
5 separate nodes from the list of 10 datanodes that got allocated a
container.
*/
Set<String> pipelineList = new TreeSet<>();
Set<UUID> pipelineList = new TreeSet<>();
for (int x = 0; x < 30; x++) {
ContainerInfo containerInfo = mapping.allocateContainer(
xceiverClientManager.getType(),
@ -128,7 +127,7 @@ public void testallocateContainerDistributesAllocation() throws Exception {
Assert.assertNotNull(containerInfo);
Assert.assertNotNull(containerInfo.getPipeline());
pipelineList.add(containerInfo.getPipeline().getLeader()
.getDatanodeUuid());
.getUuid());
}
Assert.assertTrue(pipelineList.size() > 5);
}
@ -142,8 +141,8 @@ public void testGetContainer() throws IOException {
containerOwner).getPipeline();
Assert.assertNotNull(pipeline);
Pipeline newPipeline = mapping.getContainer(containerName).getPipeline();
Assert.assertEquals(pipeline.getLeader().getDatanodeUuid(),
newPipeline.getLeader().getDatanodeUuid());
Assert.assertEquals(pipeline.getLeader().getUuid(),
newPipeline.getLeader().getUuid());
}
@Test
@ -209,7 +208,7 @@ public void testContainerCreationLeaseTimeout() throws IOException,
public void testFullContainerReport() throws IOException {
String containerName = UUID.randomUUID().toString();
ContainerInfo info = createContainer(containerName);
DatanodeID datanodeID = getDatanodeID();
DatanodeDetails datanodeDetails = TestUtils.getDatanodeDetails();
ContainerReportsRequestProto.reportType reportType =
ContainerReportsRequestProto.reportType.fullReport;
List<StorageContainerDatanodeProtocolProtos.ContainerInfo> reports =
@ -232,7 +231,7 @@ public void testFullContainerReport() throws IOException {
ContainerReportsRequestProto.Builder crBuilder =
ContainerReportsRequestProto.newBuilder();
crBuilder.setDatanodeID(datanodeID.getProtoBufMessage())
crBuilder.setDatanodeDetails(datanodeDetails.getProtoBufMessage())
.setType(reportType).addAllReports(reports);
mapping.processContainerReports(crBuilder.build());
@ -246,7 +245,7 @@ public void testFullContainerReport() throws IOException {
public void testContainerCloseWithContainerReport() throws IOException {
String containerName = UUID.randomUUID().toString();
ContainerInfo info = createContainer(containerName);
DatanodeID datanodeID = TestUtils.getDatanodeID();
DatanodeDetails datanodeDetails = TestUtils.getDatanodeDetails();
ContainerReportsRequestProto.reportType reportType =
ContainerReportsRequestProto.reportType.fullReport;
List<StorageContainerDatanodeProtocolProtos.ContainerInfo> reports =
@ -270,7 +269,7 @@ public void testContainerCloseWithContainerReport() throws IOException {
ContainerReportsRequestProto.Builder crBuilder =
ContainerReportsRequestProto.newBuilder();
crBuilder.setDatanodeID(datanodeID.getProtoBufMessage())
crBuilder.setDatanodeDetails(datanodeDetails.getProtoBufMessage())
.setType(reportType).addAllReports(reports);
mapping.processContainerReports(crBuilder.build());

View File

@ -21,13 +21,13 @@
import org.apache.commons.lang.RandomStringUtils;
import org.apache.hadoop.conf.Configuration;
import org.apache.hadoop.fs.FileUtil;
import org.apache.hadoop.hdfs.DFSTestUtil;
import org.apache.hadoop.hdfs.protocol.DatanodeID;
import org.apache.hadoop.hdsl.protocol.DatanodeDetails;
import org.apache.hadoop.ozone.OzoneConfigKeys;
import org.apache.hadoop.ozone.container.common.SCMTestUtils;
import org.apache.hadoop.hdsl.protocol.proto.HdslProtos;
import org.apache.hadoop.hdsl.protocol.proto.StorageContainerDatanodeProtocolProtos;
import org.apache.hadoop.hdsl.protocol.proto.StorageContainerDatanodeProtocolProtos.ContainerReportsRequestProto;
import org.apache.hadoop.ozone.scm.TestUtils;
import org.apache.hadoop.ozone.scm.container.ContainerMapping;
import org.apache.hadoop.ozone.scm.container.MockNodeManager;
import org.apache.hadoop.ozone.scm.container.TestContainerMapping;
@ -97,7 +97,7 @@ public void testClose() throws IOException {
long currentCount = mapping.getCloser().getCloseCount();
long runCount = mapping.getCloser().getThreadRunCount();
DatanodeID datanodeID = info.getPipeline().getLeader();
DatanodeDetails datanode = info.getPipeline().getLeader();
// Send a container report with used set to 1 GB. This should not close.
sendContainerReport(info, 1 * GIGABYTE);
@ -108,7 +108,7 @@ public void testClose() throws IOException {
Assert.assertEquals(0, mapping.getCloser().getCloseCount());
// Assert that the Close command was not queued for this Datanode.
Assert.assertEquals(0, nodeManager.getCommandCount(datanodeID));
Assert.assertEquals(0, nodeManager.getCommandCount(datanode));
long newUsed = (long) (size * 0.91f);
sendContainerReport(info, newUsed);
@ -121,7 +121,7 @@ public void testClose() throws IOException {
mapping.getCloser().getCloseCount() - currentCount);
// Assert that the Close command was Queued for this Datanode.
Assert.assertEquals(1, nodeManager.getCommandCount(datanodeID));
Assert.assertEquals(1, nodeManager.getCommandCount(datanode));
}
@Test
@ -146,7 +146,7 @@ public void testRepeatedClose() throws IOException,
long runCount = mapping.getCloser().getThreadRunCount();
DatanodeID datanodeID = info.getPipeline().getLeader();
DatanodeDetails datanodeDetails = info.getPipeline().getLeader();
// Send this command twice and assert we have only one command in the queue.
sendContainerReport(info, 5 * GIGABYTE);
@ -154,7 +154,7 @@ public void testRepeatedClose() throws IOException,
// Assert that the Close command was Queued for this Datanode.
Assert.assertEquals(1,
nodeManager.getCommandCount(datanodeID));
nodeManager.getCommandCount(datanodeDetails));
// And close count will be one.
Assert.assertEquals(1,
mapping.getCloser().getCloseCount() - currentCount);
@ -163,7 +163,7 @@ public void testRepeatedClose() throws IOException,
//send another close and the system will queue this to the command queue.
sendContainerReport(info, 5 * GIGABYTE);
Assert.assertEquals(2,
nodeManager.getCommandCount(datanodeID));
nodeManager.getCommandCount(datanodeDetails));
// but the close count will still be one, since from the point of view of
// closer we are closing only one container even if we have send multiple
// close commands to the datanode.
@ -213,8 +213,8 @@ private void sendContainerReport(ContainerInfo info, long used) throws
.setReadBytes(2000000000L)
.setWriteBytes(2000000000L)
.setContainerID(1L);
reports.setDatanodeID(
DFSTestUtil.getLocalDatanodeID().getProtoBufMessage());
reports.setDatanodeDetails(
TestUtils.getDatanodeDetails().getProtoBufMessage());
reports.addReports(ciBuilder);
mapping.processContainerReports(reports.build());
}

View File

@ -21,11 +21,10 @@
import org.apache.commons.io.IOUtils;
import org.apache.hadoop.conf.Configuration;
import org.apache.hadoop.fs.FileUtil;
import org.apache.hadoop.hdfs.protocol.DatanodeID;
import org.apache.hadoop.hdsl.protocol.DatanodeDetails;
import org.apache.hadoop.ozone.OzoneConfigKeys;
import org.apache.hadoop.hdsl.conf.OzoneConfiguration;
import org.apache.hadoop.ozone.OzoneConsts;
import org.apache.hadoop.ozone.container.common.SCMTestUtils;
import org.apache.hadoop.hdsl.protocol.proto
.StorageContainerDatanodeProtocolProtos.ReportState;
import org.apache.hadoop.hdsl.protocol.proto
@ -132,16 +131,16 @@ public void testContainerPlacementCapacity() throws IOException,
SCMNodeManager nodeManager = createNodeManager(conf);
ContainerMapping containerManager =
createContainerManager(conf, nodeManager);
List<DatanodeID> datanodes =
TestUtils.getRegisteredDatanodeIDs(nodeManager, nodeCount);
List<DatanodeDetails> datanodes =
TestUtils.getListOfRegisteredDatanodeDetails(nodeManager, nodeCount);
try {
for (DatanodeID datanodeID : datanodes) {
for (DatanodeDetails datanodeDetails : datanodes) {
SCMNodeReport.Builder nrb = SCMNodeReport.newBuilder();
SCMStorageReport.Builder srb = SCMStorageReport.newBuilder();
srb.setStorageUuid(UUID.randomUUID().toString());
srb.setCapacity(capacity).setScmUsed(used).
setRemaining(remaining).build();
nodeManager.sendHeartbeat(datanodeID,
nodeManager.sendHeartbeat(datanodeDetails.getProtoBufMessage(),
nrb.addStorageReport(srb).build(), reportState);
}

View File

@ -20,10 +20,9 @@
import com.google.common.base.Supplier;
import static java.util.concurrent.TimeUnit.*;
import org.apache.hadoop.fs.FileUtil;
import org.apache.hadoop.hdfs.protocol.DatanodeID;
import org.apache.hadoop.hdsl.protocol.DatanodeDetails;
import org.apache.hadoop.ozone.OzoneConfigKeys;
import org.apache.hadoop.hdsl.conf.OzoneConfiguration;
import org.apache.hadoop.ozone.container.common.SCMTestUtils;
import org.apache.hadoop.ozone.protocol.commands.SCMCommand;
import org.apache.hadoop.hdsl.protocol.proto.HdslProtos;
import org.apache.hadoop.hdsl.protocol.proto
@ -38,7 +37,6 @@
import org.apache.hadoop.test.GenericTestUtils;
import org.apache.hadoop.test.PathUtils;
import static org.apache.hadoop.ozone.scm.TestUtils.getDatanodeID;
import org.hamcrest.CoreMatchers;
import org.junit.Assert;
import org.junit.After;
@ -69,7 +67,8 @@
import static org.apache.hadoop.scm.ScmConfigKeys.OZONE_SCM_HEARTBEAT_INTERVAL;
import static org.apache.hadoop.scm.ScmConfigKeys
.OZONE_SCM_HEARTBEAT_PROCESS_INTERVAL;
import static org.apache.hadoop.scm.ScmConfigKeys.OZONE_SCM_MAX_HB_COUNT_TO_PROCESS;
import static org.apache.hadoop.scm.ScmConfigKeys
.OZONE_SCM_MAX_HB_COUNT_TO_PROCESS;
import static org.apache.hadoop.scm.ScmConfigKeys.OZONE_SCM_STALENODE_INTERVAL;
import static org.hamcrest.CoreMatchers.containsString;
import static org.hamcrest.MatcherAssert.assertThat;
@ -153,8 +152,10 @@ public void testScmHeartbeat() throws IOException,
try (SCMNodeManager nodeManager = createNodeManager(getConf())) {
// Send some heartbeats from different nodes.
for (int x = 0; x < nodeManager.getMinimumChillModeNodes(); x++) {
DatanodeID datanodeID = getDatanodeID(nodeManager);
nodeManager.sendHeartbeat(datanodeID, null, reportState);
DatanodeDetails datanodeDetails = TestUtils.getDatanodeDetails(
nodeManager);
nodeManager.sendHeartbeat(datanodeDetails.getProtoBufMessage(),
null, reportState);
}
// Wait for 4 seconds max.
@ -200,7 +201,8 @@ public void testScmNotEnoughHeartbeats() throws IOException,
// Need 100 nodes to come out of chill mode, only one node is sending HB.
nodeManager.setMinimumChillModeNodes(100);
nodeManager.sendHeartbeat(TestUtils.getDatanodeID(nodeManager),
nodeManager.sendHeartbeat(TestUtils.getDatanodeDetails(nodeManager)
.getProtoBufMessage(),
null, reportState);
GenericTestUtils.waitFor(() -> nodeManager.waitForHeartbeatProcessed(),
100, 4 * 1000);
@ -223,11 +225,13 @@ public void testScmSameNodeHeartbeats() throws IOException,
try (SCMNodeManager nodeManager = createNodeManager(getConf())) {
nodeManager.setMinimumChillModeNodes(3);
DatanodeID datanodeID = TestUtils.getDatanodeID(nodeManager);
DatanodeDetails datanodeDetails = TestUtils
.getDatanodeDetails(nodeManager);
// Send 10 heartbeat from same node, and assert we never leave chill mode.
for (int x = 0; x < 10; x++) {
nodeManager.sendHeartbeat(datanodeID, null, reportState);
nodeManager.sendHeartbeat(datanodeDetails.getProtoBufMessage(),
null, reportState);
}
GenericTestUtils.waitFor(() -> nodeManager.waitForHeartbeatProcessed(),
@ -253,11 +257,12 @@ public void testScmShutdown() throws IOException, InterruptedException,
conf.getTimeDuration(ScmConfigKeys.OZONE_SCM_HEARTBEAT_PROCESS_INTERVAL,
100, TimeUnit.MILLISECONDS);
SCMNodeManager nodeManager = createNodeManager(conf);
DatanodeID datanodeID = TestUtils.getDatanodeID(nodeManager);
DatanodeDetails datanodeDetails = TestUtils.getDatanodeDetails(nodeManager);
nodeManager.close();
// These should never be processed.
nodeManager.sendHeartbeat(datanodeID, null, reportState);
nodeManager.sendHeartbeat(datanodeDetails.getProtoBufMessage(),
null, reportState);
// Let us just wait for 2 seconds to prove that HBs are not processed.
Thread.sleep(2 * 1000);
@ -277,12 +282,13 @@ public void testScmHeartbeatAfterRestart() throws Exception {
OzoneConfiguration conf = getConf();
conf.getTimeDuration(ScmConfigKeys.OZONE_SCM_HEARTBEAT_PROCESS_INTERVAL,
100, TimeUnit.MILLISECONDS);
DatanodeID datanodeID = TestUtils.getDatanodeID();
DatanodeDetails datanodeDetails = TestUtils.getDatanodeDetails();
try (SCMNodeManager nodemanager = createNodeManager(conf)) {
nodemanager.register(datanodeID);
List<SCMCommand> command = nodemanager.sendHeartbeat(datanodeID,
nodemanager.register(datanodeDetails.getProtoBufMessage());
List<SCMCommand> command = nodemanager.sendHeartbeat(
datanodeDetails.getProtoBufMessage(),
null, reportState);
Assert.assertTrue(nodemanager.getAllNodes().contains(datanodeID));
Assert.assertTrue(nodemanager.getAllNodes().contains(datanodeDetails));
Assert.assertTrue("On regular HB calls, SCM responses a "
+ "datanode with an empty command list", command.isEmpty());
}
@ -291,7 +297,7 @@ public void testScmHeartbeatAfterRestart() throws Exception {
// This happens when SCM restarts.
try (SCMNodeManager nodemanager = createNodeManager(conf)) {
Assert.assertFalse(nodemanager
.getAllNodes().contains(datanodeID));
.getAllNodes().contains(datanodeDetails));
try {
// SCM handles heartbeat asynchronously.
// It may need more than one heartbeat processing to
@ -299,8 +305,8 @@ public void testScmHeartbeatAfterRestart() throws Exception {
GenericTestUtils.waitFor(new Supplier<Boolean>() {
@Override public Boolean get() {
List<SCMCommand> command =
nodemanager.sendHeartbeat(datanodeID, null,
reportState);
nodemanager.sendHeartbeat(datanodeDetails.getProtoBufMessage(),
null, reportState);
return command.size() == 1 && command.get(0).getType()
.equals(SCMCmdType.reregisterCommand);
}
@ -329,8 +335,10 @@ public void testScmHealthyNodeCount() throws IOException,
try (SCMNodeManager nodeManager = createNodeManager(conf)) {
for (int x = 0; x < count; x++) {
DatanodeID datanodeID = TestUtils.getDatanodeID(nodeManager);
nodeManager.sendHeartbeat(datanodeID, null, reportState);
DatanodeDetails datanodeDetails = TestUtils.getDatanodeDetails(
nodeManager);
nodeManager.sendHeartbeat(datanodeDetails.getProtoBufMessage(),
null, reportState);
}
GenericTestUtils.waitFor(() -> nodeManager.waitForHeartbeatProcessed(),
100, 4 * 1000);
@ -415,41 +423,42 @@ public void testScmDetectStaleAndDeadNode() throws IOException,
try (SCMNodeManager nodeManager = createNodeManager(conf)) {
List<DatanodeID> nodeList = createNodeSet(nodeManager, nodeCount,
List<DatanodeDetails> nodeList = createNodeSet(nodeManager, nodeCount,
"Node");
DatanodeID staleNode = TestUtils.getDatanodeID(nodeManager);
DatanodeDetails staleNode = TestUtils.getDatanodeDetails(nodeManager);
// Heartbeat once
nodeManager.sendHeartbeat(staleNode, null, reportState);
nodeManager.sendHeartbeat(staleNode.getProtoBufMessage(),
null, reportState);
// Heartbeat all other nodes.
for (DatanodeID dn : nodeList) {
nodeManager.sendHeartbeat(dn, null, reportState);
for (DatanodeDetails dn : nodeList) {
nodeManager.sendHeartbeat(dn.getProtoBufMessage(), null, reportState);
}
// Wait for 2 seconds .. and heartbeat good nodes again.
Thread.sleep(2 * 1000);
for (DatanodeID dn : nodeList) {
nodeManager.sendHeartbeat(dn, null, reportState);
for (DatanodeDetails dn : nodeList) {
nodeManager.sendHeartbeat(dn.getProtoBufMessage(), null, reportState);
}
// Wait for 2 seconds, wait a total of 4 seconds to make sure that the
// node moves into stale state.
Thread.sleep(2 * 1000);
List<DatanodeID> staleNodeList = nodeManager.getNodes(STALE);
List<DatanodeDetails> staleNodeList = nodeManager.getNodes(STALE);
assertEquals("Expected to find 1 stale node",
1, nodeManager.getNodeCount(STALE));
assertEquals("Expected to find 1 stale node",
1, staleNodeList.size());
assertEquals("Stale node is not the expected ID", staleNode
.getDatanodeUuid(), staleNodeList.get(0).getDatanodeUuid());
.getUuid(), staleNodeList.get(0).getUuid());
Thread.sleep(1000);
// heartbeat good nodes again.
for (DatanodeID dn : nodeList) {
nodeManager.sendHeartbeat(dn, null, reportState);
for (DatanodeDetails dn : nodeList) {
nodeManager.sendHeartbeat(dn.getProtoBufMessage(), null, reportState);
}
// 6 seconds is the dead window for this test , so we wait a total of
@ -464,13 +473,13 @@ public void testScmDetectStaleAndDeadNode() throws IOException,
0, staleNodeList.size());
// Check for the dead node now.
List<DatanodeID> deadNodeList = nodeManager.getNodes(DEAD);
List<DatanodeDetails> deadNodeList = nodeManager.getNodes(DEAD);
assertEquals("Expected to find 1 dead node", 1,
nodeManager.getNodeCount(DEAD));
assertEquals("Expected to find 1 dead node",
1, deadNodeList.size());
assertEquals("Dead node is not the expected ID", staleNode
.getDatanodeUuid(), deadNodeList.get(0).getDatanodeUuid());
.getUuid(), deadNodeList.get(0).getUuid());
}
}
@ -558,15 +567,18 @@ public void testScmClusterIsInExpectedState1() throws IOException,
* Cluster state: Healthy: All nodes are heartbeat-ing like normal.
*/
try (SCMNodeManager nodeManager = createNodeManager(conf)) {
DatanodeID healthyNode =
TestUtils.getDatanodeID(nodeManager, "HealthyNode");
DatanodeID staleNode =
TestUtils.getDatanodeID(nodeManager, "StaleNode");
DatanodeID deadNode =
TestUtils.getDatanodeID(nodeManager, "DeadNode");
nodeManager.sendHeartbeat(healthyNode, null, reportState);
nodeManager.sendHeartbeat(staleNode, null, reportState);
nodeManager.sendHeartbeat(deadNode, null, reportState);
DatanodeDetails healthyNode =
TestUtils.getDatanodeDetails(nodeManager, "HealthyNode");
DatanodeDetails staleNode =
TestUtils.getDatanodeDetails(nodeManager, "StaleNode");
DatanodeDetails deadNode =
TestUtils.getDatanodeDetails(nodeManager, "DeadNode");
nodeManager.sendHeartbeat(
healthyNode.getProtoBufMessage(), null, reportState);
nodeManager.sendHeartbeat(
staleNode.getProtoBufMessage(), null, reportState);
nodeManager.sendHeartbeat(
deadNode.getProtoBufMessage(), null, reportState);
// Sleep so that heartbeat processing thread gets to run.
Thread.sleep(500);
@ -592,12 +604,16 @@ public void testScmClusterIsInExpectedState1() throws IOException,
* the 3 second windows.
*/
nodeManager.sendHeartbeat(healthyNode, null, reportState);
nodeManager.sendHeartbeat(staleNode, null, reportState);
nodeManager.sendHeartbeat(deadNode, null, reportState);
nodeManager.sendHeartbeat(
healthyNode.getProtoBufMessage(), null, reportState);
nodeManager.sendHeartbeat(
staleNode.getProtoBufMessage(), null, reportState);
nodeManager.sendHeartbeat(
deadNode.getProtoBufMessage(), null, reportState);
Thread.sleep(1500);
nodeManager.sendHeartbeat(healthyNode, null, reportState);
nodeManager.sendHeartbeat(
healthyNode.getProtoBufMessage(), null, reportState);
Thread.sleep(2 * 1000);
assertEquals(1, nodeManager.getNodeCount(HEALTHY));
@ -605,10 +621,10 @@ public void testScmClusterIsInExpectedState1() throws IOException,
// 3.5 seconds from last heartbeat for the stale and deadNode. So those
// 2 nodes must move to Stale state and the healthy node must
// remain in the healthy State.
List<DatanodeID> healthyList = nodeManager.getNodes(HEALTHY);
List<DatanodeDetails> healthyList = nodeManager.getNodes(HEALTHY);
assertEquals("Expected one healthy node", 1, healthyList.size());
assertEquals("Healthy node is not the expected ID", healthyNode
.getDatanodeUuid(), healthyList.get(0).getDatanodeUuid());
.getUuid(), healthyList.get(0).getUuid());
assertEquals(2, nodeManager.getNodeCount(STALE));
@ -617,18 +633,21 @@ public void testScmClusterIsInExpectedState1() throws IOException,
* staleNode to move to stale state and deadNode to move to dead state.
*/
nodeManager.sendHeartbeat(healthyNode, null, reportState);
nodeManager.sendHeartbeat(staleNode, null, reportState);
nodeManager.sendHeartbeat(
healthyNode.getProtoBufMessage(), null, reportState);
nodeManager.sendHeartbeat(
staleNode.getProtoBufMessage(), null, reportState);
Thread.sleep(1500);
nodeManager.sendHeartbeat(healthyNode, null, reportState);
nodeManager.sendHeartbeat(
healthyNode.getProtoBufMessage(), null, reportState);
Thread.sleep(2 * 1000);
// 3.5 seconds have elapsed for stale node, so it moves into Stale.
// 7 seconds have elapsed for dead node, so it moves into dead.
// 2 Seconds have elapsed for healthy node, so it stays in healhty state.
healthyList = nodeManager.getNodes(HEALTHY);
List<DatanodeID> staleList = nodeManager.getNodes(STALE);
List<DatanodeID> deadList = nodeManager.getNodes(DEAD);
List<DatanodeDetails> staleList = nodeManager.getNodes(STALE);
List<DatanodeDetails> deadList = nodeManager.getNodes(DEAD);
assertEquals(3, nodeManager.getAllNodes().size());
assertEquals(1, nodeManager.getNodeCount(HEALTHY));
@ -638,24 +657,27 @@ public void testScmClusterIsInExpectedState1() throws IOException,
assertEquals("Expected one healthy node",
1, healthyList.size());
assertEquals("Healthy node is not the expected ID", healthyNode
.getDatanodeUuid(), healthyList.get(0).getDatanodeUuid());
.getUuid(), healthyList.get(0).getUuid());
assertEquals("Expected one stale node",
1, staleList.size());
assertEquals("Stale node is not the expected ID", staleNode
.getDatanodeUuid(), staleList.get(0).getDatanodeUuid());
.getUuid(), staleList.get(0).getUuid());
assertEquals("Expected one dead node",
1, deadList.size());
assertEquals("Dead node is not the expected ID", deadNode
.getDatanodeUuid(), deadList.get(0).getDatanodeUuid());
.getUuid(), deadList.get(0).getUuid());
/**
* Cluster State : let us heartbeat all the nodes and verify that we get
* back all the nodes in healthy state.
*/
nodeManager.sendHeartbeat(healthyNode, null, reportState);
nodeManager.sendHeartbeat(staleNode, null, reportState);
nodeManager.sendHeartbeat(deadNode, null, reportState);
nodeManager.sendHeartbeat(
healthyNode.getProtoBufMessage(), null, reportState);
nodeManager.sendHeartbeat(
staleNode.getProtoBufMessage(), null, reportState);
nodeManager.sendHeartbeat(
deadNode.getProtoBufMessage(), null, reportState);
Thread.sleep(500);
//Assert all nodes are healthy.
assertEquals(3, nodeManager.getAllNodes().size());
@ -671,11 +693,12 @@ public void testScmClusterIsInExpectedState1() throws IOException,
* @param sleepDuration - Duration to sleep between heartbeats.
* @throws InterruptedException
*/
private void heartbeatNodeSet(SCMNodeManager manager, List<DatanodeID> list,
private void heartbeatNodeSet(SCMNodeManager manager,
List<DatanodeDetails> list,
int sleepDuration) throws InterruptedException {
while (!Thread.currentThread().isInterrupted()) {
for (DatanodeID dn : list) {
manager.sendHeartbeat(dn, null, reportState);
for (DatanodeDetails dn : list) {
manager.sendHeartbeat(dn.getProtoBufMessage(), null, reportState);
}
Thread.sleep(sleepDuration);
}
@ -688,12 +711,12 @@ private void heartbeatNodeSet(SCMNodeManager manager, List<DatanodeID> list,
* @param prefix - A prefix string that can be used in verification.
* @return List of Nodes.
*/
private List<DatanodeID> createNodeSet(SCMNodeManager nodeManager, int
private List<DatanodeDetails> createNodeSet(SCMNodeManager nodeManager, int
count, String
prefix) {
List<DatanodeID> list = new LinkedList<>();
List<DatanodeDetails> list = new LinkedList<>();
for (int x = 0; x < count; x++) {
list.add(TestUtils.getDatanodeID(nodeManager, prefix + x));
list.add(TestUtils.getDatanodeDetails(nodeManager, prefix + x));
}
return list;
}
@ -734,11 +757,11 @@ public void testScmClusterIsInExpectedState2() throws IOException,
try (SCMNodeManager nodeManager = createNodeManager(conf)) {
List<DatanodeID> healthyNodeList = createNodeSet(nodeManager,
List<DatanodeDetails> healthyNodeList = createNodeSet(nodeManager,
healthyCount, "Healthy");
List<DatanodeID> staleNodeList = createNodeSet(nodeManager, staleCount,
"Stale");
List<DatanodeID> deadNodeList = createNodeSet(nodeManager, deadCount,
List<DatanodeDetails> staleNodeList = createNodeSet(nodeManager,
staleCount, "Stale");
List<DatanodeDetails> deadNodeList = createNodeSet(nodeManager, deadCount,
"Dead");
Runnable healthyNodeTask = () -> {
@ -761,8 +784,8 @@ public void testScmClusterIsInExpectedState2() throws IOException,
// No Thread just one time HBs the node manager, so that these will be
// marked as dead nodes eventually.
for (DatanodeID dn : deadNodeList) {
nodeManager.sendHeartbeat(dn, null, reportState);
for (DatanodeDetails dn : deadNodeList) {
nodeManager.sendHeartbeat(dn.getProtoBufMessage(), null, reportState);
}
@ -784,9 +807,9 @@ public void testScmClusterIsInExpectedState2() throws IOException,
assertEquals(deadCount, nodeManager.getNodeCount(DEAD));
List<DatanodeID> deadList = nodeManager.getNodes(DEAD);
List<DatanodeDetails> deadList = nodeManager.getNodes(DEAD);
for (DatanodeID node : deadList) {
for (DatanodeDetails node : deadList) {
assertThat(node.getHostName(), CoreMatchers.startsWith("Dead"));
}
@ -825,9 +848,10 @@ public void testScmCanHandleScale() throws IOException,
MILLISECONDS);
try (SCMNodeManager nodeManager = createNodeManager(conf)) {
List<DatanodeID> healthyList = createNodeSet(nodeManager,
List<DatanodeDetails> healthyList = createNodeSet(nodeManager,
healthyCount, "h");
List<DatanodeID> staleList = createNodeSet(nodeManager, staleCount, "s");
List<DatanodeDetails> staleList = createNodeSet(nodeManager,
staleCount, "s");
Runnable healthyNodeTask = () -> {
try {
@ -886,8 +910,8 @@ public void testScmLogsHeartbeatFlooding() throws IOException,
conf.setInt(OZONE_SCM_MAX_HB_COUNT_TO_PROCESS, 500);
try (SCMNodeManager nodeManager = createNodeManager(conf)) {
List<DatanodeID> healthyList = createNodeSet(nodeManager, healthyCount,
"h");
List<DatanodeDetails> healthyList = createNodeSet(nodeManager,
healthyCount, "h");
GenericTestUtils.LogCapturer logCapturer =
GenericTestUtils.LogCapturer.captureLogs(SCMNodeManager.LOG);
Runnable healthyNodeTask = () -> {
@ -921,8 +945,10 @@ public void testScmEnterAndExitChillMode() throws IOException,
try (SCMNodeManager nodeManager = createNodeManager(conf)) {
nodeManager.setMinimumChillModeNodes(10);
DatanodeID datanodeID = TestUtils.getDatanodeID(nodeManager);
nodeManager.sendHeartbeat(datanodeID, null, reportState);
DatanodeDetails datanodeDetails = TestUtils.getDatanodeDetails(
nodeManager);
nodeManager.sendHeartbeat(
datanodeDetails.getProtoBufMessage(), null, reportState);
String status = nodeManager.getChillModeStatus();
Assert.assertThat(status, containsString("Still in chill " +
"mode, waiting on nodes to report in."));
@ -948,8 +974,9 @@ public void testScmEnterAndExitChillMode() throws IOException,
// Assert that node manager force enter cannot be overridden by nodes HBs.
for (int x = 0; x < 20; x++) {
DatanodeID datanode = TestUtils.getDatanodeID(nodeManager);
nodeManager.sendHeartbeat(datanode, null, reportState);
DatanodeDetails datanode = TestUtils.getDatanodeDetails(nodeManager);
nodeManager.sendHeartbeat(datanode.getProtoBufMessage(),
null, reportState);
}
Thread.sleep(500);
@ -985,14 +1012,15 @@ public void testScmStatsFromNodeReport() throws IOException,
try (SCMNodeManager nodeManager = createNodeManager(conf)) {
for (int x = 0; x < nodeCount; x++) {
DatanodeID datanodeID = TestUtils.getDatanodeID(nodeManager);
DatanodeDetails datanodeDetails = TestUtils.getDatanodeDetails(
nodeManager);
SCMNodeReport.Builder nrb = SCMNodeReport.newBuilder();
SCMStorageReport.Builder srb = SCMStorageReport.newBuilder();
srb.setStorageUuid(UUID.randomUUID().toString());
srb.setCapacity(capacity).setScmUsed(used).
setRemaining(capacity - used).build();
nodeManager.sendHeartbeat(datanodeID,
nodeManager.sendHeartbeat(datanodeDetails.getProtoBufMessage(),
nrb.addStorageReport(srb).build(), reportState);
}
GenericTestUtils.waitFor(() -> nodeManager.waitForHeartbeatProcessed(),
@ -1029,7 +1057,8 @@ public void testScmNodeReportUpdate() throws IOException,
conf.setTimeDuration(OZONE_SCM_DEADNODE_INTERVAL, 6, SECONDS);
try (SCMNodeManager nodeManager = createNodeManager(conf)) {
DatanodeID datanodeID = TestUtils.getDatanodeID(nodeManager);
DatanodeDetails datanodeDetails = TestUtils.getDatanodeDetails(
nodeManager);
final long capacity = 2000;
final long usedPerHeartbeat = 100;
@ -1041,7 +1070,8 @@ public void testScmNodeReportUpdate() throws IOException,
.setRemaining(capacity - x * usedPerHeartbeat).build();
nrb.addStorageReport(srb);
nodeManager.sendHeartbeat(datanodeID, nrb.build(), reportState);
nodeManager.sendHeartbeat(
datanodeDetails.getProtoBufMessage(), nrb.build(), reportState);
Thread.sleep(100);
}
@ -1063,23 +1093,23 @@ public void testScmNodeReportUpdate() throws IOException,
// Test NodeManager#getNodeStats
assertEquals(nodeCount, nodeManager.getNodeStats().size());
long nodeCapacity = nodeManager.getNodeStat(datanodeID).get()
long nodeCapacity = nodeManager.getNodeStat(datanodeDetails).get()
.getCapacity().get();
assertEquals(capacity, nodeCapacity);
foundScmUsed = nodeManager.getNodeStat(datanodeID).get().getScmUsed()
foundScmUsed = nodeManager.getNodeStat(datanodeDetails).get().getScmUsed()
.get();
assertEquals(expectedScmUsed, foundScmUsed);
foundRemaining = nodeManager.getNodeStat(datanodeID).get()
foundRemaining = nodeManager.getNodeStat(datanodeDetails).get()
.getRemaining().get();
assertEquals(expectedRemaining, foundRemaining);
// Compare the result from
// NodeManager#getNodeStats and NodeManager#getNodeStat
SCMNodeStat stat1 = nodeManager.getNodeStats().
get(datanodeID.getDatanodeUuid());
SCMNodeStat stat2 = nodeManager.getNodeStat(datanodeID).get();
get(datanodeDetails);
SCMNodeStat stat2 = nodeManager.getNodeStat(datanodeDetails).get();
assertEquals(stat1, stat2);
// Wait up to 4s so that the node becomes stale
@ -1089,14 +1119,14 @@ public void testScmNodeReportUpdate() throws IOException,
4 * 1000);
assertEquals(nodeCount, nodeManager.getNodeStats().size());
foundCapacity = nodeManager.getNodeStat(datanodeID).get()
foundCapacity = nodeManager.getNodeStat(datanodeDetails).get()
.getCapacity().get();
assertEquals(capacity, foundCapacity);
foundScmUsed = nodeManager.getNodeStat(datanodeID).get()
foundScmUsed = nodeManager.getNodeStat(datanodeDetails).get()
.getScmUsed().get();
assertEquals(expectedScmUsed, foundScmUsed);
foundRemaining = nodeManager.getNodeStat(datanodeID).get().
foundRemaining = nodeManager.getNodeStat(datanodeDetails).get().
getRemaining().get();
assertEquals(expectedRemaining, foundRemaining);
@ -1123,7 +1153,8 @@ public void testScmNodeReportUpdate() throws IOException,
srb.setCapacity(capacity).setScmUsed(expectedScmUsed)
.setRemaining(expectedRemaining).build();
nrb.addStorageReport(srb);
nodeManager.sendHeartbeat(datanodeID, nrb.build(), reportState);
nodeManager.sendHeartbeat(
datanodeDetails.getProtoBufMessage(), nrb.build(), reportState);
// Wait up to 5 seconds so that the dead node becomes healthy
// Verify usage info should be updated.
@ -1134,13 +1165,13 @@ public void testScmNodeReportUpdate() throws IOException,
() -> nodeManager.getStats().getScmUsed().get() == expectedScmUsed,
100, 4 * 1000);
assertEquals(nodeCount, nodeManager.getNodeStats().size());
foundCapacity = nodeManager.getNodeStat(datanodeID).get()
foundCapacity = nodeManager.getNodeStat(datanodeDetails).get()
.getCapacity().get();
assertEquals(capacity, foundCapacity);
foundScmUsed = nodeManager.getNodeStat(datanodeID).get().getScmUsed()
foundScmUsed = nodeManager.getNodeStat(datanodeDetails).get().getScmUsed()
.get();
assertEquals(expectedScmUsed, foundScmUsed);
foundRemaining = nodeManager.getNodeStat(datanodeID).get()
foundRemaining = nodeManager.getNodeStat(datanodeDetails).get()
.getRemaining().get();
assertEquals(expectedRemaining, foundRemaining);
}

View File

@ -20,16 +20,15 @@
import org.apache.commons.collections.ListUtils;
import org.apache.hadoop.fs.FileUtil;
import org.apache.hadoop.hdfs.protocol.DatanodeID;
import org.apache.hadoop.hdsl.protocol.DatanodeDetails;
import org.apache.hadoop.ozone.OzoneConfigKeys;
import org.apache.hadoop.hdsl.conf.OzoneConfiguration;
import org.apache.hadoop.ozone.container.common.SCMTestUtils;
import org.apache.hadoop.ozone.scm.TestUtils;
import org.apache.hadoop.ozone.scm.container.placement.algorithms.ContainerPlacementPolicy;
import org.apache.hadoop.ozone.scm.container.placement.algorithms.SCMContainerPlacementCapacity;
import org.apache.hadoop.scm.ScmConfigKeys;
import org.apache.hadoop.test.PathUtils;
import static org.apache.hadoop.ozone.scm.TestUtils.getDatanodeIDs;
import org.junit.Rule;
import org.junit.Test;
import org.junit.rules.ExpectedException;
@ -79,21 +78,22 @@ public void testDefaultNodePool() throws IOException {
NodePoolManager npMgr = createNodePoolManager(conf);
final int nodeCount = 4;
final List<DatanodeID> nodes = getDatanodeIDs(nodeCount);
final List<DatanodeDetails> nodes = TestUtils
.getListOfDatanodeDetails(nodeCount);
assertEquals(0, npMgr.getNodePools().size());
for (DatanodeID node: nodes) {
for (DatanodeDetails node: nodes) {
npMgr.addNode(defaultPool, node);
}
List<DatanodeID> nodesRetrieved = npMgr.getNodes(defaultPool);
List<DatanodeDetails> nodesRetrieved = npMgr.getNodes(defaultPool);
assertEquals(nodeCount, nodesRetrieved.size());
assertTwoDatanodeListsEqual(nodes, nodesRetrieved);
DatanodeID nodeRemoved = nodes.remove(2);
DatanodeDetails nodeRemoved = nodes.remove(2);
npMgr.removeNode(defaultPool, nodeRemoved);
List<DatanodeID> nodesAfterRemove = npMgr.getNodes(defaultPool);
List<DatanodeDetails> nodesAfterRemove = npMgr.getNodes(defaultPool);
assertTwoDatanodeListsEqual(nodes, nodesAfterRemove);
List<DatanodeID> nonExistSet = npMgr.getNodes("NonExistSet");
List<DatanodeDetails> nonExistSet = npMgr.getNodes("NonExistSet");
assertEquals(0, nonExistSet.size());
} finally {
FileUtil.fullyDelete(testDir);
@ -111,16 +111,17 @@ public void testDefaultNodePoolReload() throws IOException {
OzoneConfiguration conf = new OzoneConfiguration();
final String defaultPool = "DefaultPool";
final int nodeCount = 4;
final List<DatanodeID> nodes = getDatanodeIDs(nodeCount);
final List<DatanodeDetails> nodes = TestUtils
.getListOfDatanodeDetails(nodeCount);
try {
try {
SCMNodePoolManager npMgr = createNodePoolManager(conf);
assertEquals(0, npMgr.getNodePools().size());
for (DatanodeID node : nodes) {
for (DatanodeDetails node : nodes) {
npMgr.addNode(defaultPool, node);
}
List<DatanodeID> nodesRetrieved = npMgr.getNodes(defaultPool);
List<DatanodeDetails> nodesRetrieved = npMgr.getNodes(defaultPool);
assertEquals(nodeCount, nodesRetrieved.size());
assertTwoDatanodeListsEqual(nodes, nodesRetrieved);
npMgr.close();
@ -132,7 +133,7 @@ public void testDefaultNodePoolReload() throws IOException {
// try reload with a new NodePoolManager instance
try {
SCMNodePoolManager npMgr = createNodePoolManager(conf);
List<DatanodeID> nodesRetrieved = npMgr.getNodes(defaultPool);
List<DatanodeDetails> nodesRetrieved = npMgr.getNodes(defaultPool);
assertEquals(nodeCount, nodesRetrieved.size());
assertTwoDatanodeListsEqual(nodes, nodesRetrieved);
} finally {
@ -148,8 +149,8 @@ public void testDefaultNodePoolReload() throws IOException {
* @param list1 - datanode list 1.
* @param list2 - datanode list 2.
*/
private void assertTwoDatanodeListsEqual(List<DatanodeID> list1,
List<DatanodeID> list2) {
private void assertTwoDatanodeListsEqual(List<DatanodeDetails> list1,
List<DatanodeDetails> list2) {
assertEquals(list1.size(), list2.size());
Collections.sort(list1);
Collections.sort(list2);

View File

@ -23,8 +23,8 @@
import org.apache.commons.cli.Option;
import org.apache.commons.cli.Options;
import org.apache.hadoop.hdsl.protocol.DatanodeDetails;
import org.apache.hadoop.hdsl.protocol.proto.ContainerProtos.ContainerData;
import org.apache.hadoop.hdfs.protocol.DatanodeID;
import org.apache.hadoop.hdsl.protocol.proto.HdslProtos;
import org.apache.hadoop.ozone.scm.cli.OzoneCommandHandler;
import org.apache.hadoop.scm.client.ScmClient;
@ -94,7 +94,7 @@ public void execute(CommandLine cmd) throws IOException {
// Print pipeline of an existing container.
logOut("LeaderID: %s", pipeline.getLeader().getHostName());
String machinesStr = pipeline.getMachines().stream().map(
DatanodeID::getHostName).collect(Collectors.joining(","));
DatanodeDetails::getHostName).collect(Collectors.joining(","));
logOut("Datanodes: [%s]", machinesStr);
}

View File

@ -119,8 +119,8 @@ public static String getHostName() {
* @param conf - Configuration
* @return the path of datanode id as string
*/
public static String getDatanodeIDPath(Configuration conf) {
return HdslUtils.getDatanodeIDPath(conf);
public static String getDatanodeIdFilePath(Configuration conf) {
return HdslUtils.getDatanodeIdFilePath(conf);
}
/**

View File

@ -23,8 +23,10 @@
import org.apache.commons.io.FileUtils;
import org.apache.hadoop.classification.InterfaceAudience;
import org.apache.hadoop.conf.Configuration;
import org.apache.hadoop.hdfs.server.datanode.DataNode;
import org.apache.hadoop.hdsl.conf.OzoneConfiguration;
import org.apache.hadoop.hdfs.MiniDFSCluster;
import org.apache.hadoop.hdsl.protocol.DatanodeDetails;
import org.apache.hadoop.ipc.Client;
import org.apache.hadoop.ipc.RPC;
import org.apache.hadoop.net.NetUtils;
@ -46,6 +48,8 @@
import org.apache.hadoop.test.GenericTestUtils;
import static org.apache.hadoop.hdfs.DFSConfigKeys.DFS_DATANODE_PLUGINS_KEY;
import org.apache.hadoop.util.ServicePlugin;
import org.slf4j.Logger;
import org.slf4j.LoggerFactory;
import org.slf4j.event.Level;
@ -116,6 +120,10 @@ protected void setupDatanodeAddress(
int i, Configuration dnConf, boolean setupHostsFile,
boolean checkDnAddrConf) throws IOException {
super.setupDatanodeAddress(i, dnConf, setupHostsFile, checkDnAddrConf);
String path = GenericTestUtils.getTempPath(
MiniOzoneClassicCluster.class.getSimpleName() + "datanode");
dnConf.setStrings(ScmConfigKeys.OZONE_SCM_DATANODE_ID,
path + "/" + i + "-datanode.id");
setConf(i, dnConf, OzoneConfigKeys.DFS_CONTAINER_RATIS_DATANODE_STORAGE_DIR,
getInstanceStorageDir(i, -1).getCanonicalPath());
String containerMetaDirs = dnConf.get(
@ -233,7 +241,7 @@ public OzoneRestClient createOzoneRestClient() throws OzoneException {
// An Ozone request may originate at any DataNode, so pick one at random.
int dnIndex = new Random().nextInt(getDataNodes().size());
String uri = String.format("http://127.0.0.1:%d",
getDataNodes().get(dnIndex).getDatanodeId().getOzoneRestPort());
getOzoneRestPort(getDataNodes().get(dnIndex)));
LOG.info("Creating Ozone client to DataNode {} with URI {} and user {}",
dnIndex, uri, USER_AUTH);
try {
@ -330,6 +338,20 @@ public void waitForHeartbeatProcessed() throws TimeoutException,
4 * 1000);
}
public static DatanodeDetails getDatanodeDetails(DataNode dataNode) {
DatanodeDetails datanodeDetails = null;
for (ServicePlugin plugin : dataNode.getPlugins()) {
if (plugin instanceof HdslDatanodeService) {
datanodeDetails = ((HdslDatanodeService) plugin).getDatanodeDetails();
}
}
return datanodeDetails;
}
public static int getOzoneRestPort(DataNode dataNode) {
return getDatanodeDetails(dataNode).getOzoneRestPort();
}
/**
* Builder for configuring the MiniOzoneCluster to run.
*/
@ -479,8 +501,8 @@ public MiniOzoneClassicCluster build() throws IOException {
conf.set(KSMConfigKeys.OZONE_KSM_HTTP_ADDRESS_KEY, "127.0.0.1:0");
conf.set(ScmConfigKeys.HDSL_REST_HTTP_ADDRESS_KEY, "127.0.0.1:0");
conf.set(DFS_DATANODE_PLUGINS_KEY,
"org.apache.hadoop.ozone.HdslServerPlugin,"
+ "org.apache.hadoop.ozone.web.ObjectStoreRestPlugin");
"org.apache.hadoop.ozone.web.ObjectStoreRestPlugin," +
"org.apache.hadoop.ozone.HdslDatanodeService");
// Configure KSM and SCM handlers
conf.setInt(ScmConfigKeys.OZONE_SCM_HANDLER_COUNT_KEY, numOfScmHandlers);
@ -538,12 +560,6 @@ private void configScmMetadata() throws IOException {
Files.createDirectories(containerPath);
conf.set(OzoneConfigKeys.OZONE_METADATA_DIRS, scmPath
.toString());
// TODO : Fix this, we need a more generic mechanism to map
// different datanode ID for different datanodes when we have lots of
// datanodes in the cluster.
conf.setStrings(ScmConfigKeys.OZONE_SCM_DATANODE_ID,
scmPath.toString() + "/datanode.id");
}
private void initializeScm() throws IOException {

View File

@ -46,10 +46,10 @@ public static DatanodeStateMachine getStateMachine(DataNode dataNode) {
return findHdslPlugin(dataNode).getDatanodeStateMachine();
}
private static HdslServerPlugin findHdslPlugin(DataNode dataNode) {
private static HdslDatanodeService findHdslPlugin(DataNode dataNode) {
for (ServicePlugin plugin : dataNode.getPlugins()) {
if (plugin instanceof HdslServerPlugin) {
return (HdslServerPlugin) plugin;
if (plugin instanceof HdslDatanodeService) {
return (HdslDatanodeService) plugin;
}
}
throw new IllegalStateException("Can't find the Hdsl server plugin in the"

View File

@ -80,7 +80,8 @@ public void close() {
}
public int getDatanodeOzoneRestPort() {
return cluster.getDataNodes().get(0).getDatanodeId().getOzoneRestPort();
return MiniOzoneClassicCluster.getOzoneRestPort(
cluster.getDataNodes().get(0));
}
}

View File

@ -20,22 +20,23 @@
import org.apache.hadoop.conf.Configuration;
import org.apache.hadoop.hdsl.conf.OzoneConfiguration;
import org.apache.hadoop.hdfs.DFSTestUtil;
import org.apache.hadoop.hdfs.protocol.DatanodeID;
import org.apache.hadoop.hdfs.server.datanode.DataNode;
import org.apache.hadoop.hdsl.protocol.DatanodeDetails;
import org.apache.hadoop.ozone.container.common.SCMTestUtils;
import org.apache.hadoop.ozone.container.common.helpers.ContainerUtils;
import org.apache.hadoop.ozone.container.common.statemachine.DatanodeStateMachine;
import org.apache.hadoop.ozone.container.ozoneimpl.TestOzoneContainer;
import org.apache.hadoop.hdsl.protocol.proto.HdslProtos;
import org.apache.hadoop.ozone.scm.TestUtils;
import org.apache.hadoop.ozone.web.utils.OzoneUtils;
import org.apache.hadoop.scm.ScmConfigKeys;
import org.apache.hadoop.scm.XceiverClient;
import org.apache.hadoop.scm.container.common.helpers.PipelineChannel;
import org.apache.hadoop.scm.container.common.helpers.Pipeline;
import org.apache.hadoop.test.PathUtils;
import org.apache.hadoop.test.TestGenericTestUtils;
import org.apache.hadoop.util.ServicePlugin;
import org.junit.AfterClass;
import org.junit.Assert;
import org.junit.BeforeClass;
import org.junit.Test;
@ -89,30 +90,23 @@ public void testStartMultipleDatanodes() throws Exception {
.numDataNodes(numberOfNodes)
.setHandlerType(OzoneConsts.OZONE_HANDLER_DISTRIBUTED)
.build();
// make sure datanode.id file is correct
File idPath = new File(
conf.get(ScmConfigKeys.OZONE_SCM_DATANODE_ID));
assertTrue(idPath.exists());
List<DatanodeID> ids = ContainerUtils.readDatanodeIDsFrom(idPath);
assertEquals(numberOfNodes, ids.size());
List<DataNode> datanodes = cluster.getDataNodes();
assertEquals(datanodes.size(), numberOfNodes);
assertEquals(numberOfNodes, datanodes.size());
for(DataNode dn : datanodes) {
// Each datanode ID should match an entry in the ID file
assertTrue("Datanode ID not found in ID file",
ids.contains(dn.getDatanodeId()));
// Create a single member pipe line
String containerName = OzoneUtils.getRequestID();
DatanodeID dnId = dn.getDatanodeId();
DatanodeDetails datanodeDetails = null;
for (ServicePlugin plugin : dn.getPlugins()) {
if (plugin instanceof HdslDatanodeService) {
datanodeDetails = ((HdslDatanodeService) plugin).getDatanodeDetails();
}
}
final PipelineChannel pipelineChannel =
new PipelineChannel(dnId.getDatanodeUuid(),
new PipelineChannel(datanodeDetails.getUuidString(),
HdslProtos.LifeCycleState.OPEN,
HdslProtos.ReplicationType.STAND_ALONE,
HdslProtos.ReplicationFactor.ONE, "test");
pipelineChannel.addMember(dnId);
pipelineChannel.addMember(datanodeDetails);
Pipeline pipeline = new Pipeline(containerName, pipelineChannel);
// Verify client is able to connect to the container
@ -126,9 +120,9 @@ public void testStartMultipleDatanodes() throws Exception {
@Test
public void testDatanodeIDPersistent() throws Exception {
// Generate IDs for testing
DatanodeID id1 = DFSTestUtil.getLocalDatanodeID(1);
DatanodeID id2 = DFSTestUtil.getLocalDatanodeID(2);
DatanodeID id3 = DFSTestUtil.getLocalDatanodeID(3);
DatanodeDetails id1 = TestUtils.getDatanodeDetails();
DatanodeDetails id2 = TestUtils.getDatanodeDetails();
DatanodeDetails id3 = TestUtils.getDatanodeDetails();
id1.setContainerPort(1);
id2.setContainerPort(2);
id3.setContainerPort(3);
@ -136,51 +130,37 @@ public void testDatanodeIDPersistent() throws Exception {
// Write a single ID to the file and read it out
File validIdsFile = new File(WRITE_TMP, "valid-values.id");
validIdsFile.delete();
ContainerUtils.writeDatanodeIDTo(id1, validIdsFile);
List<DatanodeID> validIds = ContainerUtils
.readDatanodeIDsFrom(validIdsFile);
assertEquals(1, validIds.size());
DatanodeID id11 = validIds.iterator().next();
assertEquals(id11, id1);
assertEquals(id11.getProtoBufMessage(), id1.getProtoBufMessage());
ContainerUtils.writeDatanodeDetailsTo(id1, validIdsFile);
DatanodeDetails validId = ContainerUtils.readDatanodeDetailsFrom(
validIdsFile);
// Write should avoid duplicate entries
File noDupIDFile = new File(WRITE_TMP, "no-dup-values.id");
noDupIDFile.delete();
ContainerUtils.writeDatanodeIDTo(id1, noDupIDFile);
ContainerUtils.writeDatanodeIDTo(id1, noDupIDFile);
ContainerUtils.writeDatanodeIDTo(id1, noDupIDFile);
ContainerUtils.writeDatanodeIDTo(id2, noDupIDFile);
ContainerUtils.writeDatanodeIDTo(id3, noDupIDFile);
List<DatanodeID> noDupIDs =ContainerUtils
.readDatanodeIDsFrom(noDupIDFile);
assertEquals(3, noDupIDs.size());
assertTrue(noDupIDs.contains(id1));
assertTrue(noDupIDs.contains(id2));
assertTrue(noDupIDs.contains(id3));
assertEquals(id1, validId);
assertEquals(id1.getProtoBufMessage(), validId.getProtoBufMessage());
// Write should fail if unable to create file or directory
File invalidPath = new File(WRITE_TMP, "an/invalid/path");
try {
ContainerUtils.writeDatanodeIDTo(id1, invalidPath);
ContainerUtils.writeDatanodeDetailsTo(id1, invalidPath);
Assert.fail();
} catch (Exception e) {
e.printStackTrace();
assertTrue(e instanceof IOException);
}
// Read should return an empty value if file doesn't exist
File nonExistFile = new File(READ_TMP, "non_exist.id");
nonExistFile.delete();
List<DatanodeID> emptyIDs =
ContainerUtils.readDatanodeIDsFrom(nonExistFile);
assertTrue(emptyIDs.isEmpty());
try {
ContainerUtils.readDatanodeDetailsFrom(nonExistFile);
Assert.fail();
} catch (Exception e) {
assertTrue(e instanceof IOException);
}
// Read should fail if the file is malformed
File malformedFile = new File(READ_TMP, "malformed.id");
createMalformedIDFile(malformedFile);
try {
ContainerUtils.readDatanodeIDsFrom(malformedFile);
ContainerUtils.readDatanodeDetailsFrom(malformedFile);
fail("Read a malformed ID file should fail");
} catch (Exception e) {
assertTrue(e instanceof IOException);
@ -202,11 +182,11 @@ public void testContainerRandomPort() throws IOException {
true);
try (
DatanodeStateMachine sm1 = new DatanodeStateMachine(
DFSTestUtil.getLocalDatanodeID(), ozoneConf);
TestUtils.getDatanodeDetails(), ozoneConf);
DatanodeStateMachine sm2 = new DatanodeStateMachine(
DFSTestUtil.getLocalDatanodeID(), ozoneConf);
TestUtils.getDatanodeDetails(), ozoneConf);
DatanodeStateMachine sm3 = new DatanodeStateMachine(
DFSTestUtil.getLocalDatanodeID(), ozoneConf);
TestUtils.getDatanodeDetails(), ozoneConf)
) {
HashSet<Integer> ports = new HashSet<Integer>();
assertTrue(ports.add(sm1.getContainer().getContainerServerPort()));
@ -225,11 +205,11 @@ public void testContainerRandomPort() throws IOException {
ozoneConf.setBoolean(OzoneConfigKeys.DFS_CONTAINER_IPC_RANDOM_PORT, false);
try (
DatanodeStateMachine sm1 = new DatanodeStateMachine(
DFSTestUtil.getLocalDatanodeID(), ozoneConf);
TestUtils.getDatanodeDetails(), ozoneConf);
DatanodeStateMachine sm2 = new DatanodeStateMachine(
DFSTestUtil.getLocalDatanodeID(), ozoneConf);
TestUtils.getDatanodeDetails(), ozoneConf);
DatanodeStateMachine sm3 = new DatanodeStateMachine(
DFSTestUtil.getLocalDatanodeID(), ozoneConf);
TestUtils.getDatanodeDetails(), ozoneConf)
) {
HashSet<Integer> ports = new HashSet<Integer>();
assertTrue(ports.add(sm1.getContainer().getContainerServerPort()));
@ -244,8 +224,8 @@ public void testContainerRandomPort() throws IOException {
private void createMalformedIDFile(File malformedFile)
throws IOException{
malformedFile.delete();
DatanodeID id1 = DFSTestUtil.getLocalDatanodeID(1);
ContainerUtils.writeDatanodeIDTo(id1, malformedFile);
DatanodeDetails id = TestUtils.getDatanodeDetails();
ContainerUtils.writeDatanodeDetailsTo(id, malformedFile);
FileOutputStream out = new FileOutputStream(malformedFile);
out.write("malformed".getBytes());

View File

@ -309,7 +309,8 @@ public void testBlockDeletingThrottling() throws Exception {
ReportState reportState = ReportState.newBuilder()
.setState(ReportState.states.noContainerReports).setCount(0).build();
List<SCMCommand> commands = nodeManager.sendHeartbeat(
nodeManager.getNodes(NodeState.HEALTHY).get(0), null, reportState);
nodeManager.getNodes(NodeState.HEALTHY).get(0).getProtoBufMessage(),
null, reportState);
if (commands != null) {
for (SCMCommand cmd : commands) {

View File

@ -23,9 +23,9 @@
import org.apache.hadoop.conf.Configuration;
import org.apache.hadoop.fs.StorageType;
import org.apache.hadoop.hdfs.DFSUtil;
import org.apache.hadoop.hdfs.protocol.DatanodeID;
import org.apache.hadoop.hdfs.server.datanode.DataNode;
import org.apache.hadoop.hdfs.server.datanode.ObjectStoreHandler;
import org.apache.hadoop.hdsl.protocol.DatanodeDetails;
import org.apache.hadoop.ozone.container.common.helpers.ContainerData;
import org.apache.hadoop.ozone.container.common.helpers.KeyUtils;
import org.apache.hadoop.ozone.container.ozoneimpl.OzoneContainer;
@ -159,18 +159,19 @@ private MetadataStore getContainerMetadata(String containerName)
throws IOException {
Pipeline pipeline = cluster.getStorageContainerManager()
.getContainer(containerName);
DatanodeID leadDN = pipeline.getLeader();
DatanodeDetails leadDN = pipeline.getLeader();
OzoneContainer containerServer =
getContainerServerByDatanodeID(leadDN.getDatanodeUuid());
getContainerServerByDatanodeUuid(leadDN.getUuidString());
ContainerData containerData = containerServer.getContainerManager()
.readContainer(containerName);
return KeyUtils.getDB(containerData, conf);
}
private OzoneContainer getContainerServerByDatanodeID(String dnUUID)
private OzoneContainer getContainerServerByDatanodeUuid(String dnUUID)
throws IOException {
for (DataNode dn : cluster.getDataNodes()) {
if (dn.getDatanodeId().getDatanodeUuid().equals(dnUUID)) {
if (MiniOzoneClassicCluster.getDatanodeDetails(dn).getUuidString()
.equals(dnUUID)) {
return MiniOzoneTestHelper.getOzoneContainer(dn);
}
}

View File

@ -21,6 +21,7 @@
import com.google.common.base.Preconditions;
import com.google.protobuf.ByteString;
import org.apache.commons.codec.binary.Hex;
import org.apache.hadoop.hdsl.protocol.DatanodeDetails;
import org.apache.hadoop.hdsl.protocol.proto.ContainerProtos;
import org.apache.hadoop.hdsl.protocol.proto.ContainerProtos
.ContainerCommandRequestProto;
@ -30,7 +31,6 @@
import org.apache.hadoop.hdsl.protocol.proto.HdslProtos.ReplicationFactor;
import org.apache.hadoop.hdsl.protocol.proto.HdslProtos.ReplicationType;
import org.apache.hadoop.hdsl.protocol.proto.HdslProtos.KeyValue;
import org.apache.hadoop.hdfs.protocol.DatanodeID;
import org.apache.hadoop.ozone.OzoneConfigKeys;
import org.apache.hadoop.hdsl.conf.OzoneConfiguration;
import org.apache.hadoop.ozone.OzoneConsts;
@ -89,15 +89,22 @@ public static String createLocalAddress() throws IOException {
return "127.0.0.1:" + s.getLocalPort();
}
}
public static DatanodeID createDatanodeID() throws IOException {
public static DatanodeDetails createDatanodeDetails() throws IOException {
ServerSocket socket = new ServerSocket(0);
int port = socket.getLocalPort();
DatanodeID datanodeID = new DatanodeID(socket.getInetAddress()
.getHostAddress(), socket.getInetAddress().getHostName(),
UUID.randomUUID().toString(), port, port, port, port);
datanodeID.setContainerPort(port);
DatanodeDetails datanodeDetails = DatanodeDetails.newBuilder()
.setUuid(UUID.randomUUID().toString())
.setIpAddress(socket.getInetAddress().getHostAddress())
.setHostName(socket.getInetAddress().getHostName())
.setInfoPort(port)
.setInfoSecurePort(port)
.setContainerPort(port)
.setRatisPort(port)
.setOzoneRestPort(port)
.build();
socket.close();
return datanodeID;
return datanodeDetails;
}
/**
@ -109,23 +116,23 @@ public static DatanodeID createDatanodeID() throws IOException {
public static Pipeline createPipeline(String containerName, int numNodes)
throws IOException {
Preconditions.checkArgument(numNodes >= 1);
final List<DatanodeID> ids = new ArrayList<>(numNodes);
final List<DatanodeDetails> ids = new ArrayList<>(numNodes);
for(int i = 0; i < numNodes; i++) {
ids.add(createDatanodeID());
ids.add(createDatanodeDetails());
}
return createPipeline(containerName, ids);
}
public static Pipeline createPipeline(
String containerName, Iterable<DatanodeID> ids)
String containerName, Iterable<DatanodeDetails> ids)
throws IOException {
Objects.requireNonNull(ids, "ids == null");
final Iterator<DatanodeID> i = ids.iterator();
final Iterator<DatanodeDetails> i = ids.iterator();
Preconditions.checkArgument(i.hasNext());
final DatanodeID leader = i.next();
final DatanodeDetails leader = i.next();
String pipelineName = "TEST-" + UUID.randomUUID().toString().substring(3);
final PipelineChannel pipelineChannel =
new PipelineChannel(leader.getDatanodeUuid(), LifeCycleState.OPEN,
new PipelineChannel(leader.getUuidString(), LifeCycleState.OPEN,
ReplicationType.STAND_ALONE, ReplicationFactor.ONE, pipelineName);
pipelineChannel.addMember(leader);
for(; i.hasNext();) {
@ -213,7 +220,7 @@ public static ContainerCommandRequestProto getWriteChunkRequest(
request.setCmdType(ContainerProtos.Type.WriteChunk);
request.setWriteChunk(writeRequest);
request.setTraceID(UUID.randomUUID().toString());
request.setDatanodeID(newPipeline.getLeader().getDatanodeUuid());
request.setDatanodeUuid(newPipeline.getLeader().getUuidString());
return request.build();
}
@ -259,7 +266,7 @@ public static ContainerCommandRequestProto getWriteSmallFileRequest(
request.setCmdType(ContainerProtos.Type.PutSmallFile);
request.setPutSmallFile(smallFileRequest);
request.setTraceID(UUID.randomUUID().toString());
request.setDatanodeID(newPipeline.getLeader().getDatanodeUuid());
request.setDatanodeUuid(newPipeline.getLeader().getUuidString());
return request.build();
}
@ -278,7 +285,7 @@ public static ContainerCommandRequestProto getReadSmallFileRequest(
request.setCmdType(ContainerProtos.Type.GetSmallFile);
request.setGetSmallFile(smallFileRequest);
request.setTraceID(UUID.randomUUID().toString());
request.setDatanodeID(pipeline.getLeader().getDatanodeUuid());
request.setDatanodeUuid(pipeline.getLeader().getUuidString());
return request.build();
}
@ -309,7 +316,7 @@ public static ContainerCommandRequestProto getReadChunkRequest(
newRequest.setCmdType(ContainerProtos.Type.ReadChunk);
newRequest.setReadChunk(readRequest);
newRequest.setTraceID(UUID.randomUUID().toString());
newRequest.setDatanodeID(pipeline.getLeader().getDatanodeUuid());
newRequest.setDatanodeUuid(pipeline.getLeader().getUuidString());
return newRequest.build();
}
@ -341,7 +348,7 @@ public static ContainerCommandRequestProto getDeleteChunkRequest(
request.setCmdType(ContainerProtos.Type.DeleteChunk);
request.setDeleteChunk(deleteRequest);
request.setTraceID(UUID.randomUUID().toString());
request.setDatanodeID(pipeline.getLeader().getDatanodeUuid());
request.setDatanodeUuid(pipeline.getLeader().getUuidString());
return request.build();
}
@ -371,7 +378,7 @@ public static ContainerCommandRequestProto getCreateContainerRequest(
request.setCmdType(ContainerProtos.Type.CreateContainer);
request.setCreateContainer(createRequest);
request.setTraceID(UUID.randomUUID().toString());
request.setDatanodeID(pipeline.getLeader().getDatanodeUuid().toString());
request.setDatanodeUuid(pipeline.getLeader().getUuidString());
return request.build();
}
@ -410,7 +417,7 @@ public static ContainerCommandRequestProto getUpdateContainerRequest(
request.setCmdType(ContainerProtos.Type.UpdateContainer);
request.setUpdateContainer(updateRequestBuilder.build());
request.setTraceID(UUID.randomUUID().toString());
request.setDatanodeID(pipeline.getLeader().getDatanodeUuid());
request.setDatanodeUuid(pipeline.getLeader().getUuidString());
return request.build();
}
/**
@ -461,7 +468,7 @@ public static ContainerCommandRequestProto getPutKeyRequest(
request.setCmdType(ContainerProtos.Type.PutKey);
request.setPutKey(putRequest);
request.setTraceID(UUID.randomUUID().toString());
request.setDatanodeID(pipeline.getLeader().getDatanodeUuid());
request.setDatanodeUuid(pipeline.getLeader().getUuidString());
return request.build();
}
@ -491,7 +498,7 @@ public static ContainerCommandRequestProto getKeyRequest(
request.setCmdType(ContainerProtos.Type.GetKey);
request.setGetKey(getRequest);
request.setTraceID(UUID.randomUUID().toString());
request.setDatanodeID(pipeline.getLeader().getDatanodeUuid());
request.setDatanodeUuid(pipeline.getLeader().getUuidString());
return request.build();
}
@ -529,7 +536,7 @@ public static ContainerCommandRequestProto getDeleteKeyRequest(
request.setCmdType(ContainerProtos.Type.DeleteKey);
request.setDeleteKey(delRequest);
request.setTraceID(UUID.randomUUID().toString());
request.setDatanodeID(pipeline.getLeader().getDatanodeUuid());
request.setDatanodeUuid(pipeline.getLeader().getUuidString());
return request.build();
}
@ -548,7 +555,7 @@ public static ContainerCommandRequestProto getCloseContainer(
ContainerCommandRequestProto.newBuilder().setCmdType(ContainerProtos
.Type.CloseContainer).setCloseContainer(closeRequest)
.setTraceID(UUID.randomUUID().toString())
.setDatanodeID(pipeline.getLeader().getDatanodeUuid())
.setDatanodeUuid(pipeline.getLeader().getUuidString())
.build();
return cmd;
@ -568,7 +575,8 @@ public static ContainerCommandRequestProto getRequestWithoutTraceId(
ContainerProtos.ContainerCommandRequestProto cmd =
ContainerCommandRequestProto.newBuilder().setCmdType(ContainerProtos
.Type.CloseContainer).setCloseContainer(closeRequest)
.setDatanodeID(pipeline.getLeader().getDatanodeUuid())
.setDatanodeUuid(
pipeline.getLeader().getUuidString())
.build();
return cmd;
}
@ -589,7 +597,7 @@ public static ContainerCommandRequestProto getDeleteContainer(
.setCmdType(ContainerProtos.Type.DeleteContainer)
.setDeleteContainer(deleteRequest)
.setTraceID(UUID.randomUUID().toString())
.setDatanodeID(pipeline.getLeader().getDatanodeUuid())
.setDatanodeUuid(pipeline.getLeader().getUuidString())
.build();
}
}

View File

@ -20,7 +20,6 @@
import com.google.common.collect.Lists;
import org.apache.commons.io.FileUtils;
import org.apache.hadoop.conf.Configuration;
import org.apache.hadoop.hdfs.DFSTestUtil;
import org.apache.hadoop.hdfs.DFSUtil;
import org.apache.hadoop.hdsl.protocol.proto.ContainerProtos;
import org.apache.hadoop.hdfs.server.datanode.StorageLocation;
@ -35,6 +34,7 @@
import org.apache.hadoop.ozone.container.common.impl.RandomContainerDeletionChoosingPolicy;
import org.apache.hadoop.ozone.container.common.interfaces.ContainerManager;
import org.apache.hadoop.ozone.container.common.statemachine.background.BlockDeletingService;
import org.apache.hadoop.ozone.scm.TestUtils;
import org.apache.hadoop.ozone.web.utils.OzoneUtils;
import org.apache.hadoop.scm.ScmConfigKeys;
import org.apache.hadoop.test.GenericTestUtils;
@ -115,7 +115,7 @@ private ContainerManager createContainerManager(Configuration conf)
ContainerManager containerManager = new ContainerManagerImpl();
List<StorageLocation> pathLists = new LinkedList<>();
pathLists.add(StorageLocation.parse(containersDir.getAbsolutePath()));
containerManager.init(conf, pathLists, DFSTestUtil.getLocalDatanodeID());
containerManager.init(conf, pathLists, TestUtils.getDatanodeDetails());
return containerManager;
}

View File

@ -28,15 +28,15 @@
import java.util.Random;
import org.apache.commons.io.FileUtils;
import org.apache.hadoop.hdfs.DFSTestUtil;
import org.apache.hadoop.hdfs.DFSUtil;
import org.apache.hadoop.hdfs.protocol.DatanodeID;
import org.apache.hadoop.hdfs.server.datanode.StorageLocation;
import org.apache.hadoop.hdsl.protocol.DatanodeDetails;
import org.apache.hadoop.ozone.OzoneConfigKeys;
import org.apache.hadoop.hdsl.conf.OzoneConfiguration;
import org.apache.hadoop.ozone.OzoneConsts;
import org.apache.hadoop.ozone.container.common.helpers.ContainerData;
import org.apache.hadoop.ozone.container.common.helpers.KeyUtils;
import org.apache.hadoop.ozone.scm.TestUtils;
import org.apache.hadoop.ozone.web.utils.OzoneUtils;
import org.apache.hadoop.scm.ScmConfigKeys;
import org.apache.hadoop.test.GenericTestUtils;
@ -89,7 +89,7 @@ public void testRandomChoosingPolicy() throws IOException {
List<StorageLocation> pathLists = new LinkedList<>();
pathLists.add(StorageLocation.parse(containerDir.getAbsolutePath()));
containerManager = new ContainerManagerImpl();
containerManager.init(conf, pathLists, DFSTestUtil.getLocalDatanodeID());
containerManager.init(conf, pathLists, TestUtils.getDatanodeDetails());
int numContainers = 10;
for (int i = 0; i < numContainers; i++) {
@ -135,8 +135,8 @@ public void testTopNOrderedChoosingPolicy() throws IOException {
List<StorageLocation> pathLists = new LinkedList<>();
pathLists.add(StorageLocation.parse(containerDir.getAbsolutePath()));
containerManager = new ContainerManagerImpl();
DatanodeID datanodeID = DFSTestUtil.getLocalDatanodeID();
containerManager.init(conf, pathLists, datanodeID);
DatanodeDetails datanodeDetails = TestUtils.getDatanodeDetails();
containerManager.init(conf, pathLists, datanodeDetails);
int numContainers = 10;
Random random = new Random();
@ -172,7 +172,7 @@ public void testTopNOrderedChoosingPolicy() throws IOException {
containerManager.writeLock();
containerManager.shutdown();
containerManager.writeUnlock();
containerManager.init(conf, pathLists, datanodeID);
containerManager.init(conf, pathLists, datanodeDetails);
List<ContainerData> result0 = containerManager
.chooseContainerForBlockDeletion(5);

View File

@ -21,12 +21,12 @@
import org.apache.commons.io.FileUtils;
import org.apache.commons.lang3.RandomStringUtils;
import org.apache.hadoop.fs.FileUtil;
import org.apache.hadoop.hdfs.DFSTestUtil;
import org.apache.hadoop.hdsl.protocol.proto.ContainerProtos;
import org.apache.hadoop.hdfs.server.datanode.StorageLocation;
import org.apache.hadoop.ozone.OzoneConfigKeys;
import org.apache.hadoop.hdsl.conf.OzoneConfiguration;
import org.apache.hadoop.ozone.OzoneConsts;
import org.apache.hadoop.ozone.scm.TestUtils;
import org.apache.hadoop.scm.container.common.helpers.StorageContainerException;
import org.apache.hadoop.test.GenericTestUtils;
import org.apache.hadoop.ozone.container.common.helpers.ChunkInfo;
@ -153,7 +153,7 @@ public void setupPaths() throws IOException {
FileUtils.forceMkdir(new File(location.getNormalizedUri()));
}
containerManager.init(conf, pathLists, DFSTestUtil.getLocalDatanodeID());
containerManager.init(conf, pathLists, TestUtils.getDatanodeDetails());
}
@After

View File

@ -18,6 +18,7 @@
package org.apache.hadoop.ozone.container.common.statemachine.commandhandler;
import org.apache.hadoop.hdsl.conf.OzoneConfiguration;
import org.apache.hadoop.hdsl.protocol.DatanodeDetails;
import org.apache.hadoop.ozone.MiniOzoneClassicCluster;
import org.apache.hadoop.ozone.MiniOzoneTestHelper;
import org.apache.hadoop.ozone.OzoneConsts;
@ -85,9 +86,11 @@ public void test() throws IOException, TimeoutException, InterruptedException,
Assert.assertFalse(isContainerClosed(cluster, containerName));
DatanodeDetails datanodeDetails = MiniOzoneClassicCluster
.getDatanodeDetails(cluster.getDataNodes().get(0));
//send the order to close the container
cluster.getStorageContainerManager().getScmNodeManager()
.addDatanodeCommand(cluster.getDataNodes().get(0).getDatanodeId(),
.addDatanodeCommand(datanodeDetails.getUuid(),
new CloseContainerCommand(containerName));
GenericTestUtils.waitFor(() -> isContainerClosed(cluster, containerName),

View File

@ -24,6 +24,7 @@
import org.apache.hadoop.hdfs.DFSConfigKeys;
import org.apache.hadoop.hdsl.conf.OzoneConfiguration;
import org.apache.hadoop.hdsl.protocol.DatanodeDetails;
import org.apache.hadoop.hdsl.protocol.proto.ContainerProtos;
import org.apache.hadoop.hdsl.protocol.proto.ContainerProtos.ContainerCommandRequestProto;
import org.apache.hadoop.hdsl.protocol.proto.ContainerProtos.ContainerCommandResponseProto;
@ -35,6 +36,7 @@
import org.apache.hadoop.ozone.container.common.interfaces.ChunkManager;
import org.apache.hadoop.ozone.container.common.interfaces.ContainerManager;
import org.apache.hadoop.ozone.container.common.transport.server.XceiverServer;
import org.apache.hadoop.ozone.scm.TestUtils;
import org.apache.hadoop.ozone.web.utils.OzoneUtils;
import org.apache.hadoop.scm.XceiverClient;
import org.apache.hadoop.scm.container.common.helpers.Pipeline;
@ -79,7 +81,8 @@ public void testContainerMetrics() throws Exception {
Dispatcher dispatcher = new Dispatcher(containerManager, conf);
dispatcher.init();
server = new XceiverServer(conf, dispatcher);
DatanodeDetails datanodeDetails = TestUtils.getDatanodeDetails();
server = new XceiverServer(datanodeDetails, conf, dispatcher);
client = new XceiverClient(pipeline, conf);
server.start();

View File

@ -18,7 +18,6 @@
package org.apache.hadoop.ozone.container.ozoneimpl;
import org.apache.hadoop.hdfs.DFSTestUtil;
import org.apache.hadoop.hdsl.protocol.proto.ContainerProtos;
import org.apache.hadoop.ozone.MiniOzoneClassicCluster;
import org.apache.hadoop.ozone.MiniOzoneCluster;
@ -26,6 +25,7 @@
import org.apache.hadoop.hdsl.conf.OzoneConfiguration;
import org.apache.hadoop.ozone.OzoneConsts;
import org.apache.hadoop.ozone.container.ContainerTestHelper;
import org.apache.hadoop.ozone.scm.TestUtils;
import org.apache.hadoop.ozone.web.utils.OzoneUtils;
import org.apache.hadoop.scm.XceiverClient;
import org.apache.hadoop.scm.XceiverClientSpi;
@ -68,8 +68,7 @@ public void testCreateOzoneContainer() throws Exception {
conf.setInt(OzoneConfigKeys.DFS_CONTAINER_IPC_PORT,
pipeline.getLeader().getContainerPort());
conf.setBoolean(OzoneConfigKeys.DFS_CONTAINER_IPC_RANDOM_PORT, false);
container = new OzoneContainer(DFSTestUtil.getLocalDatanodeID(1),
conf);
container = new OzoneContainer(TestUtils.getDatanodeDetails(), conf);
container.start();
XceiverClient client = new XceiverClient(pipeline, conf);

View File

@ -87,7 +87,8 @@ private static void runTest(
final List<DataNode> datanodes = cluster.getDataNodes();
final Pipeline pipeline = ContainerTestHelper.createPipeline(
containerName,
CollectionUtils.as(datanodes, DataNode::getDatanodeId));
CollectionUtils.as(datanodes,
MiniOzoneClassicCluster::getDatanodeDetails));
LOG.info("pipeline=" + pipeline);
// Create Ratis cluster
@ -97,7 +98,7 @@ private static void runTest(
// LOG.info("Created RatisCluster " + ratisId);
//
// // check Ratis cluster members
// final List<DatanodeID> dns = manager.getMembers(ratisId);
// final List<DatanodeDetails> dns = manager.getMembers(ratisId);
// Assert.assertEquals(pipeline.getMachines(), dns);
//
// // run test

View File

@ -18,8 +18,8 @@
package org.apache.hadoop.ozone.container.ozoneimpl;
import org.apache.hadoop.hdfs.protocol.DatanodeID;
import org.apache.hadoop.hdfs.server.datanode.DataNode;
import org.apache.hadoop.hdsl.protocol.DatanodeDetails;
import org.apache.hadoop.ozone.MiniOzoneClassicCluster;
import org.apache.hadoop.hdsl.conf.OzoneConfiguration;
import org.apache.hadoop.ozone.OzoneConsts;
@ -83,15 +83,17 @@ private static void runTestRatisManager(RpcType rpc) throws Exception {
cluster.waitOzoneReady();
final List<DataNode> datanodes = cluster.getDataNodes();
final List<DatanodeID> allIds = datanodes.stream()
.map(DataNode::getDatanodeId).collect(Collectors.toList());
final List<DatanodeDetails> datanodeDetailsSet = datanodes.stream()
.map(MiniOzoneClassicCluster::getDatanodeDetails).collect(
Collectors.toList());
//final RatisManager manager = RatisManager.newRatisManager(conf);
final int[] idIndex = {3, 4, 5};
for (int i = 0; i < idIndex.length; i++) {
final int previous = i == 0 ? 0 : idIndex[i - 1];
final List<DatanodeID> subIds = allIds.subList(previous, idIndex[i]);
final List<DatanodeDetails> subIds = datanodeDetailsSet.subList(
previous, idIndex[i]);
// Create Ratis cluster
final String ratisId = "ratis" + i;
@ -99,7 +101,7 @@ private static void runTestRatisManager(RpcType rpc) throws Exception {
LOG.info("Created RatisCluster " + ratisId);
// check Ratis cluster members
//final List<DatanodeID> dns = manager.getMembers(ratisId);
//final List<DatanodeDetails> dns = manager.getMembers(ratisId);
//Assert.assertEquals(subIds, dns);
}
@ -119,7 +121,7 @@ private static void runTestRatisManager(RpcType rpc) throws Exception {
//manager.updatePipeline(ratisId, allIds);
// check Ratis cluster members
//final List<DatanodeID> dns = manager.getMembers(ratisId);
//final List<DatanodeDetails> dns = manager.getMembers(ratisId);
//Assert.assertEquals(allIds, dns);
} finally {
cluster.shutdown();

View File

@ -19,10 +19,10 @@
package org.apache.hadoop.ozone.container.server;
import io.netty.channel.embedded.EmbeddedChannel;
import org.apache.hadoop.hdsl.protocol.DatanodeDetails;
import org.apache.hadoop.hdsl.protocol.proto.ContainerProtos;
import org.apache.hadoop.hdsl.protocol.proto.ContainerProtos.ContainerCommandRequestProto;
import org.apache.hadoop.hdsl.protocol.proto.ContainerProtos.ContainerCommandResponseProto;
import org.apache.hadoop.hdfs.protocol.DatanodeID;
import org.apache.hadoop.ozone.OzoneConfigKeys;
import org.apache.hadoop.hdsl.conf.OzoneConfiguration;
import org.apache.hadoop.ozone.RatisTestHelper;
@ -34,6 +34,7 @@
import org.apache.hadoop.ozone.container.common.transport.server.XceiverServerHandler;
import org.apache.hadoop.ozone.container.common.transport.server.XceiverServerSpi;
import org.apache.hadoop.ozone.container.common.transport.server.ratis.XceiverServerRatis;
import org.apache.hadoop.ozone.scm.TestUtils;
import org.apache.hadoop.ozone.web.utils.OzoneUtils;
import org.apache.hadoop.scm.XceiverClient;
import org.apache.hadoop.scm.XceiverClientRatis;
@ -90,11 +91,13 @@ public void testPipeline() throws IOException {
@Test
public void testClientServer() throws Exception {
DatanodeDetails datanodeDetails = TestUtils.getDatanodeDetails();
runTestClientServer(1,
(pipeline, conf) -> conf.setInt(OzoneConfigKeys.DFS_CONTAINER_IPC_PORT,
pipeline.getLeader().getContainerPort()),
XceiverClient::new,
(dn, conf) -> new XceiverServer(conf, new TestContainerDispatcher()),
(dn, conf) -> new XceiverServer(datanodeDetails, conf,
new TestContainerDispatcher()),
(dn, p) -> {});
}
@ -116,11 +119,10 @@ public void testClientServerRatisGrpc() throws Exception {
}
static XceiverServerRatis newXceiverServerRatis(
DatanodeID dn, OzoneConfiguration conf) throws IOException {
final String id = dn.getXferAddr();
DatanodeDetails dn, OzoneConfiguration conf) throws IOException {
conf.setInt(OzoneConfigKeys.DFS_CONTAINER_RATIS_IPC_PORT,
dn.getRatisPort());
final String dir = TEST_DIR + id.replace(':', '_');
final String dir = TEST_DIR + dn.getUuid();
conf.set(OzoneConfigKeys.DFS_CONTAINER_RATIS_DATANODE_STORAGE_DIR, dir);
final ContainerDispatcher dispatcher = new TestContainerDispatcher();
@ -128,8 +130,8 @@ static XceiverServerRatis newXceiverServerRatis(
}
static void initXceiverServerRatis(
RpcType rpc, DatanodeID id, Pipeline pipeline) throws IOException {
final RaftPeer p = RatisHelper.toRaftPeer(id);
RpcType rpc, DatanodeDetails dd, Pipeline pipeline) throws IOException {
final RaftPeer p = RatisHelper.toRaftPeer(dd);
final RaftClient client = RatisHelper.newRaftClient(rpc, p);
client.reinitialize(RatisHelper.newRaftGroup(pipeline), p.getId());
}
@ -149,9 +151,9 @@ static void runTestClientServer(
BiConsumer<Pipeline, OzoneConfiguration> initConf,
CheckedBiFunction<Pipeline, OzoneConfiguration, XceiverClientSpi,
IOException> createClient,
CheckedBiFunction<DatanodeID, OzoneConfiguration, XceiverServerSpi,
CheckedBiFunction<DatanodeDetails, OzoneConfiguration, XceiverServerSpi,
IOException> createServer,
CheckedBiConsumer<DatanodeID, Pipeline, IOException> initServer)
CheckedBiConsumer<DatanodeDetails, Pipeline, IOException> initServer)
throws Exception {
final List<XceiverServerSpi> servers = new ArrayList<>();
XceiverClientSpi client = null;
@ -162,7 +164,7 @@ static void runTestClientServer(
final OzoneConfiguration conf = new OzoneConfiguration();
initConf.accept(pipeline, conf);
for(DatanodeID dn : pipeline.getMachines()) {
for(DatanodeDetails dn : pipeline.getMachines()) {
final XceiverServerSpi s = createServer.apply(dn, conf);
servers.add(s);
s.start();
@ -203,7 +205,8 @@ public void testClientServerWithContainerDispatcher() throws Exception {
Dispatcher dispatcher =
new Dispatcher(mock(ContainerManager.class), conf);
dispatcher.init();
server = new XceiverServer(conf, dispatcher);
DatanodeDetails datanodeDetails = TestUtils.getDatanodeDetails();
server = new XceiverServer(datanodeDetails, conf, dispatcher);
client = new XceiverClient(pipeline, conf);
server.start();

View File

@ -124,7 +124,7 @@ public void testGetServiceList() throws Exception {
switch (type) {
case HTTP:
case HTTPS:
Assert.assertEquals(datanode.getDatanodeId().getOzoneRestPort(),
Assert.assertEquals(MiniOzoneClassicCluster.getOzoneRestPort(datanode),
(int) ports.get(type));
break;
default:

View File

@ -117,7 +117,7 @@ public static void init()
cluster = new MiniOzoneClassicCluster.Builder(conf)
.setHandlerType(OzoneConsts.OZONE_HANDLER_DISTRIBUTED).build();
DataNode dataNode = cluster.getDataNodes().get(0);
final int port = dataNode.getDatanodeId().getOzoneRestPort();
final int port = MiniOzoneClassicCluster.getOzoneRestPort(dataNode);
url = String.format("http://localhost:%d", port);
client = new OzoneRestClient(String.format("http://localhost:%d", port));
client.setUserAuth(OzoneConsts.OZONE_SIMPLE_HDFS_USER);

View File

@ -17,7 +17,7 @@
*/
package org.apache.hadoop.ozone.scm;
import org.apache.hadoop.hdfs.protocol.DatanodeID;
import org.apache.hadoop.hdsl.protocol.DatanodeDetails;
import org.apache.hadoop.io.IOUtils;
import org.apache.hadoop.ozone.MiniOzoneClassicCluster;
import org.apache.hadoop.ozone.OzoneConfigKeys;
@ -214,8 +214,8 @@ public void testConvertNodepoolDB() throws Exception {
// verify the sqlite db
HashMap<String, String> expectedPool = new HashMap<>();
for (DatanodeID dnid : nodeManager.getAllNodes()) {
expectedPool.put(dnid.getDatanodeUuid(), "DefaultNodePool");
for (DatanodeDetails dnid : nodeManager.getAllNodes()) {
expectedPool.put(dnid.getUuidString(), "DefaultNodePool");
}
Connection conn = connectDB(dbOutPath);
String sql = "SELECT * FROM nodePool";
@ -234,7 +234,7 @@ public void testConvertNodepoolDB() throws Exception {
public void testConvertContainerDB() throws Exception {
String dbOutPath = cluster.getDataDirectory() + "/out_sql.db";
// TODO : the following will fail due to empty Datanode list, need to fix.
//String dnUUID = cluster.getDataNodes().get(0).getDatanodeUuid();
//String dnUUID = cluster.getDataNodes().get(0).getUuid();
String dbRootPath = conf.get(OzoneConfigKeys.OZONE_METADATA_DIRS);
String dbPath = dbRootPath + "/" + SCM_CONTAINER_DB;
String[] args = {"-p", dbPath, "-o", dbOutPath};
@ -279,8 +279,8 @@ public void testConvertContainerDB() throws Exception {
count += 1;
}
// the two containers maybe on the same datanode, maybe not.
int expected = pipeline1.getLeader().getDatanodeUuid().equals(
pipeline2.getLeader().getDatanodeUuid())? 1 : 2;
int expected = pipeline1.getLeader().getUuid().equals(
pipeline2.getLeader().getUuid())? 1 : 2;
assertEquals(expected, count);
Files.delete(Paths.get(dbOutPath));
}

View File

@ -17,7 +17,7 @@
*/
package org.apache.hadoop.ozone.scm;
import org.apache.hadoop.hdfs.protocol.DatanodeID;
import org.apache.hadoop.hdsl.protocol.DatanodeDetails;
import org.apache.hadoop.io.IOUtils;
import org.apache.hadoop.ozone.MiniOzoneClassicCluster;
import org.apache.hadoop.hdsl.conf.OzoneConfiguration;
@ -232,7 +232,8 @@ public void testDeleteContainer() throws Exception {
@Test
public void testInfoContainer() throws Exception {
// The cluster has one Datanode server.
DatanodeID datanodeID = cluster.getDataNodes().get(0).getDatanodeId();
DatanodeDetails datanodeDetails = MiniOzoneClassicCluster
.getDatanodeDetails(cluster.getDataNodes().get(0));
String formatStr =
"Container Name: %s\n" +
"Container State: %s\n" +
@ -276,7 +277,7 @@ public void testInfoContainer() throws Exception {
String openStatus = data.isOpen() ? "OPEN" : "CLOSED";
String expected = String.format(formatStr, cname, openStatus,
data.getDBPath(), data.getContainerPath(), "",
datanodeID.getHostName(), datanodeID.getHostName());
datanodeDetails.getHostName(), datanodeDetails.getHostName());
assertEquals(expected, out.toString());
out.reset();
@ -297,7 +298,7 @@ public void testInfoContainer() throws Exception {
openStatus = data.isOpen() ? "OPEN" : "CLOSED";
expected = String.format(formatStr, cname, openStatus,
data.getDBPath(), data.getContainerPath(), "",
datanodeID.getHostName(), datanodeID.getHostName());
datanodeDetails.getHostName(), datanodeDetails.getHostName());
assertEquals(expected, out.toString());
out.reset();
@ -315,7 +316,7 @@ public void testInfoContainer() throws Exception {
openStatus = data.isOpen() ? "OPEN" : "CLOSED";
expected = String.format(formatStrWithHash, cname, openStatus,
data.getHash(), data.getDBPath(), data.getContainerPath(),
"", datanodeID.getHostName(), datanodeID.getHostName());
"", datanodeDetails.getHostName(), datanodeDetails.getHostName());
assertEquals(expected, out.toString());
}

View File

@ -26,12 +26,11 @@
import org.apache.commons.codec.digest.DigestUtils;
import org.apache.hadoop.hdsl.conf.OzoneConfiguration;
import org.apache.hadoop.hdfs.protocol.DatanodeID;
import org.apache.hadoop.hdfs.server.datanode.DataNode;
import org.apache.hadoop.hdsl.protocol.DatanodeDetails;
import org.apache.hadoop.metrics2.MetricsRecordBuilder;
import org.apache.hadoop.ozone.MiniOzoneClassicCluster;
import org.apache.hadoop.ozone.OzoneConsts;
import org.apache.hadoop.ozone.container.common.SCMTestUtils;
import org.apache.hadoop.ozone.container.common.helpers.ContainerReport;
import org.apache.hadoop.hdsl.protocol.proto.StorageContainerDatanodeProtocolProtos;
import org.apache.hadoop.hdsl.protocol.proto.StorageContainerDatanodeProtocolProtos.ContainerReportsRequestProto;
@ -79,7 +78,7 @@ public void testContainerMetrics() throws Exception {
ContainerReportsRequestProto request = createContainerReport(numReport,
stat, null);
String fstDatanodeID = request.getDatanodeID().getDatanodeUuid();
String fstDatanodeUuid = request.getDatanodeDetails().getUuid();
scmManager.sendContainerReport(request);
// verify container stat metrics
@ -102,7 +101,7 @@ public void testContainerMetrics() throws Exception {
// add one new report
request = createContainerReport(1, stat, null);
String sndDatanodeID = request.getDatanodeID().getDatanodeUuid();
String sndDatanodeUuid = request.getDatanodeDetails().getUuid();
scmManager.sendContainerReport(request);
scmMetrics = getMetrics(SCMMetrics.SOURCE_NAME);
@ -126,11 +125,11 @@ public void testContainerMetrics() throws Exception {
// the aggregation.
stat = new ContainerStat(100, 50, 3, 50, 60, 5, 6);
scmManager.sendContainerReport(createContainerReport(1, stat,
fstDatanodeID));
fstDatanodeUuid));
stat = new ContainerStat(1, 1, 1, 1, 1, 1, 1);
scmManager.sendContainerReport(createContainerReport(1, stat,
sndDatanodeID));
sndDatanodeUuid));
// the global container metrics value should be updated
scmMetrics = getMetrics(SCMMetrics.SOURCE_NAME);
@ -172,7 +171,8 @@ public void testStaleNodeContainerReport() throws Exception {
StorageContainerManager scmManager = cluster.getStorageContainerManager();
DataNode dataNode = cluster.getDataNodes().get(0);
String datanodeUuid = dataNode.getDatanodeId().getDatanodeUuid();
String datanodeUuid = MiniOzoneClassicCluster.getDatanodeDetails(dataNode)
.getUuidString();
ContainerReportsRequestProto request = createContainerReport(numReport,
stat, datanodeUuid);
scmManager.sendContainerReport(request);
@ -236,14 +236,23 @@ private ContainerReportsRequestProto createContainerReport(int numReport,
reportsBuilder.addReports(report.getProtoBufMessage());
}
DatanodeID datanodeID;
DatanodeDetails datanodeDetails;
if (datanodeUuid == null) {
datanodeID = TestUtils.getDatanodeID();
datanodeDetails = TestUtils.getDatanodeDetails();
} else {
datanodeID = new DatanodeID("null", "null", datanodeUuid, 0, 0, 0, 0);
datanodeDetails = DatanodeDetails.newBuilder()
.setUuid(datanodeUuid)
.setIpAddress("127.0.0.1")
.setHostName("localhost")
.setInfoPort(0)
.setInfoSecurePort(0)
.setContainerPort(0)
.setRatisPort(0)
.setOzoneRestPort(0)
.build();
}
reportsBuilder.setDatanodeID(datanodeID.getProtoBufMessage());
reportsBuilder.setDatanodeDetails(datanodeDetails.getProtoBufMessage());
reportsBuilder.setType(StorageContainerDatanodeProtocolProtos
.ContainerReportsRequestProto.reportType.fullReport);
return reportsBuilder.build();

View File

@ -67,7 +67,7 @@ public static void init() throws Exception {
cluster = new MiniOzoneClassicCluster.Builder(conf)
.setHandlerType(OzoneConsts.OZONE_HANDLER_DISTRIBUTED).build();
DataNode dataNode = cluster.getDataNodes().get(0);
port = dataNode.getDatanodeId().getOzoneRestPort();
port = MiniOzoneClassicCluster.getOzoneRestPort(dataNode);
}
/**

View File

@ -70,7 +70,7 @@ public static void init() throws Exception {
cluster = new MiniOzoneClassicCluster.Builder(conf)
.setHandlerType(OzoneConsts.OZONE_HANDLER_LOCAL).build();
DataNode dataNode = cluster.getDataNodes().get(0);
port = dataNode.getDatanodeId().getOzoneRestPort();
port = MiniOzoneClassicCluster.getOzoneRestPort(dataNode);
}
/**

View File

@ -79,7 +79,7 @@ public static void init() throws Exception {
cluster = new MiniOzoneClassicCluster.Builder(conf)
.setHandlerType(OzoneConsts.OZONE_HANDLER_LOCAL).build();
DataNode dataNode = cluster.getDataNodes().get(0);
port = dataNode.getDatanodeId().getOzoneRestPort();
port = MiniOzoneClassicCluster.getOzoneRestPort(dataNode);
}
/**

View File

@ -80,7 +80,7 @@ public static void init() throws IOException,
cluster = new MiniOzoneClassicCluster.Builder(conf)
.setHandlerType(OzoneConsts.OZONE_HANDLER_DISTRIBUTED).build();
DataNode dataNode = cluster.getDataNodes().get(0);
final int port = dataNode.getDatanodeId().getOzoneRestPort();
final int port = MiniOzoneClassicCluster.getOzoneRestPort(dataNode);
ozoneRestClient = new OzoneRestClient(
String.format("http://localhost:%d", port));
}

View File

@ -111,7 +111,7 @@ public static void init() throws Exception {
ozoneCluster = new MiniOzoneClassicCluster.Builder(conf)
.setHandlerType(OzoneConsts.OZONE_HANDLER_DISTRIBUTED).build();
DataNode dataNode = ozoneCluster.getDataNodes().get(0);
final int port = dataNode.getDatanodeId().getOzoneRestPort();
final int port = MiniOzoneClassicCluster.getOzoneRestPort(dataNode);
ozoneRestClient = new OzoneRestClient(
String.format("http://localhost:%d", port));
currentTime = Time.now();
@ -282,7 +282,7 @@ private static void restartDatanode(
cluster.restartDataNode(datanodeIdx);
// refresh the datanode endpoint uri after datanode restart
DataNode dataNode = cluster.getDataNodes().get(datanodeIdx);
final int port = dataNode.getDatanodeId().getOzoneRestPort();
final int port = MiniOzoneClassicCluster.getOzoneRestPort(dataNode);
client.setEndPoint(String.format("http://localhost:%d", port));
}

View File

@ -98,7 +98,7 @@ public static void init() throws Exception {
.setHandlerType(OzoneConsts.OZONE_HANDLER_DISTRIBUTED).build();
DataNode dataNode = cluster.getDataNodes().get(0);
endpoint = String.format("http://localhost:%d",
dataNode.getDatanodeId().getOzoneRestPort());
MiniOzoneClassicCluster.getOzoneRestPort(dataNode));
}
@AfterClass

View File

@ -90,7 +90,7 @@ public static void init() throws Exception {
cluster = new MiniOzoneClassicCluster.Builder(conf)
.setHandlerType(OzoneConsts.OZONE_HANDLER_DISTRIBUTED).build();
DataNode dataNode = cluster.getDataNodes().get(0);
final int port = dataNode.getDatanodeId().getOzoneRestPort();
final int port = MiniOzoneClassicCluster.getOzoneRestPort(dataNode);
ozoneRestClient = new OzoneRestClient(
String.format("http://localhost:%d", port));

Some files were not shown because too many files have changed in this diff Show More