Explorar o código

HDDS-88. Create separate message structure to represent ports in DatanodeDetails.
Contributed by Nanda Kumar.

Anu Engineer %!s(int64=7) %!d(string=hai) anos
pai
achega
3b34148c4f
Modificáronse 30 ficheiros con 260 adicións e 153 borrados
  1. 1 1
      hadoop-hdds/client/src/main/java/org/apache/hadoop/hdds/scm/XceiverClient.java
  2. 1 1
      hadoop-hdds/client/src/main/java/org/apache/hadoop/hdds/scm/XceiverClientGrpc.java
  3. 130 89
      hadoop-hdds/common/src/main/java/org/apache/hadoop/hdds/protocol/DatanodeDetails.java
  4. 4 2
      hadoop-hdds/common/src/main/java/org/apache/ratis/RatisHelper.java
  5. 6 4
      hadoop-hdds/common/src/main/proto/hdds.proto
  6. 2 1
      hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/common/transport/server/XceiverServer.java
  7. 2 1
      hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/common/transport/server/XceiverServerGrpc.java
  8. 2 1
      hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/common/transport/server/ratis/XceiverServerRatis.java
  9. 12 4
      hadoop-hdds/container-service/src/test/java/org/apache/hadoop/ozone/container/common/TestDatanodeStateMachine.java
  10. 9 3
      hadoop-hdds/server-scm/src/test/java/org/apache/hadoop/hdds/scm/TestUtils.java
  11. 12 6
      hadoop-hdds/server-scm/src/test/java/org/apache/hadoop/hdds/scm/block/TestDeletedBlockLog.java
  12. 10 7
      hadoop-ozone/integration-test/src/test/java/org/apache/hadoop/ozone/MiniOzoneClusterImpl.java
  13. 2 1
      hadoop-ozone/integration-test/src/test/java/org/apache/hadoop/ozone/RatisTestHelper.java
  14. 4 4
      hadoop-ozone/integration-test/src/test/java/org/apache/hadoop/ozone/TestMiniOzoneCluster.java
  15. 9 3
      hadoop-ozone/integration-test/src/test/java/org/apache/hadoop/ozone/container/ContainerTestHelper.java
  16. 2 1
      hadoop-ozone/integration-test/src/test/java/org/apache/hadoop/ozone/container/metrics/TestContainerMetrics.java
  17. 7 4
      hadoop-ozone/integration-test/src/test/java/org/apache/hadoop/ozone/container/ozoneimpl/TestOzoneContainer.java
  18. 5 3
      hadoop-ozone/integration-test/src/test/java/org/apache/hadoop/ozone/container/server/TestContainerServer.java
  19. 3 2
      hadoop-ozone/integration-test/src/test/java/org/apache/hadoop/ozone/ksm/TestKeySpaceManagerRestInterface.java
  20. 3 1
      hadoop-ozone/integration-test/src/test/java/org/apache/hadoop/ozone/web/TestDistributedOzoneVolumes.java
  21. 3 1
      hadoop-ozone/integration-test/src/test/java/org/apache/hadoop/ozone/web/TestLocalOzoneVolumes.java
  22. 3 1
      hadoop-ozone/integration-test/src/test/java/org/apache/hadoop/ozone/web/TestOzoneWebAccess.java
  23. 3 1
      hadoop-ozone/integration-test/src/test/java/org/apache/hadoop/ozone/web/client/TestOzoneClient.java
  24. 0 2
      hadoop-ozone/integration-test/src/test/java/org/apache/hadoop/ozone/web/client/TestVolume.java
  25. 3 1
      hadoop-ozone/integration-test/src/test/java/org/apache/hadoop/ozone/web/client/TestVolumeRatis.java
  26. 4 1
      hadoop-ozone/objectstore-service/src/main/java/org/apache/hadoop/ozone/web/OzoneHddsDatanodeService.java
  27. 3 1
      hadoop-ozone/ozone-manager/src/main/java/org/apache/hadoop/ozone/ksm/KeySpaceManager.java
  28. 9 3
      hadoop-ozone/tools/src/main/java/org/apache/hadoop/ozone/genesis/GenesisUtil.java
  29. 4 2
      hadoop-ozone/tools/src/main/java/org/apache/hadoop/ozone/scm/cli/SQLCLI.java
  30. 2 1
      hadoop-tools/hadoop-ozone/src/test/java/org/apache/hadoop/fs/ozone/TestOzoneFSInputStream.java

+ 1 - 1
hadoop-hdds/client/src/main/java/org/apache/hadoop/hdds/scm/XceiverClient.java

