Parcourir la source

HDDS-858. Start a Standalone Ratis Server on OM

Hanisha Koneru il y a 6 ans
Parent
commit
019836b113

+ 97 - 1
hadoop-hdds/common/src/main/resources/ozone-default.xml

@@ -1412,7 +1412,103 @@
     <value>1MB</value>
     <tag>OZONE, CLIENT, MANAGEMENT</tag>
     <description>Checksum will be computed for every bytes per checksum number
-      of bytes and stored sequentially.
+      of bytes and stored sequentially. The minimum value for this config is
+      256KB.
+    </description>
+  </property>
+
+  <property>
+    <name>ozone.om.ratis.enable</name>
+    <value>false</value>
+    <tag>OZONE, OM, RATIS, MANAGEMENT</tag>
+    <description>Property to enable or disable Ratis server on OM.
+    Please note - this is a temporary property to disable OM Ratis server.
+    </description>
+  </property>
+
+  <property>
+    <name>ozone.om.ratis.port</name>
+    <value>9872</value>
+    <tag>OZONE, OM, RATIS</tag>
+    <description>
+      The port number of the OzoneManager's Ratis server.
+    </description>
+  </property>
+
+  <property>
+    <name>ozone.om.ratis.random.port</name>
+    <value>false</value>
+    <tag>OZONE, OM, RATIS, DEBUG</tag>
+    <description>Allocates a random free port for OM's Ratis server. This is
+      used only while running unit tests.
+    </description>
+  </property>
+
+  <property>
+    <name>ozone.om.ratis.rpc.type</name>
+    <value>GRPC</value>
+    <tag>OZONE, OM, RATIS, MANAGEMENT</tag>
+    <description>Ratis supports different kinds of transports like netty, GRPC,
+      Hadoop RPC etc. This picks one of those for this cluster.
+    </description>
+  </property>
+
+  <property>
+    <name>ozone.om.ratis.storage.dir</name>
+    <value/>
+    <tag>OZONE, OM, STORAGE, MANAGEMENT, RATIS</tag>
+    <description>This directory is used for storing OM's Ratis metadata like
+      logs. If this is not set then default metadata dirs is used. A warning
+      will be logged if this not set. Ideally, this should be mapped to a
+      fast disk like an SSD.
+    </description>
+  </property>
+
+  <property>
+    <name>ozone.om.ratis.segment.size</name>
+    <value>16KB</value>
+    <tag>OZONE, OM, RATIS, PERFORMANCE</tag>
+    <description>The size of the raft segment used by Apache Ratis on OM.
+      (16 KB by default)
+    </description>
+  </property>
+
+  <property>
+    <name>ozone.om.ratis.segment.preallocated.size</name>
+    <value>128MB</value>
+    <tag>OZONE, OM, RATIS, PERFORMANCE</tag>
+    <description>The size of the buffer which is preallocated for raft segment
+      used by Apache Ratis on OM.(128 MB by default)
+    </description>
+  </property>
+
+  <property>
+    <name>ozone.om.ratis.server.request.timeout</name>
+    <value>3s</value>
+    <tag>OZONE, OM, RATIS, MANAGEMENT</tag>
+    <description>The timeout duration for OM's ratis server request .</description>
+  </property>
+
+  <property>
+    <name>ozone.om.ratis.server.retry.cache.timeout</name>
+    <value>600000ms</value>
+    <tag>OZONE, OM, RATIS, MANAGEMENT</tag>
+    <description>Retry Cache entry timeout for OM's ratis server.</description>
+  </property>
+
+  <property>
+    <name>ozone.om.ratis.minimum.timeout</name>
+    <value>1s</value>
+    <tag>OZONE, OM, RATIS, MANAGEMENT</tag>
+    <description>The minimum timeout duration for OM's Ratis server rpc.
+    </description>
+  </property>
+
+  <property>
+    <name>ozone.om.ratis.client.request.timeout</name>
+    <value>3s</value>
+    <tag>OZONE, OM, RATIS, MANAGEMENT</tag>
+    <description>The timeout duration for OM Ratis client request.
     </description>
   </property>
 

+ 9 - 6
hadoop-hdds/container-service/src/main/java/org/apache/hadoop/hdds/scm/HddsServerUtil.java

