HDFS-13300. Ozone: Remove DatanodeID dependency from HDSL and Ozone.
Contributed by Nanda kumar.
This commit is contained in:
parent
70216e7978
commit
498b84155f
|
@ -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
|
||||
|
|
|
@ -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
|
||||
|
|
|
@ -28,9 +28,9 @@ import io.netty.channel.socket.nio.NioSocketChannel;
|
|||
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 class XceiverClient extends XceiverClientSpi {
|
|||
.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 class XceiverClient extends XceiverClientSpi {
|
|||
* @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;
|
||||
|
|
|
@ -154,7 +154,7 @@ public class XceiverClientHandler extends
|
|||
|
||||
// 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");
|
||||
}
|
||||
|
||||
|
|
|
@ -21,10 +21,10 @@ package org.apache.hadoop.scm;
|
|||
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 @@ public final class XceiverClientRatis extends XceiverClientSpi {
|
|||
/**
|
||||
* {@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 final class XceiverClientRatis extends XceiverClientSpi {
|
|||
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 @@ public final class XceiverClientRatis extends XceiverClientSpi {
|
|||
* @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)) {
|
||||
|
|
|
@ -255,7 +255,7 @@ public class HdslUtils {
|
|||
* @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);
|
||||
|
|
|
@ -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);
|
||||
}
|
||||
|
||||
}
|
||||
|
||||
}
|
|
@ -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;
|
|
@ -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 class XceiverClientSpi implements Closeable {
|
|||
* @param datanodes - Datanodes
|
||||
*/
|
||||
public abstract void createPipeline(String pipelineID,
|
||||
List<DatanodeID> datanodes) throws IOException;
|
||||
List<DatanodeDetails> datanodes) throws IOException;
|
||||
|
||||
/**
|
||||
* Returns pipeline Type.
|
||||
|
|
|
@ -29,8 +29,8 @@ import com.fasterxml.jackson.databind.ser.impl.SimpleBeanPropertyFilter;
|
|||
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 class Pipeline {
|
|||
* @return First Machine.
|
||||
*/
|
||||
@JsonIgnore
|
||||
public DatanodeID getLeader() {
|
||||
public DatanodeDetails getLeader() {
|
||||
return pipelineChannel.getDatanodes().get(pipelineChannel.getLeaderID());
|
||||
}
|
||||
|
||||
|
@ -120,7 +120,7 @@ public class Pipeline {
|
|||
* @return List of Machines.
|
||||
*/
|
||||
@JsonIgnore
|
||||
public List<DatanodeID> getMachines() {
|
||||
public List<DatanodeDetails> getMachines() {
|
||||
return new ArrayList<>(pipelineChannel.getDatanodes().values());
|
||||
}
|
||||
|
||||
|
@ -131,7 +131,7 @@ public class Pipeline {
|
|||
*/
|
||||
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;
|
||||
|
|
|
@ -19,8 +19,7 @@ package org.apache.hadoop.scm.container.common.helpers;
|
|||
|
||||
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 class PipelineChannel {
|
|||
return leaderID;
|
||||
}
|
||||
|
||||
public Map<String, DatanodeID> getDatanodes() {
|
||||
public Map<String, DatanodeDetails> getDatanodes() {
|
||||
return datanodes;
|
||||
}
|
||||
|
||||
|
@ -77,15 +76,16 @@ public class PipelineChannel {
|
|||
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 class PipelineChannel {
|
|||
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;
|
||||
}
|
||||
|
|
|
@ -82,12 +82,12 @@ public final class ContainerProtocolCalls {
|
|||
.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 final class ContainerProtocolCalls {
|
|||
.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 final class ContainerProtocolCalls {
|
|||
.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 final class ContainerProtocolCalls {
|
|||
.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 final class ContainerProtocolCalls {
|
|||
.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 final class ContainerProtocolCalls {
|
|||
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 final class ContainerProtocolCalls {
|
|||
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 final class ContainerProtocolCalls {
|
|||
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 final class ContainerProtocolCalls {
|
|||
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 final class ContainerProtocolCalls {
|
|||
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);
|
||||
|
|
|
@ -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 @@ import java.util.stream.Collectors;
|
|||
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 @@ public interface RatisHelper {
|
|||
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 @@ public interface RatisHelper {
|
|||
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());
|
||||
|
|
|
@ -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 {
|
||||
|
|
|
@ -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;
|
||||
}
|
||||
|
||||
|
|
|
@ -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 {
|
||||
}
|
||||
}
|
|
@ -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;
|
||||
}
|
||||
}
|
|
@ -22,9 +22,9 @@ import com.google.common.base.Preconditions;
|
|||
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.FileOutputStream;
|
|||
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 final class ContainerUtils {
|
|||
}
|
||||
|
||||
/**
|
||||
* 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 @@ public final class ContainerUtils {
|
|||
}
|
||||
}
|
||||
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));
|
||||
} catch (IOException e) {
|
||||
throw new IOException("Failed to parse Datanode ID from "
|
||||
+ path.getAbsolutePath(), e);
|
||||
}
|
||||
}
|
||||
return DatanodeDetails.getFromProtoBuf(
|
||||
HdslProtos.DatanodeDetailsProto.parseFrom(in));
|
||||
} catch (IOException e) {
|
||||
throw new IOException("Failed to parse DatanodeDetails from "
|
||||
+ path.getAbsolutePath(), e);
|
||||
}
|
||||
return ids;
|
||||
}
|
||||
}
|
||||
|
|
|
@ -21,9 +21,9 @@ package org.apache.hadoop.ozone.container.common.interfaces;
|
|||
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.
|
||||
|
|
|
@ -19,8 +19,8 @@ package org.apache.hadoop.ozone.container.common.statemachine;
|
|||
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 class DatanodeStateMachine implements Closeable {
|
|||
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 class DatanodeStateMachine implements Closeable {
|
|||
.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.
|
||||
*
|
||||
|
|
|
@ -18,15 +18,15 @@ package org.apache.hadoop.ozone.container.common.states.datanode;
|
|||
|
||||
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 class InitDatanodeState implements DatanodeState,
|
|||
}
|
||||
|
||||
// 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);
|
||||
}
|
||||
}
|
||||
|
||||
|
|
|
@ -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.DatanodeState;
|
|||
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 class RunningDatanodeState implements DatanodeState {
|
|||
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 class RunningDatanodeState implements DatanodeState {
|
|||
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:
|
||||
|
|
|
@ -20,7 +20,8 @@ package org.apache.hadoop.ozone.container.common.states.endpoint;
|
|||
|
||||
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
|
|||
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 class HeartbeatEndpointTask
|
|||
*
|
||||
* @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 class HeartbeatEndpointTask
|
|||
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 class HeartbeatEndpointTask
|
|||
* @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 @@ public class HeartbeatEndpointTask
|
|||
public static class Builder {
|
||||
private EndpointStateMachine endPointStateMachine;
|
||||
private Configuration conf;
|
||||
private ContainerNodeIDProto containerNodeIDProto;
|
||||
private DatanodeDetails datanodeDetails;
|
||||
private StateContext context;
|
||||
|
||||
/**
|
||||
|
@ -224,11 +221,11 @@ public class HeartbeatEndpointTask
|
|||
/**
|
||||
* 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 class HeartbeatEndpointTask
|
|||
" 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;
|
||||
}
|
||||
}
|
||||
|
|
|
@ -18,12 +18,11 @@ package org.apache.hadoop.ozone.container.common.states.endpoint;
|
|||
|
||||
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 final class RegisterEndpointTask implements
|
|||
}
|
||||
|
||||
/**
|
||||
* 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 final class RegisterEndpointTask implements
|
|||
@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 final class RegisterEndpointTask implements
|
|||
|
||||
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 final class RegisterEndpointTask implements
|
|||
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 final class RegisterEndpointTask implements
|
|||
/**
|
||||
* 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 final class RegisterEndpointTask implements
|
|||
" 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;
|
||||
}
|
||||
}
|
||||
|
|
|
@ -27,6 +27,7 @@ import io.netty.channel.socket.nio.NioServerSocketChannel;
|
|||
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 final class XceiverServer implements XceiverServerSpi {
|
|||
+ "fallback to use default port {}", this.port, e);
|
||||
}
|
||||
}
|
||||
datanodeDetails.setContainerPort(port);
|
||||
this.storageContainer = dispatcher;
|
||||
}
|
||||
|
||||
|
|
|
@ -21,7 +21,7 @@ package org.apache.hadoop.ozone.container.common.transport.server.ratis;
|
|||
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 @@ public final class XceiverServerRatis implements XceiverServerSpi {
|
|||
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 @@ public final class XceiverServerRatis implements XceiverServerSpi {
|
|||
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 @@ public final class XceiverServerRatis implements XceiverServerSpi {
|
|||
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 final class XceiverServerRatis implements XceiverServerSpi {
|
|||
// 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);
|
||||
}
|
||||
|
||||
|
|
|
@ -19,8 +19,8 @@ package org.apache.hadoop.ozone.container.ozoneimpl;
|
|||
|
||||
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 class OzoneContainer {
|
|||
}
|
||||
|
||||
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 class OzoneContainer {
|
|||
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)
|
||||
};
|
||||
}
|
||||
|
||||
|
|
|
@ -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 @@ public interface StorageContainerDatanodeProtocol {
|
|||
|
||||
/**
|
||||
* 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;
|
||||
|
||||
/**
|
||||
|
|
|
@ -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);
|
||||
|
||||
}
|
||||
|
|
|
@ -18,7 +18,7 @@ package org.apache.hadoop.ozone.protocolPB;
|
|||
|
||||
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 class StorageContainerDatanodeProtocolClientSideTranslatorPB
|
|||
/**
|
||||
* 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 class StorageContainerDatanodeProtocolClientSideTranslatorPB
|
|||
/**
|
||||
* 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());
|
||||
|
|
|
@ -18,7 +18,6 @@ package org.apache.hadoop.ozone.protocolPB;
|
|||
|
||||
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 class 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 class 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);
|
||||
|
|
|
@ -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.
|
||||
|
|
|
@ -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 @@ public class ScmTestMock implements StorageContainerDatanodeProtocol {
|
|||
/**
|
||||
* 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 @@ public class ScmTestMock implements StorageContainerDatanodeProtocol {
|
|||
/**
|
||||
* 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 @@ public class ScmTestMock implements StorageContainerDatanodeProtocol {
|
|||
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) {
|
||||
|
|
|
@ -19,8 +19,7 @@ package org.apache.hadoop.ozone.container.common;
|
|||
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.nio.file.Paths;
|
|||
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 class TestDatanodeStateMachine {
|
|||
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 class TestDatanodeStateMachine {
|
|||
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 class TestDatanodeStateMachine {
|
|||
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 class TestDatanodeStateMachine {
|
|||
}
|
||||
});
|
||||
}
|
||||
|
||||
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();
|
||||
}
|
||||
}
|
||||
|
|
|
@ -28,8 +28,8 @@ import com.google.protobuf.InvalidProtocolBufferException;
|
|||
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.DeleteBlocksCommand;
|
|||
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 class StorageContainerManager extends ServiceRuntimeInfoImpl
|
|||
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 class StorageContainerManager extends ServiceRuntimeInfoImpl
|
|||
* @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 class StorageContainerManager extends ServiceRuntimeInfoImpl
|
|||
* @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 class StorageContainerManager extends ServiceRuntimeInfoImpl
|
|||
/**
|
||||
* 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 class StorageContainerManager extends ServiceRuntimeInfoImpl
|
|||
/**
|
||||
* 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 @@ public class StorageContainerManager extends ServiceRuntimeInfoImpl
|
|||
// 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
|
||||
|
|
|
@ -20,9 +20,10 @@ import java.io.IOException;
|
|||
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 class DatanodeDeletedBlockTransactions {
|
|||
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 class DatanodeDeletedBlockTransactions {
|
|||
}
|
||||
}
|
||||
|
||||
Set<DatanodeID> getDatanodes() {
|
||||
Set<UUID> getDatanodeIDs() {
|
||||
return transactions.keySet();
|
||||
}
|
||||
|
||||
|
@ -101,18 +103,18 @@ public class DatanodeDeletedBlockTransactions {
|
|||
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 {
|
||||
|
|
|
@ -20,7 +20,7 @@ import com.google.common.annotations.VisibleForTesting;
|
|||
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 static org.apache.hadoop.ozone.OzoneConfigKeys.OZONE_BLOCK_DELETING_CONTA
|
|||
|
||||
import java.io.IOException;
|
||||
import java.util.List;
|
||||
import java.util.UUID;
|
||||
import java.util.concurrent.TimeUnit;
|
||||
|
||||
/**
|
||||
|
@ -114,7 +115,7 @@ public class SCMBlockDeletingService extends BackgroundService {
|
|||
// 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 class SCMBlockDeletingService extends BackgroundService {
|
|||
}
|
||||
|
||||
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 class SCMBlockDeletingService extends BackgroundService {
|
|||
LOG.info(
|
||||
"Totally added {} delete blocks command for"
|
||||
+ " {} datanodes, task elapsed time: {}ms",
|
||||
dnTxCount, transactions.getDatanodes().size(),
|
||||
dnTxCount, transactions.getDatanodeIDs().size(),
|
||||
Time.monotonicNow() - startTime);
|
||||
}
|
||||
|
||||
|
|
|
@ -415,7 +415,7 @@ public class ContainerMapping implements Mapping {
|
|||
// 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 {
|
||||
|
|
|
@ -22,8 +22,7 @@ import com.google.common.annotations.VisibleForTesting;
|
|||
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 class ContainerCloser {
|
|||
// 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())) {
|
||||
|
|
|
@ -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;
|
||||
}
|
||||
|
|
|
@ -19,7 +19,7 @@ package org.apache.hadoop.ozone.scm.container.placement.algorithms;
|
|||
|
||||
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 abstract class SCMCommonPolicy implements ContainerPlacementPolicy {
|
|||
* @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 abstract class SCMCommonPolicy implements ContainerPlacementPolicy {
|
|||
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 abstract class SCMCommonPolicy implements ContainerPlacementPolicy {
|
|||
/**
|
||||
* 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 @@ public abstract class SCMCommonPolicy implements ContainerPlacementPolicy {
|
|||
* @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 abstract class SCMCommonPolicy implements ContainerPlacementPolicy {
|
|||
* 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);
|
||||
|
||||
|
||||
}
|
||||
|
|
|
@ -19,7 +19,7 @@ package org.apache.hadoop.ozone.scm.container.placement.algorithms;
|
|||
|
||||
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 final class SCMContainerPlacementCapacity extends SCMCommonPolicy {
|
|||
* @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 final class SCMContainerPlacementCapacity extends SCMCommonPolicy {
|
|||
*
|
||||
* @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;
|
||||
}
|
||||
}
|
||||
|
|
|
@ -19,7 +19,7 @@ package org.apache.hadoop.ozone.scm.container.placement.algorithms;
|
|||
|
||||
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 final class SCMContainerPlacementRandom extends SCMCommonPolicy
|
|||
* @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 final class SCMContainerPlacementRandom extends SCMCommonPolicy
|
|||
* @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;
|
||||
|
|
|
@ -20,7 +20,7 @@ import com.google.common.annotations.VisibleForTesting;
|
|||
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 @@ public class ContainerSupervisor implements Closeable {
|
|||
*/
|
||||
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 class ContainerSupervisor implements Closeable {
|
|||
// 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();
|
||||
}
|
||||
|
|
|
@ -18,7 +18,7 @@ package org.apache.hadoop.ozone.scm.container.replication;
|
|||
|
||||
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 org.slf4j.LoggerFactory;
|
|||
|
||||
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 final class InProgressPool {
|
|||
* 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 final class InProgressPool {
|
|||
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 final class InProgressPool {
|
|||
/**
|
||||
* 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 @@ public final class InProgressPool {
|
|||
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 @@ public final class InProgressPool {
|
|||
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 final class InProgressPool {
|
|||
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: {}",
|
||||
|
|
|
@ -19,7 +19,6 @@ package org.apache.hadoop.ozone.scm.node;
|
|||
|
||||
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.HashMap;
|
|||
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 @@ import java.util.concurrent.locks.ReentrantLock;
|
|||
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 class CommandQueue {
|
|||
* 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 @@ public class CommandQueue {
|
|||
/**
|
||||
* 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 {
|
||||
|
|
|
@ -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 @@ import static org.apache.hadoop.util.Time.monotonicNow;
|
|||
* 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 class HeartbeatQueueItem {
|
|||
* 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 class HeartbeatQueueItem {
|
|||
}
|
||||
|
||||
public HeartbeatQueueItem build() {
|
||||
return new HeartbeatQueueItem(datanodeID, recvTimestamp, nodeReport,
|
||||
return new HeartbeatQueueItem(datanodeDetails, recvTimestamp, nodeReport,
|
||||
containerReportState);
|
||||
}
|
||||
}
|
||||
|
|
|
@ -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 org.apache.hadoop.ozone.scm.container.placement.metrics.SCMNodeStat;
|
|||
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);
|
||||
}
|
||||
|
|
|
@ -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 @@ public interface NodePoolManager extends Closeable {
|
|||
* @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;
|
||||
}
|
||||
|
|
|
@ -20,8 +20,10 @@ package org.apache.hadoop.ozone.scm.node;
|
|||
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 org.slf4j.LoggerFactory;
|
|||
|
||||
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 class SCMNodeManager
|
|||
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 @@ public class SCMNodeManager
|
|||
* @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 class SCMNodeManager
|
|||
* @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 class SCMNodeManager
|
|||
/**
|
||||
* 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 class SCMNodeManager
|
|||
/**
|
||||
* 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 class SCMNodeManager
|
|||
// 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 class SCMNodeManager
|
|||
// 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 class SCMNodeManager
|
|||
// 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 @@ public class SCMNodeManager
|
|||
*
|
||||
* @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 @@ public class SCMNodeManager
|
|||
|
||||
if (scmManager != null) {
|
||||
// remove stale node's container report
|
||||
scmManager.removeContainerReport(entry.getKey());
|
||||
scmManager.removeContainerReport(entry.getKey().toString());
|
||||
}
|
||||
}
|
||||
|
||||
|
@ -573,7 +578,7 @@ public class SCMNodeManager
|
|||
*
|
||||
* @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 @@ public class SCMNodeManager
|
|||
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 @@ public class SCMNodeManager
|
|||
if (healthyNodes.containsKey(datanodeUuid)) {
|
||||
healthyNodes.put(datanodeUuid, processTimestamp);
|
||||
updateNodeStat(datanodeUuid, nodeReport);
|
||||
updateCommandQueue(datanodeID,
|
||||
updateCommandQueue(datanodeUuid,
|
||||
hbItem.getContainerReportState().getState());
|
||||
return;
|
||||
}
|
||||
|
@ -623,7 +628,7 @@ public class SCMNodeManager
|
|||
healthyNodeCount.incrementAndGet();
|
||||
staleNodeCount.decrementAndGet();
|
||||
updateNodeStat(datanodeUuid, nodeReport);
|
||||
updateCommandQueue(datanodeID,
|
||||
updateCommandQueue(datanodeUuid,
|
||||
hbItem.getContainerReportState().getState());
|
||||
return;
|
||||
}
|
||||
|
@ -636,22 +641,22 @@ public class SCMNodeManager
|
|||
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 @@ public class SCMNodeManager
|
|||
}
|
||||
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 class SCMNodeManager
|
|||
* 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 class SCMNodeManager
|
|||
// 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 class SCMNodeManager
|
|||
.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 class SCMNodeManager
|
|||
/**
|
||||
* 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 @@ public class SCMNodeManager
|
|||
/**
|
||||
* 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 class SCMNodeManager
|
|||
LOG.error("Datanode ID in heartbeat is null");
|
||||
}
|
||||
|
||||
return commandQueue.getCommand(datanodeID);
|
||||
return commandQueue.getCommand(datanodeDetails.getUuid());
|
||||
}
|
||||
|
||||
/**
|
||||
|
@ -851,18 +870,18 @@ public class SCMNodeManager
|
|||
* @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 class SCMNodeManager
|
|||
}
|
||||
|
||||
@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
|
||||
|
|
|
@ -20,9 +20,9 @@ package org.apache.hadoop.ozone.scm.node;
|
|||
|
||||
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 @@ public final class SCMNodePoolManager implements NodePoolManager {
|
|||
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 @@ public final class SCMNodePoolManager implements NodePoolManager {
|
|||
* @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 final class SCMNodePoolManager implements NodePoolManager {
|
|||
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 final class SCMNodePoolManager implements NodePoolManager {
|
|||
* @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 final class SCMNodePoolManager implements NodePoolManager {
|
|||
|
||||
// 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 final class SCMNodePoolManager implements NodePoolManager {
|
|||
* @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 final class SCMNodePoolManager implements NodePoolManager {
|
|||
|
||||
/**
|
||||
* 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);
|
||||
}
|
||||
}
|
||||
|
|
|
@ -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 @@ public abstract class PipelineManager {
|
|||
* @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 class PipelineManager {
|
|||
* 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;
|
||||
}
|
||||
|
|
|
@ -18,7 +18,7 @@ package org.apache.hadoop.ozone.scm.pipelines;
|
|||
|
||||
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 class PipelineSelector {
|
|||
* 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 class PipelineSelector {
|
|||
*/
|
||||
|
||||
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 class PipelineSelector {
|
|||
* 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 class PipelineSelector {
|
|||
*/
|
||||
|
||||
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);
|
||||
}
|
||||
|
|
|
@ -18,7 +18,7 @@ package org.apache.hadoop.ozone.scm.pipelines.ratis;
|
|||
|
||||
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 class RatisManagerImpl extends PipelineManager {
|
|||
* @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 class RatisManagerImpl extends PipelineManager {
|
|||
* @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 class RatisManagerImpl extends PipelineManager {
|
|||
* @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 class RatisManagerImpl extends PipelineManager {
|
|||
* @param newDatanodes
|
||||
*/
|
||||
@Override
|
||||
public void updatePipeline(String pipelineID, List<DatanodeID> newDatanodes)
|
||||
public void updatePipeline(String pipelineID,
|
||||
List<DatanodeDetails> newDatanodes)
|
||||
throws IOException {
|
||||
|
||||
}
|
||||
|
|
|
@ -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 class StandaloneManagerImpl extends PipelineManager {
|
|||
* @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 class StandaloneManagerImpl extends PipelineManager {
|
|||
* @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 class StandaloneManagerImpl extends PipelineManager {
|
|||
* @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 class StandaloneManagerImpl extends PipelineManager {
|
|||
* @param newDatanodes
|
||||
*/
|
||||
@Override
|
||||
public void updatePipeline(String pipelineID, List<DatanodeID>
|
||||
public void updatePipeline(String pipelineID, List<DatanodeDetails>
|
||||
newDatanodes) throws IOException {
|
||||
|
||||
}
|
||||
|
|
|
@ -19,8 +19,8 @@ package org.apache.hadoop.ozone.container.common;
|
|||
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 @@ import org.apache.hadoop.ozone.container.common.states.endpoint
|
|||
.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 @@ import org.apache.hadoop.hdsl.protocol.proto
|
|||
.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 class TestEndPoint {
|
|||
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 @@ public class TestEndPoint {
|
|||
rpcEndPoint.setState(EndpointStateMachine.EndPointStates.REGISTER);
|
||||
RegisterEndpointTask endpointTask =
|
||||
new RegisterEndpointTask(rpcEndPoint, conf);
|
||||
if (!clearContainerID) {
|
||||
ContainerNodeIDProto containerNodeID = ContainerNodeIDProto.newBuilder()
|
||||
.setClusterID(UUID.randomUUID().toString())
|
||||
.setDatanodeID(getDatanodeID().getProtoBufMessage())
|
||||
.build();
|
||||
endpointTask.setContainerNodeIDProto(containerNodeID);
|
||||
if (!clearDatanodeDetails) {
|
||||
HdslProtos.DatanodeDetailsProto datanodeDetails =
|
||||
HdslProtos.DatanodeDetailsProto.newBuilder()
|
||||
.setUuid(UUID.randomUUID().toString())
|
||||
.build();
|
||||
endpointTask.setDatanodeDetailsProto(datanodeDetails);
|
||||
}
|
||||
endpointTask.call();
|
||||
return rpcEndPoint;
|
||||
|
@ -287,7 +286,7 @@ public class TestEndPoint {
|
|||
|
||||
@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 class TestEndPoint {
|
|||
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 @@ public class TestEndPoint {
|
|||
|
||||
// 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 @@ public class TestEndPoint {
|
|||
|
||||
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 @@ public class TestEndPoint {
|
|||
reportsBuilder.addReports(getRandomContainerReport()
|
||||
.getProtoBufMessage());
|
||||
}
|
||||
reportsBuilder.setDatanodeID(getDatanodeID()
|
||||
reportsBuilder.setDatanodeDetails(getDatanodeDetails()
|
||||
.getProtoBufMessage());
|
||||
reportsBuilder.setType(StorageContainerDatanodeProtocolProtos
|
||||
.ContainerReportsRequestProto.reportType.fullReport);
|
||||
|
@ -456,7 +455,7 @@ public class TestEndPoint {
|
|||
|
||||
reportsBuilder.addReports(report.getProtoBufMessage());
|
||||
}
|
||||
reportsBuilder.setDatanodeID(getDatanodeID()
|
||||
reportsBuilder.setDatanodeDetails(getDatanodeDetails()
|
||||
.getProtoBufMessage());
|
||||
reportsBuilder.setType(StorageContainerDatanodeProtocolProtos
|
||||
.ContainerReportsRequestProto.reportType.fullReport);
|
||||
|
|
|
@ -18,7 +18,7 @@ package org.apache.hadoop.ozone.container.placement;
|
|||
|
||||
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 class TestContainerPlacement {
|
|||
|
||||
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 class TestContainerPlacement {
|
|||
}
|
||||
|
||||
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);
|
||||
}
|
||||
}
|
||||
}
|
||||
|
|
|
@ -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 class TestContainerSupervisor {
|
|||
@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 class TestContainerSupervisor {
|
|||
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 class TestContainerSupervisor {
|
|||
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 class TestContainerSupervisor {
|
|||
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();
|
||||
}
|
||||
|
|
|
@ -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 class ReplicationDatanodeStateManager {
|
|||
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 class ReplicationDatanodeStateManager {
|
|||
|
||||
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 class ReplicationDatanodeStateManager {
|
|||
.build();
|
||||
ContainerReportsRequestProto containerReport =
|
||||
ContainerReportsRequestProto.newBuilder().addReports(info)
|
||||
.setDatanodeID(id.getProtoBufMessage())
|
||||
.setDatanodeDetails(id.getProtoBufMessage())
|
||||
.setType(ContainerReportsRequestProto.reportType.fullReport)
|
||||
.build();
|
||||
containerList.add(containerReport);
|
||||
|
|
|
@ -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 org.apache.hadoop.ozone.scm.node.NodeManager;
|
|||
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 class ReplicationNodeManagerMock implements NodeManager {
|
|||
* @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 class ReplicationNodeManagerMock implements NodeManager {
|
|||
* @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 class ReplicationNodeManagerMock implements NodeManager {
|
|||
/**
|
||||
* 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 class ReplicationNodeManagerMock implements NodeManager {
|
|||
* @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 class ReplicationNodeManagerMock implements NodeManager {
|
|||
/**
|
||||
* 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 class ReplicationNodeManagerMock implements NodeManager {
|
|||
/**
|
||||
* 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 class ReplicationNodeManagerMock implements NodeManager {
|
|||
/**
|
||||
* 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);
|
||||
}
|
||||
|
||||
}
|
||||
|
|
|
@ -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 @@ import java.util.Set;
|
|||
*/
|
||||
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 class ReplicationNodePoolManagerMock implements NodePoolManager {
|
|||
* @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 class ReplicationNodePoolManagerMock implements NodePoolManager {
|
|||
* @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 class ReplicationNodePoolManagerMock implements NodePoolManager {
|
|||
@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 class ReplicationNodePoolManagerMock implements NodePoolManager {
|
|||
* 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 class ReplicationNodePoolManagerMock implements NodePoolManager {
|
|||
/**
|
||||
* 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);
|
||||
}
|
||||
|
||||
/**
|
||||
|
|
|
@ -21,8 +21,7 @@ import java.util.List;
|
|||
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;
|
||||
}
|
||||
|
|
|
@ -112,8 +112,8 @@ public class TestBlockManager {
|
|||
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 class TestBlockManager {
|
|||
// 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
|
||||
|
|
|
@ -19,8 +19,8 @@ package org.apache.hadoop.ozone.scm.block;
|
|||
|
||||
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 class TestDeletedBlockLog {
|
|||
|
||||
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 class TestDeletedBlockLog {
|
|||
|
||||
// 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 class TestDeletedBlockLog {
|
|||
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 class TestDeletedBlockLog {
|
|||
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 class TestDeletedBlockLog {
|
|||
|
||||
// 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 class TestDeletedBlockLog {
|
|||
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();
|
||||
|
|
|
@ -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.container.placement.metrics.SCMNodeStat;
|
|||
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.HashMap;
|
|||
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 class MockNodeManager implements NodeManager {
|
|||
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 class MockNodeManager implements NodeManager {
|
|||
/**
|
||||
* 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 class MockNodeManager implements NodeManager {
|
|||
* @throws UnregisteredNodeException
|
||||
*/
|
||||
@Override
|
||||
public void removeNode(DatanodeID node) throws UnregisteredNodeException {
|
||||
public void removeNode(DatanodeDetails node)
|
||||
throws UnregisteredNodeException {
|
||||
|
||||
}
|
||||
|
||||
|
@ -148,7 +150,7 @@ public class MockNodeManager implements NodeManager {
|
|||
* @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 class MockNodeManager implements NodeManager {
|
|||
*/
|
||||
@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 class MockNodeManager implements NodeManager {
|
|||
/**
|
||||
* 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 class MockNodeManager implements NodeManager {
|
|||
* @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 class MockNodeManager implements NodeManager {
|
|||
/**
|
||||
* 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 class MockNodeManager implements NodeManager {
|
|||
* 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 class MockNodeManager implements NodeManager {
|
|||
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 class MockNodeManager implements NodeManager {
|
|||
/**
|
||||
* 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);
|
||||
}
|
||||
}
|
||||
|
||||
|
|
|
@ -18,7 +18,7 @@ package org.apache.hadoop.ozone.scm.container;
|
|||
|
||||
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.ContainerInfo;
|
|||
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 class TestContainerMapping {
|
|||
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 class TestContainerMapping {
|
|||
Assert.assertNotNull(containerInfo);
|
||||
Assert.assertNotNull(containerInfo.getPipeline());
|
||||
pipelineList.add(containerInfo.getPipeline().getLeader()
|
||||
.getDatanodeUuid());
|
||||
.getUuid());
|
||||
}
|
||||
Assert.assertTrue(pipelineList.size() > 5);
|
||||
}
|
||||
|
@ -142,8 +141,8 @@ public class TestContainerMapping {
|
|||
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 class TestContainerMapping {
|
|||
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 class TestContainerMapping {
|
|||
|
||||
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 class TestContainerMapping {
|
|||
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 class TestContainerMapping {
|
|||
|
||||
ContainerReportsRequestProto.Builder crBuilder =
|
||||
ContainerReportsRequestProto.newBuilder();
|
||||
crBuilder.setDatanodeID(datanodeID.getProtoBufMessage())
|
||||
crBuilder.setDatanodeDetails(datanodeDetails.getProtoBufMessage())
|
||||
.setType(reportType).addAllReports(reports);
|
||||
|
||||
mapping.processContainerReports(crBuilder.build());
|
||||
|
|
|
@ -21,13 +21,13 @@ package org.apache.hadoop.ozone.scm.container.closer;
|
|||
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 class TestContainerCloser {
|
|||
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 class TestContainerCloser {
|
|||
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 class TestContainerCloser {
|
|||
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 class TestContainerCloser {
|
|||
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 class TestContainerCloser {
|
|||
|
||||
// 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 class TestContainerCloser {
|
|||
//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 @@ public class TestContainerCloser {
|
|||
.setReadBytes(2000000000L)
|
||||
.setWriteBytes(2000000000L)
|
||||
.setContainerID(1L);
|
||||
reports.setDatanodeID(
|
||||
DFSTestUtil.getLocalDatanodeID().getProtoBufMessage());
|
||||
reports.setDatanodeDetails(
|
||||
TestUtils.getDatanodeDetails().getProtoBufMessage());
|
||||
reports.addReports(ciBuilder);
|
||||
mapping.processContainerReports(reports.build());
|
||||
}
|
||||
|
|
|
@ -21,11 +21,10 @@ package org.apache.hadoop.ozone.scm.node;
|
|||
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 class TestContainerPlacement {
|
|||
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);
|
||||
}
|
||||
|
||||
|
|
|
@ -20,10 +20,9 @@ package org.apache.hadoop.ozone.scm.node;
|
|||
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.scm.ScmConfigKeys;
|
|||
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_DEADNODE_INTERVAL;
|
|||
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 class TestNodeManager {
|
|||
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 class TestNodeManager {
|
|||
|
||||
// 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 class TestNodeManager {
|
|||
|
||||
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 class TestNodeManager {
|
|||
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 class TestNodeManager {
|
|||
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 class TestNodeManager {
|
|||
// 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 class TestNodeManager {
|
|||
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 class TestNodeManager {
|
|||
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 class TestNodeManager {
|
|||
|
||||
|
||||
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 class TestNodeManager {
|
|||
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 class TestNodeManager {
|
|||
* 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 class TestNodeManager {
|
|||
* 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 class TestNodeManager {
|
|||
// 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 class TestNodeManager {
|
|||
* 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 class TestNodeManager {
|
|||
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 class TestNodeManager {
|
|||
* @param sleepDuration - Duration to sleep between heartbeats.
|
||||
* @throws InterruptedException
|
||||
*/
|
||||
private void heartbeatNodeSet(SCMNodeManager manager, List<DatanodeID> list,
|
||||
int sleepDuration) throws InterruptedException {
|
||||
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 @@ public class TestNodeManager {
|
|||
* @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 class TestNodeManager {
|
|||
|
||||
|
||||
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 class TestNodeManager {
|
|||
|
||||
// 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 class TestNodeManager {
|
|||
|
||||
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 class TestNodeManager {
|
|||
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 class TestNodeManager {
|
|||
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 class TestNodeManager {
|
|||
|
||||
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 class TestNodeManager {
|
|||
|
||||
// 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 class TestNodeManager {
|
|||
|
||||
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 class TestNodeManager {
|
|||
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 class TestNodeManager {
|
|||
.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 class TestNodeManager {
|
|||
|
||||
// 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 class TestNodeManager {
|
|||
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 class TestNodeManager {
|
|||
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 class TestNodeManager {
|
|||
() -> 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);
|
||||
}
|
||||
|
|
|
@ -20,16 +20,15 @@ package org.apache.hadoop.ozone.scm.node;
|
|||
|
||||
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 class TestSCMNodePoolManager {
|
|||
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 class TestSCMNodePoolManager {
|
|||
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 class TestSCMNodePoolManager {
|
|||
// 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 class TestSCMNodePoolManager {
|
|||
* @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);
|
||||
|
|
|
@ -23,8 +23,8 @@ import org.apache.commons.cli.HelpFormatter;
|
|||
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 class InfoContainerHandler extends OzoneCommandHandler {
|
|||
// 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);
|
||||
}
|
||||
|
||||
|
|
|
@ -119,8 +119,8 @@ public final class OzoneUtils {
|
|||
* @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);
|
||||
}
|
||||
|
||||
/**
|
||||
|
|
|
@ -23,8 +23,10 @@ import com.google.common.base.Preconditions;
|
|||
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.security.UserGroupInformation;
|
|||
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 @@ public final class MiniOzoneClassicCluster extends MiniDFSCluster
|
|||
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 final class MiniOzoneClassicCluster extends MiniDFSCluster
|
|||
// 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 final class MiniOzoneClassicCluster extends MiniDFSCluster
|
|||
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 final class MiniOzoneClassicCluster extends MiniDFSCluster
|
|||
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 @@ public final class MiniOzoneClassicCluster extends MiniDFSCluster
|
|||
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 {
|
||||
|
|
|
@ -46,10 +46,10 @@ public class MiniOzoneTestHelper {
|
|||
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"
|
||||
|
|
|
@ -80,7 +80,8 @@ public interface RatisTestHelper {
|
|||
}
|
||||
|
||||
public int getDatanodeOzoneRestPort() {
|
||||
return cluster.getDataNodes().get(0).getDatanodeId().getOzoneRestPort();
|
||||
return MiniOzoneClassicCluster.getOzoneRestPort(
|
||||
cluster.getDataNodes().get(0));
|
||||
}
|
||||
}
|
||||
|
||||
|
|
|
@ -20,22 +20,23 @@ package org.apache.hadoop.ozone;
|
|||
|
||||
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 class TestMiniOzoneCluster {
|
|||
.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 class TestMiniOzoneCluster {
|
|||
@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 class TestMiniOzoneCluster {
|
|||
// 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 class TestMiniOzoneCluster {
|
|||
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 class TestMiniOzoneCluster {
|
|||
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 class TestMiniOzoneCluster {
|
|||
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());
|
||||
|
|
|
@ -309,7 +309,8 @@ public class TestStorageContainerManager {
|
|||
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) {
|
||||
|
|
|
@ -23,9 +23,9 @@ import org.apache.commons.lang.RandomStringUtils;
|
|||
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 @@ public class TestStorageContainerManagerHelper {
|
|||
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);
|
||||
}
|
||||
}
|
||||
|
|
|
@ -21,6 +21,7 @@ package org.apache.hadoop.ozone.container;
|
|||
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.LifeCycleState;
|
|||
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 final class ContainerTestHelper {
|
|||
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 final class ContainerTestHelper {
|
|||
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 final class ContainerTestHelper {
|
|||
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 final class ContainerTestHelper {
|
|||
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 final class ContainerTestHelper {
|
|||
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 final class ContainerTestHelper {
|
|||
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 final class ContainerTestHelper {
|
|||
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 final class ContainerTestHelper {
|
|||
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 final class ContainerTestHelper {
|
|||
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 final class ContainerTestHelper {
|
|||
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 final class ContainerTestHelper {
|
|||
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 final class ContainerTestHelper {
|
|||
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 final class ContainerTestHelper {
|
|||
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 final class ContainerTestHelper {
|
|||
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 final class ContainerTestHelper {
|
|||
.setCmdType(ContainerProtos.Type.DeleteContainer)
|
||||
.setDeleteContainer(deleteRequest)
|
||||
.setTraceID(UUID.randomUUID().toString())
|
||||
.setDatanodeID(pipeline.getLeader().getDatanodeUuid())
|
||||
.setDatanodeUuid(pipeline.getLeader().getUuidString())
|
||||
.build();
|
||||
}
|
||||
}
|
||||
|
|
|
@ -20,7 +20,6 @@ package org.apache.hadoop.ozone.container.common;
|
|||
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.ContainerManagerImpl;
|
|||
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 @@ public class TestBlockDeletingService {
|
|||
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;
|
||||
}
|
||||
|
||||
|
|
|
@ -28,15 +28,15 @@ import java.util.Map;
|
|||
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 class TestContainerDeletionChoosingPolicy {
|
|||
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 class TestContainerDeletionChoosingPolicy {
|
|||
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 class TestContainerDeletionChoosingPolicy {
|
|||
containerManager.writeLock();
|
||||
containerManager.shutdown();
|
||||
containerManager.writeUnlock();
|
||||
containerManager.init(conf, pathLists, datanodeID);
|
||||
containerManager.init(conf, pathLists, datanodeDetails);
|
||||
|
||||
List<ContainerData> result0 = containerManager
|
||||
.chooseContainerForBlockDeletion(5);
|
||||
|
|
|
@ -21,12 +21,12 @@ import org.apache.commons.codec.binary.Hex;
|
|||
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 class TestContainerPersistence {
|
|||
FileUtils.forceMkdir(new File(location.getNormalizedUri()));
|
||||
}
|
||||
|
||||
containerManager.init(conf, pathLists, DFSTestUtil.getLocalDatanodeID());
|
||||
containerManager.init(conf, pathLists, TestUtils.getDatanodeDetails());
|
||||
}
|
||||
|
||||
@After
|
||||
|
|
|
@ -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 class TestCloseContainerHandler {
|
|||
|
||||
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),
|
||||
|
|
|
@ -24,6 +24,7 @@ import static org.mockito.Mockito.mock;
|
|||
|
||||
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.impl.Dispatcher;
|
|||
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 class TestContainerMetrics {
|
|||
|
||||
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();
|
||||
|
|
|
@ -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.ozone.OzoneConfigKeys;
|
|||
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 class TestOzoneContainer {
|
|||
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);
|
||||
|
|
|
@ -87,7 +87,8 @@ public class TestOzoneContainerRatis {
|
|||
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 @@ public class TestOzoneContainerRatis {
|
|||
// 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
|
||||
|
|
|
@ -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 @@ public class TestRatisManager {
|
|||
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 @@ public class TestRatisManager {
|
|||
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 @@ public class TestRatisManager {
|
|||
//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();
|
||||
|
|
|
@ -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.XceiverServer;
|
|||
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 class TestContainerServer {
|
|||
|
||||
@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 class TestContainerServer {
|
|||
}
|
||||
|
||||
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 @@ public class TestContainerServer {
|
|||
}
|
||||
|
||||
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 @@ public class TestContainerServer {
|
|||
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 @@ public class TestContainerServer {
|
|||
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 class TestContainerServer {
|
|||
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();
|
||||
|
|
|
@ -124,7 +124,7 @@ public class TestKeySpaceManagerRestInterface {
|
|||
switch (type) {
|
||||
case HTTP:
|
||||
case HTTPS:
|
||||
Assert.assertEquals(datanode.getDatanodeId().getOzoneRestPort(),
|
||||
Assert.assertEquals(MiniOzoneClassicCluster.getOzoneRestPort(datanode),
|
||||
(int) ports.get(type));
|
||||
break;
|
||||
default:
|
||||
|
|
|
@ -117,7 +117,7 @@ public class TestOzoneShell {
|
|||
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);
|
||||
|
|
|
@ -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 class TestContainerSQLCli {
|
|||
|
||||
// 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 class TestContainerSQLCli {
|
|||
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 class TestContainerSQLCli {
|
|||
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));
|
||||
}
|
||||
|
|
|
@ -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 class TestSCMCli {
|
|||
@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 class TestSCMCli {
|
|||
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 class TestSCMCli {
|
|||
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 class TestSCMCli {
|
|||
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());
|
||||
}
|
||||
|
||||
|
|
|
@ -26,12 +26,11 @@ import java.util.UUID;
|
|||
|
||||
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 class TestSCMMetrics {
|
|||
|
||||
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 class TestSCMMetrics {
|
|||
|
||||
// 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 class TestSCMMetrics {
|
|||
// 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 class TestSCMMetrics {
|
|||
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 @@ public class TestSCMMetrics {
|
|||
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();
|
||||
|
|
|
@ -67,7 +67,7 @@ public class TestDistributedOzoneVolumes extends TestOzoneHelper {
|
|||
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);
|
||||
}
|
||||
|
||||
/**
|
||||
|
|
|
@ -70,7 +70,7 @@ public class TestLocalOzoneVolumes extends TestOzoneHelper {
|
|||
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);
|
||||
}
|
||||
|
||||
/**
|
||||
|
|
|
@ -79,7 +79,7 @@ public class TestOzoneWebAccess {
|
|||
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);
|
||||
}
|
||||
|
||||
/**
|
||||
|
|
|
@ -80,7 +80,7 @@ public class TestBuckets {
|
|||
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));
|
||||
}
|
||||
|
|
|
@ -111,7 +111,7 @@ public class TestKeys {
|
|||
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 @@ public class TestKeys {
|
|||
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));
|
||||
}
|
||||
|
||||
|
|
|
@ -98,7 +98,7 @@ public class TestOzoneClient {
|
|||
.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
|
||||
|
|
|
@ -90,7 +90,7 @@ public class TestVolume {
|
|||
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
Loading…
Reference in New Issue