@@ -93,7 +93,7 @@ public class XceiverClient extends XceiverClientSpi {
 
 
     // read port from the data node, on failure use default configured
     // read port from the data node, on failure use default configured
     // port.
     // port.
-    int port = leader.getContainerPort();
+    int port = leader.getPort(DatanodeDetails.Port.Name.STANDALONE).getValue();
     if (port == 0) {
     if (port == 0) {
       port = config.getInt(OzoneConfigKeys.DFS_CONTAINER_IPC_PORT,
       port = config.getInt(OzoneConfigKeys.DFS_CONTAINER_IPC_PORT,
           OzoneConfigKeys.DFS_CONTAINER_IPC_PORT_DEFAULT);
           OzoneConfigKeys.DFS_CONTAINER_IPC_PORT_DEFAULT);

+ 1 - 1
hadoop-hdds/client/src/main/java/org/apache/hadoop/hdds/scm/XceiverClientGrpc.java

@@ -80,7 +80,7 @@ public class XceiverClientGrpc extends XceiverClientSpi {
 
 
     // read port from the data node, on failure use default configured
     // read port from the data node, on failure use default configured
     // port.
     // port.
-    int port = leader.getContainerPort();
+    int port = leader.getPort(DatanodeDetails.Port.Name.STANDALONE).getValue();
     if (port == 0) {
     if (port == 0) {
       port = config.getInt(OzoneConfigKeys.DFS_CONTAINER_IPC_PORT,
       port = config.getInt(OzoneConfigKeys.DFS_CONTAINER_IPC_PORT,
           OzoneConfigKeys.DFS_CONTAINER_IPC_PORT_DEFAULT);
           OzoneConfigKeys.DFS_CONTAINER_IPC_PORT_DEFAULT);

+ 130 - 89
hadoop-hdds/common/src/main/java/org/apache/hadoop/hdds/protocol/DatanodeDetails.java

@@ -23,6 +23,8 @@ import org.apache.hadoop.classification.InterfaceAudience;
 import org.apache.hadoop.classification.InterfaceStability;
 import org.apache.hadoop.classification.InterfaceStability;
 import org.apache.hadoop.hdds.protocol.proto.HddsProtos;
 import org.apache.hadoop.hdds.protocol.proto.HddsProtos;
 
 
+import java.util.ArrayList;
+import java.util.List;
 import java.util.UUID;
 import java.util.UUID;
 
 
 /**
 /**
@@ -42,9 +44,7 @@ public final class DatanodeDetails implements Comparable<DatanodeDetails> {
 
 
   private String ipAddress;
   private String ipAddress;
   private String hostName;
   private String hostName;
-  private Integer containerPort;
-  private Integer ratisPort;
-  private Integer ozoneRestPort;
+  private List<Port> ports;
 
 
 
 
   /**
   /**
@@ -53,18 +53,14 @@ public final class DatanodeDetails implements Comparable<DatanodeDetails> {
    * @param uuid DataNode's UUID
    * @param uuid DataNode's UUID
    * @param ipAddress IP Address of this DataNode
    * @param ipAddress IP Address of this DataNode
    * @param hostName DataNode's hostname
    * @param hostName DataNode's hostname
-   * @param containerPort Container Port
-   * @param ratisPort Ratis Port
-   * @param ozoneRestPort Rest Port
+   * @param ports Ports used by the DataNode
    */
    */
   private DatanodeDetails(String uuid, String ipAddress, String hostName,
   private DatanodeDetails(String uuid, String ipAddress, String hostName,
-      Integer containerPort, Integer ratisPort, Integer ozoneRestPort) {
+      List<Port> ports) {
     this.uuid = UUID.fromString(uuid);
     this.uuid = UUID.fromString(uuid);
     this.ipAddress = ipAddress;
     this.ipAddress = ipAddress;
     this.hostName = hostName;
     this.hostName = hostName;
-    this.containerPort = containerPort;
-    this.ratisPort = ratisPort;
-    this.ozoneRestPort = ozoneRestPort;
+    this.ports = ports;
   }
   }
 
 
   /**
   /**
@@ -122,54 +118,40 @@ public final class DatanodeDetails implements Comparable<DatanodeDetails> {
   }
   }
 
 
   /**
   /**
-   * Sets the Container Port.
-   * @param port ContainerPort
-   */
-  public void setContainerPort(int port) {
-    containerPort = port;
-  }
-
-  /**
-   * Returns standalone container Port.
+   * Sets a DataNode Port.
    *
    *
-   * @return Container Port
+   * @param port DataNode port
    */
    */
-  public int getContainerPort() {
-    return containerPort;
+  public void setPort(Port port) {
+    // If the port is already in the list remove it first and add the
+    // new/updated port value.
+    ports.remove(port);
+    ports.add(port);
   }
   }
 
 
   /**
   /**
-   * 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
+   * Returns all the Ports used by DataNode.
+   *
+   * @return DataNode Ports
    */
    */
-  public void setOzoneRestPort(int port) {
-    ozoneRestPort = port;
+  public List<Port> getPorts() {
+    return ports;
   }
   }
 
 
   /**
   /**
-   * Returns Ozone Rest Port.
-   * @return OzoneRestPort
+   * Given the name returns port number, null if the asked port is not found.
+   *
+   * @param name Name of the port
+   *
+   * @return Port
    */
    */
-  public int getOzoneRestPort() {
-    return ozoneRestPort;
+  public Port getPort(Port.Name name) {
+    for (Port port : ports) {
+      if (port.getName().equals(name)) {
+        return port;
+      }
+    }
+    return null;
   }
   }
 
 
   /**
   /**
@@ -188,14 +170,9 @@ public final class DatanodeDetails implements Comparable<DatanodeDetails> {
     if (datanodeDetailsProto.hasHostName()) {
     if (datanodeDetailsProto.hasHostName()) {
       builder.setHostName(datanodeDetailsProto.getHostName());
       builder.setHostName(datanodeDetailsProto.getHostName());
     }
     }
-    if (datanodeDetailsProto.hasContainerPort()) {
-      builder.setContainerPort(datanodeDetailsProto.getContainerPort());
-    }
-    if (datanodeDetailsProto.hasRatisPort()) {
-      builder.setRatisPort(datanodeDetailsProto.getRatisPort());
-    }
-    if (datanodeDetailsProto.hasOzoneRestPort()) {
-      builder.setOzoneRestPort(datanodeDetailsProto.getOzoneRestPort());
+    for (HddsProtos.Port port : datanodeDetailsProto.getPortsList()) {
+      builder.addPort(newPort(
+          Port.Name.valueOf(port.getName().toUpperCase()), port.getValue()));
     }
     }
     return builder.build();
     return builder.build();
   }
   }
@@ -214,14 +191,11 @@ public final class DatanodeDetails implements Comparable<DatanodeDetails> {
     if (hostName != null) {
     if (hostName != null) {
       builder.setHostName(hostName);
       builder.setHostName(hostName);
     }
     }
-    if (containerPort != null) {
-      builder.setContainerPort(containerPort);
-    }
-    if (ratisPort != null) {
-      builder.setRatisPort(ratisPort);
-    }
-    if (ozoneRestPort != null) {
-      builder.setOzoneRestPort(ozoneRestPort);
+    for (Port port : ports) {
+      builder.addPorts(HddsProtos.Port.newBuilder()
+          .setName(port.getName().toString())
+          .setValue(port.getValue())
+          .build());
     }
     }
     return builder.build();
     return builder.build();
   }
   }
@@ -268,9 +242,15 @@ public final class DatanodeDetails implements Comparable<DatanodeDetails> {
     private String id;
     private String id;
     private String ipAddress;
     private String ipAddress;
     private String hostName;
     private String hostName;
-    private Integer containerPort;
-    private Integer ratisPort;
-    private Integer ozoneRestPort;
+    private List<Port> ports;
+
+    /**
+     * Default private constructor. To create Builder instance use
+     * DatanodeDetails#newBuilder.
+     */
+    private Builder() {
+      ports = new ArrayList<>();
+    }
 
 
     /**
     /**
      * Sets the DatanodeUuid.
      * Sets the DatanodeUuid.
@@ -304,50 +284,111 @@ public final class DatanodeDetails implements Comparable<DatanodeDetails> {
       this.hostName = host;
       this.hostName = host;
       return this;
       return this;
     }
     }
+
     /**
     /**
-     * Sets the ContainerPort.
+     * Adds a DataNode Port.
+     *
+     * @param port DataNode port
      *
      *
-     * @param port ContainerPort
      * @return DatanodeDetails.Builder
      * @return DatanodeDetails.Builder
      */
      */
-    public Builder setContainerPort(Integer port) {
-      this.containerPort = port;
+    public Builder addPort(Port port) {
+      this.ports.add(port);
       return this;
       return this;
     }
     }
 
 
     /**
     /**
-     * Sets the RatisPort.
+     * Builds and returns DatanodeDetails instance.
      *
      *
-     * @param port RatisPort
-     * @return DatanodeDetails.Builder
+     * @return DatanodeDetails
      */
      */
-    public Builder setRatisPort(Integer port) {
-      this.ratisPort = port;
-      return this;
+    public DatanodeDetails build() {
+      Preconditions.checkNotNull(id);
+      return new DatanodeDetails(id, ipAddress, hostName, ports);
     }
     }
 
 
+  }
+
+  /**
+   * Constructs a new Port with name and value.
+   *
+   * @param name Name of the port
+   * @param value Port number
+   *
+   * @return {@code Port} instance
+   */
+  public static Port newPort(Port.Name name, Integer value) {
+    return new Port(name, value);
+  }
+
+  /**
+   * Container to hold DataNode Port details.
+   */
+  public static class Port {
+
+    /**
+     * Ports that are supported in DataNode.
+     */
+    public enum Name {
+      STANDALONE, RATIS, REST
+    }
+
+    private Name name;
+    private Integer value;
+
     /**
     /**
-     * Sets the OzoneRestPort.
+     * Private constructor for constructing Port object. Use
+     * DatanodeDetails#newPort to create a new Port object.
      *
      *
-     * @param port OzoneRestPort
-     * @return DatanodeDetails.Builder
+     * @param name
+     * @param value
      */
      */
-    public Builder setOzoneRestPort(Integer port) {
-      this.ozoneRestPort = port;
-      return this;
+    private Port(Name name, Integer value) {
+      this.name = name;
+      this.value = value;
     }
     }
 
 
     /**
     /**
-     * Builds and returns DatanodeDetails instance.
+     * Returns the name of the port.
      *
      *
-     * @return DatanodeDetails
+     * @return Port name
      */
      */
-    public DatanodeDetails build() {
-      Preconditions.checkNotNull(id);
-      return new DatanodeDetails(id, ipAddress, hostName, containerPort,
-          ratisPort, ozoneRestPort);
+    public Name getName() {
+      return name;
+    }
+
+    /**
+     * Returns the port number.
+     *
+     * @return Port number
+     */
+    public Integer getValue() {
+      return value;
+    }
+
+    @Override
+    public int hashCode() {
+      return name.hashCode();
     }
     }
 
 
+    /**
+     * Ports are considered equal if they have the same name.
+     *
+     * @param anObject
+     *          The object to compare this {@code Port} against
+     * @return {@code true} if the given object represents a {@code Port}
+               and has the same name, {@code false} otherwise
+     */
+    @Override
+    public boolean equals(Object anObject) {
+      if (this == anObject) {
+        return true;
+      }
+      if (anObject instanceof Port) {
+        return name.equals(((Port) anObject).name);
+      }
+      return false;
+    }
   }
   }
 
 
 }
 }

+ 4 - 2
hadoop-hdds/common/src/main/java/org/apache/ratis/RatisHelper.java

@@ -48,11 +48,13 @@ public interface RatisHelper {
   Logger LOG = LoggerFactory.getLogger(RatisHelper.class);
   Logger LOG = LoggerFactory.getLogger(RatisHelper.class);
 
 
   static String toRaftPeerIdString(DatanodeDetails id) {
   static String toRaftPeerIdString(DatanodeDetails id) {
-    return id.getUuidString() + "_" + id.getRatisPort();
+    return id.getUuidString() + "_" +
+        id.getPort(DatanodeDetails.Port.Name.RATIS);
   }
   }
 
 
   static String toRaftPeerAddressString(DatanodeDetails id) {
   static String toRaftPeerAddressString(DatanodeDetails id) {
-    return id.getIpAddress() + ":" + id.getRatisPort();
+    return id.getIpAddress() + ":" +
+        id.getPort(DatanodeDetails.Port.Name.RATIS);
   }
   }
 
 
   static RaftPeerId toRaftPeerId(DatanodeDetails id) {
   static RaftPeerId toRaftPeerId(DatanodeDetails id) {

+ 6 - 4
hadoop-hdds/common/src/main/proto/hdds.proto

@@ -29,13 +29,15 @@ option java_generate_equals_and_hash = true;
 package hadoop.hdds;
 package hadoop.hdds;
 
 
 message DatanodeDetailsProto {
 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 uuid = 1;  // UUID assigned to the Datanode.
     required string ipAddress = 2;     // IP address
     required string ipAddress = 2;     // IP address
     required string hostName = 3;      // hostname
     required string hostName = 3;      // hostname
-    optional uint32 containerPort = 4 [default = 0];  // Ozone stand_alone protocol
-    optional uint32 ratisPort = 5 [default = 0];      //Ozone ratis port
-    optional uint32 ozoneRestPort = 6 [default = 0];
+    repeated Port ports = 4;
+}
+
+message Port {
+    required string name = 1;
+    required uint32 value = 2;
 }
 }
 
 
 message PipelineChannel {
 message PipelineChannel {

+ 2 - 1
hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/common/transport/server/XceiverServer.java

@@ -80,7 +80,8 @@ public final class XceiverServer implements XceiverServerSpi {
             + "fallback to use default port {}", this.port, e);
             + "fallback to use default port {}", this.port, e);
       }
       }
     }
     }
-    datanodeDetails.setContainerPort(port);
+    datanodeDetails.setPort(
+        DatanodeDetails.newPort(DatanodeDetails.Port.Name.STANDALONE, port));
     this.storageContainer = dispatcher;
     this.storageContainer = dispatcher;
   }
   }
 
 

