فهرست منبع

HDFS-12017. Ozone: Container: Move IPC port to 98xx range. Contributed by Nandakumar.

Anu Engineer 7 سال پیش
والد
کامیت
2804435386

+ 2 - 2
hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/ozone/OzoneConfigKeys.java

@@ -33,7 +33,7 @@ import org.apache.hadoop.scm.ScmConfigKeys;
 public final class OzoneConfigKeys {
 public final class OzoneConfigKeys {
   public static final String DFS_CONTAINER_IPC_PORT =
   public static final String DFS_CONTAINER_IPC_PORT =
       "dfs.container.ipc";
       "dfs.container.ipc";
-  public static final int DFS_CONTAINER_IPC_PORT_DEFAULT = 50011;
+  public static final int DFS_CONTAINER_IPC_PORT_DEFAULT = 9859;
 
 
   /**
   /**
    *
    *
@@ -53,7 +53,7 @@ public final class OzoneConfigKeys {
    */
    */
   public static final String DFS_CONTAINER_RATIS_IPC_PORT =
   public static final String DFS_CONTAINER_RATIS_IPC_PORT =
       "dfs.container.ratis.ipc";
       "dfs.container.ratis.ipc";
-  public static final int DFS_CONTAINER_RATIS_IPC_PORT_DEFAULT = 50012;
+  public static final int DFS_CONTAINER_RATIS_IPC_PORT_DEFAULT = 9858;
 
 
   /**
   /**
    * When set to true, allocate a random free port for ozone container, so that
    * When set to true, allocate a random free port for ozone container, so that

+ 2 - 2
hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/ozone/client/OzoneClientUtils.java

@@ -642,8 +642,8 @@ public final class OzoneClientUtils {
    * @return port number.
    * @return port number.
    */
    */
   public static int getContainerPort(Configuration conf) {
   public static int getContainerPort(Configuration conf) {
-    return conf.getInt(ScmConfigKeys.DFS_CONTAINER_IPC_PORT, ScmConfigKeys
-        .DFS_CONTAINER_IPC_PORT_DEFAULT);
+    return conf.getInt(OzoneConfigKeys.DFS_CONTAINER_IPC_PORT,
+        OzoneConfigKeys.DFS_CONTAINER_IPC_PORT_DEFAULT);
   }
   }
 
 
   /**
   /**

+ 1 - 1
hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/ozone/client/rpc/RpcClient.java

@@ -264,7 +264,7 @@ public class RpcClient implements ClientProtocol {
     Preconditions.checkNotNull(bucketArgs);
     Preconditions.checkNotNull(bucketArgs);
 
 
     Boolean isVersionEnabled = bucketArgs.isVersionEnabled() == null ?
     Boolean isVersionEnabled = bucketArgs.isVersionEnabled() == null ?
-        false : bucketArgs.isVersionEnabled();
+        Boolean.FALSE : bucketArgs.isVersionEnabled();
     StorageType storageType = bucketArgs.getStorageType() == null ?
     StorageType storageType = bucketArgs.getStorageType() == null ?
         StorageType.DEFAULT : bucketArgs.getStorageType();
         StorageType.DEFAULT : bucketArgs.getStorageType();
     List<OzoneAcl> listOfAcls = new ArrayList<>();
     List<OzoneAcl> listOfAcls = new ArrayList<>();

+ 0 - 3
hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/scm/ScmConfigKeys.java

@@ -26,9 +26,6 @@ import org.apache.hadoop.classification.InterfaceStability;
 @InterfaceAudience.Public
 @InterfaceAudience.Public
 @InterfaceStability.Unstable
 @InterfaceStability.Unstable
 public final class ScmConfigKeys {
 public final class ScmConfigKeys {
-  public static final String DFS_CONTAINER_IPC_PORT =
-      "dfs.container.ipc";
-  public static final int DFS_CONTAINER_IPC_PORT_DEFAULT = 50011;
 
 
   public static final String SCM_CONTAINER_CLIENT_STALE_THRESHOLD_KEY =
   public static final String SCM_CONTAINER_CLIENT_STALE_THRESHOLD_KEY =
       "scm.container.client.idle.threshold";
       "scm.container.client.idle.threshold";

+ 3 - 2
hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/scm/XceiverClient.java

@@ -30,6 +30,7 @@ import io.netty.handler.logging.LoggingHandler;
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.hdfs.ozone.protocol.proto.ContainerProtos;
 import org.apache.hadoop.hdfs.ozone.protocol.proto.ContainerProtos;
 import org.apache.hadoop.hdfs.protocol.DatanodeID;
 import org.apache.hadoop.hdfs.protocol.DatanodeID;
+import org.apache.hadoop.ozone.OzoneConfigKeys;
 import org.apache.hadoop.scm.container.common.helpers.Pipeline;
 import org.apache.hadoop.scm.container.common.helpers.Pipeline;
 import org.slf4j.Logger;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 import org.slf4j.LoggerFactory;
@@ -85,8 +86,8 @@ public class XceiverClient extends XceiverClientSpi {
     // port.
     // port.
     int port = leader.getContainerPort();
     int port = leader.getContainerPort();
     if (port == 0) {
     if (port == 0) {
-      port = config.getInt(ScmConfigKeys.DFS_CONTAINER_IPC_PORT,
-          ScmConfigKeys.DFS_CONTAINER_IPC_PORT_DEFAULT);
+      port = config.getInt(OzoneConfigKeys.DFS_CONTAINER_IPC_PORT,
+          OzoneConfigKeys.DFS_CONTAINER_IPC_PORT_DEFAULT);
     }
     }
     LOG.debug("Connecting to server Port : " + port);
     LOG.debug("Connecting to server Port : " + port);
     channelFuture = b.connect(leader.getHostName(), port).sync();
     channelFuture = b.connect(leader.getHostName(), port).sync();

+ 2 - 2
hadoop-hdfs-project/hadoop-hdfs/src/main/resources/ozone-default.xml

@@ -460,7 +460,7 @@
 
 
   <property>
   <property>
     <name>dfs.container.ipc</name>
     <name>dfs.container.ipc</name>
-    <value>50011</value>
+    <value>9859</value>
     <description>
     <description>
       The ipc port number of container.
       The ipc port number of container.
     </description>
     </description>
@@ -939,7 +939,7 @@
 
 
   <property>
   <property>
     <name>dfs.container.ratis.ipc</name>
     <name>dfs.container.ratis.ipc</name>
-    <value>50012</value>
+    <value>9858</value>
     <description>
     <description>
       The ipc port number of container.
       The ipc port number of container.
     </description>
     </description>

+ 1 - 1
hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/ozone/container/common/TestDatanodeStateMachine.java

@@ -200,7 +200,7 @@ public class TestDatanodeStateMachine {
         conf.get(ScmConfigKeys.OZONE_SCM_DATANODE_ID));
         conf.get(ScmConfigKeys.OZONE_SCM_DATANODE_ID));
     idPath.delete();
     idPath.delete();
     DatanodeID dnID = DFSTestUtil.getLocalDatanodeID();
     DatanodeID dnID = DFSTestUtil.getLocalDatanodeID();
-    dnID.setContainerPort(ScmConfigKeys.DFS_CONTAINER_IPC_PORT_DEFAULT);
+    dnID.setContainerPort(OzoneConfigKeys.DFS_CONTAINER_IPC_PORT_DEFAULT);
     ContainerUtils.writeDatanodeIDTo(dnID, idPath);
     ContainerUtils.writeDatanodeIDTo(dnID, idPath);
 
 
     try (DatanodeStateMachine stateMachine = new DatanodeStateMachine(
     try (DatanodeStateMachine stateMachine = new DatanodeStateMachine(