@@ -337,13 +337,16 @@ public final class HddsServerUtil {
             OzoneConfigKeys.DFS_CONTAINER_RATIS_DATANODE_STORAGE_DIR);
 
     if (Strings.isNullOrEmpty(storageDir)) {
-      LOG.warn("Storage directory for Ratis is not configured." +
-          "Mapping Ratis storage under {}. It is a good idea " +
-          "to map this to an SSD disk. Falling back to {}",
-          storageDir, HddsConfigKeys.OZONE_METADATA_DIRS);
-      File metaDirPath = ServerUtils.getOzoneMetaDirPath(conf);
-      storageDir = (new File (metaDirPath, "ratis")).getPath();
+      storageDir = getDefaultRatisDirectory(conf);
     }
     return storageDir;
   }
+
+  public static String getDefaultRatisDirectory(Configuration conf) {
+    LOG.warn("Storage directory for Ratis is not configured. It is a good " +
+            "idea to map this to an SSD disk. Falling back to {}",
+        HddsConfigKeys.OZONE_METADATA_DIRS);
+    File metaDirPath = ServerUtils.getOzoneMetaDirPath(conf);
+    return (new File(metaDirPath, "ratis")).getPath();
+  }
 }

+ 60 - 0
hadoop-ozone/common/src/main/java/org/apache/hadoop/ozone/om/OMConfigKeys.java

@@ -17,7 +17,10 @@
 
 package org.apache.hadoop.ozone.om;
 
+import java.util.concurrent.TimeUnit;
+
 import org.apache.hadoop.ozone.OzoneAcl;
+import org.apache.ratis.util.TimeDuration;
 
 /**
  * Ozone Manager Constants.
@@ -86,4 +89,61 @@ public final class OMConfigKeys {
   public static final String OZONE_OM_METRICS_SAVE_INTERVAL =
       "ozone.om.save.metrics.interval";
   public static final String OZONE_OM_METRICS_SAVE_INTERVAL_DEFAULT = "5m";
+
+  /**
+   * OM Ratis related configurations.
+   */
+  public static final String OZONE_OM_RATIS_ENABLE_KEY
+      = "ozone.om.ratis.enable";
+  public static final boolean OZONE_OM_RATIS_ENABLE_DEFAULT
+      = false;
+  public static final String OZONE_OM_RATIS_PORT_KEY
+      = "ozone.om.ratis.port";
+  public static final int OZONE_OM_RATIS_PORT_DEFAULT
+      = 9872;
+  // When set to true, allocate a random free port for ozone ratis server
+  public static final String OZONE_OM_RATIS_RANDOM_PORT_KEY =
+      "ozone.om.ratis.random.port";
+  public static final boolean OZONE_OM_RATIS_RANDOM_PORT_KEY_DEFAULT
+      = false;
+  public static final String OZONE_OM_RATIS_RPC_TYPE_KEY
+      = "ozone.om.ratis.rpc.type";
+  public static final String OZONE_OM_RATIS_RPC_TYPE_DEFAULT
+      = "GRPC";
+
+  // OM Ratis Log configurations
+  public static final String OZONE_OM_RATIS_STORAGE_DIR
+      = "ozone.om.ratis.storage.dir";
+  public static final String OZONE_OM_RATIS_SEGMENT_SIZE_KEY
+      = "ozone.om.ratis.segment.size";
+  public static final String OZONE_OM_RATIS_SEGMENT_SIZE_DEFAULT
+      = "16KB";
+  public static final String OZONE_OM_RATIS_SEGMENT_PREALLOCATED_SIZE_KEY
+      = "ozone.om.ratis.segment.preallocated.size";
+  public static final String OZONE_OM_RATIS_SEGMENT_PREALLOCATED_SIZE_DEFAULT
+      = "128MB";
+
+  // OM Ratis server configurations
+  public static final String OZONE_OM_RATIS_SERVER_REQUEST_TIMEOUT_KEY
+      = "ozone.om.ratis.server.request.timeout";
+  public static final TimeDuration
+      OZONE_OM_RATIS_SERVER_REQUEST_TIMEOUT_DEFAULT
+      = TimeDuration.valueOf(3000, TimeUnit.MILLISECONDS);
+  public static final String
+      OZONE_OM_RATIS_SERVER_RETRY_CACHE_TIMEOUT_KEY
+      = "ozone.om.ratis.server.retry.cache.timeout";
+  public static final TimeDuration
+      OZONE_OM_RATIS_SERVER_RETRY_CACHE_TIMEOUT_DEFAULT
+      = TimeDuration.valueOf(600000, TimeUnit.MILLISECONDS);
+  public static final String OZONE_OM_RATIS_MINIMUM_TIMEOUT_KEY
+      = "ozone.om.ratis.minimum.timeout";
+  public static final TimeDuration OZONE_OM_RATIS_MINIMUM_TIMEOUT_DEFAULT
+      = TimeDuration.valueOf(1, TimeUnit.SECONDS);
+
+  // OM Ratis client configurations
+  public static final String OZONE_OM_RATIS_CLIENT_REQUEST_TIMEOUT_KEY
+      = "ozone.om.ratis.client.request.timeout";
+  public static final TimeDuration
+      OZONE_OM_RATIS_CLIENT_REQUEST_TIMEOUT_DEFAULT
+      = TimeDuration.valueOf(3000, TimeUnit.MILLISECONDS);
 }