+ 2 - 1
hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/common/transport/server/XceiverServerGrpc.java

@@ -71,7 +71,8 @@ public final class XceiverServerGrpc implements XceiverServerSpi {
             + "fallback to use default port {}", this.port, e);
             + "fallback to use default port {}", this.port, e);
       }
       }
     }
     }
-    datanodeDetails.setContainerPort(port);
+    datanodeDetails.setPort(
+        DatanodeDetails.newPort(DatanodeDetails.Port.Name.STANDALONE, port));
     server = ((NettyServerBuilder) ServerBuilder.forPort(port))
     server = ((NettyServerBuilder) ServerBuilder.forPort(port))
         .maxMessageSize(OzoneConfigKeys.DFS_CONTAINER_CHUNK_MAX_SIZE)
         .maxMessageSize(OzoneConfigKeys.DFS_CONTAINER_CHUNK_MAX_SIZE)
         .addService(new GrpcXceiverService(dispatcher))
         .addService(new GrpcXceiverService(dispatcher))

+ 2 - 1
hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/common/transport/server/ratis/XceiverServerRatis.java

@@ -203,7 +203,8 @@ public final class XceiverServerRatis implements XceiverServerSpi {
             + "fallback to use default port {}", localPort, e);
             + "fallback to use default port {}", localPort, e);
       }
       }
     }
     }
-    datanodeDetails.setRatisPort(localPort);
+    datanodeDetails.setPort(
+        DatanodeDetails.newPort(DatanodeDetails.Port.Name.RATIS, localPort));
     return new XceiverServerRatis(datanodeDetails, localPort, storageDir,
     return new XceiverServerRatis(datanodeDetails, localPort, storageDir,
         dispatcher, ozoneConf);
         dispatcher, ozoneConf);
   }
   }

+ 12 - 4
hadoop-hdds/container-service/src/test/java/org/apache/hadoop/ozone/container/common/TestDatanodeStateMachine.java

@@ -209,8 +209,10 @@ public class TestDatanodeStateMachine {
         conf.get(ScmConfigKeys.OZONE_SCM_DATANODE_ID));
         conf.get(ScmConfigKeys.OZONE_SCM_DATANODE_ID));
     idPath.delete();
     idPath.delete();
     DatanodeDetails datanodeDetails = getNewDatanodeDetails();
     DatanodeDetails datanodeDetails = getNewDatanodeDetails();
-    datanodeDetails.setContainerPort(
+    DatanodeDetails.Port port = DatanodeDetails.newPort(
+        DatanodeDetails.Port.Name.STANDALONE,
         OzoneConfigKeys.DFS_CONTAINER_IPC_PORT_DEFAULT);
         OzoneConfigKeys.DFS_CONTAINER_IPC_PORT_DEFAULT);
+    datanodeDetails.setPort(port);
     ContainerUtils.writeDatanodeDetailsTo(datanodeDetails, idPath);
     ContainerUtils.writeDatanodeDetailsTo(datanodeDetails, idPath);
 
 
     try (DatanodeStateMachine stateMachine =
     try (DatanodeStateMachine stateMachine =
@@ -360,13 +362,19 @@ public class TestDatanodeStateMachine {
   }
   }
 
 
   private DatanodeDetails getNewDatanodeDetails() {
   private DatanodeDetails getNewDatanodeDetails() {
+    DatanodeDetails.Port containerPort = DatanodeDetails.newPort(
+        DatanodeDetails.Port.Name.STANDALONE, 0);
+    DatanodeDetails.Port ratisPort = DatanodeDetails.newPort(
+        DatanodeDetails.Port.Name.RATIS, 0);
+    DatanodeDetails.Port restPort = DatanodeDetails.newPort(
+        DatanodeDetails.Port.Name.REST, 0);
     return DatanodeDetails.newBuilder()
     return DatanodeDetails.newBuilder()
         .setUuid(UUID.randomUUID().toString())
         .setUuid(UUID.randomUUID().toString())
         .setHostName("localhost")
         .setHostName("localhost")
         .setIpAddress("127.0.0.1")
         .setIpAddress("127.0.0.1")
-        .setContainerPort(0)
-        .setRatisPort(0)
-        .setOzoneRestPort(0)
+        .addPort(containerPort)
+        .addPort(ratisPort)
+        .addPort(restPort)
         .build();
         .build();
   }
   }
 }
 }

