This is an automated email from the ASF dual-hosted git repository.

licheng pushed a commit to branch HDDS-2823
in repository https://gitbox.apache.org/repos/asf/hadoop-ozone.git

commit d7728b308cb3bd326ea8370ade40a2feb727e858
Author: Li Cheng <bloodhell2...@gmail.com>
AuthorDate: Wed May 20 10:05:57 2020 +0800

    HDDS-3556 Refactor conf in SCMRatisServer to Java-based conf. (#907)
---
 .../org/apache/hadoop/hdds/scm/ScmConfigKeys.java  |   2 +-
 .../common/src/main/resources/ozone-default.xml    |   4 +-
 .../java/org/apache/hadoop/hdds/conf/Config.java   |   5 +
 .../org/apache/hadoop/hdds/conf/ConfigTag.java     |   3 +-
 .../hdds/conf/ConfigurationReflectionUtil.java     |   4 +
 .../hdds/scm/server/StorageContainerManager.java   |  89 ++++---
 .../hdds/scm/server/ratis/SCMRatisServer.java      | 282 ++++++++++++++-------
 .../java/org/apache/hadoop/hdds/scm/TestUtils.java |   2 +-
 .../hdds/scm/server/ratis/TestSCMRatisServer.java  |   6 +-
 .../apache/hadoop/ozone/genesis/GenesisUtil.java   |   2 +-
 10 files changed, 270 insertions(+), 129 deletions(-)

diff --git 
a/hadoop-hdds/common/src/main/java/org/apache/hadoop/hdds/scm/ScmConfigKeys.java
 
b/hadoop-hdds/common/src/main/java/org/apache/hadoop/hdds/scm/ScmConfigKeys.java
index 6c4f9e8..e5de261 100644
--- 
a/hadoop-hdds/common/src/main/java/org/apache/hadoop/hdds/scm/ScmConfigKeys.java
+++ 
b/hadoop-hdds/common/src/main/java/org/apache/hadoop/hdds/scm/ScmConfigKeys.java
@@ -433,7 +433,7 @@ public final class ScmConfigKeys {
   // SCM Ratis Leader Election configurations
   public static final String
       OZONE_SCM_LEADER_ELECTION_MINIMUM_TIMEOUT_DURATION_KEY =
-      "ozone.scm.leader.election.minimum.timeout.duration";
+      "ozone.scm.ratis.leader.election.minimum.timeout.duration";
   public static final TimeDuration
       OZONE_SCM_LEADER_ELECTION_MINIMUM_TIMEOUT_DURATION_DEFAULT =
       TimeDuration.valueOf(1, TimeUnit.SECONDS);
diff --git a/hadoop-hdds/common/src/main/resources/ozone-default.xml 
b/hadoop-hdds/common/src/main/resources/ozone-default.xml
index bd62474..126121b 100644
--- a/hadoop-hdds/common/src/main/resources/ozone-default.xml
+++ b/hadoop-hdds/common/src/main/resources/ozone-default.xml
@@ -2043,7 +2043,7 @@
     <name>ozone.scm.ratis.server.request.timeout</name>
     <value>3s</value>
     <tag>OZONE, SCM, HA, RATIS</tag>
-    <description>The timeout duration for SCM's ratis server request 
.</description>
+    <description>The timeout duration for SCM's ratis server 
request.</description>
   </property>
 
   <property>
@@ -2062,7 +2062,7 @@
   </property>
 
   <property>
-    <name>ozone.scm.leader.election.minimum.timeout.duration</name>
+    <name>ozone.scm.ratis.leader.election.minimum.timeout.duration</name>
     <value>1s</value>
     <tag>OZONE, SCM, HA, RATIS</tag>
     <description>The minimum timeout duration for SCM ratis leader election.
diff --git 
a/hadoop-hdds/config/src/main/java/org/apache/hadoop/hdds/conf/Config.java 
b/hadoop-hdds/config/src/main/java/org/apache/hadoop/hdds/conf/Config.java
index 316c867..5d4b477 100644
--- a/hadoop-hdds/config/src/main/java/org/apache/hadoop/hdds/conf/Config.java
+++ b/hadoop-hdds/config/src/main/java/org/apache/hadoop/hdds/conf/Config.java
@@ -55,5 +55,10 @@ public @interface Config {
    */
   TimeUnit timeUnit() default TimeUnit.MILLISECONDS;
 
+  /**
+   * If type == SIZE the unit should be defined with this attribute.
+   */
+  StorageUnit sizeUnit() default StorageUnit.BYTES;
+
   ConfigTag[] tags();
 }
diff --git 
a/hadoop-hdds/config/src/main/java/org/apache/hadoop/hdds/conf/ConfigTag.java 
b/hadoop-hdds/config/src/main/java/org/apache/hadoop/hdds/conf/ConfigTag.java
index 385840a..39d481e 100644
--- 
a/hadoop-hdds/config/src/main/java/org/apache/hadoop/hdds/conf/ConfigTag.java
+++ 
b/hadoop-hdds/config/src/main/java/org/apache/hadoop/hdds/conf/ConfigTag.java
@@ -42,5 +42,6 @@ public enum ConfigTag {
   STANDALONE,
   S3GATEWAY,
   DATANODE,
-  RECON
+  RECON,
+  HA
 }
diff --git 
a/hadoop-hdds/config/src/main/java/org/apache/hadoop/hdds/conf/ConfigurationReflectionUtil.java
 
b/hadoop-hdds/config/src/main/java/org/apache/hadoop/hdds/conf/ConfigurationReflectionUtil.java
index 48f06ea..1d5e654 100644
--- 
a/hadoop-hdds/config/src/main/java/org/apache/hadoop/hdds/conf/ConfigurationReflectionUtil.java
+++ 
b/hadoop-hdds/config/src/main/java/org/apache/hadoop/hdds/conf/ConfigurationReflectionUtil.java
@@ -84,6 +84,10 @@ public final class ConfigurationReflectionUtil {
             forcedFieldSet(field, configuration,
                 from.getTimeDuration(key, "0s", configAnnotation.timeUnit()));
             break;
+          case SIZE:
+            forcedFieldSet(field, configuration,
+                from.getStorageSize(key, "0B", configAnnotation.sizeUnit()));
+            break;
           default:
             throw new ConfigurationException(
                 "Unsupported ConfigType " + type + " on " + fieldLocation);
diff --git 
a/hadoop-hdds/server-scm/src/main/java/org/apache/hadoop/hdds/scm/server/StorageContainerManager.java
 
b/hadoop-hdds/server-scm/src/main/java/org/apache/hadoop/hdds/scm/server/StorageContainerManager.java
index 1a9b70c..6bf5da5 100644
--- 
a/hadoop-hdds/server-scm/src/main/java/org/apache/hadoop/hdds/scm/server/StorageContainerManager.java
+++ 
b/hadoop-hdds/server-scm/src/main/java/org/apache/hadoop/hdds/scm/server/StorageContainerManager.java
@@ -220,7 +220,7 @@ public final class StorageContainerManager extends 
ServiceRuntimeInfoImpl
    *
    * @param conf configuration
    */
-  public StorageContainerManager(OzoneConfiguration conf)
+  private StorageContainerManager(OzoneConfiguration conf)
       throws IOException, AuthenticationException {
     // default empty configurator means default managers will be used.
     this(conf, new SCMConfigurator());
@@ -236,7 +236,7 @@ public final class StorageContainerManager extends 
ServiceRuntimeInfoImpl
    * @param conf - Configuration
    * @param configurator - configurator
    */
-  public StorageContainerManager(OzoneConfiguration conf,
+  private StorageContainerManager(OzoneConfiguration conf,
                                  SCMConfigurator configurator)
       throws IOException, AuthenticationException  {
     super(HddsVersionInfo.HDDS_VERSION_INFO);
@@ -269,14 +269,9 @@ public final class StorageContainerManager extends 
ServiceRuntimeInfoImpl
       loginAsSCMUser(conf);
     }
 
-    if (SCMHAUtils.isSCMHAEnabled(conf)) {
-      this.scmRatisSnapshotInfo = new SCMRatisSnapshotInfo(
-          scmStorageConfig.getCurrentDir());
-      this.scmRatisSnapshotDir = SCMHAUtils.createSCMRatisDir(conf);
-      initializeRatisServer();
-    } else {
-      scmRatisServer = null;
-    }
+    this.scmRatisSnapshotInfo = new SCMRatisSnapshotInfo(
+        scmStorageConfig.getCurrentDir());
+    this.scmRatisSnapshotDir = SCMHAUtils.createSCMRatisDir(conf);
 
     // Creates the SCM DBs or opens them if it exists.
     // A valid pointer to the store is required by all the other services 
below.
@@ -388,6 +383,38 @@ public final class StorageContainerManager extends 
ServiceRuntimeInfoImpl
   }
 
   /**
+   * Create an SCM instance based on the supplied configuration.
+   *
+   * @param conf        HDDS configuration
+   * @param configurator SCM configurator
+   * @return SCM instance
+   * @throws IOException, AuthenticationException
+   */
+  public static StorageContainerManager createSCM(
+      OzoneConfiguration conf, SCMConfigurator configurator)
+      throws IOException, AuthenticationException {
+    StorageContainerManager scm = new StorageContainerManager(
+          conf, configurator);
+    if (SCMHAUtils.isSCMHAEnabled(conf) && scm.getScmRatisServer() == null) {
+      SCMRatisServer scmRatisServer = initializeRatisServer(conf, scm);
+      scm.setScmRatisServer(scmRatisServer);
+    }
+    return scm;
+  }
+
+  /**
+   * Create an SCM instance based on the supplied configuration.
+   *
+   * @param conf        HDDS configuration
+   * @return SCM instance
+   * @throws IOException, AuthenticationException
+   */
+  public static StorageContainerManager createSCM(OzoneConfiguration conf)
+      throws IOException, AuthenticationException {
+    return createSCM(conf, new SCMConfigurator());
+  }
+
+  /**
    * This function initializes the following managers. If the configurator
    * specifies a value, we will use it, else we will use the default value.
    *
@@ -634,18 +661,6 @@ public final class StorageContainerManager extends 
ServiceRuntimeInfoImpl
   }
 
   /**
-   * Create an SCM instance based on the supplied configuration.
-   *
-   * @param conf        HDDS configuration
-   * @return SCM instance
-   * @throws IOException, AuthenticationException
-   */
-  public static StorageContainerManager createSCM(OzoneConfiguration conf)
-      throws IOException, AuthenticationException {
-    return new StorageContainerManager(conf);
-  }
-
-  /**
    * Routine to set up the Version info for StorageContainerManager.
    *
    * @param conf OzoneConfiguration
@@ -1137,18 +1152,20 @@ public final class StorageContainerManager extends 
ServiceRuntimeInfoImpl
     return this.clusterMap;
   }
 
-  private void initializeRatisServer() throws IOException {
-    if (scmRatisServer == null) {
-      SCMNodeDetails scmNodeDetails = SCMNodeDetails
-          .initStandAlone(configuration);
-      //TODO enable Ratis ring
-      scmRatisServer = SCMRatisServer.newSCMRatisServer(configuration, this,
-          scmNodeDetails, Collections.EMPTY_LIST);
-      if (scmRatisServer != null) {
-        LOG.info("SCM Ratis server initialized at port {}",
-            scmRatisServer.getServerPort());
-      }
-    }
+  private static SCMRatisServer initializeRatisServer(
+      OzoneConfiguration conf, StorageContainerManager scm) throws IOException 
{
+    SCMNodeDetails scmNodeDetails = SCMNodeDetails
+        .initStandAlone(conf);
+    //TODO enable Ratis group
+    SCMRatisServer scmRatisServer = SCMRatisServer.newSCMRatisServer(
+        conf.getObject(SCMRatisServer.SCMRatisServerConfiguration.class),
+        scm, scmNodeDetails, Collections.EMPTY_LIST,
+        SCMRatisServer.getSCMRatisDirectory(conf));
+    if (scmRatisServer != null) {
+      LOG.info("SCM Ratis server initialized at port {}",
+          scmRatisServer.getServerPort());
+    } // TODO error handling for scmRatisServer creation failure
+    return scmRatisServer;
   }
 
   @VisibleForTesting
@@ -1156,6 +1173,10 @@ public final class StorageContainerManager extends 
ServiceRuntimeInfoImpl
     return scmRatisServer;
   }
 
+  public void setScmRatisServer(SCMRatisServer scmRatisServer) {
+    this.scmRatisServer = scmRatisServer;
+  }
+
   @VisibleForTesting
   public SCMRatisSnapshotInfo getSnapshotInfo() {
     return scmRatisSnapshotInfo;
diff --git 
a/hadoop-hdds/server-scm/src/main/java/org/apache/hadoop/hdds/scm/server/ratis/SCMRatisServer.java
 
b/hadoop-hdds/server-scm/src/main/java/org/apache/hadoop/hdds/scm/server/ratis/SCMRatisServer.java
index 89a9d55..9ab8c66 100644
--- 
a/hadoop-hdds/server-scm/src/main/java/org/apache/hadoop/hdds/scm/server/ratis/SCMRatisServer.java
+++ 
b/hadoop-hdds/server-scm/src/main/java/org/apache/hadoop/hdds/scm/server/ratis/SCMRatisServer.java
@@ -20,8 +20,10 @@ package org.apache.hadoop.hdds.scm.server.ratis;
 
 import com.google.common.annotations.VisibleForTesting;
 import com.google.common.base.Strings;
+import org.apache.hadoop.hdds.conf.Config;
+import org.apache.hadoop.hdds.conf.ConfigGroup;
+import org.apache.hadoop.hdds.conf.ConfigType;
 import org.apache.hadoop.hdds.conf.ConfigurationSource;
-import org.apache.hadoop.hdds.conf.StorageUnit;
 import org.apache.hadoop.hdds.scm.ScmConfigKeys;
 import org.apache.hadoop.hdds.scm.ha.SCMNodeDetails;
 import org.apache.hadoop.hdds.scm.server.StorageContainerManager;
@@ -66,6 +68,11 @@ import java.util.concurrent.TimeUnit;
 import java.util.concurrent.atomic.AtomicLong;
 import java.util.concurrent.locks.ReentrantReadWriteLock;
 
+import static org.apache.hadoop.hdds.conf.ConfigTag.OZONE;
+import static org.apache.hadoop.hdds.conf.ConfigTag.RATIS;
+import static org.apache.hadoop.hdds.conf.ConfigTag.SCM;
+import static org.apache.hadoop.hdds.conf.ConfigTag.HA;
+
 /**
  * Class for SCM Ratis Server.
  */
@@ -76,6 +83,7 @@ public final class SCMRatisServer {
   private final StorageContainerManager scm;
   private final SCMStateMachine scmStateMachine;
 
+  private final String storageDir;
   private final int port;
   private final InetSocketAddress scmRatisAddress;
   private final RaftServer server;
@@ -100,14 +108,15 @@ public final class SCMRatisServer {
    * Creates a SCM Ratis Server.
    * @throws IOException
    */
-  private SCMRatisServer(ConfigurationSource conf,
-                         StorageContainerManager scm,
+  private SCMRatisServer(SCMRatisServerConfiguration conf,
+                         StorageContainerManager scm, String ratisStorageDir,
                          String raftGroupIdStr, RaftPeerId localRaftPeerId,
                          InetSocketAddress addr, List<RaftPeer> raftPeers)
       throws IOException {
     this.scm = scm;
     this.scmRatisAddress = addr;
     this.port = addr.getPort();
+    this.storageDir = ratisStorageDir;
     RaftProperties serverProperties = newRaftProperties(conf);
 
     this.raftPeerId = localRaftPeerId;
@@ -149,8 +158,9 @@ public final class SCMRatisServer {
    * Create a SCM Ratis Server instance.
    */
   public static SCMRatisServer newSCMRatisServer(
-      ConfigurationSource conf, StorageContainerManager scm,
-      SCMNodeDetails scmNodeDetails, List<SCMNodeDetails> peers)
+      SCMRatisServerConfiguration conf, StorageContainerManager scm,
+      SCMNodeDetails scmNodeDetails, List<SCMNodeDetails> peers,
+      String ratisStorageDir)
       throws IOException {
     String scmServiceId = scmNodeDetails.getSCMServiceId();
 
@@ -174,8 +184,8 @@ public final class SCMRatisServer {
       raftPeers.add(raftPeer);
     }
 
-    return new SCMRatisServer(conf, scm, scmServiceId, localRaftPeerId,
-        ratisAddr, raftPeers);
+    return new SCMRatisServer(conf, scm, ratisStorageDir, scmServiceId,
+        localRaftPeerId, ratisAddr, raftPeers);
   }
 
   private UUID getRaftGroupIdFromOmServiceId(String scmServiceId) {
@@ -187,13 +197,10 @@ public final class SCMRatisServer {
     return new SCMStateMachine(this);
   }
 
-  private RaftProperties newRaftProperties(ConfigurationSource conf) {
+  private RaftProperties newRaftProperties(SCMRatisServerConfiguration conf) {
     final RaftProperties properties = new RaftProperties();
     // Set RPC type
-    final String rpcType = conf.get(
-        ScmConfigKeys.OZONE_SCM_RATIS_RPC_TYPE_KEY,
-        ScmConfigKeys.OZONE_SCM_RATIS_RPC_TYPE_DEFAULT);
-    final RpcType rpc = SupportedRpcType.valueOfIgnoreCase(rpcType);
+    final RpcType rpc = SupportedRpcType.valueOfIgnoreCase(conf.getRpcType());
     RaftConfigKeys.Rpc.setType(properties, rpc);
     // Set the ratis port number
     if (rpc == SupportedRpcType.GRPC) {
@@ -202,80 +209,41 @@ public final class SCMRatisServer {
       NettyConfigKeys.Server.setPort(properties, port);
     }
     // Set Ratis storage directory
-    String storageDir = SCMRatisServer.getSCMRatisDirectory(conf);
     RaftServerConfigKeys.setStorageDir(properties,
         Collections.singletonList(new File(storageDir)));
     // Set RAFT segment size
-    final int raftSegmentSize = (int) conf.getStorageSize(
-        ScmConfigKeys.OZONE_SCM_RATIS_SEGMENT_SIZE_KEY,
-        ScmConfigKeys.OZONE_SCM_RATIS_SEGMENT_SIZE_DEFAULT,
-        org.apache.hadoop.hdds.conf.StorageUnit.BYTES);
     RaftServerConfigKeys.Log.setSegmentSizeMax(properties,
-        SizeInBytes.valueOf(raftSegmentSize));
+        SizeInBytes.valueOf((long)conf.getSegmentSize()));
     // Set RAFT segment pre-allocated size
-    final int raftSegmentPreallocatedSize = (int) conf.getStorageSize(
-        ScmConfigKeys.OZONE_SCM_RATIS_SEGMENT_PREALLOCATED_SIZE_KEY,
-        ScmConfigKeys.OZONE_SCM_RATIS_SEGMENT_PREALLOCATED_SIZE_DEFAULT,
-        org.apache.hadoop.hdds.conf.StorageUnit.BYTES);
-    int logAppenderQueueNumElements = conf.getInt(
-        ScmConfigKeys.OZONE_SCM_RATIS_LOG_APPENDER_QUEUE_NUM_ELEMENTS,
-        ScmConfigKeys.OZONE_SCM_RATIS_LOG_APPENDER_QUEUE_NUM_ELEMENTS_DEFAULT);
-    final int logAppenderQueueByteLimit = (int) conf.getStorageSize(
-        ScmConfigKeys.OZONE_SCM_RATIS_LOG_APPENDER_QUEUE_BYTE_LIMIT,
-        ScmConfigKeys.OZONE_SCM_RATIS_LOG_APPENDER_QUEUE_BYTE_LIMIT_DEFAULT,
-        StorageUnit.BYTES);
     RaftServerConfigKeys.Log.Appender.setBufferElementLimit(properties,
-        logAppenderQueueNumElements);
+        (int)conf.getLogAppenderQueueByteLimit());
     RaftServerConfigKeys.Log.Appender.setBufferByteLimit(properties,
-        SizeInBytes.valueOf(logAppenderQueueByteLimit));
+        SizeInBytes.valueOf(conf.getLogAppenderQueueNum()));
     RaftServerConfigKeys.Log.setPreallocatedSize(properties,
-        SizeInBytes.valueOf(raftSegmentPreallocatedSize));
+        SizeInBytes.valueOf((int)conf.getPreallocatedSize()));
     RaftServerConfigKeys.Log.Appender.setInstallSnapshotEnabled(properties,
         false);
-    final int logPurgeGap = conf.getInt(
-        ScmConfigKeys.OZONE_SCM_RATIS_LOG_PURGE_GAP,
-        ScmConfigKeys.OZONE_SCM_RATIS_LOG_PURGE_GAP_DEFAULT);
-    RaftServerConfigKeys.Log.setPurgeGap(properties, logPurgeGap);
+    RaftServerConfigKeys.Log.setPurgeGap(properties, conf.getLogPurgeGap());
     // For grpc set the maximum message size
     // TODO: calculate the optimal max message size
     GrpcConfigKeys.setMessageSizeMax(properties,
-        SizeInBytes.valueOf(logAppenderQueueByteLimit));
+        SizeInBytes.valueOf((int)conf.getLogAppenderQueueByteLimit()));
 
     // Set the server request timeout
-    TimeUnit serverRequestTimeoutUnit =
-        ScmConfigKeys.OZONE_SCM_RATIS_SERVER_REQUEST_TIMEOUT_DEFAULT.getUnit();
-    long serverRequestTimeoutDuration = conf.getTimeDuration(
-        ScmConfigKeys.OZONE_SCM_RATIS_SERVER_REQUEST_TIMEOUT_KEY,
-        ScmConfigKeys.OZONE_SCM_RATIS_SERVER_REQUEST_TIMEOUT_DEFAULT
-            .getDuration(), serverRequestTimeoutUnit);
     final TimeDuration serverRequestTimeout = TimeDuration.valueOf(
-        serverRequestTimeoutDuration, serverRequestTimeoutUnit);
+        conf.getRequestTimeout(), TimeUnit.MILLISECONDS);
     RaftServerConfigKeys.Rpc.setRequestTimeout(properties,
         serverRequestTimeout);
     // Set timeout for server retry cache entry
-    TimeUnit retryCacheTimeoutUnit = ScmConfigKeys
-        .OZONE_SCM_RATIS_SERVER_RETRY_CACHE_TIMEOUT_DEFAULT.getUnit();
-    long retryCacheTimeoutDuration = conf.getTimeDuration(
-        ScmConfigKeys.OZONE_SCM_RATIS_SERVER_RETRY_CACHE_TIMEOUT_KEY,
-        ScmConfigKeys.OZONE_SCM_RATIS_SERVER_RETRY_CACHE_TIMEOUT_DEFAULT
-            .getDuration(), retryCacheTimeoutUnit);
     final TimeDuration retryCacheTimeout = TimeDuration.valueOf(
-        retryCacheTimeoutDuration, retryCacheTimeoutUnit);
+        conf.getRetryCacheTimeout(), TimeUnit.MILLISECONDS);
     RaftServerConfigKeys.RetryCache.setExpiryTime(properties,
         retryCacheTimeout);
     // Set the server min and max timeout
-    TimeUnit serverMinTimeoutUnit =
-        ScmConfigKeys.OZONE_SCM_RATIS_MINIMUM_TIMEOUT_DEFAULT.getUnit();
-    long serverMinTimeoutDuration = conf.getTimeDuration(
-        ScmConfigKeys.OZONE_SCM_RATIS_MINIMUM_TIMEOUT_KEY,
-        ScmConfigKeys.OZONE_SCM_RATIS_MINIMUM_TIMEOUT_DEFAULT
-            .getDuration(), serverMinTimeoutUnit);
     final TimeDuration serverMinTimeout = TimeDuration.valueOf(
-        serverMinTimeoutDuration, serverMinTimeoutUnit);
-    long serverMaxTimeoutDuration =
-        serverMinTimeout.toLong(TimeUnit.MILLISECONDS) + 200;
+        conf.getMinTimeout(), TimeUnit.MILLISECONDS);
     final TimeDuration serverMaxTimeout = TimeDuration.valueOf(
-        serverMaxTimeoutDuration, serverMinTimeoutUnit);
+        conf.getMinTimeout() + 200L, TimeUnit.MILLISECONDS);
     RaftServerConfigKeys.Rpc.setTimeoutMin(properties,
         serverMinTimeout);
     RaftServerConfigKeys.Rpc.setTimeoutMax(properties,
@@ -284,46 +252,24 @@ public final class SCMRatisServer {
     RaftServerConfigKeys.Log.setSegmentCacheNumMax(properties, 2);
     // TODO: set max write buffer size
     // Set the ratis leader election timeout
-    TimeUnit leaderElectionMinTimeoutUnit =
-        
ScmConfigKeys.OZONE_SCM_LEADER_ELECTION_MINIMUM_TIMEOUT_DURATION_DEFAULT
-            .getUnit();
-    long leaderElectionMinTimeoutduration = conf.getTimeDuration(
-        ScmConfigKeys.OZONE_SCM_LEADER_ELECTION_MINIMUM_TIMEOUT_DURATION_KEY,
-        
ScmConfigKeys.OZONE_SCM_LEADER_ELECTION_MINIMUM_TIMEOUT_DURATION_DEFAULT
-            .getDuration(), leaderElectionMinTimeoutUnit);
     final TimeDuration leaderElectionMinTimeout = TimeDuration.valueOf(
-        leaderElectionMinTimeoutduration, leaderElectionMinTimeoutUnit);
+        conf.getMinLeaderElectionTimeout(), TimeUnit.MILLISECONDS);
     RaftServerConfigKeys.Rpc.setTimeoutMin(properties,
         leaderElectionMinTimeout);
     long leaderElectionMaxTimeout = leaderElectionMinTimeout.toLong(
         TimeUnit.MILLISECONDS) + 200;
     RaftServerConfigKeys.Rpc.setTimeoutMax(properties,
         TimeDuration.valueOf(leaderElectionMaxTimeout, TimeUnit.MILLISECONDS));
-    TimeUnit nodeFailureTimeoutUnit =
-        ScmConfigKeys.OZONE_SCM_RATIS_SERVER_FAILURE_TIMEOUT_DURATION_DEFAULT
-            .getUnit();
-    long nodeFailureTimeoutDuration = conf.getTimeDuration(
-        ScmConfigKeys.OZONE_SCM_RATIS_SERVER_FAILURE_TIMEOUT_DURATION_KEY,
-        ScmConfigKeys.OZONE_SCM_RATIS_SERVER_FAILURE_TIMEOUT_DURATION_DEFAULT
-            .getDuration(), nodeFailureTimeoutUnit);
+
     final TimeDuration nodeFailureTimeout = TimeDuration.valueOf(
-        nodeFailureTimeoutDuration, nodeFailureTimeoutUnit);
+        conf.getFailureTimeout(), TimeUnit.MILLISECONDS);
     RaftServerConfigKeys.Notification.setNoLeaderTimeout(properties,
         nodeFailureTimeout);
     RaftServerConfigKeys.Rpc.setSlownessTimeout(properties,
         nodeFailureTimeout);
 
     // Ratis leader role check
-    TimeUnit roleCheckIntervalUnit =
-        ScmConfigKeys.OZONE_SCM_RATIS_SERVER_ROLE_CHECK_INTERVAL_DEFAULT
-            .getUnit();
-    long roleCheckIntervalDuration = conf.getTimeDuration(
-        ScmConfigKeys.OZONE_SCM_RATIS_SERVER_ROLE_CHECK_INTERVAL_KEY,
-        ScmConfigKeys.OZONE_SCM_RATIS_SERVER_ROLE_CHECK_INTERVAL_DEFAULT
-            .getDuration(), nodeFailureTimeoutUnit);
-    this.roleCheckIntervalMs = TimeDuration.valueOf(
-        roleCheckIntervalDuration, roleCheckIntervalUnit)
-        .toLong(TimeUnit.MILLISECONDS);
+    this.roleCheckIntervalMs = conf.getRoleCheckerInterval();
     this.roleCheckInitialDelayMs = leaderElectionMinTimeout
         .toLong(TimeUnit.MILLISECONDS);
 
@@ -479,4 +425,166 @@ public final class SCMRatisServer {
       this.roleCheckLock.writeLock().unlock();
     }
   }
+
+  /**
+   * Configuration used by SCM Ratis Server.
+   */
+  @ConfigGroup(prefix = "ozone.scm.ratis")
+  public static class SCMRatisServerConfiguration {
+    @Config(key = "rpc.type",
+        type = ConfigType.STRING,
+        defaultValue = "GRPC",
+        tags = {SCM, OZONE, HA, RATIS},
+        description = "Ratis supports different kinds of transports like" +
+            " netty, GRPC, Hadoop RPC etc. This picks one of those for" +
+            " this cluster."
+    )
+    private String rpcType;
+
+    @Config(key = "segment.size",
+        type = ConfigType.SIZE,
+        defaultValue = "16KB",
+        tags = {SCM, OZONE, HA, RATIS},
+        description = "The size of the raft segment used by Apache Ratis on" +
+            " SCM. (16 KB by default)"
+    )
+    private double segmentSize = 16 * 1024;
+
+    @Config(key = "segment.preallocated.size",
+        type = ConfigType.SIZE,
+        defaultValue = "16KB",
+        tags = {SCM, OZONE, HA, RATIS},
+        description = "The size of the buffer which is preallocated for" +
+            " raft segment used by Apache Ratis on SCM.(16 KB by default)"
+    )
+    private double preallocatedSize = 16 * 1024;
+
+    @Config(key = "log.appender.queue.num-elements",
+        type = ConfigType.INT,
+        defaultValue = "1024",
+        tags = {SCM, OZONE, HA, RATIS},
+        description = "Number of operation pending with Raft's Log Worker."
+    )
+    private int logAppenderQueueNum = 1024;
+
+    @Config(key = "log.appender.queue.byte-limit",
+        type = ConfigType.SIZE,
+        defaultValue = "32MB",
+        tags = {SCM, OZONE, HA, RATIS},
+        description = "Byte limit for Raft's Log Worker queue."
+    )
+    private double logAppenderQueueByteLimit = 32 * 1024 * 1024;
+
+    @Config(key = "log.purge.gap",
+        type = ConfigType.INT,
+        defaultValue = "1000000",
+        tags = {SCM, OZONE, HA, RATIS},
+        description = "The minimum gap between log indices for Raft server to" 
+
+            " purge its log segments after taking snapshot."
+    )
+    private int logPurgeGap = 1000000;
+
+    @Config(key = "server.request.timeout",
+        type = ConfigType.TIME,
+        defaultValue = "3s",
+        tags = {SCM, OZONE, HA, RATIS},
+        description = "The timeout duration for SCM's ratis server request."
+    )
+    private long requestTimeout = 3 * 1000L;
+
+    @Config(key = "server.retry.cache.timeout",
+        type = ConfigType.TIME,
+        defaultValue = "60s",
+        tags = {SCM, OZONE, HA, RATIS},
+        description = "Retry Cache entry timeout for SCM's ratis server."
+    )
+    private long retryCacheTimeout = 60 * 1000L;
+
+    @Config(key = "minimum.timeout",
+        type = ConfigType.TIME,
+        defaultValue = "1s",
+        tags = {SCM, OZONE, HA, RATIS},
+        description = "The minimum timeout duration for SCM's Ratis server 
rpc."
+    )
+    private long minTimeout = 1 * 1000L;
+
+    @Config(key = "leader.election.minimum.timeout.duration",
+        type = ConfigType.TIME,
+        defaultValue = "1s",
+        tags = {SCM, OZONE, HA, RATIS},
+        description = "The minimum timeout duration for SCM ratis leader" +
+            " election. Default is 1s."
+    )
+    private long minLeaderElectionTimeout = 1 * 1000L;
+
+    @Config(key = "server.failure.timeout.duration",
+        type = ConfigType.TIME,
+        defaultValue = "120s",
+        tags = {SCM, OZONE, HA, RATIS},
+        description = "The timeout duration for ratis server failure" +
+            " detection, once the threshold has reached, the ratis state" +
+            " machine will be informed about the failure in the ratis ring."
+    )
+    private long failureTimeout = 120 * 1000L;
+
+    @Config(key = "server.role.check.interval",
+        type = ConfigType.TIME,
+        defaultValue = "15s",
+        tags = {SCM, OZONE, HA, RATIS},
+        description = "The interval between SCM leader performing a role" +
+            " check on its ratis server. Ratis server informs SCM if it loses" 
+
+            " the leader role. The scheduled check is an secondary check to" +
+            " ensure that the leader role is updated periodically"
+    )
+    private long roleCheckerInterval = 15 * 1000L;
+
+    public String getRpcType() {
+      return rpcType;
+    }
+
+    public double getSegmentSize() {
+      return segmentSize;
+    }
+
+    public double getPreallocatedSize() {
+      return preallocatedSize;
+    }
+
+    public int getLogAppenderQueueNum() {
+      return logAppenderQueueNum;
+    }
+
+    public double getLogAppenderQueueByteLimit() {
+      return logAppenderQueueByteLimit;
+    }
+
+    public int getLogPurgeGap() {
+      return logPurgeGap;
+    }
+
+    public long getRequestTimeout() {
+      return requestTimeout;
+    }
+
+    public long getRetryCacheTimeout() {
+      return retryCacheTimeout;
+    }
+
+    public long getMinTimeout() {
+      return minTimeout;
+    }
+
+    public long getMinLeaderElectionTimeout() {
+      return minLeaderElectionTimeout;
+    }
+
+    public long getFailureTimeout() {
+      return failureTimeout;
+    }
+
+
+    public long getRoleCheckerInterval() {
+      return roleCheckerInterval;
+    }
+  }
 }
diff --git 
a/hadoop-hdds/server-scm/src/test/java/org/apache/hadoop/hdds/scm/TestUtils.java
 
b/hadoop-hdds/server-scm/src/test/java/org/apache/hadoop/hdds/scm/TestUtils.java
index 64752da..5d1ed46 100644
--- 
a/hadoop-hdds/server-scm/src/test/java/org/apache/hadoop/hdds/scm/TestUtils.java
+++ 
b/hadoop-hdds/server-scm/src/test/java/org/apache/hadoop/hdds/scm/TestUtils.java
@@ -497,7 +497,7 @@ public final class TestUtils {
       // writes the version file properties
       scmStore.initialize();
     }
-    return new StorageContainerManager(conf, configurator);
+    return StorageContainerManager.createSCM(conf, configurator);
   }
 
   public static ContainerInfo getContainer(
diff --git 
a/hadoop-hdds/server-scm/src/test/java/org/apache/hadoop/hdds/scm/server/ratis/TestSCMRatisServer.java
 
b/hadoop-hdds/server-scm/src/test/java/org/apache/hadoop/hdds/scm/server/ratis/TestSCMRatisServer.java
index d6981d3..4079965 100644
--- 
a/hadoop-hdds/server-scm/src/test/java/org/apache/hadoop/hdds/scm/server/ratis/TestSCMRatisServer.java
+++ 
b/hadoop-hdds/server-scm/src/test/java/org/apache/hadoop/hdds/scm/server/ratis/TestSCMRatisServer.java
@@ -117,8 +117,10 @@ public class TestSCMRatisServer {
     // Starts a single node Ratis server
     scmRatisServer.stop();
     SCMRatisServer newScmRatisServer = SCMRatisServer
-        .newSCMRatisServer(newConf, scm, nodeDetails,
-            Collections.emptyList());
+        .newSCMRatisServer(newConf.getObject(SCMRatisServer
+                .SCMRatisServerConfiguration.class), scm, nodeDetails,
+            Collections.emptyList(),
+            SCMRatisServer.getSCMRatisDirectory(newConf));
     newScmRatisServer.start();
 
     UUID uuid = UUID.nameUUIDFromBytes(customScmServiceId.getBytes());
diff --git 
a/hadoop-ozone/tools/src/main/java/org/apache/hadoop/ozone/genesis/GenesisUtil.java
 
b/hadoop-ozone/tools/src/main/java/org/apache/hadoop/ozone/genesis/GenesisUtil.java
index 797c805..b1a79d9 100644
--- 
a/hadoop-ozone/tools/src/main/java/org/apache/hadoop/ozone/genesis/GenesisUtil.java
+++ 
b/hadoop-ozone/tools/src/main/java/org/apache/hadoop/ozone/genesis/GenesisUtil.java
@@ -133,7 +133,7 @@ public final class GenesisUtil {
       // writes the version file properties
       scmStore.initialize();
     }
-    return new StorageContainerManager(conf, configurator);
+    return StorageContainerManager.createSCM(conf, configurator);
   }
 
   static void configureSCM(OzoneConfiguration conf, int numHandlers) {


---------------------------------------------------------------------
To unsubscribe, e-mail: ozone-commits-unsubscr...@hadoop.apache.org
For additional commands, e-mail: ozone-commits-h...@hadoop.apache.org

Reply via email to