+ 22 - 0
hadoop-ozone/integration-test/src/test/java/org/apache/hadoop/ozone/om/TestOzoneManager.java

@@ -62,6 +62,7 @@ import org.apache.hadoop.util.Time;
 import org.apache.hadoop.utils.db.Table;
 import org.apache.hadoop.utils.db.Table.KeyValue;
 import org.apache.hadoop.utils.db.TableIterator;
+import org.apache.ratis.util.LifeCycle;
 import org.junit.AfterClass;
 import org.junit.Assert;
 import org.junit.BeforeClass;
@@ -1368,4 +1369,25 @@ public class TestOzoneManager {
     Assert.assertEquals(NetUtils.createSocketAddr(
         conf.get(OZONE_SCM_CLIENT_ADDRESS_KEY)), scmAddress);
   }
+
+  /**
+   * Test that OM Ratis server is started only when OZONE_OM_RATIS_ENABLE_KEY is
+   * set to true.
+   */
+  @Test
+  public void testRatsiServerOnOmInitialization() throws IOException {
+    // OM Ratis server should not be started when OZONE_OM_RATIS_ENABLE_KEY
+    // is not set to true
+    Assert.assertNull("OM Ratis server started though OM Ratis is disabled.",
+        cluster.getOzoneManager().getOmRatisServerState());
+
+    // Enable OM Ratis and restart OM
+    conf.setBoolean(OMConfigKeys.OZONE_OM_RATIS_ENABLE_KEY, true);
+    cluster.restartOzoneManager();
+
+    // On enabling OM Ratis, the Ratis server should be started
+    Assert.assertEquals("OM Ratis server did not start",
+        LifeCycle.State.RUNNING,
+        cluster.getOzoneManager().getOmRatisServerState());
+  }
 }

+ 31 - 0
hadoop-ozone/ozone-manager/src/main/java/org/apache/hadoop/ozone/om/OzoneManager.java

@@ -66,6 +66,7 @@ import org.apache.hadoop.ozone.om.helpers.OpenKeySession;
 import org.apache.hadoop.ozone.om.helpers.ServiceInfo;
 import org.apache.hadoop.ozone.om.protocol.OzoneManagerProtocol;
 import org.apache.hadoop.ozone.om.protocolPB.OzoneManagerProtocolPB;
+import org.apache.hadoop.ozone.om.ratis.OzoneManagerRatisServer;
 import org.apache.hadoop.ozone.protocol.proto.OzoneManagerProtocolProtos.OzoneAclInfo;
 import org.apache.hadoop.ozone.protocol.proto.OzoneManagerProtocolProtos.ServicePort;
 import org.apache.hadoop.ozone.protocolPB.OzoneManagerProtocolServerSideTranslatorPB;
@@ -73,6 +74,7 @@ import org.apache.hadoop.security.UserGroupInformation;
 import org.apache.hadoop.util.GenericOptionsParser;
 import org.apache.hadoop.util.ShutdownHookManager;
 import org.apache.hadoop.util.StringUtils;