+ 9 - 3
hadoop-hdds/server-scm/src/test/java/org/apache/hadoop/hdds/scm/TestUtils.java

@@ -124,13 +124,19 @@ public final class TestUtils {
             .nextInt(256) + "." + random.nextInt(256);
             .nextInt(256) + "." + random.nextInt(256);
 
 
     String hostName = uuid;
     String hostName = uuid;
+    DatanodeDetails.Port containerPort = DatanodeDetails.newPort(
+        DatanodeDetails.Port.Name.STANDALONE, 0);
+    DatanodeDetails.Port ratisPort = DatanodeDetails.newPort(
+        DatanodeDetails.Port.Name.RATIS, 0);
+    DatanodeDetails.Port restPort = DatanodeDetails.newPort(
+        DatanodeDetails.Port.Name.REST, 0);
     DatanodeDetails.Builder builder = DatanodeDetails.newBuilder();
     DatanodeDetails.Builder builder = DatanodeDetails.newBuilder();
     builder.setUuid(uuid)
     builder.setUuid(uuid)
         .setHostName("localhost")
         .setHostName("localhost")
         .setIpAddress(ipAddress)
         .setIpAddress(ipAddress)
-        .setContainerPort(0)
-        .setRatisPort(0)
-        .setOzoneRestPort(0);
+        .addPort(containerPort)
+        .addPort(ratisPort)
+        .addPort(restPort);
     return builder.build();
     return builder.build();
   }
   }
 
 

+ 12 - 6
hadoop-hdds/server-scm/src/test/java/org/apache/hadoop/hdds/scm/block/TestDeletedBlockLog.java

@@ -265,21 +265,27 @@ public class TestDeletedBlockLog {
 
 
     int count = 0;
     int count = 0;
     long containerID = 0L;
     long containerID = 0L;
+    DatanodeDetails.Port containerPort = DatanodeDetails.newPort(
+        DatanodeDetails.Port.Name.STANDALONE, 0);
+    DatanodeDetails.Port ratisPort = DatanodeDetails.newPort(
+        DatanodeDetails.Port.Name.RATIS, 0);
+    DatanodeDetails.Port restPort = DatanodeDetails.newPort(
+        DatanodeDetails.Port.Name.REST, 0);
     DatanodeDetails dnId1 = DatanodeDetails.newBuilder()
     DatanodeDetails dnId1 = DatanodeDetails.newBuilder()
         .setUuid(UUID.randomUUID().toString())
         .setUuid(UUID.randomUUID().toString())
         .setIpAddress("127.0.0.1")
         .setIpAddress("127.0.0.1")
         .setHostName("localhost")
         .setHostName("localhost")
-        .setContainerPort(0)
-        .setRatisPort(0)
-        .setOzoneRestPort(0)
+        .addPort(containerPort)
+        .addPort(ratisPort)
+        .addPort(restPort)
         .build();
         .build();
     DatanodeDetails dnId2 = DatanodeDetails.newBuilder()
     DatanodeDetails dnId2 = DatanodeDetails.newBuilder()
         .setUuid(UUID.randomUUID().toString())
         .setUuid(UUID.randomUUID().toString())
         .setIpAddress("127.0.0.1")
         .setIpAddress("127.0.0.1")
         .setHostName("localhost")
         .setHostName("localhost")
-        .setContainerPort(0)
-        .setRatisPort(0)
-        .setOzoneRestPort(0)
+        .addPort(containerPort)
+        .addPort(ratisPort)
+        .addPort(restPort)
         .build();
         .build();
     Mapping mappingService = mock(ContainerMapping.class);
     Mapping mappingService = mock(ContainerMapping.class);
     // Creates {TXNum} TX in the log.
     // Creates {TXNum} TX in the log.

+ 10 - 7
hadoop-ozone/integration-test/src/test/java/org/apache/hadoop/ozone/MiniOzoneClusterImpl.java

@@ -24,6 +24,7 @@ import java.util.Optional;
 import org.apache.commons.io.FileUtils;
 import org.apache.commons.io.FileUtils;
 import org.apache.hadoop.classification.InterfaceAudience;
 import org.apache.hadoop.classification.InterfaceAudience;
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.hdds.protocol.DatanodeDetails;
 import org.apache.hadoop.hdfs.DFSConfigKeys;
 import org.apache.hadoop.hdfs.DFSConfigKeys;
 import org.apache.hadoop.hdds.conf.OzoneConfiguration;
 import org.apache.hadoop.hdds.conf.OzoneConfiguration;
 import org.apache.hadoop.ipc.Client;
 import org.apache.hadoop.ipc.Client;
@@ -219,13 +220,15 @@ public final class MiniOzoneClusterImpl implements MiniOzoneCluster {
     datanodeService.stop();
     datanodeService.stop();
     datanodeService.join();
     datanodeService.join();
     // ensure same ports are used across restarts.
     // ensure same ports are used across restarts.
-    Configuration config = datanodeService.getConf();
-    int currentPort = datanodeService.getDatanodeDetails().getContainerPort();
-    config.setInt(DFS_CONTAINER_IPC_PORT, currentPort);
-    config.setBoolean(DFS_CONTAINER_IPC_RANDOM_PORT, false);
-    int ratisPort = datanodeService.getDatanodeDetails().getRatisPort();
-    config.setInt(DFS_CONTAINER_RATIS_IPC_PORT, ratisPort);
-    config.setBoolean(DFS_CONTAINER_RATIS_IPC_RANDOM_PORT, false);
+    Configuration conf = datanodeService.getConf();
+    int currentPort = datanodeService.getDatanodeDetails()
+        .getPort(DatanodeDetails.Port.Name.STANDALONE).getValue();
+    conf.setInt(DFS_CONTAINER_IPC_PORT, currentPort);
+    conf.setBoolean(DFS_CONTAINER_IPC_RANDOM_PORT, false);
+    int ratisPort = datanodeService.getDatanodeDetails()
+        .getPort(DatanodeDetails.Port.Name.RATIS).getValue();
+    conf.setInt(DFS_CONTAINER_RATIS_IPC_PORT, ratisPort);
+    conf.setBoolean(DFS_CONTAINER_RATIS_IPC_RANDOM_PORT, false);
     datanodeService.start(null);
     datanodeService.start(null);
   }
   }
 
 

+ 2 - 1
hadoop-ozone/integration-test/src/test/java/org/apache/hadoop/ozone/RatisTestHelper.java

@@ -22,6 +22,7 @@ import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.hdds.conf.OzoneConfiguration;
 import org.apache.hadoop.hdds.conf.OzoneConfiguration;
 import org.apache.hadoop.ozone.client.protocol.ClientProtocol;
 import org.apache.hadoop.ozone.client.protocol.ClientProtocol;
 import org.apache.hadoop.ozone.client.rpc.RpcClient;
 import org.apache.hadoop.ozone.client.rpc.RpcClient;
+import org.apache.hadoop.hdds.protocol.DatanodeDetails;
 import org.apache.hadoop.ozone.container.ContainerTestHelper;
 import org.apache.hadoop.ozone.container.ContainerTestHelper;
 import org.apache.hadoop.ozone.client.rest.OzoneException;
 import org.apache.hadoop.ozone.client.rest.OzoneException;
 import org.apache.ratis.rpc.RpcType;
 import org.apache.ratis.rpc.RpcType;
@@ -78,7 +79,7 @@ public interface RatisTestHelper {
 
 
     public int getDatanodeOzoneRestPort() {
     public int getDatanodeOzoneRestPort() {
       return cluster.getHddsDatanodes().get(0).getDatanodeDetails()
       return cluster.getHddsDatanodes().get(0).getDatanodeDetails()
-          .getOzoneRestPort();
+          .getPort(DatanodeDetails.Port.Name.REST).getValue();
     }
     }
   }
   }
 
 

+ 4 - 4
hadoop-ozone/integration-test/src/test/java/org/apache/hadoop/ozone/TestMiniOzoneCluster.java

@@ -27,7 +27,6 @@ import org.apache.hadoop.ozone.container.common.statemachine.DatanodeStateMachin
 import org.apache.hadoop.ozone.container.ozoneimpl.TestOzoneContainer;
 import org.apache.hadoop.ozone.container.ozoneimpl.TestOzoneContainer;
 import org.apache.hadoop.hdds.protocol.proto.HddsProtos;
 import org.apache.hadoop.hdds.protocol.proto.HddsProtos;
 import org.apache.hadoop.hdds.scm.TestUtils;
 import org.apache.hadoop.hdds.scm.TestUtils;
-import org.apache.hadoop.ozone.web.utils.OzoneUtils;
 import org.apache.hadoop.hdds.scm.XceiverClient;
 import org.apache.hadoop.hdds.scm.XceiverClient;
 import org.apache.hadoop.hdds.scm.container.common.helpers.PipelineChannel;
 import org.apache.hadoop.hdds.scm.container.common.helpers.PipelineChannel;
 import org.apache.hadoop.hdds.scm.container.common.helpers.Pipeline;
 import org.apache.hadoop.hdds.scm.container.common.helpers.Pipeline;
@@ -44,6 +43,7 @@ import java.io.IOException;
 import java.util.HashSet;
 import java.util.HashSet;
 import java.util.List;
 import java.util.List;
 
 
+import static org.apache.hadoop.hdds.protocol.DatanodeDetails.Port;
 import static org.apache.hadoop.hdfs.DFSConfigKeys.DFS_DATANODE_DATA_DIR_KEY;
 import static org.apache.hadoop.hdfs.DFSConfigKeys.DFS_DATANODE_DATA_DIR_KEY;
 import static org.apache.hadoop.ozone.OzoneConfigKeys.DFS_CONTAINER_RATIS_IPC_RANDOM_PORT;
 import static org.apache.hadoop.ozone.OzoneConfigKeys.DFS_CONTAINER_RATIS_IPC_RANDOM_PORT;
 import static org.apache.hadoop.ozone.OzoneConfigKeys.OZONE_METADATA_DIRS;
 import static org.apache.hadoop.ozone.OzoneConfigKeys.OZONE_METADATA_DIRS;