+import org.apache.ratis.util.LifeCycle;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
@@ -136,6 +138,7 @@ public final class OzoneManager extends ServiceRuntimeInfoImpl
   private final OzoneConfiguration configuration;
   private RPC.Server omRpcServer;
   private InetSocketAddress omRpcAddress;
+  private OzoneManagerRatisServer omRatisServer;
   private final OMMetadataManager metadataManager;
   private final VolumeManager volumeManager;
   private final BucketManager bucketManager;
@@ -509,6 +512,15 @@ public final class OzoneManager extends ServiceRuntimeInfoImpl
     return omStorage;
   }
 
+  @VisibleForTesting
+  public LifeCycle.State getOmRatisServerState() {
+    if (omRatisServer == null) {
+      return null;
+    } else {
+      return omRatisServer.getServerState();
+    }
+  }
+
   /**
    * Get metadata manager.
    *
@@ -542,6 +554,22 @@ public final class OzoneManager extends ServiceRuntimeInfoImpl
     LOG.info(buildRpcServerStartMessage("OzoneManager RPC server",
         omRpcAddress));
 
+    boolean omRatisEnabled = configuration.getBoolean(
+        OMConfigKeys.OZONE_OM_RATIS_ENABLE_KEY,
+        OMConfigKeys.OZONE_OM_RATIS_ENABLE_DEFAULT);
+    // This is a temporary check. Once fully implemented, all OM state change
+    // should go through Ratis, either standalone (for non-HA) or replicated
+    // (for HA).
+    if (omRatisEnabled) {
+      omRatisServer = OzoneManagerRatisServer.newOMRatisServer(
+          omStorage.getOmId(), configuration);
+      omRatisServer.start();
+
+      LOG.info("OzoneManager Ratis server started at port {}",
+          omRatisServer.getServerPort());
+    } else {
+      omRatisServer = null;
+    }
 
     DefaultMetricsSystem.initialize("OzoneManager");
 
@@ -584,6 +612,9 @@ public final class OzoneManager extends ServiceRuntimeInfoImpl
       metricsTimer = null;
       scheduleOMMetricsWriteTask = null;
       omRpcServer.stop();
+      if (omRatisServer != null) {
+        omRatisServer.stop();
+      }
       keyManager.stop();
       httpServer.stop();
       metadataManager.stop();

+ 271 - 0
hadoop-ozone/ozone-manager/src/main/java/org/apache/hadoop/ozone/om/ratis/OzoneManagerRatisServer.java

@@ -0,0 +1,271 @@
+/*
+ * 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.om.ratis;
+
+import com.google.common.annotations.VisibleForTesting;
+import com.google.common.base.Strings;
+import java.io.File;
+import java.io.IOException;
+import java.net.InetSocketAddress;
+import java.net.ServerSocket;
+import java.net.SocketAddress;
+import java.util.Collections;
+import java.util.Objects;
+import java.util.concurrent.TimeUnit;
+
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.conf.StorageUnit;
+import org.apache.hadoop.hdds.scm.HddsServerUtil;
+import org.apache.hadoop.ozone.om.OMConfigKeys;
+import org.apache.ratis.RaftConfigKeys;
+import org.apache.ratis.client.RaftClientConfigKeys;
+import org.apache.ratis.conf.RaftProperties;
+import org.apache.ratis.grpc.GrpcConfigKeys;
+import org.apache.ratis.netty.NettyConfigKeys;
+import org.apache.ratis.protocol.RaftGroupId;
+import org.apache.ratis.protocol.RaftPeerId;
+import org.apache.ratis.rpc.RpcType;
+import org.apache.ratis.rpc.SupportedRpcType;
+import org.apache.ratis.server.RaftServer;
+import org.apache.ratis.server.RaftServerConfigKeys;
+import org.apache.ratis.statemachine.impl.BaseStateMachine;
+import org.apache.ratis.util.LifeCycle;
+import org.apache.ratis.util.SizeInBytes;
+import org.apache.ratis.util.TimeDuration;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+/**
+ * Creates a Ratis server endpoint for OM.
+ */
+public final class OzoneManagerRatisServer {
+  private static final Logger LOG = LoggerFactory
+      .getLogger(OzoneManagerRatisServer.class);
+
+  private final int port;
+  private final RaftServer server;
+
+  private OzoneManagerRatisServer(String omId, int port, Configuration conf)
+      throws IOException {
+    Objects.requireNonNull(omId, "omId == null");
+    this.port = port;
+    RaftProperties serverProperties = newRaftProperties(conf);
+
+    this.server = RaftServer.newBuilder()
+        .setServerId(RaftPeerId.valueOf(omId))
+        .setProperties(serverProperties)
+        .setStateMachineRegistry(this::getStateMachine)
+        .build();
+  }
+
+  public static OzoneManagerRatisServer newOMRatisServer(String omId,
+      Configuration ozoneConf) throws IOException {
+    int localPort = ozoneConf.getInt(
+        OMConfigKeys.OZONE_OM_RATIS_PORT_KEY,
+        OMConfigKeys.OZONE_OM_RATIS_PORT_DEFAULT);
+
+    // Get an available port on current node and
+    // use that as the container port
+    if (ozoneConf.getBoolean(
+        OMConfigKeys.OZONE_OM_RATIS_RANDOM_PORT_KEY,
+        OMConfigKeys.OZONE_OM_RATIS_RANDOM_PORT_KEY_DEFAULT)) {
+      try (ServerSocket socket = new ServerSocket()) {
+        socket.setReuseAddress(true);
+        SocketAddress address = new InetSocketAddress(0);
+        socket.bind(address);
+        localPort = socket.getLocalPort();
+        LOG.info("Found a free port for the OM Ratis server : {}", localPort);
+      } catch (IOException e) {
+        LOG.error("Unable find a random free port for the server, "
+            + "fallback to use default port {}", localPort, e);
+      }
+    }
+    return new OzoneManagerRatisServer(omId, localPort, ozoneConf);
+  }
+
+  /**
+   * Return a dummy StateMachine.
+   * TODO: Implement a state machine on OM.
+   */
+  private BaseStateMachine getStateMachine(RaftGroupId gid) {
+    return new BaseStateMachine();
+  }
+
+  public void start() throws IOException {
+    LOG.info("Starting {} {} at port {}", getClass().getSimpleName(),
+        server.getId(), port);
+    server.start();
+  }
+
+  public void stop() {
+    try {
+      server.close();
+    } catch (IOException e) {
+      throw new RuntimeException(e);
+    }
+  }
+
+  private RaftProperties newRaftProperties(Configuration conf) {
+    final RaftProperties properties = new RaftProperties();
+
+    // Set RPC type
+    final String rpcType = conf.get(
+        OMConfigKeys.OZONE_OM_RATIS_RPC_TYPE_KEY,
+        OMConfigKeys.OZONE_OM_RATIS_RPC_TYPE_DEFAULT);
+    final RpcType rpc = SupportedRpcType.valueOfIgnoreCase(rpcType);
+    RaftConfigKeys.Rpc.setType(properties, rpc);
+
+    // Set the ratis port number
+    if (rpc == SupportedRpcType.GRPC) {
+      GrpcConfigKeys.Server.setPort(properties, port);
+    } else if (rpc == SupportedRpcType.NETTY) {
+      NettyConfigKeys.Server.setPort(properties, port);
+    }
+
+    // Set Ratis storage directory
+    String storageDir = getOMRatisDirectory(conf);
+    RaftServerConfigKeys.setStorageDirs(properties,
+        Collections.singletonList(new File(storageDir)));
+
+    // Set RAFT segment size
+    final int raftSegmentSize = (int) conf.getStorageSize(
+        OMConfigKeys.OZONE_OM_RATIS_SEGMENT_SIZE_KEY,
+        OMConfigKeys.OZONE_OM_RATIS_SEGMENT_SIZE_DEFAULT,
+        StorageUnit.BYTES);
+    RaftServerConfigKeys.Log.setSegmentSizeMax(properties,
+        SizeInBytes.valueOf(raftSegmentSize));
+
+    // Set RAFT segment pre-allocated size
+    final int raftSegmentPreallocatedSize = (int) conf.getStorageSize(
+        OMConfigKeys.OZONE_OM_RATIS_SEGMENT_PREALLOCATED_SIZE_KEY,
+        OMConfigKeys.OZONE_OM_RATIS_SEGMENT_PREALLOCATED_SIZE_DEFAULT,
+        StorageUnit.BYTES);
+    RaftServerConfigKeys.Log.Appender.setBufferCapacity(properties,
+        SizeInBytes.valueOf(raftSegmentPreallocatedSize));
+    RaftServerConfigKeys.Log.setPreallocatedSize(properties,
+        SizeInBytes.valueOf(raftSegmentPreallocatedSize));
+
+    // For grpc set the maximum message size
+    // TODO: calculate the max message size based on the max size of a
+    // PutSmallFileRequest's file size limit
+    GrpcConfigKeys.setMessageSizeMax(properties,
+        SizeInBytes.valueOf(raftSegmentPreallocatedSize));
+
+    // Set the server request timeout
+    TimeUnit serverRequestTimeoutUnit =
+        OMConfigKeys.OZONE_OM_RATIS_SERVER_REQUEST_TIMEOUT_DEFAULT.getUnit();
+    long serverRequestTimeoutDuration = conf.getTimeDuration(
+        OMConfigKeys.OZONE_OM_RATIS_SERVER_REQUEST_TIMEOUT_KEY,
+        OMConfigKeys.OZONE_OM_RATIS_SERVER_REQUEST_TIMEOUT_DEFAULT
+            .getDuration(), serverRequestTimeoutUnit);
+    final TimeDuration serverRequestTimeout = TimeDuration.valueOf(
+        serverRequestTimeoutDuration, serverRequestTimeoutUnit);
+    RaftServerConfigKeys.Rpc.setRequestTimeout(properties,
+        serverRequestTimeout);
+
+    // Set timeout for server retry cache entry
+    TimeUnit retryCacheTimeoutUnit = OMConfigKeys
+        .OZONE_OM_RATIS_SERVER_RETRY_CACHE_TIMEOUT_DEFAULT.getUnit();
+    long retryCacheTimeoutDuration = conf.getTimeDuration(
+        OMConfigKeys.OZONE_OM_RATIS_SERVER_RETRY_CACHE_TIMEOUT_KEY,
+        OMConfigKeys.OZONE_OM_RATIS_SERVER_RETRY_CACHE_TIMEOUT_DEFAULT
+            .getDuration(), retryCacheTimeoutUnit);
+    final TimeDuration retryCacheTimeout = TimeDuration.valueOf(
+        retryCacheTimeoutDuration, retryCacheTimeoutUnit);
+    RaftServerConfigKeys.RetryCache
+        .setExpiryTime(properties, retryCacheTimeout);
+
+    // Set the server min and max timeout
+    TimeUnit serverMinTimeoutUnit =
+        OMConfigKeys.OZONE_OM_RATIS_MINIMUM_TIMEOUT_DEFAULT.getUnit();
+    long serverMinTimeoutDuration = conf.getTimeDuration(
+        OMConfigKeys.OZONE_OM_RATIS_MINIMUM_TIMEOUT_KEY,
+        OMConfigKeys.OZONE_OM_RATIS_MINIMUM_TIMEOUT_DEFAULT
+            .getDuration(), serverMinTimeoutUnit);
+    final TimeDuration serverMinTimeout = TimeDuration.valueOf(
+        serverMinTimeoutDuration, serverMinTimeoutUnit);
+    long serverMaxTimeoutDuration =
+        serverMinTimeout.toLong(TimeUnit.MILLISECONDS) + 200;
+    final TimeDuration serverMaxTimeout = TimeDuration.valueOf(
+        serverMaxTimeoutDuration, serverMinTimeoutUnit);
+    RaftServerConfigKeys.Rpc.setTimeoutMin(properties,
+        serverMinTimeout);
+    RaftServerConfigKeys.Rpc.setTimeoutMax(properties,
+        serverMaxTimeout);
+
+    // Enable batch append on raft server
+    RaftServerConfigKeys.Log.Appender.setBatchEnabled(properties, true);
+
+    // Set the number of maximum cached segments
+    RaftServerConfigKeys.Log.setMaxCachedSegmentNum(properties, 2);
+
+    // Set the client request timeout
+    TimeUnit clientRequestTimeoutUnit =
+        OMConfigKeys.OZONE_OM_RATIS_CLIENT_REQUEST_TIMEOUT_DEFAULT.getUnit();
+    long clientRequestTimeoutDuration = conf.getTimeDuration(
+        OMConfigKeys.OZONE_OM_RATIS_CLIENT_REQUEST_TIMEOUT_KEY,
+        OMConfigKeys.OZONE_OM_RATIS_CLIENT_REQUEST_TIMEOUT_DEFAULT
+            .getDuration(), clientRequestTimeoutUnit);
+    final TimeDuration clientRequestTimeout = TimeDuration.valueOf(
+        clientRequestTimeoutDuration, clientRequestTimeoutUnit);
+    RaftClientConfigKeys.Rpc.setRequestTimeout(properties,
+        clientRequestTimeout);
+
+    // TODO: set max write buffer size
+
+    /**
+     * TODO: when state machine is implemented, enable StateMachineData sync
+     * and set sync timeout and number of sync retries.
+     */
+
+    /**
+     * TODO: set following ratis leader election related configs when
+     * replicated ratis server is implemented.
+     * 1. leader election timeout
+     * 2. node failure timeout
+     * 3.
+     */
+
+    /**
+     * TODO: when ratis snapshots are implemented, set snapshot threshold and
+     * queue size.
+     */
+
+    return properties;
+  }
+
+  public int getServerPort() {
+    return port;
+  }
+
+  @VisibleForTesting
+  public LifeCycle.State getServerState() {
+    return server.getLifeCycleState();
+  }
+
+  public static String getOMRatisDirectory(Configuration conf) {
+    String storageDir = conf.get(OMConfigKeys.OZONE_OM_RATIS_STORAGE_DIR);
+
+    if (Strings.isNullOrEmpty(storageDir)) {
+      storageDir = HddsServerUtil.getDefaultRatisDirectory(conf);
+    }
+    return storageDir;
+  }
+
+}

+ 22 - 0
hadoop-ozone/ozone-manager/src/main/java/org/apache/hadoop/ozone/om/ratis/package-info.java

@@ -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.
+ */
+package org.apache.hadoop.ozone.om.ratis;
+
+/**
+ * This package contains classes for the OM Ratis server implementation.
+ */

+ 69 - 0
hadoop-ozone/ozone-manager/src/test/java/org/apache/hadoop/ozone/om/ratis/TestOzoneManagerRatisServer.java

@@ -0,0 +1,69 @@
+/*
+ * 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.om.ratis;
+
+import java.nio.file.Path;
+import java.nio.file.Paths;
+import java.util.UUID;
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.hdds.HddsConfigKeys;
+import org.apache.hadoop.hdds.conf.OzoneConfiguration;
+import org.apache.hadoop.test.GenericTestUtils;
+import org.apache.ratis.util.LifeCycle;
+import org.junit.After;
+import org.junit.Assert;
+import org.junit.Before;
+import org.junit.Test;
+
+/**
+ * Test OM Ratis server.
+ */
+public class TestOzoneManagerRatisServer {
+
+  private Configuration conf;
+  private OzoneManagerRatisServer omRatisServer;
+  private String omID;
+
+  @Before
+  public void init() {
+    conf = new OzoneConfiguration();
+    omID = UUID.randomUUID().toString();
+    final String path = GenericTestUtils.getTempPath(omID);
+    Path metaDirPath = Paths.get(path, "om-meta");
+    conf.set(HddsConfigKeys.OZONE_METADATA_DIRS, metaDirPath.toString());
+  }
+
+  @After
+  public void shutdown() {
+    if (omRatisServer != null) {
+      omRatisServer.stop();
+    }
+  }
+
+  /**
+   * Start a OM Ratis Server and checks its state.
+   */
+  @Test
+  public void testStartOMRatisServer() throws Exception {
+    omRatisServer = OzoneManagerRatisServer.newOMRatisServer(omID, conf);
+    omRatisServer.start();
+    Assert.assertEquals("Ratis Server should be in running state",
+        LifeCycle.State.RUNNING, omRatisServer.getServerState());
+  }
+}