@@ -114,9 +114,9 @@ public class TestMiniOzoneCluster {
     DatanodeDetails id1 = TestUtils.getDatanodeDetails();
     DatanodeDetails id1 = TestUtils.getDatanodeDetails();
     DatanodeDetails id2 = TestUtils.getDatanodeDetails();
     DatanodeDetails id2 = TestUtils.getDatanodeDetails();
     DatanodeDetails id3 = TestUtils.getDatanodeDetails();
     DatanodeDetails id3 = TestUtils.getDatanodeDetails();
-    id1.setContainerPort(1);
-    id2.setContainerPort(2);
-    id3.setContainerPort(3);
+    id1.setPort(DatanodeDetails.newPort(Port.Name.STANDALONE, 1));
+    id2.setPort(DatanodeDetails.newPort(Port.Name.STANDALONE, 2));
+    id3.setPort(DatanodeDetails.newPort(Port.Name.STANDALONE, 3));
 
 
     // Write a single ID to the file and read it out
     // Write a single ID to the file and read it out
     File validIdsFile = new File(WRITE_TMP, "valid-values.id");
     File validIdsFile = new File(WRITE_TMP, "valid-values.id");

+ 9 - 3
hadoop-ozone/integration-test/src/test/java/org/apache/hadoop/ozone/container/ContainerTestHelper.java

@@ -94,13 +94,19 @@ public final class ContainerTestHelper {
   public static DatanodeDetails createDatanodeDetails() throws IOException {
   public static DatanodeDetails createDatanodeDetails() throws IOException {
     ServerSocket socket = new ServerSocket(0);
     ServerSocket socket = new ServerSocket(0);
     int port = socket.getLocalPort();
     int port = socket.getLocalPort();
+    DatanodeDetails.Port containerPort = DatanodeDetails.newPort(
+        DatanodeDetails.Port.Name.STANDALONE, port);
+    DatanodeDetails.Port ratisPort = DatanodeDetails.newPort(
+        DatanodeDetails.Port.Name.RATIS, port);
+    DatanodeDetails.Port restPort = DatanodeDetails.newPort(
+        DatanodeDetails.Port.Name.REST, port);
     DatanodeDetails datanodeDetails = DatanodeDetails.newBuilder()
     DatanodeDetails datanodeDetails = DatanodeDetails.newBuilder()
         .setUuid(UUID.randomUUID().toString())
         .setUuid(UUID.randomUUID().toString())
         .setIpAddress(socket.getInetAddress().getHostAddress())
         .setIpAddress(socket.getInetAddress().getHostAddress())
         .setHostName(socket.getInetAddress().getHostName())
         .setHostName(socket.getInetAddress().getHostName())
-        .setContainerPort(port)
-        .setRatisPort(port)
-        .setOzoneRestPort(port)
+        .addPort(containerPort)
+        .addPort(ratisPort)
+        .addPort(restPort)
         .build();
         .build();
 
 
     socket.close();
     socket.close();

+ 2 - 1
hadoop-ozone/integration-test/src/test/java/org/apache/hadoop/ozone/container/metrics/TestContainerMetrics.java

@@ -65,7 +65,8 @@ public class TestContainerMetrics {
           .createSingleNodePipeline();
           .createSingleNodePipeline();
       OzoneConfiguration conf = new OzoneConfiguration();
       OzoneConfiguration conf = new OzoneConfiguration();
       conf.setInt(OzoneConfigKeys.DFS_CONTAINER_IPC_PORT,
       conf.setInt(OzoneConfigKeys.DFS_CONTAINER_IPC_PORT,
-          pipeline.getLeader().getContainerPort());
+          pipeline.getLeader()
+              .getPort(DatanodeDetails.Port.Name.STANDALONE).getValue());
       conf.setInt(DFSConfigKeys.DFS_METRICS_PERCENTILES_INTERVALS_KEY,
       conf.setInt(DFSConfigKeys.DFS_METRICS_PERCENTILES_INTERVALS_KEY,
           interval);
           interval);
 
 

+ 7 - 4
hadoop-ozone/integration-test/src/test/java/org/apache/hadoop/ozone/container/ozoneimpl/TestOzoneContainer.java

@@ -19,6 +19,7 @@
 package org.apache.hadoop.ozone.container.ozoneimpl;
 package org.apache.hadoop.ozone.container.ozoneimpl;
 
 
 import org.apache.hadoop.hdds.client.BlockID;
 import org.apache.hadoop.hdds.client.BlockID;
+import org.apache.hadoop.hdds.protocol.DatanodeDetails;
 import org.apache.hadoop.hdds.protocol.datanode.proto.ContainerProtos;
 import org.apache.hadoop.hdds.protocol.datanode.proto.ContainerProtos;
 import org.apache.hadoop.ozone.MiniOzoneCluster;
 import org.apache.hadoop.ozone.MiniOzoneCluster;
 import org.apache.hadoop.ozone.OzoneConfigKeys;
 import org.apache.hadoop.ozone.OzoneConfigKeys;
@@ -62,8 +63,8 @@ public class TestOzoneContainer {
       // We don't start Ozone Container via data node, we will do it
       // We don't start Ozone Container via data node, we will do it
       // independently in our test path.
       // independently in our test path.
       Pipeline pipeline = ContainerTestHelper.createSingleNodePipeline();
       Pipeline pipeline = ContainerTestHelper.createSingleNodePipeline();
-      conf.setInt(OzoneConfigKeys.DFS_CONTAINER_IPC_PORT,
-          pipeline.getLeader().getContainerPort());
+      conf.setInt(OzoneConfigKeys.DFS_CONTAINER_IPC_PORT, pipeline.getLeader()
+              .getPort(DatanodeDetails.Port.Name.STANDALONE).getValue());
       conf.setBoolean(OzoneConfigKeys.DFS_CONTAINER_IPC_RANDOM_PORT, false);
       conf.setBoolean(OzoneConfigKeys.DFS_CONTAINER_IPC_RANDOM_PORT, false);
       container = new OzoneContainer(TestUtils.getDatanodeDetails(), conf);
       container = new OzoneContainer(TestUtils.getDatanodeDetails(), conf);
       container.start();
       container.start();
@@ -101,7 +102,8 @@ public class TestOzoneContainer {
       Pipeline pipeline =
       Pipeline pipeline =
           ContainerTestHelper.createSingleNodePipeline();
           ContainerTestHelper.createSingleNodePipeline();
       conf.setInt(OzoneConfigKeys.DFS_CONTAINER_IPC_PORT,
       conf.setInt(OzoneConfigKeys.DFS_CONTAINER_IPC_PORT,
-          pipeline.getLeader().getContainerPort());
+          pipeline.getLeader()
+              .getPort(DatanodeDetails.Port.Name.STANDALONE).getValue());
 
 
       cluster = MiniOzoneCluster.newBuilder(conf)
       cluster = MiniOzoneCluster.newBuilder(conf)
           .setRandomContainerPort(false)
           .setRandomContainerPort(false)
@@ -527,7 +529,8 @@ public class TestOzoneContainer {
     Pipeline pipeline =
     Pipeline pipeline =
         ContainerTestHelper.createSingleNodePipeline();
         ContainerTestHelper.createSingleNodePipeline();
     conf.setInt(OzoneConfigKeys.DFS_CONTAINER_IPC_PORT,
     conf.setInt(OzoneConfigKeys.DFS_CONTAINER_IPC_PORT,
-        pipeline.getLeader().getContainerPort());
+        pipeline.getLeader()
+            .getPort(DatanodeDetails.Port.Name.STANDALONE).getValue());
 
 
     // This client talks to ozone container via datanode.
     // This client talks to ozone container via datanode.
     return new XceiverClient(pipeline, conf);
     return new XceiverClient(pipeline, conf);

+ 5 - 3
hadoop-ozone/integration-test/src/test/java/org/apache/hadoop/ozone/container/server/TestContainerServer.java

@@ -103,7 +103,8 @@ public class TestContainerServer {
     DatanodeDetails datanodeDetails = TestUtils.getDatanodeDetails();
     DatanodeDetails datanodeDetails = TestUtils.getDatanodeDetails();
     runTestClientServer(1,
     runTestClientServer(1,
         (pipeline, conf) -> conf.setInt(OzoneConfigKeys.DFS_CONTAINER_IPC_PORT,
         (pipeline, conf) -> conf.setInt(OzoneConfigKeys.DFS_CONTAINER_IPC_PORT,
-            pipeline.getLeader().getContainerPort()),
+            pipeline.getLeader()
+                .getPort(DatanodeDetails.Port.Name.STANDALONE).getValue()),
         XceiverClient::new,
         XceiverClient::new,
         (dn, conf) -> new XceiverServer(datanodeDetails, conf,
         (dn, conf) -> new XceiverServer(datanodeDetails, conf,
             new TestContainerDispatcher()),
             new TestContainerDispatcher()),
@@ -130,7 +131,7 @@ public class TestContainerServer {
   static XceiverServerRatis newXceiverServerRatis(
   static XceiverServerRatis newXceiverServerRatis(
       DatanodeDetails dn, OzoneConfiguration conf) throws IOException {
       DatanodeDetails dn, OzoneConfiguration conf) throws IOException {
     conf.setInt(OzoneConfigKeys.DFS_CONTAINER_RATIS_IPC_PORT,
     conf.setInt(OzoneConfigKeys.DFS_CONTAINER_RATIS_IPC_PORT,
-        dn.getRatisPort());
+        dn.getPort(DatanodeDetails.Port.Name.RATIS).getValue());
     final String dir = TEST_DIR + dn.getUuid();
     final String dir = TEST_DIR + dn.getUuid();
     conf.set(OzoneConfigKeys.DFS_CONTAINER_RATIS_DATANODE_STORAGE_DIR, dir);
     conf.set(OzoneConfigKeys.DFS_CONTAINER_RATIS_DATANODE_STORAGE_DIR, dir);
 
 
@@ -208,7 +209,8 @@ public class TestContainerServer {
       Pipeline pipeline = ContainerTestHelper.createSingleNodePipeline();
       Pipeline pipeline = ContainerTestHelper.createSingleNodePipeline();
       OzoneConfiguration conf = new OzoneConfiguration();
       OzoneConfiguration conf = new OzoneConfiguration();
       conf.setInt(OzoneConfigKeys.DFS_CONTAINER_IPC_PORT,
       conf.setInt(OzoneConfigKeys.DFS_CONTAINER_IPC_PORT,
-          pipeline.getLeader().getContainerPort());
+          pipeline.getLeader()
+              .getPort(DatanodeDetails.Port.Name.STANDALONE).getValue());
 
 
       Dispatcher dispatcher =
       Dispatcher dispatcher =
               new Dispatcher(mock(ContainerManager.class), conf);
               new Dispatcher(mock(ContainerManager.class), conf);

+ 3 - 2
hadoop-ozone/integration-test/src/test/java/org/apache/hadoop/ozone/ksm/TestKeySpaceManagerRestInterface.java

@@ -118,8 +118,9 @@ public class TestKeySpaceManagerRestInterface {
       switch (type) {
       switch (type) {
       case HTTP:
       case HTTP:
       case HTTPS:
       case HTTPS:
-        Assert.assertEquals(datanodeDetails.getOzoneRestPort(),
-            (int) ports.get(type));
+        Assert.assertEquals(
+            datanodeDetails.getPort(DatanodeDetails.Port.Name.REST).getValue(),
+            ports.get(type));
         break;
         break;
       default:
       default:
         // KSM only sends Datanode's info port details
         // KSM only sends Datanode's info port details

+ 3 - 1
hadoop-ozone/integration-test/src/test/java/org/apache/hadoop/ozone/web/TestDistributedOzoneVolumes.java

@@ -17,6 +17,7 @@
  */
  */
 package org.apache.hadoop.ozone.web;
 package org.apache.hadoop.ozone.web;
 
 
+import org.apache.hadoop.hdds.protocol.DatanodeDetails;
 import org.apache.hadoop.ozone.MiniOzoneCluster;
 import org.apache.hadoop.ozone.MiniOzoneCluster;
 import org.apache.hadoop.ozone.OzoneConfigKeys;
 import org.apache.hadoop.ozone.OzoneConfigKeys;
 import org.apache.hadoop.hdds.conf.OzoneConfiguration;
 import org.apache.hadoop.hdds.conf.OzoneConfiguration;
@@ -67,7 +68,8 @@ public class TestDistributedOzoneVolumes extends TestOzoneHelper {
     cluster = MiniOzoneCluster.newBuilder(conf).build();
     cluster = MiniOzoneCluster.newBuilder(conf).build();
     cluster.waitForClusterToBeReady();
     cluster.waitForClusterToBeReady();
     port = cluster.getHddsDatanodes().get(0)
     port = cluster.getHddsDatanodes().get(0)
-        .getDatanodeDetails().getOzoneRestPort();
+        .getDatanodeDetails()
+        .getPort(DatanodeDetails.Port.Name.REST).getValue();
   }
   }
 
 
   /**
   /**

+ 3 - 1
hadoop-ozone/integration-test/src/test/java/org/apache/hadoop/ozone/web/TestLocalOzoneVolumes.java

@@ -17,6 +17,7 @@
  */
  */
 package org.apache.hadoop.ozone.web;
 package org.apache.hadoop.ozone.web;
 
 
+import org.apache.hadoop.hdds.protocol.DatanodeDetails;
 import org.apache.hadoop.ozone.OzoneConfigKeys;
 import org.apache.hadoop.ozone.OzoneConfigKeys;
 import org.apache.hadoop.hdds.conf.OzoneConfiguration;
 import org.apache.hadoop.hdds.conf.OzoneConfiguration;
 import org.apache.hadoop.ozone.TestOzoneHelper;
 import org.apache.hadoop.ozone.TestOzoneHelper;
@@ -70,7 +71,8 @@ public class TestLocalOzoneVolumes extends TestOzoneHelper {
     cluster = MiniOzoneCluster.newBuilder(conf).build();
     cluster = MiniOzoneCluster.newBuilder(conf).build();
     cluster.waitForClusterToBeReady();
     cluster.waitForClusterToBeReady();
     port = cluster.getHddsDatanodes().get(0)
     port = cluster.getHddsDatanodes().get(0)
-        .getDatanodeDetails().getOzoneRestPort();
+        .getDatanodeDetails().getPort(
+            DatanodeDetails.Port.Name.REST).getValue();
   }
   }
 
 
   /**
   /**

+ 3 - 1
hadoop-ozone/integration-test/src/test/java/org/apache/hadoop/ozone/web/TestOzoneWebAccess.java

@@ -18,6 +18,7 @@
 
 
 package org.apache.hadoop.ozone.web;
 package org.apache.hadoop.ozone.web;
 
 
+import org.apache.hadoop.hdds.protocol.DatanodeDetails;
 import org.apache.hadoop.ozone.MiniOzoneCluster;
 import org.apache.hadoop.ozone.MiniOzoneCluster;
 import org.apache.hadoop.ozone.OzoneConfigKeys;
 import org.apache.hadoop.ozone.OzoneConfigKeys;
 import org.apache.hadoop.hdds.conf.OzoneConfiguration;
 import org.apache.hadoop.hdds.conf.OzoneConfiguration;
@@ -78,7 +79,8 @@ public class TestOzoneWebAccess {
     cluster = MiniOzoneCluster.newBuilder(conf).build();
     cluster = MiniOzoneCluster.newBuilder(conf).build();
     cluster.waitForClusterToBeReady();
     cluster.waitForClusterToBeReady();
     port = cluster.getHddsDatanodes().get(0)
     port = cluster.getHddsDatanodes().get(0)
-        .getDatanodeDetails().getOzoneRestPort();
+        .getDatanodeDetails().getPort(
+            DatanodeDetails.Port.Name.REST).getValue();
   }
   }
 
 
   /**
   /**

+ 3 - 1
hadoop-ozone/integration-test/src/test/java/org/apache/hadoop/ozone/web/client/TestOzoneClient.java

@@ -43,6 +43,7 @@ import io.netty.handler.codec.http.LastHttpContent;
 import io.netty.handler.logging.LogLevel;
 import io.netty.handler.logging.LogLevel;
 import io.netty.handler.logging.LoggingHandler;
 import io.netty.handler.logging.LoggingHandler;
 
 
+import org.apache.hadoop.hdds.protocol.DatanodeDetails;
 import org.apache.hadoop.ozone.MiniOzoneCluster;
 import org.apache.hadoop.ozone.MiniOzoneCluster;
 import org.apache.hadoop.ozone.OzoneConfigKeys;
 import org.apache.hadoop.ozone.OzoneConfigKeys;
 import org.apache.hadoop.hdds.conf.OzoneConfiguration;
 import org.apache.hadoop.hdds.conf.OzoneConfiguration;
@@ -96,7 +97,8 @@ public class TestOzoneClient {
     cluster = MiniOzoneCluster.newBuilder(conf).build();
     cluster = MiniOzoneCluster.newBuilder(conf).build();
     cluster.waitForClusterToBeReady();
     cluster.waitForClusterToBeReady();
     int port = cluster.getHddsDatanodes().get(0)
     int port = cluster.getHddsDatanodes().get(0)
-        .getDatanodeDetails().getOzoneRestPort();
+        .getDatanodeDetails()
+        .getPort(DatanodeDetails.Port.Name.REST).getValue();
     endpoint = String.format("http://localhost:%d", port);
     endpoint = String.format("http://localhost:%d", port);
   }
   }
 
 

+ 0 - 2
hadoop-ozone/integration-test/src/test/java/org/apache/hadoop/ozone/web/client/TestVolume.java

@@ -81,8 +81,6 @@ public class TestVolume {
 
 
     cluster = MiniOzoneCluster.newBuilder(conf).build();
     cluster = MiniOzoneCluster.newBuilder(conf).build();
     cluster.waitForClusterToBeReady();
     cluster.waitForClusterToBeReady();
-    final int port = cluster.getHddsDatanodes().get(0)
-        .getDatanodeDetails().getOzoneRestPort();
 
 
     client = new RpcClient(conf);
     client = new RpcClient(conf);
   }
   }

+ 3 - 1
hadoop-ozone/integration-test/src/test/java/org/apache/hadoop/ozone/web/client/TestVolumeRatis.java

@@ -19,6 +19,7 @@
 package org.apache.hadoop.ozone.web.client;
 package org.apache.hadoop.ozone.web.client;
 
 
 import org.apache.commons.io.FileUtils;
 import org.apache.commons.io.FileUtils;
+import org.apache.hadoop.hdds.protocol.DatanodeDetails;
 import org.apache.hadoop.ozone.MiniOzoneCluster;
 import org.apache.hadoop.ozone.MiniOzoneCluster;
 import org.apache.hadoop.ozone.OzoneConfigKeys;
 import org.apache.hadoop.ozone.OzoneConfigKeys;
 import org.apache.hadoop.hdds.conf.OzoneConfiguration;
 import org.apache.hadoop.hdds.conf.OzoneConfiguration;
@@ -63,7 +64,8 @@ public class TestVolumeRatis {
     cluster = MiniOzoneCluster.newBuilder(conf).setNumDatanodes(3).build();
     cluster = MiniOzoneCluster.newBuilder(conf).setNumDatanodes(3).build();
     cluster.waitForClusterToBeReady();
     cluster.waitForClusterToBeReady();
     final int port = cluster.getHddsDatanodes().get(0)
     final int port = cluster.getHddsDatanodes().get(0)
-        .getDatanodeDetails().getOzoneRestPort();
+        .getDatanodeDetails()
+        .getPort(DatanodeDetails.Port.Name.REST).getValue();
 
 
     client = new RpcClient(conf);
     client = new RpcClient(conf);
   }
   }

+ 4 - 1
hadoop-ozone/objectstore-service/src/main/java/org/apache/hadoop/ozone/web/OzoneHddsDatanodeService.java

@@ -20,6 +20,7 @@ package org.apache.hadoop.ozone.web;
 import java.io.IOException;
 import java.io.IOException;
 
 
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.hdds.protocol.DatanodeDetails;
 import org.apache.hadoop.hdfs.server.datanode.ObjectStoreHandler;
 import org.apache.hadoop.hdfs.server.datanode.ObjectStoreHandler;
 import org.apache.hadoop.ozone.HddsDatanodeService;
 import org.apache.hadoop.ozone.HddsDatanodeService;
 import org.apache.hadoop.ozone.web.netty.ObjectStoreRestHttpServer;
 import org.apache.hadoop.ozone.web.netty.ObjectStoreRestHttpServer;
@@ -51,8 +52,10 @@ public class OzoneHddsDatanodeService implements ServicePlugin {
         objectStoreRestHttpServer = new ObjectStoreRestHttpServer(
         objectStoreRestHttpServer = new ObjectStoreRestHttpServer(
             conf, null, handler);
             conf, null, handler);
         objectStoreRestHttpServer.start();
         objectStoreRestHttpServer.start();
-        hddsDatanodeService.getDatanodeDetails().setOzoneRestPort(
+        DatanodeDetails.Port restPort = DatanodeDetails.newPort(
+            DatanodeDetails.Port.Name.REST,
             objectStoreRestHttpServer.getHttpAddress().getPort());
             objectStoreRestHttpServer.getHttpAddress().getPort());
+        hddsDatanodeService.getDatanodeDetails().setPort(restPort);
 
 
       } catch (IOException e) {
       } catch (IOException e) {
         throw new RuntimeException("Can't start the Object Store Rest server",
         throw new RuntimeException("Can't start the Object Store Rest server",

+ 3 - 1
hadoop-ozone/ozone-manager/src/main/java/org/apache/hadoop/ozone/ksm/KeySpaceManager.java

@@ -21,6 +21,7 @@ import com.google.common.annotations.VisibleForTesting;
 import com.google.common.base.Preconditions;
 import com.google.common.base.Preconditions;
 import com.google.protobuf.BlockingService;
 import com.google.protobuf.BlockingService;
 import org.apache.hadoop.classification.InterfaceAudience;
 import org.apache.hadoop.classification.InterfaceAudience;
+import org.apache.hadoop.hdds.protocol.DatanodeDetails;
 import org.apache.hadoop.hdfs.DFSUtil;
 import org.apache.hadoop.hdfs.DFSUtil;
 import org.apache.hadoop.ipc.Client;
 import org.apache.hadoop.ipc.Client;
 import org.apache.hadoop.ipc.ProtobufRpcEngine;
 import org.apache.hadoop.ipc.ProtobufRpcEngine;
@@ -896,7 +897,8 @@ public final class KeySpaceManager extends ServiceRuntimeInfoImpl
 
 
       dnServiceInfoBuilder.addServicePort(ServicePort.newBuilder()
       dnServiceInfoBuilder.addServicePort(ServicePort.newBuilder()
           .setType(ServicePort.Type.HTTP)
           .setType(ServicePort.Type.HTTP)
-          .setValue(datanode.getOzoneRestPort())
+          .setValue(DatanodeDetails.getFromProtoBuf(datanode)
+              .getPort(DatanodeDetails.Port.Name.REST).getValue())
           .build());
           .build());
 
 
       services.add(dnServiceInfoBuilder.build());
       services.add(dnServiceInfoBuilder.build());

+ 9 - 3
hadoop-ozone/tools/src/main/java/org/apache/hadoop/ozone/genesis/GenesisUtil.java

@@ -78,13 +78,19 @@ public final class GenesisUtil {
         random.nextInt(256) + "." + random.nextInt(256) + "." + random
         random.nextInt(256) + "." + random.nextInt(256) + "." + random
             .nextInt(256) + "." + random.nextInt(256);
             .nextInt(256) + "." + random.nextInt(256);
 
 
+    DatanodeDetails.Port containerPort = DatanodeDetails.newPort(
+        DatanodeDetails.Port.Name.STANDALONE, 0);
+    DatanodeDetails.Port ratisPort = DatanodeDetails.newPort(
+        DatanodeDetails.Port.Name.RATIS, 0);
+    DatanodeDetails.Port restPort = DatanodeDetails.newPort(
+        DatanodeDetails.Port.Name.REST, 0);
     DatanodeDetails.Builder builder = DatanodeDetails.newBuilder();
     DatanodeDetails.Builder builder = DatanodeDetails.newBuilder();
     builder.setUuid(uuid)
     builder.setUuid(uuid)
         .setHostName("localhost")
         .setHostName("localhost")
         .setIpAddress(ipAddress)
         .setIpAddress(ipAddress)
-        .setContainerPort(0)
-        .setRatisPort(0)
-        .setOzoneRestPort(0);
+        .addPort(containerPort)
+        .addPort(ratisPort)
+        .addPort(restPort);
     return builder.build();
     return builder.build();
   }
   }
 }
 }

+ 4 - 2
hadoop-ozone/tools/src/main/java/org/apache/hadoop/ozone/scm/cli/SQLCLI.java

@@ -530,7 +530,8 @@ public class SQLCLI  extends Configured implements Tool {
         // but this seems a bit cleaner.
         // but this seems a bit cleaner.
         String ipAddr = dd.getIpAddress();
         String ipAddr = dd.getIpAddress();
         String hostName = dd.getHostName();
         String hostName = dd.getHostName();
-        int containerPort = dd.getContainerPort();
+        int containerPort = DatanodeDetails.getFromProtoBuf(dd)
+            .getPort(DatanodeDetails.Port.Name.STANDALONE).getValue();
         String insertMachineInfo = String.format(
         String insertMachineInfo = String.format(
             INSERT_DATANODE_INFO, hostName, uuid, ipAddr, containerPort);
             INSERT_DATANODE_INFO, hostName, uuid, ipAddr, containerPort);
         executeSQL(conn, insertMachineInfo);
         executeSQL(conn, insertMachineInfo);
@@ -598,7 +599,8 @@ public class SQLCLI  extends Configured implements Tool {
     String insertDatanodeDetails = String
     String insertDatanodeDetails = String
         .format(INSERT_DATANODE_INFO, datanodeDetails.getHostName(),
         .format(INSERT_DATANODE_INFO, datanodeDetails.getHostName(),
             datanodeDetails.getUuidString(), datanodeDetails.getIpAddress(),
             datanodeDetails.getUuidString(), datanodeDetails.getIpAddress(),
-            datanodeDetails.getContainerPort());
+            datanodeDetails.getPort(DatanodeDetails.Port.Name.STANDALONE)
+                .getValue());
     executeSQL(conn, insertDatanodeDetails);
     executeSQL(conn, insertDatanodeDetails);
   }
   }
 
 

+ 2 - 1
hadoop-tools/hadoop-ozone/src/test/java/org/apache/hadoop/fs/ozone/TestOzoneFSInputStream.java

@@ -90,7 +90,8 @@ public class TestOzoneFSInputStream {
     // Fetch the host and port for File System init
     // Fetch the host and port for File System init
     DatanodeDetails datanodeDetails = cluster.getHddsDatanodes().get(0)
     DatanodeDetails datanodeDetails = cluster.getHddsDatanodes().get(0)
         .getDatanodeDetails();
         .getDatanodeDetails();
-    int port = datanodeDetails.getOzoneRestPort();
+    int port = datanodeDetails
+        .getPort(DatanodeDetails.Port.Name.REST).getValue();
     String host = datanodeDetails.getHostName();
     String host = datanodeDetails.getHostName();
 
 
     // Set the fs.defaultFS and start the filesystem
     // Set the fs.defaultFS and start the filesystem