You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@ratis.apache.org by ji...@apache.org on 2017/01/31 21:16:46 UTC

[01/54] [abbrv] incubator-ratis git commit: Move o.a.r.s.* to o.a.r.s.impl.

Repository: incubator-ratis
Updated Branches:
  refs/heads/master d960d1e7f -> 813db4b78


http://git-wip-us.apache.org/repos/asf/incubator-ratis/blob/c36810ed/raft-server/src/main/java/org/apache/raft/statemachine/SnapshotInfo.java
----------------------------------------------------------------------
diff --git a/raft-server/src/main/java/org/apache/raft/statemachine/SnapshotInfo.java b/raft-server/src/main/java/org/apache/raft/statemachine/SnapshotInfo.java
index 6d82942..b7deb9a 100644
--- a/raft-server/src/main/java/org/apache/raft/statemachine/SnapshotInfo.java
+++ b/raft-server/src/main/java/org/apache/raft/statemachine/SnapshotInfo.java
@@ -17,7 +17,7 @@
  */
 package org.apache.raft.statemachine;
 
-import org.apache.raft.server.RaftConfiguration;
+import org.apache.raft.server.impl.RaftConfiguration;
 import org.apache.raft.server.protocol.TermIndex;
 import org.apache.raft.server.storage.FileInfo;
 

http://git-wip-us.apache.org/repos/asf/incubator-ratis/blob/c36810ed/raft-server/src/main/java/org/apache/raft/statemachine/SnapshotInfoImpl.java
----------------------------------------------------------------------
diff --git a/raft-server/src/main/java/org/apache/raft/statemachine/SnapshotInfoImpl.java b/raft-server/src/main/java/org/apache/raft/statemachine/SnapshotInfoImpl.java
index af8ba0e..670bfc7 100644
--- a/raft-server/src/main/java/org/apache/raft/statemachine/SnapshotInfoImpl.java
+++ b/raft-server/src/main/java/org/apache/raft/statemachine/SnapshotInfoImpl.java
@@ -17,7 +17,7 @@
  */
 package org.apache.raft.statemachine;
 
-import org.apache.raft.server.RaftConfiguration;
+import org.apache.raft.server.impl.RaftConfiguration;
 import org.apache.raft.server.protocol.TermIndex;
 import org.apache.raft.server.storage.FileInfo;
 

http://git-wip-us.apache.org/repos/asf/incubator-ratis/blob/c36810ed/raft-server/src/main/java/org/apache/raft/statemachine/StateMachine.java
----------------------------------------------------------------------
diff --git a/raft-server/src/main/java/org/apache/raft/statemachine/StateMachine.java b/raft-server/src/main/java/org/apache/raft/statemachine/StateMachine.java
index dc18175..935a83a 100644
--- a/raft-server/src/main/java/org/apache/raft/statemachine/StateMachine.java
+++ b/raft-server/src/main/java/org/apache/raft/statemachine/StateMachine.java
@@ -17,20 +17,20 @@
  */
 package org.apache.raft.statemachine;
 
-import java.io.Closeable;
-import java.io.IOException;
-import java.util.Collection;
-import java.util.concurrent.CompletableFuture;
-
 import org.apache.raft.conf.RaftProperties;
 import org.apache.raft.protocol.Message;
 import org.apache.raft.protocol.RaftClientReply;
 import org.apache.raft.protocol.RaftClientRequest;
-import org.apache.raft.server.RaftConfiguration;
 import org.apache.raft.server.RaftServerConfigKeys;
+import org.apache.raft.server.impl.RaftConfiguration;
 import org.apache.raft.server.storage.RaftStorage;
 import org.apache.raft.util.LifeCycle;
 
+import java.io.Closeable;
+import java.io.IOException;
+import java.util.Collection;
+import java.util.concurrent.CompletableFuture;
+
 public interface StateMachine extends Closeable {
   /**
    * Initializes the State Machine with the given properties and storage. The state machine is

http://git-wip-us.apache.org/repos/asf/incubator-ratis/blob/c36810ed/raft-server/src/main/java/org/apache/raft/statemachine/TermIndexTracker.java
----------------------------------------------------------------------
diff --git a/raft-server/src/main/java/org/apache/raft/statemachine/TermIndexTracker.java b/raft-server/src/main/java/org/apache/raft/statemachine/TermIndexTracker.java
index 1548e1d..694eef4 100644
--- a/raft-server/src/main/java/org/apache/raft/statemachine/TermIndexTracker.java
+++ b/raft-server/src/main/java/org/apache/raft/statemachine/TermIndexTracker.java
@@ -20,7 +20,7 @@ package org.apache.raft.statemachine;
 import com.google.common.base.Preconditions;
 import org.apache.raft.server.protocol.TermIndex;
 
-import static org.apache.raft.server.RaftServerConstants.INVALID_LOG_INDEX;
+import static org.apache.raft.server.impl.RaftServerConstants.INVALID_LOG_INDEX;
 
 /**
  * Tracks the term index that is applied to the StateMachine for simple state machines with

http://git-wip-us.apache.org/repos/asf/incubator-ratis/blob/c36810ed/raft-server/src/test/java/org/apache/raft/MiniRaftCluster.java
----------------------------------------------------------------------
diff --git a/raft-server/src/test/java/org/apache/raft/MiniRaftCluster.java b/raft-server/src/test/java/org/apache/raft/MiniRaftCluster.java
index 80cfa1f..4f0871f 100644
--- a/raft-server/src/test/java/org/apache/raft/MiniRaftCluster.java
+++ b/raft-server/src/test/java/org/apache/raft/MiniRaftCluster.java
@@ -23,7 +23,11 @@ import org.apache.raft.client.RaftClientRequestSender;
 import org.apache.raft.client.impl.RaftClientImpl;
 import org.apache.raft.conf.RaftProperties;
 import org.apache.raft.protocol.RaftPeer;
-import org.apache.raft.server.*;
+import org.apache.raft.server.RaftServerConfigKeys;
+import org.apache.raft.server.impl.DelayLocalExecutionInjection;
+import org.apache.raft.server.impl.RaftConfiguration;
+import org.apache.raft.server.impl.RaftServer;
+import org.apache.raft.server.impl.RaftServerRpc;
 import org.apache.raft.server.storage.MemoryRaftLog;
 import org.apache.raft.server.storage.RaftLog;
 import org.apache.raft.statemachine.BaseStateMachine;
@@ -40,7 +44,7 @@ import java.io.IOException;
 import java.util.*;
 import java.util.stream.Collectors;
 
-import static org.apache.raft.server.RaftServerConfigKeys.*;
+import static org.apache.raft.server.RaftServerConfigKeys.RAFT_SERVER_RPC_TIMEOUT_MIN_MS_DEFAULT;
 
 public abstract class MiniRaftCluster {
   public static final Logger LOG = LoggerFactory.getLogger(MiniRaftCluster.class);

http://git-wip-us.apache.org/repos/asf/incubator-ratis/blob/c36810ed/raft-server/src/test/java/org/apache/raft/RaftBasicTests.java
----------------------------------------------------------------------
diff --git a/raft-server/src/test/java/org/apache/raft/RaftBasicTests.java b/raft-server/src/test/java/org/apache/raft/RaftBasicTests.java
index 5a35afe..921e063 100644
--- a/raft-server/src/test/java/org/apache/raft/RaftBasicTests.java
+++ b/raft-server/src/test/java/org/apache/raft/RaftBasicTests.java
@@ -20,12 +20,8 @@ package org.apache.raft;
 import org.apache.raft.RaftTestUtil.SimpleMessage;
 import org.apache.raft.client.RaftClient;
 import org.apache.raft.conf.RaftProperties;
-import org.apache.raft.server.RaftServer;
-import org.junit.After;
-import org.junit.Assert;
-import org.junit.Before;
-import org.junit.Rule;
-import org.junit.Test;
+import org.apache.raft.server.impl.RaftServer;
+import org.junit.*;
 import org.junit.rules.Timeout;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;

http://git-wip-us.apache.org/repos/asf/incubator-ratis/blob/c36810ed/raft-server/src/test/java/org/apache/raft/RaftNotLeaderExceptionBaseTest.java
----------------------------------------------------------------------
diff --git a/raft-server/src/test/java/org/apache/raft/RaftNotLeaderExceptionBaseTest.java b/raft-server/src/test/java/org/apache/raft/RaftNotLeaderExceptionBaseTest.java
index 46a30c7..8a249e9 100644
--- a/raft-server/src/test/java/org/apache/raft/RaftNotLeaderExceptionBaseTest.java
+++ b/raft-server/src/test/java/org/apache/raft/RaftNotLeaderExceptionBaseTest.java
@@ -25,15 +25,11 @@ import org.apache.raft.client.impl.RaftClientImpl;
 import org.apache.raft.protocol.RaftClientReply;
 import org.apache.raft.protocol.RaftClientRequest;
 import org.apache.raft.protocol.RaftPeer;
-import org.apache.raft.server.RaftServer;
+import org.apache.raft.server.impl.RaftServer;
 import org.apache.raft.server.simulation.RequestHandler;
 import org.apache.raft.server.storage.RaftLog;
 import org.apache.raft.util.RaftUtils;
-import org.junit.After;
-import org.junit.Assert;
-import org.junit.Before;
-import org.junit.Rule;
-import org.junit.Test;
+import org.junit.*;
 import org.junit.rules.Timeout;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
@@ -42,7 +38,7 @@ import java.io.IOException;
 import java.util.Arrays;
 import java.util.Collection;
 
-import static org.apache.raft.server.RaftServerConstants.DEFAULT_SEQNUM;
+import static org.apache.raft.server.impl.RaftServerConstants.DEFAULT_SEQNUM;
 
 public abstract class RaftNotLeaderExceptionBaseTest {
   static {

http://git-wip-us.apache.org/repos/asf/incubator-ratis/blob/c36810ed/raft-server/src/test/java/org/apache/raft/RaftTestUtil.java
----------------------------------------------------------------------
diff --git a/raft-server/src/test/java/org/apache/raft/RaftTestUtil.java b/raft-server/src/test/java/org/apache/raft/RaftTestUtil.java
index ff7921d..92bf5c4 100644
--- a/raft-server/src/test/java/org/apache/raft/RaftTestUtil.java
+++ b/raft-server/src/test/java/org/apache/raft/RaftTestUtil.java
@@ -20,10 +20,10 @@ package org.apache.raft;
 import com.google.common.base.Preconditions;
 import org.apache.commons.lang.RandomStringUtils;
 import org.apache.raft.protocol.Message;
-import org.apache.raft.server.BlockRequestHandlingInjection;
-import org.apache.raft.server.DelayLocalExecutionInjection;
-import org.apache.raft.server.RaftServer;
 import org.apache.raft.server.RaftServerConfigKeys;
+import org.apache.raft.server.impl.BlockRequestHandlingInjection;
+import org.apache.raft.server.impl.DelayLocalExecutionInjection;
+import org.apache.raft.server.impl.RaftServer;
 import org.apache.raft.shaded.com.google.protobuf.ByteString;
 import org.apache.raft.shaded.proto.RaftProtos.LogEntryProto;
 import org.apache.raft.shaded.proto.RaftProtos.SMLogEntryProto;

http://git-wip-us.apache.org/repos/asf/incubator-ratis/blob/c36810ed/raft-server/src/test/java/org/apache/raft/server/BlockRequestHandlingInjection.java
----------------------------------------------------------------------
diff --git a/raft-server/src/test/java/org/apache/raft/server/BlockRequestHandlingInjection.java b/raft-server/src/test/java/org/apache/raft/server/BlockRequestHandlingInjection.java
deleted file mode 100644
index e1de3e5..0000000
--- a/raft-server/src/test/java/org/apache/raft/server/BlockRequestHandlingInjection.java
+++ /dev/null
@@ -1,84 +0,0 @@
-/**
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- *     http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-package org.apache.raft.server;
-
-import org.apache.raft.RaftTestUtil;
-import org.apache.raft.util.CodeInjectionForTesting;
-
-import java.util.Map;
-import java.util.concurrent.ConcurrentHashMap;
-
-/** Inject code to block a server from handling incoming requests. */
-public class BlockRequestHandlingInjection implements CodeInjectionForTesting.Code {
-  private static final BlockRequestHandlingInjection INSTANCE =
-      new BlockRequestHandlingInjection();
-
-  static {
-    CodeInjectionForTesting.put(RaftServer.REQUEST_VOTE, INSTANCE);
-    CodeInjectionForTesting.put(RaftServer.APPEND_ENTRIES, INSTANCE);
-    CodeInjectionForTesting.put(RaftServer.INSTALL_SNAPSHOT, INSTANCE);
-  }
-
-  public static BlockRequestHandlingInjection getInstance() {
-    return INSTANCE;
-  }
-
-  private final Map<String, Boolean> requestors = new ConcurrentHashMap<>();
-  private final Map<String, Boolean> repliers = new ConcurrentHashMap<>();
-
-  private BlockRequestHandlingInjection() {}
-
-  public void blockRequestor(String requestor) {
-    requestors.put(requestor, true);
-  }
-
-  public void unblockRequestor(String requestor) {
-    requestors.remove(requestor);
-  }
-
-  public void blockReplier(String replier) {
-    repliers.put(replier, true);
-  }
-
-  public void unblockReplier(String replier) {
-    repliers.remove(replier);
-  }
-
-  public void unblockAll() {
-    requestors.clear();
-    repliers.clear();
-  }
-
-  @Override
-  public boolean execute(String localId, String remoteId, Object... args) {
-    if (shouldBlock(localId, remoteId)) {
-      try {
-        RaftTestUtil.block(() -> shouldBlock(localId, remoteId));
-        return true;
-      } catch (InterruptedException e) {
-        LOG.debug("Interrupted while blocking request handling from " + remoteId
-            + " to " + localId);
-      }
-    }
-    return false;
-  }
-
-  private boolean shouldBlock(String localId, String remoteId) {
-    return repliers.containsKey(localId) || requestors.containsKey(remoteId);
-  }
-}

http://git-wip-us.apache.org/repos/asf/incubator-ratis/blob/c36810ed/raft-server/src/test/java/org/apache/raft/server/DelayLocalExecutionInjection.java
----------------------------------------------------------------------
diff --git a/raft-server/src/test/java/org/apache/raft/server/DelayLocalExecutionInjection.java b/raft-server/src/test/java/org/apache/raft/server/DelayLocalExecutionInjection.java
deleted file mode 100644
index 612b75f..0000000
--- a/raft-server/src/test/java/org/apache/raft/server/DelayLocalExecutionInjection.java
+++ /dev/null
@@ -1,67 +0,0 @@
-/**
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- *     http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-package org.apache.raft.server;
-
-import org.apache.raft.RaftTestUtil;
-import org.apache.raft.util.CodeInjectionForTesting;
-
-import java.util.Arrays;
-import java.util.Map;
-import java.util.concurrent.ConcurrentHashMap;
-import java.util.concurrent.atomic.AtomicInteger;
-
-/** Inject code to delay particular servers. */
-public class DelayLocalExecutionInjection implements CodeInjectionForTesting.Code {
-  private final Map<String, AtomicInteger> delays = new ConcurrentHashMap<>();
-
-  public DelayLocalExecutionInjection(String method) {
-    CodeInjectionForTesting.put(method, this);
-  }
-
-  public void clear() {
-    delays.clear();
-  }
-
-  public void setDelayMs(String id, int delayMs) {
-    AtomicInteger d = delays.get(id);
-    if (d == null) {
-      delays.put(id, d = new AtomicInteger());
-    }
-    d.set(delayMs);
-  }
-
-  public void removeDelay(String id) {
-    delays.remove(id);
-  }
-
-  @Override
-  public boolean execute(String localId, String remoteId, Object... args) {
-    final AtomicInteger d = delays.get(localId);
-    if (d == null) {
-      return false;
-    }
-    LOG.info("{} delay {} ms, args={}", localId, d.get(),
-        Arrays.toString(args));
-    try {
-      RaftTestUtil.delay(d::get);
-    } catch (InterruptedException e) {
-      LOG.debug("Interrupted while delaying " + localId);
-    }
-    return true;
-  }
-}

http://git-wip-us.apache.org/repos/asf/incubator-ratis/blob/c36810ed/raft-server/src/test/java/org/apache/raft/server/RaftReconfigurationBaseTest.java
----------------------------------------------------------------------
diff --git a/raft-server/src/test/java/org/apache/raft/server/RaftReconfigurationBaseTest.java b/raft-server/src/test/java/org/apache/raft/server/RaftReconfigurationBaseTest.java
deleted file mode 100644
index 1be0cd7..0000000
--- a/raft-server/src/test/java/org/apache/raft/server/RaftReconfigurationBaseTest.java
+++ /dev/null
@@ -1,576 +0,0 @@
-/**
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- *     http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-package org.apache.raft.server;
-
-import org.apache.log4j.Level;
-import org.apache.raft.MiniRaftCluster;
-import org.apache.raft.MiniRaftCluster.PeerChanges;
-import org.apache.raft.RaftTestUtil;
-import org.apache.raft.RaftTestUtil.SimpleMessage;
-import org.apache.raft.client.RaftClient;
-import org.apache.raft.client.RaftClientRequestSender;
-import org.apache.raft.client.impl.RaftClientImpl;
-import org.apache.raft.conf.RaftProperties;
-import org.apache.raft.protocol.*;
-import org.apache.raft.server.simulation.RequestHandler;
-import org.apache.raft.server.storage.RaftLog;
-import org.apache.raft.util.RaftUtils;
-import org.junit.Assert;
-import org.junit.BeforeClass;
-import org.junit.Test;
-import org.mockito.internal.util.reflection.Whitebox;
-import org.slf4j.Logger;
-import org.slf4j.LoggerFactory;
-
-import java.io.IOException;
-import java.util.ArrayList;
-import java.util.Arrays;
-import java.util.Collections;
-import java.util.concurrent.CountDownLatch;
-import java.util.concurrent.atomic.AtomicBoolean;
-import java.util.concurrent.atomic.AtomicReference;
-
-import static java.util.Arrays.asList;
-import static org.apache.raft.MiniRaftCluster.logSyncDelay;
-import static org.apache.raft.server.RaftServerConstants.DEFAULT_SEQNUM;
-import static org.apache.raft.server.RaftServerTestUtil.waitAndCheckNewConf;
-import static org.apache.raft.shaded.proto.RaftProtos.LogEntryProto.LogEntryBodyCase.CONFIGURATIONENTRY;
-
-public abstract class RaftReconfigurationBaseTest {
-  static {
-    RaftUtils.setLogLevel(RaftServer.LOG, Level.DEBUG);
-    RaftUtils.setLogLevel(RequestHandler.LOG, Level.DEBUG);
-    RaftUtils.setLogLevel(RaftClient.LOG, Level.DEBUG);
-  }
-  static final Logger LOG = LoggerFactory.getLogger(RaftReconfigurationBaseTest.class);
-
-  protected static final RaftProperties prop = new RaftProperties();
-
-  @BeforeClass
-  public static void setup() {
-    // set a small gap for tests
-    prop.setInt(RaftServerConfigKeys.RAFT_SERVER_STAGING_CATCHUP_GAP_KEY, 10);
-  }
-
-  public abstract MiniRaftCluster getCluster(int peerNum) throws IOException;
-
-  private static int getStagingGap() {
-    return prop.getInt(RaftServerConfigKeys.RAFT_SERVER_STAGING_CATCHUP_GAP_KEY,
-        RaftServerConfigKeys.RAFT_SERVER_STAGING_CATCHUP_GAP_DEFAULT);
-  }
-
-  /**
-   * add 2 new peers (3 peers -> 5 peers), no leader change
-   */
-  @Test
-  public void testAddPeers() throws Exception {
-    LOG.info("Start testAddPeers");
-    MiniRaftCluster cluster = getCluster(3);
-    cluster.start();
-    try {
-      RaftTestUtil.waitForLeader(cluster);
-
-      // add new peers
-      RaftPeer[] allPeers = cluster.addNewPeers(2, true).allPeersInNewConf;
-
-      // trigger setConfiguration
-      SetConfigurationRequest request = new SetConfigurationRequest("client",
-          cluster.getLeader().getId(), DEFAULT_SEQNUM, allPeers);
-      LOG.info("Start changing the configuration: {}", request);
-      cluster.getLeader().setConfiguration(request);
-
-      // wait for the new configuration to take effect
-      waitAndCheckNewConf(cluster, allPeers, 0, null);
-    } finally {
-      cluster.shutdown();
-    }
-  }
-
-  /**
-   * remove 2 peers (5 peers -> 3 peers), no leader change
-   */
-  @Test
-  public void testRemovePeers() throws Exception {
-    LOG.info("Start testRemovePeers");
-    MiniRaftCluster cluster = getCluster(5);
-    cluster.start();
-    try {
-      RaftTestUtil.waitForLeader(cluster);
-
-      // remove peers, leader still included in the new conf
-      RaftPeer[] allPeers = cluster
-          .removePeers(2, false, Collections.emptyList()).allPeersInNewConf;
-
-      // trigger setConfiguration
-      SetConfigurationRequest request = new SetConfigurationRequest("client",
-          cluster.getLeader().getId(), DEFAULT_SEQNUM, allPeers);
-      LOG.info("Start changing the configuration: {}", request);
-      cluster.getLeader().setConfiguration(request);
-
-      // wait for the new configuration to take effect
-      waitAndCheckNewConf(cluster, allPeers, 2, null);
-    } finally {
-      cluster.shutdown();
-    }
-  }
-
-  /**
-   * 5 peers -> 5 peers, remove 2 old, add 2 new, no leader change
-   */
-  @Test
-  public void testAddRemovePeers() throws Exception {
-    LOG.info("Start testAddRemovePeers");
-    testAddRemovePeers(false);
-  }
-
-  @Test
-  public void testLeaderStepDown() throws Exception {
-    LOG.info("Start testLeaderStepDown");
-    testAddRemovePeers(true);
-  }
-
-  private void testAddRemovePeers(boolean leaderStepdown) throws Exception {
-    MiniRaftCluster cluster = getCluster(5);
-    cluster.start();
-    try {
-      RaftTestUtil.waitForLeader(cluster);
-
-      PeerChanges change = cluster.addNewPeers(2, true);
-      RaftPeer[] allPeers = cluster.removePeers(2, leaderStepdown,
-          asList(change.newPeers)).allPeersInNewConf;
-
-      // trigger setConfiguration
-      SetConfigurationRequest request = new SetConfigurationRequest("client",
-          cluster.getLeader().getId(), DEFAULT_SEQNUM, allPeers);
-      LOG.info("Start changing the configuration: {}", request);
-      cluster.getLeader().setConfiguration(request);
-
-      // wait for the new configuration to take effect
-      waitAndCheckNewConf(cluster, allPeers, 2, null);
-    } finally {
-      cluster.shutdown();
-    }
-  }
-
-  @Test(timeout = 30000)
-  public void testReconfTwice() throws Exception {
-    LOG.info("Start testReconfTwice");
-    final MiniRaftCluster cluster = getCluster(3);
-    cluster.start();
-    try {
-      RaftTestUtil.waitForLeader(cluster);
-      final String leaderId = cluster.getLeader().getId();
-      final RaftClient client = cluster.createClient("client", leaderId);
-
-      // submit some msgs before reconf
-      for (int i = 0; i < getStagingGap() * 2; i++) {
-        RaftClientReply reply = client.send(new SimpleMessage("m" + i));
-        Assert.assertTrue(reply.isSuccess());
-      }
-
-      final AtomicBoolean reconf1 = new AtomicBoolean(false);
-      final AtomicBoolean reconf2 = new AtomicBoolean(false);
-      final AtomicReference<RaftPeer[]> finalPeers = new AtomicReference<>(null);
-      final AtomicReference<RaftPeer[]> deadPeers = new AtomicReference<>(null);
-      CountDownLatch latch = new CountDownLatch(1);
-      Thread clientThread = new Thread(() -> {
-        try {
-          PeerChanges c1 = cluster.addNewPeers(2, true);
-          LOG.info("Start changing the configuration: {}",
-              asList(c1.allPeersInNewConf));
-
-          RaftClientReply reply = client.setConfiguration(c1.allPeersInNewConf);
-          reconf1.set(reply.isSuccess());
-
-          PeerChanges c2 = cluster.removePeers(2, true, asList(c1.newPeers));
-          finalPeers.set(c2.allPeersInNewConf);
-          deadPeers.set(c2.removedPeers);
-
-          LOG.info("Start changing the configuration again: {}",
-              asList(c2.allPeersInNewConf));
-          reply = client.setConfiguration(c2.allPeersInNewConf);
-          reconf2.set(reply.isSuccess());
-
-          latch.countDown();
-          client.close();
-        } catch (IOException ignored) {
-        }
-      });
-      clientThread.start();
-
-      latch.await();
-      Assert.assertTrue(reconf1.get());
-      Assert.assertTrue(reconf2.get());
-      waitAndCheckNewConf(cluster, finalPeers.get(), 2, null);
-
-      // check configuration manager's internal state
-      // each reconf will generate two configurations: (old, new) and (new)
-      cluster.getServers().stream().filter(RaftServer::isAlive)
-          .forEach(server -> {
-        ConfigurationManager confManager =
-            (ConfigurationManager) Whitebox.getInternalState(server.getState(),
-                "configurationManager");
-        // each reconf will generate two configurations: (old, new) and (new)
-        Assert.assertEquals(5, confManager.numOfConf());
-      });
-    } finally {
-      cluster.shutdown();
-    }
-  }
-
-  @Test
-  public void testReconfTimeout() throws Exception {
-    LOG.info("Start testReconfTimeout");
-    // originally 3 peers
-    final MiniRaftCluster cluster = getCluster(3);
-    cluster.start();
-    try {
-      RaftTestUtil.waitForLeader(cluster);
-      final String leaderId = cluster.getLeader().getId();
-      final RaftClient client = cluster.createClient("client", leaderId);
-
-      PeerChanges c1 = cluster.addNewPeers(2, false);
-
-      LOG.info("Start changing the configuration: {}",
-          asList(c1.allPeersInNewConf));
-      Assert.assertFalse(cluster.getLeader().getRaftConf().isTransitional());
-
-      final RaftClientRequestSender sender = ((RaftClientImpl)client).getRequestSender();
-      final SetConfigurationRequest request = new SetConfigurationRequest(
-          "client", leaderId, DEFAULT_SEQNUM, c1.allPeersInNewConf);
-      try {
-        sender.sendRequest(request);
-        Assert.fail("did not get expected exception");
-      } catch (IOException e) {
-        Assert.assertTrue("Got exception " + e,
-            e instanceof ReconfigurationTimeoutException);
-      }
-
-      // the two new peers have not started yet, the bootstrapping must timeout
-      LOG.info(cluster.printServers());
-
-      // resend the same request, make sure the server has correctly reset its
-      // state so that we still get timeout instead of in-progress exception
-      try {
-        sender.sendRequest(request);
-        Assert.fail("did not get expected exception");
-      } catch (IOException e) {
-        Assert.assertTrue("Got exception " + e,
-            e instanceof ReconfigurationTimeoutException);
-      }
-
-      // start the two new peers
-      LOG.info("Start new peers");
-      for (RaftPeer np : c1.newPeers) {
-        cluster.startServer(np.getId());
-      }
-      Assert.assertTrue(client.setConfiguration(c1.allPeersInNewConf).isSuccess());
-      client.close();
-    } finally {
-      cluster.shutdown();
-    }
-  }
-
-  @Test
-  public void testBootstrapReconf() throws Exception {
-    LOG.info("Start testBootstrapReconf");
-    // originally 3 peers
-    final MiniRaftCluster cluster = getCluster(3);
-    cluster.start();
-    try {
-      RaftTestUtil.waitForLeader(cluster);
-      final String leaderId = cluster.getLeader().getId();
-      final RaftClient client = cluster.createClient("client", leaderId);
-
-      // submit some msgs before reconf
-      for (int i = 0; i < getStagingGap() * 2; i++) {
-        RaftClientReply reply = client.send(new SimpleMessage("m" + i));
-        Assert.assertTrue(reply.isSuccess());
-      }
-
-      PeerChanges c1 = cluster.addNewPeers(2, true);
-      LOG.info("Start changing the configuration: {}",
-          asList(c1.allPeersInNewConf));
-      final AtomicReference<Boolean> success = new AtomicReference<>();
-
-      Thread clientThread = new Thread(() -> {
-        try {
-          RaftClientReply reply = client.setConfiguration(c1.allPeersInNewConf);
-          success.set(reply.isSuccess());
-          client.close();
-        } catch (IOException ioe) {
-          LOG.error("FAILED", ioe);
-        }
-      });
-      clientThread.start();
-
-      Thread.sleep(5000);
-      LOG.info(cluster.printServers());
-      assertSuccess(success);
-
-      final RaftLog leaderLog = cluster.getLeader().getState().getLog();
-      for (RaftPeer newPeer : c1.newPeers) {
-        Assert.assertArrayEquals(leaderLog.getEntries(0, Long.MAX_VALUE),
-            cluster.getServer(newPeer.getId()).getState().getLog()
-                .getEntries(0, Long.MAX_VALUE));
-      }
-    } finally {
-      cluster.shutdown();
-    }
-  }
-
-  /**
-   * kill the leader before reconfiguration finishes. Make sure the client keeps
-   * retrying.
-   */
-  @Test
-  public void testKillLeaderDuringReconf() throws Exception {
-    LOG.info("Start testKillLeaderDuringReconf");
-    // originally 3 peers
-    final MiniRaftCluster cluster = getCluster(3);
-    cluster.start();
-    try {
-      RaftTestUtil.waitForLeader(cluster);
-      final String leaderId = cluster.getLeader().getId();
-      final RaftClient client = cluster.createClient("client", leaderId);
-
-      PeerChanges c1 = cluster.addNewPeers(2, false);
-      PeerChanges c2 = cluster.removePeers(2, false, asList(c1.newPeers));
-
-      LOG.info("Start changing the configuration: {}",
-          asList(c2.allPeersInNewConf));
-      final AtomicReference<Boolean> success = new AtomicReference<>();
-      final AtomicBoolean clientRunning = new AtomicBoolean(true);
-      Thread clientThread = new Thread(() -> {
-        try {
-          boolean r = false;
-          while (clientRunning.get() && !r) {
-            r = client.setConfiguration(c2.allPeersInNewConf).isSuccess();
-          }
-          success.set(r);
-          client.close();
-        } catch (IOException ignored) {
-        }
-      });
-      clientThread.start();
-
-      // the leader cannot generate the (old, new) conf, and it will keep
-      // bootstrapping the 2 new peers since they have not started yet
-      LOG.info(cluster.printServers());
-      Assert.assertFalse(cluster.getLeader().getRaftConf().isTransitional());
-
-      // only the first empty entry got committed
-      final long committedIndex = cluster.getLeader().getState().getLog()
-          .getLastCommittedIndex();
-      Assert.assertTrue("committedIndex is " + committedIndex,
-          committedIndex <= 1);
-
-      LOG.info("kill the current leader");
-      final String oldLeaderId = RaftTestUtil.waitAndKillLeader(cluster, true);
-      LOG.info("start the two new peers: {}", Arrays.asList(c1.newPeers));
-      for (RaftPeer np : c1.newPeers) {
-        cluster.startServer(np.getId());
-      }
-
-      Thread.sleep(3000);
-      // the client should get the NotLeaderException from the first leader, and
-      // will retry the same setConfiguration request
-      waitAndCheckNewConf(cluster, c2.allPeersInNewConf, 2,
-          Collections.singletonList(oldLeaderId));
-      clientRunning.set(false);
-      //Assert.assertTrue(success.get());
-    } finally {
-      cluster.shutdown();
-    }
-  }
-
-  static void assertSuccess(final AtomicReference<Boolean> success) {
-    final String s = "success=" + success;
-    Assert.assertNotNull(s, success.get());
-    Assert.assertTrue(s, success.get());
-  }
-
-  /**
-   * When a request's new configuration is the same with the current one, make
-   * sure we return success immediately and no log entry is recorded.
-   */
-  @Test
-  public void testNoChangeRequest() throws Exception {
-    LOG.info("Start testNoChangeRequest");
-    // originally 3 peers
-    final MiniRaftCluster cluster = getCluster(3);
-    try {
-      cluster.start();
-      RaftTestUtil.waitForLeader(cluster);
-
-      final String leaderId = cluster.getLeader().getId();
-      final RaftClient client = cluster.createClient("client", leaderId);
-      client.send(new SimpleMessage("m"));
-
-      final long committedIndex = cluster.getLeader().getState().getLog()
-          .getLastCommittedIndex();
-      final RaftConfiguration confBefore = cluster.getLeader().getRaftConf();
-
-      // no real configuration change in the request
-      RaftClientReply reply = client.setConfiguration(cluster.getPeers()
-          .toArray(new RaftPeer[0]));
-      Assert.assertTrue(reply.isSuccess());
-      Assert.assertEquals(committedIndex, cluster.getLeader().getState()
-          .getLog().getLastCommittedIndex());
-      Assert.assertSame(confBefore, cluster.getLeader().getRaftConf());
-      client.close();
-    } finally {
-      cluster.shutdown();
-    }
-  }
-
-  /**
-   * Make sure a setConfiguration request is rejected if a configuration change
-   * is still in progress (i.e., has not been committed yet).
-   */
-  @Test
-  public void testOverlappedSetConfRequests() throws Exception {
-    LOG.info("Start testOverlappedSetConfRequests");
-    // originally 3 peers
-    final MiniRaftCluster cluster = getCluster(3);
-    try {
-      cluster.start();
-      RaftTestUtil.waitForLeader(cluster);
-
-      final String leaderId = cluster.getLeader().getId();
-
-      RaftPeer[] newPeers = cluster.addNewPeers(2, true).allPeersInNewConf;
-
-      // delay every peer's logSync so that the setConf request is delayed
-      cluster.getPeers()
-          .forEach(peer -> logSyncDelay.setDelayMs(peer.getId(), 1000));
-
-      final CountDownLatch latch = new CountDownLatch(1);
-      final RaftPeer[] peersInRequest2 = cluster.getPeers().toArray(new RaftPeer[0]);
-      AtomicBoolean caughtException = new AtomicBoolean(false);
-      new Thread(() -> {
-        try(final RaftClient client2 = cluster.createClient("client2", leaderId)) {
-          latch.await();
-          LOG.info("client2 starts to change conf");
-          final RaftClientRequestSender sender2 = ((RaftClientImpl)client2).getRequestSender();
-          sender2.sendRequest(new SetConfigurationRequest(
-              "client2", leaderId, DEFAULT_SEQNUM, peersInRequest2));
-        } catch (ReconfigurationInProgressException e) {
-          caughtException.set(true);
-        } catch (Exception e) {
-          LOG.warn("Got unexpected exception when client2 changes conf", e);
-        }
-      }).start();
-
-      AtomicBoolean confChanged = new AtomicBoolean(false);
-      new Thread(() -> {
-        try(final RaftClient client1 = cluster.createClient("client1", leaderId)) {
-          LOG.info("client1 starts to change conf");
-          confChanged.set(client1.setConfiguration(newPeers).isSuccess());
-        } catch (IOException e) {
-          LOG.warn("Got unexpected exception when client1 changes conf", e);
-        }
-      }).start();
-      Thread.sleep(100);
-      latch.countDown();
-
-      for (int i = 0; i < 10 && !confChanged.get(); i++) {
-        Thread.sleep(1000);
-      }
-      Assert.assertTrue(confChanged.get());
-      Assert.assertTrue(caughtException.get());
-    } finally {
-      logSyncDelay.clear();
-      cluster.shutdown();
-    }
-  }
-
-  /**
-   * Test a scenario where the follower truncates its log entries which causes
-   * configuration change.
-   */
-  @Test
-  public void testRevertConfigurationChange() throws Exception {
-    LOG.info("Start testRevertConfigurationChange");
-    // originally 3 peers
-    final MiniRaftCluster cluster = getCluster(5);
-    try {
-      cluster.start();
-      RaftTestUtil.waitForLeader(cluster);
-
-      final String leaderId = cluster.getLeader().getId();
-
-      final RaftLog log = cluster.getServer(leaderId).getState().getLog();
-      Thread.sleep(1000);
-      Assert.assertEquals(0, log.getLatestFlushedIndex());
-
-      // we block the incoming msg for the leader and block its requests to
-      // followers, so that we force the leader change and the old leader will
-      // not know
-      LOG.info("start blocking the leader");
-      BlockRequestHandlingInjection.getInstance().blockReplier(leaderId);
-      cluster.setBlockRequestsFrom(leaderId, true);
-
-      PeerChanges change = cluster.removePeers(1, false, new ArrayList<>());
-
-      AtomicBoolean gotNotLeader = new AtomicBoolean(false);
-      new Thread(() -> {
-        try(final RaftClient client = cluster.createClient("client1", leaderId)) {
-          LOG.info("client starts to change conf");
-          final RaftClientRequestSender sender = ((RaftClientImpl)client).getRequestSender();
-          RaftClientReply reply = sender.sendRequest(new SetConfigurationRequest(
-              "client", leaderId, DEFAULT_SEQNUM, change.allPeersInNewConf));
-          if (reply.isNotLeader()) {
-            gotNotLeader.set(true);
-          }
-        } catch (IOException e) {
-          LOG.warn("Got unexpected exception when client1 changes conf", e);
-        }
-      }).start();
-
-      // wait till the old leader persist the new conf
-      for (int i = 0; i < 10 && log.getLatestFlushedIndex() < 1; i++) {
-        Thread.sleep(500);
-      }
-      Assert.assertEquals(1, log.getLatestFlushedIndex());
-      Assert.assertEquals(CONFIGURATIONENTRY,
-          log.getLastEntry().getLogEntryBodyCase());
-
-      // unblock the old leader
-      BlockRequestHandlingInjection.getInstance().unblockReplier(leaderId);
-      cluster.setBlockRequestsFrom(leaderId, false);
-
-      // the client should get NotLeaderException
-      for (int i = 0; i < 10 && !gotNotLeader.get(); i++) {
-        Thread.sleep(500);
-      }
-      Assert.assertTrue(gotNotLeader.get());
-
-      // the old leader should have truncated the setConf from the log
-      boolean newState = false;
-      for (int i = 0; i < 10 && !newState; i++) {
-        Thread.sleep(500);
-        newState = log.getLastCommittedIndex() == 1 &&
-            log.getLastEntry().getLogEntryBodyCase() != CONFIGURATIONENTRY;
-      }
-      Assert.assertTrue(newState);
-    } finally {
-      cluster.shutdown();
-    }
-  }
-}

http://git-wip-us.apache.org/repos/asf/incubator-ratis/blob/c36810ed/raft-server/src/test/java/org/apache/raft/server/RaftServerTestUtil.java
----------------------------------------------------------------------
diff --git a/raft-server/src/test/java/org/apache/raft/server/RaftServerTestUtil.java b/raft-server/src/test/java/org/apache/raft/server/RaftServerTestUtil.java
deleted file mode 100644
index d52e3ad..0000000
--- a/raft-server/src/test/java/org/apache/raft/server/RaftServerTestUtil.java
+++ /dev/null
@@ -1,67 +0,0 @@
-/**
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- *     http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-package org.apache.raft.server;
-
-import org.apache.raft.MiniRaftCluster;
-import org.apache.raft.RaftTestUtil;
-import org.apache.raft.protocol.RaftPeer;
-import org.junit.Assert;
-import org.slf4j.Logger;
-import org.slf4j.LoggerFactory;
-
-import java.util.Collection;
-
-public class RaftServerTestUtil {
-  static final Logger LOG = LoggerFactory.getLogger(RaftServerTestUtil.class);
-
-  public static void waitAndCheckNewConf(MiniRaftCluster cluster,
-      RaftPeer[] peers, int numOfRemovedPeers, Collection<String> deadPeers)
-      throws Exception {
-    final long sleepMs = cluster.getMaxTimeout() * (numOfRemovedPeers + 2);
-    RaftTestUtil.attempt(3, sleepMs,
-        () -> waitAndCheckNewConf(cluster, peers, deadPeers));
-  }
-  private static void waitAndCheckNewConf(MiniRaftCluster cluster,
-      RaftPeer[] peers, Collection<String> deadPeers)
-      throws Exception {
-    LOG.info(cluster.printServers());
-    Assert.assertNotNull(cluster.getLeader());
-
-    int numIncluded = 0;
-    int deadIncluded = 0;
-    final RaftConfiguration current = RaftConfiguration.newBuilder()
-        .setConf(peers).setLogEntryIndex(0).build();
-    for (RaftServer server : cluster.getServers()) {
-      if (deadPeers != null && deadPeers.contains(server.getId())) {
-        if (current.containsInConf(server.getId())) {
-          deadIncluded++;
-        }
-        continue;
-      }
-      if (current.containsInConf(server.getId())) {
-        numIncluded++;
-        Assert.assertTrue(server.getRaftConf().isStable());
-        Assert.assertTrue(server.getRaftConf().hasNoChange(peers));
-      } else {
-        Assert.assertFalse(server.getId() + " is still running: " + server,
-            server.isAlive());
-      }
-    }
-    Assert.assertEquals(peers.length, numIncluded + deadIncluded);
-  }
-}

http://git-wip-us.apache.org/repos/asf/incubator-ratis/blob/c36810ed/raft-server/src/test/java/org/apache/raft/server/impl/BlockRequestHandlingInjection.java
----------------------------------------------------------------------
diff --git a/raft-server/src/test/java/org/apache/raft/server/impl/BlockRequestHandlingInjection.java b/raft-server/src/test/java/org/apache/raft/server/impl/BlockRequestHandlingInjection.java
new file mode 100644
index 0000000..0980e93
--- /dev/null
+++ b/raft-server/src/test/java/org/apache/raft/server/impl/BlockRequestHandlingInjection.java
@@ -0,0 +1,84 @@
+/**
+ * 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.raft.server.impl;
+
+import org.apache.raft.RaftTestUtil;
+import org.apache.raft.util.CodeInjectionForTesting;
+
+import java.util.Map;
+import java.util.concurrent.ConcurrentHashMap;
+
+/** Inject code to block a server from handling incoming requests. */
+public class BlockRequestHandlingInjection implements CodeInjectionForTesting.Code {
+  private static final BlockRequestHandlingInjection INSTANCE =
+      new BlockRequestHandlingInjection();
+
+  static {
+    CodeInjectionForTesting.put(RaftServer.REQUEST_VOTE, INSTANCE);
+    CodeInjectionForTesting.put(RaftServer.APPEND_ENTRIES, INSTANCE);
+    CodeInjectionForTesting.put(RaftServer.INSTALL_SNAPSHOT, INSTANCE);
+  }
+
+  public static BlockRequestHandlingInjection getInstance() {
+    return INSTANCE;
+  }
+
+  private final Map<String, Boolean> requestors = new ConcurrentHashMap<>();
+  private final Map<String, Boolean> repliers = new ConcurrentHashMap<>();
+
+  private BlockRequestHandlingInjection() {}
+
+  public void blockRequestor(String requestor) {
+    requestors.put(requestor, true);
+  }
+
+  public void unblockRequestor(String requestor) {
+    requestors.remove(requestor);
+  }
+
+  public void blockReplier(String replier) {
+    repliers.put(replier, true);
+  }
+
+  public void unblockReplier(String replier) {
+    repliers.remove(replier);
+  }
+
+  public void unblockAll() {
+    requestors.clear();
+    repliers.clear();
+  }
+
+  @Override
+  public boolean execute(String localId, String remoteId, Object... args) {
+    if (shouldBlock(localId, remoteId)) {
+      try {
+        RaftTestUtil.block(() -> shouldBlock(localId, remoteId));
+        return true;
+      } catch (InterruptedException e) {
+        LOG.debug("Interrupted while blocking request handling from " + remoteId
+            + " to " + localId);
+      }
+    }
+    return false;
+  }
+
+  private boolean shouldBlock(String localId, String remoteId) {
+    return repliers.containsKey(localId) || requestors.containsKey(remoteId);
+  }
+}

http://git-wip-us.apache.org/repos/asf/incubator-ratis/blob/c36810ed/raft-server/src/test/java/org/apache/raft/server/impl/DelayLocalExecutionInjection.java
----------------------------------------------------------------------
diff --git a/raft-server/src/test/java/org/apache/raft/server/impl/DelayLocalExecutionInjection.java b/raft-server/src/test/java/org/apache/raft/server/impl/DelayLocalExecutionInjection.java
new file mode 100644
index 0000000..26b89d8
--- /dev/null
+++ b/raft-server/src/test/java/org/apache/raft/server/impl/DelayLocalExecutionInjection.java
@@ -0,0 +1,67 @@
+/**
+ * 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.raft.server.impl;
+
+import org.apache.raft.RaftTestUtil;
+import org.apache.raft.util.CodeInjectionForTesting;
+
+import java.util.Arrays;
+import java.util.Map;
+import java.util.concurrent.ConcurrentHashMap;
+import java.util.concurrent.atomic.AtomicInteger;
+
+/** Inject code to delay particular servers. */
+public class DelayLocalExecutionInjection implements CodeInjectionForTesting.Code {
+  private final Map<String, AtomicInteger> delays = new ConcurrentHashMap<>();
+
+  public DelayLocalExecutionInjection(String method) {
+    CodeInjectionForTesting.put(method, this);
+  }
+
+  public void clear() {
+    delays.clear();
+  }
+
+  public void setDelayMs(String id, int delayMs) {
+    AtomicInteger d = delays.get(id);
+    if (d == null) {
+      delays.put(id, d = new AtomicInteger());
+    }
+    d.set(delayMs);
+  }
+
+  public void removeDelay(String id) {
+    delays.remove(id);
+  }
+
+  @Override
+  public boolean execute(String localId, String remoteId, Object... args) {
+    final AtomicInteger d = delays.get(localId);
+    if (d == null) {
+      return false;
+    }
+    LOG.info("{} delay {} ms, args={}", localId, d.get(),
+        Arrays.toString(args));
+    try {
+      RaftTestUtil.delay(d::get);
+    } catch (InterruptedException e) {
+      LOG.debug("Interrupted while delaying " + localId);
+    }
+    return true;
+  }
+}

http://git-wip-us.apache.org/repos/asf/incubator-ratis/blob/c36810ed/raft-server/src/test/java/org/apache/raft/server/impl/RaftReconfigurationBaseTest.java
----------------------------------------------------------------------
diff --git a/raft-server/src/test/java/org/apache/raft/server/impl/RaftReconfigurationBaseTest.java b/raft-server/src/test/java/org/apache/raft/server/impl/RaftReconfigurationBaseTest.java
new file mode 100644
index 0000000..30f1e15
--- /dev/null
+++ b/raft-server/src/test/java/org/apache/raft/server/impl/RaftReconfigurationBaseTest.java
@@ -0,0 +1,577 @@
+/**
+ * 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.raft.server.impl;
+
+import org.apache.log4j.Level;
+import org.apache.raft.MiniRaftCluster;
+import org.apache.raft.MiniRaftCluster.PeerChanges;
+import org.apache.raft.RaftTestUtil;
+import org.apache.raft.RaftTestUtil.SimpleMessage;
+import org.apache.raft.client.RaftClient;
+import org.apache.raft.client.RaftClientRequestSender;
+import org.apache.raft.client.impl.RaftClientImpl;
+import org.apache.raft.conf.RaftProperties;
+import org.apache.raft.protocol.*;
+import org.apache.raft.server.RaftServerConfigKeys;
+import org.apache.raft.server.simulation.RequestHandler;
+import org.apache.raft.server.storage.RaftLog;
+import org.apache.raft.util.RaftUtils;
+import org.junit.Assert;
+import org.junit.BeforeClass;
+import org.junit.Test;
+import org.mockito.internal.util.reflection.Whitebox;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.io.IOException;
+import java.util.ArrayList;
+import java.util.Arrays;
+import java.util.Collections;
+import java.util.concurrent.CountDownLatch;
+import java.util.concurrent.atomic.AtomicBoolean;
+import java.util.concurrent.atomic.AtomicReference;
+
+import static java.util.Arrays.asList;
+import static org.apache.raft.MiniRaftCluster.logSyncDelay;
+import static org.apache.raft.server.impl.RaftServerConstants.DEFAULT_SEQNUM;
+import static org.apache.raft.server.impl.RaftServerTestUtil.waitAndCheckNewConf;
+import static org.apache.raft.shaded.proto.RaftProtos.LogEntryProto.LogEntryBodyCase.CONFIGURATIONENTRY;
+
+public abstract class RaftReconfigurationBaseTest {
+  static {
+    RaftUtils.setLogLevel(RaftServer.LOG, Level.DEBUG);
+    RaftUtils.setLogLevel(RequestHandler.LOG, Level.DEBUG);
+    RaftUtils.setLogLevel(RaftClient.LOG, Level.DEBUG);
+  }
+  static final Logger LOG = LoggerFactory.getLogger(RaftReconfigurationBaseTest.class);
+
+  protected static final RaftProperties prop = new RaftProperties();
+
+  @BeforeClass
+  public static void setup() {
+    // set a small gap for tests
+    prop.setInt(RaftServerConfigKeys.RAFT_SERVER_STAGING_CATCHUP_GAP_KEY, 10);
+  }
+
+  public abstract MiniRaftCluster getCluster(int peerNum) throws IOException;
+
+  private static int getStagingGap() {
+    return prop.getInt(RaftServerConfigKeys.RAFT_SERVER_STAGING_CATCHUP_GAP_KEY,
+        RaftServerConfigKeys.RAFT_SERVER_STAGING_CATCHUP_GAP_DEFAULT);
+  }
+
+  /**
+   * add 2 new peers (3 peers -> 5 peers), no leader change
+   */
+  @Test
+  public void testAddPeers() throws Exception {
+    LOG.info("Start testAddPeers");
+    MiniRaftCluster cluster = getCluster(3);
+    cluster.start();
+    try {
+      RaftTestUtil.waitForLeader(cluster);
+
+      // add new peers
+      RaftPeer[] allPeers = cluster.addNewPeers(2, true).allPeersInNewConf;
+
+      // trigger setConfiguration
+      SetConfigurationRequest request = new SetConfigurationRequest("client",
+          cluster.getLeader().getId(), DEFAULT_SEQNUM, allPeers);
+      LOG.info("Start changing the configuration: {}", request);
+      cluster.getLeader().setConfiguration(request);
+
+      // wait for the new configuration to take effect
+      waitAndCheckNewConf(cluster, allPeers, 0, null);
+    } finally {
+      cluster.shutdown();
+    }
+  }
+
+  /**
+   * remove 2 peers (5 peers -> 3 peers), no leader change
+   */
+  @Test
+  public void testRemovePeers() throws Exception {
+    LOG.info("Start testRemovePeers");
+    MiniRaftCluster cluster = getCluster(5);
+    cluster.start();
+    try {
+      RaftTestUtil.waitForLeader(cluster);
+
+      // remove peers, leader still included in the new conf
+      RaftPeer[] allPeers = cluster
+          .removePeers(2, false, Collections.emptyList()).allPeersInNewConf;
+
+      // trigger setConfiguration
+      SetConfigurationRequest request = new SetConfigurationRequest("client",
+          cluster.getLeader().getId(), DEFAULT_SEQNUM, allPeers);
+      LOG.info("Start changing the configuration: {}", request);
+      cluster.getLeader().setConfiguration(request);
+
+      // wait for the new configuration to take effect
+      waitAndCheckNewConf(cluster, allPeers, 2, null);
+    } finally {
+      cluster.shutdown();
+    }
+  }
+
+  /**
+   * 5 peers -> 5 peers, remove 2 old, add 2 new, no leader change
+   */
+  @Test
+  public void testAddRemovePeers() throws Exception {
+    LOG.info("Start testAddRemovePeers");
+    testAddRemovePeers(false);
+  }
+
+  @Test
+  public void testLeaderStepDown() throws Exception {
+    LOG.info("Start testLeaderStepDown");
+    testAddRemovePeers(true);
+  }
+
+  private void testAddRemovePeers(boolean leaderStepdown) throws Exception {
+    MiniRaftCluster cluster = getCluster(5);
+    cluster.start();
+    try {
+      RaftTestUtil.waitForLeader(cluster);
+
+      PeerChanges change = cluster.addNewPeers(2, true);
+      RaftPeer[] allPeers = cluster.removePeers(2, leaderStepdown,
+          asList(change.newPeers)).allPeersInNewConf;
+
+      // trigger setConfiguration
+      SetConfigurationRequest request = new SetConfigurationRequest("client",
+          cluster.getLeader().getId(), DEFAULT_SEQNUM, allPeers);
+      LOG.info("Start changing the configuration: {}", request);
+      cluster.getLeader().setConfiguration(request);
+
+      // wait for the new configuration to take effect
+      waitAndCheckNewConf(cluster, allPeers, 2, null);
+    } finally {
+      cluster.shutdown();
+    }
+  }
+
+  @Test(timeout = 30000)
+  public void testReconfTwice() throws Exception {
+    LOG.info("Start testReconfTwice");
+    final MiniRaftCluster cluster = getCluster(3);
+    cluster.start();
+    try {
+      RaftTestUtil.waitForLeader(cluster);
+      final String leaderId = cluster.getLeader().getId();
+      final RaftClient client = cluster.createClient("client", leaderId);
+
+      // submit some msgs before reconf
+      for (int i = 0; i < getStagingGap() * 2; i++) {
+        RaftClientReply reply = client.send(new SimpleMessage("m" + i));
+        Assert.assertTrue(reply.isSuccess());
+      }
+
+      final AtomicBoolean reconf1 = new AtomicBoolean(false);
+      final AtomicBoolean reconf2 = new AtomicBoolean(false);
+      final AtomicReference<RaftPeer[]> finalPeers = new AtomicReference<>(null);
+      final AtomicReference<RaftPeer[]> deadPeers = new AtomicReference<>(null);
+      CountDownLatch latch = new CountDownLatch(1);
+      Thread clientThread = new Thread(() -> {
+        try {
+          PeerChanges c1 = cluster.addNewPeers(2, true);
+          LOG.info("Start changing the configuration: {}",
+              asList(c1.allPeersInNewConf));
+
+          RaftClientReply reply = client.setConfiguration(c1.allPeersInNewConf);
+          reconf1.set(reply.isSuccess());
+
+          PeerChanges c2 = cluster.removePeers(2, true, asList(c1.newPeers));
+          finalPeers.set(c2.allPeersInNewConf);
+          deadPeers.set(c2.removedPeers);
+
+          LOG.info("Start changing the configuration again: {}",
+              asList(c2.allPeersInNewConf));
+          reply = client.setConfiguration(c2.allPeersInNewConf);
+          reconf2.set(reply.isSuccess());
+
+          latch.countDown();
+          client.close();
+        } catch (IOException ignored) {
+        }
+      });
+      clientThread.start();
+
+      latch.await();
+      Assert.assertTrue(reconf1.get());
+      Assert.assertTrue(reconf2.get());
+      waitAndCheckNewConf(cluster, finalPeers.get(), 2, null);
+
+      // check configuration manager's internal state
+      // each reconf will generate two configurations: (old, new) and (new)
+      cluster.getServers().stream().filter(RaftServer::isAlive)
+          .forEach(server -> {
+        ConfigurationManager confManager =
+            (ConfigurationManager) Whitebox.getInternalState(server.getState(),
+                "configurationManager");
+        // each reconf will generate two configurations: (old, new) and (new)
+        Assert.assertEquals(5, confManager.numOfConf());
+      });
+    } finally {
+      cluster.shutdown();
+    }
+  }
+
+  @Test
+  public void testReconfTimeout() throws Exception {
+    LOG.info("Start testReconfTimeout");
+    // originally 3 peers
+    final MiniRaftCluster cluster = getCluster(3);
+    cluster.start();
+    try {
+      RaftTestUtil.waitForLeader(cluster);
+      final String leaderId = cluster.getLeader().getId();
+      final RaftClient client = cluster.createClient("client", leaderId);
+
+      PeerChanges c1 = cluster.addNewPeers(2, false);
+
+      LOG.info("Start changing the configuration: {}",
+          asList(c1.allPeersInNewConf));
+      Assert.assertFalse(cluster.getLeader().getRaftConf().isTransitional());
+
+      final RaftClientRequestSender sender = ((RaftClientImpl)client).getRequestSender();
+      final SetConfigurationRequest request = new SetConfigurationRequest(
+          "client", leaderId, DEFAULT_SEQNUM, c1.allPeersInNewConf);
+      try {
+        sender.sendRequest(request);
+        Assert.fail("did not get expected exception");
+      } catch (IOException e) {
+        Assert.assertTrue("Got exception " + e,
+            e instanceof ReconfigurationTimeoutException);
+      }
+
+      // the two new peers have not started yet, the bootstrapping must timeout
+      LOG.info(cluster.printServers());
+
+      // resend the same request, make sure the server has correctly reset its
+      // state so that we still get timeout instead of in-progress exception
+      try {
+        sender.sendRequest(request);
+        Assert.fail("did not get expected exception");
+      } catch (IOException e) {
+        Assert.assertTrue("Got exception " + e,
+            e instanceof ReconfigurationTimeoutException);
+      }
+
+      // start the two new peers
+      LOG.info("Start new peers");
+      for (RaftPeer np : c1.newPeers) {
+        cluster.startServer(np.getId());
+      }
+      Assert.assertTrue(client.setConfiguration(c1.allPeersInNewConf).isSuccess());
+      client.close();
+    } finally {
+      cluster.shutdown();
+    }
+  }
+
+  @Test
+  public void testBootstrapReconf() throws Exception {
+    LOG.info("Start testBootstrapReconf");
+    // originally 3 peers
+    final MiniRaftCluster cluster = getCluster(3);
+    cluster.start();
+    try {
+      RaftTestUtil.waitForLeader(cluster);
+      final String leaderId = cluster.getLeader().getId();
+      final RaftClient client = cluster.createClient("client", leaderId);
+
+      // submit some msgs before reconf
+      for (int i = 0; i < getStagingGap() * 2; i++) {
+        RaftClientReply reply = client.send(new SimpleMessage("m" + i));
+        Assert.assertTrue(reply.isSuccess());
+      }
+
+      PeerChanges c1 = cluster.addNewPeers(2, true);
+      LOG.info("Start changing the configuration: {}",
+          asList(c1.allPeersInNewConf));
+      final AtomicReference<Boolean> success = new AtomicReference<>();
+
+      Thread clientThread = new Thread(() -> {
+        try {
+          RaftClientReply reply = client.setConfiguration(c1.allPeersInNewConf);
+          success.set(reply.isSuccess());
+          client.close();
+        } catch (IOException ioe) {
+          LOG.error("FAILED", ioe);
+        }
+      });
+      clientThread.start();
+
+      Thread.sleep(5000);
+      LOG.info(cluster.printServers());
+      assertSuccess(success);
+
+      final RaftLog leaderLog = cluster.getLeader().getState().getLog();
+      for (RaftPeer newPeer : c1.newPeers) {
+        Assert.assertArrayEquals(leaderLog.getEntries(0, Long.MAX_VALUE),
+            cluster.getServer(newPeer.getId()).getState().getLog()
+                .getEntries(0, Long.MAX_VALUE));
+      }
+    } finally {
+      cluster.shutdown();
+    }
+  }
+
+  /**
+   * kill the leader before reconfiguration finishes. Make sure the client keeps
+   * retrying.
+   */
+  @Test
+  public void testKillLeaderDuringReconf() throws Exception {
+    LOG.info("Start testKillLeaderDuringReconf");
+    // originally 3 peers
+    final MiniRaftCluster cluster = getCluster(3);
+    cluster.start();
+    try {
+      RaftTestUtil.waitForLeader(cluster);
+      final String leaderId = cluster.getLeader().getId();
+      final RaftClient client = cluster.createClient("client", leaderId);
+
+      PeerChanges c1 = cluster.addNewPeers(2, false);
+      PeerChanges c2 = cluster.removePeers(2, false, asList(c1.newPeers));
+
+      LOG.info("Start changing the configuration: {}",
+          asList(c2.allPeersInNewConf));
+      final AtomicReference<Boolean> success = new AtomicReference<>();
+      final AtomicBoolean clientRunning = new AtomicBoolean(true);
+      Thread clientThread = new Thread(() -> {
+        try {
+          boolean r = false;
+          while (clientRunning.get() && !r) {
+            r = client.setConfiguration(c2.allPeersInNewConf).isSuccess();
+          }
+          success.set(r);
+          client.close();
+        } catch (IOException ignored) {
+        }
+      });
+      clientThread.start();
+
+      // the leader cannot generate the (old, new) conf, and it will keep
+      // bootstrapping the 2 new peers since they have not started yet
+      LOG.info(cluster.printServers());
+      Assert.assertFalse(cluster.getLeader().getRaftConf().isTransitional());
+
+      // only the first empty entry got committed
+      final long committedIndex = cluster.getLeader().getState().getLog()
+          .getLastCommittedIndex();
+      Assert.assertTrue("committedIndex is " + committedIndex,
+          committedIndex <= 1);
+
+      LOG.info("kill the current leader");
+      final String oldLeaderId = RaftTestUtil.waitAndKillLeader(cluster, true);
+      LOG.info("start the two new peers: {}", Arrays.asList(c1.newPeers));
+      for (RaftPeer np : c1.newPeers) {
+        cluster.startServer(np.getId());
+      }
+
+      Thread.sleep(3000);
+      // the client should get the NotLeaderException from the first leader, and
+      // will retry the same setConfiguration request
+      waitAndCheckNewConf(cluster, c2.allPeersInNewConf, 2,
+          Collections.singletonList(oldLeaderId));
+      clientRunning.set(false);
+      //Assert.assertTrue(success.get());
+    } finally {
+      cluster.shutdown();
+    }
+  }
+
+  static void assertSuccess(final AtomicReference<Boolean> success) {
+    final String s = "success=" + success;
+    Assert.assertNotNull(s, success.get());
+    Assert.assertTrue(s, success.get());
+  }
+
+  /**
+   * When a request's new configuration is the same with the current one, make
+   * sure we return success immediately and no log entry is recorded.
+   */
+  @Test
+  public void testNoChangeRequest() throws Exception {
+    LOG.info("Start testNoChangeRequest");
+    // originally 3 peers
+    final MiniRaftCluster cluster = getCluster(3);
+    try {
+      cluster.start();
+      RaftTestUtil.waitForLeader(cluster);
+
+      final String leaderId = cluster.getLeader().getId();
+      final RaftClient client = cluster.createClient("client", leaderId);
+      client.send(new SimpleMessage("m"));
+
+      final long committedIndex = cluster.getLeader().getState().getLog()
+          .getLastCommittedIndex();
+      final RaftConfiguration confBefore = cluster.getLeader().getRaftConf();
+
+      // no real configuration change in the request
+      RaftClientReply reply = client.setConfiguration(cluster.getPeers()
+          .toArray(new RaftPeer[0]));
+      Assert.assertTrue(reply.isSuccess());
+      Assert.assertEquals(committedIndex, cluster.getLeader().getState()
+          .getLog().getLastCommittedIndex());
+      Assert.assertSame(confBefore, cluster.getLeader().getRaftConf());
+      client.close();
+    } finally {
+      cluster.shutdown();
+    }
+  }
+
+  /**
+   * Make sure a setConfiguration request is rejected if a configuration change
+   * is still in progress (i.e., has not been committed yet).
+   */
+  @Test
+  public void testOverlappedSetConfRequests() throws Exception {
+    LOG.info("Start testOverlappedSetConfRequests");
+    // originally 3 peers
+    final MiniRaftCluster cluster = getCluster(3);
+    try {
+      cluster.start();
+      RaftTestUtil.waitForLeader(cluster);
+
+      final String leaderId = cluster.getLeader().getId();
+
+      RaftPeer[] newPeers = cluster.addNewPeers(2, true).allPeersInNewConf;
+
+      // delay every peer's logSync so that the setConf request is delayed
+      cluster.getPeers()
+          .forEach(peer -> logSyncDelay.setDelayMs(peer.getId(), 1000));
+
+      final CountDownLatch latch = new CountDownLatch(1);
+      final RaftPeer[] peersInRequest2 = cluster.getPeers().toArray(new RaftPeer[0]);
+      AtomicBoolean caughtException = new AtomicBoolean(false);
+      new Thread(() -> {
+        try(final RaftClient client2 = cluster.createClient("client2", leaderId)) {
+          latch.await();
+          LOG.info("client2 starts to change conf");
+          final RaftClientRequestSender sender2 = ((RaftClientImpl)client2).getRequestSender();
+          sender2.sendRequest(new SetConfigurationRequest(
+              "client2", leaderId, DEFAULT_SEQNUM, peersInRequest2));
+        } catch (ReconfigurationInProgressException e) {
+          caughtException.set(true);
+        } catch (Exception e) {
+          LOG.warn("Got unexpected exception when client2 changes conf", e);
+        }
+      }).start();
+
+      AtomicBoolean confChanged = new AtomicBoolean(false);
+      new Thread(() -> {
+        try(final RaftClient client1 = cluster.createClient("client1", leaderId)) {
+          LOG.info("client1 starts to change conf");
+          confChanged.set(client1.setConfiguration(newPeers).isSuccess());
+        } catch (IOException e) {
+          LOG.warn("Got unexpected exception when client1 changes conf", e);
+        }
+      }).start();
+      Thread.sleep(100);
+      latch.countDown();
+
+      for (int i = 0; i < 10 && !confChanged.get(); i++) {
+        Thread.sleep(1000);
+      }
+      Assert.assertTrue(confChanged.get());
+      Assert.assertTrue(caughtException.get());
+    } finally {
+      logSyncDelay.clear();
+      cluster.shutdown();
+    }
+  }
+
+  /**
+   * Test a scenario where the follower truncates its log entries which causes
+   * configuration change.
+   */
+  @Test
+  public void testRevertConfigurationChange() throws Exception {
+    LOG.info("Start testRevertConfigurationChange");
+    // originally 3 peers
+    final MiniRaftCluster cluster = getCluster(5);
+    try {
+      cluster.start();
+      RaftTestUtil.waitForLeader(cluster);
+
+      final String leaderId = cluster.getLeader().getId();
+
+      final RaftLog log = cluster.getServer(leaderId).getState().getLog();
+      Thread.sleep(1000);
+      Assert.assertEquals(0, log.getLatestFlushedIndex());
+
+      // we block the incoming msg for the leader and block its requests to
+      // followers, so that we force the leader change and the old leader will
+      // not know
+      LOG.info("start blocking the leader");
+      BlockRequestHandlingInjection.getInstance().blockReplier(leaderId);
+      cluster.setBlockRequestsFrom(leaderId, true);
+
+      PeerChanges change = cluster.removePeers(1, false, new ArrayList<>());
+
+      AtomicBoolean gotNotLeader = new AtomicBoolean(false);
+      new Thread(() -> {
+        try(final RaftClient client = cluster.createClient("client1", leaderId)) {
+          LOG.info("client starts to change conf");
+          final RaftClientRequestSender sender = ((RaftClientImpl)client).getRequestSender();
+          RaftClientReply reply = sender.sendRequest(new SetConfigurationRequest(
+              "client", leaderId, DEFAULT_SEQNUM, change.allPeersInNewConf));
+          if (reply.isNotLeader()) {
+            gotNotLeader.set(true);
+          }
+        } catch (IOException e) {
+          LOG.warn("Got unexpected exception when client1 changes conf", e);
+        }
+      }).start();
+
+      // wait till the old leader persist the new conf
+      for (int i = 0; i < 10 && log.getLatestFlushedIndex() < 1; i++) {
+        Thread.sleep(500);
+      }
+      Assert.assertEquals(1, log.getLatestFlushedIndex());
+      Assert.assertEquals(CONFIGURATIONENTRY,
+          log.getLastEntry().getLogEntryBodyCase());
+
+      // unblock the old leader
+      BlockRequestHandlingInjection.getInstance().unblockReplier(leaderId);
+      cluster.setBlockRequestsFrom(leaderId, false);
+
+      // the client should get NotLeaderException
+      for (int i = 0; i < 10 && !gotNotLeader.get(); i++) {
+        Thread.sleep(500);
+      }
+      Assert.assertTrue(gotNotLeader.get());
+
+      // the old leader should have truncated the setConf from the log
+      boolean newState = false;
+      for (int i = 0; i < 10 && !newState; i++) {
+        Thread.sleep(500);
+        newState = log.getLastCommittedIndex() == 1 &&
+            log.getLastEntry().getLogEntryBodyCase() != CONFIGURATIONENTRY;
+      }
+      Assert.assertTrue(newState);
+    } finally {
+      cluster.shutdown();
+    }
+  }
+}

http://git-wip-us.apache.org/repos/asf/incubator-ratis/blob/c36810ed/raft-server/src/test/java/org/apache/raft/server/impl/RaftServerTestUtil.java
----------------------------------------------------------------------
diff --git a/raft-server/src/test/java/org/apache/raft/server/impl/RaftServerTestUtil.java b/raft-server/src/test/java/org/apache/raft/server/impl/RaftServerTestUtil.java
new file mode 100644
index 0000000..b30ddc9
--- /dev/null
+++ b/raft-server/src/test/java/org/apache/raft/server/impl/RaftServerTestUtil.java
@@ -0,0 +1,67 @@
+/**
+ * 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.raft.server.impl;
+
+import org.apache.raft.MiniRaftCluster;
+import org.apache.raft.RaftTestUtil;
+import org.apache.raft.protocol.RaftPeer;
+import org.junit.Assert;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.util.Collection;
+
+public class RaftServerTestUtil {
+  static final Logger LOG = LoggerFactory.getLogger(RaftServerTestUtil.class);
+
+  public static void waitAndCheckNewConf(MiniRaftCluster cluster,
+      RaftPeer[] peers, int numOfRemovedPeers, Collection<String> deadPeers)
+      throws Exception {
+    final long sleepMs = cluster.getMaxTimeout() * (numOfRemovedPeers + 2);
+    RaftTestUtil.attempt(3, sleepMs,
+        () -> waitAndCheckNewConf(cluster, peers, deadPeers));
+  }
+  private static void waitAndCheckNewConf(MiniRaftCluster cluster,
+      RaftPeer[] peers, Collection<String> deadPeers)
+      throws Exception {
+    LOG.info(cluster.printServers());
+    Assert.assertNotNull(cluster.getLeader());
+
+    int numIncluded = 0;
+    int deadIncluded = 0;
+    final RaftConfiguration current = RaftConfiguration.newBuilder()
+        .setConf(peers).setLogEntryIndex(0).build();
+    for (RaftServer server : cluster.getServers()) {
+      if (deadPeers != null && deadPeers.contains(server.getId())) {
+        if (current.containsInConf(server.getId())) {
+          deadIncluded++;
+        }
+        continue;
+      }
+      if (current.containsInConf(server.getId())) {
+        numIncluded++;
+        Assert.assertTrue(server.getRaftConf().isStable());
+        Assert.assertTrue(server.getRaftConf().hasNoChange(peers));
+      } else {
+        Assert.assertFalse(server.getId() + " is still running: " + server,
+            server.isAlive());
+      }
+    }
+    Assert.assertEquals(peers.length, numIncluded + deadIncluded);
+  }
+}

http://git-wip-us.apache.org/repos/asf/incubator-ratis/blob/c36810ed/raft-server/src/test/java/org/apache/raft/server/simulation/MiniRaftClusterWithSimulatedRpc.java
----------------------------------------------------------------------
diff --git a/raft-server/src/test/java/org/apache/raft/server/simulation/MiniRaftClusterWithSimulatedRpc.java b/raft-server/src/test/java/org/apache/raft/server/simulation/MiniRaftClusterWithSimulatedRpc.java
index fd6f6fb..360fe1e 100644
--- a/raft-server/src/test/java/org/apache/raft/server/simulation/MiniRaftClusterWithSimulatedRpc.java
+++ b/raft-server/src/test/java/org/apache/raft/server/simulation/MiniRaftClusterWithSimulatedRpc.java
@@ -21,7 +21,7 @@ import org.apache.raft.MiniRaftCluster;
 import org.apache.raft.client.RaftClientRequestSender;
 import org.apache.raft.conf.RaftProperties;
 import org.apache.raft.protocol.RaftPeer;
-import org.apache.raft.server.RaftServer;
+import org.apache.raft.server.impl.RaftServer;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 

http://git-wip-us.apache.org/repos/asf/incubator-ratis/blob/c36810ed/raft-server/src/test/java/org/apache/raft/server/simulation/SimulatedServerRpc.java
----------------------------------------------------------------------
diff --git a/raft-server/src/test/java/org/apache/raft/server/simulation/SimulatedServerRpc.java b/raft-server/src/test/java/org/apache/raft/server/simulation/SimulatedServerRpc.java
index bbb90f3..ed522d4 100644
--- a/raft-server/src/test/java/org/apache/raft/server/simulation/SimulatedServerRpc.java
+++ b/raft-server/src/test/java/org/apache/raft/server/simulation/SimulatedServerRpc.java
@@ -22,9 +22,9 @@ import org.apache.raft.protocol.RaftClientReply;
 import org.apache.raft.protocol.RaftClientRequest;
 import org.apache.raft.protocol.RaftPeer;
 import org.apache.raft.protocol.SetConfigurationRequest;
-import org.apache.raft.server.RaftServer;
-import org.apache.raft.server.RaftServerRpc;
-import org.apache.raft.server.RequestDispatcher;
+import org.apache.raft.server.impl.RaftServer;
+import org.apache.raft.server.impl.RaftServerRpc;
+import org.apache.raft.server.impl.RequestDispatcher;
 import org.apache.raft.shaded.proto.RaftProtos.*;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;

http://git-wip-us.apache.org/repos/asf/incubator-ratis/blob/c36810ed/raft-server/src/test/java/org/apache/raft/server/simulation/TestRaftReconfigurationWithSimulatedRpc.java
----------------------------------------------------------------------
diff --git a/raft-server/src/test/java/org/apache/raft/server/simulation/TestRaftReconfigurationWithSimulatedRpc.java b/raft-server/src/test/java/org/apache/raft/server/simulation/TestRaftReconfigurationWithSimulatedRpc.java
index 2fd71c7..b0eb456 100644
--- a/raft-server/src/test/java/org/apache/raft/server/simulation/TestRaftReconfigurationWithSimulatedRpc.java
+++ b/raft-server/src/test/java/org/apache/raft/server/simulation/TestRaftReconfigurationWithSimulatedRpc.java
@@ -18,7 +18,7 @@
 package org.apache.raft.server.simulation;
 
 import org.apache.raft.MiniRaftCluster;
-import org.apache.raft.server.RaftReconfigurationBaseTest;
+import org.apache.raft.server.impl.RaftReconfigurationBaseTest;
 
 import java.io.IOException;
 

http://git-wip-us.apache.org/repos/asf/incubator-ratis/blob/c36810ed/raft-server/src/test/java/org/apache/raft/server/simulation/TestRaftWithSimulatedRpc.java
----------------------------------------------------------------------
diff --git a/raft-server/src/test/java/org/apache/raft/server/simulation/TestRaftWithSimulatedRpc.java b/raft-server/src/test/java/org/apache/raft/server/simulation/TestRaftWithSimulatedRpc.java
index c772f59..669226a 100644
--- a/raft-server/src/test/java/org/apache/raft/server/simulation/TestRaftWithSimulatedRpc.java
+++ b/raft-server/src/test/java/org/apache/raft/server/simulation/TestRaftWithSimulatedRpc.java
@@ -21,7 +21,7 @@ import org.apache.log4j.Level;
 import org.apache.raft.RaftBasicTests;
 import org.apache.raft.client.RaftClient;
 import org.apache.raft.conf.RaftProperties;
-import org.apache.raft.server.RaftServer;
+import org.apache.raft.server.impl.RaftServer;
 import org.apache.raft.util.RaftUtils;
 
 import java.io.IOException;

http://git-wip-us.apache.org/repos/asf/incubator-ratis/blob/c36810ed/raft-server/src/test/java/org/apache/raft/server/storage/TestRaftLogReadWrite.java
----------------------------------------------------------------------
diff --git a/raft-server/src/test/java/org/apache/raft/server/storage/TestRaftLogReadWrite.java b/raft-server/src/test/java/org/apache/raft/server/storage/TestRaftLogReadWrite.java
index 9fe15bc..fa17696 100644
--- a/raft-server/src/test/java/org/apache/raft/server/storage/TestRaftLogReadWrite.java
+++ b/raft-server/src/test/java/org/apache/raft/server/storage/TestRaftLogReadWrite.java
@@ -21,8 +21,8 @@ import org.apache.raft.RaftTestUtil;
 import org.apache.raft.RaftTestUtil.SimpleOperation;
 import org.apache.raft.conf.RaftProperties;
 import org.apache.raft.protocol.ChecksumException;
-import org.apache.raft.server.RaftServerConstants;
-import org.apache.raft.server.RaftServerConstants.StartupOption;
+import org.apache.raft.server.impl.RaftServerConstants;
+import org.apache.raft.server.impl.RaftServerConstants.StartupOption;
 import org.apache.raft.shaded.com.google.protobuf.CodedOutputStream;
 import org.apache.raft.shaded.proto.RaftProtos.LogEntryProto;
 import org.apache.raft.util.FileUtils;

http://git-wip-us.apache.org/repos/asf/incubator-ratis/blob/c36810ed/raft-server/src/test/java/org/apache/raft/server/storage/TestRaftLogSegment.java
----------------------------------------------------------------------
diff --git a/raft-server/src/test/java/org/apache/raft/server/storage/TestRaftLogSegment.java b/raft-server/src/test/java/org/apache/raft/server/storage/TestRaftLogSegment.java
index c0814e9..470f80f 100644
--- a/raft-server/src/test/java/org/apache/raft/server/storage/TestRaftLogSegment.java
+++ b/raft-server/src/test/java/org/apache/raft/server/storage/TestRaftLogSegment.java
@@ -21,7 +21,7 @@ import org.apache.raft.RaftTestUtil;
 import org.apache.raft.RaftTestUtil.SimpleOperation;
 import org.apache.raft.conf.RaftProperties;
 import org.apache.raft.server.RaftServerConfigKeys;
-import org.apache.raft.server.RaftServerConstants.StartupOption;
+import org.apache.raft.server.impl.RaftServerConstants.StartupOption;
 import org.apache.raft.shaded.proto.RaftProtos.LogEntryProto;
 import org.apache.raft.shaded.proto.RaftProtos.SMLogEntryProto;
 import org.apache.raft.util.FileUtils;
@@ -37,10 +37,8 @@ import java.util.ArrayList;
 import java.util.Arrays;
 import java.util.List;
 
-import static org.apache.raft.server.RaftServerConfigKeys.RAFT_LOG_SEGMENT_MAX_SIZE_KEY;
-import static org.apache.raft.server.RaftServerConfigKeys.RAFT_LOG_SEGMENT_PREALLOCATED_SIZE_KEY;
-import static org.apache.raft.server.RaftServerConfigKeys.RAFT_LOG_WRITE_BUFFER_SIZE_KEY;
-import static org.apache.raft.server.RaftServerConstants.INVALID_LOG_INDEX;
+import static org.apache.raft.server.RaftServerConfigKeys.*;
+import static org.apache.raft.server.impl.RaftServerConstants.INVALID_LOG_INDEX;
 import static org.apache.raft.server.storage.LogSegment.getEntrySize;
 
 /**

http://git-wip-us.apache.org/repos/asf/incubator-ratis/blob/c36810ed/raft-server/src/test/java/org/apache/raft/server/storage/TestRaftStorage.java
----------------------------------------------------------------------
diff --git a/raft-server/src/test/java/org/apache/raft/server/storage/TestRaftStorage.java b/raft-server/src/test/java/org/apache/raft/server/storage/TestRaftStorage.java
index ee5f481..1b14199 100644
--- a/raft-server/src/test/java/org/apache/raft/server/storage/TestRaftStorage.java
+++ b/raft-server/src/test/java/org/apache/raft/server/storage/TestRaftStorage.java
@@ -20,8 +20,8 @@ package org.apache.raft.server.storage;
 import org.apache.raft.RaftTestUtil;
 import org.apache.raft.conf.RaftProperties;
 import org.apache.raft.io.nativeio.NativeIO;
-import org.apache.raft.server.RaftServerConstants.StartupOption;
 import org.apache.raft.server.RaftServerConfigKeys;
+import org.apache.raft.server.impl.RaftServerConstants.StartupOption;
 import org.apache.raft.server.protocol.TermIndex;
 import org.apache.raft.server.storage.RaftStorageDirectory.StorageState;
 import org.apache.raft.statemachine.SimpleStateMachineStorage;

http://git-wip-us.apache.org/repos/asf/incubator-ratis/blob/c36810ed/raft-server/src/test/java/org/apache/raft/server/storage/TestSegmentedRaftLog.java
----------------------------------------------------------------------
diff --git a/raft-server/src/test/java/org/apache/raft/server/storage/TestSegmentedRaftLog.java b/raft-server/src/test/java/org/apache/raft/server/storage/TestSegmentedRaftLog.java
index 0f36a72..264ba8e 100644
--- a/raft-server/src/test/java/org/apache/raft/server/storage/TestSegmentedRaftLog.java
+++ b/raft-server/src/test/java/org/apache/raft/server/storage/TestSegmentedRaftLog.java
@@ -22,9 +22,9 @@ import org.apache.raft.MiniRaftCluster;
 import org.apache.raft.RaftTestUtil;
 import org.apache.raft.RaftTestUtil.SimpleOperation;
 import org.apache.raft.conf.RaftProperties;
-import org.apache.raft.server.ConfigurationManager;
 import org.apache.raft.server.RaftServerConfigKeys;
-import org.apache.raft.server.RaftServerConstants;
+import org.apache.raft.server.impl.ConfigurationManager;
+import org.apache.raft.server.impl.RaftServerConstants;
 import org.apache.raft.shaded.proto.RaftProtos.LogEntryProto;
 import org.apache.raft.util.FileUtils;
 import org.apache.raft.util.ProtoUtils;

http://git-wip-us.apache.org/repos/asf/incubator-ratis/blob/c36810ed/raft-server/src/test/java/org/apache/raft/statemachine/RaftSnapshotBaseTest.java
----------------------------------------------------------------------
diff --git a/raft-server/src/test/java/org/apache/raft/statemachine/RaftSnapshotBaseTest.java b/raft-server/src/test/java/org/apache/raft/statemachine/RaftSnapshotBaseTest.java
index 4e7dce9..41ae9af 100644
--- a/raft-server/src/test/java/org/apache/raft/statemachine/RaftSnapshotBaseTest.java
+++ b/raft-server/src/test/java/org/apache/raft/statemachine/RaftSnapshotBaseTest.java
@@ -25,8 +25,8 @@ import org.apache.raft.client.RaftClient;
 import org.apache.raft.conf.RaftProperties;
 import org.apache.raft.protocol.RaftClientReply;
 import org.apache.raft.protocol.SetConfigurationRequest;
-import org.apache.raft.server.RaftServer;
-import org.apache.raft.server.RaftServerTestUtil;
+import org.apache.raft.server.impl.RaftServer;
+import org.apache.raft.server.impl.RaftServerTestUtil;
 import org.apache.raft.server.simulation.RequestHandler;
 import org.apache.raft.server.storage.RaftLog;
 import org.apache.raft.server.storage.RaftStorageDirectory;
@@ -48,7 +48,7 @@ import java.util.List;
 
 import static org.apache.raft.server.RaftServerConfigKeys.RAFT_SERVER_AUTO_SNAPSHOT_ENABLED_KEY;
 import static org.apache.raft.server.RaftServerConfigKeys.RAFT_SERVER_SNAPSHOT_TRIGGER_THRESHOLD_KEY;
-import static org.apache.raft.server.RaftServerConstants.DEFAULT_SEQNUM;
+import static org.apache.raft.server.impl.RaftServerConstants.DEFAULT_SEQNUM;
 
 public abstract class RaftSnapshotBaseTest {
   static {

http://git-wip-us.apache.org/repos/asf/incubator-ratis/blob/c36810ed/raft-server/src/test/java/org/apache/raft/statemachine/SimpleStateMachine4Testing.java
----------------------------------------------------------------------
diff --git a/raft-server/src/test/java/org/apache/raft/statemachine/SimpleStateMachine4Testing.java b/raft-server/src/test/java/org/apache/raft/statemachine/SimpleStateMachine4Testing.java
index d745522..227ea58 100644
--- a/raft-server/src/test/java/org/apache/raft/statemachine/SimpleStateMachine4Testing.java
+++ b/raft-server/src/test/java/org/apache/raft/statemachine/SimpleStateMachine4Testing.java
@@ -25,7 +25,7 @@ import org.apache.raft.io.MD5Hash;
 import org.apache.raft.protocol.Message;
 import org.apache.raft.protocol.RaftClientReply;
 import org.apache.raft.protocol.RaftClientRequest;
-import org.apache.raft.server.RaftServerConstants;
+import org.apache.raft.server.impl.RaftServerConstants;
 import org.apache.raft.server.protocol.TermIndex;
 import org.apache.raft.server.storage.LogInputStream;
 import org.apache.raft.server.storage.LogOutputStream;

http://git-wip-us.apache.org/repos/asf/incubator-ratis/blob/c36810ed/raft-server/src/test/java/org/apache/raft/statemachine/TestStateMachine.java
----------------------------------------------------------------------
diff --git a/raft-server/src/test/java/org/apache/raft/statemachine/TestStateMachine.java b/raft-server/src/test/java/org/apache/raft/statemachine/TestStateMachine.java
index ad606bf..c9dd99c 100644
--- a/raft-server/src/test/java/org/apache/raft/statemachine/TestStateMachine.java
+++ b/raft-server/src/test/java/org/apache/raft/statemachine/TestStateMachine.java
@@ -24,8 +24,8 @@ import org.apache.raft.client.RaftClient;
 import org.apache.raft.conf.RaftProperties;
 import org.apache.raft.protocol.Message;
 import org.apache.raft.protocol.RaftClientRequest;
-import org.apache.raft.server.RaftServer;
 import org.apache.raft.server.RaftServerConfigKeys;
+import org.apache.raft.server.impl.RaftServer;
 import org.apache.raft.server.simulation.MiniRaftClusterWithSimulatedRpc;
 import org.apache.raft.shaded.proto.RaftProtos.SMLogEntryProto;
 import org.apache.raft.util.RaftUtils;


[24/54] [abbrv] incubator-ratis git commit: Renamed the packages from raft to ratis in preperation for Apache Incubation - Moved all java packages from org.apache.raft to org.apache.ratis. - Moved native package to org_apache_ratis, and native lib to l

Posted by ji...@apache.org.
http://git-wip-us.apache.org/repos/asf/incubator-ratis/blob/7e71a2e0/ratis-grpc/src/main/java/org/apache/ratis/grpc/server/GRpcLogAppender.java
----------------------------------------------------------------------
diff --git a/ratis-grpc/src/main/java/org/apache/ratis/grpc/server/GRpcLogAppender.java b/ratis-grpc/src/main/java/org/apache/ratis/grpc/server/GRpcLogAppender.java
new file mode 100644
index 0000000..5f1d901
--- /dev/null
+++ b/ratis-grpc/src/main/java/org/apache/ratis/grpc/server/GRpcLogAppender.java
@@ -0,0 +1,416 @@
+/**
+ * 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.ratis.grpc.server;
+
+import com.google.common.base.Preconditions;
+
+import org.apache.ratis.shaded.io.grpc.Status;
+import org.apache.ratis.shaded.io.grpc.stub.StreamObserver;
+import org.apache.ratis.shaded.proto.RaftProtos.AppendEntriesReplyProto;
+import org.apache.ratis.shaded.proto.RaftProtos.AppendEntriesRequestProto;
+import org.apache.ratis.shaded.proto.RaftProtos.InstallSnapshotReplyProto;
+import org.apache.ratis.shaded.proto.RaftProtos.InstallSnapshotRequestProto;
+import org.apache.ratis.grpc.RaftGRpcService;
+import org.apache.ratis.grpc.RaftGrpcConfigKeys;
+import org.apache.ratis.server.impl.FollowerInfo;
+import org.apache.ratis.server.impl.LeaderState;
+import org.apache.ratis.server.impl.LogAppender;
+import org.apache.ratis.server.impl.RaftServerImpl;
+import org.apache.ratis.statemachine.SnapshotInfo;
+import org.apache.ratis.util.CodeInjectionForTesting;
+
+import static org.apache.ratis.grpc.RaftGRpcService.GRPC_SEND_SERVER_REQUEST;
+
+import java.util.LinkedList;
+import java.util.Queue;
+import java.util.UUID;
+import java.util.concurrent.ConcurrentLinkedQueue;
+import java.util.concurrent.atomic.AtomicBoolean;
+
+/**
+ * A new log appender implementation using grpc bi-directional stream API.
+ */
+public class GRpcLogAppender extends LogAppender {
+  private final RaftServerProtocolClient client;
+  private final Queue<AppendEntriesRequestProto> pendingRequests;
+  private final int maxPendingRequestsNum;
+  private volatile boolean firstResponseReceived = false;
+
+  private final AppendLogResponseHandler appendResponseHandler;
+  private final InstallSnapshotResponseHandler snapshotResponseHandler;
+
+  private volatile StreamObserver<AppendEntriesRequestProto> appendLogRequestObserver;
+  private StreamObserver<InstallSnapshotRequestProto> snapshotRequestObserver;
+
+  public GRpcLogAppender(RaftServerImpl server, LeaderState leaderState,
+                         FollowerInfo f) {
+    super(server, leaderState, f);
+
+    RaftGRpcService rpcService = (RaftGRpcService) server.getServerRpc();
+    client = rpcService.getRpcClient(f.getPeer());
+    maxPendingRequestsNum = server.getProperties().getInt(
+        RaftGrpcConfigKeys.RAFT_GRPC_LEADER_MAX_OUTSTANDING_APPENDS_KEY,
+        RaftGrpcConfigKeys.RAFT_GRPC_LEADER_MAX_OUTSTANDING_APPENDS_DEFAULT);
+    pendingRequests = new ConcurrentLinkedQueue<>();
+
+    appendResponseHandler = new AppendLogResponseHandler();
+    snapshotResponseHandler = new InstallSnapshotResponseHandler();
+  }
+
+  @Override
+  public void run() {
+    while (isAppenderRunning()) {
+      if (shouldSendRequest()) {
+        SnapshotInfo snapshot = shouldInstallSnapshot();
+        if (snapshot != null) {
+          installSnapshot(snapshot, snapshotResponseHandler);
+        } else {
+          // keep appending log entries or sending heartbeats
+          appendLog();
+        }
+      }
+
+      if (isAppenderRunning() && !shouldSendRequest()) {
+        // use lastSend time instead of lastResponse time
+        final long waitTime = getHeartbeatRemainingTime(
+            follower.getLastRpcTime());
+        if (waitTime > 0) {
+          synchronized (this) {
+            try {
+              LOG.debug("{} decides to wait {}ms before appending to {}",
+                  server.getId(), waitTime, follower.getPeer());
+              wait(waitTime);
+            } catch (InterruptedException ignored) {
+            }
+          }
+        }
+      }
+    }
+    appendLogRequestObserver.onCompleted();
+  }
+
+  private boolean shouldWait() {
+    return pendingRequests.size() >= maxPendingRequestsNum ||
+        shouldWaitForFirstResponse();
+  }
+
+  private void appendLog() {
+    if (appendLogRequestObserver == null) {
+      appendLogRequestObserver = client.appendEntries(appendResponseHandler);
+    }
+    AppendEntriesRequestProto pending = null;
+    final StreamObserver<AppendEntriesRequestProto> s;
+    synchronized (this) {
+      // if the queue's size >= maxSize, wait
+      while (isAppenderRunning() && shouldWait()) {
+        try {
+          LOG.debug("{} wait to send the next AppendEntries to {}",
+              server.getId(), follower.getPeer());
+          this.wait();
+        } catch (InterruptedException ignored) {
+        }
+      }
+
+      if (isAppenderRunning()) {
+        // prepare and enqueue the append request. note changes on follower's
+        // nextIndex and ops on pendingRequests should always be associated
+        // together and protected by the lock
+        pending = createRequest();
+        if (pending != null) {
+          Preconditions.checkState(pendingRequests.offer(pending));
+          updateNextIndex(pending);
+        }
+      }
+      s = appendLogRequestObserver;
+    }
+
+    if (pending != null && isAppenderRunning()) {
+      sendRequest(pending, s);
+    }
+  }
+
+  private void sendRequest(AppendEntriesRequestProto request,
+      StreamObserver<AppendEntriesRequestProto> s) {
+    CodeInjectionForTesting.execute(GRPC_SEND_SERVER_REQUEST, server.getId(),
+        null, request);
+
+    s.onNext(request);
+    follower.updateLastRpcSendTime();
+  }
+
+  private void updateNextIndex(AppendEntriesRequestProto request) {
+    final int count = request.getEntriesCount();
+    if (count > 0) {
+      follower.updateNextIndex(request.getEntries(count - 1).getIndex() + 1);
+    }
+  }
+
+  /**
+   * if this is the first append, wait for the response of the first append so
+   * that we can get the correct next index.
+   */
+  private boolean shouldWaitForFirstResponse() {
+    return pendingRequests.size() > 0 && !firstResponseReceived;
+  }
+
+  /**
+   * StreamObserver for handling responses from the follower
+   */
+  private class AppendLogResponseHandler
+      implements StreamObserver<AppendEntriesReplyProto> {
+    /**
+     * After receiving a appendEntries reply, do the following:
+     * 1. If the reply is success, update the follower's match index and submit
+     *    an event to leaderState
+     * 2. If the reply is NOT_LEADER, step down
+     * 3. If the reply is INCONSISTENCY, decrease the follower's next index
+     *    based on the response
+     */
+    @Override
+    public void onNext(AppendEntriesReplyProto reply) {
+      LOG.debug("{} received {} response from {}", server.getId(),
+          (!firstResponseReceived ? "the first" : "a"),
+          follower.getPeer());
+
+      // update the last rpc time
+      follower.updateLastRpcResponseTime();
+
+      if (!firstResponseReceived) {
+        firstResponseReceived = true;
+      }
+      switch (reply.getResult()) {
+        case SUCCESS:
+          onSuccess(reply);
+          break;
+        case NOT_LEADER:
+          onNotLeader(reply);
+          break;
+        case INCONSISTENCY:
+          onInconsistency(reply);
+          break;
+        default:
+          break;
+      }
+      notifyAppend();
+    }
+
+    /**
+     * for now we simply retry the first pending request
+     */
+    @Override
+    public void onError(Throwable t) {
+      if (!isAppenderRunning()) {
+        LOG.info("{} is stopped", GRpcLogAppender.this);
+        return;
+      }
+      LOG.warn("{} got error when appending entries to {}, exception: {}.",
+          server.getId(), follower.getPeer().getId(), t);
+
+      synchronized (this) {
+        final Status cause = Status.fromThrowable(t);
+        if (cause != null && cause.getCode() == Status.Code.INTERNAL) {
+          // TODO check other Status. Add sleep to avoid tight loop
+          LOG.debug("{} restarts Append call to {} due to error {}",
+              server.getId(), follower.getPeer(), t);
+          // recreate the StreamObserver
+          appendLogRequestObserver = client.appendEntries(appendResponseHandler);
+          // reset firstResponseReceived to false
+          firstResponseReceived = false;
+        }
+
+        // clear the pending requests queue and reset the next index of follower
+        AppendEntriesRequestProto request = pendingRequests.peek();
+        if (request != null) {
+          final long nextIndex = request.hasPreviousLog() ?
+              request.getPreviousLog().getIndex() + 1 : raftLog.getStartIndex();
+          clearPendingRequests(nextIndex);
+        }
+      }
+    }
+
+    @Override
+    public void onCompleted() {
+      LOG.info("{} stops appending log entries to follower {}", server.getId(),
+          follower);
+    }
+  }
+
+  private void clearPendingRequests(long newNextIndex) {
+    pendingRequests.clear();
+    follower.decreaseNextIndex(newNextIndex);
+  }
+
+  private void onSuccess(AppendEntriesReplyProto reply) {
+    AppendEntriesRequestProto request = pendingRequests.poll();
+    final long replyNextIndex = reply.getNextIndex();
+    Preconditions.checkNotNull(request,
+        "Got reply with next index %s but the pending queue is empty",
+        replyNextIndex);
+
+    if (request.getEntriesCount() == 0) {
+      Preconditions.checkState(!request.hasPreviousLog() ||
+              replyNextIndex - 1 == request.getPreviousLog().getIndex(),
+          "reply's next index is %s, request's previous is %s",
+          replyNextIndex, request.getPreviousLog());
+    } else {
+      // check if the reply and the pending request is consistent
+      final long lastEntryIndex = request
+          .getEntries(request.getEntriesCount() - 1).getIndex();
+      Preconditions.checkState(replyNextIndex == lastEntryIndex + 1,
+          "reply's next index is %s, request's last entry index is %s",
+          replyNextIndex, lastEntryIndex);
+      follower.updateMatchIndex(lastEntryIndex);
+      submitEventOnSuccessAppend();
+    }
+  }
+
+  private void onNotLeader(AppendEntriesReplyProto reply) {
+    checkResponseTerm(reply.getTerm());
+    // the running loop will end and the connection will onComplete
+  }
+
+  private synchronized void onInconsistency(AppendEntriesReplyProto reply) {
+    AppendEntriesRequestProto request = pendingRequests.peek();
+    Preconditions.checkState(request.hasPreviousLog());
+    if (request.getPreviousLog().getIndex() >= reply.getNextIndex()) {
+      clearPendingRequests(reply.getNextIndex());
+    }
+  }
+
+  private class InstallSnapshotResponseHandler
+      implements StreamObserver<InstallSnapshotReplyProto> {
+    private final Queue<Integer> pending;
+    private final AtomicBoolean done = new AtomicBoolean(false);
+
+    InstallSnapshotResponseHandler() {
+      pending = new LinkedList<>();
+    }
+
+    synchronized void addPending(InstallSnapshotRequestProto request) {
+      pending.offer(request.getRequestIndex());
+    }
+
+    synchronized void removePending(InstallSnapshotReplyProto reply) {
+      int index = pending.poll();
+      Preconditions.checkState(index == reply.getRequestIndex());
+    }
+
+    boolean isDone() {
+      return done.get();
+    }
+
+    void close() {
+      done.set(true);
+      GRpcLogAppender.this.notifyAppend();
+    }
+
+    synchronized boolean hasAllResponse() {
+      return pending.isEmpty();
+    }
+
+    @Override
+    public void onNext(InstallSnapshotReplyProto reply) {
+      LOG.debug("{} received {} response from {}", server.getId(),
+          (!firstResponseReceived ? "the first" : "a"),
+          follower.getPeer());
+
+      // update the last rpc time
+      follower.updateLastRpcResponseTime();
+
+      if (!firstResponseReceived) {
+        firstResponseReceived = true;
+      }
+
+      switch (reply.getResult()) {
+        case SUCCESS:
+          removePending(reply);
+          break;
+        case NOT_LEADER:
+          checkResponseTerm(reply.getTerm());
+          break;
+        case UNRECOGNIZED:
+          break;
+      }
+    }
+
+    @Override
+    public void onError(Throwable t) {
+      if (!isAppenderRunning()) {
+        LOG.info("{} is stopped", GRpcLogAppender.this);
+        return;
+      }
+      LOG.info("{} got error when installing snapshot to {}, exception: {}",
+          server.getId(), follower.getPeer(), t);
+      close();
+    }
+
+    @Override
+    public void onCompleted() {
+      LOG.info("{} stops sending snapshots to follower {}", server.getId(),
+          follower);
+      close();
+    }
+  }
+
+  private void installSnapshot(SnapshotInfo snapshot,
+      InstallSnapshotResponseHandler responseHandler) {
+    LOG.info("{}: follower {}'s next index is {}," +
+            " log's start index is {}, need to install snapshot",
+        server.getId(), follower.getPeer(), follower.getNextIndex(),
+        raftLog.getStartIndex());
+
+    snapshotRequestObserver = client.installSnapshot(snapshotResponseHandler);
+    final String requestId = UUID.randomUUID().toString();
+    try {
+      for (InstallSnapshotRequestProto request :
+          new SnapshotRequestIter(snapshot, requestId)) {
+        if (isAppenderRunning()) {
+          snapshotRequestObserver.onNext(request);
+          follower.updateLastRpcSendTime();
+          responseHandler.addPending(request);
+        } else {
+          break;
+        }
+      }
+      snapshotRequestObserver.onCompleted();
+    } catch (Exception e) {
+      LOG.warn("{} failed to install snapshot {}. Exception: {}", this,
+          snapshot.getFiles(), e);
+      snapshotRequestObserver.onError(e);
+      return;
+    } finally {
+      snapshotRequestObserver = null;
+    }
+
+    synchronized (this) {
+      while (isAppenderRunning() && !responseHandler.isDone()) {
+        try {
+          wait();
+        } catch (InterruptedException ignored) {
+        }
+      }
+    }
+
+    if (responseHandler.hasAllResponse()) {
+      follower.updateMatchIndex(snapshot.getTermIndex().getIndex());
+      follower.updateNextIndex(snapshot.getTermIndex().getIndex() + 1);
+      LOG.info("{}: install snapshot-{} successfully on follower {}",
+          server.getId(), snapshot.getTermIndex().getIndex(), follower.getPeer());
+    }
+  }
+}

http://git-wip-us.apache.org/repos/asf/incubator-ratis/blob/7e71a2e0/ratis-grpc/src/main/java/org/apache/ratis/grpc/server/PipelinedLogAppenderFactory.java
----------------------------------------------------------------------
diff --git a/ratis-grpc/src/main/java/org/apache/ratis/grpc/server/PipelinedLogAppenderFactory.java b/ratis-grpc/src/main/java/org/apache/ratis/grpc/server/PipelinedLogAppenderFactory.java
new file mode 100644
index 0000000..d30b391
--- /dev/null
+++ b/ratis-grpc/src/main/java/org/apache/ratis/grpc/server/PipelinedLogAppenderFactory.java
@@ -0,0 +1,32 @@
+/**
+ * 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.ratis.grpc.server;
+
+import org.apache.ratis.server.impl.FollowerInfo;
+import org.apache.ratis.server.impl.LeaderState;
+import org.apache.ratis.server.impl.LogAppender;
+import org.apache.ratis.server.impl.LogAppenderFactory;
+import org.apache.ratis.server.impl.RaftServerImpl;
+
+public class PipelinedLogAppenderFactory implements LogAppenderFactory {
+  @Override
+  public LogAppender getLogAppender(RaftServerImpl server, LeaderState state,
+                                    FollowerInfo f) {
+    return new GRpcLogAppender(server, state, f);
+  }
+}

http://git-wip-us.apache.org/repos/asf/incubator-ratis/blob/7e71a2e0/ratis-grpc/src/main/java/org/apache/ratis/grpc/server/RaftServerProtocolClient.java
----------------------------------------------------------------------
diff --git a/ratis-grpc/src/main/java/org/apache/ratis/grpc/server/RaftServerProtocolClient.java b/ratis-grpc/src/main/java/org/apache/ratis/grpc/server/RaftServerProtocolClient.java
new file mode 100644
index 0000000..53df10d
--- /dev/null
+++ b/ratis-grpc/src/main/java/org/apache/ratis/grpc/server/RaftServerProtocolClient.java
@@ -0,0 +1,63 @@
+/**
+ * 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.ratis.grpc.server;
+
+import org.apache.ratis.shaded.io.grpc.ManagedChannel;
+import org.apache.ratis.shaded.io.grpc.ManagedChannelBuilder;
+import org.apache.ratis.shaded.io.grpc.stub.StreamObserver;
+import org.apache.ratis.shaded.proto.RaftProtos.*;
+import org.apache.ratis.shaded.proto.grpc.RaftServerProtocolServiceGrpc;
+import org.apache.ratis.shaded.proto.grpc.RaftServerProtocolServiceGrpc.RaftServerProtocolServiceBlockingStub;
+import org.apache.ratis.shaded.proto.grpc.RaftServerProtocolServiceGrpc.RaftServerProtocolServiceStub;
+import org.apache.ratis.protocol.RaftPeer;
+
+/**
+ * This is a RaftClient implementation that supports streaming data to the raft
+ * ring. The stream implementation utilizes gRPC.
+ */
+public class RaftServerProtocolClient {
+  private final ManagedChannel channel;
+  private final RaftServerProtocolServiceBlockingStub blockingStub;
+  private final RaftServerProtocolServiceStub asyncStub;
+
+  public RaftServerProtocolClient(RaftPeer target) {
+    channel = ManagedChannelBuilder.forTarget(target.getAddress())
+        .usePlaintext(true).build();
+    blockingStub = RaftServerProtocolServiceGrpc.newBlockingStub(channel);
+    asyncStub = RaftServerProtocolServiceGrpc.newStub(channel);
+  }
+
+  public void shutdown() {
+    channel.shutdownNow();
+  }
+
+  public RequestVoteReplyProto requestVote(RequestVoteRequestProto request) {
+    // the StatusRuntimeException will be handled by the caller
+    return blockingStub.requestVote(request);
+  }
+
+  StreamObserver<AppendEntriesRequestProto> appendEntries(
+      StreamObserver<AppendEntriesReplyProto> responseHandler) {
+    return asyncStub.appendEntries(responseHandler);
+  }
+
+  StreamObserver<InstallSnapshotRequestProto> installSnapshot(
+      StreamObserver<InstallSnapshotReplyProto> responseHandler) {
+    return asyncStub.installSnapshot(responseHandler);
+  }
+}

http://git-wip-us.apache.org/repos/asf/incubator-ratis/blob/7e71a2e0/ratis-grpc/src/main/java/org/apache/ratis/grpc/server/RaftServerProtocolService.java
----------------------------------------------------------------------
diff --git a/ratis-grpc/src/main/java/org/apache/ratis/grpc/server/RaftServerProtocolService.java b/ratis-grpc/src/main/java/org/apache/ratis/grpc/server/RaftServerProtocolService.java
new file mode 100644
index 0000000..08e6a51
--- /dev/null
+++ b/ratis-grpc/src/main/java/org/apache/ratis/grpc/server/RaftServerProtocolService.java
@@ -0,0 +1,116 @@
+/**
+ * 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.ratis.grpc.server;
+
+import org.apache.ratis.grpc.RaftGrpcUtil;
+import org.apache.ratis.server.protocol.RaftServerProtocol;
+import org.apache.ratis.shaded.io.grpc.stub.StreamObserver;
+import org.apache.ratis.shaded.proto.RaftProtos.AppendEntriesReplyProto;
+import org.apache.ratis.shaded.proto.RaftProtos.AppendEntriesRequestProto;
+import org.apache.ratis.shaded.proto.RaftProtos.InstallSnapshotReplyProto;
+import org.apache.ratis.shaded.proto.RaftProtos.InstallSnapshotRequestProto;
+import org.apache.ratis.shaded.proto.RaftProtos.RequestVoteReplyProto;
+import org.apache.ratis.shaded.proto.RaftProtos.RequestVoteRequestProto;
+import org.apache.ratis.shaded.proto.grpc.RaftServerProtocolServiceGrpc.RaftServerProtocolServiceImplBase;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+public class RaftServerProtocolService extends RaftServerProtocolServiceImplBase {
+  public static final Logger LOG = LoggerFactory.getLogger(RaftServerProtocolService.class);
+
+  private final String id;
+  private final RaftServerProtocol server;
+
+  public RaftServerProtocolService(String id, RaftServerProtocol server) {
+    this.id = id;
+    this.server = server;
+  }
+
+  @Override
+  public void requestVote(RequestVoteRequestProto request,
+      StreamObserver<RequestVoteReplyProto> responseObserver) {
+    try {
+      final RequestVoteReplyProto reply = server.requestVote(request);
+      responseObserver.onNext(reply);
+      responseObserver.onCompleted();
+    } catch (Throwable e) {
+      LOG.info("{} got exception when handling requestVote {}: {}",
+          id, request.getServerRequest(), e);
+      responseObserver.onError(RaftGrpcUtil.wrapException(e));
+    }
+  }
+
+  @Override
+  public StreamObserver<AppendEntriesRequestProto> appendEntries(
+      StreamObserver<AppendEntriesReplyProto> responseObserver) {
+    return new StreamObserver<AppendEntriesRequestProto>() {
+      @Override
+      public void onNext(AppendEntriesRequestProto request) {
+        try {
+          final AppendEntriesReplyProto reply = server.appendEntries(request);
+          responseObserver.onNext(reply);
+        } catch (Throwable e) {
+          LOG.info("{} got exception when handling appendEntries {}: {}",
+              id, request.getServerRequest(), e);
+          responseObserver.onError(RaftGrpcUtil.wrapException(e));
+        }
+      }
+
+      @Override
+      public void onError(Throwable t) {
+        // for now we just log a msg
+        LOG.info("{}: appendEntries on error. Exception: {}", id, t);
+      }
+
+      @Override
+      public void onCompleted() {
+        LOG.info("{}: appendEntries completed", id);
+        responseObserver.onCompleted();
+      }
+    };
+  }
+
+  @Override
+  public StreamObserver<InstallSnapshotRequestProto> installSnapshot(
+      StreamObserver<InstallSnapshotReplyProto> responseObserver) {
+    return new StreamObserver<InstallSnapshotRequestProto>() {
+      @Override
+      public void onNext(InstallSnapshotRequestProto request) {
+        try {
+          final InstallSnapshotReplyProto reply = server.installSnapshot(request);
+          responseObserver.onNext(reply);
+        } catch (Throwable e) {
+          LOG.info("{} got exception when handling installSnapshot {}: {}",
+              id, request.getServerRequest(), e);
+          responseObserver.onError(RaftGrpcUtil.wrapException(e));
+        }
+      }
+
+      @Override
+      public void onError(Throwable t) {
+        LOG.info("{}: installSnapshot on error. Exception: {}", id, t);
+      }
+
+      @Override
+      public void onCompleted() {
+        LOG.info("{}: installSnapshot completed", id);
+        responseObserver.onCompleted();
+      }
+    };
+  }
+}

http://git-wip-us.apache.org/repos/asf/incubator-ratis/blob/7e71a2e0/ratis-grpc/src/test/java/org/apache/ratis/grpc/MiniRaftClusterWithGRpc.java
----------------------------------------------------------------------
diff --git a/ratis-grpc/src/test/java/org/apache/ratis/grpc/MiniRaftClusterWithGRpc.java b/ratis-grpc/src/test/java/org/apache/ratis/grpc/MiniRaftClusterWithGRpc.java
new file mode 100644
index 0000000..f5c7b3f
--- /dev/null
+++ b/ratis-grpc/src/test/java/org/apache/ratis/grpc/MiniRaftClusterWithGRpc.java
@@ -0,0 +1,138 @@
+/**
+ * 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.ratis.grpc;
+
+import com.google.common.base.Preconditions;
+
+import org.apache.ratis.MiniRaftCluster;
+import org.apache.ratis.RaftTestUtil;
+import org.apache.ratis.client.RaftClientRequestSender;
+import org.apache.ratis.conf.RaftProperties;
+import org.apache.ratis.grpc.RaftGRpcService;
+import org.apache.ratis.grpc.RaftGrpcConfigKeys;
+import org.apache.ratis.grpc.client.RaftClientSenderWithGrpc;
+import org.apache.ratis.grpc.server.PipelinedLogAppenderFactory;
+import org.apache.ratis.protocol.RaftPeer;
+import org.apache.ratis.server.impl.BlockRequestHandlingInjection;
+import org.apache.ratis.server.impl.DelayLocalExecutionInjection;
+import org.apache.ratis.server.impl.LogAppenderFactory;
+import org.apache.ratis.server.impl.RaftServerImpl;
+import org.apache.ratis.util.NetUtils;
+
+import static org.apache.ratis.server.RaftServerConfigKeys.RAFT_SERVER_LOG_APPENDER_FACTORY_CLASS_KEY;
+
+import java.io.IOException;
+import java.util.ArrayList;
+import java.util.Collection;
+import java.util.HashMap;
+import java.util.Map;
+
+public class MiniRaftClusterWithGRpc extends MiniRaftCluster.RpcBase {
+  public static final Factory<MiniRaftClusterWithGRpc> FACTORY
+      = new Factory<MiniRaftClusterWithGRpc>() {
+    @Override
+    public MiniRaftClusterWithGRpc newCluster(
+        String[] ids, RaftProperties prop, boolean formatted) throws IOException {
+      return new MiniRaftClusterWithGRpc(ids, prop, formatted);
+    }
+  };
+
+  public static final DelayLocalExecutionInjection sendServerRequestInjection =
+      new DelayLocalExecutionInjection(RaftGRpcService.GRPC_SEND_SERVER_REQUEST);
+
+  public MiniRaftClusterWithGRpc(int numServers, RaftProperties properties)
+      throws IOException {
+    this(generateIds(numServers, 0), properties, true);
+  }
+
+  public MiniRaftClusterWithGRpc(String[] ids, RaftProperties properties,
+      boolean formatted) throws IOException {
+    super(ids, getPropForGrpc(properties), formatted);
+    init(initRpcServices(getServers(), properties));
+  }
+
+  private static RaftProperties getPropForGrpc(RaftProperties prop) {
+    RaftProperties newProp = new RaftProperties(prop);
+    newProp.setClass(RAFT_SERVER_LOG_APPENDER_FACTORY_CLASS_KEY,
+        PipelinedLogAppenderFactory.class, LogAppenderFactory.class);
+    return newProp;
+  }
+
+  private static Map<RaftPeer, RaftGRpcService> initRpcServices(
+      Collection<RaftServerImpl> servers, RaftProperties prop) throws IOException {
+    final Map<RaftPeer, RaftGRpcService> peerRpcs = new HashMap<>();
+
+    for (RaftServerImpl s : servers) {
+      final RaftGRpcService rpc = new RaftGRpcService(s, prop);
+      peerRpcs.put(new RaftPeer(s.getId(), rpc.getInetSocketAddress()), rpc);
+    }
+    return peerRpcs;
+  }
+
+  @Override
+  public RaftClientRequestSender getRaftClientRequestSender() {
+    return new RaftClientSenderWithGrpc(getPeers());
+  }
+
+  @Override
+  protected Collection<RaftPeer> addNewPeers(Collection<RaftPeer> newPeers,
+                                             Collection<RaftServerImpl> newServers, boolean startService)
+      throws IOException {
+    final Map<RaftPeer, RaftGRpcService> peers = initRpcServices(newServers, properties);
+    for (Map.Entry<RaftPeer, RaftGRpcService> entry : peers.entrySet()) {
+      RaftServerImpl server = servers.get(entry.getKey().getId());
+      server.setServerRpc(entry.getValue());
+      if (!startService) {
+        BlockRequestHandlingInjection.getInstance().blockReplier(server.getId());
+      } else {
+        server.start();
+      }
+    }
+    return new ArrayList<>(peers.keySet());
+  }
+
+  @Override
+  protected RaftServerImpl setPeerRpc(RaftPeer peer) throws IOException {
+    RaftServerImpl server = servers.get(peer.getId());
+    int port = NetUtils.newInetSocketAddress(peer.getAddress()).getPort();
+    int oldPort = properties.getInt(RaftGrpcConfigKeys.RAFT_GRPC_SERVER_PORT_KEY,
+        RaftGrpcConfigKeys.RAFT_GRPC_SERVER_PORT_DEFAULT);
+    properties.setInt(RaftGrpcConfigKeys.RAFT_GRPC_SERVER_PORT_KEY, port);
+    final RaftGRpcService rpc = new RaftGRpcService(server, properties);
+    Preconditions.checkState(
+        rpc.getInetSocketAddress().toString().contains(peer.getAddress()),
+        "address in the raft conf: %s, address in rpc server: %s",
+        peer.getAddress(), rpc.getInetSocketAddress().toString());
+    server.setServerRpc(rpc);
+    properties.setInt(RaftGrpcConfigKeys.RAFT_GRPC_SERVER_PORT_KEY, oldPort);
+    return server;
+  }
+
+  @Override
+  public void startServer(String id) {
+    super.startServer(id);
+    BlockRequestHandlingInjection.getInstance().unblockReplier(id);
+  }
+
+  @Override
+  protected void blockQueueAndSetDelay(String leaderId, int delayMs)
+      throws InterruptedException {
+    RaftTestUtil.blockQueueAndSetDelay(getServers(), sendServerRequestInjection,
+        leaderId, delayMs, getMaxTimeout());
+  }
+}

http://git-wip-us.apache.org/repos/asf/incubator-ratis/blob/7e71a2e0/ratis-grpc/src/test/java/org/apache/ratis/grpc/TestNotLeaderExceptionWithGrpc.java
----------------------------------------------------------------------
diff --git a/ratis-grpc/src/test/java/org/apache/ratis/grpc/TestNotLeaderExceptionWithGrpc.java b/ratis-grpc/src/test/java/org/apache/ratis/grpc/TestNotLeaderExceptionWithGrpc.java
new file mode 100644
index 0000000..351e406
--- /dev/null
+++ b/ratis-grpc/src/test/java/org/apache/ratis/grpc/TestNotLeaderExceptionWithGrpc.java
@@ -0,0 +1,39 @@
+/**
+ * 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.ratis.grpc;
+
+import org.apache.ratis.MiniRaftCluster;
+import org.apache.ratis.RaftNotLeaderExceptionBaseTest;
+import org.apache.ratis.conf.RaftProperties;
+import org.apache.ratis.grpc.server.PipelinedLogAppenderFactory;
+import org.apache.ratis.server.impl.LogAppenderFactory;
+
+import static org.apache.ratis.server.RaftServerConfigKeys.RAFT_SERVER_LOG_APPENDER_FACTORY_CLASS_KEY;
+
+import java.io.IOException;
+
+public class TestNotLeaderExceptionWithGrpc extends RaftNotLeaderExceptionBaseTest {
+  @Override
+  public MiniRaftCluster initCluster() throws IOException {
+    String[] s = MiniRaftCluster.generateIds(NUM_PEERS, 0);
+    RaftProperties prop = new RaftProperties();
+    prop.setClass(RAFT_SERVER_LOG_APPENDER_FACTORY_CLASS_KEY,
+        PipelinedLogAppenderFactory.class, LogAppenderFactory.class);
+    return new MiniRaftClusterWithGRpc(s, prop, true);
+  }
+}

http://git-wip-us.apache.org/repos/asf/incubator-ratis/blob/7e71a2e0/ratis-grpc/src/test/java/org/apache/ratis/grpc/TestRaftReconfigurationWithGRpc.java
----------------------------------------------------------------------
diff --git a/ratis-grpc/src/test/java/org/apache/ratis/grpc/TestRaftReconfigurationWithGRpc.java b/ratis-grpc/src/test/java/org/apache/ratis/grpc/TestRaftReconfigurationWithGRpc.java
new file mode 100644
index 0000000..450eb6e
--- /dev/null
+++ b/ratis-grpc/src/test/java/org/apache/ratis/grpc/TestRaftReconfigurationWithGRpc.java
@@ -0,0 +1,47 @@
+/**
+ * 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.ratis.grpc;
+
+import org.apache.log4j.Level;
+import org.apache.ratis.grpc.server.PipelinedLogAppenderFactory;
+import org.apache.ratis.grpc.server.RaftServerProtocolService;
+import org.apache.ratis.server.impl.LogAppenderFactory;
+import org.apache.ratis.server.impl.RaftReconfigurationBaseTest;
+import org.apache.ratis.util.RaftUtils;
+import org.junit.BeforeClass;
+
+import static org.apache.ratis.server.RaftServerConfigKeys.RAFT_SERVER_LOG_APPENDER_FACTORY_CLASS_KEY;
+
+import java.io.IOException;
+
+public class TestRaftReconfigurationWithGRpc extends RaftReconfigurationBaseTest {
+  static {
+    RaftUtils.setLogLevel(RaftServerProtocolService.LOG, Level.DEBUG);
+  }
+
+  @BeforeClass
+  public static void setProp() {
+    prop.setClass(RAFT_SERVER_LOG_APPENDER_FACTORY_CLASS_KEY,
+        PipelinedLogAppenderFactory.class, LogAppenderFactory.class);
+  }
+
+  @Override
+  public MiniRaftClusterWithGRpc getCluster(int peerNum) throws IOException {
+    return new MiniRaftClusterWithGRpc(peerNum, prop);
+  }
+}

http://git-wip-us.apache.org/repos/asf/incubator-ratis/blob/7e71a2e0/ratis-grpc/src/test/java/org/apache/ratis/grpc/TestRaftSnapshotWithGrpc.java
----------------------------------------------------------------------
diff --git a/ratis-grpc/src/test/java/org/apache/ratis/grpc/TestRaftSnapshotWithGrpc.java b/ratis-grpc/src/test/java/org/apache/ratis/grpc/TestRaftSnapshotWithGrpc.java
new file mode 100644
index 0000000..8a26435
--- /dev/null
+++ b/ratis-grpc/src/test/java/org/apache/ratis/grpc/TestRaftSnapshotWithGrpc.java
@@ -0,0 +1,32 @@
+/**
+ * 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.ratis.grpc;
+
+import org.apache.ratis.MiniRaftCluster;
+import org.apache.ratis.conf.RaftProperties;
+import org.apache.ratis.statemachine.RaftSnapshotBaseTest;
+
+import java.io.IOException;
+
+public class TestRaftSnapshotWithGrpc extends RaftSnapshotBaseTest {
+  @Override
+  public MiniRaftCluster initCluster(int numServer, RaftProperties prop)
+      throws IOException {
+    return MiniRaftClusterWithGRpc.FACTORY.newCluster(numServer, prop, true);
+  }
+}

http://git-wip-us.apache.org/repos/asf/incubator-ratis/blob/7e71a2e0/ratis-grpc/src/test/java/org/apache/ratis/grpc/TestRaftStream.java
----------------------------------------------------------------------
diff --git a/ratis-grpc/src/test/java/org/apache/ratis/grpc/TestRaftStream.java b/ratis-grpc/src/test/java/org/apache/ratis/grpc/TestRaftStream.java
new file mode 100644
index 0000000..f4c8d27
--- /dev/null
+++ b/ratis-grpc/src/test/java/org/apache/ratis/grpc/TestRaftStream.java
@@ -0,0 +1,319 @@
+/**
+ * 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.ratis.grpc;
+
+import org.apache.log4j.Level;
+import org.apache.ratis.shaded.proto.RaftProtos.LogEntryProto;
+import org.apache.ratis.RaftTestUtil;
+import org.apache.ratis.conf.RaftProperties;
+import org.apache.ratis.grpc.client.AppendStreamer;
+import org.apache.ratis.grpc.client.RaftOutputStream;
+import org.apache.ratis.grpc.server.PipelinedLogAppenderFactory;
+import org.apache.ratis.server.impl.LogAppenderFactory;
+import org.apache.ratis.server.impl.RaftServerImpl;
+import org.apache.ratis.server.storage.RaftLog;
+import org.apache.ratis.util.RaftUtils;
+import org.junit.After;
+import org.junit.Assert;
+import org.junit.BeforeClass;
+import org.junit.Test;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.io.IOException;
+import java.util.*;
+import java.util.concurrent.CountDownLatch;
+import java.util.concurrent.TimeUnit;
+import java.util.concurrent.atomic.AtomicBoolean;
+import java.util.concurrent.atomic.AtomicInteger;
+import java.util.function.Supplier;
+
+import static org.apache.ratis.RaftTestUtil.waitForLeader;
+import static org.apache.ratis.grpc.RaftGrpcConfigKeys.RAFT_OUTPUTSTREAM_BUFFER_SIZE_KEY;
+import static org.apache.ratis.server.RaftServerConfigKeys.RAFT_SERVER_LOG_APPENDER_FACTORY_CLASS_KEY;
+import static org.junit.Assert.fail;
+
+public class TestRaftStream {
+  static {
+    RaftUtils.setLogLevel(AppendStreamer.LOG, Level.ALL);
+  }
+  static final Logger LOG = LoggerFactory.getLogger(TestRaftStream.class);
+
+  private static final RaftProperties prop = new RaftProperties();
+  private static final int NUM_SERVERS = 3;
+
+  private MiniRaftClusterWithGRpc cluster;
+
+
+  @BeforeClass
+  public static void setProp() {
+    prop.setClass(RAFT_SERVER_LOG_APPENDER_FACTORY_CLASS_KEY,
+        PipelinedLogAppenderFactory.class, LogAppenderFactory.class);
+  }
+
+  @After
+  public void tearDown() {
+    if (cluster != null) {
+      cluster.shutdown();
+    }
+  }
+
+  private byte[] genContent(int count) {
+    return toBytes(count);
+  }
+
+  private byte[] toBytes(int i) {
+    byte[] b = new byte[4];
+    b[0] = (byte) ((i >>> 24) & 0xFF);
+    b[1] = (byte) ((i >>> 16) & 0xFF);
+    b[2] = (byte) ((i >>> 8) & 0xFF);
+    b[3] = (byte) (i & 0xFF);
+    return b;
+  }
+
+  @Test
+  public void testSimpleWrite() throws Exception {
+    LOG.info("Running testSimpleWrite");
+
+    // default 64K is too large for a test
+    prop.setInt(RAFT_OUTPUTSTREAM_BUFFER_SIZE_KEY, 4);
+    cluster = new MiniRaftClusterWithGRpc(NUM_SERVERS, prop);
+
+    cluster.start();
+    RaftServerImpl leader = waitForLeader(cluster);
+
+    int count = 1;
+    try (RaftOutputStream out = new RaftOutputStream(prop, "writer-1",
+        cluster.getPeers(), leader.getId())) {
+      for (int i = 0; i < 500; i++) { // generate 500 requests
+        out.write(genContent(count++));
+      }
+    }
+
+    // check the leader's raft log
+    final RaftLog raftLog = leader.getState().getLog();
+    final AtomicInteger currentNum = new AtomicInteger(1);
+    checkLog(raftLog, 500, () -> {
+      int value = currentNum.getAndIncrement();
+      return toBytes(value);
+    });
+  }
+
+  private void checkLog(RaftLog raftLog, long expectedCommittedIndex,
+      Supplier<byte[]> s) {
+    long committedIndex = raftLog.getLastCommittedIndex();
+    Assert.assertEquals(expectedCommittedIndex, committedIndex);
+    // check the log content
+    LogEntryProto[] entries = raftLog.getEntries(1, expectedCommittedIndex + 1);
+    for (LogEntryProto entry : entries) {
+      byte[] logData = entry.getSmLogEntry().getData().toByteArray();
+      byte[] expected = s.get();
+      Assert.assertEquals("log entry: " + entry,
+          expected.length, logData.length);
+      Assert.assertArrayEquals(expected, logData);
+    }
+  }
+
+  @Test
+  public void testWriteAndFlush() throws Exception {
+    LOG.info("Running testWriteAndFlush");
+
+    prop.setInt(RAFT_OUTPUTSTREAM_BUFFER_SIZE_KEY, ByteValue.BUFFERSIZE);
+    cluster = new MiniRaftClusterWithGRpc(NUM_SERVERS, prop);
+    cluster.start();
+
+    RaftServerImpl leader = waitForLeader(cluster);
+    RaftOutputStream out = new RaftOutputStream(prop, "writer",
+        cluster.getPeers(), leader.getId());
+
+    int[] lengths = new int[]{1, 500, 1023, 1024, 1025, 2048, 3000, 3072};
+    ByteValue[] values = new ByteValue[lengths.length];
+    for (int i = 0; i < values.length; i++) {
+      values[i] = new ByteValue(lengths[i], (byte) 9);
+    }
+
+    List<byte[]> expectedTxs = new ArrayList<>();
+    for (ByteValue v : values) {
+      byte[] data = v.genData();
+      expectedTxs.addAll(v.getTransactions());
+      out.write(data);
+      out.flush();
+
+      // make sure after the flush the data has been committed
+      Assert.assertEquals(expectedTxs.size(),
+          leader.getState().getLastAppliedIndex());
+    }
+    out.close();
+
+    try {
+      out.write(0);
+      fail("The OutputStream has been closed");
+    } catch (IOException ignored) {
+    }
+
+    LOG.info("Start to check leader's log");
+    final AtomicInteger index = new AtomicInteger(0);
+    checkLog(leader.getState().getLog(), expectedTxs.size(),
+        () -> expectedTxs.get(index.getAndIncrement()));
+  }
+
+  private static class ByteValue {
+    final static int BUFFERSIZE = 1024;
+
+    final int length;
+    final byte value;
+    final int numTx;
+    byte[] data;
+
+    ByteValue(int length, byte value) {
+      this.length = length;
+      this.value = value;
+      numTx = (length - 1) / BUFFERSIZE + 1;
+    }
+
+    byte[] genData() {
+      data = new byte[length];
+      Arrays.fill(data, value);
+      return data;
+    }
+
+    Collection<byte[]> getTransactions() {
+      if (data.length <= BUFFERSIZE) {
+        return Collections.singletonList(data);
+      } else {
+        List<byte[]> list = new ArrayList<>();
+        for (int i = 0; i < numTx; i++) {
+          int txSize = Math.min(BUFFERSIZE, length - BUFFERSIZE * i);
+          byte[] t = new byte[txSize];
+          Arrays.fill(t, value);
+          list.add(t);
+        }
+        return list;
+      }
+    }
+  }
+
+  @Test
+  public void testWriteWithOffset() throws Exception {
+    LOG.info("Running testWriteWithOffset");
+    prop.setInt(RAFT_OUTPUTSTREAM_BUFFER_SIZE_KEY, ByteValue.BUFFERSIZE);
+
+    cluster = new MiniRaftClusterWithGRpc(NUM_SERVERS, prop);
+    cluster.start();
+    RaftServerImpl leader = waitForLeader(cluster);
+
+    RaftOutputStream out = new RaftOutputStream(prop, "writer",
+        cluster.getPeers(), leader.getId());
+
+    byte[] b1 = new byte[ByteValue.BUFFERSIZE / 2];
+    Arrays.fill(b1, (byte) 1);
+    byte[] b2 = new byte[ByteValue.BUFFERSIZE];
+    Arrays.fill(b2, (byte) 2);
+    byte[] b3 = new byte[ByteValue.BUFFERSIZE * 2 + ByteValue.BUFFERSIZE / 2];
+    Arrays.fill(b3, (byte) 3);
+    byte[] b4 = new byte[ByteValue.BUFFERSIZE * 4];
+    Arrays.fill(b3, (byte) 4);
+
+    byte[] expected = new byte[ByteValue.BUFFERSIZE * 8];
+    byte[][] data = new byte[][]{b1, b2, b3, b4};
+    final Random random = new Random();
+    int totalSize = 0;
+    for (byte[] b : data) {
+      System.arraycopy(b, 0, expected, totalSize, b.length);
+      totalSize += b.length;
+
+      int written = 0;
+      while (written < b.length) {
+        int toWrite = random.nextInt(b.length - written) + 1;
+        LOG.info("write {} bytes", toWrite);
+        out.write(b, written, toWrite);
+        written += toWrite;
+      }
+    }
+    out.close();
+
+    final RaftLog log = leader.getState().getLog();
+    // 0.5 + 1 + 2.5 + 4 = 8
+    Assert.assertEquals(8, leader.getState().getLastAppliedIndex());
+    Assert.assertEquals(8, log.getLastCommittedIndex());
+    LogEntryProto[] entries = log.getEntries(1, 9);
+    byte[] actual = new byte[ByteValue.BUFFERSIZE * 8];
+    totalSize = 0;
+    for (LogEntryProto e : entries) {
+      byte[] eValue = e.getSmLogEntry().getData().toByteArray();
+      Assert.assertEquals(ByteValue.BUFFERSIZE, eValue.length);
+      System.arraycopy(eValue, 0, actual, totalSize, eValue.length);
+      totalSize += eValue.length;
+    }
+    Assert.assertArrayEquals(expected, actual);
+  }
+
+  /**
+   * Write while leader is killed
+   */
+  @Test
+  public void testKillLeader() throws Exception {
+    LOG.info("Running testChangeLeader");
+
+    prop.setInt(RAFT_OUTPUTSTREAM_BUFFER_SIZE_KEY, 4);
+    cluster = new MiniRaftClusterWithGRpc(NUM_SERVERS, prop);
+    cluster.start();
+    final RaftServerImpl leader = waitForLeader(cluster);
+
+    final AtomicBoolean running  = new AtomicBoolean(true);
+    final AtomicBoolean success = new AtomicBoolean(false);
+    final AtomicInteger result = new AtomicInteger(0);
+    final CountDownLatch latch = new CountDownLatch(1);
+
+    new Thread(() -> {
+      LOG.info("Writer thread starts");
+      int count = 0;
+      try (RaftOutputStream out = new RaftOutputStream(prop, "writer",
+          cluster.getPeers(), leader.getId())) {
+        while (running.get()) {
+          out.write(toBytes(count++));
+          Thread.sleep(10);
+        }
+        success.set(true);
+        result.set(count);
+      } catch (Exception e) {
+        LOG.info("Got exception when writing", e);
+        success.set(false);
+      } finally {
+        latch.countDown();
+      }
+    }).start();
+
+    // force change the leader
+    RaftTestUtil.waitAndKillLeader(cluster, true);
+    final RaftServerImpl newLeader = waitForLeader(cluster);
+    Assert.assertNotEquals(leader.getId(), newLeader.getId());
+    Thread.sleep(500);
+
+    running.set(false);
+    latch.await(5, TimeUnit.SECONDS);
+    Assert.assertTrue(success.get());
+    // total number of tx should be >= result + 2, where 2 means two NoOp from
+    // leaders. It may be larger than result+2 because the client may resend
+    // requests and we do not have retry cache on servers yet.
+    LOG.info("last applied index: {}. total number of requests: {}",
+        newLeader.getState().getLastAppliedIndex(), result.get());
+    Assert.assertTrue(
+        newLeader.getState().getLastAppliedIndex() >= result.get() + 1);
+  }
+}

http://git-wip-us.apache.org/repos/asf/incubator-ratis/blob/7e71a2e0/ratis-grpc/src/test/java/org/apache/ratis/grpc/TestRaftWithGrpc.java
----------------------------------------------------------------------
diff --git a/ratis-grpc/src/test/java/org/apache/ratis/grpc/TestRaftWithGrpc.java b/ratis-grpc/src/test/java/org/apache/ratis/grpc/TestRaftWithGrpc.java
new file mode 100644
index 0000000..b60e30d
--- /dev/null
+++ b/ratis-grpc/src/test/java/org/apache/ratis/grpc/TestRaftWithGrpc.java
@@ -0,0 +1,73 @@
+/**
+ * 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.ratis.grpc;
+
+import org.apache.log4j.Level;
+import org.apache.ratis.RaftBasicTests;
+import org.apache.ratis.grpc.server.PipelinedLogAppenderFactory;
+import org.apache.ratis.server.impl.BlockRequestHandlingInjection;
+import org.apache.ratis.server.impl.LogAppenderFactory;
+import org.apache.ratis.server.impl.RaftServerImpl;
+import org.apache.ratis.util.RaftUtils;
+import org.junit.Assert;
+import org.junit.BeforeClass;
+import org.junit.Test;
+
+import static org.apache.ratis.server.RaftServerConfigKeys.RAFT_SERVER_LOG_APPENDER_FACTORY_CLASS_KEY;
+
+import java.io.IOException;
+
+public class TestRaftWithGrpc extends RaftBasicTests {
+  static {
+    RaftUtils.setLogLevel(RaftServerImpl.LOG, Level.DEBUG);
+  }
+
+  private final MiniRaftClusterWithGRpc cluster;
+
+  @BeforeClass
+  public static void setProp() {
+    properties.setClass(RAFT_SERVER_LOG_APPENDER_FACTORY_CLASS_KEY,
+        PipelinedLogAppenderFactory.class, LogAppenderFactory.class);
+  }
+
+  public TestRaftWithGrpc() throws IOException {
+    cluster = new MiniRaftClusterWithGRpc(NUM_SERVERS, properties);
+    Assert.assertNull(cluster.getLeader());
+  }
+
+  @Override
+  public MiniRaftClusterWithGRpc getCluster() {
+    return cluster;
+  }
+
+  @Override
+  @Test
+  public void testEnforceLeader() throws Exception {
+    super.testEnforceLeader();
+
+    MiniRaftClusterWithGRpc.sendServerRequestInjection.clear();
+    BlockRequestHandlingInjection.getInstance().unblockAll();
+  }
+
+  @Override
+  @Test
+  public void testWithLoad() throws Exception {
+    super.testWithLoad();
+    BlockRequestHandlingInjection.getInstance().unblockAll();
+  }
+}

http://git-wip-us.apache.org/repos/asf/incubator-ratis/blob/7e71a2e0/ratis-grpc/src/test/resources/log4j.properties
----------------------------------------------------------------------
diff --git a/ratis-grpc/src/test/resources/log4j.properties b/ratis-grpc/src/test/resources/log4j.properties
new file mode 100644
index 0000000..ced0687
--- /dev/null
+++ b/ratis-grpc/src/test/resources/log4j.properties
@@ -0,0 +1,18 @@
+#   Licensed 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.
+# log4j configuration used during build and unit tests
+
+log4j.rootLogger=info,stdout
+log4j.threshold=ALL
+log4j.appender.stdout=org.apache.log4j.ConsoleAppender
+log4j.appender.stdout.layout=org.apache.log4j.PatternLayout
+log4j.appender.stdout.layout.ConversionPattern=%d{ISO8601} %-5p %c{2} (%F:%M(%L)) - %m%n

http://git-wip-us.apache.org/repos/asf/incubator-ratis/blob/7e71a2e0/ratis-hadoop/pom.xml
----------------------------------------------------------------------
diff --git a/ratis-hadoop/pom.xml b/ratis-hadoop/pom.xml
new file mode 100644
index 0000000..ab7fd9f
--- /dev/null
+++ b/ratis-hadoop/pom.xml
@@ -0,0 +1,99 @@
+<?xml version="1.0" encoding="UTF-8"?>
+<!--
+  Licensed 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. See accompanying LICENSE file.
+-->
+<project xmlns="http://maven.apache.org/POM/4.0.0"
+         xmlns:xsi="http://www.w3.org/2001/XMLSchema-instance"
+         xsi:schemaLocation="http://maven.apache.org/POM/4.0.0 http://maven.apache.org/xsd/maven-4.0.0.xsd">
+  <modelVersion>4.0.0</modelVersion>
+  <parent>
+    <artifactId>ratis-project-dist</artifactId>
+    <groupId>org.apache.ratis</groupId>
+    <version>1.0-SNAPSHOT</version>
+    <relativePath>../ratis-project-dist</relativePath>
+  </parent>
+
+  <artifactId>ratis-hadoop</artifactId>
+  <name>Ratis Hadoop Support</name>
+
+  <dependencies>
+    <dependency>
+      <artifactId>ratis-proto-shaded</artifactId>
+      <groupId>org.apache.ratis</groupId>
+      <scope>provided</scope>
+    </dependency>
+
+    <dependency>
+      <artifactId>ratis-common</artifactId>
+      <groupId>org.apache.ratis</groupId>
+      <scope>provided</scope>
+    </dependency>
+    <dependency>
+      <artifactId>ratis-common</artifactId>
+      <groupId>org.apache.ratis</groupId>
+      <scope>test</scope>
+      <type>test-jar</type>
+    </dependency>
+    
+    <dependency>
+      <artifactId>ratis-client</artifactId>
+      <groupId>org.apache.ratis</groupId>
+      <scope>provided</scope>
+    </dependency>
+    <dependency>
+      <artifactId>ratis-client</artifactId>
+      <groupId>org.apache.ratis</groupId>
+      <scope>test</scope>
+      <type>test-jar</type>
+    </dependency>
+    
+    <dependency>
+      <artifactId>ratis-server</artifactId>
+      <groupId>org.apache.ratis</groupId>
+      <scope>provided</scope>
+    </dependency>
+    <dependency>
+      <artifactId>ratis-server</artifactId>
+      <groupId>org.apache.ratis</groupId>
+      <scope>test</scope>
+      <type>test-jar</type>
+    </dependency>
+    
+    <dependency>
+      <groupId>org.slf4j</groupId>
+      <artifactId>slf4j-api</artifactId>
+    </dependency>
+
+    <dependency>
+      <groupId>com.google.guava</groupId>
+      <artifactId>guava</artifactId>
+    </dependency>
+
+    <dependency>
+      <groupId>junit</groupId>
+      <artifactId>junit</artifactId>
+      <scope>test</scope>
+    </dependency>
+    <dependency>
+      <groupId>org.mockito</groupId>
+      <artifactId>mockito-all</artifactId>
+      <scope>test</scope>
+    </dependency>
+
+    <dependency>
+      <groupId>org.apache.hadoop</groupId>
+      <artifactId>hadoop-common</artifactId>
+      <version>${hadoop.version}</version>
+    </dependency>    
+  </dependencies>
+</project>

http://git-wip-us.apache.org/repos/asf/incubator-ratis/blob/7e71a2e0/ratis-hadoop/src/main/java/org/apache/hadoop/ipc/ProtobufRpcEngineShaded.java
----------------------------------------------------------------------
diff --git a/ratis-hadoop/src/main/java/org/apache/hadoop/ipc/ProtobufRpcEngineShaded.java b/ratis-hadoop/src/main/java/org/apache/hadoop/ipc/ProtobufRpcEngineShaded.java
new file mode 100644
index 0000000..a5a9654
--- /dev/null
+++ b/ratis-hadoop/src/main/java/org/apache/hadoop/ipc/ProtobufRpcEngineShaded.java
@@ -0,0 +1,623 @@
+/**
+ * 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.ipc;
+
+import com.google.common.annotations.VisibleForTesting;
+import org.apache.commons.logging.Log;
+import org.apache.commons.logging.LogFactory;
+import org.apache.hadoop.classification.InterfaceAudience;
+import org.apache.hadoop.classification.InterfaceStability;
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.io.DataOutputOutputStream;
+import org.apache.hadoop.io.Writable;
+import org.apache.hadoop.io.retry.RetryPolicy;
+import org.apache.hadoop.ipc.Client.ConnectionId;
+import org.apache.hadoop.ipc.RPC.RpcInvoker;
+import org.apache.hadoop.security.UserGroupInformation;
+import org.apache.hadoop.security.token.SecretManager;
+import org.apache.hadoop.security.token.TokenIdentifier;
+import org.apache.hadoop.util.ProtoUtil;
+import org.apache.hadoop.util.Time;
+import org.apache.ratis.shaded.com.google.protobuf.*;
+import org.apache.ratis.shaded.com.google.protobuf.Descriptors.MethodDescriptor;
+import org.apache.ratis.shaded.org.apache.hadoop.ipc.protobuf.ProtobufRpcEngineProtos.RequestHeaderProto;
+import org.apache.ratis.shaded.org.apache.hadoop.ipc.protobuf.RpcHeaderProtos.RpcRequestHeaderProto;
+import org.apache.ratis.shaded.org.apache.hadoop.ipc.protobuf.RpcHeaderProtos.RpcResponseHeaderProto;
+
+import javax.net.SocketFactory;
+import java.io.DataInput;
+import java.io.DataOutput;
+import java.io.IOException;
+import java.io.OutputStream;
+import java.lang.reflect.Method;
+import java.lang.reflect.Proxy;
+import java.net.InetSocketAddress;
+import java.util.Map;
+import java.util.concurrent.ConcurrentHashMap;
+import java.util.concurrent.atomic.AtomicBoolean;
+
+/**
+ * Copied from {@link org.apache.hadoop.ipc.ProtobufRpcEngine}
+ * and replaced the protobuf classes with the shaded classes.
+ */
+@InterfaceStability.Evolving
+public class ProtobufRpcEngineShaded implements RpcEngine {
+  public static final Log LOG = LogFactory.getLog(ProtobufRpcEngineShaded.class);
+
+  static { // Register the rpcRequest deserializer for WritableRpcEngine
+    org.apache.hadoop.ipc.Server.registerProtocolEngine(
+        RPC.RpcKind.RPC_PROTOCOL_BUFFER, RpcRequestWrapper.class,
+        new Server.ProtoBufRpcInvoker());
+  }
+
+  private static final ClientCache CLIENTS = new ClientCache();
+
+  public <T> ProtocolProxy<T> getProxy(Class<T> protocol, long clientVersion,
+      InetSocketAddress addr, UserGroupInformation ticket, Configuration conf,
+      SocketFactory factory, int rpcTimeout) throws IOException {
+    return getProxy(protocol, clientVersion, addr, ticket, conf, factory,
+        rpcTimeout, null);
+  }
+
+  @Override
+  public <T> ProtocolProxy<T> getProxy(Class<T> protocol, long clientVersion,
+      InetSocketAddress addr, UserGroupInformation ticket, Configuration conf,
+      SocketFactory factory, int rpcTimeout, RetryPolicy connectionRetryPolicy
+      ) throws IOException {
+    return getProxy(protocol, clientVersion, addr, ticket, conf, factory,
+      rpcTimeout, connectionRetryPolicy, null);
+  }
+
+  @Override
+  @SuppressWarnings("unchecked")
+  public <T> ProtocolProxy<T> getProxy(Class<T> protocol, long clientVersion,
+      InetSocketAddress addr, UserGroupInformation ticket, Configuration conf,
+      SocketFactory factory, int rpcTimeout, RetryPolicy connectionRetryPolicy,
+      AtomicBoolean fallbackToSimpleAuth) throws IOException {
+
+    final Invoker invoker = new Invoker(protocol, addr, ticket, conf, factory,
+        rpcTimeout, connectionRetryPolicy, fallbackToSimpleAuth);
+    return new ProtocolProxy<T>(protocol, (T) Proxy.newProxyInstance(
+        protocol.getClassLoader(), new Class[]{protocol}, invoker), false);
+  }
+
+  @Override
+  public ProtocolProxy<ProtocolMetaInfoPB> getProtocolMetaInfoProxy(
+      ConnectionId connId, Configuration conf, SocketFactory factory)
+      throws IOException {
+    Class<ProtocolMetaInfoPB> protocol = ProtocolMetaInfoPB.class;
+    return new ProtocolProxy<ProtocolMetaInfoPB>(protocol,
+        (ProtocolMetaInfoPB) Proxy.newProxyInstance(protocol.getClassLoader(),
+            new Class[] { protocol }, new Invoker(protocol, connId, conf,
+                factory)), false);
+  }
+
+  private static class Invoker implements RpcInvocationHandler {
+    private final Map<String, Message> returnTypes =
+        new ConcurrentHashMap<String, Message>();
+    private boolean isClosed = false;
+    private final Client.ConnectionId remoteId;
+    private final Client client;
+    private final long clientProtocolVersion;
+    private final String protocolName;
+    private AtomicBoolean fallbackToSimpleAuth;
+
+    private Invoker(Class<?> protocol, InetSocketAddress addr,
+        UserGroupInformation ticket, Configuration conf, SocketFactory factory,
+        int rpcTimeout, RetryPolicy connectionRetryPolicy,
+        AtomicBoolean fallbackToSimpleAuth) throws IOException {
+      this(protocol, Client.ConnectionId.getConnectionId(
+          addr, protocol, ticket, rpcTimeout, connectionRetryPolicy, conf),
+          conf, factory);
+      this.fallbackToSimpleAuth = fallbackToSimpleAuth;
+    }
+
+    /**
+     * This constructor takes a connectionId, instead of creating a new one.
+     */
+    private Invoker(Class<?> protocol, Client.ConnectionId connId,
+        Configuration conf, SocketFactory factory) {
+      this.remoteId = connId;
+      this.client = CLIENTS.getClient(conf, factory, RpcResponseWrapper.class);
+      this.protocolName = RPC.getProtocolName(protocol);
+      this.clientProtocolVersion = RPC
+          .getProtocolVersion(protocol);
+    }
+
+    private RequestHeaderProto constructRpcRequestHeader(Method method) {
+      RequestHeaderProto.Builder builder = RequestHeaderProto
+          .newBuilder();
+      builder.setMethodName(method.getName());
+
+
+      // For protobuf, {@code protocol} used when creating client side proxy is
+      // the interface extending BlockingInterface, which has the annotations
+      // such as ProtocolName etc.
+      //
+      // Using Method.getDeclaringClass(), as in WritableEngine to get at
+      // the protocol interface will return BlockingInterface, from where
+      // the annotation ProtocolName and Version cannot be
+      // obtained.
+      //
+      // Hence we simply use the protocol class used to create the proxy.
+      // For PB this may limit the use of mixins on client side.
+      builder.setDeclaringClassProtocolName(protocolName);
+      builder.setClientProtocolVersion(clientProtocolVersion);
+      return builder.build();
+    }
+
+    /**
+     * This is the client side invoker of RPC method. It only throws
+     * ServiceException, since the invocation proxy expects only
+     * ServiceException to be thrown by the method in case protobuf service.
+     *
+     * ServiceException has the following causes:
+     * <ol>
+     * <li>Exceptions encountered on the client side in this method are
+     * set as cause in ServiceException as is.</li>
+     * <li>Exceptions from the server are wrapped in RemoteException and are
+     * set as cause in ServiceException</li>
+     * </ol>
+     *
+     * Note that the client calling protobuf RPC methods, must handle
+     * ServiceException by getting the cause from the ServiceException. If the
+     * cause is RemoteException, then unwrap it to get the exception thrown by
+     * the server.
+     */
+    @Override
+    public Object invoke(Object proxy, Method method, Object[] args)
+        throws ServiceException {
+      long startTime = 0;
+      if (LOG.isDebugEnabled()) {
+        startTime = Time.now();
+      }
+
+      if (args.length != 2) { // RpcController + Message
+        throw new ServiceException("Too many parameters for request. Method: ["
+            + method.getName() + "]" + ", Expected: 2, Actual: "
+            + args.length);
+      }
+      if (args[1] == null) {
+        throw new ServiceException("null param while calling Method: ["
+            + method.getName() + "]");
+      }
+
+      RequestHeaderProto rpcRequestHeader = constructRpcRequestHeader(method);
+
+      if (LOG.isTraceEnabled()) {
+        LOG.trace(Thread.currentThread().getId() + ": Call -> " +
+            remoteId + ": " + method.getName() +
+            " {" + TextFormat.shortDebugString((Message) args[1]) + "}");
+      }
+
+
+      Message theRequest = (Message) args[1];
+      final RpcResponseWrapper val;
+      try {
+        val = (RpcResponseWrapper) client.call(RPC.RpcKind.RPC_PROTOCOL_BUFFER,
+            new RpcRequestWrapper(rpcRequestHeader, theRequest), remoteId,
+            fallbackToSimpleAuth);
+
+      } catch (Throwable e) {
+        if (LOG.isTraceEnabled()) {
+          LOG.trace(Thread.currentThread().getId() + ": Exception <- " +
+              remoteId + ": " + method.getName() +
+                " {" + e + "}");
+        }
+        throw new ServiceException(e);
+      }
+
+      if (LOG.isDebugEnabled()) {
+        long callTime = Time.now() - startTime;
+        LOG.debug("Call: " + method.getName() + " took " + callTime + "ms");
+      }
+
+      Message prototype = null;
+      try {
+        prototype = getReturnProtoType(method);
+      } catch (Exception e) {
+        throw new ServiceException(e);
+      }
+      Message returnMessage;
+      try {
+        returnMessage = prototype.newBuilderForType()
+            .mergeFrom(val.theResponseRead).build();
+
+        if (LOG.isTraceEnabled()) {
+          LOG.trace(Thread.currentThread().getId() + ": Response <- " +
+              remoteId + ": " + method.getName() +
+                " {" + TextFormat.shortDebugString(returnMessage) + "}");
+        }
+
+      } catch (Throwable e) {
+        throw new ServiceException(e);
+      }
+      return returnMessage;
+    }
+
+    @Override
+    public void close() throws IOException {
+      if (!isClosed) {
+        isClosed = true;
+        CLIENTS.stopClient(client);
+      }
+    }
+
+    private Message getReturnProtoType(Method method) throws Exception {
+      if (returnTypes.containsKey(method.getName())) {
+        return returnTypes.get(method.getName());
+      }
+
+      Class<?> returnType = method.getReturnType();
+      Method newInstMethod = returnType.getMethod("getDefaultInstance");
+      newInstMethod.setAccessible(true);
+      Message prototype = (Message) newInstMethod.invoke(null, (Object[]) null);
+      returnTypes.put(method.getName(), prototype);
+      return prototype;
+    }
+
+    @Override //RpcInvocationHandler
+    public ConnectionId getConnectionId() {
+      return remoteId;
+    }
+  }
+
+  interface RpcWrapper extends Writable {
+    int getLength();
+  }
+  /**
+   * Wrapper for Protocol Buffer Requests
+   *
+   * Note while this wrapper is writable, the request on the wire is in
+   * Protobuf. Several methods on {@link org.apache.hadoop.ipc.Server and RPC}
+   * use type Writable as a wrapper to work across multiple RpcEngine kinds.
+   */
+  private static abstract class RpcMessageWithHeader<T extends GeneratedMessage>
+    implements RpcWrapper {
+    T requestHeader;
+    Message theRequest; // for clientSide, the request is here
+    byte[] theRequestRead; // for server side, the request is here
+
+    public RpcMessageWithHeader() {
+    }
+
+    public RpcMessageWithHeader(T requestHeader, Message theRequest) {
+      this.requestHeader = requestHeader;
+      this.theRequest = theRequest;
+    }
+
+    @Override
+    public void write(DataOutput out) throws IOException {
+      OutputStream os = DataOutputOutputStream.constructOutputStream(out);
+
+      ((Message)requestHeader).writeDelimitedTo(os);
+      theRequest.writeDelimitedTo(os);
+    }
+
+    @Override
+    public void readFields(DataInput in) throws IOException {
+      requestHeader = parseHeaderFrom(readVarintBytes(in));
+      theRequestRead = readMessageRequest(in);
+    }
+
+    abstract T parseHeaderFrom(byte[] bytes) throws IOException;
+
+    byte[] readMessageRequest(DataInput in) throws IOException {
+      return readVarintBytes(in);
+    }
+
+    private static byte[] readVarintBytes(DataInput in) throws IOException {
+      final int length = ProtoUtil.readRawVarint32(in);
+      final byte[] bytes = new byte[length];
+      in.readFully(bytes);
+      return bytes;
+    }
+
+    public T getMessageHeader() {
+      return requestHeader;
+    }
+
+    public byte[] getMessageBytes() {
+      return theRequestRead;
+    }
+
+    @Override
+    public int getLength() {
+      int headerLen = requestHeader.getSerializedSize();
+      int reqLen;
+      if (theRequest != null) {
+        reqLen = theRequest.getSerializedSize();
+      } else if (theRequestRead != null ) {
+        reqLen = theRequestRead.length;
+      } else {
+        throw new IllegalArgumentException(
+            "getLength on uninitialized RpcWrapper");
+      }
+      return CodedOutputStream.computeRawVarint32Size(headerLen) +  headerLen
+          + CodedOutputStream.computeRawVarint32Size(reqLen) + reqLen;
+    }
+  }
+
+  private static class RpcRequestWrapper
+  extends RpcMessageWithHeader<RequestHeaderProto> {
+    @SuppressWarnings("unused")
+    public RpcRequestWrapper() {}
+
+    public RpcRequestWrapper(
+        RequestHeaderProto requestHeader, Message theRequest) {
+      super(requestHeader, theRequest);
+    }
+
+    @Override
+    RequestHeaderProto parseHeaderFrom(byte[] bytes) throws IOException {
+      return RequestHeaderProto.parseFrom(bytes);
+    }
+
+    @Override
+    public String toString() {
+      return requestHeader.getDeclaringClassProtocolName() + "." +
+          requestHeader.getMethodName();
+    }
+  }
+
+  @InterfaceAudience.LimitedPrivate({"RPC"})
+  public static class RpcRequestMessageWrapper
+  extends RpcMessageWithHeader<RpcRequestHeaderProto> {
+    public RpcRequestMessageWrapper() {}
+
+    public RpcRequestMessageWrapper(
+        RpcRequestHeaderProto requestHeader, Message theRequest) {
+      super(requestHeader, theRequest);
+    }
+
+    @Override
+    RpcRequestHeaderProto parseHeaderFrom(byte[] bytes) throws IOException {
+      return RpcRequestHeaderProto.parseFrom(bytes);
+    }
+  }
+
+  @InterfaceAudience.LimitedPrivate({"RPC"})
+  public static class RpcResponseMessageWrapper
+  extends RpcMessageWithHeader<RpcResponseHeaderProto> {
+    public RpcResponseMessageWrapper() {}
+
+    public RpcResponseMessageWrapper(
+        RpcResponseHeaderProto responseHeader, Message theRequest) {
+      super(responseHeader, theRequest);
+    }
+
+    @Override
+    byte[] readMessageRequest(DataInput in) throws IOException {
+      // error message contain no message body
+      switch (requestHeader.getStatus()) {
+        case ERROR:
+        case FATAL:
+          return null;
+        default:
+          return super.readMessageRequest(in);
+      }
+    }
+
+    @Override
+    RpcResponseHeaderProto parseHeaderFrom(byte[] bytes) throws IOException {
+      return RpcResponseHeaderProto.parseFrom(bytes);
+    }
+  }
+
+  /**
+   *  Wrapper for Protocol Buffer Responses
+   *
+   * Note while this wrapper is writable, the request on the wire is in
+   * Protobuf. Several methods on {@link org.apache.hadoop.ipc.Server and RPC}
+   * use type Writable as a wrapper to work across multiple RpcEngine kinds.
+   */
+  @InterfaceAudience.LimitedPrivate({"RPC"}) // temporarily exposed
+  public static class RpcResponseWrapper implements RpcWrapper {
+    Message theResponse; // for senderSide, the response is here
+    byte[] theResponseRead; // for receiver side, the response is here
+
+    public RpcResponseWrapper() {
+    }
+
+    public RpcResponseWrapper(Message message) {
+      this.theResponse = message;
+    }
+
+    @Override
+    public void write(DataOutput out) throws IOException {
+      OutputStream os = DataOutputOutputStream.constructOutputStream(out);
+      theResponse.writeDelimitedTo(os);
+    }
+
+    @Override
+    public void readFields(DataInput in) throws IOException {
+      int length = ProtoUtil.readRawVarint32(in);
+      theResponseRead = new byte[length];
+      in.readFully(theResponseRead);
+    }
+
+    @Override
+    public int getLength() {
+      int resLen;
+      if (theResponse != null) {
+        resLen = theResponse.getSerializedSize();
+      } else if (theResponseRead != null ) {
+        resLen = theResponseRead.length;
+      } else {
+        throw new IllegalArgumentException(
+            "getLength on uninitialized RpcWrapper");
+      }
+      return CodedOutputStream.computeRawVarint32Size(resLen) + resLen;
+    }
+  }
+
+  @VisibleForTesting
+  @InterfaceAudience.Private
+  @InterfaceStability.Unstable
+  static Client getClient(Configuration conf) {
+    return CLIENTS.getClient(conf, SocketFactory.getDefault(),
+        RpcResponseWrapper.class);
+  }
+
+
+
+  @Override
+  public RPC.Server getServer(Class<?> protocol, Object protocolImpl,
+      String bindAddress, int port, int numHandlers, int numReaders,
+      int queueSizePerHandler, boolean verbose, Configuration conf,
+      SecretManager<? extends TokenIdentifier> secretManager,
+      String portRangeConfig)
+      throws IOException {
+    return new Server(protocol, protocolImpl, conf, bindAddress, port,
+        numHandlers, numReaders, queueSizePerHandler, verbose, secretManager,
+        portRangeConfig);
+  }
+
+  public static class Server extends RPC.Server {
+    /**
+     * Construct an RPC server.
+     *
+     * @param protocolClass the class of protocol
+     * @param protocolImpl the protocolImpl whose methods will be called
+     * @param conf the configuration to use
+     * @param bindAddress the address to bind on to listen for connection
+     * @param port the port to listen for connections on
+     * @param numHandlers the number of method handler threads to run
+     * @param verbose whether each call should be logged
+     * @param portRangeConfig A config parameter that can be used to restrict
+     * the range of ports used when port is 0 (an ephemeral port)
+     */
+    public Server(Class<?> protocolClass, Object protocolImpl,
+        Configuration conf, String bindAddress, int port, int numHandlers,
+        int numReaders, int queueSizePerHandler, boolean verbose,
+        SecretManager<? extends TokenIdentifier> secretManager,
+        String portRangeConfig)
+        throws IOException {
+      super(bindAddress, port, null, numHandlers,
+          numReaders, queueSizePerHandler, conf, classNameBase(protocolImpl
+              .getClass().getName()), secretManager, portRangeConfig);
+      this.verbose = verbose;
+      registerProtocolAndImpl(RPC.RpcKind.RPC_PROTOCOL_BUFFER, protocolClass,
+          protocolImpl);
+    }
+
+    /**
+     * Protobuf invoker for {@link RpcInvoker}
+     */
+    static class ProtoBufRpcInvoker implements RpcInvoker {
+      private static ProtoClassProtoImpl getProtocolImpl(RPC.Server server,
+          String protoName, long clientVersion) throws RpcServerException {
+        ProtoNameVer pv = new ProtoNameVer(protoName, clientVersion);
+        ProtoClassProtoImpl impl =
+            server.getProtocolImplMap(RPC.RpcKind.RPC_PROTOCOL_BUFFER).get(pv);
+        if (impl == null) { // no match for Protocol AND Version
+          VerProtocolImpl highest =
+              server.getHighestSupportedProtocol(RPC.RpcKind.RPC_PROTOCOL_BUFFER,
+                  protoName);
+          if (highest == null) {
+            throw new RpcNoSuchProtocolException(
+                "Unknown protocol: " + protoName);
+          }
+          // protocol supported but not the version that client wants
+          throw new RPC.VersionMismatch(protoName, clientVersion,
+              highest.version);
+        }
+        return impl;
+      }
+
+      @Override
+      /**
+       * This is a server side method, which is invoked over RPC. On success
+       * the return response has protobuf response payload. On failure, the
+       * exception name and the stack trace are return in the resposne.
+       * See {@link HadoopRpcResponseProto}
+       *
+       * In this method there three types of exceptions possible and they are
+       * returned in response as follows.
+       * <ol>
+       * <li> Exceptions encountered in this method that are returned
+       * as {@link RpcServerException} </li>
+       * <li> Exceptions thrown by the service is wrapped in ServiceException.
+       * In that this method returns in response the exception thrown by the
+       * service.</li>
+       * <li> Other exceptions thrown by the service. They are returned as
+       * it is.</li>
+       * </ol>
+       */
+      public Writable call(RPC.Server server, String protocol,
+          Writable writableRequest, long receiveTime) throws Exception {
+        RpcRequestWrapper request = (RpcRequestWrapper) writableRequest;
+        RequestHeaderProto rpcRequest = request.requestHeader;
+        String methodName = rpcRequest.getMethodName();
+        String protoName = rpcRequest.getDeclaringClassProtocolName();
+        long clientVersion = rpcRequest.getClientProtocolVersion();
+        if (server.verbose)
+          LOG.info("Call: protocol=" + protocol + ", method=" + methodName);
+
+        ProtoClassProtoImpl protocolImpl = getProtocolImpl(server, protoName,
+            clientVersion);
+        BlockingService service = (BlockingService) protocolImpl.protocolImpl;
+        MethodDescriptor methodDescriptor = service.getDescriptorForType()
+            .findMethodByName(methodName);
+        if (methodDescriptor == null) {
+          String msg = "Unknown method " + methodName + " called on " + protocol
+              + " protocol.";
+          LOG.warn(msg);
+          throw new RpcNoSuchMethodException(msg);
+        }
+        Message prototype = service.getRequestPrototype(methodDescriptor);
+        Message param = prototype.newBuilderForType()
+            .mergeFrom(request.theRequestRead).build();
+
+        Message result;
+        long startTime = Time.now();
+        int qTime = (int) (startTime - receiveTime);
+        Exception exception = null;
+        try {
+          server.rpcDetailedMetrics.init(protocolImpl.protocolClass);
+          result = service.callBlockingMethod(methodDescriptor, null, param);
+        } catch (ServiceException e) {
+          exception = (Exception) e.getCause();
+          throw (Exception) e.getCause();
+        } catch (Exception e) {
+          exception = e;
+          throw e;
+        } finally {
+          int processingTime = (int) (Time.now() - startTime);
+          if (LOG.isDebugEnabled()) {
+            String msg = "Served: " + methodName + " queueTime= " + qTime +
+                " procesingTime= " + processingTime;
+            if (exception != null) {
+              msg += " exception= " + exception.getClass().getSimpleName();
+            }
+            LOG.debug(msg);
+          }
+          String detailedMetricsName = (exception == null) ?
+              methodName :
+              exception.getClass().getSimpleName();
+          server.rpcMetrics.addRpcQueueTime(qTime);
+          server.rpcMetrics.addRpcProcessingTime(processingTime);
+          server.rpcDetailedMetrics.addProcessingTime(detailedMetricsName,
+              processingTime);
+        }
+        return new RpcResponseWrapper(result);
+      }
+    }
+  }
+}

http://git-wip-us.apache.org/repos/asf/incubator-ratis/blob/7e71a2e0/ratis-hadoop/src/main/java/org/apache/ratis/hadooprpc/HadoopConstants.java
----------------------------------------------------------------------
diff --git a/ratis-hadoop/src/main/java/org/apache/ratis/hadooprpc/HadoopConstants.java b/ratis-hadoop/src/main/java/org/apache/ratis/hadooprpc/HadoopConstants.java
new file mode 100644
index 0000000..a50b938
--- /dev/null
+++ b/ratis-hadoop/src/main/java/org/apache/ratis/hadooprpc/HadoopConstants.java
@@ -0,0 +1,29 @@
+/**
+ * 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.ratis.hadooprpc;
+
+public interface HadoopConstants {
+  String RAFT_SERVER_KERBEROS_PRINCIPAL_KEY
+      = "raft.server.kerberos.principal";
+  String RAFT_CLIENT_KERBEROS_PRINCIPAL_KEY
+      = "raft.client.kerberos.principal";
+  String RAFT_SERVER_PROTOCOL_NAME
+      = "org.apache.hadoop.raft.server.protocol.RaftServerProtocol";
+  String RAFT_CLIENT_PROTOCOL_NAME
+      = "org.apache.hadoop.raft.protocol.RaftClientProtocol";
+}

http://git-wip-us.apache.org/repos/asf/incubator-ratis/blob/7e71a2e0/ratis-hadoop/src/main/java/org/apache/ratis/hadooprpc/Proxy.java
----------------------------------------------------------------------
diff --git a/ratis-hadoop/src/main/java/org/apache/ratis/hadooprpc/Proxy.java b/ratis-hadoop/src/main/java/org/apache/ratis/hadooprpc/Proxy.java
new file mode 100644
index 0000000..22edef1
--- /dev/null
+++ b/ratis-hadoop/src/main/java/org/apache/ratis/hadooprpc/Proxy.java
@@ -0,0 +1,55 @@
+/**
+ * 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.ratis.hadooprpc;
+
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.ipc.ProtobufRpcEngineShaded;
+import org.apache.hadoop.ipc.RPC;
+import org.apache.hadoop.net.NetUtils;
+import org.apache.hadoop.security.UserGroupInformation;
+
+import java.io.Closeable;
+import java.io.IOException;
+
+public class Proxy<PROTOCOL> implements Closeable {
+  public static <PROTOCOL> PROTOCOL getProxy(
+      Class<PROTOCOL> clazz, String addressStr, Configuration conf)
+      throws IOException {
+    RPC.setProtocolEngine(conf, clazz, ProtobufRpcEngineShaded.class);
+    return RPC.getProxy(clazz, RPC.getProtocolVersion(clazz),
+        org.apache.ratis.util.NetUtils.newInetSocketAddress(addressStr),
+        UserGroupInformation.getCurrentUser(),
+        conf, NetUtils.getSocketFactory(conf, clazz));
+  }
+
+  private final PROTOCOL protocol;
+
+  public Proxy(Class<PROTOCOL> clazz, String addressStr, Configuration conf)
+      throws IOException {
+    this.protocol = getProxy(clazz, addressStr, conf);
+  }
+
+  public PROTOCOL getProtocol() {
+    return protocol;
+  }
+
+  @Override
+  public void close() {
+    RPC.stopProxy(protocol);
+  }
+}


[23/54] [abbrv] incubator-ratis git commit: Renamed the packages from raft to ratis in preperation for Apache Incubation - Moved all java packages from org.apache.raft to org.apache.ratis. - Moved native package to org_apache_ratis, and native lib to l

Posted by ji...@apache.org.
http://git-wip-us.apache.org/repos/asf/incubator-ratis/blob/7e71a2e0/ratis-hadoop/src/main/java/org/apache/ratis/hadooprpc/client/HadoopClientRequestSender.java
----------------------------------------------------------------------
diff --git a/ratis-hadoop/src/main/java/org/apache/ratis/hadooprpc/client/HadoopClientRequestSender.java b/ratis-hadoop/src/main/java/org/apache/ratis/hadooprpc/client/HadoopClientRequestSender.java
new file mode 100644
index 0000000..aac7c31
--- /dev/null
+++ b/ratis-hadoop/src/main/java/org/apache/ratis/hadooprpc/client/HadoopClientRequestSender.java
@@ -0,0 +1,68 @@
+/**
+ * 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.ratis.hadooprpc.client;
+
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.ipc.RemoteException;
+import org.apache.ratis.client.RaftClientRequestSender;
+import org.apache.ratis.protocol.*;
+import org.apache.ratis.util.PeerProxyMap;
+
+import java.io.IOException;
+import java.util.Collection;
+
+public class HadoopClientRequestSender implements RaftClientRequestSender {
+
+  private final PeerProxyMap<RaftClientProtocolClientSideTranslatorPB> proxies;
+
+  public HadoopClientRequestSender(
+      Collection<RaftPeer> peers, final Configuration conf) {
+    this.proxies  = new PeerProxyMap<>(
+        p -> new RaftClientProtocolClientSideTranslatorPB(p.getAddress(), conf));
+    proxies.addPeers(peers);
+  }
+
+  @Override
+  public RaftClientReply sendRequest(RaftClientRequest request)
+      throws IOException {
+    final String serverId = request.getReplierId();
+    final RaftClientProtocolClientSideTranslatorPB proxy =
+        proxies.getProxy(serverId);
+    try {
+      if (request instanceof SetConfigurationRequest) {
+        return proxy.setConfiguration((SetConfigurationRequest) request);
+      } else {
+        return proxy.submitClientRequest(request);
+      }
+    } catch (RemoteException e) {
+      throw e.unwrapRemoteException(StateMachineException.class,
+          ReconfigurationTimeoutException.class,
+          ReconfigurationInProgressException.class, RaftException.class);
+    }
+  }
+
+  @Override
+  public void addServers(Iterable<RaftPeer> servers) {
+    proxies.addPeers(servers);
+  }
+
+  @Override
+  public void close() {
+    proxies.close();
+  }
+}

http://git-wip-us.apache.org/repos/asf/incubator-ratis/blob/7e71a2e0/ratis-hadoop/src/main/java/org/apache/ratis/hadooprpc/client/RaftClientProtocolClientSideTranslatorPB.java
----------------------------------------------------------------------
diff --git a/ratis-hadoop/src/main/java/org/apache/ratis/hadooprpc/client/RaftClientProtocolClientSideTranslatorPB.java b/ratis-hadoop/src/main/java/org/apache/ratis/hadooprpc/client/RaftClientProtocolClientSideTranslatorPB.java
new file mode 100644
index 0000000..a5c1a13
--- /dev/null
+++ b/ratis-hadoop/src/main/java/org/apache/ratis/hadooprpc/client/RaftClientProtocolClientSideTranslatorPB.java
@@ -0,0 +1,70 @@
+/**
+ * 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.ratis.hadooprpc.client;
+
+import java.io.IOException;
+
+import org.apache.hadoop.classification.InterfaceAudience;
+import org.apache.hadoop.conf.Configuration;
+import org.apache.ratis.client.impl.ClientProtoUtils;
+import org.apache.ratis.hadooprpc.Proxy;
+import org.apache.ratis.protocol.RaftClientProtocol;
+import org.apache.ratis.protocol.RaftClientReply;
+import org.apache.ratis.protocol.RaftClientRequest;
+import org.apache.ratis.protocol.SetConfigurationRequest;
+import org.apache.ratis.shaded.com.google.protobuf.ServiceException;
+import org.apache.ratis.shaded.proto.RaftProtos.RaftClientReplyProto;
+import org.apache.ratis.shaded.proto.RaftProtos.RaftClientRequestProto;
+import org.apache.ratis.shaded.proto.RaftProtos.SetConfigurationRequestProto;
+import org.apache.ratis.util.ProtoUtils;
+
+@InterfaceAudience.Private
+public class RaftClientProtocolClientSideTranslatorPB
+    extends Proxy<RaftClientProtocolPB>
+    implements RaftClientProtocol {
+
+  public RaftClientProtocolClientSideTranslatorPB(
+      String addressStr, Configuration conf) throws IOException {
+    super(RaftClientProtocolPB.class, addressStr, conf);
+  }
+
+  @Override
+  public RaftClientReply submitClientRequest(RaftClientRequest request)
+      throws IOException {
+    final RaftClientRequestProto p = ClientProtoUtils.toRaftClientRequestProto(request);
+    try {
+      final RaftClientReplyProto reply = getProtocol().submitClientRequest(null, p);
+      return ClientProtoUtils.toRaftClientReply(reply);
+    } catch (ServiceException se) {
+      throw ProtoUtils.toIOException(se);
+    }
+  }
+
+  @Override
+  public RaftClientReply setConfiguration(SetConfigurationRequest request)
+      throws IOException {
+    final SetConfigurationRequestProto p
+        = ClientProtoUtils.toSetConfigurationRequestProto(request);
+    try {
+      final RaftClientReplyProto reply = getProtocol().setConfiguration(null, p);
+      return ClientProtoUtils.toRaftClientReply(reply);
+    } catch (ServiceException se) {
+      throw ProtoUtils.toIOException(se);
+    }
+  }
+}

http://git-wip-us.apache.org/repos/asf/incubator-ratis/blob/7e71a2e0/ratis-hadoop/src/main/java/org/apache/ratis/hadooprpc/client/RaftClientProtocolPB.java
----------------------------------------------------------------------
diff --git a/ratis-hadoop/src/main/java/org/apache/ratis/hadooprpc/client/RaftClientProtocolPB.java b/ratis-hadoop/src/main/java/org/apache/ratis/hadooprpc/client/RaftClientProtocolPB.java
new file mode 100644
index 0000000..908cd99
--- /dev/null
+++ b/ratis-hadoop/src/main/java/org/apache/ratis/hadooprpc/client/RaftClientProtocolPB.java
@@ -0,0 +1,37 @@
+/**
+ * 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.ratis.hadooprpc.client;
+
+import org.apache.hadoop.classification.InterfaceAudience;
+import org.apache.hadoop.classification.InterfaceStability;
+import org.apache.hadoop.ipc.ProtocolInfo;
+import org.apache.hadoop.security.KerberosInfo;
+import org.apache.ratis.hadooprpc.HadoopConstants;
+import org.apache.ratis.shaded.proto.hadoop.HadoopProtos.RaftClientProtocolService;
+
+@InterfaceAudience.Private
+@InterfaceStability.Unstable
+@KerberosInfo(
+    serverPrincipal = HadoopConstants.RAFT_SERVER_KERBEROS_PRINCIPAL_KEY,
+    clientPrincipal = HadoopConstants.RAFT_CLIENT_KERBEROS_PRINCIPAL_KEY)
+@ProtocolInfo(
+    protocolName = HadoopConstants.RAFT_CLIENT_PROTOCOL_NAME,
+    protocolVersion = 1)
+public interface RaftClientProtocolPB extends
+    RaftClientProtocolService.BlockingInterface {
+}

http://git-wip-us.apache.org/repos/asf/incubator-ratis/blob/7e71a2e0/ratis-hadoop/src/main/java/org/apache/ratis/hadooprpc/client/RaftClientProtocolServerSideTranslatorPB.java
----------------------------------------------------------------------
diff --git a/ratis-hadoop/src/main/java/org/apache/ratis/hadooprpc/client/RaftClientProtocolServerSideTranslatorPB.java b/ratis-hadoop/src/main/java/org/apache/ratis/hadooprpc/client/RaftClientProtocolServerSideTranslatorPB.java
new file mode 100644
index 0000000..08cf589
--- /dev/null
+++ b/ratis-hadoop/src/main/java/org/apache/ratis/hadooprpc/client/RaftClientProtocolServerSideTranslatorPB.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.ratis.hadooprpc.client;
+
+import java.io.IOException;
+
+import org.apache.hadoop.classification.InterfaceAudience;
+import org.apache.ratis.client.impl.ClientProtoUtils;
+import org.apache.ratis.protocol.RaftClientProtocol;
+import org.apache.ratis.protocol.RaftClientReply;
+import org.apache.ratis.protocol.RaftClientRequest;
+import org.apache.ratis.protocol.SetConfigurationRequest;
+import org.apache.ratis.shaded.com.google.protobuf.RpcController;
+import org.apache.ratis.shaded.com.google.protobuf.ServiceException;
+import org.apache.ratis.shaded.proto.RaftProtos.RaftClientReplyProto;
+import org.apache.ratis.shaded.proto.RaftProtos.RaftClientRequestProto;
+import org.apache.ratis.shaded.proto.RaftProtos.SetConfigurationRequestProto;
+
+@InterfaceAudience.Private
+public class RaftClientProtocolServerSideTranslatorPB
+    implements RaftClientProtocolPB {
+  private final RaftClientProtocol impl;
+
+  public RaftClientProtocolServerSideTranslatorPB(RaftClientProtocol impl) {
+    this.impl = impl;
+  }
+
+  @Override
+  public RaftClientReplyProto submitClientRequest(
+      RpcController unused, RaftClientRequestProto proto)
+      throws ServiceException {
+    final RaftClientRequest request = ClientProtoUtils.toRaftClientRequest(proto);
+    try {
+      final RaftClientReply reply = impl.submitClientRequest(request);
+      return ClientProtoUtils.toRaftClientReplyProto(reply);
+    } catch(IOException ioe) {
+      throw new ServiceException(ioe);
+    }
+  }
+
+  @Override
+  public RaftClientReplyProto setConfiguration(
+      RpcController unused, SetConfigurationRequestProto proto)
+      throws ServiceException {
+    final SetConfigurationRequest request;
+    try {
+      request = ClientProtoUtils.toSetConfigurationRequest(proto);
+      final RaftClientReply reply = impl.setConfiguration(request);
+      return ClientProtoUtils.toRaftClientReplyProto(reply);
+    } catch(IOException ioe) {
+      throw new ServiceException(ioe);
+    }
+  }
+}

http://git-wip-us.apache.org/repos/asf/incubator-ratis/blob/7e71a2e0/ratis-hadoop/src/main/java/org/apache/ratis/hadooprpc/server/HadoopRpcService.java
----------------------------------------------------------------------
diff --git a/ratis-hadoop/src/main/java/org/apache/ratis/hadooprpc/server/HadoopRpcService.java b/ratis-hadoop/src/main/java/org/apache/ratis/hadooprpc/server/HadoopRpcService.java
new file mode 100644
index 0000000..b7ac64a
--- /dev/null
+++ b/ratis-hadoop/src/main/java/org/apache/ratis/hadooprpc/server/HadoopRpcService.java
@@ -0,0 +1,184 @@
+/**
+ * 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.ratis.hadooprpc.server;
+
+import java.io.IOException;
+import java.net.InetSocketAddress;
+
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.ipc.ProtobufRpcEngineShaded;
+import org.apache.hadoop.ipc.RPC;
+import org.apache.ratis.hadooprpc.Proxy;
+import org.apache.ratis.hadooprpc.client.RaftClientProtocolPB;
+import org.apache.ratis.hadooprpc.client.RaftClientProtocolServerSideTranslatorPB;
+import org.apache.ratis.protocol.RaftClientProtocol;
+import org.apache.ratis.protocol.RaftPeer;
+import org.apache.ratis.server.RaftServer;
+import org.apache.ratis.server.RaftServerConfigKeys;
+import org.apache.ratis.server.RaftServerRpc;
+import org.apache.ratis.server.protocol.RaftServerProtocol;
+import org.apache.ratis.shaded.com.google.protobuf.BlockingService;
+import org.apache.ratis.shaded.com.google.protobuf.ServiceException;
+import org.apache.ratis.shaded.proto.RaftProtos.AppendEntriesReplyProto;
+import org.apache.ratis.shaded.proto.RaftProtos.AppendEntriesRequestProto;
+import org.apache.ratis.shaded.proto.RaftProtos.InstallSnapshotReplyProto;
+import org.apache.ratis.shaded.proto.RaftProtos.InstallSnapshotRequestProto;
+import org.apache.ratis.shaded.proto.RaftProtos.RequestVoteReplyProto;
+import org.apache.ratis.shaded.proto.RaftProtos.RequestVoteRequestProto;
+import org.apache.ratis.shaded.proto.hadoop.HadoopProtos.RaftClientProtocolService;
+import org.apache.ratis.shaded.proto.hadoop.HadoopProtos.RaftServerProtocolService;
+import org.apache.ratis.util.CodeInjectionForTesting;
+import org.apache.ratis.util.PeerProxyMap;
+import org.apache.ratis.util.ProtoUtils;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import com.google.common.base.Preconditions;
+
+/** Server side Hadoop RPC service. */
+public class HadoopRpcService implements RaftServerRpc {
+  public static final Logger LOG = LoggerFactory.getLogger(HadoopRpcService.class);
+  static final String CLASS_NAME = HadoopRpcService.class.getSimpleName();
+  public static final String SEND_SERVER_REQUEST = CLASS_NAME + ".sendServerRequest";
+
+  private final String id;
+  private final RPC.Server ipcServer;
+  private final InetSocketAddress ipcServerAddress;
+
+  private final PeerProxyMap<Proxy<RaftServerProtocolPB>> proxies;
+
+  public HadoopRpcService(RaftServer server, final Configuration conf)
+      throws IOException {
+    this.proxies = new PeerProxyMap<>(
+        p -> new Proxy(RaftServerProtocolPB.class, p.getAddress(), conf));
+    this.id = server.getId();
+    this.ipcServer = newRpcServer(server, conf);
+    this.ipcServerAddress = ipcServer.getListenerAddress();
+
+    addRaftClientProtocol(server, conf);
+
+    LOG.info(getClass().getSimpleName() + " created RPC.Server at "
+        + ipcServerAddress);
+  }
+
+  @Override
+  public InetSocketAddress getInetSocketAddress() {
+    return ipcServerAddress;
+  }
+
+  private RPC.Server newRpcServer(RaftServerProtocol serverProtocol, final Configuration conf)
+      throws IOException {
+    final RaftServerConfigKeys.Get get = new RaftServerConfigKeys.Get() {
+      @Override
+      protected int getInt(String key, int defaultValue) {
+        return conf.getInt(key, defaultValue);
+      }
+
+      @Override
+      protected String getTrimmed(String key, String defaultValue) {
+        return conf.getTrimmed(key, defaultValue);
+      }
+    };
+
+    final int handlerCount = get.ipc().handlers();
+    final InetSocketAddress address = get.ipc().address();
+
+    final BlockingService service
+        = RaftServerProtocolService.newReflectiveBlockingService(
+            new RaftServerProtocolServerSideTranslatorPB(serverProtocol));
+    RPC.setProtocolEngine(conf, RaftServerProtocolPB.class, ProtobufRpcEngineShaded.class);
+    return new RPC.Builder(conf)
+        .setProtocol(RaftServerProtocolPB.class)
+        .setInstance(service)
+        .setBindAddress(address.getHostName())
+        .setPort(address.getPort())
+        .setNumHandlers(handlerCount)
+        .setVerbose(false)
+        .build();
+  }
+
+  private void addRaftClientProtocol(RaftClientProtocol clientProtocol, Configuration conf) {
+    final Class<?> protocol = RaftClientProtocolPB.class;
+    RPC.setProtocolEngine(conf,protocol, ProtobufRpcEngineShaded.class);
+
+    final BlockingService service
+        = RaftClientProtocolService.newReflectiveBlockingService(
+        new RaftClientProtocolServerSideTranslatorPB(clientProtocol));
+    ipcServer.addProtocol(RPC.RpcKind.RPC_PROTOCOL_BUFFER, protocol, service);
+  }
+
+  @Override
+  public void start() {
+    ipcServer.start();
+  }
+
+  @Override
+  public void close() {
+    ipcServer.stop();
+  }
+
+  @Override
+  public AppendEntriesReplyProto appendEntries(
+      AppendEntriesRequestProto request) throws IOException {
+    Preconditions.checkArgument(id.equals(request.getServerRequest().getRequestorId()));
+    CodeInjectionForTesting.execute(SEND_SERVER_REQUEST, id, null, request);
+
+    final RaftServerProtocolPB proxy = proxies.getProxy(
+        request.getServerRequest().getReplyId()).getProtocol();
+    try {
+      return proxy.appendEntries(null, request);
+    } catch (ServiceException se) {
+      throw ProtoUtils.toIOException(se);
+    }
+  }
+
+  @Override
+  public InstallSnapshotReplyProto installSnapshot(
+      InstallSnapshotRequestProto request) throws IOException {
+    Preconditions.checkArgument(id.equals(request.getServerRequest().getRequestorId()));
+    CodeInjectionForTesting.execute(SEND_SERVER_REQUEST, id, null, request);
+
+    final RaftServerProtocolPB proxy = proxies.getProxy(
+        request.getServerRequest().getReplyId()).getProtocol();
+    try {
+      return proxy.installSnapshot(null, request);
+    } catch (ServiceException se) {
+      throw ProtoUtils.toIOException(se);
+    }
+  }
+
+  @Override
+  public RequestVoteReplyProto requestVote(
+      RequestVoteRequestProto request) throws IOException {
+    Preconditions.checkArgument(id.equals(request.getServerRequest().getRequestorId()));
+    CodeInjectionForTesting.execute(SEND_SERVER_REQUEST, id, null, request);
+
+    final RaftServerProtocolPB proxy = proxies.getProxy(
+        request.getServerRequest().getReplyId()).getProtocol();
+    try {
+      return proxy.requestVote(null, request);
+    } catch (ServiceException se) {
+      throw ProtoUtils.toIOException(se);
+    }
+  }
+
+  @Override
+  public void addPeers(Iterable<RaftPeer> peers) {
+    proxies.addPeers(peers);
+  }
+}

http://git-wip-us.apache.org/repos/asf/incubator-ratis/blob/7e71a2e0/ratis-hadoop/src/main/java/org/apache/ratis/hadooprpc/server/RaftServerProtocolPB.java
----------------------------------------------------------------------
diff --git a/ratis-hadoop/src/main/java/org/apache/ratis/hadooprpc/server/RaftServerProtocolPB.java b/ratis-hadoop/src/main/java/org/apache/ratis/hadooprpc/server/RaftServerProtocolPB.java
new file mode 100644
index 0000000..8b92cc4
--- /dev/null
+++ b/ratis-hadoop/src/main/java/org/apache/ratis/hadooprpc/server/RaftServerProtocolPB.java
@@ -0,0 +1,37 @@
+/**
+ * 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.ratis.hadooprpc.server;
+
+import org.apache.hadoop.classification.InterfaceAudience;
+import org.apache.hadoop.classification.InterfaceStability;
+import org.apache.hadoop.ipc.ProtocolInfo;
+import org.apache.hadoop.security.KerberosInfo;
+import org.apache.ratis.hadooprpc.HadoopConstants;
+import org.apache.ratis.shaded.proto.hadoop.HadoopProtos.RaftServerProtocolService;
+
+@InterfaceAudience.Private
+@InterfaceStability.Unstable
+@KerberosInfo(
+    serverPrincipal = HadoopConstants.RAFT_SERVER_KERBEROS_PRINCIPAL_KEY,
+    clientPrincipal = HadoopConstants.RAFT_SERVER_KERBEROS_PRINCIPAL_KEY)
+@ProtocolInfo(
+    protocolName = HadoopConstants.RAFT_SERVER_PROTOCOL_NAME,
+    protocolVersion = 1)
+public interface RaftServerProtocolPB extends
+    RaftServerProtocolService.BlockingInterface {
+}

http://git-wip-us.apache.org/repos/asf/incubator-ratis/blob/7e71a2e0/ratis-hadoop/src/main/java/org/apache/ratis/hadooprpc/server/RaftServerProtocolServerSideTranslatorPB.java
----------------------------------------------------------------------
diff --git a/ratis-hadoop/src/main/java/org/apache/ratis/hadooprpc/server/RaftServerProtocolServerSideTranslatorPB.java b/ratis-hadoop/src/main/java/org/apache/ratis/hadooprpc/server/RaftServerProtocolServerSideTranslatorPB.java
new file mode 100644
index 0000000..a496793
--- /dev/null
+++ b/ratis-hadoop/src/main/java/org/apache/ratis/hadooprpc/server/RaftServerProtocolServerSideTranslatorPB.java
@@ -0,0 +1,73 @@
+/**
+ * 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.ratis.hadooprpc.server;
+
+import java.io.IOException;
+
+import org.apache.hadoop.classification.InterfaceAudience;
+import org.apache.ratis.server.protocol.RaftServerProtocol;
+import org.apache.ratis.shaded.com.google.protobuf.RpcController;
+import org.apache.ratis.shaded.com.google.protobuf.ServiceException;
+import org.apache.ratis.shaded.proto.RaftProtos.AppendEntriesReplyProto;
+import org.apache.ratis.shaded.proto.RaftProtos.AppendEntriesRequestProto;
+import org.apache.ratis.shaded.proto.RaftProtos.InstallSnapshotReplyProto;
+import org.apache.ratis.shaded.proto.RaftProtos.InstallSnapshotRequestProto;
+import org.apache.ratis.shaded.proto.RaftProtos.RequestVoteReplyProto;
+import org.apache.ratis.shaded.proto.RaftProtos.RequestVoteRequestProto;
+
+@InterfaceAudience.Private
+public class RaftServerProtocolServerSideTranslatorPB
+    implements RaftServerProtocolPB {
+  private final RaftServerProtocol impl;
+
+  public RaftServerProtocolServerSideTranslatorPB(RaftServerProtocol impl) {
+    this.impl = impl;
+  }
+
+  @Override
+  public RequestVoteReplyProto requestVote(
+      RpcController unused, RequestVoteRequestProto request)
+      throws ServiceException {
+    try {
+      return impl.requestVote(request);
+    } catch(IOException ioe) {
+      throw new ServiceException(ioe);
+    }
+  }
+
+  @Override
+  public AppendEntriesReplyProto appendEntries(
+      RpcController unused, AppendEntriesRequestProto request)
+      throws ServiceException {
+    try {
+      return impl.appendEntries(request);
+    } catch(IOException ioe) {
+      throw new ServiceException(ioe);
+    }
+  }
+
+  @Override
+  public InstallSnapshotReplyProto installSnapshot(RpcController controller,
+      InstallSnapshotRequestProto request) throws ServiceException {
+    try {
+      return impl.installSnapshot(request);
+    } catch(IOException ioe) {
+      throw new ServiceException(ioe);
+    }
+  }
+}

http://git-wip-us.apache.org/repos/asf/incubator-ratis/blob/7e71a2e0/ratis-hadoop/src/test/java/org/apache/ratis/hadooprpc/MiniRaftClusterWithHadoopRpc.java
----------------------------------------------------------------------
diff --git a/ratis-hadoop/src/test/java/org/apache/ratis/hadooprpc/MiniRaftClusterWithHadoopRpc.java b/ratis-hadoop/src/test/java/org/apache/ratis/hadooprpc/MiniRaftClusterWithHadoopRpc.java
new file mode 100644
index 0000000..964f3a2
--- /dev/null
+++ b/ratis-hadoop/src/test/java/org/apache/ratis/hadooprpc/MiniRaftClusterWithHadoopRpc.java
@@ -0,0 +1,117 @@
+/**
+ * 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.ratis.hadooprpc;
+
+import com.google.common.base.Preconditions;
+import org.apache.hadoop.conf.Configuration;
+import org.apache.ratis.MiniRaftCluster;
+import org.apache.ratis.RaftTestUtil;
+import org.apache.ratis.client.RaftClientRequestSender;
+import org.apache.ratis.conf.RaftProperties;
+import org.apache.ratis.hadooprpc.client.HadoopClientRequestSender;
+import org.apache.ratis.hadooprpc.server.HadoopRpcService;
+import org.apache.ratis.protocol.RaftPeer;
+import org.apache.ratis.server.RaftServerConfigKeys;
+import org.apache.ratis.server.impl.DelayLocalExecutionInjection;
+import org.apache.ratis.server.impl.RaftServerImpl;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.io.IOException;
+import java.util.Collection;
+import java.util.HashMap;
+import java.util.Map;
+
+public class MiniRaftClusterWithHadoopRpc extends MiniRaftCluster.RpcBase {
+  static final Logger LOG = LoggerFactory.getLogger(MiniRaftClusterWithHadoopRpc.class);
+
+  public static final Factory<MiniRaftClusterWithHadoopRpc> FACTORY
+      = new Factory<MiniRaftClusterWithHadoopRpc>() {
+    @Override
+    public MiniRaftClusterWithHadoopRpc newCluster(
+        String[] ids, RaftProperties prop, boolean formatted) throws IOException {
+      final Configuration conf = new Configuration();
+      conf.set(RaftServerConfigKeys.Ipc.ADDRESS_KEY, "0.0.0.0:0");
+      return new MiniRaftClusterWithHadoopRpc(ids, prop, conf, formatted);
+    }
+  };
+
+  public static final DelayLocalExecutionInjection sendServerRequest =
+      new DelayLocalExecutionInjection(HadoopRpcService.SEND_SERVER_REQUEST);
+
+  private final Configuration hadoopConf;
+
+  public MiniRaftClusterWithHadoopRpc(int numServers, RaftProperties properties,
+      Configuration conf) throws IOException {
+    this(generateIds(numServers, 0), properties, conf, true);
+  }
+
+  public MiniRaftClusterWithHadoopRpc(String[] ids, RaftProperties properties,
+      Configuration hadoopConf, boolean formatted) throws IOException {
+    super(ids, properties, formatted);
+    this.hadoopConf = hadoopConf;
+
+    init(initRpcServices(getServers(), hadoopConf));
+  }
+
+  private static Map<RaftPeer, HadoopRpcService> initRpcServices(
+      Collection<RaftServerImpl> servers, Configuration hadoopConf) throws IOException {
+    final Map<RaftPeer, HadoopRpcService> peerRpcs = new HashMap<>();
+
+    for(RaftServerImpl s : servers) {
+      final HadoopRpcService rpc = new HadoopRpcService(s, hadoopConf);
+      peerRpcs.put(new RaftPeer(s.getId(), rpc.getInetSocketAddress()), rpc);
+    }
+    return peerRpcs;
+  }
+
+  @Override
+  protected RaftServerImpl setPeerRpc(RaftPeer peer) throws IOException {
+    Configuration hconf = new Configuration(hadoopConf);
+    hconf.set(RaftServerConfigKeys.Ipc.ADDRESS_KEY, peer.getAddress());
+
+    RaftServerImpl server = servers.get(peer.getId());
+    final HadoopRpcService rpc = new HadoopRpcService(server, hconf);
+    Preconditions.checkState(
+        rpc.getInetSocketAddress().toString().contains(peer.getAddress()),
+        "address in the raft conf: %s, address in rpc server: %s",
+        peer.getAddress(), rpc.getInetSocketAddress().toString());
+    server.setServerRpc(rpc);
+    return server;
+  }
+
+  @Override
+  public Collection<RaftPeer> addNewPeers(Collection<RaftPeer> newPeers,
+                                          Collection<RaftServerImpl> newServers, boolean startService)
+      throws IOException {
+    return addNewPeers(initRpcServices(newServers, hadoopConf),
+        newServers, startService);
+  }
+
+  @Override
+  public RaftClientRequestSender getRaftClientRequestSender() {
+    return new HadoopClientRequestSender(getPeers(), hadoopConf);
+  }
+
+  @Override
+  public void blockQueueAndSetDelay(String leaderId, int delayMs)
+      throws InterruptedException {
+    RaftTestUtil.blockQueueAndSetDelay(getServers(), sendServerRequest,
+        leaderId, delayMs, getMaxTimeout());
+  }
+}

http://git-wip-us.apache.org/repos/asf/incubator-ratis/blob/7e71a2e0/ratis-hadoop/src/test/java/org/apache/ratis/hadooprpc/TestNotLeaderExceptionWithHadoopRpc.java
----------------------------------------------------------------------
diff --git a/ratis-hadoop/src/test/java/org/apache/ratis/hadooprpc/TestNotLeaderExceptionWithHadoopRpc.java b/ratis-hadoop/src/test/java/org/apache/ratis/hadooprpc/TestNotLeaderExceptionWithHadoopRpc.java
new file mode 100644
index 0000000..8bc5ae6
--- /dev/null
+++ b/ratis-hadoop/src/test/java/org/apache/ratis/hadooprpc/TestNotLeaderExceptionWithHadoopRpc.java
@@ -0,0 +1,37 @@
+/**
+ * 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.ratis.hadooprpc;
+
+import org.apache.hadoop.conf.Configuration;
+import org.apache.ratis.MiniRaftCluster;
+import org.apache.ratis.RaftNotLeaderExceptionBaseTest;
+import org.apache.ratis.conf.RaftProperties;
+import org.apache.ratis.server.RaftServerConfigKeys;
+
+import java.io.IOException;
+
+public class TestNotLeaderExceptionWithHadoopRpc extends RaftNotLeaderExceptionBaseTest {
+  @Override
+  public MiniRaftCluster initCluster() throws IOException {
+    String[] s = MiniRaftCluster.generateIds(NUM_PEERS, 0);
+    final Configuration conf = new Configuration();
+    conf.set(RaftServerConfigKeys.Ipc.ADDRESS_KEY, "0.0.0.0:0");
+    RaftProperties prop = new RaftProperties();
+    return new MiniRaftClusterWithHadoopRpc(s, prop, conf, true);
+  }
+}

http://git-wip-us.apache.org/repos/asf/incubator-ratis/blob/7e71a2e0/ratis-hadoop/src/test/java/org/apache/ratis/hadooprpc/TestRaftReconfigurationWithHadoopRpc.java
----------------------------------------------------------------------
diff --git a/ratis-hadoop/src/test/java/org/apache/ratis/hadooprpc/TestRaftReconfigurationWithHadoopRpc.java b/ratis-hadoop/src/test/java/org/apache/ratis/hadooprpc/TestRaftReconfigurationWithHadoopRpc.java
new file mode 100644
index 0000000..cd502ac
--- /dev/null
+++ b/ratis-hadoop/src/test/java/org/apache/ratis/hadooprpc/TestRaftReconfigurationWithHadoopRpc.java
@@ -0,0 +1,40 @@
+/**
+ * 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.ratis.hadooprpc;
+
+import org.apache.hadoop.conf.Configuration;
+import org.apache.ratis.MiniRaftCluster;
+import org.apache.ratis.server.RaftServerConfigKeys;
+import org.apache.ratis.server.impl.RaftReconfigurationBaseTest;
+
+import java.io.IOException;
+
+import static org.apache.hadoop.fs.CommonConfigurationKeysPublic.IPC_CLIENT_CONNECT_MAX_RETRIES_KEY;
+import static org.apache.hadoop.fs.CommonConfigurationKeysPublic.IPC_CLIENT_CONNECT_TIMEOUT_KEY;
+
+public class TestRaftReconfigurationWithHadoopRpc
+    extends RaftReconfigurationBaseTest {
+  @Override
+  public MiniRaftCluster getCluster(int peerNum) throws IOException {
+    final Configuration hadoopConf = new Configuration();
+    hadoopConf.setInt(IPC_CLIENT_CONNECT_TIMEOUT_KEY, 1000);
+    hadoopConf.setInt(IPC_CLIENT_CONNECT_MAX_RETRIES_KEY, 0);
+    hadoopConf.set(RaftServerConfigKeys.Ipc.ADDRESS_KEY, "0.0.0.0:0");
+    return new MiniRaftClusterWithHadoopRpc(peerNum, prop, hadoopConf);
+  }
+}

http://git-wip-us.apache.org/repos/asf/incubator-ratis/blob/7e71a2e0/ratis-hadoop/src/test/java/org/apache/ratis/hadooprpc/TestRaftSnapshotWithHadoopRpc.java
----------------------------------------------------------------------
diff --git a/ratis-hadoop/src/test/java/org/apache/ratis/hadooprpc/TestRaftSnapshotWithHadoopRpc.java b/ratis-hadoop/src/test/java/org/apache/ratis/hadooprpc/TestRaftSnapshotWithHadoopRpc.java
new file mode 100644
index 0000000..7bdafb0
--- /dev/null
+++ b/ratis-hadoop/src/test/java/org/apache/ratis/hadooprpc/TestRaftSnapshotWithHadoopRpc.java
@@ -0,0 +1,32 @@
+/**
+ * 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.ratis.hadooprpc;
+
+import org.apache.ratis.MiniRaftCluster;
+import org.apache.ratis.conf.RaftProperties;
+import org.apache.ratis.statemachine.RaftSnapshotBaseTest;
+
+import java.io.IOException;
+
+public class TestRaftSnapshotWithHadoopRpc extends RaftSnapshotBaseTest {
+  @Override
+  public MiniRaftCluster initCluster(int numServer, RaftProperties prop)
+      throws IOException {
+    return MiniRaftClusterWithHadoopRpc.FACTORY.newCluster(numServer, prop, true);
+  }
+}

http://git-wip-us.apache.org/repos/asf/incubator-ratis/blob/7e71a2e0/ratis-hadoop/src/test/java/org/apache/ratis/hadooprpc/TestRaftWithHadoopRpc.java
----------------------------------------------------------------------
diff --git a/ratis-hadoop/src/test/java/org/apache/ratis/hadooprpc/TestRaftWithHadoopRpc.java b/ratis-hadoop/src/test/java/org/apache/ratis/hadooprpc/TestRaftWithHadoopRpc.java
new file mode 100644
index 0000000..9dd3f27
--- /dev/null
+++ b/ratis-hadoop/src/test/java/org/apache/ratis/hadooprpc/TestRaftWithHadoopRpc.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.ratis.hadooprpc;
+
+import org.apache.hadoop.conf.Configuration;
+import org.apache.log4j.Level;
+import org.apache.ratis.RaftBasicTests;
+import org.apache.ratis.client.RaftClient;
+import org.apache.ratis.server.RaftServerConfigKeys;
+import org.apache.ratis.server.impl.BlockRequestHandlingInjection;
+import org.apache.ratis.server.impl.RaftServerImpl;
+import org.apache.ratis.util.RaftUtils;
+import org.junit.Test;
+
+import static org.apache.ratis.hadooprpc.MiniRaftClusterWithHadoopRpc.sendServerRequest;
+
+import java.io.IOException;
+
+public class TestRaftWithHadoopRpc extends RaftBasicTests {
+  static {
+    RaftUtils.setLogLevel(RaftServerImpl.LOG, Level.DEBUG);
+    RaftUtils.setLogLevel(RaftClient.LOG, Level.DEBUG);
+    RaftUtils.setLogLevel(MiniRaftClusterWithHadoopRpc.LOG, Level.DEBUG);
+  }
+
+  private final MiniRaftClusterWithHadoopRpc cluster;
+
+  public TestRaftWithHadoopRpc() throws IOException {
+    Configuration conf = new Configuration();
+    conf.set(RaftServerConfigKeys.Ipc.ADDRESS_KEY, "0.0.0.0:0");
+    cluster = new MiniRaftClusterWithHadoopRpc(NUM_SERVERS, getProperties(), conf);
+  }
+
+  @Override
+  public MiniRaftClusterWithHadoopRpc getCluster() {
+    return cluster;
+  }
+
+  @Override
+  @Test
+  public void testEnforceLeader() throws Exception {
+    super.testEnforceLeader();
+
+    sendServerRequest.clear();
+    BlockRequestHandlingInjection.getInstance().unblockAll();
+  }
+
+  @Override
+  @Test
+  public void testWithLoad() throws Exception {
+    super.testWithLoad();
+    BlockRequestHandlingInjection.getInstance().unblockAll();
+  }
+}

http://git-wip-us.apache.org/repos/asf/incubator-ratis/blob/7e71a2e0/ratis-hadoop/src/test/resources/log4j.properties
----------------------------------------------------------------------
diff --git a/ratis-hadoop/src/test/resources/log4j.properties b/ratis-hadoop/src/test/resources/log4j.properties
new file mode 100644
index 0000000..ced0687
--- /dev/null
+++ b/ratis-hadoop/src/test/resources/log4j.properties
@@ -0,0 +1,18 @@
+#   Licensed 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.
+# log4j configuration used during build and unit tests
+
+log4j.rootLogger=info,stdout
+log4j.threshold=ALL
+log4j.appender.stdout=org.apache.log4j.ConsoleAppender
+log4j.appender.stdout.layout=org.apache.log4j.PatternLayout
+log4j.appender.stdout.layout.ConversionPattern=%d{ISO8601} %-5p %c{2} (%F:%M(%L)) - %m%n

http://git-wip-us.apache.org/repos/asf/incubator-ratis/blob/7e71a2e0/ratis-netty/pom.xml
----------------------------------------------------------------------
diff --git a/ratis-netty/pom.xml b/ratis-netty/pom.xml
new file mode 100644
index 0000000..cdffdc2
--- /dev/null
+++ b/ratis-netty/pom.xml
@@ -0,0 +1,99 @@
+<?xml version="1.0" encoding="UTF-8"?>
+<!--
+  Licensed 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. See accompanying LICENSE file.
+-->
+<project xmlns="http://maven.apache.org/POM/4.0.0"
+         xmlns:xsi="http://www.w3.org/2001/XMLSchema-instance"
+         xsi:schemaLocation="http://maven.apache.org/POM/4.0.0 http://maven.apache.org/xsd/maven-4.0.0.xsd">
+  <modelVersion>4.0.0</modelVersion>
+  <parent>
+    <artifactId>ratis-project-dist</artifactId>
+    <groupId>org.apache.ratis</groupId>
+    <version>1.0-SNAPSHOT</version>
+    <relativePath>../ratis-project-dist</relativePath>
+  </parent>
+
+  <artifactId>ratis-netty</artifactId>
+  <name>Ratis Netty Support</name>
+
+  <dependencies>
+    <dependency>
+      <artifactId>ratis-proto-shaded</artifactId>
+      <groupId>org.apache.ratis</groupId>
+      <scope>provided</scope>
+    </dependency>
+
+    <dependency>
+      <artifactId>ratis-common</artifactId>
+      <groupId>org.apache.ratis</groupId>
+      <scope>provided</scope>
+    </dependency>
+    <dependency>
+      <artifactId>ratis-common</artifactId>
+      <groupId>org.apache.ratis</groupId>
+      <scope>test</scope>
+      <type>test-jar</type>
+    </dependency>
+    
+    <dependency>
+      <artifactId>ratis-client</artifactId>
+      <groupId>org.apache.ratis</groupId>
+      <scope>provided</scope>
+    </dependency>
+    <dependency>
+      <artifactId>ratis-client</artifactId>
+      <groupId>org.apache.ratis</groupId>
+      <scope>test</scope>
+      <type>test-jar</type>
+    </dependency>
+    
+    <dependency>
+      <artifactId>ratis-server</artifactId>
+      <groupId>org.apache.ratis</groupId>
+      <scope>provided</scope>
+    </dependency>
+    <dependency>
+      <artifactId>ratis-server</artifactId>
+      <groupId>org.apache.ratis</groupId>
+      <scope>test</scope>
+      <type>test-jar</type>
+    </dependency>
+    
+    <dependency>
+      <groupId>org.slf4j</groupId>
+      <artifactId>slf4j-api</artifactId>
+    </dependency>
+
+    <dependency>
+      <groupId>com.google.guava</groupId>
+      <artifactId>guava</artifactId>
+    </dependency>
+
+    <dependency>
+      <groupId>junit</groupId>
+      <artifactId>junit</artifactId>
+      <scope>test</scope>
+    </dependency>
+    <dependency>
+      <groupId>org.mockito</groupId>
+      <artifactId>mockito-all</artifactId>
+      <scope>test</scope>
+    </dependency>
+
+    <dependency>
+      <groupId>io.netty</groupId>
+      <artifactId>netty-all</artifactId>
+      <scope>compile</scope>
+    </dependency>
+  </dependencies>
+</project>

http://git-wip-us.apache.org/repos/asf/incubator-ratis/blob/7e71a2e0/ratis-netty/src/main/java/org/apache/ratis/netty/NettyClient.java
----------------------------------------------------------------------
diff --git a/ratis-netty/src/main/java/org/apache/ratis/netty/NettyClient.java b/ratis-netty/src/main/java/org/apache/ratis/netty/NettyClient.java
new file mode 100644
index 0000000..1aea87b
--- /dev/null
+++ b/ratis-netty/src/main/java/org/apache/ratis/netty/NettyClient.java
@@ -0,0 +1,73 @@
+/*
+ * 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.ratis.netty;
+
+import com.google.common.base.Preconditions;
+import io.netty.bootstrap.Bootstrap;
+import io.netty.channel.Channel;
+import io.netty.channel.ChannelFuture;
+import io.netty.channel.ChannelInitializer;
+import io.netty.channel.EventLoopGroup;
+import io.netty.channel.socket.SocketChannel;
+import io.netty.channel.socket.nio.NioSocketChannel;
+import io.netty.handler.logging.LogLevel;
+import io.netty.handler.logging.LoggingHandler;
+
+import java.io.Closeable;
+import java.net.InetSocketAddress;
+
+import org.apache.ratis.util.LifeCycle;
+import org.apache.ratis.util.NetUtils;
+
+public class NettyClient implements Closeable {
+  private final LifeCycle lifeCycle = new LifeCycle(getClass().getSimpleName());
+
+  private Channel channel;
+
+  /** Connects to the given server address. */
+  public void connect(String serverAddress, EventLoopGroup group,
+                      ChannelInitializer<SocketChannel> initializer)
+      throws InterruptedException {
+    final InetSocketAddress address = NetUtils.newInetSocketAddress(serverAddress);
+    Preconditions.checkNotNull(address,
+        "Failed to create InetSocketAddress from %s.", serverAddress);
+
+    lifeCycle.startAndTransition(
+        () -> channel = new Bootstrap()
+            .group(group)
+            .channel(NioSocketChannel.class)
+            .handler(new LoggingHandler(LogLevel.INFO))
+            .handler(initializer)
+            .connect(address)
+            .sync()
+            .channel(),
+        InterruptedException.class);
+  }
+
+  @Override
+  public void close() {
+    lifeCycle.checkStateAndClose(() -> {
+      channel.close().syncUninterruptibly();
+    });
+  }
+
+  public ChannelFuture writeAndFlush(Object msg) {
+    lifeCycle.assertCurrentState(LifeCycle.State.RUNNING);
+    return channel.writeAndFlush(msg);
+  }
+}

http://git-wip-us.apache.org/repos/asf/incubator-ratis/blob/7e71a2e0/ratis-netty/src/main/java/org/apache/ratis/netty/NettyRpcProxy.java
----------------------------------------------------------------------
diff --git a/ratis-netty/src/main/java/org/apache/ratis/netty/NettyRpcProxy.java b/ratis-netty/src/main/java/org/apache/ratis/netty/NettyRpcProxy.java
new file mode 100644
index 0000000..9b8553b
--- /dev/null
+++ b/ratis-netty/src/main/java/org/apache/ratis/netty/NettyRpcProxy.java
@@ -0,0 +1,186 @@
+/*
+ * 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.ratis.netty;
+
+import static org.apache.ratis.shaded.proto.netty.NettyProtos.RaftNettyServerReplyProto.RaftNettyServerReplyCase.EXCEPTIONREPLY;
+
+import java.io.Closeable;
+import java.io.IOException;
+import java.util.LinkedList;
+import java.util.Queue;
+import java.util.concurrent.CompletableFuture;
+import java.util.concurrent.ExecutionException;
+
+import org.apache.ratis.protocol.RaftPeer;
+import org.apache.ratis.shaded.io.netty.handler.codec.protobuf.ProtobufDecoder;
+import org.apache.ratis.shaded.io.netty.handler.codec.protobuf.ProtobufEncoder;
+import org.apache.ratis.shaded.io.netty.handler.codec.protobuf.ProtobufVarint32FrameDecoder;
+import org.apache.ratis.shaded.io.netty.handler.codec.protobuf.ProtobufVarint32LengthFieldPrepender;
+import org.apache.ratis.shaded.proto.RaftProtos.RaftRpcRequestProto;
+import org.apache.ratis.shaded.proto.netty.NettyProtos.RaftNettyServerReplyProto;
+import org.apache.ratis.shaded.proto.netty.NettyProtos.RaftNettyServerRequestProto;
+import org.apache.ratis.util.PeerProxyMap;
+import org.apache.ratis.util.ProtoUtils;
+import org.apache.ratis.util.RaftUtils;
+
+import io.netty.channel.ChannelFuture;
+import io.netty.channel.ChannelHandlerContext;
+import io.netty.channel.ChannelInboundHandler;
+import io.netty.channel.ChannelInitializer;
+import io.netty.channel.ChannelPipeline;
+import io.netty.channel.EventLoopGroup;
+import io.netty.channel.SimpleChannelInboundHandler;
+import io.netty.channel.nio.NioEventLoopGroup;
+import io.netty.channel.socket.SocketChannel;
+
+public class NettyRpcProxy implements Closeable {
+  public static class PeerMap extends PeerProxyMap<NettyRpcProxy> {
+    private final EventLoopGroup group = new NioEventLoopGroup();
+
+    @Override
+    public NettyRpcProxy createProxyImpl(RaftPeer peer)
+        throws IOException {
+      try {
+        return new NettyRpcProxy(peer, group);
+      } catch (InterruptedException e) {
+        throw RaftUtils.toInterruptedIOException("Failed connecting to " + peer, e);
+      }
+    }
+
+    @Override
+    public void close() {
+      super.close();
+      group.shutdownGracefully();
+    }
+  }
+
+  public static long getSeqNum(RaftNettyServerReplyProto proto) {
+    switch (proto.getRaftNettyServerReplyCase()) {
+      case REQUESTVOTEREPLY:
+        return proto.getRequestVoteReply().getServerReply().getSeqNum();
+      case APPENDENTRIESREPLY:
+        return proto.getAppendEntriesReply().getServerReply().getSeqNum();
+      case INSTALLSNAPSHOTREPLY:
+        return proto.getInstallSnapshotReply().getServerReply().getSeqNum();
+      case RAFTCLIENTREPLY:
+        return proto.getRaftClientReply().getRpcReply().getSeqNum();
+      case EXCEPTIONREPLY:
+        return proto.getExceptionReply().getRpcReply().getSeqNum();
+      case RAFTNETTYSERVERREPLY_NOT_SET:
+        throw new IllegalArgumentException("Reply case not set in proto: "
+            + proto.getRaftNettyServerReplyCase());
+      default:
+        throw new UnsupportedOperationException("Reply case not supported: "
+            + proto.getRaftNettyServerReplyCase());
+    }
+  }
+
+
+  class Connection implements Closeable {
+    private final NettyClient client = new NettyClient();
+    private final Queue<CompletableFuture<RaftNettyServerReplyProto>> replies
+        = new LinkedList<>();
+
+    Connection(EventLoopGroup group) throws InterruptedException {
+      final ChannelInboundHandler inboundHandler
+          = new SimpleChannelInboundHandler<RaftNettyServerReplyProto>() {
+        @Override
+        protected void channelRead0(ChannelHandlerContext ctx,
+                                    RaftNettyServerReplyProto proto) {
+          final CompletableFuture<RaftNettyServerReplyProto> future = pollReply();
+          if (future == null) {
+            throw new IllegalStateException("Request #" + getSeqNum(proto)
+                + " not found");
+          }
+          if (proto.getRaftNettyServerReplyCase() == EXCEPTIONREPLY) {
+            final Object ioe = ProtoUtils.toObject(proto.getExceptionReply().getException());
+            future.completeExceptionally((IOException)ioe);
+          } else {
+            future.complete(proto);
+          }
+        }
+      };
+      final ChannelInitializer<SocketChannel> initializer
+          = new ChannelInitializer<SocketChannel>() {
+        @Override
+        protected void initChannel(SocketChannel ch) throws Exception {
+          final ChannelPipeline p = ch.pipeline();
+
+          p.addLast(new ProtobufVarint32FrameDecoder());
+          p.addLast(new ProtobufDecoder(RaftNettyServerReplyProto.getDefaultInstance()));
+          p.addLast(new ProtobufVarint32LengthFieldPrepender());
+          p.addLast(new ProtobufEncoder());
+
+          p.addLast(inboundHandler);
+        }
+      };
+
+      client.connect(peer.getAddress(), group, initializer);
+    }
+
+    synchronized ChannelFuture offer(RaftNettyServerRequestProto request,
+        CompletableFuture<RaftNettyServerReplyProto> reply) {
+      replies.offer(reply);
+      return client.writeAndFlush(request);
+    }
+
+    synchronized CompletableFuture<RaftNettyServerReplyProto> pollReply() {
+      return replies.poll();
+    }
+
+    @Override
+    public synchronized void close() {
+      client.close();
+      if (!replies.isEmpty()) {
+        final IOException e = new IOException("Connection to " + peer + " is closed.");
+        replies.stream().forEach(f -> f.completeExceptionally(e));
+        replies.clear();
+      }
+    }
+  }
+
+  private final RaftPeer peer;
+  private final Connection connection;
+
+  public NettyRpcProxy(RaftPeer peer, EventLoopGroup group) throws InterruptedException {
+    this.peer = peer;
+    this.connection = new Connection(group);
+  }
+
+  @Override
+  public void close() {
+    connection.close();
+  }
+
+  public RaftNettyServerReplyProto send(
+      RaftRpcRequestProto request, RaftNettyServerRequestProto proto)
+      throws IOException {
+    final CompletableFuture<RaftNettyServerReplyProto> reply = new CompletableFuture<>();
+    final ChannelFuture channelFuture = connection.offer(proto, reply);
+
+    try {
+      channelFuture.sync();
+      return reply.get();
+    } catch (InterruptedException e) {
+      throw RaftUtils.toInterruptedIOException(ProtoUtils.toString(request)
+          + " sending from " + peer + " is interrupted.", e);
+    } catch (ExecutionException e) {
+      throw RaftUtils.toIOException(e);
+    }
+  }
+}

http://git-wip-us.apache.org/repos/asf/incubator-ratis/blob/7e71a2e0/ratis-netty/src/main/java/org/apache/ratis/netty/client/NettyClientRequestSender.java
----------------------------------------------------------------------
diff --git a/ratis-netty/src/main/java/org/apache/ratis/netty/client/NettyClientRequestSender.java b/ratis-netty/src/main/java/org/apache/ratis/netty/client/NettyClientRequestSender.java
new file mode 100644
index 0000000..38d806b
--- /dev/null
+++ b/ratis-netty/src/main/java/org/apache/ratis/netty/client/NettyClientRequestSender.java
@@ -0,0 +1,71 @@
+/*
+ * 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.ratis.netty.client;
+
+import org.apache.ratis.shaded.proto.RaftProtos.RaftClientRequestProto;
+import org.apache.ratis.shaded.proto.RaftProtos.RaftRpcRequestProto;
+import org.apache.ratis.shaded.proto.RaftProtos.SetConfigurationRequestProto;
+import org.apache.ratis.shaded.proto.netty.NettyProtos.RaftNettyServerRequestProto;
+import org.apache.ratis.client.RaftClientRequestSender;
+import org.apache.ratis.client.impl.ClientProtoUtils;
+import org.apache.ratis.netty.NettyRpcProxy;
+import org.apache.ratis.protocol.RaftClientReply;
+import org.apache.ratis.protocol.RaftClientRequest;
+import org.apache.ratis.protocol.RaftPeer;
+import org.apache.ratis.protocol.SetConfigurationRequest;
+
+import java.io.IOException;
+
+public class NettyClientRequestSender implements RaftClientRequestSender {
+  private final NettyRpcProxy.PeerMap proxies = new NettyRpcProxy.PeerMap();
+
+  public NettyClientRequestSender(Iterable<RaftPeer> servers) {
+    addServers(servers);
+  }
+
+  @Override
+  public RaftClientReply sendRequest(RaftClientRequest request) throws IOException {
+    final String serverId = request.getReplierId();
+    final NettyRpcProxy proxy = proxies.getProxy(serverId);
+
+    final RaftNettyServerRequestProto.Builder b = RaftNettyServerRequestProto.newBuilder();
+    final RaftRpcRequestProto rpcRequest;
+    if (request instanceof SetConfigurationRequest) {
+      final SetConfigurationRequestProto proto = ClientProtoUtils.toSetConfigurationRequestProto(
+          (SetConfigurationRequest)request);
+      b.setSetConfigurationRequest(proto);
+      rpcRequest = proto.getRpcRequest();
+    } else {
+      final RaftClientRequestProto proto = ClientProtoUtils.toRaftClientRequestProto(request);
+      b.setRaftClientRequest(proto);
+      rpcRequest = proto.getRpcRequest();
+    }
+    return ClientProtoUtils.toRaftClientReply(
+        proxy.send(rpcRequest, b.build()).getRaftClientReply());
+  }
+
+  @Override
+  public void addServers(Iterable<RaftPeer> servers) {
+    proxies.addPeers(servers);
+  }
+
+  @Override
+  public void close() {
+    proxies.close();
+  }
+}

http://git-wip-us.apache.org/repos/asf/incubator-ratis/blob/7e71a2e0/ratis-netty/src/main/java/org/apache/ratis/netty/server/NettyRpcService.java
----------------------------------------------------------------------
diff --git a/ratis-netty/src/main/java/org/apache/ratis/netty/server/NettyRpcService.java b/ratis-netty/src/main/java/org/apache/ratis/netty/server/NettyRpcService.java
new file mode 100644
index 0000000..153f61e
--- /dev/null
+++ b/ratis-netty/src/main/java/org/apache/ratis/netty/server/NettyRpcService.java
@@ -0,0 +1,254 @@
+/*
+ * 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.ratis.netty.server;
+
+import com.google.common.base.Preconditions;
+import io.netty.bootstrap.ServerBootstrap;
+import io.netty.channel.*;
+import io.netty.channel.nio.NioEventLoopGroup;
+import io.netty.channel.socket.SocketChannel;
+import io.netty.channel.socket.nio.NioServerSocketChannel;
+import io.netty.handler.logging.LogLevel;
+import io.netty.handler.logging.LoggingHandler;
+
+import org.apache.ratis.shaded.io.netty.handler.codec.protobuf.ProtobufDecoder;
+import org.apache.ratis.shaded.io.netty.handler.codec.protobuf.ProtobufEncoder;
+import org.apache.ratis.shaded.io.netty.handler.codec.protobuf.ProtobufVarint32FrameDecoder;
+import org.apache.ratis.shaded.io.netty.handler.codec.protobuf.ProtobufVarint32LengthFieldPrepender;
+import org.apache.ratis.shaded.proto.RaftProtos.*;
+import org.apache.ratis.shaded.proto.netty.NettyProtos.RaftNettyExceptionReplyProto;
+import org.apache.ratis.shaded.proto.netty.NettyProtos.RaftNettyServerReplyProto;
+import org.apache.ratis.shaded.proto.netty.NettyProtos.RaftNettyServerRequestProto;
+import org.apache.ratis.client.impl.ClientProtoUtils;
+import org.apache.ratis.netty.NettyRpcProxy;
+import org.apache.ratis.protocol.RaftClientReply;
+import org.apache.ratis.protocol.RaftPeer;
+import org.apache.ratis.server.RaftServer;
+import org.apache.ratis.server.RaftServerRpc;
+import org.apache.ratis.util.CodeInjectionForTesting;
+import org.apache.ratis.util.LifeCycle;
+import org.apache.ratis.util.ProtoUtils;
+
+import java.io.IOException;
+import java.net.InetSocketAddress;
+import java.nio.channels.ClosedChannelException;
+
+/**
+ * A netty server endpoint that acts as the communication layer.
+ */
+public final class NettyRpcService implements RaftServerRpc {
+  static final String CLASS_NAME = NettyRpcService.class.getSimpleName();
+  public static final String SEND_SERVER_REQUEST = CLASS_NAME + ".sendServerRequest";
+
+  private final LifeCycle lifeCycle = new LifeCycle(getClass().getSimpleName());
+  private final RaftServer server;
+  private final String id;
+
+  private final EventLoopGroup bossGroup = new NioEventLoopGroup();
+  private final EventLoopGroup workerGroup = new NioEventLoopGroup();
+  private final ChannelFuture channelFuture;
+
+  private final NettyRpcProxy.PeerMap proxies = new NettyRpcProxy.PeerMap();
+
+  @ChannelHandler.Sharable
+  class InboundHandler extends SimpleChannelInboundHandler<RaftNettyServerRequestProto> {
+    @Override
+    protected void channelRead0(ChannelHandlerContext ctx, RaftNettyServerRequestProto proto) {
+      final RaftNettyServerReplyProto reply = handle(proto);
+      ctx.writeAndFlush(reply);
+    }
+  }
+
+  /** Constructs a netty server with the given port. */
+  public NettyRpcService(int port, RaftServer server) {
+    this.server = server;
+    this.id = server.getId();
+
+    final ChannelInitializer<SocketChannel> initializer
+        = new ChannelInitializer<SocketChannel>() {
+      @Override
+      protected void initChannel(SocketChannel ch) throws Exception {
+        final ChannelPipeline p = ch.pipeline();
+
+        p.addLast(new ProtobufVarint32FrameDecoder());
+        p.addLast(new ProtobufDecoder(RaftNettyServerRequestProto.getDefaultInstance()));
+        p.addLast(new ProtobufVarint32LengthFieldPrepender());
+        p.addLast(new ProtobufEncoder());
+
+        p.addLast(new InboundHandler());
+      }
+    };
+
+    channelFuture = new ServerBootstrap()
+        .group(bossGroup, workerGroup)
+        .channel(NioServerSocketChannel.class)
+        .handler(new LoggingHandler(LogLevel.INFO))
+        .childHandler(initializer)
+        .bind(port);
+  }
+
+  private Channel getChannel() {
+    return channelFuture.awaitUninterruptibly().channel();
+  }
+
+  @Override
+  public void start() {
+    lifeCycle.startAndTransition(() -> channelFuture.syncUninterruptibly());
+  }
+
+  @Override
+  public void close() {
+    lifeCycle.checkStateAndClose(() -> {
+      bossGroup.shutdownGracefully();
+      workerGroup.shutdownGracefully();
+      final ChannelFuture f = getChannel().close();
+      proxies.close();
+      f.syncUninterruptibly();
+    });
+  }
+
+  @Override
+  public InetSocketAddress getInetSocketAddress() {
+    return (InetSocketAddress)getChannel().localAddress();
+  }
+
+  RaftNettyServerReplyProto handle(RaftNettyServerRequestProto proto) {
+    RaftRpcRequestProto rpcRequest = null;
+    try {
+      switch (proto.getRaftNettyServerRequestCase()) {
+        case REQUESTVOTEREQUEST: {
+          final RequestVoteRequestProto request = proto.getRequestVoteRequest();
+          rpcRequest = request.getServerRequest();
+          final RequestVoteReplyProto reply = server.requestVote(request);
+          return RaftNettyServerReplyProto.newBuilder()
+              .setRequestVoteReply(reply)
+              .build();
+        }
+        case APPENDENTRIESREQUEST: {
+          final AppendEntriesRequestProto request = proto.getAppendEntriesRequest();
+          rpcRequest = request.getServerRequest();
+          final AppendEntriesReplyProto reply = server.appendEntries(request);
+          return RaftNettyServerReplyProto.newBuilder()
+              .setAppendEntriesReply(reply)
+              .build();
+        }
+        case INSTALLSNAPSHOTREQUEST: {
+          final InstallSnapshotRequestProto request = proto.getInstallSnapshotRequest();
+          rpcRequest = request.getServerRequest();
+          final InstallSnapshotReplyProto reply = server.installSnapshot(request);
+          return RaftNettyServerReplyProto.newBuilder()
+              .setInstallSnapshotReply(reply)
+              .build();
+        }
+        case RAFTCLIENTREQUEST: {
+          final RaftClientRequestProto request = proto.getRaftClientRequest();
+          rpcRequest = request.getRpcRequest();
+          final RaftClientReply reply = server.submitClientRequest(
+              ClientProtoUtils.toRaftClientRequest(request));
+          return RaftNettyServerReplyProto.newBuilder()
+              .setRaftClientReply(ClientProtoUtils.toRaftClientReplyProto(reply))
+              .build();
+        }
+        case SETCONFIGURATIONREQUEST: {
+          final SetConfigurationRequestProto request = proto.getSetConfigurationRequest();
+          rpcRequest = request.getRpcRequest();
+          final RaftClientReply reply = server.setConfiguration(
+              ClientProtoUtils.toSetConfigurationRequest(request));
+          return RaftNettyServerReplyProto.newBuilder()
+              .setRaftClientReply(ClientProtoUtils.toRaftClientReplyProto(reply))
+              .build();
+        }
+        case RAFTNETTYSERVERREQUEST_NOT_SET:
+          throw new IllegalArgumentException("Request case not set in proto: "
+              + proto.getRaftNettyServerRequestCase());
+        default:
+          throw new UnsupportedOperationException("Request case not supported: "
+              + proto.getRaftNettyServerRequestCase());
+      }
+    } catch (IOException ioe) {
+      Preconditions.checkNotNull(rpcRequest);
+      return toRaftNettyServerReplyProto(rpcRequest, ioe);
+    }
+  }
+
+  private static RaftNettyServerReplyProto toRaftNettyServerReplyProto(
+      RaftRpcRequestProto request, IOException e) {
+    final RaftRpcReplyProto.Builder rpcReply = ClientProtoUtils.toRaftRpcReplyProtoBuilder(
+        request.getRequestorId(),
+        request.getReplyId(),
+        request.getSeqNum(), false);
+    final RaftNettyExceptionReplyProto.Builder ioe = RaftNettyExceptionReplyProto.newBuilder()
+        .setRpcReply(rpcReply)
+        .setException(ProtoUtils.toByteString(e));
+    return RaftNettyServerReplyProto.newBuilder().setExceptionReply(ioe).build();
+  }
+
+  @Override
+  public RequestVoteReplyProto requestVote(RequestVoteRequestProto request) throws IOException {
+    Preconditions.checkArgument(id.equals(request.getServerRequest().getRequestorId()));
+    CodeInjectionForTesting.execute(SEND_SERVER_REQUEST, id, null, request);
+
+    final RaftNettyServerRequestProto proto = RaftNettyServerRequestProto.newBuilder()
+        .setRequestVoteRequest(request)
+        .build();
+    final RaftRpcRequestProto serverRequest = request.getServerRequest();
+    return sendRaftNettyServerRequestProto(serverRequest, proto).getRequestVoteReply();
+  }
+
+  @Override
+  public AppendEntriesReplyProto appendEntries(AppendEntriesRequestProto request) throws IOException {
+    Preconditions.checkArgument(id.equals(request.getServerRequest().getRequestorId()));
+    CodeInjectionForTesting.execute(SEND_SERVER_REQUEST, id, null, request);
+
+    final RaftNettyServerRequestProto proto = RaftNettyServerRequestProto.newBuilder()
+        .setAppendEntriesRequest(request)
+        .build();
+    final RaftRpcRequestProto serverRequest = request.getServerRequest();
+    return sendRaftNettyServerRequestProto(serverRequest, proto).getAppendEntriesReply();
+  }
+
+  @Override
+  public InstallSnapshotReplyProto installSnapshot(InstallSnapshotRequestProto request) throws IOException {
+    Preconditions.checkArgument(id.equals(request.getServerRequest().getRequestorId()));
+    CodeInjectionForTesting.execute(SEND_SERVER_REQUEST, id, null, request);
+
+    final RaftNettyServerRequestProto proto = RaftNettyServerRequestProto.newBuilder()
+        .setInstallSnapshotRequest(request)
+        .build();
+    final RaftRpcRequestProto serverRequest = request.getServerRequest();
+    return sendRaftNettyServerRequestProto(serverRequest, proto).getInstallSnapshotReply();
+  }
+
+  private RaftNettyServerReplyProto sendRaftNettyServerRequestProto(
+      RaftRpcRequestProto request, RaftNettyServerRequestProto proto)
+      throws IOException {
+    final String id = request.getReplyId();
+    final NettyRpcProxy p = proxies.getProxy(id);
+    try {
+      return p.send(request, proto);
+    } catch (ClosedChannelException cce) {
+      proxies.resetProxy(id);
+      throw cce;
+    }
+  }
+
+  @Override
+  public void addPeers(Iterable<RaftPeer> peers) {
+    proxies.addPeers(peers);
+  }
+}

http://git-wip-us.apache.org/repos/asf/incubator-ratis/blob/7e71a2e0/ratis-netty/src/test/java/org/apache/ratis/netty/MiniRaftClusterWithNetty.java
----------------------------------------------------------------------
diff --git a/ratis-netty/src/test/java/org/apache/ratis/netty/MiniRaftClusterWithNetty.java b/ratis-netty/src/test/java/org/apache/ratis/netty/MiniRaftClusterWithNetty.java
new file mode 100644
index 0000000..92e7722
--- /dev/null
+++ b/ratis-netty/src/test/java/org/apache/ratis/netty/MiniRaftClusterWithNetty.java
@@ -0,0 +1,117 @@
+/**
+ * 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.ratis.netty;
+
+import org.apache.ratis.MiniRaftCluster;
+import org.apache.ratis.RaftTestUtil;
+import org.apache.ratis.client.RaftClientRequestSender;
+import org.apache.ratis.conf.RaftProperties;
+import org.apache.ratis.netty.client.NettyClientRequestSender;
+import org.apache.ratis.netty.server.NettyRpcService;
+import org.apache.ratis.protocol.RaftPeer;
+import org.apache.ratis.server.impl.DelayLocalExecutionInjection;
+import org.apache.ratis.server.impl.RaftConfiguration;
+import org.apache.ratis.server.impl.RaftServerImpl;
+import org.apache.ratis.util.NetUtils;
+
+import java.io.IOException;
+import java.util.Collection;
+import java.util.HashMap;
+import java.util.Map;
+
+public class MiniRaftClusterWithNetty extends MiniRaftCluster.RpcBase {
+  public static final Factory<MiniRaftClusterWithNetty> FACTORY
+      = new Factory<MiniRaftClusterWithNetty>() {
+    @Override
+    public MiniRaftClusterWithNetty newCluster(
+        String[] ids, RaftProperties prop, boolean formatted) {
+      return new MiniRaftClusterWithNetty(ids, prop, formatted);
+    }
+  };
+
+  public static final DelayLocalExecutionInjection sendServerRequest
+      = new DelayLocalExecutionInjection(NettyRpcService.SEND_SERVER_REQUEST);
+
+  public MiniRaftClusterWithNetty(int numServers, RaftProperties properties) {
+    this(generateIds(numServers, 0), properties, true);
+  }
+
+  public MiniRaftClusterWithNetty(
+      String[] ids, RaftProperties properties, boolean formatted) {
+    super(ids, properties, formatted);
+    init(initRpcServices(getServers(), getConf()));
+  }
+
+  private static String getAddress(String id, RaftConfiguration conf) {
+    final RaftPeer peer = conf.getPeer(id);
+    if (peer != null) {
+      final String address = peer.getAddress();
+      if (address != null) {
+        return address;
+      }
+    }
+    return "0.0.0.0:0";
+  }
+
+  private static NettyRpcService newNettyRpcService(
+      RaftServerImpl s, RaftConfiguration conf) {
+    final String address = getAddress(s.getId(), conf);
+    final int port = NetUtils.newInetSocketAddress(address).getPort();
+    return new NettyRpcService(port, s);
+  }
+
+  private static Map<RaftPeer, NettyRpcService> initRpcServices(
+      Collection<RaftServerImpl> servers, RaftConfiguration conf) {
+    final Map<RaftPeer, NettyRpcService> peerRpcs = new HashMap<>();
+
+    for (RaftServerImpl s : servers) {
+      final NettyRpcService rpc = newNettyRpcService(s, conf);
+      peerRpcs.put(new RaftPeer(s.getId(), rpc.getInetSocketAddress()), rpc);
+    }
+
+    return peerRpcs;
+  }
+
+  @Override
+  protected RaftServerImpl setPeerRpc(RaftPeer peer) throws IOException {
+    final RaftServerImpl s = servers.get(peer.getId());
+    final NettyRpcService rpc = newNettyRpcService(s, conf);
+    s.setServerRpc(rpc);
+    return s;
+  }
+
+  @Override
+  protected Collection<RaftPeer> addNewPeers(
+      Collection<RaftPeer> newPeers, Collection<RaftServerImpl> newServers,
+      boolean startService) throws IOException {
+    return addNewPeers(initRpcServices(newServers, conf),
+        newServers, startService);
+  }
+
+  @Override
+  public RaftClientRequestSender getRaftClientRequestSender() {
+    return new NettyClientRequestSender(getPeers());
+  }
+
+  @Override
+  protected void blockQueueAndSetDelay(String leaderId, int delayMs)
+      throws InterruptedException {
+    RaftTestUtil.blockQueueAndSetDelay(getServers(), sendServerRequest,
+        leaderId, delayMs, getMaxTimeout());
+  }
+}

http://git-wip-us.apache.org/repos/asf/incubator-ratis/blob/7e71a2e0/ratis-netty/src/test/java/org/apache/ratis/netty/TestNotLeaderExceptionWithNetty.java
----------------------------------------------------------------------
diff --git a/ratis-netty/src/test/java/org/apache/ratis/netty/TestNotLeaderExceptionWithNetty.java b/ratis-netty/src/test/java/org/apache/ratis/netty/TestNotLeaderExceptionWithNetty.java
new file mode 100644
index 0000000..9c267e7
--- /dev/null
+++ b/ratis-netty/src/test/java/org/apache/ratis/netty/TestNotLeaderExceptionWithNetty.java
@@ -0,0 +1,33 @@
+/**
+ * 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.ratis.netty;
+
+import org.apache.ratis.MiniRaftCluster;
+import org.apache.ratis.RaftNotLeaderExceptionBaseTest;
+import org.apache.ratis.conf.RaftProperties;
+
+import java.io.IOException;
+
+public class TestNotLeaderExceptionWithNetty extends RaftNotLeaderExceptionBaseTest {
+  @Override
+  public MiniRaftCluster initCluster() throws IOException {
+    String[] s = MiniRaftCluster.generateIds(NUM_PEERS, 0);
+    RaftProperties prop = new RaftProperties();
+    return new MiniRaftClusterWithNetty(s, prop, true);
+  }
+}

http://git-wip-us.apache.org/repos/asf/incubator-ratis/blob/7e71a2e0/ratis-netty/src/test/java/org/apache/ratis/netty/TestRaftReconfigurationWithNetty.java
----------------------------------------------------------------------
diff --git a/ratis-netty/src/test/java/org/apache/ratis/netty/TestRaftReconfigurationWithNetty.java b/ratis-netty/src/test/java/org/apache/ratis/netty/TestRaftReconfigurationWithNetty.java
new file mode 100644
index 0000000..cfa9729
--- /dev/null
+++ b/ratis-netty/src/test/java/org/apache/ratis/netty/TestRaftReconfigurationWithNetty.java
@@ -0,0 +1,31 @@
+/**
+ * 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.ratis.netty;
+
+import java.io.IOException;
+
+import org.apache.ratis.MiniRaftCluster;
+import org.apache.ratis.server.impl.RaftReconfigurationBaseTest;
+
+public class TestRaftReconfigurationWithNetty
+    extends RaftReconfigurationBaseTest {
+  @Override
+  public MiniRaftCluster getCluster(int peerNum) throws IOException {
+    return MiniRaftClusterWithNetty.FACTORY.newCluster(peerNum, prop, true);
+  }
+}

http://git-wip-us.apache.org/repos/asf/incubator-ratis/blob/7e71a2e0/ratis-netty/src/test/java/org/apache/ratis/netty/TestRaftSnapshotWithNetty.java
----------------------------------------------------------------------
diff --git a/ratis-netty/src/test/java/org/apache/ratis/netty/TestRaftSnapshotWithNetty.java b/ratis-netty/src/test/java/org/apache/ratis/netty/TestRaftSnapshotWithNetty.java
new file mode 100644
index 0000000..18807c0
--- /dev/null
+++ b/ratis-netty/src/test/java/org/apache/ratis/netty/TestRaftSnapshotWithNetty.java
@@ -0,0 +1,32 @@
+/**
+ * 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.ratis.netty;
+
+import org.apache.ratis.MiniRaftCluster;
+import org.apache.ratis.conf.RaftProperties;
+import org.apache.ratis.statemachine.RaftSnapshotBaseTest;
+
+import java.io.IOException;
+
+public class TestRaftSnapshotWithNetty extends RaftSnapshotBaseTest {
+  @Override
+  public MiniRaftCluster initCluster(int numServer, RaftProperties prop)
+      throws IOException {
+    return MiniRaftClusterWithNetty.FACTORY.newCluster(numServer, prop, true);
+  }
+}

http://git-wip-us.apache.org/repos/asf/incubator-ratis/blob/7e71a2e0/ratis-netty/src/test/java/org/apache/ratis/netty/TestRaftWithNetty.java
----------------------------------------------------------------------
diff --git a/ratis-netty/src/test/java/org/apache/ratis/netty/TestRaftWithNetty.java b/ratis-netty/src/test/java/org/apache/ratis/netty/TestRaftWithNetty.java
new file mode 100644
index 0000000..3954065
--- /dev/null
+++ b/ratis-netty/src/test/java/org/apache/ratis/netty/TestRaftWithNetty.java
@@ -0,0 +1,62 @@
+/**
+ * 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.ratis.netty;
+
+import org.apache.log4j.Level;
+import org.apache.ratis.RaftBasicTests;
+import org.apache.ratis.client.RaftClient;
+import org.apache.ratis.server.impl.BlockRequestHandlingInjection;
+import org.apache.ratis.server.impl.RaftServerImpl;
+import org.apache.ratis.util.RaftUtils;
+import org.junit.Test;
+
+import java.io.IOException;
+
+public class TestRaftWithNetty extends RaftBasicTests {
+  static {
+    RaftUtils.setLogLevel(RaftServerImpl.LOG, Level.DEBUG);
+    RaftUtils.setLogLevel(RaftClient.LOG, Level.DEBUG);
+  }
+
+  private final MiniRaftClusterWithNetty cluster;
+
+  public TestRaftWithNetty() throws IOException {
+    cluster = new MiniRaftClusterWithNetty(NUM_SERVERS, getProperties());
+  }
+
+  @Override
+  public MiniRaftClusterWithNetty getCluster() {
+    return cluster;
+  }
+
+  @Override
+  @Test
+  public void testEnforceLeader() throws Exception {
+    super.testEnforceLeader();
+
+    MiniRaftClusterWithNetty.sendServerRequest.clear();
+    BlockRequestHandlingInjection.getInstance().unblockAll();
+  }
+
+  @Override
+  @Test
+  public void testWithLoad() throws Exception {
+    super.testWithLoad();
+    BlockRequestHandlingInjection.getInstance().unblockAll();
+  }
+}

http://git-wip-us.apache.org/repos/asf/incubator-ratis/blob/7e71a2e0/ratis-netty/src/test/resources/log4j.properties
----------------------------------------------------------------------
diff --git a/ratis-netty/src/test/resources/log4j.properties b/ratis-netty/src/test/resources/log4j.properties
new file mode 100644
index 0000000..ced0687
--- /dev/null
+++ b/ratis-netty/src/test/resources/log4j.properties
@@ -0,0 +1,18 @@
+#   Licensed 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.
+# log4j configuration used during build and unit tests
+
+log4j.rootLogger=info,stdout
+log4j.threshold=ALL
+log4j.appender.stdout=org.apache.log4j.ConsoleAppender
+log4j.appender.stdout.layout=org.apache.log4j.PatternLayout
+log4j.appender.stdout.layout.ConversionPattern=%d{ISO8601} %-5p %c{2} (%F:%M(%L)) - %m%n


[26/54] [abbrv] incubator-ratis git commit: Renamed the packages from raft to ratis in preperation for Apache Incubation - Moved all java packages from org.apache.raft to org.apache.ratis. - Moved native package to org_apache_ratis, and native lib to l

Posted by ji...@apache.org.
http://git-wip-us.apache.org/repos/asf/incubator-ratis/blob/7e71a2e0/ratis-common/src/main/native/src/org/apache/ratis/util/crc32c_tables.h
----------------------------------------------------------------------
diff --git a/ratis-common/src/main/native/src/org/apache/ratis/util/crc32c_tables.h b/ratis-common/src/main/native/src/org/apache/ratis/util/crc32c_tables.h
new file mode 100644
index 0000000..d54c4db
--- /dev/null
+++ b/ratis-common/src/main/native/src/org/apache/ratis/util/crc32c_tables.h
@@ -0,0 +1,550 @@
+/*
+ * 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.
+ */
+
+/*
+ * CRC-32 lookup tables generated by the polynomial 82F63B78
+ * See also TestPureJavaCrc32.Table.
+ */
+const uint32_t CRC32C_T8_0[256] = {
+  0x00000000, 0xF26B8303, 0xE13B70F7, 0x1350F3F4, 
+  0xC79A971F, 0x35F1141C, 0x26A1E7E8, 0xD4CA64EB, 
+  0x8AD958CF, 0x78B2DBCC, 0x6BE22838, 0x9989AB3B, 
+  0x4D43CFD0, 0xBF284CD3, 0xAC78BF27, 0x5E133C24, 
+  0x105EC76F, 0xE235446C, 0xF165B798, 0x030E349B, 
+  0xD7C45070, 0x25AFD373, 0x36FF2087, 0xC494A384, 
+  0x9A879FA0, 0x68EC1CA3, 0x7BBCEF57, 0x89D76C54, 
+  0x5D1D08BF, 0xAF768BBC, 0xBC267848, 0x4E4DFB4B, 
+  0x20BD8EDE, 0xD2D60DDD, 0xC186FE29, 0x33ED7D2A, 
+  0xE72719C1, 0x154C9AC2, 0x061C6936, 0xF477EA35, 
+  0xAA64D611, 0x580F5512, 0x4B5FA6E6, 0xB93425E5, 
+  0x6DFE410E, 0x9F95C20D, 0x8CC531F9, 0x7EAEB2FA, 
+  0x30E349B1, 0xC288CAB2, 0xD1D83946, 0x23B3BA45, 
+  0xF779DEAE, 0x05125DAD, 0x1642AE59, 0xE4292D5A, 
+  0xBA3A117E, 0x4851927D, 0x5B016189, 0xA96AE28A, 
+  0x7DA08661, 0x8FCB0562, 0x9C9BF696, 0x6EF07595, 
+  0x417B1DBC, 0xB3109EBF, 0xA0406D4B, 0x522BEE48, 
+  0x86E18AA3, 0x748A09A0, 0x67DAFA54, 0x95B17957, 
+  0xCBA24573, 0x39C9C670, 0x2A993584, 0xD8F2B687, 
+  0x0C38D26C, 0xFE53516F, 0xED03A29B, 0x1F682198, 
+  0x5125DAD3, 0xA34E59D0, 0xB01EAA24, 0x42752927, 
+  0x96BF4DCC, 0x64D4CECF, 0x77843D3B, 0x85EFBE38, 
+  0xDBFC821C, 0x2997011F, 0x3AC7F2EB, 0xC8AC71E8, 
+  0x1C661503, 0xEE0D9600, 0xFD5D65F4, 0x0F36E6F7, 
+  0x61C69362, 0x93AD1061, 0x80FDE395, 0x72966096, 
+  0xA65C047D, 0x5437877E, 0x4767748A, 0xB50CF789, 
+  0xEB1FCBAD, 0x197448AE, 0x0A24BB5A, 0xF84F3859, 
+  0x2C855CB2, 0xDEEEDFB1, 0xCDBE2C45, 0x3FD5AF46, 
+  0x7198540D, 0x83F3D70E, 0x90A324FA, 0x62C8A7F9, 
+  0xB602C312, 0x44694011, 0x5739B3E5, 0xA55230E6, 
+  0xFB410CC2, 0x092A8FC1, 0x1A7A7C35, 0xE811FF36, 
+  0x3CDB9BDD, 0xCEB018DE, 0xDDE0EB2A, 0x2F8B6829, 
+  0x82F63B78, 0x709DB87B, 0x63CD4B8F, 0x91A6C88C, 
+  0x456CAC67, 0xB7072F64, 0xA457DC90, 0x563C5F93, 
+  0x082F63B7, 0xFA44E0B4, 0xE9141340, 0x1B7F9043, 
+  0xCFB5F4A8, 0x3DDE77AB, 0x2E8E845F, 0xDCE5075C, 
+  0x92A8FC17, 0x60C37F14, 0x73938CE0, 0x81F80FE3, 
+  0x55326B08, 0xA759E80B, 0xB4091BFF, 0x466298FC, 
+  0x1871A4D8, 0xEA1A27DB, 0xF94AD42F, 0x0B21572C, 
+  0xDFEB33C7, 0x2D80B0C4, 0x3ED04330, 0xCCBBC033, 
+  0xA24BB5A6, 0x502036A5, 0x4370C551, 0xB11B4652, 
+  0x65D122B9, 0x97BAA1BA, 0x84EA524E, 0x7681D14D, 
+  0x2892ED69, 0xDAF96E6A, 0xC9A99D9E, 0x3BC21E9D, 
+  0xEF087A76, 0x1D63F975, 0x0E330A81, 0xFC588982, 
+  0xB21572C9, 0x407EF1CA, 0x532E023E, 0xA145813D, 
+  0x758FE5D6, 0x87E466D5, 0x94B49521, 0x66DF1622, 
+  0x38CC2A06, 0xCAA7A905, 0xD9F75AF1, 0x2B9CD9F2, 
+  0xFF56BD19, 0x0D3D3E1A, 0x1E6DCDEE, 0xEC064EED, 
+  0xC38D26C4, 0x31E6A5C7, 0x22B65633, 0xD0DDD530, 
+  0x0417B1DB, 0xF67C32D8, 0xE52CC12C, 0x1747422F, 
+  0x49547E0B, 0xBB3FFD08, 0xA86F0EFC, 0x5A048DFF, 
+  0x8ECEE914, 0x7CA56A17, 0x6FF599E3, 0x9D9E1AE0, 
+  0xD3D3E1AB, 0x21B862A8, 0x32E8915C, 0xC083125F, 
+  0x144976B4, 0xE622F5B7, 0xF5720643, 0x07198540, 
+  0x590AB964, 0xAB613A67, 0xB831C993, 0x4A5A4A90, 
+  0x9E902E7B, 0x6CFBAD78, 0x7FAB5E8C, 0x8DC0DD8F, 
+  0xE330A81A, 0x115B2B19, 0x020BD8ED, 0xF0605BEE, 
+  0x24AA3F05, 0xD6C1BC06, 0xC5914FF2, 0x37FACCF1, 
+  0x69E9F0D5, 0x9B8273D6, 0x88D28022, 0x7AB90321, 
+  0xAE7367CA, 0x5C18E4C9, 0x4F48173D, 0xBD23943E, 
+  0xF36E6F75, 0x0105EC76, 0x12551F82, 0xE03E9C81, 
+  0x34F4F86A, 0xC69F7B69, 0xD5CF889D, 0x27A40B9E, 
+  0x79B737BA, 0x8BDCB4B9, 0x988C474D, 0x6AE7C44E, 
+  0xBE2DA0A5, 0x4C4623A6, 0x5F16D052, 0xAD7D5351
+};
+const uint32_t CRC32C_T8_1[256] = {
+  0x00000000, 0x13A29877, 0x274530EE, 0x34E7A899, 
+  0x4E8A61DC, 0x5D28F9AB, 0x69CF5132, 0x7A6DC945, 
+  0x9D14C3B8, 0x8EB65BCF, 0xBA51F356, 0xA9F36B21, 
+  0xD39EA264, 0xC03C3A13, 0xF4DB928A, 0xE7790AFD, 
+  0x3FC5F181, 0x2C6769F6, 0x1880C16F, 0x0B225918, 
+  0x714F905D, 0x62ED082A, 0x560AA0B3, 0x45A838C4, 
+  0xA2D13239, 0xB173AA4E, 0x859402D7, 0x96369AA0, 
+  0xEC5B53E5, 0xFFF9CB92, 0xCB1E630B, 0xD8BCFB7C, 
+  0x7F8BE302, 0x6C297B75, 0x58CED3EC, 0x4B6C4B9B, 
+  0x310182DE, 0x22A31AA9, 0x1644B230, 0x05E62A47, 
+  0xE29F20BA, 0xF13DB8CD, 0xC5DA1054, 0xD6788823, 
+  0xAC154166, 0xBFB7D911, 0x8B507188, 0x98F2E9FF, 
+  0x404E1283, 0x53EC8AF4, 0x670B226D, 0x74A9BA1A, 
+  0x0EC4735F, 0x1D66EB28, 0x298143B1, 0x3A23DBC6, 
+  0xDD5AD13B, 0xCEF8494C, 0xFA1FE1D5, 0xE9BD79A2, 
+  0x93D0B0E7, 0x80722890, 0xB4958009, 0xA737187E, 
+  0xFF17C604, 0xECB55E73, 0xD852F6EA, 0xCBF06E9D, 
+  0xB19DA7D8, 0xA23F3FAF, 0x96D89736, 0x857A0F41, 
+  0x620305BC, 0x71A19DCB, 0x45463552, 0x56E4AD25, 
+  0x2C896460, 0x3F2BFC17, 0x0BCC548E, 0x186ECCF9, 
+  0xC0D23785, 0xD370AFF2, 0xE797076B, 0xF4359F1C, 
+  0x8E585659, 0x9DFACE2E, 0xA91D66B7, 0xBABFFEC0, 
+  0x5DC6F43D, 0x4E646C4A, 0x7A83C4D3, 0x69215CA4, 
+  0x134C95E1, 0x00EE0D96, 0x3409A50F, 0x27AB3D78, 
+  0x809C2506, 0x933EBD71, 0xA7D915E8, 0xB47B8D9F, 
+  0xCE1644DA, 0xDDB4DCAD, 0xE9537434, 0xFAF1EC43, 
+  0x1D88E6BE, 0x0E2A7EC9, 0x3ACDD650, 0x296F4E27, 
+  0x53028762, 0x40A01F15, 0x7447B78C, 0x67E52FFB, 
+  0xBF59D487, 0xACFB4CF0, 0x981CE469, 0x8BBE7C1E, 
+  0xF1D3B55B, 0xE2712D2C, 0xD69685B5, 0xC5341DC2, 
+  0x224D173F, 0x31EF8F48, 0x050827D1, 0x16AABFA6, 
+  0x6CC776E3, 0x7F65EE94, 0x4B82460D, 0x5820DE7A, 
+  0xFBC3FAF9, 0xE861628E, 0xDC86CA17, 0xCF245260, 
+  0xB5499B25, 0xA6EB0352, 0x920CABCB, 0x81AE33BC, 
+  0x66D73941, 0x7575A136, 0x419209AF, 0x523091D8, 
+  0x285D589D, 0x3BFFC0EA, 0x0F186873, 0x1CBAF004, 
+  0xC4060B78, 0xD7A4930F, 0xE3433B96, 0xF0E1A3E1, 
+  0x8A8C6AA4, 0x992EF2D3, 0xADC95A4A, 0xBE6BC23D, 
+  0x5912C8C0, 0x4AB050B7, 0x7E57F82E, 0x6DF56059, 
+  0x1798A91C, 0x043A316B, 0x30DD99F2, 0x237F0185, 
+  0x844819FB, 0x97EA818C, 0xA30D2915, 0xB0AFB162, 
+  0xCAC27827, 0xD960E050, 0xED8748C9, 0xFE25D0BE, 
+  0x195CDA43, 0x0AFE4234, 0x3E19EAAD, 0x2DBB72DA, 
+  0x57D6BB9F, 0x447423E8, 0x70938B71, 0x63311306, 
+  0xBB8DE87A, 0xA82F700D, 0x9CC8D894, 0x8F6A40E3, 
+  0xF50789A6, 0xE6A511D1, 0xD242B948, 0xC1E0213F, 
+  0x26992BC2, 0x353BB3B5, 0x01DC1B2C, 0x127E835B, 
+  0x68134A1E, 0x7BB1D269, 0x4F567AF0, 0x5CF4E287, 
+  0x04D43CFD, 0x1776A48A, 0x23910C13, 0x30339464, 
+  0x4A5E5D21, 0x59FCC556, 0x6D1B6DCF, 0x7EB9F5B8, 
+  0x99C0FF45, 0x8A626732, 0xBE85CFAB, 0xAD2757DC, 
+  0xD74A9E99, 0xC4E806EE, 0xF00FAE77, 0xE3AD3600, 
+  0x3B11CD7C, 0x28B3550B, 0x1C54FD92, 0x0FF665E5, 
+  0x759BACA0, 0x663934D7, 0x52DE9C4E, 0x417C0439, 
+  0xA6050EC4, 0xB5A796B3, 0x81403E2A, 0x92E2A65D, 
+  0xE88F6F18, 0xFB2DF76F, 0xCFCA5FF6, 0xDC68C781, 
+  0x7B5FDFFF, 0x68FD4788, 0x5C1AEF11, 0x4FB87766, 
+  0x35D5BE23, 0x26772654, 0x12908ECD, 0x013216BA, 
+  0xE64B1C47, 0xF5E98430, 0xC10E2CA9, 0xD2ACB4DE, 
+  0xA8C17D9B, 0xBB63E5EC, 0x8F844D75, 0x9C26D502, 
+  0x449A2E7E, 0x5738B609, 0x63DF1E90, 0x707D86E7, 
+  0x0A104FA2, 0x19B2D7D5, 0x2D557F4C, 0x3EF7E73B, 
+  0xD98EEDC6, 0xCA2C75B1, 0xFECBDD28, 0xED69455F, 
+  0x97048C1A, 0x84A6146D, 0xB041BCF4, 0xA3E32483
+};
+const uint32_t CRC32C_T8_2[256] = {
+  0x00000000, 0xA541927E, 0x4F6F520D, 0xEA2EC073, 
+  0x9EDEA41A, 0x3B9F3664, 0xD1B1F617, 0x74F06469, 
+  0x38513EC5, 0x9D10ACBB, 0x773E6CC8, 0xD27FFEB6, 
+  0xA68F9ADF, 0x03CE08A1, 0xE9E0C8D2, 0x4CA15AAC, 
+  0x70A27D8A, 0xD5E3EFF4, 0x3FCD2F87, 0x9A8CBDF9, 
+  0xEE7CD990, 0x4B3D4BEE, 0xA1138B9D, 0x045219E3, 
+  0x48F3434F, 0xEDB2D131, 0x079C1142, 0xA2DD833C, 
+  0xD62DE755, 0x736C752B, 0x9942B558, 0x3C032726, 
+  0xE144FB14, 0x4405696A, 0xAE2BA919, 0x0B6A3B67, 
+  0x7F9A5F0E, 0xDADBCD70, 0x30F50D03, 0x95B49F7D, 
+  0xD915C5D1, 0x7C5457AF, 0x967A97DC, 0x333B05A2, 
+  0x47CB61CB, 0xE28AF3B5, 0x08A433C6, 0xADE5A1B8, 
+  0x91E6869E, 0x34A714E0, 0xDE89D493, 0x7BC846ED, 
+  0x0F382284, 0xAA79B0FA, 0x40577089, 0xE516E2F7, 
+  0xA9B7B85B, 0x0CF62A25, 0xE6D8EA56, 0x43997828, 
+  0x37691C41, 0x92288E3F, 0x78064E4C, 0xDD47DC32, 
+  0xC76580D9, 0x622412A7, 0x880AD2D4, 0x2D4B40AA, 
+  0x59BB24C3, 0xFCFAB6BD, 0x16D476CE, 0xB395E4B0, 
+  0xFF34BE1C, 0x5A752C62, 0xB05BEC11, 0x151A7E6F, 
+  0x61EA1A06, 0xC4AB8878, 0x2E85480B, 0x8BC4DA75, 
+  0xB7C7FD53, 0x12866F2D, 0xF8A8AF5E, 0x5DE93D20, 
+  0x29195949, 0x8C58CB37, 0x66760B44, 0xC337993A, 
+  0x8F96C396, 0x2AD751E8, 0xC0F9919B, 0x65B803E5, 
+  0x1148678C, 0xB409F5F2, 0x5E273581, 0xFB66A7FF, 
+  0x26217BCD, 0x8360E9B3, 0x694E29C0, 0xCC0FBBBE, 
+  0xB8FFDFD7, 0x1DBE4DA9, 0xF7908DDA, 0x52D11FA4, 
+  0x1E704508, 0xBB31D776, 0x511F1705, 0xF45E857B, 
+  0x80AEE112, 0x25EF736C, 0xCFC1B31F, 0x6A802161, 
+  0x56830647, 0xF3C29439, 0x19EC544A, 0xBCADC634, 
+  0xC85DA25D, 0x6D1C3023, 0x8732F050, 0x2273622E, 
+  0x6ED23882, 0xCB93AAFC, 0x21BD6A8F, 0x84FCF8F1, 
+  0xF00C9C98, 0x554D0EE6, 0xBF63CE95, 0x1A225CEB, 
+  0x8B277743, 0x2E66E53D, 0xC448254E, 0x6109B730, 
+  0x15F9D359, 0xB0B84127, 0x5A968154, 0xFFD7132A, 
+  0xB3764986, 0x1637DBF8, 0xFC191B8B, 0x595889F5, 
+  0x2DA8ED9C, 0x88E97FE2, 0x62C7BF91, 0xC7862DEF, 
+  0xFB850AC9, 0x5EC498B7, 0xB4EA58C4, 0x11ABCABA, 
+  0x655BAED3, 0xC01A3CAD, 0x2A34FCDE, 0x8F756EA0, 
+  0xC3D4340C, 0x6695A672, 0x8CBB6601, 0x29FAF47F, 
+  0x5D0A9016, 0xF84B0268, 0x1265C21B, 0xB7245065, 
+  0x6A638C57, 0xCF221E29, 0x250CDE5A, 0x804D4C24, 
+  0xF4BD284D, 0x51FCBA33, 0xBBD27A40, 0x1E93E83E, 
+  0x5232B292, 0xF77320EC, 0x1D5DE09F, 0xB81C72E1, 
+  0xCCEC1688, 0x69AD84F6, 0x83834485, 0x26C2D6FB, 
+  0x1AC1F1DD, 0xBF8063A3, 0x55AEA3D0, 0xF0EF31AE, 
+  0x841F55C7, 0x215EC7B9, 0xCB7007CA, 0x6E3195B4, 
+  0x2290CF18, 0x87D15D66, 0x6DFF9D15, 0xC8BE0F6B, 
+  0xBC4E6B02, 0x190FF97C, 0xF321390F, 0x5660AB71, 
+  0x4C42F79A, 0xE90365E4, 0x032DA597, 0xA66C37E9, 
+  0xD29C5380, 0x77DDC1FE, 0x9DF3018D, 0x38B293F3, 
+  0x7413C95F, 0xD1525B21, 0x3B7C9B52, 0x9E3D092C, 
+  0xEACD6D45, 0x4F8CFF3B, 0xA5A23F48, 0x00E3AD36, 
+  0x3CE08A10, 0x99A1186E, 0x738FD81D, 0xD6CE4A63, 
+  0xA23E2E0A, 0x077FBC74, 0xED517C07, 0x4810EE79, 
+  0x04B1B4D5, 0xA1F026AB, 0x4BDEE6D8, 0xEE9F74A6, 
+  0x9A6F10CF, 0x3F2E82B1, 0xD50042C2, 0x7041D0BC, 
+  0xAD060C8E, 0x08479EF0, 0xE2695E83, 0x4728CCFD, 
+  0x33D8A894, 0x96993AEA, 0x7CB7FA99, 0xD9F668E7, 
+  0x9557324B, 0x3016A035, 0xDA386046, 0x7F79F238, 
+  0x0B899651, 0xAEC8042F, 0x44E6C45C, 0xE1A75622, 
+  0xDDA47104, 0x78E5E37A, 0x92CB2309, 0x378AB177, 
+  0x437AD51E, 0xE63B4760, 0x0C158713, 0xA954156D, 
+  0xE5F54FC1, 0x40B4DDBF, 0xAA9A1DCC, 0x0FDB8FB2, 
+  0x7B2BEBDB, 0xDE6A79A5, 0x3444B9D6, 0x91052BA8
+};
+const uint32_t CRC32C_T8_3[256] = {
+  0x00000000, 0xDD45AAB8, 0xBF672381, 0x62228939, 
+  0x7B2231F3, 0xA6679B4B, 0xC4451272, 0x1900B8CA, 
+  0xF64463E6, 0x2B01C95E, 0x49234067, 0x9466EADF, 
+  0x8D665215, 0x5023F8AD, 0x32017194, 0xEF44DB2C, 
+  0xE964B13D, 0x34211B85, 0x560392BC, 0x8B463804, 
+  0x924680CE, 0x4F032A76, 0x2D21A34F, 0xF06409F7, 
+  0x1F20D2DB, 0xC2657863, 0xA047F15A, 0x7D025BE2, 
+  0x6402E328, 0xB9474990, 0xDB65C0A9, 0x06206A11, 
+  0xD725148B, 0x0A60BE33, 0x6842370A, 0xB5079DB2, 
+  0xAC072578, 0x71428FC0, 0x136006F9, 0xCE25AC41, 
+  0x2161776D, 0xFC24DDD5, 0x9E0654EC, 0x4343FE54, 
+  0x5A43469E, 0x8706EC26, 0xE524651F, 0x3861CFA7, 
+  0x3E41A5B6, 0xE3040F0E, 0x81268637, 0x5C632C8F, 
+  0x45639445, 0x98263EFD, 0xFA04B7C4, 0x27411D7C, 
+  0xC805C650, 0x15406CE8, 0x7762E5D1, 0xAA274F69, 
+  0xB327F7A3, 0x6E625D1B, 0x0C40D422, 0xD1057E9A, 
+  0xABA65FE7, 0x76E3F55F, 0x14C17C66, 0xC984D6DE, 
+  0xD0846E14, 0x0DC1C4AC, 0x6FE34D95, 0xB2A6E72D, 
+  0x5DE23C01, 0x80A796B9, 0xE2851F80, 0x3FC0B538, 
+  0x26C00DF2, 0xFB85A74A, 0x99A72E73, 0x44E284CB, 
+  0x42C2EEDA, 0x9F874462, 0xFDA5CD5B, 0x20E067E3, 
+  0x39E0DF29, 0xE4A57591, 0x8687FCA8, 0x5BC25610, 
+  0xB4868D3C, 0x69C32784, 0x0BE1AEBD, 0xD6A40405, 
+  0xCFA4BCCF, 0x12E11677, 0x70C39F4E, 0xAD8635F6, 
+  0x7C834B6C, 0xA1C6E1D4, 0xC3E468ED, 0x1EA1C255, 
+  0x07A17A9F, 0xDAE4D027, 0xB8C6591E, 0x6583F3A6, 
+  0x8AC7288A, 0x57828232, 0x35A00B0B, 0xE8E5A1B3, 
+  0xF1E51979, 0x2CA0B3C1, 0x4E823AF8, 0x93C79040, 
+  0x95E7FA51, 0x48A250E9, 0x2A80D9D0, 0xF7C57368, 
+  0xEEC5CBA2, 0x3380611A, 0x51A2E823, 0x8CE7429B, 
+  0x63A399B7, 0xBEE6330F, 0xDCC4BA36, 0x0181108E, 
+  0x1881A844, 0xC5C402FC, 0xA7E68BC5, 0x7AA3217D, 
+  0x52A0C93F, 0x8FE56387, 0xEDC7EABE, 0x30824006, 
+  0x2982F8CC, 0xF4C75274, 0x96E5DB4D, 0x4BA071F5, 
+  0xA4E4AAD9, 0x79A10061, 0x1B838958, 0xC6C623E0, 
+  0xDFC69B2A, 0x02833192, 0x60A1B8AB, 0xBDE41213, 
+  0xBBC47802, 0x6681D2BA, 0x04A35B83, 0xD9E6F13B, 
+  0xC0E649F1, 0x1DA3E349, 0x7F816A70, 0xA2C4C0C8, 
+  0x4D801BE4, 0x90C5B15C, 0xF2E73865, 0x2FA292DD, 
+  0x36A22A17, 0xEBE780AF, 0x89C50996, 0x5480A32E, 
+  0x8585DDB4, 0x58C0770C, 0x3AE2FE35, 0xE7A7548D, 
+  0xFEA7EC47, 0x23E246FF, 0x41C0CFC6, 0x9C85657E, 
+  0x73C1BE52, 0xAE8414EA, 0xCCA69DD3, 0x11E3376B, 
+  0x08E38FA1, 0xD5A62519, 0xB784AC20, 0x6AC10698, 
+  0x6CE16C89, 0xB1A4C631, 0xD3864F08, 0x0EC3E5B0, 
+  0x17C35D7A, 0xCA86F7C2, 0xA8A47EFB, 0x75E1D443, 
+  0x9AA50F6F, 0x47E0A5D7, 0x25C22CEE, 0xF8878656, 
+  0xE1873E9C, 0x3CC29424, 0x5EE01D1D, 0x83A5B7A5, 
+  0xF90696D8, 0x24433C60, 0x4661B559, 0x9B241FE1, 
+  0x8224A72B, 0x5F610D93, 0x3D4384AA, 0xE0062E12, 
+  0x0F42F53E, 0xD2075F86, 0xB025D6BF, 0x6D607C07, 
+  0x7460C4CD, 0xA9256E75, 0xCB07E74C, 0x16424DF4, 
+  0x106227E5, 0xCD278D5D, 0xAF050464, 0x7240AEDC, 
+  0x6B401616, 0xB605BCAE, 0xD4273597, 0x09629F2F, 
+  0xE6264403, 0x3B63EEBB, 0x59416782, 0x8404CD3A, 
+  0x9D0475F0, 0x4041DF48, 0x22635671, 0xFF26FCC9, 
+  0x2E238253, 0xF36628EB, 0x9144A1D2, 0x4C010B6A, 
+  0x5501B3A0, 0x88441918, 0xEA669021, 0x37233A99, 
+  0xD867E1B5, 0x05224B0D, 0x6700C234, 0xBA45688C, 
+  0xA345D046, 0x7E007AFE, 0x1C22F3C7, 0xC167597F, 
+  0xC747336E, 0x1A0299D6, 0x782010EF, 0xA565BA57, 
+  0xBC65029D, 0x6120A825, 0x0302211C, 0xDE478BA4, 
+  0x31035088, 0xEC46FA30, 0x8E647309, 0x5321D9B1, 
+  0x4A21617B, 0x9764CBC3, 0xF54642FA, 0x2803E842
+};
+const uint32_t CRC32C_T8_4[256] = {
+  0x00000000, 0x38116FAC, 0x7022DF58, 0x4833B0F4, 
+  0xE045BEB0, 0xD854D11C, 0x906761E8, 0xA8760E44, 
+  0xC5670B91, 0xFD76643D, 0xB545D4C9, 0x8D54BB65, 
+  0x2522B521, 0x1D33DA8D, 0x55006A79, 0x6D1105D5, 
+  0x8F2261D3, 0xB7330E7F, 0xFF00BE8B, 0xC711D127, 
+  0x6F67DF63, 0x5776B0CF, 0x1F45003B, 0x27546F97, 
+  0x4A456A42, 0x725405EE, 0x3A67B51A, 0x0276DAB6, 
+  0xAA00D4F2, 0x9211BB5E, 0xDA220BAA, 0xE2336406, 
+  0x1BA8B557, 0x23B9DAFB, 0x6B8A6A0F, 0x539B05A3, 
+  0xFBED0BE7, 0xC3FC644B, 0x8BCFD4BF, 0xB3DEBB13, 
+  0xDECFBEC6, 0xE6DED16A, 0xAEED619E, 0x96FC0E32, 
+  0x3E8A0076, 0x069B6FDA, 0x4EA8DF2E, 0x76B9B082, 
+  0x948AD484, 0xAC9BBB28, 0xE4A80BDC, 0xDCB96470, 
+  0x74CF6A34, 0x4CDE0598, 0x04EDB56C, 0x3CFCDAC0, 
+  0x51EDDF15, 0x69FCB0B9, 0x21CF004D, 0x19DE6FE1, 
+  0xB1A861A5, 0x89B90E09, 0xC18ABEFD, 0xF99BD151, 
+  0x37516AAE, 0x0F400502, 0x4773B5F6, 0x7F62DA5A, 
+  0xD714D41E, 0xEF05BBB2, 0xA7360B46, 0x9F2764EA, 
+  0xF236613F, 0xCA270E93, 0x8214BE67, 0xBA05D1CB, 
+  0x1273DF8F, 0x2A62B023, 0x625100D7, 0x5A406F7B, 
+  0xB8730B7D, 0x806264D1, 0xC851D425, 0xF040BB89, 
+  0x5836B5CD, 0x6027DA61, 0x28146A95, 0x10050539, 
+  0x7D1400EC, 0x45056F40, 0x0D36DFB4, 0x3527B018, 
+  0x9D51BE5C, 0xA540D1F0, 0xED736104, 0xD5620EA8, 
+  0x2CF9DFF9, 0x14E8B055, 0x5CDB00A1, 0x64CA6F0D, 
+  0xCCBC6149, 0xF4AD0EE5, 0xBC9EBE11, 0x848FD1BD, 
+  0xE99ED468, 0xD18FBBC4, 0x99BC0B30, 0xA1AD649C, 
+  0x09DB6AD8, 0x31CA0574, 0x79F9B580, 0x41E8DA2C, 
+  0xA3DBBE2A, 0x9BCAD186, 0xD3F96172, 0xEBE80EDE, 
+  0x439E009A, 0x7B8F6F36, 0x33BCDFC2, 0x0BADB06E, 
+  0x66BCB5BB, 0x5EADDA17, 0x169E6AE3, 0x2E8F054F, 
+  0x86F90B0B, 0xBEE864A7, 0xF6DBD453, 0xCECABBFF, 
+  0x6EA2D55C, 0x56B3BAF0, 0x1E800A04, 0x269165A8, 
+  0x8EE76BEC, 0xB6F60440, 0xFEC5B4B4, 0xC6D4DB18, 
+  0xABC5DECD, 0x93D4B161, 0xDBE70195, 0xE3F66E39, 
+  0x4B80607D, 0x73910FD1, 0x3BA2BF25, 0x03B3D089, 
+  0xE180B48F, 0xD991DB23, 0x91A26BD7, 0xA9B3047B, 
+  0x01C50A3F, 0x39D46593, 0x71E7D567, 0x49F6BACB, 
+  0x24E7BF1E, 0x1CF6D0B2, 0x54C56046, 0x6CD40FEA, 
+  0xC4A201AE, 0xFCB36E02, 0xB480DEF6, 0x8C91B15A, 
+  0x750A600B, 0x4D1B0FA7, 0x0528BF53, 0x3D39D0FF, 
+  0x954FDEBB, 0xAD5EB117, 0xE56D01E3, 0xDD7C6E4F, 
+  0xB06D6B9A, 0x887C0436, 0xC04FB4C2, 0xF85EDB6E, 
+  0x5028D52A, 0x6839BA86, 0x200A0A72, 0x181B65DE, 
+  0xFA2801D8, 0xC2396E74, 0x8A0ADE80, 0xB21BB12C, 
+  0x1A6DBF68, 0x227CD0C4, 0x6A4F6030, 0x525E0F9C, 
+  0x3F4F0A49, 0x075E65E5, 0x4F6DD511, 0x777CBABD, 
+  0xDF0AB4F9, 0xE71BDB55, 0xAF286BA1, 0x9739040D, 
+  0x59F3BFF2, 0x61E2D05E, 0x29D160AA, 0x11C00F06, 
+  0xB9B60142, 0x81A76EEE, 0xC994DE1A, 0xF185B1B6, 
+  0x9C94B463, 0xA485DBCF, 0xECB66B3B, 0xD4A70497, 
+  0x7CD10AD3, 0x44C0657F, 0x0CF3D58B, 0x34E2BA27, 
+  0xD6D1DE21, 0xEEC0B18D, 0xA6F30179, 0x9EE26ED5, 
+  0x36946091, 0x0E850F3D, 0x46B6BFC9, 0x7EA7D065, 
+  0x13B6D5B0, 0x2BA7BA1C, 0x63940AE8, 0x5B856544, 
+  0xF3F36B00, 0xCBE204AC, 0x83D1B458, 0xBBC0DBF4, 
+  0x425B0AA5, 0x7A4A6509, 0x3279D5FD, 0x0A68BA51, 
+  0xA21EB415, 0x9A0FDBB9, 0xD23C6B4D, 0xEA2D04E1, 
+  0x873C0134, 0xBF2D6E98, 0xF71EDE6C, 0xCF0FB1C0, 
+  0x6779BF84, 0x5F68D028, 0x175B60DC, 0x2F4A0F70, 
+  0xCD796B76, 0xF56804DA, 0xBD5BB42E, 0x854ADB82, 
+  0x2D3CD5C6, 0x152DBA6A, 0x5D1E0A9E, 0x650F6532, 
+  0x081E60E7, 0x300F0F4B, 0x783CBFBF, 0x402DD013, 
+  0xE85BDE57, 0xD04AB1FB, 0x9879010F, 0xA0686EA3
+};
+const uint32_t CRC32C_T8_5[256] = {
+  0x00000000, 0xEF306B19, 0xDB8CA0C3, 0x34BCCBDA, 
+  0xB2F53777, 0x5DC55C6E, 0x697997B4, 0x8649FCAD, 
+  0x6006181F, 0x8F367306, 0xBB8AB8DC, 0x54BAD3C5, 
+  0xD2F32F68, 0x3DC34471, 0x097F8FAB, 0xE64FE4B2, 
+  0xC00C303E, 0x2F3C5B27, 0x1B8090FD, 0xF4B0FBE4, 
+  0x72F90749, 0x9DC96C50, 0xA975A78A, 0x4645CC93, 
+  0xA00A2821, 0x4F3A4338, 0x7B8688E2, 0x94B6E3FB, 
+  0x12FF1F56, 0xFDCF744F, 0xC973BF95, 0x2643D48C, 
+  0x85F4168D, 0x6AC47D94, 0x5E78B64E, 0xB148DD57, 
+  0x370121FA, 0xD8314AE3, 0xEC8D8139, 0x03BDEA20, 
+  0xE5F20E92, 0x0AC2658B, 0x3E7EAE51, 0xD14EC548, 
+  0x570739E5, 0xB83752FC, 0x8C8B9926, 0x63BBF23F, 
+  0x45F826B3, 0xAAC84DAA, 0x9E748670, 0x7144ED69, 
+  0xF70D11C4, 0x183D7ADD, 0x2C81B107, 0xC3B1DA1E, 
+  0x25FE3EAC, 0xCACE55B5, 0xFE729E6F, 0x1142F576, 
+  0x970B09DB, 0x783B62C2, 0x4C87A918, 0xA3B7C201, 
+  0x0E045BEB, 0xE13430F2, 0xD588FB28, 0x3AB89031, 
+  0xBCF16C9C, 0x53C10785, 0x677DCC5F, 0x884DA746, 
+  0x6E0243F4, 0x813228ED, 0xB58EE337, 0x5ABE882E, 
+  0xDCF77483, 0x33C71F9A, 0x077BD440, 0xE84BBF59, 
+  0xCE086BD5, 0x213800CC, 0x1584CB16, 0xFAB4A00F, 
+  0x7CFD5CA2, 0x93CD37BB, 0xA771FC61, 0x48419778, 
+  0xAE0E73CA, 0x413E18D3, 0x7582D309, 0x9AB2B810, 
+  0x1CFB44BD, 0xF3CB2FA4, 0xC777E47E, 0x28478F67, 
+  0x8BF04D66, 0x64C0267F, 0x507CEDA5, 0xBF4C86BC, 
+  0x39057A11, 0xD6351108, 0xE289DAD2, 0x0DB9B1CB, 
+  0xEBF65579, 0x04C63E60, 0x307AF5BA, 0xDF4A9EA3, 
+  0x5903620E, 0xB6330917, 0x828FC2CD, 0x6DBFA9D4, 
+  0x4BFC7D58, 0xA4CC1641, 0x9070DD9B, 0x7F40B682, 
+  0xF9094A2F, 0x16392136, 0x2285EAEC, 0xCDB581F5, 
+  0x2BFA6547, 0xC4CA0E5E, 0xF076C584, 0x1F46AE9D, 
+  0x990F5230, 0x763F3929, 0x4283F2F3, 0xADB399EA, 
+  0x1C08B7D6, 0xF338DCCF, 0xC7841715, 0x28B47C0C, 
+  0xAEFD80A1, 0x41CDEBB8, 0x75712062, 0x9A414B7B, 
+  0x7C0EAFC9, 0x933EC4D0, 0xA7820F0A, 0x48B26413, 
+  0xCEFB98BE, 0x21CBF3A7, 0x1577387D, 0xFA475364, 
+  0xDC0487E8, 0x3334ECF1, 0x0788272B, 0xE8B84C32, 
+  0x6EF1B09F, 0x81C1DB86, 0xB57D105C, 0x5A4D7B45, 
+  0xBC029FF7, 0x5332F4EE, 0x678E3F34, 0x88BE542D, 
+  0x0EF7A880, 0xE1C7C399, 0xD57B0843, 0x3A4B635A, 
+  0x99FCA15B, 0x76CCCA42, 0x42700198, 0xAD406A81, 
+  0x2B09962C, 0xC439FD35, 0xF08536EF, 0x1FB55DF6, 
+  0xF9FAB944, 0x16CAD25D, 0x22761987, 0xCD46729E, 
+  0x4B0F8E33, 0xA43FE52A, 0x90832EF0, 0x7FB345E9, 
+  0x59F09165, 0xB6C0FA7C, 0x827C31A6, 0x6D4C5ABF, 
+  0xEB05A612, 0x0435CD0B, 0x308906D1, 0xDFB96DC8, 
+  0x39F6897A, 0xD6C6E263, 0xE27A29B9, 0x0D4A42A0, 
+  0x8B03BE0D, 0x6433D514, 0x508F1ECE, 0xBFBF75D7, 
+  0x120CEC3D, 0xFD3C8724, 0xC9804CFE, 0x26B027E7, 
+  0xA0F9DB4A, 0x4FC9B053, 0x7B757B89, 0x94451090, 
+  0x720AF422, 0x9D3A9F3B, 0xA98654E1, 0x46B63FF8, 
+  0xC0FFC355, 0x2FCFA84C, 0x1B736396, 0xF443088F, 
+  0xD200DC03, 0x3D30B71A, 0x098C7CC0, 0xE6BC17D9, 
+  0x60F5EB74, 0x8FC5806D, 0xBB794BB7, 0x544920AE, 
+  0xB206C41C, 0x5D36AF05, 0x698A64DF, 0x86BA0FC6, 
+  0x00F3F36B, 0xEFC39872, 0xDB7F53A8, 0x344F38B1, 
+  0x97F8FAB0, 0x78C891A9, 0x4C745A73, 0xA344316A, 
+  0x250DCDC7, 0xCA3DA6DE, 0xFE816D04, 0x11B1061D, 
+  0xF7FEE2AF, 0x18CE89B6, 0x2C72426C, 0xC3422975, 
+  0x450BD5D8, 0xAA3BBEC1, 0x9E87751B, 0x71B71E02, 
+  0x57F4CA8E, 0xB8C4A197, 0x8C786A4D, 0x63480154, 
+  0xE501FDF9, 0x0A3196E0, 0x3E8D5D3A, 0xD1BD3623, 
+  0x37F2D291, 0xD8C2B988, 0xEC7E7252, 0x034E194B, 
+  0x8507E5E6, 0x6A378EFF, 0x5E8B4525, 0xB1BB2E3C
+};
+const uint32_t CRC32C_T8_6[256] = {
+  0x00000000, 0x68032CC8, 0xD0065990, 0xB8057558, 
+  0xA5E0C5D1, 0xCDE3E919, 0x75E69C41, 0x1DE5B089, 
+  0x4E2DFD53, 0x262ED19B, 0x9E2BA4C3, 0xF628880B, 
+  0xEBCD3882, 0x83CE144A, 0x3BCB6112, 0x53C84DDA, 
+  0x9C5BFAA6, 0xF458D66E, 0x4C5DA336, 0x245E8FFE, 
+  0x39BB3F77, 0x51B813BF, 0xE9BD66E7, 0x81BE4A2F, 
+  0xD27607F5, 0xBA752B3D, 0x02705E65, 0x6A7372AD, 
+  0x7796C224, 0x1F95EEEC, 0xA7909BB4, 0xCF93B77C, 
+  0x3D5B83BD, 0x5558AF75, 0xED5DDA2D, 0x855EF6E5, 
+  0x98BB466C, 0xF0B86AA4, 0x48BD1FFC, 0x20BE3334, 
+  0x73767EEE, 0x1B755226, 0xA370277E, 0xCB730BB6, 
+  0xD696BB3F, 0xBE9597F7, 0x0690E2AF, 0x6E93CE67, 
+  0xA100791B, 0xC90355D3, 0x7106208B, 0x19050C43, 
+  0x04E0BCCA, 0x6CE39002, 0xD4E6E55A, 0xBCE5C992, 
+  0xEF2D8448, 0x872EA880, 0x3F2BDDD8, 0x5728F110, 
+  0x4ACD4199, 0x22CE6D51, 0x9ACB1809, 0xF2C834C1, 
+  0x7AB7077A, 0x12B42BB2, 0xAAB15EEA, 0xC2B27222, 
+  0xDF57C2AB, 0xB754EE63, 0x0F519B3B, 0x6752B7F3, 
+  0x349AFA29, 0x5C99D6E1, 0xE49CA3B9, 0x8C9F8F71, 
+  0x917A3FF8, 0xF9791330, 0x417C6668, 0x297F4AA0, 
+  0xE6ECFDDC, 0x8EEFD114, 0x36EAA44C, 0x5EE98884, 
+  0x430C380D, 0x2B0F14C5, 0x930A619D, 0xFB094D55, 
+  0xA8C1008F, 0xC0C22C47, 0x78C7591F, 0x10C475D7, 
+  0x0D21C55E, 0x6522E996, 0xDD279CCE, 0xB524B006, 
+  0x47EC84C7, 0x2FEFA80F, 0x97EADD57, 0xFFE9F19F, 
+  0xE20C4116, 0x8A0F6DDE, 0x320A1886, 0x5A09344E, 
+  0x09C17994, 0x61C2555C, 0xD9C72004, 0xB1C40CCC, 
+  0xAC21BC45, 0xC422908D, 0x7C27E5D5, 0x1424C91D, 
+  0xDBB77E61, 0xB3B452A9, 0x0BB127F1, 0x63B20B39, 
+  0x7E57BBB0, 0x16549778, 0xAE51E220, 0xC652CEE8, 
+  0x959A8332, 0xFD99AFFA, 0x459CDAA2, 0x2D9FF66A, 
+  0x307A46E3, 0x58796A2B, 0xE07C1F73, 0x887F33BB, 
+  0xF56E0EF4, 0x9D6D223C, 0x25685764, 0x4D6B7BAC, 
+  0x508ECB25, 0x388DE7ED, 0x808892B5, 0xE88BBE7D, 
+  0xBB43F3A7, 0xD340DF6F, 0x6B45AA37, 0x034686FF, 
+  0x1EA33676, 0x76A01ABE, 0xCEA56FE6, 0xA6A6432E, 
+  0x6935F452, 0x0136D89A, 0xB933ADC2, 0xD130810A, 
+  0xCCD53183, 0xA4D61D4B, 0x1CD36813, 0x74D044DB, 
+  0x27180901, 0x4F1B25C9, 0xF71E5091, 0x9F1D7C59, 
+  0x82F8CCD0, 0xEAFBE018, 0x52FE9540, 0x3AFDB988, 
+  0xC8358D49, 0xA036A181, 0x1833D4D9, 0x7030F811, 
+  0x6DD54898, 0x05D66450, 0xBDD31108, 0xD5D03DC0, 
+  0x8618701A, 0xEE1B5CD2, 0x561E298A, 0x3E1D0542, 
+  0x23F8B5CB, 0x4BFB9903, 0xF3FEEC5B, 0x9BFDC093, 
+  0x546E77EF, 0x3C6D5B27, 0x84682E7F, 0xEC6B02B7, 
+  0xF18EB23E, 0x998D9EF6, 0x2188EBAE, 0x498BC766, 
+  0x1A438ABC, 0x7240A674, 0xCA45D32C, 0xA246FFE4, 
+  0xBFA34F6D, 0xD7A063A5, 0x6FA516FD, 0x07A63A35, 
+  0x8FD9098E, 0xE7DA2546, 0x5FDF501E, 0x37DC7CD6, 
+  0x2A39CC5F, 0x423AE097, 0xFA3F95CF, 0x923CB907, 
+  0xC1F4F4DD, 0xA9F7D815, 0x11F2AD4D, 0x79F18185, 
+  0x6414310C, 0x0C171DC4, 0xB412689C, 0xDC114454, 
+  0x1382F328, 0x7B81DFE0, 0xC384AAB8, 0xAB878670, 
+  0xB66236F9, 0xDE611A31, 0x66646F69, 0x0E6743A1, 
+  0x5DAF0E7B, 0x35AC22B3, 0x8DA957EB, 0xE5AA7B23, 
+  0xF84FCBAA, 0x904CE762, 0x2849923A, 0x404ABEF2, 
+  0xB2828A33, 0xDA81A6FB, 0x6284D3A3, 0x0A87FF6B, 
+  0x17624FE2, 0x7F61632A, 0xC7641672, 0xAF673ABA, 
+  0xFCAF7760, 0x94AC5BA8, 0x2CA92EF0, 0x44AA0238, 
+  0x594FB2B1, 0x314C9E79, 0x8949EB21, 0xE14AC7E9, 
+  0x2ED97095, 0x46DA5C5D, 0xFEDF2905, 0x96DC05CD, 
+  0x8B39B544, 0xE33A998C, 0x5B3FECD4, 0x333CC01C, 
+  0x60F48DC6, 0x08F7A10E, 0xB0F2D456, 0xD8F1F89E, 
+  0xC5144817, 0xAD1764DF, 0x15121187, 0x7D113D4F
+};
+const uint32_t CRC32C_T8_7[256] = {
+  0x00000000, 0x493C7D27, 0x9278FA4E, 0xDB448769, 
+  0x211D826D, 0x6821FF4A, 0xB3657823, 0xFA590504, 
+  0x423B04DA, 0x0B0779FD, 0xD043FE94, 0x997F83B3, 
+  0x632686B7, 0x2A1AFB90, 0xF15E7CF9, 0xB86201DE, 
+  0x847609B4, 0xCD4A7493, 0x160EF3FA, 0x5F328EDD, 
+  0xA56B8BD9, 0xEC57F6FE, 0x37137197, 0x7E2F0CB0, 
+  0xC64D0D6E, 0x8F717049, 0x5435F720, 0x1D098A07, 
+  0xE7508F03, 0xAE6CF224, 0x7528754D, 0x3C14086A, 
+  0x0D006599, 0x443C18BE, 0x9F789FD7, 0xD644E2F0, 
+  0x2C1DE7F4, 0x65219AD3, 0xBE651DBA, 0xF759609D, 
+  0x4F3B6143, 0x06071C64, 0xDD439B0D, 0x947FE62A, 
+  0x6E26E32E, 0x271A9E09, 0xFC5E1960, 0xB5626447, 
+  0x89766C2D, 0xC04A110A, 0x1B0E9663, 0x5232EB44, 
+  0xA86BEE40, 0xE1579367, 0x3A13140E, 0x732F6929, 
+  0xCB4D68F7, 0x827115D0, 0x593592B9, 0x1009EF9E, 
+  0xEA50EA9A, 0xA36C97BD, 0x782810D4, 0x31146DF3, 
+  0x1A00CB32, 0x533CB615, 0x8878317C, 0xC1444C5B, 
+  0x3B1D495F, 0x72213478, 0xA965B311, 0xE059CE36, 
+  0x583BCFE8, 0x1107B2CF, 0xCA4335A6, 0x837F4881, 
+  0x79264D85, 0x301A30A2, 0xEB5EB7CB, 0xA262CAEC, 
+  0x9E76C286, 0xD74ABFA1, 0x0C0E38C8, 0x453245EF, 
+  0xBF6B40EB, 0xF6573DCC, 0x2D13BAA5, 0x642FC782, 
+  0xDC4DC65C, 0x9571BB7B, 0x4E353C12, 0x07094135, 
+  0xFD504431, 0xB46C3916, 0x6F28BE7F, 0x2614C358, 
+  0x1700AEAB, 0x5E3CD38C, 0x857854E5, 0xCC4429C2, 
+  0x361D2CC6, 0x7F2151E1, 0xA465D688, 0xED59ABAF, 
+  0x553BAA71, 0x1C07D756, 0xC743503F, 0x8E7F2D18, 
+  0x7426281C, 0x3D1A553B, 0xE65ED252, 0xAF62AF75, 
+  0x9376A71F, 0xDA4ADA38, 0x010E5D51, 0x48322076, 
+  0xB26B2572, 0xFB575855, 0x2013DF3C, 0x692FA21B, 
+  0xD14DA3C5, 0x9871DEE2, 0x4335598B, 0x0A0924AC, 
+  0xF05021A8, 0xB96C5C8F, 0x6228DBE6, 0x2B14A6C1, 
+  0x34019664, 0x7D3DEB43, 0xA6796C2A, 0xEF45110D, 
+  0x151C1409, 0x5C20692E, 0x8764EE47, 0xCE589360, 
+  0x763A92BE, 0x3F06EF99, 0xE44268F0, 0xAD7E15D7, 
+  0x572710D3, 0x1E1B6DF4, 0xC55FEA9D, 0x8C6397BA, 
+  0xB0779FD0, 0xF94BE2F7, 0x220F659E, 0x6B3318B9, 
+  0x916A1DBD, 0xD856609A, 0x0312E7F3, 0x4A2E9AD4, 
+  0xF24C9B0A, 0xBB70E62D, 0x60346144, 0x29081C63, 
+  0xD3511967, 0x9A6D6440, 0x4129E329, 0x08159E0E, 
+  0x3901F3FD, 0x703D8EDA, 0xAB7909B3, 0xE2457494, 
+  0x181C7190, 0x51200CB7, 0x8A648BDE, 0xC358F6F9, 
+  0x7B3AF727, 0x32068A00, 0xE9420D69, 0xA07E704E, 
+  0x5A27754A, 0x131B086D, 0xC85F8F04, 0x8163F223, 
+  0xBD77FA49, 0xF44B876E, 0x2F0F0007, 0x66337D20, 
+  0x9C6A7824, 0xD5560503, 0x0E12826A, 0x472EFF4D, 
+  0xFF4CFE93, 0xB67083B4, 0x6D3404DD, 0x240879FA, 
+  0xDE517CFE, 0x976D01D9, 0x4C2986B0, 0x0515FB97, 
+  0x2E015D56, 0x673D2071, 0xBC79A718, 0xF545DA3F, 
+  0x0F1CDF3B, 0x4620A21C, 0x9D642575, 0xD4585852, 
+  0x6C3A598C, 0x250624AB, 0xFE42A3C2, 0xB77EDEE5, 
+  0x4D27DBE1, 0x041BA6C6, 0xDF5F21AF, 0x96635C88, 
+  0xAA7754E2, 0xE34B29C5, 0x380FAEAC, 0x7133D38B, 
+  0x8B6AD68F, 0xC256ABA8, 0x19122CC1, 0x502E51E6, 
+  0xE84C5038, 0xA1702D1F, 0x7A34AA76, 0x3308D751, 
+  0xC951D255, 0x806DAF72, 0x5B29281B, 0x1215553C, 
+  0x230138CF, 0x6A3D45E8, 0xB179C281, 0xF845BFA6, 
+  0x021CBAA2, 0x4B20C785, 0x906440EC, 0xD9583DCB, 
+  0x613A3C15, 0x28064132, 0xF342C65B, 0xBA7EBB7C, 
+  0x4027BE78, 0x091BC35F, 0xD25F4436, 0x9B633911, 
+  0xA777317B, 0xEE4B4C5C, 0x350FCB35, 0x7C33B612, 
+  0x866AB316, 0xCF56CE31, 0x14124958, 0x5D2E347F, 
+  0xE54C35A1, 0xAC704886, 0x7734CFEF, 0x3E08B2C8, 
+  0xC451B7CC, 0x8D6DCAEB, 0x56294D82, 0x1F1530A5
+};

http://git-wip-us.apache.org/repos/asf/incubator-ratis/blob/7e71a2e0/ratis-common/src/main/native/src/org/apache/ratis/util/gcc_optimizations.h
----------------------------------------------------------------------
diff --git a/ratis-common/src/main/native/src/org/apache/ratis/util/gcc_optimizations.h b/ratis-common/src/main/native/src/org/apache/ratis/util/gcc_optimizations.h
new file mode 100644
index 0000000..8c0fbff
--- /dev/null
+++ b/ratis-common/src/main/native/src/org/apache/ratis/util/gcc_optimizations.h
@@ -0,0 +1,30 @@
+/*
+ * 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.
+ */
+#ifndef __GCC_OPTIMIZATIONS_H_INCLUDED
+#define __GCC_OPTIMIZATIONS_H_INCLUDED
+
+// Hints to gcc optimizer -- compiled out on non-GCC
+#ifdef __GNUC__
+#define likely(x)       __builtin_expect((x),1)
+#define unlikely(x)     __builtin_expect((x),0)
+#else
+#define likely(x)       (x)
+#define unlikely(x)     (x)
+#endif
+
+#endif

http://git-wip-us.apache.org/repos/asf/incubator-ratis/blob/7e71a2e0/ratis-common/src/main/native/src/org_apache_ratis.h
----------------------------------------------------------------------
diff --git a/ratis-common/src/main/native/src/org_apache_ratis.h b/ratis-common/src/main/native/src/org_apache_ratis.h
new file mode 100644
index 0000000..0de9c81
--- /dev/null
+++ b/ratis-common/src/main/native/src/org_apache_ratis.h
@@ -0,0 +1,189 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+/**
+ * This file includes some common utilities
+ * for all native code used in ratis.
+ */
+
+#if !defined ORG_APACHE_RATIS_H
+#define ORG_APACHE_RATIS_H
+
+#if defined(_WIN32)
+#undef UNIX
+#define WINDOWS
+#else
+#undef WINDOWS
+#define UNIX
+#endif
+
+/* A helper macro to 'throw' a java exception. */
+#define THROW(env, exception_name, message) \
+  { \
+	jclass ecls = (*env)->FindClass(env, exception_name); \
+	if (ecls) { \
+	  (*env)->ThrowNew(env, ecls, message); \
+	  (*env)->DeleteLocalRef(env, ecls); \
+	} \
+  }
+
+/* Helper macro to return if an exception is pending */
+#define PASS_EXCEPTIONS(env) \
+  { \
+    if ((*env)->ExceptionCheck(env)) return; \
+  }
+
+#define PASS_EXCEPTIONS_GOTO(env, target) \
+  { \
+    if ((*env)->ExceptionCheck(env)) goto target; \
+  }
+
+#define PASS_EXCEPTIONS_RET(env, ret) \
+  { \
+    if ((*env)->ExceptionCheck(env)) return (ret); \
+  }
+
+/**
+ * Unix definitions
+ */
+#ifdef UNIX
+#include <config.h>
+#include <dlfcn.h>
+#include <jni.h>
+
+/**
+ * A helper function to dlsym a 'symbol' from a given library-handle.
+ *
+ * @param env jni handle to report contingencies.
+ * @param handle handle to the dlopen'ed library.
+ * @param symbol symbol to load.
+ * @return returns the address where the symbol is loaded in memory,
+ *         <code>NULL</code> on error.
+ */
+static __attribute__ ((unused))
+void *do_dlsym(JNIEnv *env, void *handle, const char *symbol) {
+  if (!env || !handle || !symbol) {
+  	THROW(env, "java/lang/InternalError", NULL);
+  	return NULL;
+  }
+  char *error = NULL;
+  void *func_ptr = dlsym(handle, symbol);
+  if ((error = dlerror()) != NULL) {
+  	THROW(env, "java/lang/UnsatisfiedLinkError", symbol);
+  	return NULL;
+  }
+  return func_ptr;
+}
+
+/* A helper macro to dlsym the requisite dynamic symbol and bail-out on error. */
+#define LOAD_DYNAMIC_SYMBOL(func_ptr, env, handle, symbol) \
+  if ((func_ptr = do_dlsym(env, handle, symbol)) == NULL) { \
+    return; \
+  }
+#endif
+// Unix part end
+
+
+/**
+ * Windows definitions
+ */
+#ifdef WINDOWS
+
+/* Force using Unicode throughout the code */
+#ifndef UNICODE
+#define UNICODE
+#endif
+
+/* Microsoft C Compiler does not support the C99 inline keyword */
+#ifndef __cplusplus
+#define inline __inline;
+#endif // _cplusplus
+
+/* Optimization macros supported by GCC but for which there is no
+   direct equivalent in the Microsoft C compiler */
+#define likely(_c) (_c)
+#define unlikely(_c) (_c)
+
+/* Disable certain warnings in the native CRC32 code. */
+#pragma warning(disable:4018)		// Signed/unsigned mismatch.
+#pragma warning(disable:4244)		// Possible loss of data in conversion.
+#pragma warning(disable:4267)		// Possible loss of data.
+#pragma warning(disable:4996)		// Use of deprecated function.
+
+#include <Windows.h>
+#include <stdio.h>
+#include <jni.h>
+
+#define snprintf(a, b ,c, d) _snprintf_s((a), (b), _TRUNCATE, (c), (d))
+
+/* A helper macro to dlsym the requisite dynamic symbol and bail-out on error. */
+#define LOAD_DYNAMIC_SYMBOL(func_type, func_ptr, env, handle, symbol) \
+  if ((func_ptr = (func_type) do_dlsym(env, handle, symbol)) == NULL) { \
+    return; \
+  }
+
+/**
+ * A helper function to dynamic load a 'symbol' from a given library-handle.
+ *
+ * @param env jni handle to report contingencies.
+ * @param handle handle to the dynamic library.
+ * @param symbol symbol to load.
+ * @return returns the address where the symbol is loaded in memory,
+ *         <code>NULL</code> on error.
+ */
+static FARPROC WINAPI do_dlsym(JNIEnv *env, HMODULE handle, LPCSTR symbol) {
+  DWORD dwErrorCode = ERROR_SUCCESS;
+  FARPROC func_ptr = NULL;
+
+  if (!env || !handle || !symbol) {
+    THROW(env, "java/lang/InternalError", NULL);
+    return NULL;
+  }
+
+  func_ptr = GetProcAddress(handle, symbol);
+  if (func_ptr == NULL)
+  {
+    THROW(env, "java/lang/UnsatisfiedLinkError", symbol);
+  }
+  return func_ptr;
+}
+#endif
+// Windows part end
+
+
+#define LOCK_CLASS(env, clazz, classname) \
+  if ((*env)->MonitorEnter(env, clazz) != 0) { \
+    char exception_msg[128]; \
+    snprintf(exception_msg, 128, "Failed to lock %s", classname); \
+    THROW(env, "java/lang/InternalError", exception_msg); \
+  }
+
+#define UNLOCK_CLASS(env, clazz, classname) \
+  if ((*env)->MonitorExit(env, clazz) != 0) { \
+    char exception_msg[128]; \
+    snprintf(exception_msg, 128, "Failed to unlock %s", classname); \
+    THROW(env, "java/lang/InternalError", exception_msg); \
+  }
+
+#define RETRY_ON_EINTR(ret, expr) do { \
+  ret = expr; \
+} while ((ret == -1) && (errno == EINTR));
+
+#endif
+
+//vim: sw=2: ts=2: et

http://git-wip-us.apache.org/repos/asf/incubator-ratis/blob/7e71a2e0/ratis-common/src/main/native/src/test/org/apache/ratis/util/test_bulk_crc32.c
----------------------------------------------------------------------
diff --git a/ratis-common/src/main/native/src/test/org/apache/ratis/util/test_bulk_crc32.c b/ratis-common/src/main/native/src/test/org/apache/ratis/util/test_bulk_crc32.c
new file mode 100644
index 0000000..afd7262
--- /dev/null
+++ b/ratis-common/src/main/native/src/test/org/apache/ratis/util/test_bulk_crc32.c
@@ -0,0 +1,113 @@
+/**
+ * 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.
+ */
+
+#include "org_apache_ratis.h"
+
+#include "bulk_crc32.h"
+
+#include <stdint.h>
+#include <stdio.h>
+#include <stdlib.h>
+#include <time.h>
+
+#define EXPECT_ZERO(x) \
+    do { \
+        int __my_ret__ = x; \
+        if (__my_ret__) { \
+            fprintf(stderr, "TEST_ERROR: failed on line %d with return " \
+              "code %d: got nonzero from %s\n", __LINE__, __my_ret__, #x); \
+            return __my_ret__; \
+        } \
+    } while (0);
+
+static int testBulkVerifyCrc(int dataLen, int crcType, int bytesPerChecksum)
+{
+  int i;
+  uint8_t *data;
+  uint32_t *sums;
+  crc32_error_t errorData;
+
+  data = malloc(dataLen);
+  for (i = 0; i < dataLen; i++) {
+    data[i] = (i % 16) + 1;
+  }
+  sums = calloc(sizeof(uint32_t),
+                (dataLen + bytesPerChecksum - 1) / bytesPerChecksum);
+
+  EXPECT_ZERO(bulk_crc(data, dataLen, sums, crcType,
+                                 bytesPerChecksum, NULL));
+  EXPECT_ZERO(bulk_crc(data, dataLen, sums, crcType,
+                            bytesPerChecksum, &errorData));
+  free(data);
+  free(sums);
+  return 0;
+}
+
+static int timeBulkCrc(int dataLen, int crcType, int bytesPerChecksum, int iterations)
+{
+  int i;
+  uint8_t *data;
+  uint32_t *sums;
+  crc32_error_t errorData;
+  clock_t start, fini;
+
+  data = malloc(dataLen);
+  for (i = 0; i < dataLen; i++) {
+    data[i] = (i % 16) + 1;
+  }
+  sums = calloc(sizeof(uint32_t),
+                (dataLen + bytesPerChecksum - 1) / bytesPerChecksum);
+
+  start = clock();
+  for (i = 0; i < iterations; i++) {
+    EXPECT_ZERO(bulk_crc(data, dataLen, sums, crcType,
+				   bytesPerChecksum, NULL));
+    EXPECT_ZERO(bulk_crc(data, dataLen, sums, crcType,
+			      bytesPerChecksum, &errorData));
+  }
+  fini = clock();
+  printf("CRC %d bytes @ %d bytes per checksum X %d iterations = %g\n",
+          dataLen, bytesPerChecksum, iterations, (double)(fini-start)/CLOCKS_PER_SEC);
+  free(data);
+  free(sums);
+  return 0;
+}
+
+int main(int argc, char **argv)
+{
+  /* Test running bulk_calculate_crc with some different algorithms and
+   * bytePerChecksum values. */
+  EXPECT_ZERO(testBulkVerifyCrc(4096, CRC32C_POLYNOMIAL, 512));
+  EXPECT_ZERO(testBulkVerifyCrc(4096, CRC32_ZLIB_POLYNOMIAL, 512));
+  EXPECT_ZERO(testBulkVerifyCrc(256, CRC32C_POLYNOMIAL, 1));
+  EXPECT_ZERO(testBulkVerifyCrc(256, CRC32_ZLIB_POLYNOMIAL, 1));
+  EXPECT_ZERO(testBulkVerifyCrc(1, CRC32C_POLYNOMIAL, 1));
+  EXPECT_ZERO(testBulkVerifyCrc(1, CRC32_ZLIB_POLYNOMIAL, 1));
+  EXPECT_ZERO(testBulkVerifyCrc(2, CRC32C_POLYNOMIAL, 1));
+  EXPECT_ZERO(testBulkVerifyCrc(17, CRC32C_POLYNOMIAL, 1));
+  EXPECT_ZERO(testBulkVerifyCrc(17, CRC32C_POLYNOMIAL, 2));
+  EXPECT_ZERO(testBulkVerifyCrc(17, CRC32_ZLIB_POLYNOMIAL, 2));
+  EXPECT_ZERO(testBulkVerifyCrc(17, CRC32C_POLYNOMIAL, 4));
+  EXPECT_ZERO(testBulkVerifyCrc(17, CRC32_ZLIB_POLYNOMIAL, 4));
+
+  EXPECT_ZERO(timeBulkCrc(16 * 1024, CRC32C_POLYNOMIAL, 512, 1000000));
+  EXPECT_ZERO(timeBulkCrc(16 * 1024, CRC32_ZLIB_POLYNOMIAL, 512, 1000000));
+
+  fprintf(stderr, "%s: SUCCESS.\n", argv[0]);
+  return EXIT_SUCCESS;
+}

http://git-wip-us.apache.org/repos/asf/incubator-ratis/blob/7e71a2e0/ratis-common/src/test/java/org/apache/ratis/util/TestLifeCycle.java
----------------------------------------------------------------------
diff --git a/ratis-common/src/test/java/org/apache/ratis/util/TestLifeCycle.java b/ratis-common/src/test/java/org/apache/ratis/util/TestLifeCycle.java
new file mode 100644
index 0000000..82816c9
--- /dev/null
+++ b/ratis-common/src/test/java/org/apache/ratis/util/TestLifeCycle.java
@@ -0,0 +1,54 @@
+/**
+ * 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.ratis.util;
+
+import org.apache.ratis.util.LifeCycle;
+import org.junit.Assert;
+import org.junit.Test;
+
+import static org.apache.ratis.util.LifeCycle.State.*;
+
+import java.util.*;
+
+public class TestLifeCycle {
+  /**
+   * Test if the successor map and the predecessor map are consistent.
+   * {@link LifeCycle} uses predecessors to validate transitions
+   * while this test uses successors.
+   */
+  @Test
+  public void testIsValid() throws Exception {
+    final Map<LifeCycle.State, List<LifeCycle.State>> successors
+        = new EnumMap<>(LifeCycle.State.class);
+    put(NEW,       successors, STARTING, CLOSED);
+    put(STARTING,  successors, NEW, RUNNING, CLOSING, EXCEPTION);
+    put(RUNNING,   successors, CLOSING, PAUSING, EXCEPTION);
+    put(PAUSING,   successors, PAUSED, CLOSING, EXCEPTION);
+    put(PAUSED,    successors, STARTING, CLOSING);
+    put(EXCEPTION, successors, CLOSING);
+    put(CLOSING ,  successors, CLOSED);
+    put(CLOSED,    successors);
+
+    final List<LifeCycle.State> states = Arrays.asList(LifeCycle.State.values());
+    states.stream().forEach(
+        from -> states.forEach(
+            to -> Assert.assertEquals(from + " -> " + to,
+                successors.get(from).contains(to),
+                isValid(from, to))));
+  }
+}

http://git-wip-us.apache.org/repos/asf/incubator-ratis/blob/7e71a2e0/ratis-examples/pom.xml
----------------------------------------------------------------------
diff --git a/ratis-examples/pom.xml b/ratis-examples/pom.xml
new file mode 100644
index 0000000..5330099
--- /dev/null
+++ b/ratis-examples/pom.xml
@@ -0,0 +1,130 @@
+<?xml version="1.0" encoding="UTF-8"?>
+<!--
+  Licensed 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. See accompanying LICENSE file.
+-->
+<project xmlns="http://maven.apache.org/POM/4.0.0"
+         xmlns:xsi="http://www.w3.org/2001/XMLSchema-instance"
+         xsi:schemaLocation="http://maven.apache.org/POM/4.0.0 http://maven.apache.org/xsd/maven-4.0.0.xsd">
+  <modelVersion>4.0.0</modelVersion>
+  <parent>
+    <artifactId>ratis-project-dist</artifactId>
+    <groupId>org.apache.ratis</groupId>
+    <version>1.0-SNAPSHOT</version>
+    <relativePath>../ratis-project-dist</relativePath>
+  </parent>
+
+  <artifactId>ratis-examples</artifactId>
+  <name>Ratis Examples</name>
+
+  <dependencies>
+    <dependency>
+      <artifactId>ratis-proto-shaded</artifactId>
+      <groupId>org.apache.ratis</groupId>
+      <scope>provided</scope>
+    </dependency>
+
+    <dependency>
+      <artifactId>ratis-common</artifactId>
+      <groupId>org.apache.ratis</groupId>
+      <scope>provided</scope>
+    </dependency>
+    <dependency>
+      <artifactId>ratis-common</artifactId>
+      <groupId>org.apache.ratis</groupId>
+      <scope>test</scope>
+      <type>test-jar</type>
+    </dependency>
+    
+    <dependency>
+      <artifactId>ratis-client</artifactId>
+      <groupId>org.apache.ratis</groupId>
+      <scope>provided</scope>
+    </dependency>
+    <dependency>
+      <artifactId>ratis-client</artifactId>
+      <groupId>org.apache.ratis</groupId>
+      <scope>test</scope>
+      <type>test-jar</type>
+    </dependency>
+    
+    <dependency>
+      <artifactId>ratis-server</artifactId>
+      <groupId>org.apache.ratis</groupId>
+      <scope>provided</scope>
+    </dependency>
+    <dependency>
+      <artifactId>ratis-server</artifactId>
+      <groupId>org.apache.ratis</groupId>
+      <scope>test</scope>
+      <type>test-jar</type>
+    </dependency>
+    
+    <dependency>
+      <artifactId>ratis-hadoop</artifactId>
+      <groupId>org.apache.ratis</groupId>
+      <scope>provided</scope>
+    </dependency>
+    <dependency>
+      <artifactId>ratis-hadoop</artifactId>
+      <groupId>org.apache.ratis</groupId>
+      <scope>test</scope>
+      <type>test-jar</type>
+    </dependency>
+
+    <dependency>
+      <artifactId>ratis-grpc</artifactId>
+      <groupId>org.apache.ratis</groupId>
+      <scope>provided</scope>
+    </dependency>
+    <dependency>
+      <artifactId>ratis-grpc</artifactId>
+      <groupId>org.apache.ratis</groupId>
+      <scope>test</scope>
+      <type>test-jar</type>
+    </dependency>
+
+    <dependency>
+      <artifactId>ratis-netty</artifactId>
+      <groupId>org.apache.ratis</groupId>
+      <scope>provided</scope>
+    </dependency>
+    <dependency>
+      <artifactId>ratis-netty</artifactId>
+      <groupId>org.apache.ratis</groupId>
+      <scope>test</scope>
+      <type>test-jar</type>
+    </dependency>
+
+    <dependency>
+      <groupId>org.slf4j</groupId>
+      <artifactId>slf4j-api</artifactId>
+    </dependency>
+
+    <dependency>
+      <groupId>com.google.guava</groupId>
+      <artifactId>guava</artifactId>
+    </dependency>
+
+    <dependency>
+      <groupId>junit</groupId>
+      <artifactId>junit</artifactId>
+      <scope>test</scope>
+    </dependency>
+    <dependency>
+      <groupId>org.mockito</groupId>
+      <artifactId>mockito-all</artifactId>
+      <scope>test</scope>
+    </dependency>
+  </dependencies>
+</project>
+    

http://git-wip-us.apache.org/repos/asf/incubator-ratis/blob/7e71a2e0/ratis-examples/src/main/java/org/apache/ratis/examples/arithmetic/ArithmeticStateMachine.java
----------------------------------------------------------------------
diff --git a/ratis-examples/src/main/java/org/apache/ratis/examples/arithmetic/ArithmeticStateMachine.java b/ratis-examples/src/main/java/org/apache/ratis/examples/arithmetic/ArithmeticStateMachine.java
new file mode 100644
index 0000000..b241f3a
--- /dev/null
+++ b/ratis-examples/src/main/java/org/apache/ratis/examples/arithmetic/ArithmeticStateMachine.java
@@ -0,0 +1,181 @@
+/**
+ * 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.ratis.examples.arithmetic;
+
+import com.google.common.base.Preconditions;
+
+import org.apache.ratis.conf.RaftProperties;
+import org.apache.ratis.examples.arithmetic.expression.Expression;
+import org.apache.ratis.protocol.Message;
+import org.apache.ratis.protocol.RaftClientReply;
+import org.apache.ratis.protocol.RaftClientRequest;
+import org.apache.ratis.server.impl.RaftServerConstants;
+import org.apache.ratis.server.protocol.TermIndex;
+import org.apache.ratis.server.storage.RaftStorage;
+import org.apache.ratis.shaded.proto.RaftProtos.LogEntryProto;
+import org.apache.ratis.statemachine.*;
+import org.apache.ratis.util.AutoCloseableLock;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.io.*;
+import java.util.HashMap;
+import java.util.Map;
+import java.util.concurrent.CompletableFuture;
+import java.util.concurrent.ConcurrentHashMap;
+import java.util.concurrent.atomic.AtomicReference;
+import java.util.concurrent.locks.ReentrantReadWriteLock;
+
+public class ArithmeticStateMachine extends BaseStateMachine {
+  static final Logger LOG = LoggerFactory.getLogger(ArithmeticStateMachine.class);
+
+  private final Map<String, Double> variables = new ConcurrentHashMap<>();
+
+  private final SimpleStateMachineStorage storage = new SimpleStateMachineStorage();
+  private final AtomicReference<TermIndex> latestTermIndex = new AtomicReference<>();
+
+  private final ReentrantReadWriteLock lock = new ReentrantReadWriteLock(true);
+
+  private AutoCloseableLock readLock() {
+    return AutoCloseableLock.acquire(lock.readLock());
+  }
+
+  private AutoCloseableLock writeLock() {
+    return AutoCloseableLock.acquire(lock.writeLock());
+  }
+
+  void reset() {
+    variables.clear();
+    latestTermIndex.set(null);
+  }
+
+  @Override
+  public void initialize(String id, RaftProperties properties, RaftStorage raftStorage)
+      throws IOException {
+    super.initialize(id, properties, raftStorage);
+    this.storage.init(raftStorage);
+    loadSnapshot(storage.getLatestSnapshot());
+  }
+
+  @Override
+  public void reinitialize(String id, RaftProperties properties, RaftStorage storage)
+      throws IOException {
+    close();
+    this.initialize(id, properties, storage);
+  }
+
+  @Override
+  public long takeSnapshot() throws IOException {
+    final Map<String, Double> copy;
+    final TermIndex last;
+    try(final AutoCloseableLock readLock = readLock()) {
+      copy = new HashMap<>(variables);
+      last = latestTermIndex.get();
+    }
+
+    File snapshotFile =  new File(SimpleStateMachineStorage.getSnapshotFileName(
+        last.getTerm(), last.getIndex()));
+
+    try(final ObjectOutputStream out = new ObjectOutputStream(
+        new BufferedOutputStream(new FileOutputStream(snapshotFile)))) {
+      out.writeObject(copy);
+    } catch(IOException ioe) {
+      LOG.warn("Failed to write snapshot file \"" + snapshotFile
+          + "\", last applied index=" + last);
+    }
+
+    return last.getIndex();
+  }
+
+  public long loadSnapshot(SingleFileSnapshotInfo snapshot) throws IOException {
+    return load(snapshot, false);
+  }
+
+  private long load(SingleFileSnapshotInfo snapshot, boolean reload) throws IOException {
+    if (snapshot == null || !snapshot.getFile().getPath().toFile().exists()) {
+      LOG.warn("The snapshot file {} does not exist", snapshot);
+      return RaftServerConstants.INVALID_LOG_INDEX;
+    }
+
+    File snapshotFile =snapshot.getFile().getPath().toFile();
+    final TermIndex last = SimpleStateMachineStorage.getTermIndexFromSnapshotFile(snapshotFile);
+    try(final AutoCloseableLock writeLock = writeLock();
+        final ObjectInputStream in = new ObjectInputStream(
+            new BufferedInputStream(new FileInputStream(snapshotFile)))) {
+      if (reload) {
+        reset();
+      }
+      latestTermIndex.set(last);
+      variables.putAll((Map<String, Double>) in.readObject());
+    } catch (ClassNotFoundException e) {
+      throw new IllegalStateException(e);
+    }
+    return last.getIndex();
+  }
+
+  @Override
+  public StateMachineStorage getStateMachineStorage() {
+    return storage;
+  }
+
+  @Override
+  public CompletableFuture<RaftClientReply> query(
+      RaftClientRequest request) {
+    final Expression q = Expression.Utils.bytes2Expression(
+        request.getMessage().getContent().toByteArray(), 0);
+    final Double result;
+    try(final AutoCloseableLock readLock = readLock()) {
+      result = q.evaluate(variables);
+    }
+    final Expression r = Expression.Utils.double2Expression(result);
+    LOG.debug("QUERY: {} = {}", q, r);
+    final RaftClientReply reply = new RaftClientReply(request,
+        Expression.Utils.toMessage(r));
+    return CompletableFuture.completedFuture(reply);
+  }
+
+  @Override
+  public void close() {
+    reset();
+  }
+
+  @Override
+  public CompletableFuture<Message> applyTransaction(TransactionContext trx) {
+    final LogEntryProto entry = trx.getLogEntry().get();
+    final AssignmentMessage assignment = new AssignmentMessage(
+        () -> entry.getSmLogEntry().getData());
+
+    final long index = entry.getIndex();
+    final Double result;
+    try(final AutoCloseableLock writeLock = writeLock()) {
+      result = assignment.evaluate(variables);
+      updateLatestTermIndex(entry.getTerm(), index);
+    }
+    final Expression r = Expression.Utils.double2Expression(result);
+    LOG.debug("{}: {} = {}, variables={}", index, assignment, r, variables);
+    return CompletableFuture.completedFuture(Expression.Utils.toMessage(r));
+  }
+
+  private void updateLatestTermIndex(long term, long index) {
+    final TermIndex newTI = TermIndex.newTermIndex(term, index);
+    final TermIndex oldTI = latestTermIndex.getAndSet(newTI);
+    if (oldTI != null) {
+      Preconditions.checkArgument(newTI.compareTo(oldTI) >= 0);
+    }
+  }
+}

http://git-wip-us.apache.org/repos/asf/incubator-ratis/blob/7e71a2e0/ratis-examples/src/main/java/org/apache/ratis/examples/arithmetic/AssignmentMessage.java
----------------------------------------------------------------------
diff --git a/ratis-examples/src/main/java/org/apache/ratis/examples/arithmetic/AssignmentMessage.java b/ratis-examples/src/main/java/org/apache/ratis/examples/arithmetic/AssignmentMessage.java
new file mode 100644
index 0000000..e4e7ca8
--- /dev/null
+++ b/ratis-examples/src/main/java/org/apache/ratis/examples/arithmetic/AssignmentMessage.java
@@ -0,0 +1,83 @@
+/**
+ * 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.ratis.examples.arithmetic;
+
+import static org.apache.ratis.util.ProtoUtils.toByteString;
+
+import java.nio.charset.Charset;
+import java.util.Map;
+
+import org.apache.ratis.examples.arithmetic.expression.Expression;
+import org.apache.ratis.examples.arithmetic.expression.Variable;
+import org.apache.ratis.protocol.Message;
+import org.apache.ratis.shaded.com.google.protobuf.ByteString;
+
+public class AssignmentMessage implements Message, Evaluable {
+  public static final Charset UTF8 = Charset.forName("UTF-8");
+
+  private final Variable variable;
+  private final Expression expression;
+
+  public AssignmentMessage(Variable variable, Expression expression) {
+    this.variable = variable;
+    this.expression = expression;
+  }
+
+  public AssignmentMessage(byte[] buf, int offset) {
+    variable = new Variable(buf, offset);
+    expression = Expression.Utils.bytes2Expression(buf, offset + variable.length());
+  }
+
+  public AssignmentMessage(Message message) {
+    this(message.getContent().toByteArray(), 0);
+  }
+
+  public Variable getVariable() {
+    return variable;
+  }
+
+  public Expression getExpression() {
+    return expression;
+  }
+
+  @Override
+  public ByteString getContent() {
+    final int length = variable.length() + expression.length();
+    final byte[] bytes = new byte[length];
+    final int offset = variable.toBytes(bytes, 0);
+    expression.toBytes(bytes, offset);
+    return toByteString(bytes);
+  }
+
+  @Override
+  public String toString() {
+    return variable + " = " + expression;
+  }
+
+  @Override
+  public Double evaluate(Map<String, Double> variableMap) {
+    final Double value = expression.evaluate(variableMap);
+    final String name = variable.getName();
+    if (value == null) {
+      variableMap.remove(name);
+    } else {
+      variableMap.put(name, value);
+    }
+    return value;
+  }
+}

http://git-wip-us.apache.org/repos/asf/incubator-ratis/blob/7e71a2e0/ratis-examples/src/main/java/org/apache/ratis/examples/arithmetic/Evaluable.java
----------------------------------------------------------------------
diff --git a/ratis-examples/src/main/java/org/apache/ratis/examples/arithmetic/Evaluable.java b/ratis-examples/src/main/java/org/apache/ratis/examples/arithmetic/Evaluable.java
new file mode 100644
index 0000000..8afb080
--- /dev/null
+++ b/ratis-examples/src/main/java/org/apache/ratis/examples/arithmetic/Evaluable.java
@@ -0,0 +1,24 @@
+/**
+ * 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.ratis.examples.arithmetic;
+
+import java.util.Map;
+
+public interface Evaluable {
+  Double evaluate(Map<String, Double> variableMap);
+}

http://git-wip-us.apache.org/repos/asf/incubator-ratis/blob/7e71a2e0/ratis-examples/src/main/java/org/apache/ratis/examples/arithmetic/expression/BinaryExpression.java
----------------------------------------------------------------------
diff --git a/ratis-examples/src/main/java/org/apache/ratis/examples/arithmetic/expression/BinaryExpression.java b/ratis-examples/src/main/java/org/apache/ratis/examples/arithmetic/expression/BinaryExpression.java
new file mode 100644
index 0000000..fceaef9
--- /dev/null
+++ b/ratis-examples/src/main/java/org/apache/ratis/examples/arithmetic/expression/BinaryExpression.java
@@ -0,0 +1,103 @@
+/**
+ * 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.ratis.examples.arithmetic.expression;
+
+import com.google.common.base.Preconditions;
+
+import java.util.Map;
+
+public class BinaryExpression implements Expression {
+  public enum Op {
+    ADD("+"), SUBTRACT("-"), MULT("*"), DIV("/");
+
+    final String symbol;
+
+    Op(String symbol) {
+      this.symbol = symbol;
+    }
+
+    byte byteValue() {
+      return (byte) ordinal();
+    }
+
+    @Override
+    public String toString() {
+      return symbol;
+    }
+
+    static final Op[] VALUES = Op.values();
+
+    static Op valueOf(byte b) {
+      Preconditions.checkArgument(b < VALUES.length);
+      return VALUES[b];
+    }
+  }
+
+  private final Op op;
+  private final Expression left, right;
+
+  BinaryExpression(byte[] buf, final int offset) {
+    Preconditions.checkArgument(buf[offset] == Type.BINARY.byteValue());
+    op = Op.valueOf(buf[offset + 1]);
+    left = Utils.bytes2Expression(buf, offset + 2);
+    right = Utils.bytes2Expression(buf, offset + 2 + left.length());
+  }
+
+  public BinaryExpression(Op op, Expression left, Expression right) {
+    this.op = op;
+    this.left = left;
+    this.right = right;
+  }
+
+  @Override
+  public int toBytes(byte[] buf, final int offset) {
+    buf[offset] = Type.BINARY.byteValue();
+    buf[offset + 1] = op.byteValue();
+    final int l = left.toBytes(buf, offset + 2);
+    final int r = right.toBytes(buf, offset + 2 + l);
+    return 2 + l + r;
+  }
+
+  @Override
+  public int length() {
+    return 2 + left.length() + right.length();
+  }
+
+  @Override
+  public Double evaluate(Map<String, Double> variableMap) {
+    final double l = left.evaluate(variableMap);
+    final double r = right.evaluate(variableMap);
+    switch (op) {
+      case ADD:
+        return l + r;
+      case SUBTRACT:
+        return l - r;
+      case MULT:
+        return l * r;
+      case DIV:
+        return l / r;
+      default:
+        throw new AssertionError("Unexpected op value: " + op);
+    }
+  }
+
+  @Override
+  public String toString() {
+    return "(" + left + " " + op + " " + right + ")";
+  }
+}

http://git-wip-us.apache.org/repos/asf/incubator-ratis/blob/7e71a2e0/ratis-examples/src/main/java/org/apache/ratis/examples/arithmetic/expression/DoubleValue.java
----------------------------------------------------------------------
diff --git a/ratis-examples/src/main/java/org/apache/ratis/examples/arithmetic/expression/DoubleValue.java b/ratis-examples/src/main/java/org/apache/ratis/examples/arithmetic/expression/DoubleValue.java
new file mode 100644
index 0000000..7de3847
--- /dev/null
+++ b/ratis-examples/src/main/java/org/apache/ratis/examples/arithmetic/expression/DoubleValue.java
@@ -0,0 +1,61 @@
+/**
+ * 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.ratis.examples.arithmetic.expression;
+
+import com.google.common.base.Preconditions;
+
+import java.util.Map;
+
+public class DoubleValue implements Expression {
+  public static final DoubleValue ZERO = new DoubleValue(0);
+  public static final DoubleValue ONE = new DoubleValue(1);
+
+  private final double value;
+
+  public DoubleValue(double value) {
+    this.value = value;
+  }
+
+  DoubleValue(byte[] buf, int offset) {
+    this(Utils.bytes2double(buf, offset + 1));
+    Preconditions.checkArgument(buf[offset] == Type.DOUBLE.byteValue());
+  }
+
+  @Override
+  public int toBytes(byte[] buf, int offset) {
+    Preconditions.checkArgument(offset + length() <= buf.length);
+    buf[offset++] = Type.DOUBLE.byteValue();
+    Utils.double2bytes(value, buf, offset);
+    return length();
+  }
+
+  @Override
+  public int length() {
+    return 9;
+  }
+
+  @Override
+  public Double evaluate(Map<String, Double> variableMap) {
+    return value;
+  }
+
+  @Override
+  public String toString() {
+    return String.valueOf(value);
+  }
+}

http://git-wip-us.apache.org/repos/asf/incubator-ratis/blob/7e71a2e0/ratis-examples/src/main/java/org/apache/ratis/examples/arithmetic/expression/Expression.java
----------------------------------------------------------------------
diff --git a/ratis-examples/src/main/java/org/apache/ratis/examples/arithmetic/expression/Expression.java b/ratis-examples/src/main/java/org/apache/ratis/examples/arithmetic/expression/Expression.java
new file mode 100644
index 0000000..c90814a
--- /dev/null
+++ b/ratis-examples/src/main/java/org/apache/ratis/examples/arithmetic/expression/Expression.java
@@ -0,0 +1,112 @@
+/*
+ * 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.ratis.examples.arithmetic.expression;
+
+import com.google.common.base.Preconditions;
+
+import static org.apache.ratis.util.ProtoUtils.toByteString;
+
+import org.apache.ratis.examples.arithmetic.Evaluable;
+import org.apache.ratis.protocol.Message;
+
+public interface Expression extends Evaluable {
+  enum Type {
+    NULL, VARIABLE, DOUBLE, BINARY, UNARY;
+
+    byte byteValue() {
+      return (byte) ordinal();
+    }
+
+    private static final Type[] VALUES = Type.values();
+
+    static Type valueOf(byte b) {
+      Preconditions.checkArgument(b >= 0);
+      Preconditions.checkArgument(b < VALUES.length);
+      return VALUES[b];
+    }
+  }
+
+  int toBytes(byte[] buf, int offset);
+
+  int length();
+
+  class Utils {
+    public static Message toMessage(final Expression e) {
+      return () -> {
+        final byte[] buf = new byte[e.length()];
+        final int length = e.toBytes(buf, 0);
+        Preconditions.checkState(length == buf.length);
+        return toByteString(buf);
+      };
+    }
+
+    public static Expression double2Expression(Double d) {
+      return d == null? NullValue.getInstance(): new DoubleValue(d);
+    }
+
+    public static Expression bytes2Expression(byte[] buf, int offset) {
+      final Type type = Type.valueOf(buf[offset]);
+      switch(type) {
+        case NULL: return NullValue.getInstance();
+        case DOUBLE: return new DoubleValue(buf, offset);
+        case VARIABLE: return new Variable(buf, offset);
+        case BINARY: return new BinaryExpression(buf, offset);
+        case UNARY: return new UnaryExpression(buf, offset);
+        default:
+          throw new AssertionError("Unknown expression type " + type);
+      }
+    }
+
+    public static int int2bytes(int v, byte[] buf, int offset) {
+      buf[offset    ] = (byte) (v >>> 24);
+      buf[offset + 1] = (byte) (v >>> 16);
+      buf[offset + 2] = (byte) (v >>> 8);
+      buf[offset + 3] = (byte) (v);
+      return 4;
+    }
+
+    public static int long2bytes(long v, byte[] buf, int offset) {
+      int2bytes((int)(v >>> 32), buf, offset);
+      int2bytes((int) v        , buf, offset + 4);
+      return 8;
+    }
+
+    public static int double2bytes(double d, byte[] buf, int offset) {
+      final long v = Double.doubleToRawLongBits(d);
+      return long2bytes(v, buf, offset);
+    }
+
+    public static int bytes2int(byte[] buf, int offset) {
+      return ((int) buf[offset] << 24)
+          + ((0xFF & buf[offset + 1]) << 16)
+          + ((0xFF & buf[offset + 2]) <<  8)
+          +  (0xFF & buf[offset + 3]);
+    }
+
+    public static long bytes2long(byte[] buf, int offset) {
+      return ((long)bytes2int(buf, offset) << 32)
+          + (0xFFFFFFFFL & bytes2int(buf, offset + 4));
+    }
+
+    public static double bytes2double(byte[] buf, int offset) {
+      final long v = bytes2long(buf, offset);
+      return Double.longBitsToDouble(v);
+    }
+  }
+}
+

http://git-wip-us.apache.org/repos/asf/incubator-ratis/blob/7e71a2e0/ratis-examples/src/main/java/org/apache/ratis/examples/arithmetic/expression/NullValue.java
----------------------------------------------------------------------
diff --git a/ratis-examples/src/main/java/org/apache/ratis/examples/arithmetic/expression/NullValue.java b/ratis-examples/src/main/java/org/apache/ratis/examples/arithmetic/expression/NullValue.java
new file mode 100644
index 0000000..7f087d0
--- /dev/null
+++ b/ratis-examples/src/main/java/org/apache/ratis/examples/arithmetic/expression/NullValue.java
@@ -0,0 +1,55 @@
+/**
+ * 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.ratis.examples.arithmetic.expression;
+
+import com.google.common.base.Preconditions;
+
+import java.util.Map;
+
+public class NullValue implements Expression {
+  private static final NullValue INSTANCE = new NullValue();
+
+  public static NullValue getInstance() {
+    return INSTANCE;
+  }
+
+  private NullValue() {
+  }
+
+  @Override
+  public int toBytes(byte[] buf, int offset) {
+    Preconditions.checkArgument(offset + length() <= buf.length);
+    buf[offset++] = Type.NULL.byteValue();
+    return length();
+  }
+
+  @Override
+  public int length() {
+    return 1;
+  }
+
+  @Override
+  public Double evaluate(Map<String, Double> variableMap) {
+    return null;
+  }
+
+  @Override
+  public String toString() {
+    return "null";
+  }
+}

http://git-wip-us.apache.org/repos/asf/incubator-ratis/blob/7e71a2e0/ratis-examples/src/main/java/org/apache/ratis/examples/arithmetic/expression/UnaryExpression.java
----------------------------------------------------------------------
diff --git a/ratis-examples/src/main/java/org/apache/ratis/examples/arithmetic/expression/UnaryExpression.java b/ratis-examples/src/main/java/org/apache/ratis/examples/arithmetic/expression/UnaryExpression.java
new file mode 100644
index 0000000..c340a1e
--- /dev/null
+++ b/ratis-examples/src/main/java/org/apache/ratis/examples/arithmetic/expression/UnaryExpression.java
@@ -0,0 +1,95 @@
+/**
+ * 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.ratis.examples.arithmetic.expression;
+
+import com.google.common.base.Preconditions;
+
+import java.util.Map;
+
+public class UnaryExpression implements Expression {
+  public enum Op {
+    NEG("~"), SQRT("\u221a");
+
+    final String symbol;
+
+    Op(String symbol) {
+      this.symbol = symbol;
+    }
+
+    byte byteValue() {
+      return (byte) ordinal();
+    }
+
+    @Override
+    public String toString() {
+      return symbol;
+    }
+
+    static final Op[] VALUES = Op.values();
+
+    static Op valueOf(byte b) {
+      Preconditions.checkArgument(b < VALUES.length);
+      return VALUES[b];
+    }
+  }
+
+  final Op op;
+  final Expression expression;
+
+  UnaryExpression(byte[] buf, int offset) {
+    Preconditions.checkArgument(buf[offset] == Type.UNARY.byteValue());
+    op = Op.valueOf(buf[offset + 1]);
+    expression = Utils.bytes2Expression(buf, offset + 2);
+  }
+
+  public UnaryExpression(Op op, Expression expression) {
+    this.op = op;
+    this.expression = expression;
+  }
+
+  @Override
+  public int toBytes(byte[] buf, int offset) {
+    buf[offset] = Type.UNARY.byteValue();
+    buf[offset + 1] = op.byteValue();
+    final int length = expression.toBytes(buf, offset + 2);
+    return 2 + length;
+  }
+
+  @Override
+  public int length() {
+    return 2 + expression.length();
+  }
+
+  @Override
+  public Double evaluate(Map<String, Double> variableMap) {
+    final double value = expression.evaluate(variableMap);
+    switch (op) {
+      case NEG:
+        return -value;
+      case SQRT:
+        return Math.sqrt(value);
+      default:
+        throw new AssertionError("Unexpected op value: " + op);
+    }
+  }
+
+  @Override
+  public String toString() {
+    return op + " " + expression;
+  }
+}

http://git-wip-us.apache.org/repos/asf/incubator-ratis/blob/7e71a2e0/ratis-examples/src/main/java/org/apache/ratis/examples/arithmetic/expression/Variable.java
----------------------------------------------------------------------
diff --git a/ratis-examples/src/main/java/org/apache/ratis/examples/arithmetic/expression/Variable.java b/ratis-examples/src/main/java/org/apache/ratis/examples/arithmetic/expression/Variable.java
new file mode 100644
index 0000000..4e9a834
--- /dev/null
+++ b/ratis-examples/src/main/java/org/apache/ratis/examples/arithmetic/expression/Variable.java
@@ -0,0 +1,126 @@
+/**
+ * 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.ratis.examples.arithmetic.expression;
+
+import com.google.common.base.Preconditions;
+
+import java.util.Map;
+import java.util.regex.Pattern;
+
+import org.apache.ratis.examples.arithmetic.AssignmentMessage;
+
+public class Variable implements Expression {
+  static final int LENGTH_LIMIT = 32;
+  static final String REGEX = "[a-zA-Z]\\w*";
+  static final Pattern PATTERN = Pattern.compile(REGEX);
+
+  static byte[] string2bytes(String s) {
+    final byte[] stringBytes = s.getBytes(AssignmentMessage.UTF8);
+    final byte[] bytes = new byte[stringBytes.length + 2];
+    bytes[0] = Type.VARIABLE.byteValue();
+    bytes[1] = (byte)stringBytes.length;
+    System.arraycopy(stringBytes, 0, bytes, 2, stringBytes.length);
+    return bytes;
+  }
+
+  static String extractString(byte[] buf, int offset) {
+    Preconditions.checkArgument(buf[offset] == Type.VARIABLE.byteValue());
+    final int length = buf[offset + 1];
+    final byte[] stringBytes = new byte[length];
+    System.arraycopy(buf, offset + 2, stringBytes, 0, length);
+    return new String(stringBytes, AssignmentMessage.UTF8);
+  }
+
+  static byte[] copyBytes(byte[] buf, int offset) {
+    Preconditions.checkArgument(buf[offset] == Type.VARIABLE.byteValue());
+    final int length = buf[offset + 1];
+    final byte[] copy = new byte[length + 2];
+    System.arraycopy(buf, offset, copy, 0, copy.length);
+    return copy;
+  }
+
+  private final String name;
+  private final byte[] encoded;
+
+  private Variable(String name, byte[] encoded) {
+    this.name = name;
+    this.encoded = encoded;
+
+    if (!PATTERN.matcher(name).matches()) {
+      throw new IllegalArgumentException("The variable name \"" + name
+          + "\" does not match the pattern \"" + PATTERN + "\"");
+    }
+    if (encoded.length - 2 > LENGTH_LIMIT) {
+      throw new IllegalArgumentException("The variable name \"" + name
+          + "\" is longer than the limit = " + LENGTH_LIMIT);
+    }
+  }
+
+  public Variable(byte[] buf, int offset) {
+    this(extractString(buf, offset), copyBytes(buf, offset));
+  }
+
+  public Variable(String name) {
+    this(name, string2bytes(name));
+  }
+
+  public String getName() {
+    return name;
+  }
+
+  @Override
+  public int toBytes(byte[] buf, int offset) {
+    System.arraycopy(encoded, 0, buf, offset, encoded.length);
+    return encoded.length;
+  }
+
+  @Override
+  public int length() {
+    return encoded.length;
+  }
+
+  @Override
+  public Double evaluate(Map<String, Double> variableMap) {
+    final Double value = variableMap.get(name);
+    if (value == null) {
+      throw new IllegalStateException("Undefined variable \"" + name + "\"");
+    }
+    return value;
+  }
+
+  @Override
+  public boolean equals(Object obj) {
+    if (obj == this) {
+      return true;
+    } else if (obj == null || !(obj instanceof Variable)) {
+      return false;
+    }
+    final Variable that = (Variable)obj;
+    return this.getName().equals(that.getName());
+  }
+
+  @Override
+  public int hashCode() {
+    return name.hashCode();
+  }
+
+  @Override
+  public String toString() {
+    return name;
+  }
+}

http://git-wip-us.apache.org/repos/asf/incubator-ratis/blob/7e71a2e0/ratis-examples/src/test/java/org/apache/ratis/TestBatchAppend.java
----------------------------------------------------------------------
diff --git a/ratis-examples/src/test/java/org/apache/ratis/TestBatchAppend.java b/ratis-examples/src/test/java/org/apache/ratis/TestBatchAppend.java
new file mode 100644
index 0000000..15f65a9
--- /dev/null
+++ b/ratis-examples/src/test/java/org/apache/ratis/TestBatchAppend.java
@@ -0,0 +1,172 @@
+/**
+ * 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.ratis;
+
+import org.apache.log4j.Level;
+import org.apache.ratis.MiniRaftCluster;
+import org.apache.ratis.RaftTestUtil;
+import org.apache.ratis.RaftTestUtil.SimpleMessage;
+import org.apache.ratis.client.RaftClient;
+import org.apache.ratis.conf.RaftProperties;
+import org.apache.ratis.examples.RaftExamplesTestUtil;
+import org.apache.ratis.server.impl.RaftServerImpl;
+import org.apache.ratis.server.simulation.RequestHandler;
+import org.apache.ratis.statemachine.SimpleStateMachine4Testing;
+import org.apache.ratis.statemachine.StateMachine;
+import org.apache.ratis.util.RaftUtils;
+import org.junit.After;
+import org.junit.Assert;
+import org.junit.Rule;
+import org.junit.Test;
+import org.junit.rules.Timeout;
+import org.junit.runner.RunWith;
+import org.junit.runners.Parameterized;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import static org.apache.ratis.server.RaftServerConfigKeys.*;
+
+import java.io.IOException;
+import java.util.Arrays;
+import java.util.Collection;
+import java.util.List;
+import java.util.concurrent.CountDownLatch;
+import java.util.concurrent.atomic.AtomicBoolean;
+import java.util.stream.Collectors;
+import java.util.stream.Stream;
+
+/**
+ * Enable raft.server.log.appender.batch.enabled and test LogAppender
+ */
+@RunWith(Parameterized.class)
+public class TestBatchAppend {
+  static Logger LOG = LoggerFactory.getLogger(TestBatchAppend.class);
+  static {
+    RaftUtils.setLogLevel(RaftServerImpl.LOG, Level.DEBUG);
+    RaftUtils.setLogLevel(RequestHandler.LOG, Level.DEBUG);
+    RaftUtils.setLogLevel(RaftClient.LOG, Level.DEBUG);
+  }
+
+  @Parameterized.Parameters
+  public static Collection<Object[]> data() throws IOException {
+    RaftProperties prop = new RaftProperties();
+    prop.setClass(MiniRaftCluster.STATEMACHINE_CLASS_KEY,
+        SimpleStateMachine4Testing.class, StateMachine.class);
+    prop.setInt(RAFT_LOG_SEGMENT_MAX_SIZE_KEY, 1024 * 8);
+    // enable batch appending
+    prop.setBoolean(RAFT_SERVER_LOG_APPENDER_BATCH_ENABLED_KEY, true);
+    // set batch appending buffer size to 4KB
+    prop.setInt(RAFT_SERVER_LOG_APPENDER_BUFFER_CAPACITY_KEY, 4 * 1024);
+
+    return RaftExamplesTestUtil.getMiniRaftClusters(prop, 3);
+  }
+
+  @Parameterized.Parameter
+  public MiniRaftCluster cluster;
+
+  @Rule
+  public Timeout globalTimeout = new Timeout(60 * 1000);
+
+  @After
+  public void tearDown() {
+    if (cluster != null) {
+      cluster.shutdown();
+    }
+  }
+
+  private class Sender extends Thread {
+    private final RaftClient client;
+    private final CountDownLatch latch;
+    private final SimpleMessage[] msgs;
+    private final AtomicBoolean succeed = new AtomicBoolean(false);
+
+    Sender(String clientId, String leaderId, CountDownLatch latch, int numMsg) {
+      this.latch = latch;
+      this.client = cluster.createClient(clientId, leaderId);
+      msgs = generateMsgs(numMsg);
+    }
+
+    SimpleMessage[] generateMsgs(int num) {
+      SimpleMessage[] msgs = new SimpleMessage[num * 6];
+      for (int i = 0; i < num; i++) {
+        for (int j = 0; j < 6; j++) {
+          byte[] bytes = new byte[1024 * (j + 1)];
+          Arrays.fill(bytes, (byte) j);
+          msgs[i * 6 + j] = new SimpleMessage(new String(bytes));
+        }
+      }
+      return msgs;
+    }
+
+    @Override
+    public void run() {
+      try {
+        latch.await();
+      } catch (InterruptedException ignored) {
+        LOG.warn("Client {} waiting for countdown latch got interrupted",
+            client.getId());
+      }
+      for (SimpleMessage msg : msgs) {
+        try {
+          client.send(msg);
+        } catch (IOException e) {
+          succeed.set(false);
+          LOG.warn("Client {} hit exception {}", client.getId(), e);
+          return;
+        }
+      }
+      succeed.set(true);
+      try {
+        client.close();
+      } catch (IOException ignore) {
+      }
+    }
+  }
+
+  @Test
+  public void testAppend() throws Exception {
+    final int numMsgs = 10;
+    final int numClients = 5;
+    cluster.start();
+    RaftTestUtil.waitForLeader(cluster);
+    final String leaderId = cluster.getLeader().getId();
+
+    // start several clients and write concurrently
+    CountDownLatch latch = new CountDownLatch(1);
+    final List<Sender> senders = Stream.iterate(0, i -> i+1).limit(numClients)
+        .map(i -> new Sender("c" + i, leaderId, latch, numMsgs))
+        .collect(Collectors.toList());
+    senders.forEach(Thread::start);
+
+    latch.countDown();
+
+    senders.forEach(sender -> {
+      try {
+        sender.join();
+      } catch (InterruptedException ignored) {
+      }
+    });
+
+    for (Sender s : senders) {
+      Assert.assertTrue(s.succeed.get());
+    }
+
+    Assert.assertEquals(6 * numMsgs * numClients,
+        cluster.getLeader().getState().getLastAppliedIndex());
+  }
+}


[19/54] [abbrv] incubator-ratis git commit: Renamed the packages from raft to ratis in preperation for Apache Incubation - Moved all java packages from org.apache.raft to org.apache.ratis. - Moved native package to org_apache_ratis, and native lib to l

Posted by ji...@apache.org.
http://git-wip-us.apache.org/repos/asf/incubator-ratis/blob/7e71a2e0/ratis-server/src/main/java/org/apache/ratis/server/storage/LogOutputStream.java
----------------------------------------------------------------------
diff --git a/ratis-server/src/main/java/org/apache/ratis/server/storage/LogOutputStream.java b/ratis-server/src/main/java/org/apache/ratis/server/storage/LogOutputStream.java
new file mode 100644
index 0000000..6ed70fa
--- /dev/null
+++ b/ratis-server/src/main/java/org/apache/ratis/server/storage/LogOutputStream.java
@@ -0,0 +1,181 @@
+/**
+ * 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.ratis.server.storage;
+
+import static org.apache.ratis.server.RaftServerConfigKeys.RAFT_LOG_SEGMENT_MAX_SIZE_DEFAULT;
+import static org.apache.ratis.server.RaftServerConfigKeys.RAFT_LOG_SEGMENT_MAX_SIZE_KEY;
+import static org.apache.ratis.server.RaftServerConfigKeys.RAFT_LOG_SEGMENT_PREALLOCATED_SIZE_DEFAULT;
+import static org.apache.ratis.server.RaftServerConfigKeys.RAFT_LOG_SEGMENT_PREALLOCATED_SIZE_KEY;
+import static org.apache.ratis.server.RaftServerConfigKeys.RAFT_LOG_WRITE_BUFFER_SIZE_DEFAULT;
+import static org.apache.ratis.server.RaftServerConfigKeys.RAFT_LOG_WRITE_BUFFER_SIZE_KEY;
+
+import java.io.Closeable;
+import java.io.File;
+import java.io.IOException;
+import java.io.RandomAccessFile;
+import java.nio.ByteBuffer;
+import java.nio.channels.FileChannel;
+import java.util.zip.Checksum;
+
+import org.apache.ratis.conf.RaftProperties;
+import org.apache.ratis.server.impl.RaftServerConstants;
+import org.apache.ratis.shaded.com.google.protobuf.CodedOutputStream;
+import org.apache.ratis.shaded.proto.RaftProtos.LogEntryProto;
+import org.apache.ratis.util.PureJavaCrc32C;
+import org.apache.ratis.util.RaftUtils;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+public class LogOutputStream implements Closeable {
+  private static final Logger LOG = LoggerFactory.getLogger(LogOutputStream.class);
+
+  private static final ByteBuffer fill;
+  private static final int BUFFER_SIZE = 1024 * 1024; // 1 MB
+  static {
+    fill = ByteBuffer.allocateDirect(BUFFER_SIZE);
+    fill.position(0);
+    for (int i = 0; i < fill.capacity(); i++) {
+      fill.put(RaftServerConstants.LOG_TERMINATE_BYTE);
+    }
+  }
+
+  private File file;
+  private FileChannel fc; // channel of the file stream for sync
+  private BufferedWriteChannel out; // buffered FileChannel for writing
+  private final Checksum checksum;
+
+  private final long segmentMaxSize;
+  private final long preallocatedSize;
+  private long preallocatedPos;
+
+  public LogOutputStream(File file, boolean append, RaftProperties properties)
+      throws IOException {
+    this.file = file;
+    this.checksum = new PureJavaCrc32C();
+    this.segmentMaxSize = properties.getLong(RAFT_LOG_SEGMENT_MAX_SIZE_KEY,
+        RAFT_LOG_SEGMENT_MAX_SIZE_DEFAULT);
+    this.preallocatedSize = properties.getLong(
+        RAFT_LOG_SEGMENT_PREALLOCATED_SIZE_KEY,
+        RAFT_LOG_SEGMENT_PREALLOCATED_SIZE_DEFAULT);
+    RandomAccessFile rp = new RandomAccessFile(file, "rw");
+    fc = rp.getChannel();
+    fc.position(fc.size());
+    preallocatedPos = fc.size();
+
+    int bufferSize = properties.getInt(RAFT_LOG_WRITE_BUFFER_SIZE_KEY,
+        RAFT_LOG_WRITE_BUFFER_SIZE_DEFAULT);
+    out = new BufferedWriteChannel(fc, bufferSize);
+
+    if (!append) {
+      create();
+    }
+  }
+
+  /**
+   * Format:
+   * LogEntryProto's protobuf
+   * 4-byte checksum of the above protobuf
+   */
+  public void write(LogEntryProto entry) throws IOException {
+    final int serialized = entry.getSerializedSize();
+    final int bufferSize = CodedOutputStream.computeUInt32SizeNoTag(serialized)
+        + serialized;
+
+    preallocateIfNecessary(bufferSize + 4);
+
+    byte[] buf = new byte[bufferSize];
+    CodedOutputStream cout = CodedOutputStream.newInstance(buf);
+    cout.writeUInt32NoTag(serialized);
+    entry.writeTo(cout);
+
+    checksum.reset();
+    checksum.update(buf, 0, buf.length);
+    final int sum = (int) checksum.getValue();
+
+    out.write(buf);
+    writeInt(sum);
+  }
+
+  private void writeInt(int v) throws IOException {
+    out.write((v >>> 24) & 0xFF);
+    out.write((v >>> 16) & 0xFF);
+    out.write((v >>>  8) & 0xFF);
+    out.write((v) & 0xFF);
+  }
+
+  private void create() throws IOException {
+    fc.truncate(0);
+    fc.position(0);
+    preallocatedPos = 0;
+    preallocate(); // preallocate file
+
+    out.write(SegmentedRaftLog.HEADER_BYTES);
+    flush();
+  }
+
+  @Override
+  public void close() throws IOException {
+    try {
+      out.flush(false);
+      if (fc != null && fc.isOpen()) {
+        fc.truncate(fc.position());
+      }
+    } finally {
+      RaftUtils.cleanup(LOG, fc, out);
+      fc = null;
+      out = null;
+    }
+  }
+
+  /**
+   * Flush data to persistent store.
+   * Collect sync metrics.
+   */
+  public void flush() throws IOException {
+    if (out == null) {
+      throw new IOException("Trying to use aborted output stream");
+    }
+    out.flush(true);
+  }
+
+  private void preallocate() throws IOException {
+    fill.position(0);
+    long targetSize = Math.min(segmentMaxSize - fc.size(), preallocatedSize);
+    int allocated = 0;
+    while (allocated < targetSize) {
+      int size = (int) Math.min(BUFFER_SIZE, targetSize - allocated);
+      ByteBuffer buffer = fill.slice();
+      buffer.limit(size);
+      RaftUtils.writeFully(fc, buffer, preallocatedPos);
+      preallocatedPos += size;
+      allocated += size;
+    }
+    LOG.debug("Pre-allocated {} bytes for the log segment", allocated);
+  }
+
+  private void preallocateIfNecessary(int size) throws IOException {
+    if (out.position() + size > preallocatedPos) {
+      preallocate();
+    }
+  }
+
+  @Override
+  public String toString() {
+    return this.getClass().getSimpleName() + "(" + file + ")";
+  }
+}

http://git-wip-us.apache.org/repos/asf/incubator-ratis/blob/7e71a2e0/ratis-server/src/main/java/org/apache/ratis/server/storage/LogReader.java
----------------------------------------------------------------------
diff --git a/ratis-server/src/main/java/org/apache/ratis/server/storage/LogReader.java b/ratis-server/src/main/java/org/apache/ratis/server/storage/LogReader.java
new file mode 100644
index 0000000..0e5a168
--- /dev/null
+++ b/ratis-server/src/main/java/org/apache/ratis/server/storage/LogReader.java
@@ -0,0 +1,302 @@
+/**
+ * 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.ratis.server.storage;
+
+import java.io.BufferedInputStream;
+import java.io.Closeable;
+import java.io.DataInputStream;
+import java.io.EOFException;
+import java.io.File;
+import java.io.FileInputStream;
+import java.io.FileNotFoundException;
+import java.io.FilterInputStream;
+import java.io.IOException;
+import java.io.InputStream;
+import java.util.zip.Checksum;
+
+import org.apache.commons.io.Charsets;
+import org.apache.ratis.protocol.ChecksumException;
+import org.apache.ratis.server.impl.RaftServerConstants;
+import org.apache.ratis.shaded.com.google.protobuf.CodedInputStream;
+import org.apache.ratis.shaded.com.google.protobuf.CodedOutputStream;
+import org.apache.ratis.shaded.proto.RaftProtos.LogEntryProto;
+import org.apache.ratis.util.PureJavaCrc32C;
+import org.apache.ratis.util.RaftUtils;
+
+import com.google.common.base.Preconditions;
+
+public class LogReader implements Closeable {
+  /**
+   * InputStream wrapper that keeps track of the current stream position.
+   *
+   * This stream also allows us to set a limit on how many bytes we can read
+   * without getting an exception.
+   */
+  public static class LimitedInputStream extends FilterInputStream {
+    private long curPos = 0;
+    private long markPos = -1;
+    private long limitPos = Long.MAX_VALUE;
+
+    public LimitedInputStream(InputStream is) {
+      super(is);
+    }
+
+    private void checkLimit(long amt) throws IOException {
+      long extra = (curPos + amt) - limitPos;
+      if (extra > 0) {
+        throw new IOException("Tried to read " + amt + " byte(s) past " +
+            "the limit at offset " + limitPos);
+      }
+    }
+
+    @Override
+    public int read() throws IOException {
+      checkLimit(1);
+      int ret = super.read();
+      if (ret != -1) curPos++;
+      return ret;
+    }
+
+    @Override
+    public int read(byte[] data) throws IOException {
+      checkLimit(data.length);
+      int ret = super.read(data);
+      if (ret > 0) curPos += ret;
+      return ret;
+    }
+
+    @Override
+    public int read(byte[] data, int offset, int length) throws IOException {
+      checkLimit(length);
+      int ret = super.read(data, offset, length);
+      if (ret > 0) curPos += ret;
+      return ret;
+    }
+
+    public void setLimit(long limit) {
+      limitPos = curPos + limit;
+    }
+
+    public void clearLimit() {
+      limitPos = Long.MAX_VALUE;
+    }
+
+    @Override
+    public void mark(int limit) {
+      super.mark(limit);
+      markPos = curPos;
+    }
+
+    @Override
+    public void reset() throws IOException {
+      if (markPos == -1) {
+        throw new IOException("Not marked!");
+      }
+      super.reset();
+      curPos = markPos;
+      markPos = -1;
+    }
+
+    public long getPos() {
+      return curPos;
+    }
+
+    @Override
+    public long skip(long amt) throws IOException {
+      long extra = (curPos + amt) - limitPos;
+      if (extra > 0) {
+        throw new IOException("Tried to skip " + extra + " bytes past " +
+            "the limit at offset " + limitPos);
+      }
+      long ret = super.skip(amt);
+      curPos += ret;
+      return ret;
+    }
+  }
+
+  private static final int maxOpSize = 32 * 1024 * 1024;
+
+  private final LimitedInputStream limiter;
+  private final DataInputStream in;
+  private byte[] temp = new byte[4096];
+  private final Checksum checksum;
+
+  LogReader(File file) throws FileNotFoundException {
+    this.limiter = new LimitedInputStream(
+        new BufferedInputStream(new FileInputStream(file)));
+    in = new DataInputStream(limiter);
+    checksum = new PureJavaCrc32C();
+  }
+
+  String readLogHeader() throws IOException {
+    byte[] header = new byte[SegmentedRaftLog.HEADER_BYTES.length];
+    int num = in.read(header);
+    if (num < header.length) {
+      throw new EOFException("EOF before reading a complete log header");
+    }
+    return new String(header, Charsets.UTF_8);
+  }
+
+  /**
+   * Read a log entry from the input stream.
+   *
+   * @return the operation read from the stream, or null at the end of the
+   *         file
+   * @throws IOException on error.  This function should only throw an
+   *         exception when skipBrokenEdits is false.
+   */
+  LogEntryProto readEntry() throws IOException {
+    try {
+      return decodeEntry();
+    } catch (IOException e) {
+      in.reset();
+
+      throw e;
+    } catch (Throwable e) {
+      // raft log requires no gap between any two entries. thus if an entry is
+      // broken, throw the exception instead of skipping broken entries
+      in.reset();
+      throw new IOException("got unexpected exception " + e.getMessage(), e);
+    }
+  }
+
+  /**
+   * Scan and validate a log entry.
+   * @return the index of the log entry
+   */
+  long scanEntry() throws IOException {
+    LogEntryProto entry = decodeEntry();
+    return entry != null ? entry.getIndex() : RaftServerConstants.INVALID_LOG_INDEX;
+  }
+
+  void verifyTerminator() throws IOException {
+     // The end of the log should contain 0x00 bytes.
+     // If it contains other bytes, the log itself may be corrupt.
+    limiter.clearLimit();
+    int numRead = -1, idx = 0;
+    while (true) {
+      try {
+        numRead = -1;
+        numRead = in.read(temp);
+        if (numRead == -1) {
+          return;
+        }
+        for (idx = 0; idx < numRead; idx++) {
+          if (temp[idx] != RaftServerConstants.LOG_TERMINATE_BYTE) {
+            throw new IOException("Read extra bytes after the terminator!");
+          }
+        }
+      } finally {
+        // After reading each group of bytes, we reposition the mark one
+        // byte before the next group. Similarly, if there is an error, we
+        // want to reposition the mark one byte before the error
+        if (numRead != -1) {
+          in.reset();
+          RaftUtils.skipFully(in, idx);
+          in.mark(temp.length + 1);
+          RaftUtils.skipFully(in, 1);
+        }
+      }
+    }
+  }
+
+  /**
+   * Decode the log entry "frame". This includes reading the log entry, and
+   * validating the checksum.
+   *
+   * The input stream will be advanced to the end of the op at the end of this
+   * function.
+   *
+   * @return The log entry, or null if we hit EOF.
+   */
+  private LogEntryProto decodeEntry() throws IOException {
+    limiter.setLimit(maxOpSize);
+    in.mark(maxOpSize);
+
+    byte nextByte;
+    try {
+      nextByte = in.readByte();
+    } catch (EOFException eof) {
+      // EOF at an opcode boundary is expected.
+      return null;
+    }
+    // Each log entry starts with a var-int. Thus a valid entry's first byte
+    // should not be 0. So if the terminate byte is 0, we should hit the end
+    // of the segment.
+    if (nextByte == RaftServerConstants.LOG_TERMINATE_BYTE) {
+      verifyTerminator();
+      return null;
+    }
+
+    // Here, we verify that the Op size makes sense and that the
+    // data matches its checksum before attempting to construct an Op.
+    int entryLength = CodedInputStream.readRawVarint32(nextByte, in);
+    if (entryLength > maxOpSize) {
+      throw new IOException("Entry has size " + entryLength
+          + ", but maxOpSize = " + maxOpSize);
+    }
+
+    final int varintLength = CodedOutputStream.computeUInt32SizeNoTag(
+        entryLength);
+    final int totalLength = varintLength + entryLength;
+    checkBufferSize(totalLength);
+    in.reset();
+    in.mark(maxOpSize);
+    RaftUtils.readFully(in, temp, 0, totalLength);
+
+    // verify checksum
+    checksum.reset();
+    checksum.update(temp, 0, totalLength);
+    int expectedChecksum = in.readInt();
+    int calculatedChecksum = (int) checksum.getValue();
+    if (expectedChecksum != calculatedChecksum) {
+      throw new ChecksumException("LogEntry is corrupt. Calculated checksum is "
+          + calculatedChecksum + " but read checksum " + expectedChecksum,
+          limiter.markPos);
+    }
+
+    // parse the buffer
+    return LogEntryProto.parseFrom(
+        CodedInputStream.newInstance(temp, varintLength, entryLength));
+  }
+
+  private void checkBufferSize(int entryLength) {
+    Preconditions.checkArgument(entryLength <= maxOpSize);
+    int length = temp.length;
+    if (length < entryLength) {
+      while (length < entryLength) {
+        length = Math.min(length * 2, maxOpSize);
+      }
+      temp = new byte[length];
+    }
+  }
+
+  long getPos() {
+    return limiter.getPos();
+  }
+
+  void skipFully(long length) throws IOException {
+    limiter.clearLimit();
+    RaftUtils.skipFully(limiter, length);
+  }
+
+  @Override
+  public void close() throws IOException {
+    RaftUtils.cleanup(null, in);
+  }
+}

http://git-wip-us.apache.org/repos/asf/incubator-ratis/blob/7e71a2e0/ratis-server/src/main/java/org/apache/ratis/server/storage/LogSegment.java
----------------------------------------------------------------------
diff --git a/ratis-server/src/main/java/org/apache/ratis/server/storage/LogSegment.java b/ratis-server/src/main/java/org/apache/ratis/server/storage/LogSegment.java
new file mode 100644
index 0000000..af9ee66
--- /dev/null
+++ b/ratis-server/src/main/java/org/apache/ratis/server/storage/LogSegment.java
@@ -0,0 +1,233 @@
+/**
+ * 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.ratis.server.storage;
+
+import static org.apache.ratis.shaded.proto.RaftProtos.LogEntryProto.LogEntryBodyCase.CONFIGURATIONENTRY;
+
+import java.io.File;
+import java.io.IOException;
+import java.util.ArrayList;
+import java.util.List;
+
+import org.apache.ratis.server.impl.ConfigurationManager;
+import org.apache.ratis.server.impl.ServerProtoUtils;
+import org.apache.ratis.shaded.com.google.protobuf.CodedOutputStream;
+import org.apache.ratis.shaded.proto.RaftProtos.LogEntryProto;
+import org.apache.ratis.util.FileUtils;
+
+import com.google.common.base.Preconditions;
+
+/**
+ * In-memory cache for a log segment file. All the updates will be first written
+ * into LogSegment then into corresponding files in the same order.
+ *
+ * This class will be protected by the RaftServer's lock.
+ */
+class LogSegment implements Comparable<Long> {
+  static class LogRecord {
+    /** starting offset in the file */
+    final long offset;
+    final LogEntryProto entry;
+
+    LogRecord(long offset, LogEntryProto entry) {
+      this.offset = offset;
+      this.entry = entry;
+    }
+  }
+
+  static class SegmentFileInfo {
+    final long startIndex; // start index of the
+    final long endIndex; // original end index
+    final boolean isOpen;
+    final long targetLength; // position for truncation
+    final long newEndIndex; // new end index after the truncation
+
+    SegmentFileInfo(long start, long end, boolean isOpen, long targetLength,
+        long newEndIndex) {
+      this.startIndex = start;
+      this.endIndex = end;
+      this.isOpen = isOpen;
+      this.targetLength = targetLength;
+      this.newEndIndex = newEndIndex;
+    }
+  }
+
+  static long getEntrySize(LogEntryProto entry) {
+    final int serialized = entry.getSerializedSize();
+    return serialized + CodedOutputStream.computeUInt32SizeNoTag(serialized) + 4;
+  }
+
+  private boolean isOpen;
+  private final List<LogRecord> records = new ArrayList<>();
+  private long totalSize;
+  private final long startIndex;
+  private long endIndex;
+
+  private LogSegment(boolean isOpen, long start, long end) {
+    this.isOpen = isOpen;
+    this.startIndex = start;
+    this.endIndex = end;
+    totalSize = SegmentedRaftLog.HEADER_BYTES.length;
+  }
+
+  static LogSegment newOpenSegment(long start) {
+    Preconditions.checkArgument(start >= 0);
+    return new LogSegment(true, start, start - 1);
+  }
+
+  private static LogSegment newCloseSegment(long start, long end) {
+    Preconditions.checkArgument(start >= 0 && end >= start);
+    return new LogSegment(false, start, end);
+  }
+
+  static LogSegment loadSegment(File file, long start, long end, boolean isOpen,
+      ConfigurationManager confManager) throws IOException {
+    final LogSegment segment;
+    try (LogInputStream in = new LogInputStream(file, start, end, isOpen)) {
+      segment = isOpen ? LogSegment.newOpenSegment(start) :
+          LogSegment.newCloseSegment(start, end);
+      LogEntryProto next;
+      LogEntryProto prev = null;
+      while ((next = in.nextEntry()) != null) {
+        if (prev != null) {
+          Preconditions.checkState(next.getIndex() == prev.getIndex() + 1,
+              "gap between entry %s and entry %s", prev, next);
+        }
+        segment.append(next);
+        if (confManager != null &&
+            next.getLogEntryBodyCase() == CONFIGURATIONENTRY) {
+          confManager.addConfiguration(next.getIndex(),
+              ServerProtoUtils.toRaftConfiguration(next.getIndex(),
+                  next.getConfigurationEntry()));
+        }
+        prev = next;
+      }
+    }
+
+    // truncate padding if necessary
+    if (file.length() > segment.getTotalSize()) {
+      FileUtils.truncateFile(file, segment.getTotalSize());
+    }
+
+    Preconditions.checkState(start == segment.records.get(0).entry.getIndex());
+    if (!isOpen) {
+      Preconditions.checkState(segment.getEndIndex() == end);
+    }
+    return segment;
+  }
+
+  long getStartIndex() {
+    return startIndex;
+  }
+
+  long getEndIndex() {
+    return endIndex;
+  }
+
+  boolean isOpen() {
+    return isOpen;
+  }
+
+  int numOfEntries() {
+    return (int) (endIndex - startIndex + 1);
+  }
+
+  void appendToOpenSegment(LogEntryProto... entries) {
+    Preconditions.checkState(isOpen(),
+        "The log segment %s is not open for append", this.toString());
+    append(entries);
+  }
+
+  private void append(LogEntryProto... entries) {
+    Preconditions.checkArgument(entries != null && entries.length > 0);
+    final long term = entries[0].getTerm();
+    if (records.isEmpty()) {
+      Preconditions.checkArgument(entries[0].getIndex() == startIndex,
+          "gap between start index %s and first entry to append %s",
+          startIndex, entries[0].getIndex());
+    }
+    for (LogEntryProto entry : entries) {
+      // all these entries should be of the same term
+      Preconditions.checkArgument(entry.getTerm() == term,
+          "expected term:%s, term of the entry:%s", term, entry.getTerm());
+      final LogRecord currentLast = getLastRecord();
+      if (currentLast != null) {
+        Preconditions.checkArgument(
+            entry.getIndex() == currentLast.entry.getIndex() + 1,
+            "gap between entries %s and %s", entry.getIndex(),
+            currentLast.entry.getIndex());
+      }
+
+      final LogRecord record = new LogRecord(totalSize, entry);
+      records.add(record);
+      totalSize += getEntrySize(entry);
+      endIndex = entry.getIndex();
+    }
+  }
+
+  LogRecord getLogRecord(long index) {
+    if (index >= startIndex && index <= endIndex) {
+      return records.get((int) (index - startIndex));
+    }
+    return null;
+  }
+
+  LogRecord getLastRecord() {
+    return records.isEmpty() ? null : records.get(records.size() - 1);
+  }
+
+  long getTotalSize() {
+    return totalSize;
+  }
+
+  /**
+   * Remove records from the given index (inclusive)
+   */
+  void truncate(long fromIndex) {
+    Preconditions.checkArgument(fromIndex >= startIndex && fromIndex <= endIndex);
+    LogRecord record = records.get((int) (fromIndex - startIndex));
+    for (long index = endIndex; index >= fromIndex; index--) {
+      records.remove((int)(index - startIndex));
+    }
+    totalSize = record.offset;
+    isOpen = false;
+    this.endIndex = fromIndex - 1;
+  }
+
+  void close() {
+    Preconditions.checkState(isOpen());
+    isOpen = false;
+  }
+
+  @Override
+  public String toString() {
+    return isOpen() ? "log-" + startIndex + "-inprogress" :
+        "log-" + startIndex + "-" + endIndex;
+  }
+
+  @Override
+  public int compareTo(Long l) {
+    return (l >= getStartIndex() && l <= getEndIndex()) ? 0 :
+        (this.getEndIndex() < l ? -1 : 1);
+  }
+
+  void clear() {
+    records.clear();
+    endIndex = startIndex - 1;
+  }
+}

http://git-wip-us.apache.org/repos/asf/incubator-ratis/blob/7e71a2e0/ratis-server/src/main/java/org/apache/ratis/server/storage/MemoryRaftLog.java
----------------------------------------------------------------------
diff --git a/ratis-server/src/main/java/org/apache/ratis/server/storage/MemoryRaftLog.java b/ratis-server/src/main/java/org/apache/ratis/server/storage/MemoryRaftLog.java
new file mode 100644
index 0000000..8a275ec
--- /dev/null
+++ b/ratis-server/src/main/java/org/apache/ratis/server/storage/MemoryRaftLog.java
@@ -0,0 +1,183 @@
+/**
+ * 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.ratis.server.storage;
+
+import java.util.ArrayList;
+import java.util.List;
+
+import org.apache.ratis.server.impl.RaftConfiguration;
+import org.apache.ratis.server.impl.RaftServerConstants;
+import org.apache.ratis.server.impl.ServerProtoUtils;
+import org.apache.ratis.shaded.proto.RaftProtos.LogEntryProto;
+import org.apache.ratis.util.AutoCloseableLock;
+import org.apache.ratis.util.CodeInjectionForTesting;
+
+import com.google.common.base.Preconditions;
+
+/**
+ * A simple RaftLog implementation in memory. Used only for testing.
+ */
+public class MemoryRaftLog extends RaftLog {
+  private final List<LogEntryProto> entries = new ArrayList<>();
+
+  public MemoryRaftLog(String selfId) {
+    super(selfId);
+  }
+
+  @Override
+  public LogEntryProto get(long index) {
+    checkLogState();
+    try(AutoCloseableLock readLock = readLock()) {
+      final int i = (int) index;
+      return i >= 0 && i < entries.size() ? entries.get(i) : null;
+    }
+  }
+
+  @Override
+  public LogEntryProto[] getEntries(long startIndex, long endIndex) {
+    checkLogState();
+    try(AutoCloseableLock readLock = readLock()) {
+      final int i = (int) startIndex;
+      if (startIndex >= entries.size()) {
+        return null;
+      }
+      final int toIndex = (int) Math.min(entries.size(), endIndex);
+      return entries.subList(i, toIndex).toArray(EMPTY_LOGENTRY_ARRAY);
+    }
+  }
+
+  @Override
+  void truncate(long index) {
+    checkLogState();
+    try(AutoCloseableLock writeLock = writeLock()) {
+      Preconditions.checkArgument(index >= 0);
+      final int truncateIndex = (int) index;
+      for (int i = entries.size() - 1; i >= truncateIndex; i--) {
+        entries.remove(i);
+      }
+    }
+  }
+
+  @Override
+  public LogEntryProto getLastEntry() {
+    checkLogState();
+    try(AutoCloseableLock readLock = readLock()) {
+      final int size = entries.size();
+      return size == 0 ? null : entries.get(size - 1);
+    }
+  }
+
+  @Override
+  void appendEntry(LogEntryProto entry) {
+    checkLogState();
+    try(AutoCloseableLock writeLock = writeLock()) {
+      entries.add(entry);
+    }
+  }
+
+  @Override
+  public long append(long term, RaftConfiguration newConf) {
+    checkLogState();
+    try(AutoCloseableLock writeLock = writeLock()) {
+      final long nextIndex = getNextIndex();
+      final LogEntryProto e = ServerProtoUtils.toLogEntryProto(newConf, term,
+          nextIndex);
+      entries.add(e);
+      return nextIndex;
+    }
+  }
+
+  @Override
+  public long getStartIndex() {
+    return entries.isEmpty() ? RaftServerConstants.INVALID_LOG_INDEX :
+        entries.get(0).getIndex();
+  }
+
+  @Override
+  public void append(LogEntryProto... entries) {
+    checkLogState();
+    try(AutoCloseableLock writeLock = writeLock()) {
+      if (entries == null || entries.length == 0) {
+        return;
+      }
+      // Before truncating the entries, we first need to check if some
+      // entries are duplicated. If the leader sends entry 6, entry 7, then
+      // entry 6 again, without this check the follower may truncate entry 7
+      // when receiving entry 6 again. Then before the leader detects this
+      // truncation in the next appendEntries RPC, leader may think entry 7 has
+      // been committed but in the system the entry has not been committed to
+      // the quorum of peers' disks.
+      // TODO add a unit test for this
+      boolean toTruncate = false;
+      int truncateIndex = (int) entries[0].getIndex();
+      int index = 0;
+      for (; truncateIndex < getNextIndex() && index < entries.length;
+           index++, truncateIndex++) {
+        if (this.entries.get(truncateIndex).getTerm() !=
+            entries[index].getTerm()) {
+          toTruncate = true;
+          break;
+        }
+      }
+      if (toTruncate) {
+        truncate(truncateIndex);
+      }
+      //  Collections.addAll(this.entries, entries);
+      for (int i = index; i < entries.length; i++) {
+        this.entries.add(entries[i]);
+      }
+    }
+  }
+
+  @Override
+  public String toString() {
+    return "last=" + ServerProtoUtils.toString(getLastEntry())
+        + ", committed="
+        + ServerProtoUtils.toString(get(getLastCommittedIndex()));
+  }
+
+  public String getEntryString() {
+    return "entries=" + entries;
+  }
+
+  @Override
+  public void logSync() {
+    CodeInjectionForTesting.execute(LOG_SYNC, getSelfId(), null);
+    // do nothing
+  }
+
+  @Override
+  public long getLatestFlushedIndex() {
+    return getNextIndex() - 1;
+  }
+
+  @Override
+  public void writeMetadata(long term, String votedFor) {
+    // do nothing
+  }
+
+  @Override
+  public Metadata loadMetadata() {
+    return new Metadata(null, 0);
+  }
+
+  @Override
+  public void syncWithSnapshot(long lastSnapshotIndex) {
+    // do nothing
+  }
+}

http://git-wip-us.apache.org/repos/asf/incubator-ratis/blob/7e71a2e0/ratis-server/src/main/java/org/apache/ratis/server/storage/MetaFile.java
----------------------------------------------------------------------
diff --git a/ratis-server/src/main/java/org/apache/ratis/server/storage/MetaFile.java b/ratis-server/src/main/java/org/apache/ratis/server/storage/MetaFile.java
new file mode 100644
index 0000000..8deb7e3
--- /dev/null
+++ b/ratis-server/src/main/java/org/apache/ratis/server/storage/MetaFile.java
@@ -0,0 +1,131 @@
+/*
+ * 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.ratis.server.storage;
+
+import com.google.common.base.Charsets;
+
+import org.apache.ratis.util.AtomicFileOutputStream;
+import org.apache.ratis.util.RaftUtils;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.io.*;
+import java.util.Properties;
+
+/**
+ * Class that represents a file on disk which persistently stores
+ * a single <code>long</code> value. The file is updated atomically
+ * and durably (i.e fsynced).
+ */
+class MetaFile {
+  private static final Logger LOG = LoggerFactory.getLogger(MetaFile.class);
+  private static final String TERM_KEY = "term";
+  private static final String VOTEDFOR_KEY = "votedFor";
+  static final long DEFAULT_TERM = 0;
+  static final String EMPTY_VOTEFOR = "";
+
+  private final File file;
+  private boolean loaded = false;
+  private long term;
+  private String votedFor;
+
+  MetaFile(File file) {
+    this.file = file;
+    term = DEFAULT_TERM;
+    votedFor = EMPTY_VOTEFOR;
+  }
+
+  boolean exists() {
+    return this.file.exists();
+  }
+
+  long getTerm() throws IOException {
+    if (!loaded) {
+      readFile();
+      loaded = true;
+    }
+    return term;
+  }
+
+  String getVotedFor() throws IOException {
+    if (!loaded) {
+      readFile();
+      loaded = true;
+    }
+    return votedFor;
+  }
+
+  void set(long newTerm, String newVotedFor) throws IOException {
+    newVotedFor = newVotedFor == null ? EMPTY_VOTEFOR : newVotedFor;
+    if (!loaded || (newTerm != term || !newVotedFor.equals(votedFor))) {
+      writeFile(newTerm, newVotedFor);
+    }
+    term = newTerm;
+    votedFor = newVotedFor;
+    loaded = true;
+  }
+
+  /**
+   * Atomically write the given term and votedFor information to the given file,
+   * including fsyncing.
+   *
+   * @throws IOException if the file cannot be written
+   */
+  void writeFile(long term, String votedFor) throws IOException {
+    AtomicFileOutputStream fos = new AtomicFileOutputStream(file);
+    Properties properties = new Properties();
+    properties.setProperty(TERM_KEY, Long.toString(term));
+    properties.setProperty(VOTEDFOR_KEY, votedFor);
+    try {
+      properties.store(
+          new BufferedWriter(new OutputStreamWriter(fos, Charsets.UTF_8)), "");
+      fos.close();
+      fos = null;
+    } finally {
+      if (fos != null) {
+        fos.abort();
+      }
+    }
+  }
+
+  void readFile() throws IOException {
+    term = DEFAULT_TERM;
+    votedFor = EMPTY_VOTEFOR;
+    if (file.exists()) {
+      BufferedReader br = new BufferedReader(
+          new InputStreamReader(new FileInputStream(file), Charsets.UTF_8));
+      try {
+        Properties properties = new Properties();
+        properties.load(br);
+        if (properties.containsKey(TERM_KEY) &&
+            properties.containsKey(VOTEDFOR_KEY)) {
+          term = Long.parseLong((String) properties.get(TERM_KEY));
+          votedFor = (String) properties.get(VOTEDFOR_KEY);
+        } else {
+          throw new IOException("Corrupted term/votedFor properties: "
+              + properties);
+        }
+      } catch(IOException e) {
+        LOG.warn("Cannot load term/votedFor properties from {}", file, e);
+        throw e;
+      } finally {
+        RaftUtils.cleanup(LOG, br);
+      }
+    }
+  }
+}

http://git-wip-us.apache.org/repos/asf/incubator-ratis/blob/7e71a2e0/ratis-server/src/main/java/org/apache/ratis/server/storage/RaftLog.java
----------------------------------------------------------------------
diff --git a/ratis-server/src/main/java/org/apache/ratis/server/storage/RaftLog.java b/ratis-server/src/main/java/org/apache/ratis/server/storage/RaftLog.java
new file mode 100644
index 0000000..05307f2
--- /dev/null
+++ b/ratis-server/src/main/java/org/apache/ratis/server/storage/RaftLog.java
@@ -0,0 +1,293 @@
+/**
+ * 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.ratis.server.storage;
+
+import java.io.Closeable;
+import java.io.IOException;
+import java.util.concurrent.atomic.AtomicLong;
+import java.util.concurrent.locks.ReentrantReadWriteLock;
+
+import org.apache.ratis.server.impl.ConfigurationManager;
+import org.apache.ratis.server.impl.RaftConfiguration;
+import org.apache.ratis.server.impl.RaftServerConstants;
+import org.apache.ratis.server.impl.ServerProtoUtils;
+import org.apache.ratis.server.protocol.TermIndex;
+import org.apache.ratis.shaded.proto.RaftProtos.LogEntryProto;
+import org.apache.ratis.statemachine.TransactionContext;
+import org.apache.ratis.util.AutoCloseableLock;
+import org.apache.ratis.util.ProtoUtils;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import com.google.common.base.Preconditions;
+
+/**
+ * Base class of RaftLog. Currently we provide two types of RaftLog
+ * implementation:
+ * 1. MemoryRaftLog: all the log entries are stored in memory. This is only used
+ *    for testing.
+ * 2. Segmented RaftLog: the log entries are persisted on disk, and are stored
+ *    in segments.
+ */
+public abstract class RaftLog implements Closeable {
+  public static final Logger LOG = LoggerFactory.getLogger(RaftLog.class);
+  public static final LogEntryProto[] EMPTY_LOGENTRY_ARRAY = new LogEntryProto[0];
+  public static final String LOG_SYNC = RaftLog.class.getSimpleName() + ".logSync";
+
+  /**
+   * The largest committed index. Note the last committed log may be included
+   * in the latest snapshot file.
+   */
+  protected final AtomicLong lastCommitted =
+      new AtomicLong(RaftServerConstants.INVALID_LOG_INDEX);
+  private final String selfId;
+
+  private final ReentrantReadWriteLock lock = new ReentrantReadWriteLock(true);
+  private volatile boolean isOpen = false;
+
+  public RaftLog(String selfId) {
+    this.selfId = selfId;
+  }
+
+  public long getLastCommittedIndex() {
+    return lastCommitted.get();
+  }
+
+  public void checkLogState() {
+    Preconditions.checkState(isOpen,
+        "The RaftLog has not been opened or has been closed");
+  }
+
+  /**
+   * Update the last committed index.
+   * @param majorityIndex the index that has achieved majority.
+   * @param currentTerm the current term.
+   */
+  public void updateLastCommitted(long majorityIndex, long currentTerm) {
+    try(AutoCloseableLock writeLock = writeLock()) {
+      if (lastCommitted.get() < majorityIndex) {
+        // Only update last committed index for current term. See �5.4.2 in
+        // paper for details.
+        final LogEntryProto entry = get(majorityIndex);
+        if (entry != null && entry.getTerm() == currentTerm) {
+          LOG.debug("{}: Updating lastCommitted to {}", selfId, majorityIndex);
+          lastCommitted.set(majorityIndex);
+        }
+      }
+    }
+  }
+
+  /**
+   * Does the log contains the given term and index? Used to check the
+   * consistency between the local log of a follower and the log entries sent
+   * by the leader.
+   */
+  public boolean contains(TermIndex ti) {
+    if (ti == null) {
+      return false;
+    }
+    LogEntryProto entry = get(ti.getIndex());
+    TermIndex local = ServerProtoUtils.toTermIndex(entry);
+    return ti.equals(local);
+  }
+
+  /**
+   * @return the index of the next log entry to append.
+   */
+  public long getNextIndex() {
+    final LogEntryProto last = getLastEntry();
+    if (last == null) {
+      // if the log is empty, the last committed index should be consistent with
+      // the last index included in the latest snapshot.
+      return getLastCommittedIndex() + 1;
+    }
+    return last.getIndex() + 1;
+  }
+
+  /**
+   * Generate a log entry for the given term and message, and append the entry.
+   * Used by the leader.
+   * @return the index of the new log entry.
+   */
+  public long append(long term, TransactionContext operation) throws IOException {
+    checkLogState();
+    try(AutoCloseableLock writeLock = writeLock()) {
+      final long nextIndex = getNextIndex();
+
+      // This is called here to guarantee strict serialization of callback executions in case
+      // the SM wants to attach a logic depending on ordered execution in the log commit order.
+      operation = operation.preAppendTransaction();
+
+      // build the log entry after calling the StateMachine
+      final LogEntryProto e = ProtoUtils.toLogEntryProto(
+          operation.getSMLogEntry().get(), term, nextIndex);
+
+      appendEntry(e);
+      operation.setLogEntry(e);
+      return nextIndex;
+    }
+  }
+
+  /**
+   * Generate a log entry for the given term and configurations,
+   * and append the entry. Used by the leader.
+   * @return the index of the new log entry.
+   */
+  public long append(long term, RaftConfiguration newConf) {
+    checkLogState();
+    try(AutoCloseableLock writeLock = writeLock()) {
+      final long nextIndex = getNextIndex();
+      final LogEntryProto e = ServerProtoUtils.toLogEntryProto(newConf, term,
+          nextIndex);
+      appendEntry(e);
+      return nextIndex;
+    }
+  }
+
+  public void open(ConfigurationManager confManager, long lastIndexInSnapshot)
+      throws IOException {
+    isOpen = true;
+  }
+
+  public abstract long getStartIndex();
+
+  /**
+   * Get the log entry of the given index.
+   *
+   * @param index The given index.
+   * @return The log entry associated with the given index.
+   *         Null if there is no log entry with the index.
+   */
+  public abstract LogEntryProto get(long index);
+
+  /**
+   * @param startIndex the starting log index (inclusive)
+   * @param endIndex the ending log index (exclusive)
+   * @return all log entries within the given index range. Null if startIndex
+   *         is greater than the smallest available index.
+   */
+  public abstract LogEntryProto[] getEntries(long startIndex, long endIndex);
+
+  /**
+   * @return the last log entry.
+   */
+  public abstract LogEntryProto getLastEntry();
+
+  /**
+   * Truncate the log entries till the given index. The log with the given index
+   * will also be truncated (i.e., inclusive).
+   */
+  abstract void truncate(long index);
+
+  /**
+   * Used by the leader when appending a new entry based on client's request
+   * or configuration change.
+   */
+  abstract void appendEntry(LogEntryProto entry);
+
+  /**
+   * Append all the given log entries. Used by the followers.
+   *
+   * If an existing entry conflicts with a new one (same index but different
+   * terms), delete the existing entry and all entries that follow it (�5.3).
+   *
+   * This method, {@link #append(long, TransactionContext)},
+   * {@link #append(long, RaftConfiguration)}, and {@link #truncate(long)},
+   * do not guarantee the changes are persisted.
+   * Need to call {@link #logSync()} to persist the changes.
+   */
+  public abstract void append(LogEntryProto... entries);
+
+  /**
+   * Flush and sync the log.
+   * It is triggered by AppendEntries RPC request from the leader.
+   */
+  public abstract void logSync() throws InterruptedException;
+
+  /**
+   * @return the index of the latest entry that has been flushed to the local
+   *         storage.
+   */
+  public abstract long getLatestFlushedIndex();
+
+  /**
+   * Write and flush the metadata (votedFor and term) into the meta file.
+   *
+   * We need to guarantee that the order of writeMetadata calls is the same with
+   * that when we change the in-memory term/votedFor. Otherwise we may persist
+   * stale term/votedFor in file.
+   *
+   * Since the leader change is not frequent, currently we simply put this call
+   * in the RaftPeer's lock. Later we can use an IO task queue to enforce the
+   * order.
+   */
+  public abstract void writeMetadata(long term, String votedFor)
+      throws IOException;
+
+  public abstract Metadata loadMetadata() throws IOException;
+
+  public abstract void syncWithSnapshot(long lastSnapshotIndex);
+
+  @Override
+  public String toString() {
+    return ServerProtoUtils.toString(getLastEntry());
+  }
+
+  public static class Metadata {
+    private final String votedFor;
+    private final long term;
+
+    public Metadata(String votedFor, long term) {
+      this.votedFor = votedFor;
+      this.term = term;
+    }
+
+    public String getVotedFor() {
+      return votedFor;
+    }
+
+    public long getTerm() {
+      return term;
+    }
+  }
+
+  public AutoCloseableLock readLock() {
+    return AutoCloseableLock.acquire(lock.readLock());
+  }
+
+  public AutoCloseableLock writeLock() {
+    return AutoCloseableLock.acquire(lock.writeLock());
+  }
+
+  public boolean hasWriteLock() {
+    return this.lock.isWriteLockedByCurrentThread();
+  }
+
+  public boolean hasReadLock() {
+    return this.lock.getReadHoldCount() > 0 || hasWriteLock();
+  }
+
+  @Override
+  public void close() throws IOException {
+    isOpen = false;
+  }
+
+  public String getSelfId() {
+    return selfId;
+  }
+}

http://git-wip-us.apache.org/repos/asf/incubator-ratis/blob/7e71a2e0/ratis-server/src/main/java/org/apache/ratis/server/storage/RaftLogCache.java
----------------------------------------------------------------------
diff --git a/ratis-server/src/main/java/org/apache/ratis/server/storage/RaftLogCache.java b/ratis-server/src/main/java/org/apache/ratis/server/storage/RaftLogCache.java
new file mode 100644
index 0000000..90dd7fd
--- /dev/null
+++ b/ratis-server/src/main/java/org/apache/ratis/server/storage/RaftLogCache.java
@@ -0,0 +1,328 @@
+/**
+ * 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.ratis.server.storage;
+
+import static org.apache.ratis.server.impl.RaftServerConstants.INVALID_LOG_INDEX;
+
+import java.util.ArrayList;
+import java.util.Collections;
+import java.util.Iterator;
+import java.util.List;
+import java.util.NoSuchElementException;
+
+import org.apache.ratis.server.impl.RaftServerConstants;
+import org.apache.ratis.server.storage.LogSegment.LogRecord;
+import org.apache.ratis.server.storage.LogSegment.SegmentFileInfo;
+import org.apache.ratis.shaded.proto.RaftProtos.LogEntryProto;
+
+import com.google.common.annotations.VisibleForTesting;
+import com.google.common.base.Preconditions;
+
+/**
+ * In-memory RaftLog Cache. Currently we provide a simple implementation that
+ * caches all the segments in the memory. The cache is not thread-safe and
+ * requires external lock protection.
+ */
+class RaftLogCache {
+  private LogSegment openSegment;
+  private final List<LogSegment> closedSegments;
+
+  RaftLogCache() {
+    closedSegments = new ArrayList<>();
+  }
+
+  private boolean areConsecutiveSegments(LogSegment prev, LogSegment segment) {
+    return !prev.isOpen() && prev.getEndIndex() + 1 == segment.getStartIndex();
+  }
+
+  private LogSegment getLastClosedSegment() {
+    return closedSegments.isEmpty() ?
+        null : closedSegments.get(closedSegments.size() - 1);
+  }
+
+  private void validateAdding(LogSegment segment) {
+    final LogSegment lastClosed = getLastClosedSegment();
+    if (!segment.isOpen()) {
+      Preconditions.checkState(lastClosed == null ||
+          areConsecutiveSegments(lastClosed, segment));
+    } else {
+      Preconditions.checkState(openSegment == null &&
+          (lastClosed == null || areConsecutiveSegments(lastClosed, segment)));
+    }
+  }
+
+  void addSegment(LogSegment segment) {
+    validateAdding(segment);
+    if (segment.isOpen()) {
+      openSegment = segment;
+    } else {
+      closedSegments.add(segment);
+    }
+  }
+
+  LogEntryProto getEntry(long index) {
+    if (openSegment != null && index >= openSegment.getStartIndex()) {
+      final LogRecord record = openSegment.getLogRecord(index);
+      return record == null ? null : record.entry;
+    } else {
+      int segmentIndex = Collections.binarySearch(closedSegments, index);
+      if (segmentIndex < 0) {
+        return null;
+      } else {
+        return closedSegments.get(segmentIndex).getLogRecord(index).entry;
+      }
+    }
+  }
+
+  /**
+   * @param startIndex inclusive
+   * @param endIndex exclusive
+   */
+  LogEntryProto[] getEntries(final long startIndex, final long endIndex) {
+    if (startIndex < 0 || startIndex < getStartIndex()) {
+      throw new IndexOutOfBoundsException("startIndex = " + startIndex
+          + ", log cache starts from index " + getStartIndex());
+    }
+    if (startIndex > endIndex) {
+      throw new IndexOutOfBoundsException("startIndex(" + startIndex
+          + ") > endIndex(" + endIndex + ")");
+    }
+    final long realEnd = Math.min(getEndIndex() + 1, endIndex);
+    if (startIndex >= realEnd) {
+      return RaftLog.EMPTY_LOGENTRY_ARRAY;
+    }
+
+    LogEntryProto[] entries = new LogEntryProto[(int) (realEnd - startIndex)];
+    int segmentIndex = Collections.binarySearch(closedSegments, startIndex);
+    if (segmentIndex < 0) {
+      getEntriesFromSegment(openSegment, startIndex, entries, 0, entries.length);
+    } else {
+      long index = startIndex;
+      for (int i = segmentIndex; i < closedSegments.size() && index < realEnd; i++) {
+        LogSegment s = closedSegments.get(i);
+        int numberFromSegment = (int) Math.min(realEnd - index,
+            s.getEndIndex() - index + 1);
+        getEntriesFromSegment(s, index, entries, (int) (index - startIndex),
+            numberFromSegment);
+        index += numberFromSegment;
+      }
+      if (index < realEnd) {
+        getEntriesFromSegment(openSegment, index, entries,
+            (int) (index - startIndex), (int) (realEnd - index));
+      }
+    }
+    return entries;
+  }
+
+  private void getEntriesFromSegment(LogSegment segment, long startIndex,
+      LogEntryProto[] entries, int offset, int size) {
+    long endIndex = segment.getEndIndex();
+    endIndex = Math.min(endIndex, startIndex + size - 1);
+    int index = offset;
+    for (long i = startIndex; i <= endIndex; i++) {
+      entries[index++] = segment.getLogRecord(i).entry;
+    }
+  }
+
+  long getStartIndex() {
+    if (closedSegments.isEmpty()) {
+      return openSegment != null ? openSegment.getStartIndex() :
+          RaftServerConstants.INVALID_LOG_INDEX;
+    } else {
+      return closedSegments.get(0).getStartIndex();
+    }
+  }
+
+  @VisibleForTesting
+  long getEndIndex() {
+    return openSegment != null ? openSegment.getEndIndex() :
+        (closedSegments.isEmpty() ?
+            INVALID_LOG_INDEX :
+            closedSegments.get(closedSegments.size() - 1).getEndIndex());
+  }
+
+  LogEntryProto getLastEntry() {
+    return (openSegment != null && openSegment.numOfEntries() > 0) ?
+        openSegment.getLastRecord().entry :
+        (closedSegments.isEmpty() ? null :
+            closedSegments.get(closedSegments.size() - 1).getLastRecord().entry);
+  }
+
+  LogSegment getOpenSegment() {
+    return openSegment;
+  }
+
+  void appendEntry(LogEntryProto entry) {
+    // SegmentedRaftLog does the segment creation/rolling work. Here we just
+    // simply append the entry into the open segment.
+    Preconditions.checkState(openSegment != null);
+    openSegment.appendToOpenSegment(entry);
+  }
+
+  /**
+   * finalize the current open segment, and start a new open segment
+   */
+  void rollOpenSegment(boolean createNewOpen) {
+    Preconditions.checkState(openSegment != null
+        && openSegment.numOfEntries() > 0);
+    final long nextIndex = openSegment.getEndIndex() + 1;
+    openSegment.close();
+    closedSegments.add(openSegment);
+    if (createNewOpen) {
+      openSegment = LogSegment.newOpenSegment(nextIndex);
+    } else {
+      openSegment = null;
+    }
+  }
+
+  private SegmentFileInfo deleteOpenSegment() {
+    final long oldEnd = openSegment.getEndIndex();
+    openSegment.clear();
+    SegmentFileInfo info = new SegmentFileInfo(openSegment.getStartIndex(),
+        oldEnd, true, 0, openSegment.getEndIndex());
+    openSegment = null;
+    return info;
+  }
+
+  /**
+   * truncate log entries starting from the given index (inclusive)
+   */
+  TruncationSegments truncate(long index) {
+    int segmentIndex = Collections.binarySearch(closedSegments, index);
+    if (segmentIndex == -closedSegments.size() - 1) {
+      if (openSegment != null && openSegment.getEndIndex() >= index) {
+        final long oldEnd = openSegment.getEndIndex();
+        if (index == openSegment.getStartIndex()) {
+          // the open segment should be deleted
+          return new TruncationSegments(null,
+              Collections.singletonList(deleteOpenSegment()));
+        } else {
+          openSegment.truncate(index);
+          Preconditions.checkState(!openSegment.isOpen());
+          SegmentFileInfo info = new SegmentFileInfo(openSegment.getStartIndex(),
+              oldEnd, true, openSegment.getTotalSize(),
+              openSegment.getEndIndex());
+          closedSegments.add(openSegment);
+          openSegment = null;
+          return new TruncationSegments(info, Collections.emptyList());
+        }
+      }
+    } else if (segmentIndex >= 0) {
+      LogSegment ts = closedSegments.get(segmentIndex);
+      final long oldEnd = ts.getEndIndex();
+      List<SegmentFileInfo> list = new ArrayList<>();
+      ts.truncate(index);
+      final int size = closedSegments.size();
+      for (int i = size - 1;
+           i >= (ts.numOfEntries() == 0 ? segmentIndex : segmentIndex + 1);
+           i-- ) {
+        LogSegment s = closedSegments.remove(i);
+        final long endOfS = i == segmentIndex ? oldEnd : s.getEndIndex();
+        s.clear();
+        list.add(new SegmentFileInfo(s.getStartIndex(), endOfS, false, 0,
+            s.getEndIndex()));
+      }
+      if (openSegment != null) {
+        list.add(deleteOpenSegment());
+      }
+      SegmentFileInfo t = ts.numOfEntries() == 0 ? null :
+          new SegmentFileInfo(ts.getStartIndex(), oldEnd, false,
+              ts.getTotalSize(), ts.getEndIndex());
+      return new TruncationSegments(t, list);
+    }
+    return null;
+  }
+
+  static class TruncationSegments {
+    final SegmentFileInfo toTruncate; // name of the file to be truncated
+    final SegmentFileInfo[] toDelete; // names of the files to be deleted
+
+    TruncationSegments(SegmentFileInfo toTruncate,
+        List<SegmentFileInfo> toDelete) {
+      this.toDelete = toDelete == null ? null :
+          toDelete.toArray(new SegmentFileInfo[toDelete.size()]);
+      this.toTruncate = toTruncate;
+    }
+  }
+
+  Iterator<LogEntryProto> iterator(long startIndex) {
+    return new EntryIterator(startIndex);
+  }
+
+  private class EntryIterator implements Iterator<LogEntryProto> {
+    private long nextIndex;
+    private LogSegment currentSegment;
+    private int segmentIndex;
+
+    EntryIterator(long start) {
+      this.nextIndex = start;
+      segmentIndex = Collections.binarySearch(closedSegments, nextIndex);
+      if (segmentIndex >= 0) {
+        currentSegment = closedSegments.get(segmentIndex);
+      } else {
+        segmentIndex = -segmentIndex - 1;
+        if (segmentIndex == closedSegments.size()) {
+          currentSegment = openSegment;
+        } else {
+          // the start index is smaller than the first closed segment's start
+          // index. We no longer keep the log entry (because of the snapshot) or
+          // the start index is invalid.
+          Preconditions.checkState(segmentIndex == 0);
+          throw new IndexOutOfBoundsException();
+        }
+      }
+    }
+
+    @Override
+    public boolean hasNext() {
+      return currentSegment != null &&
+          currentSegment.getLogRecord(nextIndex) != null;
+    }
+
+    @Override
+    public LogEntryProto next() {
+      LogRecord record;
+      if (currentSegment == null ||
+          (record = currentSegment.getLogRecord(nextIndex)) == null) {
+        throw new NoSuchElementException();
+      }
+      if (++nextIndex > currentSegment.getEndIndex()) {
+        if (currentSegment != openSegment) {
+          segmentIndex++;
+          currentSegment = segmentIndex == closedSegments.size() ?
+              openSegment : closedSegments.get(segmentIndex);
+        }
+      }
+      return record.entry;
+    }
+  }
+
+  @VisibleForTesting
+  int getNumOfSegments() {
+    return closedSegments.size() + (openSegment == null ? 0 : 1);
+  }
+
+  boolean isEmpty() {
+    return closedSegments.isEmpty() && openSegment == null;
+  }
+
+  void clear() {
+    openSegment = null;
+    closedSegments.clear();
+  }
+}

http://git-wip-us.apache.org/repos/asf/incubator-ratis/blob/7e71a2e0/ratis-server/src/main/java/org/apache/ratis/server/storage/RaftLogWorker.java
----------------------------------------------------------------------
diff --git a/ratis-server/src/main/java/org/apache/ratis/server/storage/RaftLogWorker.java b/ratis-server/src/main/java/org/apache/ratis/server/storage/RaftLogWorker.java
new file mode 100644
index 0000000..e057a51
--- /dev/null
+++ b/ratis-server/src/main/java/org/apache/ratis/server/storage/RaftLogWorker.java
@@ -0,0 +1,372 @@
+/**
+ * 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.ratis.server.storage;
+
+import static org.apache.ratis.server.RaftServerConfigKeys.RAFT_LOG_FORCE_SYNC_NUM_DEFAULT;
+import static org.apache.ratis.server.RaftServerConfigKeys.RAFT_LOG_FORCE_SYNC_NUM_KEY;
+
+import java.io.File;
+import java.io.IOException;
+import java.util.concurrent.ArrayBlockingQueue;
+import java.util.concurrent.BlockingQueue;
+import java.util.concurrent.TimeUnit;
+
+import org.apache.ratis.conf.RaftProperties;
+import org.apache.ratis.io.nativeio.NativeIO;
+import org.apache.ratis.server.impl.RaftServerConstants;
+import org.apache.ratis.server.impl.RaftServerImpl;
+import org.apache.ratis.server.storage.LogSegment.SegmentFileInfo;
+import org.apache.ratis.server.storage.RaftLogCache.TruncationSegments;
+import org.apache.ratis.server.storage.SegmentedRaftLog.Task;
+import org.apache.ratis.shaded.proto.RaftProtos.LogEntryProto;
+import org.apache.ratis.util.ExitUtils;
+import org.apache.ratis.util.FileUtils;
+import org.apache.ratis.util.RaftUtils;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import com.google.common.base.Preconditions;
+
+/**
+ * This class takes the responsibility of all the raft log related I/O ops for a
+ * raft peer.
+ */
+class RaftLogWorker implements Runnable {
+  static final Logger LOG = LoggerFactory.getLogger(RaftLogWorker.class);
+  /**
+   * The task queue accessed by rpc handler threads and the io worker thread.
+   */
+  private final BlockingQueue<Task> queue = new ArrayBlockingQueue<>(4096);
+  private volatile boolean running = true;
+  private final Thread workerThread;
+
+  private final RaftStorage storage;
+  private LogOutputStream out;
+  private final RaftServerImpl raftServer;
+
+  /**
+   * The number of entries that have been written into the LogOutputStream but
+   * has not been flushed.
+   */
+  private int pendingFlushNum = 0;
+  /** the index of the last entry that has been written */
+  private long lastWrittenIndex;
+  /** the largest index of the entry that has been flushed */
+  private volatile long flushedIndex;
+
+  private final int forceSyncNum;
+
+  private final  RaftProperties properties;
+
+  RaftLogWorker(RaftServerImpl raftServer, RaftStorage storage,
+                RaftProperties properties) {
+    this.raftServer = raftServer;
+    this.storage = storage;
+    this.properties = properties;
+    this.forceSyncNum = properties.getInt(RAFT_LOG_FORCE_SYNC_NUM_KEY,
+        RAFT_LOG_FORCE_SYNC_NUM_DEFAULT);
+    workerThread = new Thread(this,
+        getClass().getSimpleName() + " for " + storage);
+  }
+
+  void start(long latestIndex, File openSegmentFile) throws IOException {
+    lastWrittenIndex = latestIndex;
+    flushedIndex = latestIndex;
+    if (openSegmentFile != null) {
+      Preconditions.checkArgument(openSegmentFile.exists());
+      out = new LogOutputStream(openSegmentFile, true, properties);
+    }
+    workerThread.start();
+  }
+
+  void close() {
+    this.running = false;
+    workerThread.interrupt();
+    try {
+      workerThread.join();
+    } catch (InterruptedException ignored) {
+    }
+  }
+
+  /**
+   * A snapshot has just been installed on the follower. Need to update the IO
+   * worker's state accordingly.
+   */
+  void syncWithSnapshot(long lastSnapshotIndex) {
+    queue.clear();
+    lastWrittenIndex = lastSnapshotIndex;
+    flushedIndex = lastSnapshotIndex;
+    pendingFlushNum = 0;
+  }
+
+  @Override
+  public String toString() {
+    return this.getClass().getSimpleName() + "-"
+        + (raftServer != null ? raftServer.getId() : "");
+  }
+
+  /**
+   * This is protected by the RaftServer and RaftLog's lock.
+   */
+  private Task addIOTask(Task task) {
+    LOG.debug("add task {}", task);
+    try {
+      if (!queue.offer(task, 1, TimeUnit.SECONDS)) {
+        Preconditions.checkState(isAlive(),
+            "the worker thread is not alive");
+        queue.put(task);
+      }
+    } catch (Throwable t) {
+      if (t instanceof InterruptedException && !running) {
+        LOG.info("Got InterruptedException when adding task " + task
+            + ". The RaftLogWorker already stopped.");
+      } else {
+        ExitUtils.terminate(2, "Failed to add IO task " + task, t, LOG);
+      }
+    }
+    return task;
+  }
+
+  boolean isAlive() {
+    return running && workerThread.isAlive();
+  }
+
+  @Override
+  public void run() {
+    while (running) {
+      try {
+        Task task = queue.poll(1, TimeUnit.SECONDS);
+        if (task != null) {
+          try {
+            task.execute();
+          } catch (IOException e) {
+            if (task.getEndIndex() < lastWrittenIndex) {
+              LOG.info("Ignore IOException when handling task " + task
+                  + " which is smaller than the lastWrittenIndex."
+                  + " There should be a snapshot installed.", e);
+            } else {
+              throw e;
+            }
+          }
+          task.done();
+        }
+      } catch (InterruptedException e) {
+        LOG.info(Thread.currentThread().getName()
+            + " was interrupted, exiting. There are " + queue.size()
+            + " tasks remaining in the queue.");
+      } catch (Throwable t) {
+        // TODO avoid terminating the jvm by supporting multiple log directories
+        ExitUtils.terminate(1, Thread.currentThread().getName() + " failed.", t, LOG);
+      }
+    }
+  }
+
+  private boolean shouldFlush() {
+    return pendingFlushNum >= forceSyncNum ||
+        (pendingFlushNum > 0 && queue.isEmpty());
+  }
+
+  private void flushWrites() throws IOException {
+    if (out != null) {
+      LOG.debug("flush data to " + out + ", reset pending_sync_number to 0");
+      out.flush();
+      updateFlushedIndex();
+    }
+  }
+
+  private void updateFlushedIndex() {
+    flushedIndex = lastWrittenIndex;
+    pendingFlushNum = 0;
+    if (raftServer != null) {
+      raftServer.submitLocalSyncEvent();
+    }
+  }
+
+  /**
+   * The following several methods (startLogSegment, rollLogSegment,
+   * writeLogEntry, and truncate) are only called by SegmentedRaftLog which is
+   * protected by RaftServer's lock.
+   *
+   * Thus all the tasks are created and added sequentially.
+   */
+  Task startLogSegment(long startIndex) {
+    return addIOTask(new StartLogSegment(startIndex));
+  }
+
+  Task rollLogSegment(LogSegment segmentToClose) {
+    addIOTask(new FinalizeLogSegment(segmentToClose));
+    return addIOTask(new StartLogSegment(segmentToClose.getEndIndex() + 1));
+  }
+
+  Task writeLogEntry(LogEntryProto entry) {
+    return addIOTask(new WriteLog(entry));
+  }
+
+  Task truncate(TruncationSegments ts) {
+    return addIOTask(new TruncateLog(ts));
+  }
+
+  // TODO we can add another level of buffer for writing here
+  private class WriteLog extends Task {
+    private final LogEntryProto entry;
+
+    WriteLog(LogEntryProto entry) {
+      this.entry = entry;
+    }
+
+    @Override
+    public void execute() throws IOException {
+      Preconditions.checkState(out != null);
+      Preconditions.checkState(lastWrittenIndex + 1 == entry.getIndex(),
+          "lastWrittenIndex == %s, entry == %s", lastWrittenIndex, entry);
+      out.write(entry);
+      lastWrittenIndex = entry.getIndex();
+      pendingFlushNum++;
+      if (shouldFlush()) {
+        flushWrites();
+      }
+    }
+
+    @Override
+    long getEndIndex() {
+      return entry.getIndex();
+    }
+  }
+
+  private class FinalizeLogSegment extends Task {
+    private final LogSegment segmentToClose;
+
+    FinalizeLogSegment(LogSegment segmentToClose) {
+      this.segmentToClose = segmentToClose;
+    }
+
+    @Override
+    public void execute() throws IOException {
+      RaftUtils.cleanup(null, out);
+      out = null;
+      Preconditions.checkState(segmentToClose != null);
+
+      File openFile = storage.getStorageDir()
+          .getOpenLogFile(segmentToClose.getStartIndex());
+      Preconditions.checkState(openFile.exists(),
+          "File %s does not exist.", openFile);
+      if (segmentToClose.numOfEntries() > 0) {
+        // finalize the current open segment
+        File dstFile = storage.getStorageDir().getClosedLogFile(
+            segmentToClose.getStartIndex(), segmentToClose.getEndIndex());
+        Preconditions.checkState(!dstFile.exists());
+
+        NativeIO.renameTo(openFile, dstFile);
+      } else { // delete the file of the empty segment
+        FileUtils.deleteFile(openFile);
+      }
+      updateFlushedIndex();
+    }
+
+    @Override
+    long getEndIndex() {
+      return segmentToClose.getEndIndex();
+    }
+  }
+
+  private class StartLogSegment extends Task {
+    private final long newStartIndex;
+
+    StartLogSegment(long newStartIndex) {
+      this.newStartIndex = newStartIndex;
+    }
+
+    @Override
+    void execute() throws IOException {
+      File openFile = storage.getStorageDir().getOpenLogFile(newStartIndex);
+      Preconditions.checkState(!openFile.exists(), "open file %s exists for %s",
+          openFile.getAbsolutePath(), RaftLogWorker.this.toString());
+      Preconditions.checkState(out == null && pendingFlushNum == 0);
+      out = new LogOutputStream(openFile, false, properties);
+    }
+
+    @Override
+    long getEndIndex() {
+      return newStartIndex;
+    }
+  }
+
+  private class TruncateLog extends Task {
+    private final TruncationSegments segments;
+
+    TruncateLog(TruncationSegments ts) {
+      this.segments = ts;
+    }
+
+    @Override
+    void execute() throws IOException {
+      RaftUtils.cleanup(null, out);
+      out = null;
+      if (segments.toTruncate != null) {
+        File fileToTruncate = segments.toTruncate.isOpen ?
+            storage.getStorageDir().getOpenLogFile(
+                segments.toTruncate.startIndex) :
+            storage.getStorageDir().getClosedLogFile(
+                segments.toTruncate.startIndex,
+                segments.toTruncate.endIndex);
+        FileUtils.truncateFile(fileToTruncate, segments.toTruncate.targetLength);
+
+        // rename the file
+        File dstFile = storage.getStorageDir().getClosedLogFile(
+            segments.toTruncate.startIndex, segments.toTruncate.newEndIndex);
+        Preconditions.checkState(!dstFile.exists());
+        NativeIO.renameTo(fileToTruncate, dstFile);
+
+        // update lastWrittenIndex
+        lastWrittenIndex = segments.toTruncate.newEndIndex;
+      }
+      if (segments.toDelete != null && segments.toDelete.length > 0) {
+        long minStart = segments.toDelete[0].startIndex;
+        for (SegmentFileInfo del : segments.toDelete) {
+          final File delFile;
+          if (del.isOpen) {
+            delFile = storage.getStorageDir().getOpenLogFile(del.startIndex);
+          } else {
+            delFile = storage.getStorageDir()
+                .getClosedLogFile(del.startIndex, del.endIndex);
+          }
+          FileUtils.deleteFile(delFile);
+          minStart = Math.min(minStart, del.startIndex);
+        }
+        if (segments.toTruncate == null) {
+          lastWrittenIndex = minStart - 1;
+        }
+      }
+      updateFlushedIndex();
+    }
+
+    @Override
+    long getEndIndex() {
+      if (segments.toTruncate != null) {
+        return segments.toTruncate.newEndIndex;
+      } else if (segments.toDelete.length > 0) {
+        return segments.toDelete[segments.toDelete.length - 1].endIndex;
+      }
+      return RaftServerConstants.INVALID_LOG_INDEX;
+    }
+  }
+
+  long getFlushedIndex() {
+    return flushedIndex;
+  }
+}

http://git-wip-us.apache.org/repos/asf/incubator-ratis/blob/7e71a2e0/ratis-server/src/main/java/org/apache/ratis/server/storage/RaftStorage.java
----------------------------------------------------------------------
diff --git a/ratis-server/src/main/java/org/apache/ratis/server/storage/RaftStorage.java b/ratis-server/src/main/java/org/apache/ratis/server/storage/RaftStorage.java
new file mode 100644
index 0000000..09ea55c
--- /dev/null
+++ b/ratis-server/src/main/java/org/apache/ratis/server/storage/RaftStorage.java
@@ -0,0 +1,145 @@
+/**
+ * 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.ratis.server.storage;
+
+import com.google.common.base.Preconditions;
+
+import org.apache.ratis.conf.RaftProperties;
+import org.apache.ratis.server.impl.RaftServerConstants;
+import org.apache.ratis.server.storage.RaftStorageDirectory.StorageState;
+import org.apache.ratis.statemachine.SnapshotInfo;
+import org.apache.ratis.statemachine.StateMachineStorage;
+import org.apache.ratis.util.FileUtils;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import static org.apache.ratis.server.RaftServerConfigKeys.RAFT_SERVER_STORAGE_DIR_DEFAULT;
+import static org.apache.ratis.server.RaftServerConfigKeys.RAFT_SERVER_STORAGE_DIR_KEY;
+
+import java.io.Closeable;
+import java.io.File;
+import java.io.IOException;
+import java.nio.file.Files;
+
+public class RaftStorage implements Closeable {
+  private static final Logger LOG = LoggerFactory.getLogger(RaftStorage.class);
+
+  // TODO support multiple storage directories
+  private final RaftStorageDirectory storageDir;
+  private final StorageState state;
+  private volatile MetaFile metaFile;
+  private StateMachineStorage stateMachineStorage;
+
+  public RaftStorage(RaftProperties prop, RaftServerConstants.StartupOption option)
+      throws IOException {
+    final String dir = prop.get(RAFT_SERVER_STORAGE_DIR_KEY,
+        RAFT_SERVER_STORAGE_DIR_DEFAULT);
+    storageDir = new RaftStorageDirectory(
+        new File(FileUtils.stringAsURI(dir).getPath()));
+    if (option == RaftServerConstants.StartupOption.FORMAT) {
+      if (storageDir.analyzeStorage(false) == StorageState.NON_EXISTENT) {
+        throw new IOException("Cannot format " + storageDir);
+      }
+      storageDir.lock();
+      format();
+      state = storageDir.analyzeStorage(false);
+      Preconditions.checkState(state == StorageState.NORMAL);
+    } else {
+      state = analyzeAndRecoverStorage(true); // metaFile is initialized here
+      if (state != StorageState.NORMAL) {
+        storageDir.unlock();
+        throw new IOException("Cannot load " + storageDir
+            + ". Its state: " + state);
+      }
+    }
+  }
+
+  StorageState getState() {
+    return state;
+  }
+
+  private void format() throws IOException {
+    storageDir.clearDirectory();
+    metaFile = writeMetaFile(MetaFile.DEFAULT_TERM, MetaFile.EMPTY_VOTEFOR);
+    LOG.info("Storage directory " + storageDir.getRoot()
+        + " has been successfully formatted.");
+  }
+
+  private MetaFile writeMetaFile(long term, String votedFor) throws IOException {
+    MetaFile metaFile = new MetaFile(storageDir.getMetaFile());
+    metaFile.set(term, votedFor);
+    return metaFile;
+  }
+
+  private void cleanMetaTmpFile() throws IOException {
+    Files.deleteIfExists(storageDir.getMetaTmpFile().toPath());
+  }
+
+  private StorageState analyzeAndRecoverStorage(boolean toLock)
+      throws IOException {
+    StorageState storageState = storageDir.analyzeStorage(toLock);
+    if (storageState == StorageState.NORMAL) {
+      metaFile = new MetaFile(storageDir.getMetaFile());
+      assert metaFile.exists();
+      metaFile.readFile();
+      // Existence of raft-meta.tmp means the change of votedFor/term has not
+      // been committed. Thus we should delete the tmp file.
+      cleanMetaTmpFile();
+      return StorageState.NORMAL;
+    } else if (storageState == StorageState.NOT_FORMATTED &&
+        storageDir.isCurrentEmpty()) {
+      format();
+      return StorageState.NORMAL;
+    } else {
+      return storageState;
+    }
+  }
+
+  public RaftStorageDirectory getStorageDir() {
+    return storageDir;
+  }
+
+  @Override
+  public void close() throws IOException {
+    storageDir.unlock();
+  }
+
+  MetaFile getMetaFile() {
+    return metaFile;
+  }
+
+  public SnapshotInfo getLastestSnapshot() throws IOException {
+    return getStateMachineStorage().getLatestSnapshot();
+  }
+
+  /**
+   * Called by the state machine after it has initialized the StateMachineStorage.
+   */
+  public void setStateMachineStorage(StateMachineStorage smStorage) {
+    this.stateMachineStorage = smStorage;
+  }
+
+  public StateMachineStorage getStateMachineStorage() {
+    return stateMachineStorage;
+  }
+
+  @Override
+  public String toString() {
+    return getStorageDir() + "";
+  }
+}

http://git-wip-us.apache.org/repos/asf/incubator-ratis/blob/7e71a2e0/ratis-server/src/main/java/org/apache/ratis/server/storage/RaftStorageDirectory.java
----------------------------------------------------------------------
diff --git a/ratis-server/src/main/java/org/apache/ratis/server/storage/RaftStorageDirectory.java b/ratis-server/src/main/java/org/apache/ratis/server/storage/RaftStorageDirectory.java
new file mode 100644
index 0000000..bfa691d
--- /dev/null
+++ b/ratis-server/src/main/java/org/apache/ratis/server/storage/RaftStorageDirectory.java
@@ -0,0 +1,361 @@
+/**
+ * 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.ratis.server.storage;
+
+import com.google.common.annotations.VisibleForTesting;
+import com.google.common.base.Charsets;
+import com.google.common.base.Preconditions;
+import com.google.common.collect.ImmutableList;
+
+import org.apache.ratis.util.AtomicFileOutputStream;
+import org.apache.ratis.util.FileUtils;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.io.File;
+import java.io.IOException;
+import java.io.RandomAccessFile;
+import java.lang.management.ManagementFactory;
+import java.nio.channels.FileLock;
+import java.nio.channels.OverlappingFileLockException;
+import java.nio.file.DirectoryStream;
+import java.nio.file.Files;
+import java.nio.file.Path;
+import java.util.*;
+import java.util.regex.Matcher;
+import java.util.regex.Pattern;
+
+import static java.nio.file.Files.newDirectoryStream;
+import static org.apache.ratis.server.impl.RaftServerConstants.INVALID_LOG_INDEX;
+
+public class RaftStorageDirectory {
+  static final Logger LOG = LoggerFactory.getLogger(RaftStorageDirectory.class);
+
+  static final String STORAGE_DIR_CURRENT = "current";
+  static final String STORAGE_FILE_LOCK = "in_use.lock";
+  static final String META_FILE_NAME = "raft-meta";
+  static final String LOG_FILE_INPROGRESS = "inprogress";
+  static final String LOG_FILE_PREFIX = "log";
+  static final String STATE_MACHINE = "sm"; // directory containing state machine snapshots
+  static final String TEMP = "tmp";
+  static final Pattern CLOSED_SEGMENT_REGEX = Pattern.compile("log_(\\d+)-(\\d+)");
+  static final Pattern OPEN_SEGMENT_REGEX = Pattern.compile("log_inprogress_(\\d+)(?:\\..*)?");
+
+  private static final List<Pattern> LOGSEGMENTS_REGEXES =
+      ImmutableList.of(CLOSED_SEGMENT_REGEX, OPEN_SEGMENT_REGEX);
+
+  enum StorageState {
+    NON_EXISTENT,
+    NOT_FORMATTED,
+    NORMAL
+  }
+
+  public static class LogPathAndIndex {
+    public final Path path;
+    public final long startIndex;
+    public final long endIndex;
+
+    LogPathAndIndex(Path path, long startIndex, long endIndex) {
+      this.path = path;
+      this.startIndex = startIndex;
+      this.endIndex = endIndex;
+    }
+
+    @Override
+    public String toString() {
+      return path + "-" + startIndex + "-" + endIndex;
+    }
+  }
+
+  private final File root; // root directory
+  private FileLock lock;   // storage lock
+
+  /**
+   * Constructor
+   * @param dir directory corresponding to the storage
+   */
+  RaftStorageDirectory(File dir) {
+    this.root = dir;
+    this.lock = null;
+  }
+
+  /**
+   * Get root directory of this storage
+   */
+  //TODO
+  public File getRoot() {
+    return root;
+  }
+
+  /**
+   * Clear and re-create storage directory.
+   * <p>
+   * Removes contents of the current directory and creates an empty directory.
+   *
+   * This does not fully format storage directory.
+   * It cannot write the version file since it should be written last after
+   * all other storage type dependent files are written.
+   * Derived storage is responsible for setting specific storage values and
+   * writing the version file to disk.
+   */
+  void clearDirectory() throws IOException {
+    File curDir = this.getCurrentDir();
+    clearDirectory(curDir);
+    clearDirectory(getStateMachineDir());
+  }
+
+  void clearDirectory(File dir) throws IOException {
+    if (dir.exists()) {
+      File[] files = FileUtils.listFiles(dir);
+      LOG.info("Will remove files: " + Arrays.toString(files));
+      if (!(FileUtils.fullyDelete(dir)))
+        throw new IOException("Cannot remove directory: " + dir);
+    }
+    if (!dir.mkdirs())
+      throw new IOException("Cannot create directory " + dir);
+  }
+
+  /**
+   * Directory {@code current} contains latest files defining
+   * the file system meta-data.
+   *
+   * @return the directory path
+   */
+  File getCurrentDir() {
+    return new File(root, STORAGE_DIR_CURRENT);
+  }
+
+  File getMetaFile() {
+    return new File(getCurrentDir(), META_FILE_NAME);
+  }
+
+  File getMetaTmpFile() {
+    return new File(getCurrentDir(), META_FILE_NAME
+        + AtomicFileOutputStream.TMP_EXTENSION);
+  }
+
+  File getOpenLogFile(long startIndex) {
+    return new File(getCurrentDir(), getOpenLogFileName(startIndex));
+  }
+
+  static String getOpenLogFileName(long startIndex) {
+    return LOG_FILE_PREFIX + "_" + LOG_FILE_INPROGRESS + "_" + startIndex;
+  }
+
+  File getClosedLogFile(long startIndex, long endIndex) {
+    return new File(getCurrentDir(), getClosedLogFileName(startIndex, endIndex));
+  }
+
+  static String getClosedLogFileName(long startIndex, long endIndex) {
+    return LOG_FILE_PREFIX + "_" + startIndex + "-" + endIndex;
+  }
+
+  public File getStateMachineDir() {
+    return new File(getRoot(), STATE_MACHINE);
+  }
+
+  /** Returns a uniquely named temporary directory under $rootdir/tmp/ */
+  public File getNewTempDir() {
+    return new File(new File(getRoot(), TEMP), UUID.randomUUID().toString());
+  }
+
+  public Path relativizeToRoot(Path p) {
+    if (p.isAbsolute()) {
+      return getRoot().toPath().relativize(p);
+    }
+    return p;
+  }
+
+  /**
+   * @return log segment files sorted based on their index.
+   */
+  @VisibleForTesting
+  public List<LogPathAndIndex> getLogSegmentFiles() throws IOException {
+    List<LogPathAndIndex> list = new ArrayList<>();
+    try (DirectoryStream<Path> stream =
+             Files.newDirectoryStream(getCurrentDir().toPath())) {
+      for (Path path : stream) {
+        for (Pattern pattern : LOGSEGMENTS_REGEXES) {
+          Matcher matcher = pattern.matcher(path.getFileName().toString());
+          if (matcher.matches()) {
+            final long startIndex = Long.parseLong(matcher.group(1));
+            final long endIndex = matcher.groupCount() == 2 ?
+                Long.parseLong(matcher.group(2)) : INVALID_LOG_INDEX;
+            list.add(new LogPathAndIndex(path, startIndex, endIndex));
+          }
+        }
+      }
+    }
+    Collections.sort(list,
+        (o1, o2) -> o1.startIndex == o2.startIndex ?
+            0 : (o1.startIndex < o2.startIndex ? -1 : 1));
+    return list;
+  }
+
+  /**
+   * Check to see if current/ directory is empty.
+   */
+  boolean isCurrentEmpty() throws IOException {
+    File currentDir = getCurrentDir();
+    if(!currentDir.exists()) {
+      // if current/ does not exist, it's safe to format it.
+      return true;
+    }
+    try(DirectoryStream<Path> dirStream =
+            newDirectoryStream(currentDir.toPath())) {
+      if (dirStream.iterator().hasNext()) {
+        return false;
+      }
+    }
+    return true;
+  }
+
+  /**
+   * Check consistency of the storage directory.
+   *
+   * @return state {@link StorageState} of the storage directory
+   */
+  StorageState analyzeStorage(boolean toLock) throws IOException {
+    Preconditions.checkState(root != null, "root directory is null");
+
+    String rootPath = root.getCanonicalPath();
+    try { // check that storage exists
+      if (!root.exists()) {
+        LOG.info(rootPath + " does not exist. Creating ...");
+        if (!root.mkdirs()) {
+          throw new IOException("Cannot create directory " + rootPath);
+        }
+      }
+      // or is inaccessible
+      if (!root.isDirectory()) {
+        LOG.warn(rootPath + "is not a directory");
+        return StorageState.NON_EXISTENT;
+      }
+      if (!FileUtils.canWrite(root)) {
+        LOG.warn("Cannot access storage directory " + rootPath);
+        return StorageState.NON_EXISTENT;
+      }
+    } catch(SecurityException ex) {
+      LOG.warn("Cannot access storage directory " + rootPath, ex);
+      return StorageState.NON_EXISTENT;
+    }
+
+    if (toLock) {
+      this.lock(); // lock storage if it exists
+    }
+
+    // check whether current directory is valid
+    if (hasMetaFile()) {
+      return StorageState.NORMAL;
+    } else {
+      return StorageState.NOT_FORMATTED;
+    }
+  }
+
+  boolean hasMetaFile() throws IOException {
+    return getMetaFile().exists();
+  }
+
+  /**
+   * Lock storage to provide exclusive access.
+   *
+   * <p> Locking is not supported by all file systems.
+   * E.g., NFS does not consistently support exclusive locks.
+   *
+   * <p> If locking is supported we guarantee exclusive access to the
+   * storage directory. Otherwise, no guarantee is given.
+   *
+   * @throws IOException if locking fails
+   */
+  public void lock() throws IOException {
+    FileLock newLock = tryLock();
+    if (newLock == null) {
+      String msg = "Cannot lock storage " + this.root
+          + ". The directory is already locked";
+      LOG.info(msg);
+      throw new IOException(msg);
+    }
+    // Don't overwrite lock until success - this way if we accidentally
+    // call lock twice, the internal state won't be cleared by the second
+    // (failed) lock attempt
+    lock = newLock;
+  }
+
+  /**
+   * Attempts to acquire an exclusive lock on the storage.
+   *
+   * @return A lock object representing the newly-acquired lock or
+   * <code>null</code> if storage is already locked.
+   * @throws IOException if locking fails.
+   */
+  private FileLock tryLock() throws IOException {
+    boolean deletionHookAdded = false;
+    File lockF = new File(root, STORAGE_FILE_LOCK);
+    if (!lockF.exists()) {
+      lockF.deleteOnExit();
+      deletionHookAdded = true;
+    }
+    RandomAccessFile file = new RandomAccessFile(lockF, "rws");
+    String jvmName = ManagementFactory.getRuntimeMXBean().getName();
+    FileLock res;
+    try {
+      res = file.getChannel().tryLock();
+      if (null == res) {
+        LOG.error("Unable to acquire file lock on path " + lockF.toString());
+        throw new OverlappingFileLockException();
+      }
+      file.write(jvmName.getBytes(Charsets.UTF_8));
+      LOG.info("Lock on " + lockF + " acquired by nodename " + jvmName);
+    } catch (OverlappingFileLockException oe) {
+      // Cannot read from the locked file on Windows.
+      LOG.error("It appears that another process "
+          + "has already locked the storage directory: " + root, oe);
+      file.close();
+      return null;
+    } catch(IOException e) {
+      LOG.error("Failed to acquire lock on " + lockF
+          + ". If this storage directory is mounted via NFS, "
+          + "ensure that the appropriate nfs lock services are running.", e);
+      file.close();
+      throw e;
+    }
+    if (!deletionHookAdded) {
+      // If the file existed prior to our startup, we didn't
+      // call deleteOnExit above. But since we successfully locked
+      // the dir, we can take care of cleaning it up.
+      lockF.deleteOnExit();
+    }
+    return res;
+  }
+
+  /**
+   * Unlock storage.
+   */
+  public void unlock() throws IOException {
+    if (this.lock == null)
+      return;
+    this.lock.release();
+    lock.channel().close();
+    lock = null;
+  }
+
+  @Override
+  public String toString() {
+    return "Storage Directory " + this.root;
+  }
+}



[13/54] [abbrv] incubator-ratis git commit: Convert TermIndex to an interface.

Posted by ji...@apache.org.
Convert TermIndex to an interface.


Project: http://git-wip-us.apache.org/repos/asf/incubator-ratis/repo
Commit: http://git-wip-us.apache.org/repos/asf/incubator-ratis/commit/59460582
Tree: http://git-wip-us.apache.org/repos/asf/incubator-ratis/tree/59460582
Diff: http://git-wip-us.apache.org/repos/asf/incubator-ratis/diff/59460582

Branch: refs/heads/master
Commit: 594605829133f98d51b16d42fd0596fa26bde6d2
Parents: 8dbb64f
Author: Tsz-Wo Nicholas Sze <sz...@hortonworks.com>
Authored: Fri Jan 6 14:44:36 2017 +0800
Committer: Tsz-Wo Nicholas Sze <sz...@hortonworks.com>
Committed: Fri Jan 6 14:44:36 2017 +0800

----------------------------------------------------------------------
 .../arithmetic/ArithmeticStateMachine.java      |  2 +-
 .../raft/server/impl/ServerProtoUtils.java      |  5 +-
 .../apache/raft/server/impl/ServerUtils.java    | 81 ++++++++++++++++++++
 .../apache/raft/server/protocol/TermIndex.java  | 58 +++-----------
 .../org/apache/raft/server/storage/RaftLog.java |  3 +-
 .../statemachine/SimpleStateMachineStorage.java |  2 +-
 .../raft/statemachine/SnapshotInfoImpl.java     |  2 +-
 .../SimpleStateMachine4Testing.java             |  7 +-
 .../raft/statemachine/TermIndexTracker.java     |  2 +-
 9 files changed, 104 insertions(+), 58 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-ratis/blob/59460582/raft-examples/src/main/java/org/apache/raft/examples/arithmetic/ArithmeticStateMachine.java
----------------------------------------------------------------------
diff --git a/raft-examples/src/main/java/org/apache/raft/examples/arithmetic/ArithmeticStateMachine.java b/raft-examples/src/main/java/org/apache/raft/examples/arithmetic/ArithmeticStateMachine.java
index 91776eb..53b6122 100644
--- a/raft-examples/src/main/java/org/apache/raft/examples/arithmetic/ArithmeticStateMachine.java
+++ b/raft-examples/src/main/java/org/apache/raft/examples/arithmetic/ArithmeticStateMachine.java
@@ -172,7 +172,7 @@ public class ArithmeticStateMachine extends BaseStateMachine {
   }
 
   private void updateLatestTermIndex(long term, long index) {
-    final TermIndex newTI = new TermIndex(term, index);
+    final TermIndex newTI = TermIndex.newTermIndex(term, index);
     final TermIndex oldTI = latestTermIndex.getAndSet(newTI);
     if (oldTI != null) {
       Preconditions.checkArgument(newTI.compareTo(oldTI) >= 0);

http://git-wip-us.apache.org/repos/asf/incubator-ratis/blob/59460582/raft-server/src/main/java/org/apache/raft/server/impl/ServerProtoUtils.java
----------------------------------------------------------------------
diff --git a/raft-server/src/main/java/org/apache/raft/server/impl/ServerProtoUtils.java b/raft-server/src/main/java/org/apache/raft/server/impl/ServerProtoUtils.java
index 4594666..e30b979 100644
--- a/raft-server/src/main/java/org/apache/raft/server/impl/ServerProtoUtils.java
+++ b/raft-server/src/main/java/org/apache/raft/server/impl/ServerProtoUtils.java
@@ -29,9 +29,10 @@ import java.util.stream.Collectors;
 import static org.apache.raft.server.impl.RaftServerConstants.DEFAULT_SEQNUM;
 import static org.apache.raft.shaded.proto.RaftProtos.AppendEntriesReplyProto.AppendResult.SUCCESS;
 
+/** Server proto utilities for internal use. */
 public class ServerProtoUtils {
   public static TermIndex toTermIndex(TermIndexProto p) {
-    return p == null? null: new TermIndex(p.getTerm(), p.getIndex());
+    return p == null? null: TermIndex.newTermIndex(p.getTerm(), p.getIndex());
   }
 
   public static TermIndexProto toTermIndexProto(TermIndex ti) {
@@ -43,7 +44,7 @@ public class ServerProtoUtils {
 
   public static TermIndex toTermIndex(LogEntryProto entry) {
     return entry == null ? null :
-        new TermIndex(entry.getTerm(), entry.getIndex());
+        TermIndex.newTermIndex(entry.getTerm(), entry.getIndex());
   }
 
   public static String toString(LogEntryProto... entries) {

http://git-wip-us.apache.org/repos/asf/incubator-ratis/blob/59460582/raft-server/src/main/java/org/apache/raft/server/impl/ServerUtils.java
----------------------------------------------------------------------
diff --git a/raft-server/src/main/java/org/apache/raft/server/impl/ServerUtils.java b/raft-server/src/main/java/org/apache/raft/server/impl/ServerUtils.java
new file mode 100644
index 0000000..ac21386
--- /dev/null
+++ b/raft-server/src/main/java/org/apache/raft/server/impl/ServerUtils.java
@@ -0,0 +1,81 @@
+/**
+ * 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.raft.server.impl;
+
+import org.apache.commons.lang.builder.HashCodeBuilder;
+import org.apache.raft.server.protocol.TermIndex;
+
+/** Server utilities for internal use. */
+public class ServerUtils {
+  public static TermIndex newTermIndex(long term, long index) {
+    return new TermIndexImpl(term, index);
+  }
+
+  private static class TermIndexImpl implements TermIndex {
+    private final long term;
+    private final long index; //log index; first index is 1.
+
+    TermIndexImpl(long term, long logIndex) {
+      this.term = term;
+      this.index = logIndex;
+    }
+
+    @Override
+    public long getTerm() {
+      return term;
+    }
+
+    @Override
+    public long getIndex() {
+      return index;
+    }
+
+    @Override
+    public int compareTo(TermIndex that) {
+      final int d = Long.compare(this.getTerm(), that.getTerm());
+      return d != 0 ? d : Long.compare(this.getIndex(), that.getIndex());
+    }
+
+    @Override
+    public boolean equals(Object obj) {
+      if (obj == this) {
+        return true;
+      } else if (obj == null || !(obj instanceof TermIndexImpl)) {
+        return false;
+      }
+
+      final TermIndexImpl that = (TermIndexImpl) obj;
+      return this.getTerm() == that.getTerm()
+          && this.getIndex() == that.getIndex();
+    }
+
+    @Override
+    public int hashCode() {
+      return new HashCodeBuilder().append(term).append(index).hashCode();
+    }
+
+    private static String toString(long n) {
+      return n < 0 ? "~" : "" + n;
+    }
+
+    @Override
+    public String toString() {
+      return "(t:" + toString(term) + ", i:" + toString(index) + ")";
+    }
+  }
+}

http://git-wip-us.apache.org/repos/asf/incubator-ratis/blob/59460582/raft-server/src/main/java/org/apache/raft/server/protocol/TermIndex.java
----------------------------------------------------------------------
diff --git a/raft-server/src/main/java/org/apache/raft/server/protocol/TermIndex.java b/raft-server/src/main/java/org/apache/raft/server/protocol/TermIndex.java
index 4bf9d71..df401d6 100644
--- a/raft-server/src/main/java/org/apache/raft/server/protocol/TermIndex.java
+++ b/raft-server/src/main/java/org/apache/raft/server/protocol/TermIndex.java
@@ -17,56 +17,20 @@
  */
 package org.apache.raft.server.protocol;
 
-import org.apache.commons.lang.builder.HashCodeBuilder;
+import org.apache.raft.server.impl.ServerUtils;
 
-public class TermIndex implements Comparable<TermIndex> {
-  private final long term;
-  private final long index; //log index; first index is 1.
+/** The term and the log index defined in the Raft consensus algorithm. */
+public interface TermIndex extends Comparable<TermIndex> {
+  /** @return the term. */
+  long getTerm();
 
-  public TermIndex(long term, long logIndex) {
-    this.term = term;
-    this.index = logIndex;
-  }
-
-  public TermIndex(TermIndex other) {
-    this(other.getTerm(), other.getIndex());
-  }
-
-  public long getTerm() {
-    return term;
-  }
-
-  public long getIndex() {
-    return index;
-  }
+  /** @return the index. */
+  long getIndex();
 
-  @Override
-  public int compareTo(TermIndex that) {
-    final int diff = Long.compare(this.term, that.term);
-    return diff != 0 ? diff : Long.compare(this.index, that.index);
-  }
-
-  @Override
-  public boolean equals(Object o) {
-    if (o instanceof TermIndex) {
-      final TermIndex ti = (TermIndex) o;
-      return this == ti ||
-          (this.term == ti.getTerm() && this.index == ti.getIndex());
-    }
-    return false;
+  /** Create a new {@link TermIndex} instance. */
+  static TermIndex newTermIndex(long term, long index) {
+    return ServerUtils.newTermIndex(term, index);
   }
+}
 
-  @Override
-  public int hashCode() {
-    return new HashCodeBuilder().append(term).append(index).hashCode();
-  }
 
-  private static String toString(long n) {
-    return n < 0 ? "~" : "" + n;
-  }
-
-  @Override
-  public String toString() {
-    return "(t:" + toString(term) + ", i:" + toString(index) + ")";
-  }
-}

http://git-wip-us.apache.org/repos/asf/incubator-ratis/blob/59460582/raft-server/src/main/java/org/apache/raft/server/storage/RaftLog.java
----------------------------------------------------------------------
diff --git a/raft-server/src/main/java/org/apache/raft/server/storage/RaftLog.java b/raft-server/src/main/java/org/apache/raft/server/storage/RaftLog.java
index acd44b0..de79911 100644
--- a/raft-server/src/main/java/org/apache/raft/server/storage/RaftLog.java
+++ b/raft-server/src/main/java/org/apache/raft/server/storage/RaftLog.java
@@ -101,8 +101,7 @@ public abstract class RaftLog implements Closeable {
       return false;
     }
     LogEntryProto entry = get(ti.getIndex());
-    TermIndex local = entry == null ? null :
-        new TermIndex(entry.getTerm(), entry.getIndex());
+    TermIndex local = ServerProtoUtils.toTermIndex(entry);
     return ti.equals(local);
   }
 

http://git-wip-us.apache.org/repos/asf/incubator-ratis/blob/59460582/raft-server/src/main/java/org/apache/raft/statemachine/SimpleStateMachineStorage.java
----------------------------------------------------------------------
diff --git a/raft-server/src/main/java/org/apache/raft/statemachine/SimpleStateMachineStorage.java b/raft-server/src/main/java/org/apache/raft/statemachine/SimpleStateMachineStorage.java
index bedb5b0..c317eb8 100644
--- a/raft-server/src/main/java/org/apache/raft/statemachine/SimpleStateMachineStorage.java
+++ b/raft-server/src/main/java/org/apache/raft/statemachine/SimpleStateMachineStorage.java
@@ -89,7 +89,7 @@ public class SimpleStateMachineStorage implements StateMachineStorage {
     }
     final long term = Long.parseLong(m.group(1));
     final long index = Long.parseLong(m.group(2));
-    return new TermIndex(term, index);
+    return TermIndex.newTermIndex(term, index);
   }
 
   protected static String getTmpSnapshotFileName(long term, long endIndex) {

http://git-wip-us.apache.org/repos/asf/incubator-ratis/blob/59460582/raft-server/src/main/java/org/apache/raft/statemachine/SnapshotInfoImpl.java
----------------------------------------------------------------------
diff --git a/raft-server/src/main/java/org/apache/raft/statemachine/SnapshotInfoImpl.java b/raft-server/src/main/java/org/apache/raft/statemachine/SnapshotInfoImpl.java
index 670bfc7..1929614 100644
--- a/raft-server/src/main/java/org/apache/raft/statemachine/SnapshotInfoImpl.java
+++ b/raft-server/src/main/java/org/apache/raft/statemachine/SnapshotInfoImpl.java
@@ -33,7 +33,7 @@ public class SnapshotInfoImpl implements SnapshotInfo {
                           List<FileInfo> files, long term, long index) {
     this.raftConfiguration = raftConfiguration;
     this.files = files;
-    this.termIndex = new TermIndex(term, index);
+    this.termIndex = TermIndex.newTermIndex(term, index);
   }
 
   @Override

http://git-wip-us.apache.org/repos/asf/incubator-ratis/blob/59460582/raft-server/src/test/java/org/apache/raft/statemachine/SimpleStateMachine4Testing.java
----------------------------------------------------------------------
diff --git a/raft-server/src/test/java/org/apache/raft/statemachine/SimpleStateMachine4Testing.java b/raft-server/src/test/java/org/apache/raft/statemachine/SimpleStateMachine4Testing.java
index d11bfd4..0709d22 100644
--- a/raft-server/src/test/java/org/apache/raft/statemachine/SimpleStateMachine4Testing.java
+++ b/raft-server/src/test/java/org/apache/raft/statemachine/SimpleStateMachine4Testing.java
@@ -27,6 +27,7 @@ import org.apache.raft.protocol.RaftClientRequest;
 import org.apache.raft.server.impl.RaftServerConstants;
 import org.apache.raft.server.impl.RaftServerImpl;
 import org.apache.raft.server.impl.RaftServerTestUtil;
+import org.apache.raft.server.impl.ServerProtoUtils;
 import org.apache.raft.server.protocol.TermIndex;
 import org.apache.raft.server.storage.LogInputStream;
 import org.apache.raft.server.storage.LogOutputStream;
@@ -127,8 +128,9 @@ public class SimpleStateMachine4Testing extends BaseStateMachine {
   @Override
   public CompletableFuture<Message> applyTransaction(TransactionContext trx) {
     LogEntryProto entry = trx.getLogEntry().get();
+    Preconditions.checkNotNull(entry);
     list.add(entry);
-    termIndexTracker.update(new TermIndex(entry.getTerm(), entry.getIndex()));
+    termIndexTracker.update(ServerProtoUtils.toTermIndex(entry));
     return CompletableFuture.completedFuture(
         new SimpleMessage(entry.getIndex() + " OK"));
   }
@@ -197,8 +199,7 @@ public class SimpleStateMachine4Testing extends BaseStateMachine {
         LogEntryProto entry;
         while ((entry = in.nextEntry()) != null) {
           list.add(entry);
-          termIndexTracker.update(
-              new TermIndex(entry.getTerm(), entry.getIndex()));
+          termIndexTracker.update(ServerProtoUtils.toTermIndex(entry));
         }
       }
       Preconditions.checkState(

http://git-wip-us.apache.org/repos/asf/incubator-ratis/blob/59460582/raft-server/src/test/java/org/apache/raft/statemachine/TermIndexTracker.java
----------------------------------------------------------------------
diff --git a/raft-server/src/test/java/org/apache/raft/statemachine/TermIndexTracker.java b/raft-server/src/test/java/org/apache/raft/statemachine/TermIndexTracker.java
index fa9c130..b08fe11 100644
--- a/raft-server/src/test/java/org/apache/raft/statemachine/TermIndexTracker.java
+++ b/raft-server/src/test/java/org/apache/raft/statemachine/TermIndexTracker.java
@@ -28,7 +28,7 @@ import static org.apache.raft.server.impl.RaftServerConstants.INVALID_LOG_INDEX;
  */
 class TermIndexTracker {
   static final TermIndex INIT_TERMINDEX =
-      new TermIndex(INVALID_LOG_INDEX, INVALID_LOG_INDEX);
+      TermIndex.newTermIndex(INVALID_LOG_INDEX, INVALID_LOG_INDEX);
 
   private TermIndex current = INIT_TERMINDEX;
 


[43/54] [abbrv] incubator-ratis git commit: Renamed the packages from raft to ratis in preperation for Apache Incubation - Moved all java packages from org.apache.raft to org.apache.ratis. - Moved native package to org_apache_ratis, and native lib to l

Posted by ji...@apache.org.
http://git-wip-us.apache.org/repos/asf/incubator-ratis/blob/7e71a2e0/raft-common/src/main/native/src/org/apache/raft/util/crc32c_tables.h
----------------------------------------------------------------------
diff --git a/raft-common/src/main/native/src/org/apache/raft/util/crc32c_tables.h b/raft-common/src/main/native/src/org/apache/raft/util/crc32c_tables.h
deleted file mode 100644
index d54c4db..0000000
--- a/raft-common/src/main/native/src/org/apache/raft/util/crc32c_tables.h
+++ /dev/null
@@ -1,550 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- *     http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-
-/*
- * CRC-32 lookup tables generated by the polynomial 82F63B78
- * See also TestPureJavaCrc32.Table.
- */
-const uint32_t CRC32C_T8_0[256] = {
-  0x00000000, 0xF26B8303, 0xE13B70F7, 0x1350F3F4, 
-  0xC79A971F, 0x35F1141C, 0x26A1E7E8, 0xD4CA64EB, 
-  0x8AD958CF, 0x78B2DBCC, 0x6BE22838, 0x9989AB3B, 
-  0x4D43CFD0, 0xBF284CD3, 0xAC78BF27, 0x5E133C24, 
-  0x105EC76F, 0xE235446C, 0xF165B798, 0x030E349B, 
-  0xD7C45070, 0x25AFD373, 0x36FF2087, 0xC494A384, 
-  0x9A879FA0, 0x68EC1CA3, 0x7BBCEF57, 0x89D76C54, 
-  0x5D1D08BF, 0xAF768BBC, 0xBC267848, 0x4E4DFB4B, 
-  0x20BD8EDE, 0xD2D60DDD, 0xC186FE29, 0x33ED7D2A, 
-  0xE72719C1, 0x154C9AC2, 0x061C6936, 0xF477EA35, 
-  0xAA64D611, 0x580F5512, 0x4B5FA6E6, 0xB93425E5, 
-  0x6DFE410E, 0x9F95C20D, 0x8CC531F9, 0x7EAEB2FA, 
-  0x30E349B1, 0xC288CAB2, 0xD1D83946, 0x23B3BA45, 
-  0xF779DEAE, 0x05125DAD, 0x1642AE59, 0xE4292D5A, 
-  0xBA3A117E, 0x4851927D, 0x5B016189, 0xA96AE28A, 
-  0x7DA08661, 0x8FCB0562, 0x9C9BF696, 0x6EF07595, 
-  0x417B1DBC, 0xB3109EBF, 0xA0406D4B, 0x522BEE48, 
-  0x86E18AA3, 0x748A09A0, 0x67DAFA54, 0x95B17957, 
-  0xCBA24573, 0x39C9C670, 0x2A993584, 0xD8F2B687, 
-  0x0C38D26C, 0xFE53516F, 0xED03A29B, 0x1F682198, 
-  0x5125DAD3, 0xA34E59D0, 0xB01EAA24, 0x42752927, 
-  0x96BF4DCC, 0x64D4CECF, 0x77843D3B, 0x85EFBE38, 
-  0xDBFC821C, 0x2997011F, 0x3AC7F2EB, 0xC8AC71E8, 
-  0x1C661503, 0xEE0D9600, 0xFD5D65F4, 0x0F36E6F7, 
-  0x61C69362, 0x93AD1061, 0x80FDE395, 0x72966096, 
-  0xA65C047D, 0x5437877E, 0x4767748A, 0xB50CF789, 
-  0xEB1FCBAD, 0x197448AE, 0x0A24BB5A, 0xF84F3859, 
-  0x2C855CB2, 0xDEEEDFB1, 0xCDBE2C45, 0x3FD5AF46, 
-  0x7198540D, 0x83F3D70E, 0x90A324FA, 0x62C8A7F9, 
-  0xB602C312, 0x44694011, 0x5739B3E5, 0xA55230E6, 
-  0xFB410CC2, 0x092A8FC1, 0x1A7A7C35, 0xE811FF36, 
-  0x3CDB9BDD, 0xCEB018DE, 0xDDE0EB2A, 0x2F8B6829, 
-  0x82F63B78, 0x709DB87B, 0x63CD4B8F, 0x91A6C88C, 
-  0x456CAC67, 0xB7072F64, 0xA457DC90, 0x563C5F93, 
-  0x082F63B7, 0xFA44E0B4, 0xE9141340, 0x1B7F9043, 
-  0xCFB5F4A8, 0x3DDE77AB, 0x2E8E845F, 0xDCE5075C, 
-  0x92A8FC17, 0x60C37F14, 0x73938CE0, 0x81F80FE3, 
-  0x55326B08, 0xA759E80B, 0xB4091BFF, 0x466298FC, 
-  0x1871A4D8, 0xEA1A27DB, 0xF94AD42F, 0x0B21572C, 
-  0xDFEB33C7, 0x2D80B0C4, 0x3ED04330, 0xCCBBC033, 
-  0xA24BB5A6, 0x502036A5, 0x4370C551, 0xB11B4652, 
-  0x65D122B9, 0x97BAA1BA, 0x84EA524E, 0x7681D14D, 
-  0x2892ED69, 0xDAF96E6A, 0xC9A99D9E, 0x3BC21E9D, 
-  0xEF087A76, 0x1D63F975, 0x0E330A81, 0xFC588982, 
-  0xB21572C9, 0x407EF1CA, 0x532E023E, 0xA145813D, 
-  0x758FE5D6, 0x87E466D5, 0x94B49521, 0x66DF1622, 
-  0x38CC2A06, 0xCAA7A905, 0xD9F75AF1, 0x2B9CD9F2, 
-  0xFF56BD19, 0x0D3D3E1A, 0x1E6DCDEE, 0xEC064EED, 
-  0xC38D26C4, 0x31E6A5C7, 0x22B65633, 0xD0DDD530, 
-  0x0417B1DB, 0xF67C32D8, 0xE52CC12C, 0x1747422F, 
-  0x49547E0B, 0xBB3FFD08, 0xA86F0EFC, 0x5A048DFF, 
-  0x8ECEE914, 0x7CA56A17, 0x6FF599E3, 0x9D9E1AE0, 
-  0xD3D3E1AB, 0x21B862A8, 0x32E8915C, 0xC083125F, 
-  0x144976B4, 0xE622F5B7, 0xF5720643, 0x07198540, 
-  0x590AB964, 0xAB613A67, 0xB831C993, 0x4A5A4A90, 
-  0x9E902E7B, 0x6CFBAD78, 0x7FAB5E8C, 0x8DC0DD8F, 
-  0xE330A81A, 0x115B2B19, 0x020BD8ED, 0xF0605BEE, 
-  0x24AA3F05, 0xD6C1BC06, 0xC5914FF2, 0x37FACCF1, 
-  0x69E9F0D5, 0x9B8273D6, 0x88D28022, 0x7AB90321, 
-  0xAE7367CA, 0x5C18E4C9, 0x4F48173D, 0xBD23943E, 
-  0xF36E6F75, 0x0105EC76, 0x12551F82, 0xE03E9C81, 
-  0x34F4F86A, 0xC69F7B69, 0xD5CF889D, 0x27A40B9E, 
-  0x79B737BA, 0x8BDCB4B9, 0x988C474D, 0x6AE7C44E, 
-  0xBE2DA0A5, 0x4C4623A6, 0x5F16D052, 0xAD7D5351
-};
-const uint32_t CRC32C_T8_1[256] = {
-  0x00000000, 0x13A29877, 0x274530EE, 0x34E7A899, 
-  0x4E8A61DC, 0x5D28F9AB, 0x69CF5132, 0x7A6DC945, 
-  0x9D14C3B8, 0x8EB65BCF, 0xBA51F356, 0xA9F36B21, 
-  0xD39EA264, 0xC03C3A13, 0xF4DB928A, 0xE7790AFD, 
-  0x3FC5F181, 0x2C6769F6, 0x1880C16F, 0x0B225918, 
-  0x714F905D, 0x62ED082A, 0x560AA0B3, 0x45A838C4, 
-  0xA2D13239, 0xB173AA4E, 0x859402D7, 0x96369AA0, 
-  0xEC5B53E5, 0xFFF9CB92, 0xCB1E630B, 0xD8BCFB7C, 
-  0x7F8BE302, 0x6C297B75, 0x58CED3EC, 0x4B6C4B9B, 
-  0x310182DE, 0x22A31AA9, 0x1644B230, 0x05E62A47, 
-  0xE29F20BA, 0xF13DB8CD, 0xC5DA1054, 0xD6788823, 
-  0xAC154166, 0xBFB7D911, 0x8B507188, 0x98F2E9FF, 
-  0x404E1283, 0x53EC8AF4, 0x670B226D, 0x74A9BA1A, 
-  0x0EC4735F, 0x1D66EB28, 0x298143B1, 0x3A23DBC6, 
-  0xDD5AD13B, 0xCEF8494C, 0xFA1FE1D5, 0xE9BD79A2, 
-  0x93D0B0E7, 0x80722890, 0xB4958009, 0xA737187E, 
-  0xFF17C604, 0xECB55E73, 0xD852F6EA, 0xCBF06E9D, 
-  0xB19DA7D8, 0xA23F3FAF, 0x96D89736, 0x857A0F41, 
-  0x620305BC, 0x71A19DCB, 0x45463552, 0x56E4AD25, 
-  0x2C896460, 0x3F2BFC17, 0x0BCC548E, 0x186ECCF9, 
-  0xC0D23785, 0xD370AFF2, 0xE797076B, 0xF4359F1C, 
-  0x8E585659, 0x9DFACE2E, 0xA91D66B7, 0xBABFFEC0, 
-  0x5DC6F43D, 0x4E646C4A, 0x7A83C4D3, 0x69215CA4, 
-  0x134C95E1, 0x00EE0D96, 0x3409A50F, 0x27AB3D78, 
-  0x809C2506, 0x933EBD71, 0xA7D915E8, 0xB47B8D9F, 
-  0xCE1644DA, 0xDDB4DCAD, 0xE9537434, 0xFAF1EC43, 
-  0x1D88E6BE, 0x0E2A7EC9, 0x3ACDD650, 0x296F4E27, 
-  0x53028762, 0x40A01F15, 0x7447B78C, 0x67E52FFB, 
-  0xBF59D487, 0xACFB4CF0, 0x981CE469, 0x8BBE7C1E, 
-  0xF1D3B55B, 0xE2712D2C, 0xD69685B5, 0xC5341DC2, 
-  0x224D173F, 0x31EF8F48, 0x050827D1, 0x16AABFA6, 
-  0x6CC776E3, 0x7F65EE94, 0x4B82460D, 0x5820DE7A, 
-  0xFBC3FAF9, 0xE861628E, 0xDC86CA17, 0xCF245260, 
-  0xB5499B25, 0xA6EB0352, 0x920CABCB, 0x81AE33BC, 
-  0x66D73941, 0x7575A136, 0x419209AF, 0x523091D8, 
-  0x285D589D, 0x3BFFC0EA, 0x0F186873, 0x1CBAF004, 
-  0xC4060B78, 0xD7A4930F, 0xE3433B96, 0xF0E1A3E1, 
-  0x8A8C6AA4, 0x992EF2D3, 0xADC95A4A, 0xBE6BC23D, 
-  0x5912C8C0, 0x4AB050B7, 0x7E57F82E, 0x6DF56059, 
-  0x1798A91C, 0x043A316B, 0x30DD99F2, 0x237F0185, 
-  0x844819FB, 0x97EA818C, 0xA30D2915, 0xB0AFB162, 
-  0xCAC27827, 0xD960E050, 0xED8748C9, 0xFE25D0BE, 
-  0x195CDA43, 0x0AFE4234, 0x3E19EAAD, 0x2DBB72DA, 
-  0x57D6BB9F, 0x447423E8, 0x70938B71, 0x63311306, 
-  0xBB8DE87A, 0xA82F700D, 0x9CC8D894, 0x8F6A40E3, 
-  0xF50789A6, 0xE6A511D1, 0xD242B948, 0xC1E0213F, 
-  0x26992BC2, 0x353BB3B5, 0x01DC1B2C, 0x127E835B, 
-  0x68134A1E, 0x7BB1D269, 0x4F567AF0, 0x5CF4E287, 
-  0x04D43CFD, 0x1776A48A, 0x23910C13, 0x30339464, 
-  0x4A5E5D21, 0x59FCC556, 0x6D1B6DCF, 0x7EB9F5B8, 
-  0x99C0FF45, 0x8A626732, 0xBE85CFAB, 0xAD2757DC, 
-  0xD74A9E99, 0xC4E806EE, 0xF00FAE77, 0xE3AD3600, 
-  0x3B11CD7C, 0x28B3550B, 0x1C54FD92, 0x0FF665E5, 
-  0x759BACA0, 0x663934D7, 0x52DE9C4E, 0x417C0439, 
-  0xA6050EC4, 0xB5A796B3, 0x81403E2A, 0x92E2A65D, 
-  0xE88F6F18, 0xFB2DF76F, 0xCFCA5FF6, 0xDC68C781, 
-  0x7B5FDFFF, 0x68FD4788, 0x5C1AEF11, 0x4FB87766, 
-  0x35D5BE23, 0x26772654, 0x12908ECD, 0x013216BA, 
-  0xE64B1C47, 0xF5E98430, 0xC10E2CA9, 0xD2ACB4DE, 
-  0xA8C17D9B, 0xBB63E5EC, 0x8F844D75, 0x9C26D502, 
-  0x449A2E7E, 0x5738B609, 0x63DF1E90, 0x707D86E7, 
-  0x0A104FA2, 0x19B2D7D5, 0x2D557F4C, 0x3EF7E73B, 
-  0xD98EEDC6, 0xCA2C75B1, 0xFECBDD28, 0xED69455F, 
-  0x97048C1A, 0x84A6146D, 0xB041BCF4, 0xA3E32483
-};
-const uint32_t CRC32C_T8_2[256] = {
-  0x00000000, 0xA541927E, 0x4F6F520D, 0xEA2EC073, 
-  0x9EDEA41A, 0x3B9F3664, 0xD1B1F617, 0x74F06469, 
-  0x38513EC5, 0x9D10ACBB, 0x773E6CC8, 0xD27FFEB6, 
-  0xA68F9ADF, 0x03CE08A1, 0xE9E0C8D2, 0x4CA15AAC, 
-  0x70A27D8A, 0xD5E3EFF4, 0x3FCD2F87, 0x9A8CBDF9, 
-  0xEE7CD990, 0x4B3D4BEE, 0xA1138B9D, 0x045219E3, 
-  0x48F3434F, 0xEDB2D131, 0x079C1142, 0xA2DD833C, 
-  0xD62DE755, 0x736C752B, 0x9942B558, 0x3C032726, 
-  0xE144FB14, 0x4405696A, 0xAE2BA919, 0x0B6A3B67, 
-  0x7F9A5F0E, 0xDADBCD70, 0x30F50D03, 0x95B49F7D, 
-  0xD915C5D1, 0x7C5457AF, 0x967A97DC, 0x333B05A2, 
-  0x47CB61CB, 0xE28AF3B5, 0x08A433C6, 0xADE5A1B8, 
-  0x91E6869E, 0x34A714E0, 0xDE89D493, 0x7BC846ED, 
-  0x0F382284, 0xAA79B0FA, 0x40577089, 0xE516E2F7, 
-  0xA9B7B85B, 0x0CF62A25, 0xE6D8EA56, 0x43997828, 
-  0x37691C41, 0x92288E3F, 0x78064E4C, 0xDD47DC32, 
-  0xC76580D9, 0x622412A7, 0x880AD2D4, 0x2D4B40AA, 
-  0x59BB24C3, 0xFCFAB6BD, 0x16D476CE, 0xB395E4B0, 
-  0xFF34BE1C, 0x5A752C62, 0xB05BEC11, 0x151A7E6F, 
-  0x61EA1A06, 0xC4AB8878, 0x2E85480B, 0x8BC4DA75, 
-  0xB7C7FD53, 0x12866F2D, 0xF8A8AF5E, 0x5DE93D20, 
-  0x29195949, 0x8C58CB37, 0x66760B44, 0xC337993A, 
-  0x8F96C396, 0x2AD751E8, 0xC0F9919B, 0x65B803E5, 
-  0x1148678C, 0xB409F5F2, 0x5E273581, 0xFB66A7FF, 
-  0x26217BCD, 0x8360E9B3, 0x694E29C0, 0xCC0FBBBE, 
-  0xB8FFDFD7, 0x1DBE4DA9, 0xF7908DDA, 0x52D11FA4, 
-  0x1E704508, 0xBB31D776, 0x511F1705, 0xF45E857B, 
-  0x80AEE112, 0x25EF736C, 0xCFC1B31F, 0x6A802161, 
-  0x56830647, 0xF3C29439, 0x19EC544A, 0xBCADC634, 
-  0xC85DA25D, 0x6D1C3023, 0x8732F050, 0x2273622E, 
-  0x6ED23882, 0xCB93AAFC, 0x21BD6A8F, 0x84FCF8F1, 
-  0xF00C9C98, 0x554D0EE6, 0xBF63CE95, 0x1A225CEB, 
-  0x8B277743, 0x2E66E53D, 0xC448254E, 0x6109B730, 
-  0x15F9D359, 0xB0B84127, 0x5A968154, 0xFFD7132A, 
-  0xB3764986, 0x1637DBF8, 0xFC191B8B, 0x595889F5, 
-  0x2DA8ED9C, 0x88E97FE2, 0x62C7BF91, 0xC7862DEF, 
-  0xFB850AC9, 0x5EC498B7, 0xB4EA58C4, 0x11ABCABA, 
-  0x655BAED3, 0xC01A3CAD, 0x2A34FCDE, 0x8F756EA0, 
-  0xC3D4340C, 0x6695A672, 0x8CBB6601, 0x29FAF47F, 
-  0x5D0A9016, 0xF84B0268, 0x1265C21B, 0xB7245065, 
-  0x6A638C57, 0xCF221E29, 0x250CDE5A, 0x804D4C24, 
-  0xF4BD284D, 0x51FCBA33, 0xBBD27A40, 0x1E93E83E, 
-  0x5232B292, 0xF77320EC, 0x1D5DE09F, 0xB81C72E1, 
-  0xCCEC1688, 0x69AD84F6, 0x83834485, 0x26C2D6FB, 
-  0x1AC1F1DD, 0xBF8063A3, 0x55AEA3D0, 0xF0EF31AE, 
-  0x841F55C7, 0x215EC7B9, 0xCB7007CA, 0x6E3195B4, 
-  0x2290CF18, 0x87D15D66, 0x6DFF9D15, 0xC8BE0F6B, 
-  0xBC4E6B02, 0x190FF97C, 0xF321390F, 0x5660AB71, 
-  0x4C42F79A, 0xE90365E4, 0x032DA597, 0xA66C37E9, 
-  0xD29C5380, 0x77DDC1FE, 0x9DF3018D, 0x38B293F3, 
-  0x7413C95F, 0xD1525B21, 0x3B7C9B52, 0x9E3D092C, 
-  0xEACD6D45, 0x4F8CFF3B, 0xA5A23F48, 0x00E3AD36, 
-  0x3CE08A10, 0x99A1186E, 0x738FD81D, 0xD6CE4A63, 
-  0xA23E2E0A, 0x077FBC74, 0xED517C07, 0x4810EE79, 
-  0x04B1B4D5, 0xA1F026AB, 0x4BDEE6D8, 0xEE9F74A6, 
-  0x9A6F10CF, 0x3F2E82B1, 0xD50042C2, 0x7041D0BC, 
-  0xAD060C8E, 0x08479EF0, 0xE2695E83, 0x4728CCFD, 
-  0x33D8A894, 0x96993AEA, 0x7CB7FA99, 0xD9F668E7, 
-  0x9557324B, 0x3016A035, 0xDA386046, 0x7F79F238, 
-  0x0B899651, 0xAEC8042F, 0x44E6C45C, 0xE1A75622, 
-  0xDDA47104, 0x78E5E37A, 0x92CB2309, 0x378AB177, 
-  0x437AD51E, 0xE63B4760, 0x0C158713, 0xA954156D, 
-  0xE5F54FC1, 0x40B4DDBF, 0xAA9A1DCC, 0x0FDB8FB2, 
-  0x7B2BEBDB, 0xDE6A79A5, 0x3444B9D6, 0x91052BA8
-};
-const uint32_t CRC32C_T8_3[256] = {
-  0x00000000, 0xDD45AAB8, 0xBF672381, 0x62228939, 
-  0x7B2231F3, 0xA6679B4B, 0xC4451272, 0x1900B8CA, 
-  0xF64463E6, 0x2B01C95E, 0x49234067, 0x9466EADF, 
-  0x8D665215, 0x5023F8AD, 0x32017194, 0xEF44DB2C, 
-  0xE964B13D, 0x34211B85, 0x560392BC, 0x8B463804, 
-  0x924680CE, 0x4F032A76, 0x2D21A34F, 0xF06409F7, 
-  0x1F20D2DB, 0xC2657863, 0xA047F15A, 0x7D025BE2, 
-  0x6402E328, 0xB9474990, 0xDB65C0A9, 0x06206A11, 
-  0xD725148B, 0x0A60BE33, 0x6842370A, 0xB5079DB2, 
-  0xAC072578, 0x71428FC0, 0x136006F9, 0xCE25AC41, 
-  0x2161776D, 0xFC24DDD5, 0x9E0654EC, 0x4343FE54, 
-  0x5A43469E, 0x8706EC26, 0xE524651F, 0x3861CFA7, 
-  0x3E41A5B6, 0xE3040F0E, 0x81268637, 0x5C632C8F, 
-  0x45639445, 0x98263EFD, 0xFA04B7C4, 0x27411D7C, 
-  0xC805C650, 0x15406CE8, 0x7762E5D1, 0xAA274F69, 
-  0xB327F7A3, 0x6E625D1B, 0x0C40D422, 0xD1057E9A, 
-  0xABA65FE7, 0x76E3F55F, 0x14C17C66, 0xC984D6DE, 
-  0xD0846E14, 0x0DC1C4AC, 0x6FE34D95, 0xB2A6E72D, 
-  0x5DE23C01, 0x80A796B9, 0xE2851F80, 0x3FC0B538, 
-  0x26C00DF2, 0xFB85A74A, 0x99A72E73, 0x44E284CB, 
-  0x42C2EEDA, 0x9F874462, 0xFDA5CD5B, 0x20E067E3, 
-  0x39E0DF29, 0xE4A57591, 0x8687FCA8, 0x5BC25610, 
-  0xB4868D3C, 0x69C32784, 0x0BE1AEBD, 0xD6A40405, 
-  0xCFA4BCCF, 0x12E11677, 0x70C39F4E, 0xAD8635F6, 
-  0x7C834B6C, 0xA1C6E1D4, 0xC3E468ED, 0x1EA1C255, 
-  0x07A17A9F, 0xDAE4D027, 0xB8C6591E, 0x6583F3A6, 
-  0x8AC7288A, 0x57828232, 0x35A00B0B, 0xE8E5A1B3, 
-  0xF1E51979, 0x2CA0B3C1, 0x4E823AF8, 0x93C79040, 
-  0x95E7FA51, 0x48A250E9, 0x2A80D9D0, 0xF7C57368, 
-  0xEEC5CBA2, 0x3380611A, 0x51A2E823, 0x8CE7429B, 
-  0x63A399B7, 0xBEE6330F, 0xDCC4BA36, 0x0181108E, 
-  0x1881A844, 0xC5C402FC, 0xA7E68BC5, 0x7AA3217D, 
-  0x52A0C93F, 0x8FE56387, 0xEDC7EABE, 0x30824006, 
-  0x2982F8CC, 0xF4C75274, 0x96E5DB4D, 0x4BA071F5, 
-  0xA4E4AAD9, 0x79A10061, 0x1B838958, 0xC6C623E0, 
-  0xDFC69B2A, 0x02833192, 0x60A1B8AB, 0xBDE41213, 
-  0xBBC47802, 0x6681D2BA, 0x04A35B83, 0xD9E6F13B, 
-  0xC0E649F1, 0x1DA3E349, 0x7F816A70, 0xA2C4C0C8, 
-  0x4D801BE4, 0x90C5B15C, 0xF2E73865, 0x2FA292DD, 
-  0x36A22A17, 0xEBE780AF, 0x89C50996, 0x5480A32E, 
-  0x8585DDB4, 0x58C0770C, 0x3AE2FE35, 0xE7A7548D, 
-  0xFEA7EC47, 0x23E246FF, 0x41C0CFC6, 0x9C85657E, 
-  0x73C1BE52, 0xAE8414EA, 0xCCA69DD3, 0x11E3376B, 
-  0x08E38FA1, 0xD5A62519, 0xB784AC20, 0x6AC10698, 
-  0x6CE16C89, 0xB1A4C631, 0xD3864F08, 0x0EC3E5B0, 
-  0x17C35D7A, 0xCA86F7C2, 0xA8A47EFB, 0x75E1D443, 
-  0x9AA50F6F, 0x47E0A5D7, 0x25C22CEE, 0xF8878656, 
-  0xE1873E9C, 0x3CC29424, 0x5EE01D1D, 0x83A5B7A5, 
-  0xF90696D8, 0x24433C60, 0x4661B559, 0x9B241FE1, 
-  0x8224A72B, 0x5F610D93, 0x3D4384AA, 0xE0062E12, 
-  0x0F42F53E, 0xD2075F86, 0xB025D6BF, 0x6D607C07, 
-  0x7460C4CD, 0xA9256E75, 0xCB07E74C, 0x16424DF4, 
-  0x106227E5, 0xCD278D5D, 0xAF050464, 0x7240AEDC, 
-  0x6B401616, 0xB605BCAE, 0xD4273597, 0x09629F2F, 
-  0xE6264403, 0x3B63EEBB, 0x59416782, 0x8404CD3A, 
-  0x9D0475F0, 0x4041DF48, 0x22635671, 0xFF26FCC9, 
-  0x2E238253, 0xF36628EB, 0x9144A1D2, 0x4C010B6A, 
-  0x5501B3A0, 0x88441918, 0xEA669021, 0x37233A99, 
-  0xD867E1B5, 0x05224B0D, 0x6700C234, 0xBA45688C, 
-  0xA345D046, 0x7E007AFE, 0x1C22F3C7, 0xC167597F, 
-  0xC747336E, 0x1A0299D6, 0x782010EF, 0xA565BA57, 
-  0xBC65029D, 0x6120A825, 0x0302211C, 0xDE478BA4, 
-  0x31035088, 0xEC46FA30, 0x8E647309, 0x5321D9B1, 
-  0x4A21617B, 0x9764CBC3, 0xF54642FA, 0x2803E842
-};
-const uint32_t CRC32C_T8_4[256] = {
-  0x00000000, 0x38116FAC, 0x7022DF58, 0x4833B0F4, 
-  0xE045BEB0, 0xD854D11C, 0x906761E8, 0xA8760E44, 
-  0xC5670B91, 0xFD76643D, 0xB545D4C9, 0x8D54BB65, 
-  0x2522B521, 0x1D33DA8D, 0x55006A79, 0x6D1105D5, 
-  0x8F2261D3, 0xB7330E7F, 0xFF00BE8B, 0xC711D127, 
-  0x6F67DF63, 0x5776B0CF, 0x1F45003B, 0x27546F97, 
-  0x4A456A42, 0x725405EE, 0x3A67B51A, 0x0276DAB6, 
-  0xAA00D4F2, 0x9211BB5E, 0xDA220BAA, 0xE2336406, 
-  0x1BA8B557, 0x23B9DAFB, 0x6B8A6A0F, 0x539B05A3, 
-  0xFBED0BE7, 0xC3FC644B, 0x8BCFD4BF, 0xB3DEBB13, 
-  0xDECFBEC6, 0xE6DED16A, 0xAEED619E, 0x96FC0E32, 
-  0x3E8A0076, 0x069B6FDA, 0x4EA8DF2E, 0x76B9B082, 
-  0x948AD484, 0xAC9BBB28, 0xE4A80BDC, 0xDCB96470, 
-  0x74CF6A34, 0x4CDE0598, 0x04EDB56C, 0x3CFCDAC0, 
-  0x51EDDF15, 0x69FCB0B9, 0x21CF004D, 0x19DE6FE1, 
-  0xB1A861A5, 0x89B90E09, 0xC18ABEFD, 0xF99BD151, 
-  0x37516AAE, 0x0F400502, 0x4773B5F6, 0x7F62DA5A, 
-  0xD714D41E, 0xEF05BBB2, 0xA7360B46, 0x9F2764EA, 
-  0xF236613F, 0xCA270E93, 0x8214BE67, 0xBA05D1CB, 
-  0x1273DF8F, 0x2A62B023, 0x625100D7, 0x5A406F7B, 
-  0xB8730B7D, 0x806264D1, 0xC851D425, 0xF040BB89, 
-  0x5836B5CD, 0x6027DA61, 0x28146A95, 0x10050539, 
-  0x7D1400EC, 0x45056F40, 0x0D36DFB4, 0x3527B018, 
-  0x9D51BE5C, 0xA540D1F0, 0xED736104, 0xD5620EA8, 
-  0x2CF9DFF9, 0x14E8B055, 0x5CDB00A1, 0x64CA6F0D, 
-  0xCCBC6149, 0xF4AD0EE5, 0xBC9EBE11, 0x848FD1BD, 
-  0xE99ED468, 0xD18FBBC4, 0x99BC0B30, 0xA1AD649C, 
-  0x09DB6AD8, 0x31CA0574, 0x79F9B580, 0x41E8DA2C, 
-  0xA3DBBE2A, 0x9BCAD186, 0xD3F96172, 0xEBE80EDE, 
-  0x439E009A, 0x7B8F6F36, 0x33BCDFC2, 0x0BADB06E, 
-  0x66BCB5BB, 0x5EADDA17, 0x169E6AE3, 0x2E8F054F, 
-  0x86F90B0B, 0xBEE864A7, 0xF6DBD453, 0xCECABBFF, 
-  0x6EA2D55C, 0x56B3BAF0, 0x1E800A04, 0x269165A8, 
-  0x8EE76BEC, 0xB6F60440, 0xFEC5B4B4, 0xC6D4DB18, 
-  0xABC5DECD, 0x93D4B161, 0xDBE70195, 0xE3F66E39, 
-  0x4B80607D, 0x73910FD1, 0x3BA2BF25, 0x03B3D089, 
-  0xE180B48F, 0xD991DB23, 0x91A26BD7, 0xA9B3047B, 
-  0x01C50A3F, 0x39D46593, 0x71E7D567, 0x49F6BACB, 
-  0x24E7BF1E, 0x1CF6D0B2, 0x54C56046, 0x6CD40FEA, 
-  0xC4A201AE, 0xFCB36E02, 0xB480DEF6, 0x8C91B15A, 
-  0x750A600B, 0x4D1B0FA7, 0x0528BF53, 0x3D39D0FF, 
-  0x954FDEBB, 0xAD5EB117, 0xE56D01E3, 0xDD7C6E4F, 
-  0xB06D6B9A, 0x887C0436, 0xC04FB4C2, 0xF85EDB6E, 
-  0x5028D52A, 0x6839BA86, 0x200A0A72, 0x181B65DE, 
-  0xFA2801D8, 0xC2396E74, 0x8A0ADE80, 0xB21BB12C, 
-  0x1A6DBF68, 0x227CD0C4, 0x6A4F6030, 0x525E0F9C, 
-  0x3F4F0A49, 0x075E65E5, 0x4F6DD511, 0x777CBABD, 
-  0xDF0AB4F9, 0xE71BDB55, 0xAF286BA1, 0x9739040D, 
-  0x59F3BFF2, 0x61E2D05E, 0x29D160AA, 0x11C00F06, 
-  0xB9B60142, 0x81A76EEE, 0xC994DE1A, 0xF185B1B6, 
-  0x9C94B463, 0xA485DBCF, 0xECB66B3B, 0xD4A70497, 
-  0x7CD10AD3, 0x44C0657F, 0x0CF3D58B, 0x34E2BA27, 
-  0xD6D1DE21, 0xEEC0B18D, 0xA6F30179, 0x9EE26ED5, 
-  0x36946091, 0x0E850F3D, 0x46B6BFC9, 0x7EA7D065, 
-  0x13B6D5B0, 0x2BA7BA1C, 0x63940AE8, 0x5B856544, 
-  0xF3F36B00, 0xCBE204AC, 0x83D1B458, 0xBBC0DBF4, 
-  0x425B0AA5, 0x7A4A6509, 0x3279D5FD, 0x0A68BA51, 
-  0xA21EB415, 0x9A0FDBB9, 0xD23C6B4D, 0xEA2D04E1, 
-  0x873C0134, 0xBF2D6E98, 0xF71EDE6C, 0xCF0FB1C0, 
-  0x6779BF84, 0x5F68D028, 0x175B60DC, 0x2F4A0F70, 
-  0xCD796B76, 0xF56804DA, 0xBD5BB42E, 0x854ADB82, 
-  0x2D3CD5C6, 0x152DBA6A, 0x5D1E0A9E, 0x650F6532, 
-  0x081E60E7, 0x300F0F4B, 0x783CBFBF, 0x402DD013, 
-  0xE85BDE57, 0xD04AB1FB, 0x9879010F, 0xA0686EA3
-};
-const uint32_t CRC32C_T8_5[256] = {
-  0x00000000, 0xEF306B19, 0xDB8CA0C3, 0x34BCCBDA, 
-  0xB2F53777, 0x5DC55C6E, 0x697997B4, 0x8649FCAD, 
-  0x6006181F, 0x8F367306, 0xBB8AB8DC, 0x54BAD3C5, 
-  0xD2F32F68, 0x3DC34471, 0x097F8FAB, 0xE64FE4B2, 
-  0xC00C303E, 0x2F3C5B27, 0x1B8090FD, 0xF4B0FBE4, 
-  0x72F90749, 0x9DC96C50, 0xA975A78A, 0x4645CC93, 
-  0xA00A2821, 0x4F3A4338, 0x7B8688E2, 0x94B6E3FB, 
-  0x12FF1F56, 0xFDCF744F, 0xC973BF95, 0x2643D48C, 
-  0x85F4168D, 0x6AC47D94, 0x5E78B64E, 0xB148DD57, 
-  0x370121FA, 0xD8314AE3, 0xEC8D8139, 0x03BDEA20, 
-  0xE5F20E92, 0x0AC2658B, 0x3E7EAE51, 0xD14EC548, 
-  0x570739E5, 0xB83752FC, 0x8C8B9926, 0x63BBF23F, 
-  0x45F826B3, 0xAAC84DAA, 0x9E748670, 0x7144ED69, 
-  0xF70D11C4, 0x183D7ADD, 0x2C81B107, 0xC3B1DA1E, 
-  0x25FE3EAC, 0xCACE55B5, 0xFE729E6F, 0x1142F576, 
-  0x970B09DB, 0x783B62C2, 0x4C87A918, 0xA3B7C201, 
-  0x0E045BEB, 0xE13430F2, 0xD588FB28, 0x3AB89031, 
-  0xBCF16C9C, 0x53C10785, 0x677DCC5F, 0x884DA746, 
-  0x6E0243F4, 0x813228ED, 0xB58EE337, 0x5ABE882E, 
-  0xDCF77483, 0x33C71F9A, 0x077BD440, 0xE84BBF59, 
-  0xCE086BD5, 0x213800CC, 0x1584CB16, 0xFAB4A00F, 
-  0x7CFD5CA2, 0x93CD37BB, 0xA771FC61, 0x48419778, 
-  0xAE0E73CA, 0x413E18D3, 0x7582D309, 0x9AB2B810, 
-  0x1CFB44BD, 0xF3CB2FA4, 0xC777E47E, 0x28478F67, 
-  0x8BF04D66, 0x64C0267F, 0x507CEDA5, 0xBF4C86BC, 
-  0x39057A11, 0xD6351108, 0xE289DAD2, 0x0DB9B1CB, 
-  0xEBF65579, 0x04C63E60, 0x307AF5BA, 0xDF4A9EA3, 
-  0x5903620E, 0xB6330917, 0x828FC2CD, 0x6DBFA9D4, 
-  0x4BFC7D58, 0xA4CC1641, 0x9070DD9B, 0x7F40B682, 
-  0xF9094A2F, 0x16392136, 0x2285EAEC, 0xCDB581F5, 
-  0x2BFA6547, 0xC4CA0E5E, 0xF076C584, 0x1F46AE9D, 
-  0x990F5230, 0x763F3929, 0x4283F2F3, 0xADB399EA, 
-  0x1C08B7D6, 0xF338DCCF, 0xC7841715, 0x28B47C0C, 
-  0xAEFD80A1, 0x41CDEBB8, 0x75712062, 0x9A414B7B, 
-  0x7C0EAFC9, 0x933EC4D0, 0xA7820F0A, 0x48B26413, 
-  0xCEFB98BE, 0x21CBF3A7, 0x1577387D, 0xFA475364, 
-  0xDC0487E8, 0x3334ECF1, 0x0788272B, 0xE8B84C32, 
-  0x6EF1B09F, 0x81C1DB86, 0xB57D105C, 0x5A4D7B45, 
-  0xBC029FF7, 0x5332F4EE, 0x678E3F34, 0x88BE542D, 
-  0x0EF7A880, 0xE1C7C399, 0xD57B0843, 0x3A4B635A, 
-  0x99FCA15B, 0x76CCCA42, 0x42700198, 0xAD406A81, 
-  0x2B09962C, 0xC439FD35, 0xF08536EF, 0x1FB55DF6, 
-  0xF9FAB944, 0x16CAD25D, 0x22761987, 0xCD46729E, 
-  0x4B0F8E33, 0xA43FE52A, 0x90832EF0, 0x7FB345E9, 
-  0x59F09165, 0xB6C0FA7C, 0x827C31A6, 0x6D4C5ABF, 
-  0xEB05A612, 0x0435CD0B, 0x308906D1, 0xDFB96DC8, 
-  0x39F6897A, 0xD6C6E263, 0xE27A29B9, 0x0D4A42A0, 
-  0x8B03BE0D, 0x6433D514, 0x508F1ECE, 0xBFBF75D7, 
-  0x120CEC3D, 0xFD3C8724, 0xC9804CFE, 0x26B027E7, 
-  0xA0F9DB4A, 0x4FC9B053, 0x7B757B89, 0x94451090, 
-  0x720AF422, 0x9D3A9F3B, 0xA98654E1, 0x46B63FF8, 
-  0xC0FFC355, 0x2FCFA84C, 0x1B736396, 0xF443088F, 
-  0xD200DC03, 0x3D30B71A, 0x098C7CC0, 0xE6BC17D9, 
-  0x60F5EB74, 0x8FC5806D, 0xBB794BB7, 0x544920AE, 
-  0xB206C41C, 0x5D36AF05, 0x698A64DF, 0x86BA0FC6, 
-  0x00F3F36B, 0xEFC39872, 0xDB7F53A8, 0x344F38B1, 
-  0x97F8FAB0, 0x78C891A9, 0x4C745A73, 0xA344316A, 
-  0x250DCDC7, 0xCA3DA6DE, 0xFE816D04, 0x11B1061D, 
-  0xF7FEE2AF, 0x18CE89B6, 0x2C72426C, 0xC3422975, 
-  0x450BD5D8, 0xAA3BBEC1, 0x9E87751B, 0x71B71E02, 
-  0x57F4CA8E, 0xB8C4A197, 0x8C786A4D, 0x63480154, 
-  0xE501FDF9, 0x0A3196E0, 0x3E8D5D3A, 0xD1BD3623, 
-  0x37F2D291, 0xD8C2B988, 0xEC7E7252, 0x034E194B, 
-  0x8507E5E6, 0x6A378EFF, 0x5E8B4525, 0xB1BB2E3C
-};
-const uint32_t CRC32C_T8_6[256] = {
-  0x00000000, 0x68032CC8, 0xD0065990, 0xB8057558, 
-  0xA5E0C5D1, 0xCDE3E919, 0x75E69C41, 0x1DE5B089, 
-  0x4E2DFD53, 0x262ED19B, 0x9E2BA4C3, 0xF628880B, 
-  0xEBCD3882, 0x83CE144A, 0x3BCB6112, 0x53C84DDA, 
-  0x9C5BFAA6, 0xF458D66E, 0x4C5DA336, 0x245E8FFE, 
-  0x39BB3F77, 0x51B813BF, 0xE9BD66E7, 0x81BE4A2F, 
-  0xD27607F5, 0xBA752B3D, 0x02705E65, 0x6A7372AD, 
-  0x7796C224, 0x1F95EEEC, 0xA7909BB4, 0xCF93B77C, 
-  0x3D5B83BD, 0x5558AF75, 0xED5DDA2D, 0x855EF6E5, 
-  0x98BB466C, 0xF0B86AA4, 0x48BD1FFC, 0x20BE3334, 
-  0x73767EEE, 0x1B755226, 0xA370277E, 0xCB730BB6, 
-  0xD696BB3F, 0xBE9597F7, 0x0690E2AF, 0x6E93CE67, 
-  0xA100791B, 0xC90355D3, 0x7106208B, 0x19050C43, 
-  0x04E0BCCA, 0x6CE39002, 0xD4E6E55A, 0xBCE5C992, 
-  0xEF2D8448, 0x872EA880, 0x3F2BDDD8, 0x5728F110, 
-  0x4ACD4199, 0x22CE6D51, 0x9ACB1809, 0xF2C834C1, 
-  0x7AB7077A, 0x12B42BB2, 0xAAB15EEA, 0xC2B27222, 
-  0xDF57C2AB, 0xB754EE63, 0x0F519B3B, 0x6752B7F3, 
-  0x349AFA29, 0x5C99D6E1, 0xE49CA3B9, 0x8C9F8F71, 
-  0x917A3FF8, 0xF9791330, 0x417C6668, 0x297F4AA0, 
-  0xE6ECFDDC, 0x8EEFD114, 0x36EAA44C, 0x5EE98884, 
-  0x430C380D, 0x2B0F14C5, 0x930A619D, 0xFB094D55, 
-  0xA8C1008F, 0xC0C22C47, 0x78C7591F, 0x10C475D7, 
-  0x0D21C55E, 0x6522E996, 0xDD279CCE, 0xB524B006, 
-  0x47EC84C7, 0x2FEFA80F, 0x97EADD57, 0xFFE9F19F, 
-  0xE20C4116, 0x8A0F6DDE, 0x320A1886, 0x5A09344E, 
-  0x09C17994, 0x61C2555C, 0xD9C72004, 0xB1C40CCC, 
-  0xAC21BC45, 0xC422908D, 0x7C27E5D5, 0x1424C91D, 
-  0xDBB77E61, 0xB3B452A9, 0x0BB127F1, 0x63B20B39, 
-  0x7E57BBB0, 0x16549778, 0xAE51E220, 0xC652CEE8, 
-  0x959A8332, 0xFD99AFFA, 0x459CDAA2, 0x2D9FF66A, 
-  0x307A46E3, 0x58796A2B, 0xE07C1F73, 0x887F33BB, 
-  0xF56E0EF4, 0x9D6D223C, 0x25685764, 0x4D6B7BAC, 
-  0x508ECB25, 0x388DE7ED, 0x808892B5, 0xE88BBE7D, 
-  0xBB43F3A7, 0xD340DF6F, 0x6B45AA37, 0x034686FF, 
-  0x1EA33676, 0x76A01ABE, 0xCEA56FE6, 0xA6A6432E, 
-  0x6935F452, 0x0136D89A, 0xB933ADC2, 0xD130810A, 
-  0xCCD53183, 0xA4D61D4B, 0x1CD36813, 0x74D044DB, 
-  0x27180901, 0x4F1B25C9, 0xF71E5091, 0x9F1D7C59, 
-  0x82F8CCD0, 0xEAFBE018, 0x52FE9540, 0x3AFDB988, 
-  0xC8358D49, 0xA036A181, 0x1833D4D9, 0x7030F811, 
-  0x6DD54898, 0x05D66450, 0xBDD31108, 0xD5D03DC0, 
-  0x8618701A, 0xEE1B5CD2, 0x561E298A, 0x3E1D0542, 
-  0x23F8B5CB, 0x4BFB9903, 0xF3FEEC5B, 0x9BFDC093, 
-  0x546E77EF, 0x3C6D5B27, 0x84682E7F, 0xEC6B02B7, 
-  0xF18EB23E, 0x998D9EF6, 0x2188EBAE, 0x498BC766, 
-  0x1A438ABC, 0x7240A674, 0xCA45D32C, 0xA246FFE4, 
-  0xBFA34F6D, 0xD7A063A5, 0x6FA516FD, 0x07A63A35, 
-  0x8FD9098E, 0xE7DA2546, 0x5FDF501E, 0x37DC7CD6, 
-  0x2A39CC5F, 0x423AE097, 0xFA3F95CF, 0x923CB907, 
-  0xC1F4F4DD, 0xA9F7D815, 0x11F2AD4D, 0x79F18185, 
-  0x6414310C, 0x0C171DC4, 0xB412689C, 0xDC114454, 
-  0x1382F328, 0x7B81DFE0, 0xC384AAB8, 0xAB878670, 
-  0xB66236F9, 0xDE611A31, 0x66646F69, 0x0E6743A1, 
-  0x5DAF0E7B, 0x35AC22B3, 0x8DA957EB, 0xE5AA7B23, 
-  0xF84FCBAA, 0x904CE762, 0x2849923A, 0x404ABEF2, 
-  0xB2828A33, 0xDA81A6FB, 0x6284D3A3, 0x0A87FF6B, 
-  0x17624FE2, 0x7F61632A, 0xC7641672, 0xAF673ABA, 
-  0xFCAF7760, 0x94AC5BA8, 0x2CA92EF0, 0x44AA0238, 
-  0x594FB2B1, 0x314C9E79, 0x8949EB21, 0xE14AC7E9, 
-  0x2ED97095, 0x46DA5C5D, 0xFEDF2905, 0x96DC05CD, 
-  0x8B39B544, 0xE33A998C, 0x5B3FECD4, 0x333CC01C, 
-  0x60F48DC6, 0x08F7A10E, 0xB0F2D456, 0xD8F1F89E, 
-  0xC5144817, 0xAD1764DF, 0x15121187, 0x7D113D4F
-};
-const uint32_t CRC32C_T8_7[256] = {
-  0x00000000, 0x493C7D27, 0x9278FA4E, 0xDB448769, 
-  0x211D826D, 0x6821FF4A, 0xB3657823, 0xFA590504, 
-  0x423B04DA, 0x0B0779FD, 0xD043FE94, 0x997F83B3, 
-  0x632686B7, 0x2A1AFB90, 0xF15E7CF9, 0xB86201DE, 
-  0x847609B4, 0xCD4A7493, 0x160EF3FA, 0x5F328EDD, 
-  0xA56B8BD9, 0xEC57F6FE, 0x37137197, 0x7E2F0CB0, 
-  0xC64D0D6E, 0x8F717049, 0x5435F720, 0x1D098A07, 
-  0xE7508F03, 0xAE6CF224, 0x7528754D, 0x3C14086A, 
-  0x0D006599, 0x443C18BE, 0x9F789FD7, 0xD644E2F0, 
-  0x2C1DE7F4, 0x65219AD3, 0xBE651DBA, 0xF759609D, 
-  0x4F3B6143, 0x06071C64, 0xDD439B0D, 0x947FE62A, 
-  0x6E26E32E, 0x271A9E09, 0xFC5E1960, 0xB5626447, 
-  0x89766C2D, 0xC04A110A, 0x1B0E9663, 0x5232EB44, 
-  0xA86BEE40, 0xE1579367, 0x3A13140E, 0x732F6929, 
-  0xCB4D68F7, 0x827115D0, 0x593592B9, 0x1009EF9E, 
-  0xEA50EA9A, 0xA36C97BD, 0x782810D4, 0x31146DF3, 
-  0x1A00CB32, 0x533CB615, 0x8878317C, 0xC1444C5B, 
-  0x3B1D495F, 0x72213478, 0xA965B311, 0xE059CE36, 
-  0x583BCFE8, 0x1107B2CF, 0xCA4335A6, 0x837F4881, 
-  0x79264D85, 0x301A30A2, 0xEB5EB7CB, 0xA262CAEC, 
-  0x9E76C286, 0xD74ABFA1, 0x0C0E38C8, 0x453245EF, 
-  0xBF6B40EB, 0xF6573DCC, 0x2D13BAA5, 0x642FC782, 
-  0xDC4DC65C, 0x9571BB7B, 0x4E353C12, 0x07094135, 
-  0xFD504431, 0xB46C3916, 0x6F28BE7F, 0x2614C358, 
-  0x1700AEAB, 0x5E3CD38C, 0x857854E5, 0xCC4429C2, 
-  0x361D2CC6, 0x7F2151E1, 0xA465D688, 0xED59ABAF, 
-  0x553BAA71, 0x1C07D756, 0xC743503F, 0x8E7F2D18, 
-  0x7426281C, 0x3D1A553B, 0xE65ED252, 0xAF62AF75, 
-  0x9376A71F, 0xDA4ADA38, 0x010E5D51, 0x48322076, 
-  0xB26B2572, 0xFB575855, 0x2013DF3C, 0x692FA21B, 
-  0xD14DA3C5, 0x9871DEE2, 0x4335598B, 0x0A0924AC, 
-  0xF05021A8, 0xB96C5C8F, 0x6228DBE6, 0x2B14A6C1, 
-  0x34019664, 0x7D3DEB43, 0xA6796C2A, 0xEF45110D, 
-  0x151C1409, 0x5C20692E, 0x8764EE47, 0xCE589360, 
-  0x763A92BE, 0x3F06EF99, 0xE44268F0, 0xAD7E15D7, 
-  0x572710D3, 0x1E1B6DF4, 0xC55FEA9D, 0x8C6397BA, 
-  0xB0779FD0, 0xF94BE2F7, 0x220F659E, 0x6B3318B9, 
-  0x916A1DBD, 0xD856609A, 0x0312E7F3, 0x4A2E9AD4, 
-  0xF24C9B0A, 0xBB70E62D, 0x60346144, 0x29081C63, 
-  0xD3511967, 0x9A6D6440, 0x4129E329, 0x08159E0E, 
-  0x3901F3FD, 0x703D8EDA, 0xAB7909B3, 0xE2457494, 
-  0x181C7190, 0x51200CB7, 0x8A648BDE, 0xC358F6F9, 
-  0x7B3AF727, 0x32068A00, 0xE9420D69, 0xA07E704E, 
-  0x5A27754A, 0x131B086D, 0xC85F8F04, 0x8163F223, 
-  0xBD77FA49, 0xF44B876E, 0x2F0F0007, 0x66337D20, 
-  0x9C6A7824, 0xD5560503, 0x0E12826A, 0x472EFF4D, 
-  0xFF4CFE93, 0xB67083B4, 0x6D3404DD, 0x240879FA, 
-  0xDE517CFE, 0x976D01D9, 0x4C2986B0, 0x0515FB97, 
-  0x2E015D56, 0x673D2071, 0xBC79A718, 0xF545DA3F, 
-  0x0F1CDF3B, 0x4620A21C, 0x9D642575, 0xD4585852, 
-  0x6C3A598C, 0x250624AB, 0xFE42A3C2, 0xB77EDEE5, 
-  0x4D27DBE1, 0x041BA6C6, 0xDF5F21AF, 0x96635C88, 
-  0xAA7754E2, 0xE34B29C5, 0x380FAEAC, 0x7133D38B, 
-  0x8B6AD68F, 0xC256ABA8, 0x19122CC1, 0x502E51E6, 
-  0xE84C5038, 0xA1702D1F, 0x7A34AA76, 0x3308D751, 
-  0xC951D255, 0x806DAF72, 0x5B29281B, 0x1215553C, 
-  0x230138CF, 0x6A3D45E8, 0xB179C281, 0xF845BFA6, 
-  0x021CBAA2, 0x4B20C785, 0x906440EC, 0xD9583DCB, 
-  0x613A3C15, 0x28064132, 0xF342C65B, 0xBA7EBB7C, 
-  0x4027BE78, 0x091BC35F, 0xD25F4436, 0x9B633911, 
-  0xA777317B, 0xEE4B4C5C, 0x350FCB35, 0x7C33B612, 
-  0x866AB316, 0xCF56CE31, 0x14124958, 0x5D2E347F, 
-  0xE54C35A1, 0xAC704886, 0x7734CFEF, 0x3E08B2C8, 
-  0xC451B7CC, 0x8D6DCAEB, 0x56294D82, 0x1F1530A5
-};

http://git-wip-us.apache.org/repos/asf/incubator-ratis/blob/7e71a2e0/raft-common/src/main/native/src/org/apache/raft/util/gcc_optimizations.h
----------------------------------------------------------------------
diff --git a/raft-common/src/main/native/src/org/apache/raft/util/gcc_optimizations.h b/raft-common/src/main/native/src/org/apache/raft/util/gcc_optimizations.h
deleted file mode 100644
index 8c0fbff..0000000
--- a/raft-common/src/main/native/src/org/apache/raft/util/gcc_optimizations.h
+++ /dev/null
@@ -1,30 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- *     http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-#ifndef __GCC_OPTIMIZATIONS_H_INCLUDED
-#define __GCC_OPTIMIZATIONS_H_INCLUDED
-
-// Hints to gcc optimizer -- compiled out on non-GCC
-#ifdef __GNUC__
-#define likely(x)       __builtin_expect((x),1)
-#define unlikely(x)     __builtin_expect((x),0)
-#else
-#define likely(x)       (x)
-#define unlikely(x)     (x)
-#endif
-
-#endif

http://git-wip-us.apache.org/repos/asf/incubator-ratis/blob/7e71a2e0/raft-common/src/main/native/src/org_apache_raft.h
----------------------------------------------------------------------
diff --git a/raft-common/src/main/native/src/org_apache_raft.h b/raft-common/src/main/native/src/org_apache_raft.h
deleted file mode 100644
index e61f14b..0000000
--- a/raft-common/src/main/native/src/org_apache_raft.h
+++ /dev/null
@@ -1,189 +0,0 @@
-/**
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- *     http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-
-/**
- * This file includes some common utilities
- * for all native code used in raft.
- */
-
-#if !defined ORG_APACHE_RAFT_H
-#define ORG_APACHE_RAFT_H
-
-#if defined(_WIN32)
-#undef UNIX
-#define WINDOWS
-#else
-#undef WINDOWS
-#define UNIX
-#endif
-
-/* A helper macro to 'throw' a java exception. */
-#define THROW(env, exception_name, message) \
-  { \
-	jclass ecls = (*env)->FindClass(env, exception_name); \
-	if (ecls) { \
-	  (*env)->ThrowNew(env, ecls, message); \
-	  (*env)->DeleteLocalRef(env, ecls); \
-	} \
-  }
-
-/* Helper macro to return if an exception is pending */
-#define PASS_EXCEPTIONS(env) \
-  { \
-    if ((*env)->ExceptionCheck(env)) return; \
-  }
-
-#define PASS_EXCEPTIONS_GOTO(env, target) \
-  { \
-    if ((*env)->ExceptionCheck(env)) goto target; \
-  }
-
-#define PASS_EXCEPTIONS_RET(env, ret) \
-  { \
-    if ((*env)->ExceptionCheck(env)) return (ret); \
-  }
-
-/**
- * Unix definitions
- */
-#ifdef UNIX
-#include <config.h>
-#include <dlfcn.h>
-#include <jni.h>
-
-/**
- * A helper function to dlsym a 'symbol' from a given library-handle.
- *
- * @param env jni handle to report contingencies.
- * @param handle handle to the dlopen'ed library.
- * @param symbol symbol to load.
- * @return returns the address where the symbol is loaded in memory,
- *         <code>NULL</code> on error.
- */
-static __attribute__ ((unused))
-void *do_dlsym(JNIEnv *env, void *handle, const char *symbol) {
-  if (!env || !handle || !symbol) {
-  	THROW(env, "java/lang/InternalError", NULL);
-  	return NULL;
-  }
-  char *error = NULL;
-  void *func_ptr = dlsym(handle, symbol);
-  if ((error = dlerror()) != NULL) {
-  	THROW(env, "java/lang/UnsatisfiedLinkError", symbol);
-  	return NULL;
-  }
-  return func_ptr;
-}
-
-/* A helper macro to dlsym the requisite dynamic symbol and bail-out on error. */
-#define LOAD_DYNAMIC_SYMBOL(func_ptr, env, handle, symbol) \
-  if ((func_ptr = do_dlsym(env, handle, symbol)) == NULL) { \
-    return; \
-  }
-#endif
-// Unix part end
-
-
-/**
- * Windows definitions
- */
-#ifdef WINDOWS
-
-/* Force using Unicode throughout the code */
-#ifndef UNICODE
-#define UNICODE
-#endif
-
-/* Microsoft C Compiler does not support the C99 inline keyword */
-#ifndef __cplusplus
-#define inline __inline;
-#endif // _cplusplus
-
-/* Optimization macros supported by GCC but for which there is no
-   direct equivalent in the Microsoft C compiler */
-#define likely(_c) (_c)
-#define unlikely(_c) (_c)
-
-/* Disable certain warnings in the native CRC32 code. */
-#pragma warning(disable:4018)		// Signed/unsigned mismatch.
-#pragma warning(disable:4244)		// Possible loss of data in conversion.
-#pragma warning(disable:4267)		// Possible loss of data.
-#pragma warning(disable:4996)		// Use of deprecated function.
-
-#include <Windows.h>
-#include <stdio.h>
-#include <jni.h>
-
-#define snprintf(a, b ,c, d) _snprintf_s((a), (b), _TRUNCATE, (c), (d))
-
-/* A helper macro to dlsym the requisite dynamic symbol and bail-out on error. */
-#define LOAD_DYNAMIC_SYMBOL(func_type, func_ptr, env, handle, symbol) \
-  if ((func_ptr = (func_type) do_dlsym(env, handle, symbol)) == NULL) { \
-    return; \
-  }
-
-/**
- * A helper function to dynamic load a 'symbol' from a given library-handle.
- *
- * @param env jni handle to report contingencies.
- * @param handle handle to the dynamic library.
- * @param symbol symbol to load.
- * @return returns the address where the symbol is loaded in memory,
- *         <code>NULL</code> on error.
- */
-static FARPROC WINAPI do_dlsym(JNIEnv *env, HMODULE handle, LPCSTR symbol) {
-  DWORD dwErrorCode = ERROR_SUCCESS;
-  FARPROC func_ptr = NULL;
-
-  if (!env || !handle || !symbol) {
-    THROW(env, "java/lang/InternalError", NULL);
-    return NULL;
-  }
-
-  func_ptr = GetProcAddress(handle, symbol);
-  if (func_ptr == NULL)
-  {
-    THROW(env, "java/lang/UnsatisfiedLinkError", symbol);
-  }
-  return func_ptr;
-}
-#endif
-// Windows part end
-
-
-#define LOCK_CLASS(env, clazz, classname) \
-  if ((*env)->MonitorEnter(env, clazz) != 0) { \
-    char exception_msg[128]; \
-    snprintf(exception_msg, 128, "Failed to lock %s", classname); \
-    THROW(env, "java/lang/InternalError", exception_msg); \
-  }
-
-#define UNLOCK_CLASS(env, clazz, classname) \
-  if ((*env)->MonitorExit(env, clazz) != 0) { \
-    char exception_msg[128]; \
-    snprintf(exception_msg, 128, "Failed to unlock %s", classname); \
-    THROW(env, "java/lang/InternalError", exception_msg); \
-  }
-
-#define RETRY_ON_EINTR(ret, expr) do { \
-  ret = expr; \
-} while ((ret == -1) && (errno == EINTR));
-
-#endif
-
-//vim: sw=2: ts=2: et

http://git-wip-us.apache.org/repos/asf/incubator-ratis/blob/7e71a2e0/raft-common/src/main/native/src/test/org/apache/raft/util/test_bulk_crc32.c
----------------------------------------------------------------------
diff --git a/raft-common/src/main/native/src/test/org/apache/raft/util/test_bulk_crc32.c b/raft-common/src/main/native/src/test/org/apache/raft/util/test_bulk_crc32.c
deleted file mode 100644
index 787d285..0000000
--- a/raft-common/src/main/native/src/test/org/apache/raft/util/test_bulk_crc32.c
+++ /dev/null
@@ -1,113 +0,0 @@
-/**
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- *     http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-
-#include "org_apache_raft.h"
-
-#include "bulk_crc32.h"
-
-#include <stdint.h>
-#include <stdio.h>
-#include <stdlib.h>
-#include <time.h>
-
-#define EXPECT_ZERO(x) \
-    do { \
-        int __my_ret__ = x; \
-        if (__my_ret__) { \
-            fprintf(stderr, "TEST_ERROR: failed on line %d with return " \
-              "code %d: got nonzero from %s\n", __LINE__, __my_ret__, #x); \
-            return __my_ret__; \
-        } \
-    } while (0);
-
-static int testBulkVerifyCrc(int dataLen, int crcType, int bytesPerChecksum)
-{
-  int i;
-  uint8_t *data;
-  uint32_t *sums;
-  crc32_error_t errorData;
-
-  data = malloc(dataLen);
-  for (i = 0; i < dataLen; i++) {
-    data[i] = (i % 16) + 1;
-  }
-  sums = calloc(sizeof(uint32_t),
-                (dataLen + bytesPerChecksum - 1) / bytesPerChecksum);
-
-  EXPECT_ZERO(bulk_crc(data, dataLen, sums, crcType,
-                                 bytesPerChecksum, NULL));
-  EXPECT_ZERO(bulk_crc(data, dataLen, sums, crcType,
-                            bytesPerChecksum, &errorData));
-  free(data);
-  free(sums);
-  return 0;
-}
-
-static int timeBulkCrc(int dataLen, int crcType, int bytesPerChecksum, int iterations)
-{
-  int i;
-  uint8_t *data;
-  uint32_t *sums;
-  crc32_error_t errorData;
-  clock_t start, fini;
-
-  data = malloc(dataLen);
-  for (i = 0; i < dataLen; i++) {
-    data[i] = (i % 16) + 1;
-  }
-  sums = calloc(sizeof(uint32_t),
-                (dataLen + bytesPerChecksum - 1) / bytesPerChecksum);
-
-  start = clock();
-  for (i = 0; i < iterations; i++) {
-    EXPECT_ZERO(bulk_crc(data, dataLen, sums, crcType,
-				   bytesPerChecksum, NULL));
-    EXPECT_ZERO(bulk_crc(data, dataLen, sums, crcType,
-			      bytesPerChecksum, &errorData));
-  }
-  fini = clock();
-  printf("CRC %d bytes @ %d bytes per checksum X %d iterations = %g\n",
-          dataLen, bytesPerChecksum, iterations, (double)(fini-start)/CLOCKS_PER_SEC);
-  free(data);
-  free(sums);
-  return 0;
-}
-
-int main(int argc, char **argv)
-{
-  /* Test running bulk_calculate_crc with some different algorithms and
-   * bytePerChecksum values. */
-  EXPECT_ZERO(testBulkVerifyCrc(4096, CRC32C_POLYNOMIAL, 512));
-  EXPECT_ZERO(testBulkVerifyCrc(4096, CRC32_ZLIB_POLYNOMIAL, 512));
-  EXPECT_ZERO(testBulkVerifyCrc(256, CRC32C_POLYNOMIAL, 1));
-  EXPECT_ZERO(testBulkVerifyCrc(256, CRC32_ZLIB_POLYNOMIAL, 1));
-  EXPECT_ZERO(testBulkVerifyCrc(1, CRC32C_POLYNOMIAL, 1));
-  EXPECT_ZERO(testBulkVerifyCrc(1, CRC32_ZLIB_POLYNOMIAL, 1));
-  EXPECT_ZERO(testBulkVerifyCrc(2, CRC32C_POLYNOMIAL, 1));
-  EXPECT_ZERO(testBulkVerifyCrc(17, CRC32C_POLYNOMIAL, 1));
-  EXPECT_ZERO(testBulkVerifyCrc(17, CRC32C_POLYNOMIAL, 2));
-  EXPECT_ZERO(testBulkVerifyCrc(17, CRC32_ZLIB_POLYNOMIAL, 2));
-  EXPECT_ZERO(testBulkVerifyCrc(17, CRC32C_POLYNOMIAL, 4));
-  EXPECT_ZERO(testBulkVerifyCrc(17, CRC32_ZLIB_POLYNOMIAL, 4));
-
-  EXPECT_ZERO(timeBulkCrc(16 * 1024, CRC32C_POLYNOMIAL, 512, 1000000));
-  EXPECT_ZERO(timeBulkCrc(16 * 1024, CRC32_ZLIB_POLYNOMIAL, 512, 1000000));
-
-  fprintf(stderr, "%s: SUCCESS.\n", argv[0]);
-  return EXIT_SUCCESS;
-}

http://git-wip-us.apache.org/repos/asf/incubator-ratis/blob/7e71a2e0/raft-common/src/test/java/org/apache/raft/util/TestLifeCycle.java
----------------------------------------------------------------------
diff --git a/raft-common/src/test/java/org/apache/raft/util/TestLifeCycle.java b/raft-common/src/test/java/org/apache/raft/util/TestLifeCycle.java
deleted file mode 100644
index 53923f7..0000000
--- a/raft-common/src/test/java/org/apache/raft/util/TestLifeCycle.java
+++ /dev/null
@@ -1,53 +0,0 @@
-/**
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- *     http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-package org.apache.raft.util;
-
-import org.junit.Assert;
-import org.junit.Test;
-
-import java.util.*;
-
-import static org.apache.raft.util.LifeCycle.State.*;
-
-public class TestLifeCycle {
-  /**
-   * Test if the successor map and the predecessor map are consistent.
-   * {@link LifeCycle} uses predecessors to validate transitions
-   * while this test uses successors.
-   */
-  @Test
-  public void testIsValid() throws Exception {
-    final Map<LifeCycle.State, List<LifeCycle.State>> successors
-        = new EnumMap<>(LifeCycle.State.class);
-    put(NEW,       successors, STARTING, CLOSED);
-    put(STARTING,  successors, NEW, RUNNING, CLOSING, EXCEPTION);
-    put(RUNNING,   successors, CLOSING, PAUSING, EXCEPTION);
-    put(PAUSING,   successors, PAUSED, CLOSING, EXCEPTION);
-    put(PAUSED,    successors, STARTING, CLOSING);
-    put(EXCEPTION, successors, CLOSING);
-    put(CLOSING ,  successors, CLOSED);
-    put(CLOSED,    successors);
-
-    final List<LifeCycle.State> states = Arrays.asList(LifeCycle.State.values());
-    states.stream().forEach(
-        from -> states.forEach(
-            to -> Assert.assertEquals(from + " -> " + to,
-                successors.get(from).contains(to),
-                isValid(from, to))));
-  }
-}

http://git-wip-us.apache.org/repos/asf/incubator-ratis/blob/7e71a2e0/raft-examples/pom.xml
----------------------------------------------------------------------
diff --git a/raft-examples/pom.xml b/raft-examples/pom.xml
deleted file mode 100644
index f11b384..0000000
--- a/raft-examples/pom.xml
+++ /dev/null
@@ -1,130 +0,0 @@
-<?xml version="1.0" encoding="UTF-8"?>
-<!--
-  Licensed 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. See accompanying LICENSE file.
--->
-<project xmlns="http://maven.apache.org/POM/4.0.0"
-         xmlns:xsi="http://www.w3.org/2001/XMLSchema-instance"
-         xsi:schemaLocation="http://maven.apache.org/POM/4.0.0 http://maven.apache.org/xsd/maven-4.0.0.xsd">
-  <modelVersion>4.0.0</modelVersion>
-  <parent>
-    <artifactId>raft-project-dist</artifactId>
-    <groupId>com.hortonworks.raft</groupId>
-    <version>1.0-SNAPSHOT</version>
-    <relativePath>../raft-project-dist</relativePath>
-  </parent>
-
-  <artifactId>raft-examples</artifactId>
-  <name>Raft Examples</name>
-
-  <dependencies>
-    <dependency>
-      <artifactId>raft-proto-shaded</artifactId>
-      <groupId>com.hortonworks.raft</groupId>
-      <scope>provided</scope>
-    </dependency>
-
-    <dependency>
-      <artifactId>raft-common</artifactId>
-      <groupId>com.hortonworks.raft</groupId>
-      <scope>provided</scope>
-    </dependency>
-    <dependency>
-      <artifactId>raft-common</artifactId>
-      <groupId>com.hortonworks.raft</groupId>
-      <scope>test</scope>
-      <type>test-jar</type>
-    </dependency>
-    
-    <dependency>
-      <artifactId>raft-client</artifactId>
-      <groupId>com.hortonworks.raft</groupId>
-      <scope>provided</scope>
-    </dependency>
-    <dependency>
-      <artifactId>raft-client</artifactId>
-      <groupId>com.hortonworks.raft</groupId>
-      <scope>test</scope>
-      <type>test-jar</type>
-    </dependency>
-    
-    <dependency>
-      <artifactId>raft-server</artifactId>
-      <groupId>com.hortonworks.raft</groupId>
-      <scope>provided</scope>
-    </dependency>
-    <dependency>
-      <artifactId>raft-server</artifactId>
-      <groupId>com.hortonworks.raft</groupId>
-      <scope>test</scope>
-      <type>test-jar</type>
-    </dependency>
-    
-    <dependency>
-      <artifactId>raft-hadoop</artifactId>
-      <groupId>com.hortonworks.raft</groupId>
-      <scope>provided</scope>
-    </dependency>
-    <dependency>
-      <artifactId>raft-hadoop</artifactId>
-      <groupId>com.hortonworks.raft</groupId>
-      <scope>test</scope>
-      <type>test-jar</type>
-    </dependency>
-
-    <dependency>
-      <artifactId>raft-grpc</artifactId>
-      <groupId>com.hortonworks.raft</groupId>
-      <scope>provided</scope>
-    </dependency>
-    <dependency>
-      <artifactId>raft-grpc</artifactId>
-      <groupId>com.hortonworks.raft</groupId>
-      <scope>test</scope>
-      <type>test-jar</type>
-    </dependency>
-
-    <dependency>
-      <artifactId>raft-netty</artifactId>
-      <groupId>com.hortonworks.raft</groupId>
-      <scope>provided</scope>
-    </dependency>
-    <dependency>
-      <artifactId>raft-netty</artifactId>
-      <groupId>com.hortonworks.raft</groupId>
-      <scope>test</scope>
-      <type>test-jar</type>
-    </dependency>
-
-    <dependency>
-      <groupId>org.slf4j</groupId>
-      <artifactId>slf4j-api</artifactId>
-    </dependency>
-
-    <dependency>
-      <groupId>com.google.guava</groupId>
-      <artifactId>guava</artifactId>
-    </dependency>
-
-    <dependency>
-      <groupId>junit</groupId>
-      <artifactId>junit</artifactId>
-      <scope>test</scope>
-    </dependency>
-    <dependency>
-      <groupId>org.mockito</groupId>
-      <artifactId>mockito-all</artifactId>
-      <scope>test</scope>
-    </dependency>
-  </dependencies>
-</project>
-    

http://git-wip-us.apache.org/repos/asf/incubator-ratis/blob/7e71a2e0/raft-examples/src/main/java/org/apache/raft/examples/arithmetic/ArithmeticStateMachine.java
----------------------------------------------------------------------
diff --git a/raft-examples/src/main/java/org/apache/raft/examples/arithmetic/ArithmeticStateMachine.java b/raft-examples/src/main/java/org/apache/raft/examples/arithmetic/ArithmeticStateMachine.java
deleted file mode 100644
index 5112e14..0000000
--- a/raft-examples/src/main/java/org/apache/raft/examples/arithmetic/ArithmeticStateMachine.java
+++ /dev/null
@@ -1,180 +0,0 @@
-/**
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- *     http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-package org.apache.raft.examples.arithmetic;
-
-import com.google.common.base.Preconditions;
-import org.apache.raft.conf.RaftProperties;
-import org.apache.raft.examples.arithmetic.expression.Expression;
-import org.apache.raft.protocol.Message;
-import org.apache.raft.protocol.RaftClientReply;
-import org.apache.raft.protocol.RaftClientRequest;
-import org.apache.raft.server.impl.RaftServerConstants;
-import org.apache.raft.server.protocol.TermIndex;
-import org.apache.raft.server.storage.RaftStorage;
-import org.apache.raft.shaded.proto.RaftProtos.LogEntryProto;
-import org.apache.raft.statemachine.*;
-import org.apache.raft.util.AutoCloseableLock;
-import org.slf4j.Logger;
-import org.slf4j.LoggerFactory;
-
-import java.io.*;
-import java.util.HashMap;
-import java.util.Map;
-import java.util.concurrent.CompletableFuture;
-import java.util.concurrent.ConcurrentHashMap;
-import java.util.concurrent.atomic.AtomicReference;
-import java.util.concurrent.locks.ReentrantReadWriteLock;
-
-public class ArithmeticStateMachine extends BaseStateMachine {
-  static final Logger LOG = LoggerFactory.getLogger(ArithmeticStateMachine.class);
-
-  private final Map<String, Double> variables = new ConcurrentHashMap<>();
-
-  private final SimpleStateMachineStorage storage = new SimpleStateMachineStorage();
-  private final AtomicReference<TermIndex> latestTermIndex = new AtomicReference<>();
-
-  private final ReentrantReadWriteLock lock = new ReentrantReadWriteLock(true);
-
-  private AutoCloseableLock readLock() {
-    return AutoCloseableLock.acquire(lock.readLock());
-  }
-
-  private AutoCloseableLock writeLock() {
-    return AutoCloseableLock.acquire(lock.writeLock());
-  }
-
-  void reset() {
-    variables.clear();
-    latestTermIndex.set(null);
-  }
-
-  @Override
-  public void initialize(String id, RaftProperties properties, RaftStorage raftStorage)
-      throws IOException {
-    super.initialize(id, properties, raftStorage);
-    this.storage.init(raftStorage);
-    loadSnapshot(storage.getLatestSnapshot());
-  }
-
-  @Override
-  public void reinitialize(String id, RaftProperties properties, RaftStorage storage)
-      throws IOException {
-    close();
-    this.initialize(id, properties, storage);
-  }
-
-  @Override
-  public long takeSnapshot() throws IOException {
-    final Map<String, Double> copy;
-    final TermIndex last;
-    try(final AutoCloseableLock readLock = readLock()) {
-      copy = new HashMap<>(variables);
-      last = latestTermIndex.get();
-    }
-
-    File snapshotFile =  new File(SimpleStateMachineStorage.getSnapshotFileName(
-        last.getTerm(), last.getIndex()));
-
-    try(final ObjectOutputStream out = new ObjectOutputStream(
-        new BufferedOutputStream(new FileOutputStream(snapshotFile)))) {
-      out.writeObject(copy);
-    } catch(IOException ioe) {
-      LOG.warn("Failed to write snapshot file \"" + snapshotFile
-          + "\", last applied index=" + last);
-    }
-
-    return last.getIndex();
-  }
-
-  public long loadSnapshot(SingleFileSnapshotInfo snapshot) throws IOException {
-    return load(snapshot, false);
-  }
-
-  private long load(SingleFileSnapshotInfo snapshot, boolean reload) throws IOException {
-    if (snapshot == null || !snapshot.getFile().getPath().toFile().exists()) {
-      LOG.warn("The snapshot file {} does not exist", snapshot);
-      return RaftServerConstants.INVALID_LOG_INDEX;
-    }
-
-    File snapshotFile =snapshot.getFile().getPath().toFile();
-    final TermIndex last = SimpleStateMachineStorage.getTermIndexFromSnapshotFile(snapshotFile);
-    try(final AutoCloseableLock writeLock = writeLock();
-        final ObjectInputStream in = new ObjectInputStream(
-            new BufferedInputStream(new FileInputStream(snapshotFile)))) {
-      if (reload) {
-        reset();
-      }
-      latestTermIndex.set(last);
-      variables.putAll((Map<String, Double>) in.readObject());
-    } catch (ClassNotFoundException e) {
-      throw new IllegalStateException(e);
-    }
-    return last.getIndex();
-  }
-
-  @Override
-  public StateMachineStorage getStateMachineStorage() {
-    return storage;
-  }
-
-  @Override
-  public CompletableFuture<RaftClientReply> query(
-      RaftClientRequest request) {
-    final Expression q = Expression.Utils.bytes2Expression(
-        request.getMessage().getContent().toByteArray(), 0);
-    final Double result;
-    try(final AutoCloseableLock readLock = readLock()) {
-      result = q.evaluate(variables);
-    }
-    final Expression r = Expression.Utils.double2Expression(result);
-    LOG.debug("QUERY: {} = {}", q, r);
-    final RaftClientReply reply = new RaftClientReply(request,
-        Expression.Utils.toMessage(r));
-    return CompletableFuture.completedFuture(reply);
-  }
-
-  @Override
-  public void close() {
-    reset();
-  }
-
-  @Override
-  public CompletableFuture<Message> applyTransaction(TransactionContext trx) {
-    final LogEntryProto entry = trx.getLogEntry().get();
-    final AssignmentMessage assignment = new AssignmentMessage(
-        () -> entry.getSmLogEntry().getData());
-
-    final long index = entry.getIndex();
-    final Double result;
-    try(final AutoCloseableLock writeLock = writeLock()) {
-      result = assignment.evaluate(variables);
-      updateLatestTermIndex(entry.getTerm(), index);
-    }
-    final Expression r = Expression.Utils.double2Expression(result);
-    LOG.debug("{}: {} = {}, variables={}", index, assignment, r, variables);
-    return CompletableFuture.completedFuture(Expression.Utils.toMessage(r));
-  }
-
-  private void updateLatestTermIndex(long term, long index) {
-    final TermIndex newTI = TermIndex.newTermIndex(term, index);
-    final TermIndex oldTI = latestTermIndex.getAndSet(newTI);
-    if (oldTI != null) {
-      Preconditions.checkArgument(newTI.compareTo(oldTI) >= 0);
-    }
-  }
-}

http://git-wip-us.apache.org/repos/asf/incubator-ratis/blob/7e71a2e0/raft-examples/src/main/java/org/apache/raft/examples/arithmetic/AssignmentMessage.java
----------------------------------------------------------------------
diff --git a/raft-examples/src/main/java/org/apache/raft/examples/arithmetic/AssignmentMessage.java b/raft-examples/src/main/java/org/apache/raft/examples/arithmetic/AssignmentMessage.java
deleted file mode 100644
index 5106318..0000000
--- a/raft-examples/src/main/java/org/apache/raft/examples/arithmetic/AssignmentMessage.java
+++ /dev/null
@@ -1,83 +0,0 @@
-/**
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- *     http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-package org.apache.raft.examples.arithmetic;
-
-import org.apache.raft.examples.arithmetic.expression.Expression;
-import org.apache.raft.examples.arithmetic.expression.Variable;
-import org.apache.raft.protocol.Message;
-import org.apache.raft.shaded.com.google.protobuf.ByteString;
-
-import java.nio.charset.Charset;
-import java.util.Map;
-
-import static org.apache.raft.util.ProtoUtils.toByteString;
-
-public class AssignmentMessage implements Message, Evaluable {
-  public static final Charset UTF8 = Charset.forName("UTF-8");
-
-  private final Variable variable;
-  private final Expression expression;
-
-  public AssignmentMessage(Variable variable, Expression expression) {
-    this.variable = variable;
-    this.expression = expression;
-  }
-
-  public AssignmentMessage(byte[] buf, int offset) {
-    variable = new Variable(buf, offset);
-    expression = Expression.Utils.bytes2Expression(buf, offset + variable.length());
-  }
-
-  public AssignmentMessage(Message message) {
-    this(message.getContent().toByteArray(), 0);
-  }
-
-  public Variable getVariable() {
-    return variable;
-  }
-
-  public Expression getExpression() {
-    return expression;
-  }
-
-  @Override
-  public ByteString getContent() {
-    final int length = variable.length() + expression.length();
-    final byte[] bytes = new byte[length];
-    final int offset = variable.toBytes(bytes, 0);
-    expression.toBytes(bytes, offset);
-    return toByteString(bytes);
-  }
-
-  @Override
-  public String toString() {
-    return variable + " = " + expression;
-  }
-
-  @Override
-  public Double evaluate(Map<String, Double> variableMap) {
-    final Double value = expression.evaluate(variableMap);
-    final String name = variable.getName();
-    if (value == null) {
-      variableMap.remove(name);
-    } else {
-      variableMap.put(name, value);
-    }
-    return value;
-  }
-}

http://git-wip-us.apache.org/repos/asf/incubator-ratis/blob/7e71a2e0/raft-examples/src/main/java/org/apache/raft/examples/arithmetic/Evaluable.java
----------------------------------------------------------------------
diff --git a/raft-examples/src/main/java/org/apache/raft/examples/arithmetic/Evaluable.java b/raft-examples/src/main/java/org/apache/raft/examples/arithmetic/Evaluable.java
deleted file mode 100644
index 295af2e..0000000
--- a/raft-examples/src/main/java/org/apache/raft/examples/arithmetic/Evaluable.java
+++ /dev/null
@@ -1,24 +0,0 @@
-/**
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- *     http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-package org.apache.raft.examples.arithmetic;
-
-import java.util.Map;
-
-public interface Evaluable {
-  Double evaluate(Map<String, Double> variableMap);
-}

http://git-wip-us.apache.org/repos/asf/incubator-ratis/blob/7e71a2e0/raft-examples/src/main/java/org/apache/raft/examples/arithmetic/expression/BinaryExpression.java
----------------------------------------------------------------------
diff --git a/raft-examples/src/main/java/org/apache/raft/examples/arithmetic/expression/BinaryExpression.java b/raft-examples/src/main/java/org/apache/raft/examples/arithmetic/expression/BinaryExpression.java
deleted file mode 100644
index 8585ceb..0000000
--- a/raft-examples/src/main/java/org/apache/raft/examples/arithmetic/expression/BinaryExpression.java
+++ /dev/null
@@ -1,103 +0,0 @@
-/**
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- *     http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-package org.apache.raft.examples.arithmetic.expression;
-
-import com.google.common.base.Preconditions;
-
-import java.util.Map;
-
-public class BinaryExpression implements Expression {
-  public enum Op {
-    ADD("+"), SUBTRACT("-"), MULT("*"), DIV("/");
-
-    final String symbol;
-
-    Op(String symbol) {
-      this.symbol = symbol;
-    }
-
-    byte byteValue() {
-      return (byte) ordinal();
-    }
-
-    @Override
-    public String toString() {
-      return symbol;
-    }
-
-    static final Op[] VALUES = Op.values();
-
-    static Op valueOf(byte b) {
-      Preconditions.checkArgument(b < VALUES.length);
-      return VALUES[b];
-    }
-  }
-
-  private final Op op;
-  private final Expression left, right;
-
-  BinaryExpression(byte[] buf, final int offset) {
-    Preconditions.checkArgument(buf[offset] == Type.BINARY.byteValue());
-    op = Op.valueOf(buf[offset + 1]);
-    left = Utils.bytes2Expression(buf, offset + 2);
-    right = Utils.bytes2Expression(buf, offset + 2 + left.length());
-  }
-
-  public BinaryExpression(Op op, Expression left, Expression right) {
-    this.op = op;
-    this.left = left;
-    this.right = right;
-  }
-
-  @Override
-  public int toBytes(byte[] buf, final int offset) {
-    buf[offset] = Type.BINARY.byteValue();
-    buf[offset + 1] = op.byteValue();
-    final int l = left.toBytes(buf, offset + 2);
-    final int r = right.toBytes(buf, offset + 2 + l);
-    return 2 + l + r;
-  }
-
-  @Override
-  public int length() {
-    return 2 + left.length() + right.length();
-  }
-
-  @Override
-  public Double evaluate(Map<String, Double> variableMap) {
-    final double l = left.evaluate(variableMap);
-    final double r = right.evaluate(variableMap);
-    switch (op) {
-      case ADD:
-        return l + r;
-      case SUBTRACT:
-        return l - r;
-      case MULT:
-        return l * r;
-      case DIV:
-        return l / r;
-      default:
-        throw new AssertionError("Unexpected op value: " + op);
-    }
-  }
-
-  @Override
-  public String toString() {
-    return "(" + left + " " + op + " " + right + ")";
-  }
-}

http://git-wip-us.apache.org/repos/asf/incubator-ratis/blob/7e71a2e0/raft-examples/src/main/java/org/apache/raft/examples/arithmetic/expression/DoubleValue.java
----------------------------------------------------------------------
diff --git a/raft-examples/src/main/java/org/apache/raft/examples/arithmetic/expression/DoubleValue.java b/raft-examples/src/main/java/org/apache/raft/examples/arithmetic/expression/DoubleValue.java
deleted file mode 100644
index e7f15ca..0000000
--- a/raft-examples/src/main/java/org/apache/raft/examples/arithmetic/expression/DoubleValue.java
+++ /dev/null
@@ -1,61 +0,0 @@
-/**
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- *     http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-package org.apache.raft.examples.arithmetic.expression;
-
-import com.google.common.base.Preconditions;
-
-import java.util.Map;
-
-public class DoubleValue implements Expression {
-  public static final DoubleValue ZERO = new DoubleValue(0);
-  public static final DoubleValue ONE = new DoubleValue(1);
-
-  private final double value;
-
-  public DoubleValue(double value) {
-    this.value = value;
-  }
-
-  DoubleValue(byte[] buf, int offset) {
-    this(Utils.bytes2double(buf, offset + 1));
-    Preconditions.checkArgument(buf[offset] == Type.DOUBLE.byteValue());
-  }
-
-  @Override
-  public int toBytes(byte[] buf, int offset) {
-    Preconditions.checkArgument(offset + length() <= buf.length);
-    buf[offset++] = Type.DOUBLE.byteValue();
-    Utils.double2bytes(value, buf, offset);
-    return length();
-  }
-
-  @Override
-  public int length() {
-    return 9;
-  }
-
-  @Override
-  public Double evaluate(Map<String, Double> variableMap) {
-    return value;
-  }
-
-  @Override
-  public String toString() {
-    return String.valueOf(value);
-  }
-}

http://git-wip-us.apache.org/repos/asf/incubator-ratis/blob/7e71a2e0/raft-examples/src/main/java/org/apache/raft/examples/arithmetic/expression/Expression.java
----------------------------------------------------------------------
diff --git a/raft-examples/src/main/java/org/apache/raft/examples/arithmetic/expression/Expression.java b/raft-examples/src/main/java/org/apache/raft/examples/arithmetic/expression/Expression.java
deleted file mode 100644
index 594c26f..0000000
--- a/raft-examples/src/main/java/org/apache/raft/examples/arithmetic/expression/Expression.java
+++ /dev/null
@@ -1,111 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- *  with the License.  You may obtain a copy of the License at
- *
- *      http://www.apache.org/licenses/LICENSE-2.0
- *
- *  Unless required by applicable law or agreed to in writing, software
- *  distributed under the License is distributed on an "AS IS" BASIS,
- *  WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- *  See the License for the specific language governing permissions and
- *  limitations under the License.
- */
-package org.apache.raft.examples.arithmetic.expression;
-
-import com.google.common.base.Preconditions;
-import org.apache.raft.examples.arithmetic.Evaluable;
-import org.apache.raft.protocol.Message;
-
-import static org.apache.raft.util.ProtoUtils.toByteString;
-
-public interface Expression extends Evaluable {
-  enum Type {
-    NULL, VARIABLE, DOUBLE, BINARY, UNARY;
-
-    byte byteValue() {
-      return (byte) ordinal();
-    }
-
-    private static final Type[] VALUES = Type.values();
-
-    static Type valueOf(byte b) {
-      Preconditions.checkArgument(b >= 0);
-      Preconditions.checkArgument(b < VALUES.length);
-      return VALUES[b];
-    }
-  }
-
-  int toBytes(byte[] buf, int offset);
-
-  int length();
-
-  class Utils {
-    public static Message toMessage(final Expression e) {
-      return () -> {
-        final byte[] buf = new byte[e.length()];
-        final int length = e.toBytes(buf, 0);
-        Preconditions.checkState(length == buf.length);
-        return toByteString(buf);
-      };
-    }
-
-    public static Expression double2Expression(Double d) {
-      return d == null? NullValue.getInstance(): new DoubleValue(d);
-    }
-
-    public static Expression bytes2Expression(byte[] buf, int offset) {
-      final Type type = Type.valueOf(buf[offset]);
-      switch(type) {
-        case NULL: return NullValue.getInstance();
-        case DOUBLE: return new DoubleValue(buf, offset);
-        case VARIABLE: return new Variable(buf, offset);
-        case BINARY: return new BinaryExpression(buf, offset);
-        case UNARY: return new UnaryExpression(buf, offset);
-        default:
-          throw new AssertionError("Unknown expression type " + type);
-      }
-    }
-
-    public static int int2bytes(int v, byte[] buf, int offset) {
-      buf[offset    ] = (byte) (v >>> 24);
-      buf[offset + 1] = (byte) (v >>> 16);
-      buf[offset + 2] = (byte) (v >>> 8);
-      buf[offset + 3] = (byte) (v);
-      return 4;
-    }
-
-    public static int long2bytes(long v, byte[] buf, int offset) {
-      int2bytes((int)(v >>> 32), buf, offset);
-      int2bytes((int) v        , buf, offset + 4);
-      return 8;
-    }
-
-    public static int double2bytes(double d, byte[] buf, int offset) {
-      final long v = Double.doubleToRawLongBits(d);
-      return long2bytes(v, buf, offset);
-    }
-
-    public static int bytes2int(byte[] buf, int offset) {
-      return ((int) buf[offset] << 24)
-          + ((0xFF & buf[offset + 1]) << 16)
-          + ((0xFF & buf[offset + 2]) <<  8)
-          +  (0xFF & buf[offset + 3]);
-    }
-
-    public static long bytes2long(byte[] buf, int offset) {
-      return ((long)bytes2int(buf, offset) << 32)
-          + (0xFFFFFFFFL & bytes2int(buf, offset + 4));
-    }
-
-    public static double bytes2double(byte[] buf, int offset) {
-      final long v = bytes2long(buf, offset);
-      return Double.longBitsToDouble(v);
-    }
-  }
-}
-

http://git-wip-us.apache.org/repos/asf/incubator-ratis/blob/7e71a2e0/raft-examples/src/main/java/org/apache/raft/examples/arithmetic/expression/NullValue.java
----------------------------------------------------------------------
diff --git a/raft-examples/src/main/java/org/apache/raft/examples/arithmetic/expression/NullValue.java b/raft-examples/src/main/java/org/apache/raft/examples/arithmetic/expression/NullValue.java
deleted file mode 100644
index 1fe0fb5..0000000
--- a/raft-examples/src/main/java/org/apache/raft/examples/arithmetic/expression/NullValue.java
+++ /dev/null
@@ -1,55 +0,0 @@
-/**
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- *     http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-package org.apache.raft.examples.arithmetic.expression;
-
-import com.google.common.base.Preconditions;
-
-import java.util.Map;
-
-public class NullValue implements Expression {
-  private static final NullValue INSTANCE = new NullValue();
-
-  public static NullValue getInstance() {
-    return INSTANCE;
-  }
-
-  private NullValue() {
-  }
-
-  @Override
-  public int toBytes(byte[] buf, int offset) {
-    Preconditions.checkArgument(offset + length() <= buf.length);
-    buf[offset++] = Type.NULL.byteValue();
-    return length();
-  }
-
-  @Override
-  public int length() {
-    return 1;
-  }
-
-  @Override
-  public Double evaluate(Map<String, Double> variableMap) {
-    return null;
-  }
-
-  @Override
-  public String toString() {
-    return "null";
-  }
-}

http://git-wip-us.apache.org/repos/asf/incubator-ratis/blob/7e71a2e0/raft-examples/src/main/java/org/apache/raft/examples/arithmetic/expression/UnaryExpression.java
----------------------------------------------------------------------
diff --git a/raft-examples/src/main/java/org/apache/raft/examples/arithmetic/expression/UnaryExpression.java b/raft-examples/src/main/java/org/apache/raft/examples/arithmetic/expression/UnaryExpression.java
deleted file mode 100644
index 3d5b17c..0000000
--- a/raft-examples/src/main/java/org/apache/raft/examples/arithmetic/expression/UnaryExpression.java
+++ /dev/null
@@ -1,95 +0,0 @@
-/**
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- *     http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-package org.apache.raft.examples.arithmetic.expression;
-
-import com.google.common.base.Preconditions;
-
-import java.util.Map;
-
-public class UnaryExpression implements Expression {
-  public enum Op {
-    NEG("~"), SQRT("\u221a");
-
-    final String symbol;
-
-    Op(String symbol) {
-      this.symbol = symbol;
-    }
-
-    byte byteValue() {
-      return (byte) ordinal();
-    }
-
-    @Override
-    public String toString() {
-      return symbol;
-    }
-
-    static final Op[] VALUES = Op.values();
-
-    static Op valueOf(byte b) {
-      Preconditions.checkArgument(b < VALUES.length);
-      return VALUES[b];
-    }
-  }
-
-  final Op op;
-  final Expression expression;
-
-  UnaryExpression(byte[] buf, int offset) {
-    Preconditions.checkArgument(buf[offset] == Type.UNARY.byteValue());
-    op = Op.valueOf(buf[offset + 1]);
-    expression = Utils.bytes2Expression(buf, offset + 2);
-  }
-
-  public UnaryExpression(Op op, Expression expression) {
-    this.op = op;
-    this.expression = expression;
-  }
-
-  @Override
-  public int toBytes(byte[] buf, int offset) {
-    buf[offset] = Type.UNARY.byteValue();
-    buf[offset + 1] = op.byteValue();
-    final int length = expression.toBytes(buf, offset + 2);
-    return 2 + length;
-  }
-
-  @Override
-  public int length() {
-    return 2 + expression.length();
-  }
-
-  @Override
-  public Double evaluate(Map<String, Double> variableMap) {
-    final double value = expression.evaluate(variableMap);
-    switch (op) {
-      case NEG:
-        return -value;
-      case SQRT:
-        return Math.sqrt(value);
-      default:
-        throw new AssertionError("Unexpected op value: " + op);
-    }
-  }
-
-  @Override
-  public String toString() {
-    return op + " " + expression;
-  }
-}

http://git-wip-us.apache.org/repos/asf/incubator-ratis/blob/7e71a2e0/raft-examples/src/main/java/org/apache/raft/examples/arithmetic/expression/Variable.java
----------------------------------------------------------------------
diff --git a/raft-examples/src/main/java/org/apache/raft/examples/arithmetic/expression/Variable.java b/raft-examples/src/main/java/org/apache/raft/examples/arithmetic/expression/Variable.java
deleted file mode 100644
index 0cd7474..0000000
--- a/raft-examples/src/main/java/org/apache/raft/examples/arithmetic/expression/Variable.java
+++ /dev/null
@@ -1,125 +0,0 @@
-/**
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- *     http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-package org.apache.raft.examples.arithmetic.expression;
-
-import org.apache.raft.examples.arithmetic.AssignmentMessage;
-import com.google.common.base.Preconditions;
-
-import java.util.Map;
-import java.util.regex.Pattern;
-
-public class Variable implements Expression {
-  static final int LENGTH_LIMIT = 32;
-  static final String REGEX = "[a-zA-Z]\\w*";
-  static final Pattern PATTERN = Pattern.compile(REGEX);
-
-  static byte[] string2bytes(String s) {
-    final byte[] stringBytes = s.getBytes(AssignmentMessage.UTF8);
-    final byte[] bytes = new byte[stringBytes.length + 2];
-    bytes[0] = Type.VARIABLE.byteValue();
-    bytes[1] = (byte)stringBytes.length;
-    System.arraycopy(stringBytes, 0, bytes, 2, stringBytes.length);
-    return bytes;
-  }
-
-  static String extractString(byte[] buf, int offset) {
-    Preconditions.checkArgument(buf[offset] == Type.VARIABLE.byteValue());
-    final int length = buf[offset + 1];
-    final byte[] stringBytes = new byte[length];
-    System.arraycopy(buf, offset + 2, stringBytes, 0, length);
-    return new String(stringBytes, AssignmentMessage.UTF8);
-  }
-
-  static byte[] copyBytes(byte[] buf, int offset) {
-    Preconditions.checkArgument(buf[offset] == Type.VARIABLE.byteValue());
-    final int length = buf[offset + 1];
-    final byte[] copy = new byte[length + 2];
-    System.arraycopy(buf, offset, copy, 0, copy.length);
-    return copy;
-  }
-
-  private final String name;
-  private final byte[] encoded;
-
-  private Variable(String name, byte[] encoded) {
-    this.name = name;
-    this.encoded = encoded;
-
-    if (!PATTERN.matcher(name).matches()) {
-      throw new IllegalArgumentException("The variable name \"" + name
-          + "\" does not match the pattern \"" + PATTERN + "\"");
-    }
-    if (encoded.length - 2 > LENGTH_LIMIT) {
-      throw new IllegalArgumentException("The variable name \"" + name
-          + "\" is longer than the limit = " + LENGTH_LIMIT);
-    }
-  }
-
-  public Variable(byte[] buf, int offset) {
-    this(extractString(buf, offset), copyBytes(buf, offset));
-  }
-
-  public Variable(String name) {
-    this(name, string2bytes(name));
-  }
-
-  public String getName() {
-    return name;
-  }
-
-  @Override
-  public int toBytes(byte[] buf, int offset) {
-    System.arraycopy(encoded, 0, buf, offset, encoded.length);
-    return encoded.length;
-  }
-
-  @Override
-  public int length() {
-    return encoded.length;
-  }
-
-  @Override
-  public Double evaluate(Map<String, Double> variableMap) {
-    final Double value = variableMap.get(name);
-    if (value == null) {
-      throw new IllegalStateException("Undefined variable \"" + name + "\"");
-    }
-    return value;
-  }
-
-  @Override
-  public boolean equals(Object obj) {
-    if (obj == this) {
-      return true;
-    } else if (obj == null || !(obj instanceof Variable)) {
-      return false;
-    }
-    final Variable that = (Variable)obj;
-    return this.getName().equals(that.getName());
-  }
-
-  @Override
-  public int hashCode() {
-    return name.hashCode();
-  }
-
-  @Override
-  public String toString() {
-    return name;
-  }
-}

http://git-wip-us.apache.org/repos/asf/incubator-ratis/blob/7e71a2e0/raft-examples/src/test/java/org/apache/raft/TestBatchAppend.java
----------------------------------------------------------------------
diff --git a/raft-examples/src/test/java/org/apache/raft/TestBatchAppend.java b/raft-examples/src/test/java/org/apache/raft/TestBatchAppend.java
deleted file mode 100644
index 1fd8ba0..0000000
--- a/raft-examples/src/test/java/org/apache/raft/TestBatchAppend.java
+++ /dev/null
@@ -1,170 +0,0 @@
-/**
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- *     http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-package org.apache.raft;
-
-import org.apache.log4j.Level;
-import org.apache.raft.RaftTestUtil.SimpleMessage;
-import org.apache.raft.client.RaftClient;
-import org.apache.raft.conf.RaftProperties;
-import org.apache.raft.examples.RaftExamplesTestUtil;
-import org.apache.raft.server.impl.RaftServerImpl;
-import org.apache.raft.server.simulation.RequestHandler;
-import org.apache.raft.statemachine.SimpleStateMachine4Testing;
-import org.apache.raft.statemachine.StateMachine;
-import org.apache.raft.util.RaftUtils;
-import org.junit.After;
-import org.junit.Assert;
-import org.junit.Rule;
-import org.junit.Test;
-import org.junit.rules.Timeout;
-import org.junit.runner.RunWith;
-import org.junit.runners.Parameterized;
-import org.slf4j.Logger;
-import org.slf4j.LoggerFactory;
-
-import java.io.IOException;
-import java.util.Arrays;
-import java.util.Collection;
-import java.util.List;
-import java.util.concurrent.CountDownLatch;
-import java.util.concurrent.atomic.AtomicBoolean;
-import java.util.stream.Collectors;
-import java.util.stream.Stream;
-
-import static org.apache.raft.server.RaftServerConfigKeys.*;
-
-/**
- * Enable raft.server.log.appender.batch.enabled and test LogAppender
- */
-@RunWith(Parameterized.class)
-public class TestBatchAppend {
-  static Logger LOG = LoggerFactory.getLogger(TestBatchAppend.class);
-  static {
-    RaftUtils.setLogLevel(RaftServerImpl.LOG, Level.DEBUG);
-    RaftUtils.setLogLevel(RequestHandler.LOG, Level.DEBUG);
-    RaftUtils.setLogLevel(RaftClient.LOG, Level.DEBUG);
-  }
-
-  @Parameterized.Parameters
-  public static Collection<Object[]> data() throws IOException {
-    RaftProperties prop = new RaftProperties();
-    prop.setClass(MiniRaftCluster.STATEMACHINE_CLASS_KEY,
-        SimpleStateMachine4Testing.class, StateMachine.class);
-    prop.setInt(RAFT_LOG_SEGMENT_MAX_SIZE_KEY, 1024 * 8);
-    // enable batch appending
-    prop.setBoolean(RAFT_SERVER_LOG_APPENDER_BATCH_ENABLED_KEY, true);
-    // set batch appending buffer size to 4KB
-    prop.setInt(RAFT_SERVER_LOG_APPENDER_BUFFER_CAPACITY_KEY, 4 * 1024);
-
-    return RaftExamplesTestUtil.getMiniRaftClusters(prop, 3);
-  }
-
-  @Parameterized.Parameter
-  public MiniRaftCluster cluster;
-
-  @Rule
-  public Timeout globalTimeout = new Timeout(60 * 1000);
-
-  @After
-  public void tearDown() {
-    if (cluster != null) {
-      cluster.shutdown();
-    }
-  }
-
-  private class Sender extends Thread {
-    private final RaftClient client;
-    private final CountDownLatch latch;
-    private final SimpleMessage[] msgs;
-    private final AtomicBoolean succeed = new AtomicBoolean(false);
-
-    Sender(String clientId, String leaderId, CountDownLatch latch, int numMsg) {
-      this.latch = latch;
-      this.client = cluster.createClient(clientId, leaderId);
-      msgs = generateMsgs(numMsg);
-    }
-
-    SimpleMessage[] generateMsgs(int num) {
-      SimpleMessage[] msgs = new SimpleMessage[num * 6];
-      for (int i = 0; i < num; i++) {
-        for (int j = 0; j < 6; j++) {
-          byte[] bytes = new byte[1024 * (j + 1)];
-          Arrays.fill(bytes, (byte) j);
-          msgs[i * 6 + j] = new SimpleMessage(new String(bytes));
-        }
-      }
-      return msgs;
-    }
-
-    @Override
-    public void run() {
-      try {
-        latch.await();
-      } catch (InterruptedException ignored) {
-        LOG.warn("Client {} waiting for countdown latch got interrupted",
-            client.getId());
-      }
-      for (SimpleMessage msg : msgs) {
-        try {
-          client.send(msg);
-        } catch (IOException e) {
-          succeed.set(false);
-          LOG.warn("Client {} hit exception {}", client.getId(), e);
-          return;
-        }
-      }
-      succeed.set(true);
-      try {
-        client.close();
-      } catch (IOException ignore) {
-      }
-    }
-  }
-
-  @Test
-  public void testAppend() throws Exception {
-    final int numMsgs = 10;
-    final int numClients = 5;
-    cluster.start();
-    RaftTestUtil.waitForLeader(cluster);
-    final String leaderId = cluster.getLeader().getId();
-
-    // start several clients and write concurrently
-    CountDownLatch latch = new CountDownLatch(1);
-    final List<Sender> senders = Stream.iterate(0, i -> i+1).limit(numClients)
-        .map(i -> new Sender("c" + i, leaderId, latch, numMsgs))
-        .collect(Collectors.toList());
-    senders.forEach(Thread::start);
-
-    latch.countDown();
-
-    senders.forEach(sender -> {
-      try {
-        sender.join();
-      } catch (InterruptedException ignored) {
-      }
-    });
-
-    for (Sender s : senders) {
-      Assert.assertTrue(s.succeed.get());
-    }
-
-    Assert.assertEquals(6 * numMsgs * numClients,
-        cluster.getLeader().getState().getLastAppliedIndex());
-  }
-}


[21/54] [abbrv] incubator-ratis git commit: Renamed the packages from raft to ratis in preperation for Apache Incubation - Moved all java packages from org.apache.raft to org.apache.ratis. - Moved native package to org_apache_ratis, and native lib to l

Posted by ji...@apache.org.
http://git-wip-us.apache.org/repos/asf/incubator-ratis/blob/7e71a2e0/ratis-server/src/main/java/org/apache/ratis/server/impl/LeaderElection.java
----------------------------------------------------------------------
diff --git a/ratis-server/src/main/java/org/apache/ratis/server/impl/LeaderElection.java b/ratis-server/src/main/java/org/apache/ratis/server/impl/LeaderElection.java
new file mode 100644
index 0000000..7b3845a
--- /dev/null
+++ b/ratis-server/src/main/java/org/apache/ratis/server/impl/LeaderElection.java
@@ -0,0 +1,247 @@
+/**
+ * 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.ratis.server.impl;
+
+import java.io.IOException;
+import java.util.ArrayList;
+import java.util.Collection;
+import java.util.List;
+import java.util.concurrent.ExecutionException;
+import java.util.concurrent.ExecutorCompletionService;
+import java.util.concurrent.ExecutorService;
+import java.util.concurrent.Executors;
+import java.util.concurrent.Future;
+import java.util.concurrent.TimeUnit;
+import java.util.stream.Collectors;
+
+import org.apache.ratis.protocol.RaftPeer;
+import org.apache.ratis.server.protocol.TermIndex;
+import org.apache.ratis.shaded.proto.RaftProtos.RequestVoteReplyProto;
+import org.apache.ratis.shaded.proto.RaftProtos.RequestVoteRequestProto;
+import org.apache.ratis.statemachine.SnapshotInfo;
+import org.apache.ratis.util.Daemon;
+import org.apache.ratis.util.ProtoUtils;
+import org.apache.ratis.util.Timestamp;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import com.google.common.base.Preconditions;
+import com.google.common.util.concurrent.ThreadFactoryBuilder;
+
+class LeaderElection extends Daemon {
+  public static final Logger LOG = LoggerFactory.getLogger(LeaderElection.class);
+
+  private ResultAndTerm logAndReturn(Result result,
+      List<RequestVoteReplyProto> responses,
+      List<Exception> exceptions, long newTerm) {
+    LOG.info(server.getId() + ": Election " + result + "; received "
+        + responses.size() + " response(s) "
+        + responses.stream().map(r -> ProtoUtils.toString(r)).collect(Collectors.toList())
+        + " and " + exceptions.size() + " exception(s); " + server.getState());
+    int i = 0;
+    for(Exception e : exceptions) {
+      LOG.info("  " + i++ + ": " + e);
+      LOG.trace("TRACE", e);
+    }
+    return new ResultAndTerm(result, newTerm);
+  }
+
+  enum Result {PASSED, REJECTED, TIMEOUT, DISCOVERED_A_NEW_TERM, SHUTDOWN}
+
+  private static class ResultAndTerm {
+    final Result result;
+    final long term;
+
+    ResultAndTerm(Result result, long term) {
+      this.result = result;
+      this.term = term;
+    }
+  }
+
+  private final RaftServerImpl server;
+  private ExecutorCompletionService<RequestVoteReplyProto> service;
+  private ExecutorService executor;
+  private volatile boolean running;
+  /**
+   * The Raft configuration should not change while the peer is in candidate
+   * state. If the configuration changes, another peer should be acting as a
+   * leader and this LeaderElection session should end.
+   */
+  private final RaftConfiguration conf;
+  private final Collection<RaftPeer> others;
+
+  LeaderElection(RaftServerImpl server) {
+    this.server = server;
+    conf = server.getRaftConf();
+    others = conf.getOtherPeers(server.getId());
+    this.running = true;
+  }
+
+  void stopRunning() {
+    this.running = false;
+  }
+
+  private void initExecutor() {
+    Preconditions.checkState(!others.isEmpty());
+    executor = Executors.newFixedThreadPool(others.size(),
+        new ThreadFactoryBuilder().setDaemon(true).build());
+    service = new ExecutorCompletionService<>(executor);
+  }
+
+  @Override
+  public void run() {
+    try {
+      askForVotes();
+    } catch (InterruptedException e) {
+      // the leader election thread is interrupted. The peer may already step
+      // down to a follower. The leader election should skip.
+      LOG.info(server.getId() + " " + getClass().getSimpleName()
+          + " thread is interrupted gracefully; server=" + server);
+    } catch (IOException e) {
+      LOG.warn("Failed to persist votedFor/term. Exit the leader election.", e);
+      stopRunning();
+    }
+  }
+
+  /**
+   * After a peer changes its role to candidate, it invokes this method to
+   * send out requestVote rpc to all other peers.
+   */
+  private void askForVotes() throws InterruptedException, IOException {
+    final ServerState state = server.getState();
+    while (running && server.isCandidate()) {
+      // one round of requestVotes
+      final long electionTerm;
+      synchronized (server) {
+        electionTerm = state.initElection();
+        server.getState().persistMetadata();
+      }
+      LOG.info(state.getSelfId() + ": begin an election in Term "
+          + electionTerm);
+
+      TermIndex lastEntry = ServerProtoUtils.toTermIndex(
+          state.getLog().getLastEntry());
+      if (lastEntry == null) {
+        // lastEntry may need to be derived from snapshot
+        SnapshotInfo snapshot = state.getLatestSnapshot();
+        if (snapshot != null) {
+          lastEntry = snapshot.getTermIndex();
+        }
+      }
+
+      final ResultAndTerm r;
+      if (others.isEmpty()) {
+        r = new ResultAndTerm(Result.PASSED, electionTerm);
+      } else {
+        try {
+          initExecutor();
+          int submitted = submitRequests(electionTerm, lastEntry);
+          r = waitForResults(electionTerm, submitted);
+        } finally {
+          if (executor != null) {
+            executor.shutdown();
+          }
+        }
+      }
+
+      synchronized (server) {
+        if (electionTerm != state.getCurrentTerm() || !running ||
+            !server.isCandidate()) {
+          return; // term already passed or no longer a candidate.
+        }
+
+        switch (r.result) {
+          case PASSED:
+            server.changeToLeader();
+            return;
+          case SHUTDOWN:
+            LOG.info("{} received shutdown response when requesting votes.",
+                server.getId());
+            server.close();
+            return;
+          case REJECTED:
+          case DISCOVERED_A_NEW_TERM:
+            final long term = r.term > server.getState().getCurrentTerm() ?
+                r.term : server.getState().getCurrentTerm();
+            server.changeToFollower(term, true);
+            return;
+          case TIMEOUT:
+            // should start another election
+        }
+      }
+    }
+  }
+
+  private int submitRequests(final long electionTerm, final TermIndex lastEntry) {
+    int submitted = 0;
+    for (final RaftPeer peer : others) {
+      final RequestVoteRequestProto r = server.createRequestVoteRequest(
+          peer.getId(), electionTerm, lastEntry);
+      service.submit(
+          () -> server.getServerRpc().requestVote(r));
+      submitted++;
+    }
+    return submitted;
+  }
+
+  private ResultAndTerm waitForResults(final long electionTerm,
+      final int submitted) throws InterruptedException {
+    final Timestamp timeout = new Timestamp().addTimeMs(server.getRandomTimeoutMs());
+    final List<RequestVoteReplyProto> responses = new ArrayList<>();
+    final List<Exception> exceptions = new ArrayList<>();
+    int waitForNum = submitted;
+    Collection<String> votedPeers = new ArrayList<>();
+    while (waitForNum > 0 && running && server.isCandidate()) {
+      final long waitTime = -timeout.elapsedTimeMs();
+      if (waitTime <= 0) {
+        return logAndReturn(Result.TIMEOUT, responses, exceptions, -1);
+      }
+
+      try {
+        final Future<RequestVoteReplyProto> future = service.poll(
+            waitTime, TimeUnit.MILLISECONDS);
+        if (future == null) {
+          continue; // poll timeout, continue to return Result.TIMEOUT
+        }
+
+        final RequestVoteReplyProto r = future.get();
+        responses.add(r);
+        if (r.getShouldShutdown()) {
+          return logAndReturn(Result.SHUTDOWN, responses, exceptions, -1);
+        }
+        if (r.getTerm() > electionTerm) {
+          return logAndReturn(Result.DISCOVERED_A_NEW_TERM, responses,
+              exceptions, r.getTerm());
+        }
+        if (r.getServerReply().getSuccess()) {
+          votedPeers.add(r.getServerReply().getReplyId());
+          if (conf.hasMajority(votedPeers, server.getId())) {
+            return logAndReturn(Result.PASSED, responses, exceptions, -1);
+          }
+        }
+      } catch(ExecutionException e) {
+        LOG.info("Got exception when requesting votes: " + e);
+        LOG.trace("TRACE", e);
+        exceptions.add(e);
+      }
+      waitForNum--;
+    }
+    // received all the responses
+    return logAndReturn(Result.REJECTED, responses, exceptions, -1);
+  }
+}

http://git-wip-us.apache.org/repos/asf/incubator-ratis/blob/7e71a2e0/ratis-server/src/main/java/org/apache/ratis/server/impl/LeaderState.java
----------------------------------------------------------------------
diff --git a/ratis-server/src/main/java/org/apache/ratis/server/impl/LeaderState.java b/ratis-server/src/main/java/org/apache/ratis/server/impl/LeaderState.java
new file mode 100644
index 0000000..e4b2889
--- /dev/null
+++ b/ratis-server/src/main/java/org/apache/ratis/server/impl/LeaderState.java
@@ -0,0 +1,601 @@
+/**
+ * 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.ratis.server.impl;
+
+import static org.apache.ratis.server.RaftServerConfigKeys.RAFT_SERVER_RPC_SLEEP_TIME_MS_DEFAULT;
+import static org.apache.ratis.server.RaftServerConfigKeys.RAFT_SERVER_RPC_SLEEP_TIME_MS_KEY;
+import static org.apache.ratis.server.RaftServerConfigKeys.RAFT_SERVER_STAGING_CATCHUP_GAP_DEFAULT;
+import static org.apache.ratis.server.RaftServerConfigKeys.RAFT_SERVER_STAGING_CATCHUP_GAP_KEY;
+import static org.apache.ratis.server.RaftServerConfigKeys.RAFT_SNAPSHOT_CHUNK_MAX_SIZE_DEFAULT;
+import static org.apache.ratis.server.RaftServerConfigKeys.RAFT_SNAPSHOT_CHUNK_MAX_SIZE_KEY;
+import static org.apache.ratis.server.impl.LeaderState.StateUpdateEventType.STAGINGPROGRESS;
+import static org.apache.ratis.server.impl.LeaderState.StateUpdateEventType.STEPDOWN;
+import static org.apache.ratis.server.impl.LeaderState.StateUpdateEventType.UPDATECOMMIT;
+
+import java.io.IOException;
+import java.util.ArrayList;
+import java.util.Arrays;
+import java.util.Collection;
+import java.util.Collections;
+import java.util.HashMap;
+import java.util.Iterator;
+import java.util.List;
+import java.util.Map;
+import java.util.concurrent.ArrayBlockingQueue;
+import java.util.concurrent.BlockingQueue;
+import java.util.concurrent.CompletableFuture;
+import java.util.concurrent.TimeUnit;
+import java.util.stream.Collectors;
+
+import org.apache.ratis.conf.RaftProperties;
+import org.apache.ratis.protocol.Message;
+import org.apache.ratis.protocol.RaftClientRequest;
+import org.apache.ratis.protocol.RaftPeer;
+import org.apache.ratis.protocol.ReconfigurationTimeoutException;
+import org.apache.ratis.protocol.SetConfigurationRequest;
+import org.apache.ratis.server.storage.RaftLog;
+import org.apache.ratis.shaded.proto.RaftProtos.LeaderNoOp;
+import org.apache.ratis.shaded.proto.RaftProtos.LogEntryProto;
+import org.apache.ratis.statemachine.TransactionContext;
+import org.apache.ratis.util.Daemon;
+import org.apache.ratis.util.ProtoUtils;
+import org.apache.ratis.util.Timestamp;
+import org.slf4j.Logger;
+
+import com.google.common.base.Preconditions;
+
+/**
+ * States for leader only. It contains three different types of processors:
+ * 1. RPC senders: each thread is appending log to a follower
+ * 2. EventProcessor: a single thread updating the raft server's state based on
+ *                    status of log appending response
+ * 3. PendingRequestHandler: a handler sending back responses to clients when
+ *                           corresponding log entries are committed
+ */
+public class LeaderState {
+  private static final Logger LOG = RaftServerImpl.LOG;
+
+  enum StateUpdateEventType {
+    STEPDOWN, UPDATECOMMIT, STAGINGPROGRESS
+  }
+
+  enum BootStrapProgress {
+    NOPROGRESS, PROGRESSING, CAUGHTUP
+  }
+
+  static class StateUpdateEvent {
+    final StateUpdateEventType type;
+    final long newTerm;
+
+    StateUpdateEvent(StateUpdateEventType type, long newTerm) {
+      this.type = type;
+      this.newTerm = newTerm;
+    }
+  }
+
+  static final StateUpdateEvent UPDATE_COMMIT_EVENT =
+      new StateUpdateEvent(StateUpdateEventType.UPDATECOMMIT, -1);
+  static final StateUpdateEvent STAGING_PROGRESS_EVENT =
+      new StateUpdateEvent(StateUpdateEventType.STAGINGPROGRESS, -1);
+
+  private final RaftServerImpl server;
+  private final RaftLog raftLog;
+  private final long currentTerm;
+  private volatile ConfigurationStagingState stagingState;
+  private List<List<FollowerInfo>> voterLists;
+
+  /**
+   * The list of threads appending entries to followers.
+   * The list is protected by the RaftServer's lock.
+   */
+  private final List<LogAppender> senders;
+  private final BlockingQueue<StateUpdateEvent> eventQ;
+  private final EventProcessor processor;
+  private final PendingRequests pendingRequests;
+  private volatile boolean running = true;
+
+  private final int stagingCatchupGap;
+  private final int snapshotChunkMaxSize;
+  private final int syncInterval;
+
+  LeaderState(RaftServerImpl server, RaftProperties properties) {
+    this.server = server;
+
+    stagingCatchupGap = properties.getInt(
+        RAFT_SERVER_STAGING_CATCHUP_GAP_KEY,
+        RAFT_SERVER_STAGING_CATCHUP_GAP_DEFAULT);
+    snapshotChunkMaxSize = properties.getInt(
+        RAFT_SNAPSHOT_CHUNK_MAX_SIZE_KEY,
+        RAFT_SNAPSHOT_CHUNK_MAX_SIZE_DEFAULT);
+    syncInterval = properties.getInt(
+        RAFT_SERVER_RPC_SLEEP_TIME_MS_KEY,
+        RAFT_SERVER_RPC_SLEEP_TIME_MS_DEFAULT);
+
+    final ServerState state = server.getState();
+    this.raftLog = state.getLog();
+    this.currentTerm = state.getCurrentTerm();
+    eventQ = new ArrayBlockingQueue<>(4096);
+    processor = new EventProcessor();
+    pendingRequests = new PendingRequests(server);
+
+    final RaftConfiguration conf = server.getRaftConf();
+    Collection<RaftPeer> others = conf.getOtherPeers(state.getSelfId());
+    final Timestamp t = new Timestamp().addTimeMs(-server.getMaxTimeoutMs());
+    final long nextIndex = raftLog.getNextIndex();
+    senders = new ArrayList<>(others.size());
+    for (RaftPeer p : others) {
+      FollowerInfo f = new FollowerInfo(p, t, nextIndex, true);
+      senders.add(server.getLogAppenderFactory().getLogAppender(server, this, f));
+    }
+    voterLists = divideFollowers(conf);
+  }
+
+  void start() {
+    // In the beginning of the new term, replicate an empty entry in order
+    // to finally commit entries in the previous term.
+    // Also this message can help identify the last committed index when
+    // the leader peer is just started.
+    final LogEntryProto placeHolder = LogEntryProto.newBuilder()
+        .setTerm(server.getState().getCurrentTerm())
+        .setIndex(raftLog.getNextIndex())
+        .setNoOp(LeaderNoOp.newBuilder()).build();
+    raftLog.append(placeHolder);
+
+    processor.start();
+    startSenders();
+  }
+
+  private void startSenders() {
+    senders.forEach(Thread::start);
+  }
+
+  void stop() {
+    this.running = false;
+    // do not interrupt event processor since it may be in the middle of logSync
+    for (LogAppender sender : senders) {
+      sender.stopSender();
+      sender.interrupt();
+    }
+    try {
+      pendingRequests.sendNotLeaderResponses();
+    } catch (IOException e) {
+      LOG.warn("Caught exception in sendNotLeaderResponses", e);
+    }
+  }
+
+  void notifySenders() {
+    senders.forEach(LogAppender::notifyAppend);
+  }
+
+  boolean inStagingState() {
+    return stagingState != null;
+  }
+
+  ConfigurationStagingState getStagingState() {
+    return stagingState;
+  }
+
+  long getCurrentTerm() {
+    return currentTerm;
+  }
+
+  int getSnapshotChunkMaxSize() {
+    return snapshotChunkMaxSize;
+  }
+
+  int getSyncInterval() {
+    return syncInterval;
+  }
+
+  /**
+   * Start bootstrapping new peers
+   */
+  PendingRequest startSetConfiguration(SetConfigurationRequest request) {
+    Preconditions.checkState(running && !inStagingState());
+
+    RaftPeer[] peersInNewConf = request.getPeersInNewConf();
+    Collection<RaftPeer> peersToBootStrap = RaftConfiguration
+        .computeNewPeers(peersInNewConf, server.getRaftConf());
+
+    // add the request to the pending queue
+    final PendingRequest pending = pendingRequests.addConfRequest(request);
+
+    ConfigurationStagingState stagingState = new ConfigurationStagingState(
+        peersToBootStrap, new PeerConfiguration(Arrays.asList(peersInNewConf)));
+    Collection<RaftPeer> newPeers = stagingState.getNewPeers();
+    // set the staging state
+    this.stagingState = stagingState;
+
+    if (newPeers.isEmpty()) {
+      applyOldNewConf();
+    } else {
+      // update the LeaderState's sender list
+      addSenders(newPeers);
+    }
+    return pending;
+  }
+
+  PendingRequest addPendingRequest(long index, RaftClientRequest request,
+      TransactionContext entry) {
+    return pendingRequests.addPendingRequest(index, request, entry);
+  }
+
+  private void applyOldNewConf() {
+    final ServerState state = server.getState();
+    final RaftConfiguration current = server.getRaftConf();
+    final RaftConfiguration oldNewConf= stagingState.generateOldNewConf(current,
+        state.getLog().getNextIndex());
+    // apply the (old, new) configuration to log, and use it as the current conf
+    long index = state.getLog().append(state.getCurrentTerm(), oldNewConf);
+    updateConfiguration(index, oldNewConf);
+
+    this.stagingState = null;
+    notifySenders();
+  }
+
+  private void updateConfiguration(long logIndex, RaftConfiguration newConf) {
+    voterLists = divideFollowers(newConf);
+    server.getState().setRaftConf(logIndex, newConf);
+  }
+
+  /**
+   * After receiving a setConfiguration request, the leader should update its
+   * RpcSender list.
+   */
+  void addSenders(Collection<RaftPeer> newMembers) {
+    final Timestamp t = new Timestamp().addTimeMs(-server.getMaxTimeoutMs());
+    final long nextIndex = raftLog.getNextIndex();
+    for (RaftPeer peer : newMembers) {
+      FollowerInfo f = new FollowerInfo(peer, t, nextIndex, false);
+      LogAppender sender = server.getLogAppenderFactory()
+          .getLogAppender(server, this, f);
+      senders.add(sender);
+      sender.start();
+    }
+  }
+
+  /**
+   * Update the RpcSender list based on the current configuration
+   */
+  private void updateSenders(RaftConfiguration conf) {
+    Preconditions.checkState(conf.isStable() && !inStagingState());
+    Iterator<LogAppender> iterator = senders.iterator();
+    while (iterator.hasNext()) {
+      LogAppender sender = iterator.next();
+      if (!conf.containsInConf(sender.getFollower().getPeer().getId())) {
+        iterator.remove();
+        sender.stopSender();
+        sender.interrupt();
+      }
+    }
+  }
+
+  void submitUpdateStateEvent(StateUpdateEvent event) {
+    try {
+      eventQ.put(event);
+    } catch (InterruptedException e) {
+      LOG.info("Interrupted when adding event {} into the queue", event);
+    }
+  }
+
+  private void prepare() {
+    synchronized (server) {
+      if (running) {
+        final RaftConfiguration conf = server.getRaftConf();
+        if (conf.isTransitional() && server.getState().isConfCommitted()) {
+          // the configuration is in transitional state, and has been committed
+          // so it is time to generate and replicate (new) conf.
+          replicateNewConf();
+        }
+      }
+    }
+  }
+
+  /**
+   * The processor thread takes the responsibility to update the raft server's
+   * state, such as changing to follower, or updating the committed index.
+   */
+  private class EventProcessor extends Daemon {
+    @Override
+    public void run() {
+      // apply an empty message; check if necessary to replicate (new) conf
+      prepare();
+
+      while (running) {
+        try {
+          StateUpdateEvent event = eventQ.poll(server.getMaxTimeoutMs(),
+              TimeUnit.MILLISECONDS);
+          synchronized (server) {
+            if (running) {
+              handleEvent(event);
+            }
+          }
+          // the updated configuration does not need to be sync'ed here
+        } catch (InterruptedException e) {
+          final String s = server.getId() + " " + getClass().getSimpleName()
+              + " thread is interrupted ";
+          if (!running) {
+            LOG.info(s + " gracefully; server=" + server);
+          } else {
+            LOG.warn(s + " UNEXPECTEDLY; server=" + server, e);
+            throw new RuntimeException(e);
+          }
+        } catch (IOException e) {
+          LOG.warn("Failed to persist new votedFor/term.", e);
+          // the failure should happen while changing the state to follower
+          // thus the in-memory state should have been updated
+          Preconditions.checkState(!running);
+        }
+      }
+    }
+  }
+
+  private void handleEvent(StateUpdateEvent e) throws IOException {
+    if (e == null) {
+      if (inStagingState()) {
+        checkNewPeers();
+      }
+    } else {
+      if (e.type == STEPDOWN) {
+        server.changeToFollower(e.newTerm, true);
+      } else if (e.type == UPDATECOMMIT) {
+        updateLastCommitted();
+      } else if (e.type == STAGINGPROGRESS) {
+        checkNewPeers();
+      }
+    }
+  }
+
+  /**
+   * So far we use a simple implementation for catchup checking:
+   * 1. If the latest rpc time of the remote peer is before 3 * max_timeout,
+   *    the peer made no progress for that long. We should fail the whole
+   *    setConfiguration request.
+   * 2. If the peer's matching index is just behind for a small gap, and the
+   *    peer was updated recently (within max_timeout), declare the peer as
+   *    caught-up.
+   * 3. Otherwise the peer is making progressing. Keep waiting.
+   */
+  private BootStrapProgress checkProgress(FollowerInfo follower,
+      long committed) {
+    Preconditions.checkArgument(!follower.isAttendingVote());
+    final Timestamp progressTime = new Timestamp().addTimeMs(-server.getMaxTimeoutMs());
+    final Timestamp timeoutTime = new Timestamp().addTimeMs(-3*server.getMaxTimeoutMs());
+    if (follower.getLastRpcResponseTime().compareTo(timeoutTime) < 0) {
+      LOG.debug("{} detects a follower {} timeout for bootstrapping," +
+              " timeoutTime: {}", server.getId(), follower, timeoutTime);
+      return BootStrapProgress.NOPROGRESS;
+    } else if (follower.getMatchIndex() + stagingCatchupGap > committed
+        && follower.getLastRpcResponseTime().compareTo(progressTime) > 0) {
+      return BootStrapProgress.CAUGHTUP;
+    } else {
+      return BootStrapProgress.PROGRESSING;
+    }
+  }
+
+  private Collection<BootStrapProgress> checkAllProgress(long committed) {
+    Preconditions.checkState(inStagingState());
+    return senders.stream()
+        .filter(sender -> !sender.getFollower().isAttendingVote())
+        .map(sender -> checkProgress(sender.getFollower(), committed))
+        .collect(Collectors.toCollection(ArrayList::new));
+  }
+
+  private void checkNewPeers() {
+    if (!inStagingState()) {
+      // it is possible that the bootstrapping is done and we still have
+      // remaining STAGINGPROGRESS event to handle.
+      updateLastCommitted();
+    } else {
+      final long committedIndex = server.getState().getLog()
+          .getLastCommittedIndex();
+      Collection<BootStrapProgress> reports = checkAllProgress(committedIndex);
+      if (reports.contains(BootStrapProgress.NOPROGRESS)) {
+        LOG.debug("{} fails the setConfiguration request", server.getId());
+        stagingState.fail();
+      } else if (!reports.contains(BootStrapProgress.PROGRESSING)) {
+        // all caught up!
+        applyOldNewConf();
+        for (LogAppender sender : senders) {
+          sender.getFollower().startAttendVote();
+        }
+      }
+    }
+  }
+
+  boolean isBootStrappingPeer(String peerId) {
+    return inStagingState() && getStagingState().contains(peerId);
+  }
+
+  private void updateLastCommitted() {
+    final String selfId = server.getId();
+    final RaftConfiguration conf = server.getRaftConf();
+    long majorityInNewConf = computeLastCommitted(voterLists.get(0),
+        conf.containsInConf(selfId));
+    final long oldLastCommitted = raftLog.getLastCommittedIndex();
+    final LogEntryProto[] entriesToCommit;
+    if (!conf.isTransitional()) {
+      // copy the entries that may get committed out of the raftlog, to prevent
+      // the possible race that the log gets purged after the statemachine does
+      // a snapshot
+      entriesToCommit = raftLog.getEntries(oldLastCommitted + 1,
+          Math.max(majorityInNewConf, oldLastCommitted) + 1);
+      server.getState().updateStatemachine(majorityInNewConf, currentTerm);
+    } else { // configuration is in transitional state
+      long majorityInOldConf = computeLastCommitted(voterLists.get(1),
+          conf.containsInOldConf(selfId));
+      final long majority = Math.min(majorityInNewConf, majorityInOldConf);
+      entriesToCommit = raftLog.getEntries(oldLastCommitted + 1,
+          Math.max(majority, oldLastCommitted) + 1);
+      server.getState().updateStatemachine(majority, currentTerm);
+    }
+    checkAndUpdateConfiguration(entriesToCommit);
+  }
+
+  private boolean committedConf(LogEntryProto[] entries) {
+    final long currentCommitted = raftLog.getLastCommittedIndex();
+    for (LogEntryProto entry : entries) {
+      if (entry.getIndex() <= currentCommitted &&
+          ProtoUtils.isConfigurationLogEntry(entry)) {
+        return true;
+      }
+    }
+    return false;
+  }
+
+  private void checkAndUpdateConfiguration(LogEntryProto[] entriesToCheck) {
+    final RaftConfiguration conf = server.getRaftConf();
+    if (committedConf(entriesToCheck)) {
+      if (conf.isTransitional()) {
+        replicateNewConf();
+      } else { // the (new) log entry has been committed
+        LOG.debug("{} sends success to setConfiguration request", server.getId());
+        pendingRequests.replySetConfiguration();
+        // if the leader is not included in the current configuration, step down
+        if (!conf.containsInConf(server.getId())) {
+          LOG.info("{} is not included in the new configuration {}. Step down.",
+              server.getId(), conf);
+          try {
+            // leave some time for all RPC senders to send out new conf entry
+            Thread.sleep(server.getMinTimeoutMs());
+          } catch (InterruptedException ignored) {
+          }
+          // the pending request handler will send NotLeaderException for
+          // pending client requests when it stops
+          server.close();
+        }
+      }
+    }
+  }
+
+  /**
+   * when the (old, new) log entry has been committed, should replicate (new):
+   * 1) append (new) to log
+   * 2) update conf to (new)
+   * 3) update RpcSenders list
+   * 4) start replicating the log entry
+   */
+  private void replicateNewConf() {
+    final RaftConfiguration conf = server.getRaftConf();
+    final RaftConfiguration newConf = RaftConfiguration.newBuilder()
+        .setConf(conf)
+        .setLogEntryIndex(raftLog.getNextIndex())
+        .build();
+    // stop the LogAppender if the corresponding follower is no longer in the conf
+    updateSenders(newConf);
+    long index = raftLog.append(server.getState().getCurrentTerm(), newConf);
+    updateConfiguration(index, newConf);
+    notifySenders();
+  }
+
+  private long computeLastCommitted(List<FollowerInfo> followers,
+      boolean includeSelf) {
+    final int length = includeSelf ? followers.size() + 1 : followers.size();
+    final long[] indices = new long[length];
+    for (int i = 0; i < followers.size(); i++) {
+      indices[i] = followers.get(i).getMatchIndex();
+    }
+    if (includeSelf) {
+      // note that we also need to wait for the local disk I/O
+      indices[length - 1] = raftLog.getLatestFlushedIndex();
+    }
+
+    Arrays.sort(indices);
+    return indices[(indices.length - 1) / 2];
+  }
+
+  private List<List<FollowerInfo>> divideFollowers(RaftConfiguration conf) {
+    List<List<FollowerInfo>> lists = new ArrayList<>(2);
+    List<FollowerInfo> listForNew = senders.stream()
+        .filter(sender -> conf.containsInConf(sender.getFollower().getPeer().getId()))
+        .map(LogAppender::getFollower)
+        .collect(Collectors.toList());
+    lists.add(listForNew);
+    if (conf.isTransitional()) {
+      List<FollowerInfo> listForOld = senders.stream()
+          .filter(sender -> conf.containsInOldConf(sender.getFollower().getPeer().getId()))
+          .map(LogAppender::getFollower)
+          .collect(Collectors.toList());
+      lists.add(listForOld);
+    }
+    return lists;
+  }
+
+  PendingRequest returnNoConfChange(SetConfigurationRequest r) {
+    PendingRequest pending = new PendingRequest(r);
+    pending.setSuccessReply(null);
+    return pending;
+  }
+
+  void replyPendingRequest(long logIndex, CompletableFuture<Message> message) {
+    pendingRequests.replyPendingRequest(logIndex, message);
+  }
+
+  TransactionContext getTransactionContext(long index) {
+    return pendingRequests.getTransactionContext(index);
+  }
+
+  private class ConfigurationStagingState {
+    private final Map<String, RaftPeer> newPeers;
+    private final PeerConfiguration newConf;
+
+    ConfigurationStagingState(Collection<RaftPeer> newPeers,
+        PeerConfiguration newConf) {
+      Map<String, RaftPeer> map = new HashMap<>();
+      for (RaftPeer peer : newPeers) {
+        map.put(peer.getId(), peer);
+      }
+      this.newPeers = Collections.unmodifiableMap(map);
+      this.newConf = newConf;
+    }
+
+    RaftConfiguration generateOldNewConf(RaftConfiguration current,
+        long logIndex) {
+      return RaftConfiguration.newBuilder()
+          .setConf(newConf)
+          .setOldConf(current)
+          .setLogEntryIndex(logIndex)
+          .build();
+    }
+
+    Collection<RaftPeer> getNewPeers() {
+      return newPeers.values();
+    }
+
+    boolean contains(String peerId) {
+      return newPeers.containsKey(peerId);
+    }
+
+    void fail() {
+      Iterator<LogAppender> iterator = senders.iterator();
+      while (iterator.hasNext()) {
+        LogAppender sender = iterator.next();
+        if (!sender.getFollower().isAttendingVote()) {
+          iterator.remove();
+          sender.stopSender();
+          sender.interrupt();
+        }
+      }
+      LeaderState.this.stagingState = null;
+      // send back failure response to client's request
+      pendingRequests.failSetConfiguration(
+          new ReconfigurationTimeoutException("Fail to set configuration "
+              + newConf + ". Timeout when bootstrapping new peers."));
+    }
+  }
+}

http://git-wip-us.apache.org/repos/asf/incubator-ratis/blob/7e71a2e0/ratis-server/src/main/java/org/apache/ratis/server/impl/LogAppender.java
----------------------------------------------------------------------
diff --git a/ratis-server/src/main/java/org/apache/ratis/server/impl/LogAppender.java b/ratis-server/src/main/java/org/apache/ratis/server/impl/LogAppender.java
new file mode 100644
index 0000000..5599699
--- /dev/null
+++ b/ratis-server/src/main/java/org/apache/ratis/server/impl/LogAppender.java
@@ -0,0 +1,494 @@
+/**
+ * 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.ratis.server.impl;
+
+import static org.apache.ratis.server.RaftServerConfigKeys.RAFT_SERVER_LOG_APPENDER_BATCH_ENABLED_DEFAULT;
+import static org.apache.ratis.server.RaftServerConfigKeys.RAFT_SERVER_LOG_APPENDER_BATCH_ENABLED_KEY;
+import static org.apache.ratis.server.RaftServerConfigKeys.RAFT_SERVER_LOG_APPENDER_BUFFER_CAPACITY_DEFAULT;
+import static org.apache.ratis.server.RaftServerConfigKeys.RAFT_SERVER_LOG_APPENDER_BUFFER_CAPACITY_KEY;
+import static org.apache.ratis.server.impl.RaftServerConstants.INVALID_LOG_INDEX;
+
+import java.io.File;
+import java.io.FileInputStream;
+import java.io.IOException;
+import java.io.InterruptedIOException;
+import java.nio.file.Path;
+import java.util.ArrayList;
+import java.util.Iterator;
+import java.util.List;
+import java.util.NoSuchElementException;
+import java.util.UUID;
+
+import org.apache.commons.io.IOUtils;
+import org.apache.ratis.server.impl.LeaderState.StateUpdateEventType;
+import org.apache.ratis.server.protocol.TermIndex;
+import org.apache.ratis.server.storage.FileInfo;
+import org.apache.ratis.server.storage.RaftLog;
+import org.apache.ratis.shaded.com.google.protobuf.ByteString;
+import org.apache.ratis.shaded.proto.RaftProtos.AppendEntriesReplyProto;
+import org.apache.ratis.shaded.proto.RaftProtos.AppendEntriesRequestProto;
+import org.apache.ratis.shaded.proto.RaftProtos.FileChunkProto;
+import org.apache.ratis.shaded.proto.RaftProtos.InstallSnapshotReplyProto;
+import org.apache.ratis.shaded.proto.RaftProtos.InstallSnapshotRequestProto;
+import org.apache.ratis.shaded.proto.RaftProtos.InstallSnapshotResult;
+import org.apache.ratis.shaded.proto.RaftProtos.LogEntryProto;
+import org.apache.ratis.statemachine.SnapshotInfo;
+import org.apache.ratis.util.Daemon;
+import org.apache.ratis.util.ProtoUtils;
+import org.apache.ratis.util.Timestamp;
+import org.slf4j.Logger;
+
+import com.google.common.base.Preconditions;
+import com.google.common.collect.Lists;
+
+/**
+ * A daemon thread appending log entries to a follower peer.
+ */
+public class LogAppender extends Daemon {
+  public static final Logger LOG = RaftServerImpl.LOG;
+
+  protected final RaftServerImpl server;
+  private final LeaderState leaderState;
+  protected final RaftLog raftLog;
+  protected final FollowerInfo follower;
+  private final int maxBufferSize;
+  private final boolean batchSending;
+  private final LogEntryBuffer buffer;
+  private final long leaderTerm;
+
+  private volatile boolean sending = true;
+
+  public LogAppender(RaftServerImpl server, LeaderState leaderState, FollowerInfo f) {
+    this.follower = f;
+    this.server = server;
+    this.leaderState = leaderState;
+    this.raftLog = server.getState().getLog();
+    this.maxBufferSize = server.getProperties().getInt(
+        RAFT_SERVER_LOG_APPENDER_BUFFER_CAPACITY_KEY,
+        RAFT_SERVER_LOG_APPENDER_BUFFER_CAPACITY_DEFAULT);
+    this.batchSending = server.getProperties().getBoolean(
+        RAFT_SERVER_LOG_APPENDER_BATCH_ENABLED_KEY,
+        RAFT_SERVER_LOG_APPENDER_BATCH_ENABLED_DEFAULT);
+    this.buffer = new LogEntryBuffer();
+    this.leaderTerm = server.getState().getCurrentTerm();
+  }
+
+  @Override
+  public String toString() {
+    return getClass().getSimpleName() + "(" + server.getId() + " -> " +
+        follower.getPeer().getId() + ")";
+  }
+
+  @Override
+  public void run() {
+    try {
+      checkAndSendAppendEntries();
+    } catch (InterruptedException | InterruptedIOException e) {
+      LOG.info(this + " was interrupted: " + e);
+    }
+  }
+
+  protected boolean isAppenderRunning() {
+    return sending;
+  }
+
+  public void stopSender() {
+    this.sending = false;
+  }
+
+  public FollowerInfo getFollower() {
+    return follower;
+  }
+
+  /**
+   * A buffer for log entries with size limitation.
+   */
+  private class LogEntryBuffer {
+    private final List<LogEntryProto> buf = new ArrayList<>();
+    private int totalSize = 0;
+
+    void addEntry(LogEntryProto entry) {
+      buf.add(entry);
+      totalSize += entry.getSerializedSize();
+    }
+
+    boolean isFull() {
+      return totalSize >= maxBufferSize;
+    }
+
+    boolean isEmpty() {
+      return buf.isEmpty();
+    }
+
+    AppendEntriesRequestProto getAppendRequest(TermIndex previous) {
+      final AppendEntriesRequestProto request = server
+          .createAppendEntriesRequest(leaderTerm, follower.getPeer().getId(),
+              previous, buf, !follower.isAttendingVote());
+      buf.clear();
+      totalSize = 0;
+      return request;
+    }
+
+    int getPendingEntryNum() {
+      return buf.size();
+    }
+  }
+
+  private TermIndex getPrevious() {
+    TermIndex previous = ServerProtoUtils.toTermIndex(
+        raftLog.get(follower.getNextIndex() - 1));
+    if (previous == null) {
+      // if previous is null, nextIndex must be equal to the log start
+      // index (otherwise we will install snapshot).
+      Preconditions.checkState(follower.getNextIndex() == raftLog.getStartIndex(),
+          "follower's next index %s, local log start index %s",
+          follower.getNextIndex(), raftLog.getStartIndex());
+      SnapshotInfo snapshot = server.getState().getLatestSnapshot();
+      previous = snapshot == null ? null : snapshot.getTermIndex();
+    }
+    return previous;
+  }
+
+  protected AppendEntriesRequestProto createRequest() {
+    final TermIndex previous = getPrevious();
+    final long leaderNext = raftLog.getNextIndex();
+    long next = follower.getNextIndex() + buffer.getPendingEntryNum();
+    boolean toSend = false;
+
+    if (leaderNext == next && !buffer.isEmpty()) {
+      // no new entries, then send out the entries in the buffer
+      toSend = true;
+    } else if (leaderNext > next) {
+      while (leaderNext > next && !buffer.isFull()) {
+        // stop adding entry once the buffer size is >= the max size
+        buffer.addEntry(raftLog.get(next++));
+      }
+      if (buffer.isFull() || !batchSending) {
+        // buffer is full or batch sending is disabled, send out a request
+        toSend = true;
+      }
+    }
+
+    if (toSend || shouldHeartbeat()) {
+      return buffer.getAppendRequest(previous);
+    }
+    return null;
+  }
+
+  /** Send an appendEntries RPC; retry indefinitely. */
+  private AppendEntriesReplyProto sendAppendEntriesWithRetries()
+      throws InterruptedException, InterruptedIOException {
+    int retry = 0;
+    AppendEntriesRequestProto request = null;
+    while (isAppenderRunning()) { // keep retrying for IOException
+      try {
+        if (request == null || request.getEntriesCount() == 0) {
+          request = createRequest();
+        }
+
+        if (request == null) {
+          LOG.trace("{} need not send AppendEntries now." +
+              " Wait for more entries.", server.getId());
+          return null;
+        } else if (!isAppenderRunning()) {
+          LOG.debug("LogAppender {} has been stopped. Skip the request.", this);
+          return null;
+        }
+
+        follower.updateLastRpcSendTime();
+        final AppendEntriesReplyProto r = server.getServerRpc()
+            .appendEntries(request);
+        follower.updateLastRpcResponseTime();
+
+        return r;
+      } catch (InterruptedIOException iioe) {
+        throw iioe;
+      } catch (IOException ioe) {
+        LOG.debug(this + ": failed to send appendEntries; retry " + retry++, ioe);
+      }
+      if (isAppenderRunning()) {
+        Thread.sleep(leaderState.getSyncInterval());
+      }
+    }
+    return null;
+  }
+
+  protected class SnapshotRequestIter
+      implements Iterable<InstallSnapshotRequestProto> {
+    private final SnapshotInfo snapshot;
+    private final List<FileInfo> files;
+    private FileInputStream in;
+    private int fileIndex = 0;
+
+    private FileInfo currentFileInfo;
+    private byte[] currentBuf;
+    private long currentFileSize;
+    private long currentOffset = 0;
+    private int chunkIndex = 0;
+
+    private final String requestId;
+    private int requestIndex = 0;
+
+    public SnapshotRequestIter(SnapshotInfo snapshot, String requestId)
+        throws IOException {
+      this.snapshot = snapshot;
+      this.requestId = requestId;
+      this.files = snapshot.getFiles();
+      if (files.size() > 0) {
+        startReadFile();
+      }
+    }
+
+    private void startReadFile() throws IOException {
+      currentFileInfo = files.get(fileIndex);
+      File snapshotFile = currentFileInfo.getPath().toFile();
+      currentFileSize = snapshotFile.length();
+      final int bufLength =
+          (int) Math.min(leaderState.getSnapshotChunkMaxSize(), currentFileSize);
+      currentBuf = new byte[bufLength];
+      currentOffset = 0;
+      chunkIndex = 0;
+      in = new FileInputStream(snapshotFile);
+    }
+
+    @Override
+    public Iterator<InstallSnapshotRequestProto> iterator() {
+      return new Iterator<InstallSnapshotRequestProto>() {
+        @Override
+        public boolean hasNext() {
+          return fileIndex < files.size();
+        }
+
+        @Override
+        public InstallSnapshotRequestProto next() {
+          if (fileIndex >= files.size()) {
+            throw new NoSuchElementException();
+          }
+          int targetLength = (int) Math.min(currentFileSize - currentOffset,
+              leaderState.getSnapshotChunkMaxSize());
+          FileChunkProto chunk;
+          try {
+            chunk = readFileChunk(currentFileInfo, in, currentBuf,
+                targetLength, currentOffset, chunkIndex);
+            boolean done = (fileIndex == files.size() - 1) &&
+                chunk.getDone();
+            InstallSnapshotRequestProto request =
+                server.createInstallSnapshotRequest(follower.getPeer().getId(),
+                    requestId, requestIndex++, snapshot,
+                    Lists.newArrayList(chunk), done);
+            currentOffset += targetLength;
+            chunkIndex++;
+
+            if (currentOffset >= currentFileSize) {
+              in.close();
+              fileIndex++;
+              if (fileIndex < files.size()) {
+                startReadFile();
+              }
+            }
+
+            return request;
+          } catch (IOException e) {
+            if (in != null) {
+              try {
+                in.close();
+              } catch (IOException ignored) {
+              }
+            }
+            LOG.warn("Got exception when preparing InstallSnapshot request", e);
+            throw new RuntimeException(e);
+          }
+        }
+      };
+    }
+  }
+
+  private FileChunkProto readFileChunk(FileInfo fileInfo,
+      FileInputStream in, byte[] buf, int length, long offset, int chunkIndex)
+      throws IOException {
+    FileChunkProto.Builder builder = FileChunkProto.newBuilder()
+        .setOffset(offset).setChunkIndex(chunkIndex);
+    IOUtils.readFully(in, buf, 0, length);
+    Path relativePath = server.getState().getStorage().getStorageDir()
+        .relativizeToRoot(fileInfo.getPath());
+    builder.setFilename(relativePath.toString());
+    builder.setDone(offset + length == fileInfo.getFileSize());
+    builder.setFileDigest(
+        ByteString.copyFrom(fileInfo.getFileDigest().getDigest()));
+    builder.setData(ByteString.copyFrom(buf, 0, length));
+    return builder.build();
+  }
+
+  private InstallSnapshotReplyProto installSnapshot(SnapshotInfo snapshot)
+      throws InterruptedException, InterruptedIOException {
+    String requestId = UUID.randomUUID().toString();
+    InstallSnapshotReplyProto reply = null;
+    try {
+      for (InstallSnapshotRequestProto request :
+          new SnapshotRequestIter(snapshot, requestId)) {
+        follower.updateLastRpcSendTime();
+        reply = server.getServerRpc().installSnapshot(request);
+        follower.updateLastRpcResponseTime();
+
+        if (!reply.getServerReply().getSuccess()) {
+          return reply;
+        }
+      }
+    } catch (InterruptedIOException iioe) {
+      throw iioe;
+    } catch (Exception ioe) {
+      LOG.warn(this + ": failed to install SnapshotInfo " + snapshot.getFiles(),
+          ioe);
+      return null;
+    }
+
+    if (reply != null) {
+      follower.updateMatchIndex(snapshot.getTermIndex().getIndex());
+      follower.updateNextIndex(snapshot.getTermIndex().getIndex() + 1);
+      LOG.info("{}: install snapshot-{} successfully on follower {}",
+          server.getId(), snapshot.getTermIndex().getIndex(), follower.getPeer());
+    }
+    return reply;
+  }
+
+  protected SnapshotInfo shouldInstallSnapshot() {
+    final long logStartIndex = raftLog.getStartIndex();
+    // we should install snapshot if the follower needs to catch up and:
+    // 1. there is no local log entry but there is snapshot
+    // 2. or the follower's next index is smaller than the log start index
+    if (follower.getNextIndex() < raftLog.getNextIndex()) {
+      SnapshotInfo snapshot = server.getState().getLatestSnapshot();
+      if (follower.getNextIndex() < logStartIndex ||
+          (logStartIndex == INVALID_LOG_INDEX && snapshot != null)) {
+        return snapshot;
+      }
+    }
+    return null;
+  }
+
+  /** Check and send appendEntries RPC */
+  private void checkAndSendAppendEntries()
+      throws InterruptedException, InterruptedIOException {
+    while (isAppenderRunning()) {
+      if (shouldSendRequest()) {
+        SnapshotInfo snapshot = shouldInstallSnapshot();
+        if (snapshot != null) {
+          LOG.info("{}: follower {}'s next index is {}," +
+              " log's start index is {}, need to install snapshot",
+              server.getId(), follower.getPeer(), follower.getNextIndex(),
+              raftLog.getStartIndex());
+
+          final InstallSnapshotReplyProto r = installSnapshot(snapshot);
+          if (r != null && r.getResult() == InstallSnapshotResult.NOT_LEADER) {
+            checkResponseTerm(r.getTerm());
+          } // otherwise if r is null, retry the snapshot installation
+        } else {
+          final AppendEntriesReplyProto r = sendAppendEntriesWithRetries();
+          if (r != null) {
+            handleReply(r);
+          }
+        }
+      }
+      if (isAppenderRunning() && !shouldAppendEntries(
+          follower.getNextIndex() + buffer.getPendingEntryNum())) {
+        final long waitTime = getHeartbeatRemainingTime(
+            follower.getLastRpcTime());
+        if (waitTime > 0) {
+          synchronized (this) {
+            wait(waitTime);
+          }
+        }
+      }
+    }
+  }
+
+  private void handleReply(AppendEntriesReplyProto reply) {
+    if (reply != null) {
+      switch (reply.getResult()) {
+        case SUCCESS:
+          final long oldNextIndex = follower.getNextIndex();
+          final long nextIndex = reply.getNextIndex();
+          if (nextIndex < oldNextIndex) {
+            throw new IllegalStateException("nextIndex=" + nextIndex
+                + " < oldNextIndex=" + oldNextIndex
+                + ", reply=" + ProtoUtils.toString(reply));
+          }
+
+          if (nextIndex > oldNextIndex) {
+            follower.updateMatchIndex(nextIndex - 1);
+            follower.updateNextIndex(nextIndex);
+            submitEventOnSuccessAppend();
+          }
+          break;
+        case NOT_LEADER:
+          // check if should step down
+          checkResponseTerm(reply.getTerm());
+          break;
+        case INCONSISTENCY:
+          follower.decreaseNextIndex(reply.getNextIndex());
+          break;
+        case UNRECOGNIZED:
+          LOG.warn("{} received UNRECOGNIZED AppendResult from {}",
+              server.getId(), follower.getPeer().getId());
+          break;
+      }
+    }
+  }
+
+  protected void submitEventOnSuccessAppend() {
+    LeaderState.StateUpdateEvent e = follower.isAttendingVote() ?
+        LeaderState.UPDATE_COMMIT_EVENT :
+        LeaderState.STAGING_PROGRESS_EVENT;
+    leaderState.submitUpdateStateEvent(e);
+  }
+
+  public synchronized void notifyAppend() {
+    this.notify();
+  }
+
+  /** Should the leader send appendEntries RPC to this follower? */
+  protected boolean shouldSendRequest() {
+    return shouldAppendEntries(follower.getNextIndex()) || shouldHeartbeat();
+  }
+
+  private boolean shouldAppendEntries(long followerIndex) {
+    return followerIndex < raftLog.getNextIndex();
+  }
+
+  private boolean shouldHeartbeat() {
+    return getHeartbeatRemainingTime(follower.getLastRpcTime()) <= 0;
+  }
+
+  /**
+   * @return the time in milliseconds that the leader should send a heartbeat.
+   */
+  protected long getHeartbeatRemainingTime(Timestamp lastTime) {
+    return server.getMinTimeoutMs() / 2 - lastTime.elapsedTimeMs();
+  }
+
+  protected void checkResponseTerm(long responseTerm) {
+    synchronized (server) {
+      if (isAppenderRunning() && follower.isAttendingVote()
+          && responseTerm > leaderState.getCurrentTerm()) {
+        leaderState.submitUpdateStateEvent(
+            new LeaderState.StateUpdateEvent(StateUpdateEventType.STEPDOWN,
+                responseTerm));
+      }
+    }
+  }
+}

http://git-wip-us.apache.org/repos/asf/incubator-ratis/blob/7e71a2e0/ratis-server/src/main/java/org/apache/ratis/server/impl/LogAppenderFactory.java
----------------------------------------------------------------------
diff --git a/ratis-server/src/main/java/org/apache/ratis/server/impl/LogAppenderFactory.java b/ratis-server/src/main/java/org/apache/ratis/server/impl/LogAppenderFactory.java
new file mode 100644
index 0000000..e6cc213
--- /dev/null
+++ b/ratis-server/src/main/java/org/apache/ratis/server/impl/LogAppenderFactory.java
@@ -0,0 +1,31 @@
+/**
+ * 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.ratis.server.impl;
+
+public interface LogAppenderFactory {
+  LogAppender getLogAppender(RaftServerImpl server, LeaderState state,
+                             FollowerInfo f);
+
+  class SynchronousLogAppenderFactory implements LogAppenderFactory {
+    @Override
+    public LogAppender getLogAppender(RaftServerImpl server, LeaderState state,
+                                      FollowerInfo f) {
+      return new LogAppender(server, state, f);
+    }
+  }
+}

http://git-wip-us.apache.org/repos/asf/incubator-ratis/blob/7e71a2e0/ratis-server/src/main/java/org/apache/ratis/server/impl/PeerConfiguration.java
----------------------------------------------------------------------
diff --git a/ratis-server/src/main/java/org/apache/ratis/server/impl/PeerConfiguration.java b/ratis-server/src/main/java/org/apache/ratis/server/impl/PeerConfiguration.java
new file mode 100644
index 0000000..b532303
--- /dev/null
+++ b/ratis-server/src/main/java/org/apache/ratis/server/impl/PeerConfiguration.java
@@ -0,0 +1,91 @@
+/**
+ * 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.ratis.server.impl;
+
+import com.google.common.base.Preconditions;
+
+import java.util.*;
+
+import org.apache.ratis.protocol.RaftPeer;
+
+/**
+ * The peer configuration of a raft cluster.
+ *
+ * The objects of this class are immutable.
+ */
+class PeerConfiguration {
+  private final Map<String, RaftPeer> peers;
+
+  PeerConfiguration(Iterable<RaftPeer> peers) {
+    Preconditions.checkNotNull(peers);
+    Map<String, RaftPeer> map = new HashMap<>();
+    for(RaftPeer p : peers) {
+      map.put(p.getId(), p);
+    }
+    this.peers = Collections.unmodifiableMap(map);
+    Preconditions.checkState(!this.peers.isEmpty());
+  }
+
+  Collection<RaftPeer> getPeers() {
+    return Collections.unmodifiableCollection(peers.values());
+  }
+
+  int size() {
+    return peers.size();
+  }
+
+  @Override
+  public String toString() {
+    return peers.values().toString();
+  }
+
+  RaftPeer getPeer(String id) {
+    return peers.get(id);
+  }
+
+  boolean contains(String id) {
+    return peers.containsKey(id);
+  }
+
+  List<RaftPeer> getOtherPeers(String selfId) {
+    List<RaftPeer> others = new ArrayList<>();
+    for (Map.Entry<String, RaftPeer> entry : peers.entrySet()) {
+      if (!selfId.equals(entry.getValue().getId())) {
+        others.add(entry.getValue());
+      }
+    }
+    return others;
+  }
+
+  boolean hasMajority(Collection<String> others, String selfId) {
+    Preconditions.checkArgument(!others.contains(selfId));
+    int num = 0;
+    if (contains(selfId)) {
+      num++;
+    }
+    for (String other : others) {
+      if (contains(other)) {
+        num++;
+      }
+      if (num > size() / 2) {
+        return true;
+      }
+    }
+    return false;
+  }
+}

http://git-wip-us.apache.org/repos/asf/incubator-ratis/blob/7e71a2e0/ratis-server/src/main/java/org/apache/ratis/server/impl/PendingRequest.java
----------------------------------------------------------------------
diff --git a/ratis-server/src/main/java/org/apache/ratis/server/impl/PendingRequest.java b/ratis-server/src/main/java/org/apache/ratis/server/impl/PendingRequest.java
new file mode 100644
index 0000000..bf47cdc
--- /dev/null
+++ b/ratis-server/src/main/java/org/apache/ratis/server/impl/PendingRequest.java
@@ -0,0 +1,88 @@
+/**
+ * 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
+ * <p>
+ * http://www.apache.org/licenses/LICENSE-2.0
+ * <p>
+ * 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.ratis.server.impl;
+
+import com.google.common.base.Preconditions;
+
+import org.apache.ratis.protocol.Message;
+import org.apache.ratis.protocol.RaftClientReply;
+import org.apache.ratis.protocol.RaftClientRequest;
+import org.apache.ratis.protocol.SetConfigurationRequest;
+import org.apache.ratis.statemachine.TransactionContext;
+
+import java.util.concurrent.CompletableFuture;
+
+public class PendingRequest implements Comparable<PendingRequest> {
+  private final Long index;
+  private final RaftClientRequest request;
+  private final TransactionContext entry;
+  private final CompletableFuture<RaftClientReply> future;
+
+  PendingRequest(long index, RaftClientRequest request,
+                 TransactionContext entry) {
+    this.index = index;
+    this.request = request;
+    this.entry = entry;
+    this.future = new CompletableFuture<>();
+  }
+
+  PendingRequest(SetConfigurationRequest request) {
+    this(RaftServerConstants.INVALID_LOG_INDEX, request, null);
+  }
+
+  long getIndex() {
+    return index;
+  }
+
+  RaftClientRequest getRequest() {
+    return request;
+  }
+
+  public CompletableFuture<RaftClientReply> getFuture() {
+    return future;
+  }
+
+  TransactionContext getEntry() {
+    return entry;
+  }
+
+  synchronized void setException(Throwable e) {
+    Preconditions.checkArgument(e != null);
+    future.completeExceptionally(e);
+  }
+
+  synchronized void setReply(RaftClientReply r) {
+    Preconditions.checkArgument(r != null);
+    future.complete(r);
+  }
+
+  void setSuccessReply(Message message) {
+    setReply(new RaftClientReply(getRequest(), message));
+  }
+
+  @Override
+  public int compareTo(PendingRequest that) {
+    return Long.compare(this.index, that.index);
+  }
+
+  @Override
+  public String toString() {
+    return getClass().getSimpleName() + "(index=" + index
+        + ", request=" + request;
+  }
+}

http://git-wip-us.apache.org/repos/asf/incubator-ratis/blob/7e71a2e0/ratis-server/src/main/java/org/apache/ratis/server/impl/PendingRequests.java
----------------------------------------------------------------------
diff --git a/ratis-server/src/main/java/org/apache/ratis/server/impl/PendingRequests.java b/ratis-server/src/main/java/org/apache/ratis/server/impl/PendingRequests.java
new file mode 100644
index 0000000..6343344
--- /dev/null
+++ b/ratis-server/src/main/java/org/apache/ratis/server/impl/PendingRequests.java
@@ -0,0 +1,130 @@
+/**
+ * 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.ratis.server.impl;
+
+import com.google.common.base.Preconditions;
+
+import org.apache.ratis.protocol.*;
+import org.apache.ratis.statemachine.TransactionContext;
+import org.slf4j.Logger;
+
+import java.io.IOException;
+import java.util.Collection;
+import java.util.concurrent.CompletableFuture;
+import java.util.concurrent.ConcurrentHashMap;
+import java.util.concurrent.ConcurrentMap;
+import java.util.stream.Collectors;
+
+class PendingRequests {
+  private static final Logger LOG = RaftServerImpl.LOG;
+
+  private PendingRequest pendingSetConf;
+  private final RaftServerImpl server;
+  private final ConcurrentMap<Long, PendingRequest> pendingRequests = new ConcurrentHashMap<>();
+  private PendingRequest last = null;
+
+  PendingRequests(RaftServerImpl server) {
+    this.server = server;
+  }
+
+  PendingRequest addPendingRequest(long index, RaftClientRequest request,
+      TransactionContext entry) {
+    // externally synced for now
+    Preconditions.checkArgument(!request.isReadOnly());
+    Preconditions.checkState(last == null || index == last.getIndex() + 1);
+    return add(index, request, entry);
+  }
+
+  private PendingRequest add(long index, RaftClientRequest request,
+      TransactionContext entry) {
+    final PendingRequest pending = new PendingRequest(index, request, entry);
+    pendingRequests.put(index, pending);
+    last = pending;
+    return pending;
+  }
+
+  PendingRequest addConfRequest(SetConfigurationRequest request) {
+    Preconditions.checkState(pendingSetConf == null);
+    pendingSetConf = new PendingRequest(request);
+    return pendingSetConf;
+  }
+
+  void replySetConfiguration() {
+    // we allow the pendingRequest to be null in case that the new leader
+    // commits the new configuration while it has not received the retry
+    // request from the client
+    if (pendingSetConf != null) {
+      // for setConfiguration we do not need to wait for statemachine. send back
+      // reply after it's committed.
+      pendingSetConf.setSuccessReply(null);
+      pendingSetConf = null;
+    }
+  }
+
+  void failSetConfiguration(RaftException e) {
+    Preconditions.checkState(pendingSetConf != null);
+    pendingSetConf.setException(e);
+    pendingSetConf = null;
+  }
+
+  TransactionContext getTransactionContext(long index) {
+    PendingRequest pendingRequest = pendingRequests.get(index);
+    // it is possible that the pendingRequest is null if this peer just becomes
+    // the new leader and commits transactions received by the previous leader
+    return pendingRequest != null ? pendingRequest.getEntry() : null;
+  }
+
+  void replyPendingRequest(long index, CompletableFuture<Message> messageFuture) {
+    final PendingRequest pending = pendingRequests.get(index);
+    if (pending != null) {
+      Preconditions.checkState(pending.getIndex() == index);
+
+      messageFuture.whenComplete((reply, exception) -> {
+        if (exception == null) {
+          pending.setSuccessReply(reply);
+        } else {
+          pending.setException(exception);
+        }
+      });
+    }
+  }
+
+  /**
+   * The leader state is stopped. Send NotLeaderException to all the pending
+   * requests since they have not got applied to the state machine yet.
+   */
+  void sendNotLeaderResponses() throws IOException {
+    LOG.info("{} sends responses before shutting down PendingRequestsHandler",
+        server.getId());
+
+    Collection<TransactionContext> pendingEntries = pendingRequests.values().stream()
+        .map(PendingRequest::getEntry).collect(Collectors.toList());
+    // notify the state machine about stepping down
+    server.getStateMachine().notifyNotLeader(pendingEntries);
+    pendingRequests.values().forEach(this::setNotLeaderException);
+    if (pendingSetConf != null) {
+      setNotLeaderException(pendingSetConf);
+    }
+  }
+
+  private void setNotLeaderException(PendingRequest pending) {
+    RaftClientReply reply = new RaftClientReply(pending.getRequest(),
+        server.generateNotLeaderException());
+    pending.setReply(reply);
+  }
+}

http://git-wip-us.apache.org/repos/asf/incubator-ratis/blob/7e71a2e0/ratis-server/src/main/java/org/apache/ratis/server/impl/RaftConfiguration.java
----------------------------------------------------------------------
diff --git a/ratis-server/src/main/java/org/apache/ratis/server/impl/RaftConfiguration.java b/ratis-server/src/main/java/org/apache/ratis/server/impl/RaftConfiguration.java
new file mode 100644
index 0000000..8fdd628
--- /dev/null
+++ b/ratis-server/src/main/java/org/apache/ratis/server/impl/RaftConfiguration.java
@@ -0,0 +1,262 @@
+/**
+ * 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.ratis.server.impl;
+
+import com.google.common.annotations.VisibleForTesting;
+import com.google.common.base.Preconditions;
+
+import java.util.*;
+import java.util.concurrent.ThreadLocalRandom;
+
+import org.apache.ratis.protocol.RaftPeer;
+
+/**
+ * The configuration of the raft cluster.
+ *
+ * The configuration is stable if there is no on-going peer change. Otherwise,
+ * the configuration is transitional, i.e. in the middle of a peer change.
+ *
+ * The objects of this class are immutable.
+ */
+public class RaftConfiguration {
+  /** Create a {@link Builder}. */
+  public static Builder newBuilder() {
+    return new Builder();
+  }
+
+  /** To build {@link RaftConfiguration} objects. */
+  public static class Builder {
+    private PeerConfiguration oldConf;
+    private PeerConfiguration conf;
+    private long logEntryIndex = RaftServerConstants.INVALID_LOG_INDEX;
+
+    private boolean forceStable = false;
+    private boolean forceTransitional = false;
+
+    private Builder() {}
+
+    public Builder setConf(PeerConfiguration conf) {
+      Preconditions.checkNotNull(conf);
+      Preconditions.checkState(this.conf == null, "conf is already set.");
+      this.conf = conf;
+      return this;
+    }
+
+    public Builder setConf(Iterable<RaftPeer> peers) {
+      return setConf(new PeerConfiguration(peers));
+    }
+
+    public Builder setConf(RaftPeer[] peers) {
+      return setConf(Arrays.asList(peers));
+    }
+
+    Builder setConf(RaftConfiguration transitionalConf) {
+      Preconditions.checkNotNull(transitionalConf);
+      Preconditions.checkState(transitionalConf.isTransitional());
+
+      Preconditions.checkState(!forceTransitional);
+      forceStable = true;
+      return setConf(transitionalConf.conf);
+    }
+
+
+    public Builder setOldConf(PeerConfiguration oldConf) {
+      Preconditions.checkNotNull(oldConf);
+      Preconditions.checkState(this.oldConf == null, "oldConf is already set.");
+      this.oldConf = oldConf;
+      return this;
+    }
+
+    public Builder setOldConf(Iterable<RaftPeer> oldPeers) {
+      return setOldConf(new PeerConfiguration(oldPeers));
+    }
+
+    public Builder setOldConf(RaftPeer[] oldPeers) {
+      return setOldConf(Arrays.asList(oldPeers));
+    }
+
+    Builder setOldConf(RaftConfiguration stableConf) {
+      Preconditions.checkNotNull(stableConf);
+      Preconditions.checkState(stableConf.isStable());
+
+      Preconditions.checkState(!forceStable);
+      forceTransitional = true;
+      return setOldConf(stableConf.conf);
+    }
+
+    public Builder setLogEntryIndex(long logEntryIndex) {
+      Preconditions.checkArgument(
+          logEntryIndex != RaftServerConstants.INVALID_LOG_INDEX);
+      Preconditions.checkState(
+          this.logEntryIndex == RaftServerConstants.INVALID_LOG_INDEX,
+          "logEntryIndex is already set.");
+      this.logEntryIndex = logEntryIndex;
+      return this;
+    }
+
+    /** Build a {@link RaftConfiguration}. */
+    public RaftConfiguration build() {
+      if (forceTransitional) {
+        Preconditions.checkState(oldConf != null);
+      }
+      if (forceStable) {
+        Preconditions.checkState(oldConf == null);
+      }
+      return new RaftConfiguration(conf, oldConf, logEntryIndex);
+    }
+  }
+
+  /** Non-null only if this configuration is transitional. */
+  private final PeerConfiguration oldConf;
+  /**
+   * The current peer configuration while this configuration is stable;
+   * or the new peer configuration while this configuration is transitional.
+   */
+  private final PeerConfiguration conf;
+
+  /** The index of the corresponding log entry for this configuration. */
+  private final long logEntryIndex;
+
+  private RaftConfiguration(PeerConfiguration conf, PeerConfiguration oldConf,
+      long logEntryIndex) {
+    Preconditions.checkNotNull(conf);
+    this.conf = conf;
+    this.oldConf = oldConf;
+    this.logEntryIndex = logEntryIndex;
+  }
+
+  /** Is this configuration transitional, i.e. in the middle of a peer change? */
+  boolean isTransitional() {
+    return oldConf != null;
+  }
+
+  /** Is this configuration stable, i.e. no on-going peer change? */
+  boolean isStable() {
+    return oldConf == null;
+  }
+
+  boolean containsInConf(String peerId) {
+    return conf.contains(peerId);
+  }
+
+  boolean containsInOldConf(String peerId) {
+    return oldConf != null && oldConf.contains(peerId);
+  }
+
+  boolean contains(String peerId) {
+    return containsInConf(peerId) && (oldConf == null || containsInOldConf(peerId));
+  }
+
+  /**
+   * @return the peer corresponding to the given id;
+   *         or return null if the peer is not in this configuration.
+   */
+  public RaftPeer getPeer(String id) {
+    if (id == null) {
+      return null;
+    }
+    RaftPeer peer = conf.getPeer(id);
+    if (peer != null) {
+      return peer;
+    } else if (oldConf != null) {
+      return oldConf.getPeer(id);
+    }
+    return null;
+  }
+
+  /** @return all the peers from the conf, and the old conf if it exists. */
+  public Collection<RaftPeer> getPeers() {
+    final Collection<RaftPeer> peers = new ArrayList<>(conf.getPeers());
+    if (oldConf != null) {
+      oldConf.getPeers().stream().filter(p -> !peers.contains(p))
+          .forEach(peers::add);
+    }
+    return peers;
+  }
+
+  /**
+   * @return all the peers other than the given self id from the conf,
+   *         and the old conf if it exists.
+   */
+  public Collection<RaftPeer> getOtherPeers(String selfId) {
+    Collection<RaftPeer> others = conf.getOtherPeers(selfId);
+    if (oldConf != null) {
+      oldConf.getOtherPeers(selfId).stream()
+          .filter(p -> !others.contains(p))
+          .forEach(others::add);
+    }
+    return others;
+  }
+
+  /** @return true if the self id together with the others are in the majority. */
+  boolean hasMajority(Collection<String> others, String selfId) {
+    Preconditions.checkArgument(!others.contains(selfId));
+    return conf.hasMajority(others, selfId) &&
+        (oldConf == null || oldConf.hasMajority(others, selfId));
+  }
+
+  @Override
+  public String toString() {
+    return conf + (oldConf != null ? "old:" + oldConf : "");
+  }
+
+  @VisibleForTesting
+  boolean hasNoChange(RaftPeer[] newMembers) {
+    if (!isStable() || conf.size() != newMembers.length) {
+      return false;
+    }
+    for (RaftPeer peer : newMembers) {
+      if (!conf.contains(peer.getId())) {
+        return false;
+      }
+    }
+    return true;
+  }
+
+  long getLogEntryIndex() {
+    return logEntryIndex;
+  }
+
+  static Collection<RaftPeer> computeNewPeers(RaftPeer[] newMembers,
+      RaftConfiguration old) {
+    List<RaftPeer> peers = new ArrayList<>();
+    for (RaftPeer p : newMembers) {
+      if (!old.containsInConf(p.getId())) {
+        peers.add(p);
+      }
+    }
+    return peers;
+  }
+
+  RaftPeer getRandomPeer(String exclusiveId) {
+    final List<RaftPeer> peers = conf.getOtherPeers(exclusiveId);
+    if (peers.isEmpty()) {
+      return null;
+    }
+    final int index = ThreadLocalRandom.current().nextInt(peers.size());
+    return peers.get(index);
+  }
+
+  Collection<RaftPeer> getPeersInOldConf() {
+    return oldConf != null ? oldConf.getPeers() : Collections.emptyList();
+  }
+
+  Collection<RaftPeer> getPeersInConf() {
+    return conf.getPeers();
+  }
+}

http://git-wip-us.apache.org/repos/asf/incubator-ratis/blob/7e71a2e0/ratis-server/src/main/java/org/apache/ratis/server/impl/RaftServerConstants.java
----------------------------------------------------------------------
diff --git a/ratis-server/src/main/java/org/apache/ratis/server/impl/RaftServerConstants.java b/ratis-server/src/main/java/org/apache/ratis/server/impl/RaftServerConstants.java
new file mode 100644
index 0000000..caf9c4d
--- /dev/null
+++ b/ratis-server/src/main/java/org/apache/ratis/server/impl/RaftServerConstants.java
@@ -0,0 +1,46 @@
+/**
+ * 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.ratis.server.impl;
+
+import org.apache.ratis.client.RaftClient;
+
+public interface RaftServerConstants {
+  long INVALID_LOG_INDEX = -1;
+  byte LOG_TERMINATE_BYTE = 0;
+  long DEFAULT_SEQNUM = RaftClient.DEFAULT_SEQNUM;
+
+  enum StartupOption {
+    FORMAT("format"),
+    REGULAR("regular");
+
+    private final String option;
+
+    StartupOption(String arg) {
+      this.option = arg;
+    }
+
+    public static StartupOption getOption(String arg) {
+      for (StartupOption s : StartupOption.values()) {
+        if (s.option.equals(arg)) {
+          return s;
+        }
+      }
+      return REGULAR;
+    }
+  }
+}


[48/54] [abbrv] incubator-ratis git commit: Renamed the packages from raft to ratis in preperation for Apache Incubation - Moved all java packages from org.apache.raft to org.apache.ratis. - Moved native package to org_apache_ratis, and native lib to l

Posted by ji...@apache.org.
http://git-wip-us.apache.org/repos/asf/incubator-ratis/blob/7e71a2e0/raft-common/src/main/java/org/apache/raft/conf/RaftProperties.java
----------------------------------------------------------------------
diff --git a/raft-common/src/main/java/org/apache/raft/conf/RaftProperties.java b/raft-common/src/main/java/org/apache/raft/conf/RaftProperties.java
deleted file mode 100644
index e482f88..0000000
--- a/raft-common/src/main/java/org/apache/raft/conf/RaftProperties.java
+++ /dev/null
@@ -1,1657 +0,0 @@
-/**
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- *     http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-
-package org.apache.raft.conf;
-
-import com.google.common.base.Preconditions;
-import org.apache.raft.util.StringUtils;
-import org.slf4j.Logger;
-import org.slf4j.LoggerFactory;
-import org.w3c.dom.*;
-import org.xml.sax.SAXException;
-
-import javax.xml.parsers.DocumentBuilder;
-import javax.xml.parsers.DocumentBuilderFactory;
-import javax.xml.parsers.ParserConfigurationException;
-import javax.xml.transform.Transformer;
-import javax.xml.transform.TransformerException;
-import javax.xml.transform.TransformerFactory;
-import javax.xml.transform.dom.DOMSource;
-import javax.xml.transform.stream.StreamResult;
-import java.io.*;
-import java.lang.ref.WeakReference;
-import java.net.JarURLConnection;
-import java.net.URL;
-import java.net.URLConnection;
-import java.util.*;
-import java.util.Map.Entry;
-import java.util.concurrent.ConcurrentHashMap;
-import java.util.concurrent.CopyOnWriteArrayList;
-import java.util.concurrent.TimeUnit;
-import java.util.regex.Matcher;
-import java.util.regex.Pattern;
-import java.util.regex.PatternSyntaxException;
-
-/** 
- * Provides access to configuration parameters. The current implementation is a
- * simplified version of hadoop's Configuration.
- */
-public class RaftProperties {
-  private static final Logger LOG = LoggerFactory.getLogger(RaftProperties.class);
-
-  private static class Resource {
-    private final Object resource;
-    private final String name;
-
-    public Resource(Object resource) {
-      this(resource, resource.toString());
-    }
-
-    public Resource(Object resource, String name) {
-      this.resource = resource;
-      this.name = name;
-    }
-
-    public String getName(){
-      return name;
-    }
-
-    public Object getResource() {
-      return resource;
-    }
-
-    @Override
-    public String toString() {
-      return name;
-    }
-  }
-
-  /**
-   * List of configuration resources.
-   */
-  private ArrayList<Resource> resources = new ArrayList<>();
-
-  /**
-   * The value reported as the setting resource when a key is set
-   * by code rather than a file resource by dumpConfiguration.
-   */
-  static final String UNKNOWN_RESOURCE = "Unknown";
-
-  /**
-   * List of configuration parameters marked <b>final</b>.
-   */
-  private Set<String> finalParameters = Collections.newSetFromMap(
-      new ConcurrentHashMap<String, Boolean>());
-
-  private boolean loadDefaults = true;
-
-  /**
-   * Configuration objects
-   */
-  private static final WeakHashMap<RaftProperties, Object> REGISTRY = new WeakHashMap<>();
-
-  /**
-   * List of default Resources. Resources are loaded in the order of the list
-   * entries
-   */
-  private static final CopyOnWriteArrayList<String> defaultResources =
-    new CopyOnWriteArrayList<>();
-
-  /**
-   * Stores the mapping of key to the resource which modifies or loads
-   * the key most recently
-   */
-  private Map<String, String[]> updatingResource;
-
-  private Properties properties;
-  private Properties overlay;
-  private ClassLoader classLoader;
-  {
-    classLoader = Thread.currentThread().getContextClassLoader();
-    if (classLoader == null) {
-      classLoader = RaftProperties.class.getClassLoader();
-    }
-  }
-
-  /** A new configuration. */
-  public RaftProperties() {
-    this(true);
-  }
-
-  /** A new configuration where the behavior of reading from the default
-   * resources can be turned off.
-   *
-   * If the parameter {@code loadDefaults} is false, the new instance
-   * will not load resources from the default files.
-   * @param loadDefaults specifies whether to load from the default files
-   */
-  public RaftProperties(boolean loadDefaults) {
-    this.loadDefaults = loadDefaults;
-    updatingResource = new ConcurrentHashMap<>();
-    synchronized(RaftProperties.class) {
-      REGISTRY.put(this, null);
-    }
-  }
-
-  /**
-   * A new RaftProperties with the same settings cloned from another.
-   *
-   * @param other the RaftProperties from which to clone settings.
-   */
-  @SuppressWarnings("unchecked")
-  public RaftProperties(RaftProperties other) {
-    this.resources = (ArrayList<Resource>) other.resources.clone();
-    synchronized(other) {
-      if (other.properties != null) {
-        this.properties = (Properties)other.properties.clone();
-      }
-
-      if (other.overlay!=null) {
-        this.overlay = (Properties)other.overlay.clone();
-      }
-
-      this.updatingResource = new ConcurrentHashMap<>(other.updatingResource);
-      this.finalParameters = Collections.newSetFromMap(
-          new ConcurrentHashMap<String, Boolean>());
-      this.finalParameters.addAll(other.finalParameters);
-    }
-
-    synchronized(RaftProperties.class) {
-      REGISTRY.put(this, null);
-    }
-    this.classLoader = other.classLoader;
-    this.loadDefaults = other.loadDefaults;
-  }
-
-  /**
-   * Add a default resource. Resources are loaded in the order of the resources
-   * added.
-   * @param name file name. File should be present in the classpath.
-   */
-  public static synchronized void addDefaultResource(String name) {
-    if(!defaultResources.contains(name)) {
-      defaultResources.add(name);
-      REGISTRY.keySet().stream().filter(conf -> conf.loadDefaults)
-          .forEach(RaftProperties::reloadConfiguration);
-    }
-  }
-
-  /**
-   * Add a configuration resource.
-   *
-   * The properties of this resource will override properties of previously
-   * added resources, unless they were marked <a href="#Final">final</a>.
-   *
-   * @param name resource to be added, the classpath is examined for a file
-   *             with that name.
-   */
-  public void addResource(String name) {
-    addResourceObject(new Resource(name));
-  }
-
-  /**
-   * Add a configuration resource.
-   *
-   * The properties of this resource will override properties of previously
-   * added resources, unless they were marked <a href="#Final">final</a>.
-   *
-   * WARNING: The contents of the InputStream will be cached, by this method.
-   * So use this sparingly because it does increase the memory consumption.
-   *
-   * @param in InputStream to deserialize the object from. In will be read from
-   * when a get or set is called next.  After it is read the stream will be
-   * closed.
-   */
-  public void addResource(InputStream in) {
-    addResourceObject(new Resource(in));
-  }
-
-  /**
-   * Add a configuration resource.
-   *
-   * The properties of this resource will override properties of previously
-   * added resources, unless they were marked <a href="#Final">final</a>.
-   *
-   * @param in InputStream to deserialize the object from.
-   * @param name the name of the resource because InputStream.toString is not
-   * very descriptive some times.
-   */
-  public void addResource(InputStream in, String name) {
-    addResourceObject(new Resource(in, name));
-  }
-
-  /**
-   * Add a configuration resource.
-   *
-   * The properties of this resource will override properties of previously
-   * added resources, unless they were marked <a href="#Final">final</a>.
-   *
-   * @param conf Configuration object from which to load properties
-   */
-  public void addResource(RaftProperties conf) {
-    addResourceObject(new Resource(conf.getProps()));
-  }
-
-
-
-  /**
-   * Reload configuration from previously added resources.
-   *
-   * This method will clear all the configuration read from the added
-   * resources, and final parameters. This will make the resources to
-   * be read again before accessing the values. Values that are added
-   * via set methods will overlay values read from the resources.
-   */
-  public synchronized void reloadConfiguration() {
-    properties = null;                            // trigger reload
-    finalParameters.clear();                      // clear site-limits
-  }
-
-  private synchronized void addResourceObject(Resource resource) {
-    resources.add(resource);                      // add to resources
-    reloadConfiguration();
-  }
-
-  private static final int MAX_SUBST = 20;
-
-  private static final int SUB_START_IDX = 0;
-  private static final int SUB_END_IDX = SUB_START_IDX + 1;
-
-  /**
-   * This is a manual implementation of the following regex
-   * "\\$\\{[^\\}\\$\u0020]+\\}".
-   *
-   * @param eval a string that may contain variables requiring expansion.
-   * @return a 2-element int array res such that
-   * eval.substring(res[0], res[1]) is "var" for the left-most occurrence of
-   * ${var} in eval. If no variable is found -1, -1 is returned.
-   */
-  private static int[] findSubVariable(String eval) {
-    int[] result = {-1, -1};
-
-    int matchStart;
-    int leftBrace;
-
-    // scanning for a brace first because it's less frequent than $
-    // that can occur in nested class names
-    //
-    match_loop:
-    for (matchStart = 1, leftBrace = eval.indexOf('{', matchStart);
-         // minimum left brace position (follows '$')
-         leftBrace > 0
-         // right brace of a smallest valid expression "${c}"
-         && leftBrace + "{c".length() < eval.length();
-         leftBrace = eval.indexOf('{', matchStart)) {
-      int matchedLen = 0;
-      if (eval.charAt(leftBrace - 1) == '$') {
-        int subStart = leftBrace + 1; // after '{'
-        for (int i = subStart; i < eval.length(); i++) {
-          switch (eval.charAt(i)) {
-            case '}':
-              if (matchedLen > 0) { // match
-                result[SUB_START_IDX] = subStart;
-                result[SUB_END_IDX] = subStart + matchedLen;
-                break match_loop;
-              }
-              // fall through to skip 1 char
-            case ' ':
-            case '$':
-              matchStart = i + 1;
-              continue match_loop;
-            default:
-              matchedLen++;
-          }
-        }
-        // scanned from "${"  to the end of eval, and no reset via ' ', '$':
-        //    no match!
-        break;
-      } else {
-        // not a start of a variable
-        //
-        matchStart = leftBrace + 1;
-      }
-    }
-    return result;
-  }
-
-  /**
-   * Attempts to repeatedly expand the value {@code expr} by replacing the
-   * left-most substring of the form "${var}" in the following precedence order
-   * <ol>
-   *   <li>by the value of the environment variable "var" if defined</li>
-   *   <li>by the value of the Java system property "var" if defined</li>
-   *   <li>by the value of the configuration key "var" if defined</li>
-   * </ol>
-   *
-   * If var is unbounded the current state of expansion "prefix${var}suffix" is
-   * returned.
-   *
-   * If a cycle is detected: replacing var1 requires replacing var2 ... requires
-   * replacing var1, i.e., the cycle is shorter than
-   * {@link RaftProperties#MAX_SUBST} then the original expr is returned.
-   *
-   * @param expr the literal value of a config key
-   * @return null if expr is null, otherwise the value resulting from expanding
-   * expr using the algorithm above.
-   * @throws IllegalArgumentException when more than
-   * {@link RaftProperties#MAX_SUBST} replacements are required
-   */
-  private String substituteVars(String expr) {
-    if (expr == null) {
-      return null;
-    }
-    String eval = expr;
-    Set<String> evalSet = null;
-    for(int s = 0; s < MAX_SUBST; s++) {
-      final int[] varBounds = findSubVariable(eval);
-      if (varBounds[SUB_START_IDX] == -1) {
-        return eval;
-      }
-      final String var = eval.substring(varBounds[SUB_START_IDX],
-          varBounds[SUB_END_IDX]);
-      String val = null;
-      try {
-        if (var.startsWith("env.") && 4 < var.length()) {
-          String v = var.substring(4);
-          int i = 0;
-          for (; i < v.length(); i++) {
-            char c = v.charAt(i);
-            if (c == ':' && i < v.length() - 1 && v.charAt(i + 1) == '-') {
-              val = getenv(v.substring(0, i));
-              if (val == null || val.length() == 0) {
-                val = v.substring(i + 2);
-              }
-              break;
-            } else if (c == '-') {
-              val = getenv(v.substring(0, i));
-              if (val == null) {
-                val = v.substring(i + 1);
-              }
-              break;
-            }
-          }
-          if (i == v.length()) {
-            val = getenv(v);
-          }
-        } else {
-          val = getProperty(var);
-        }
-      } catch(SecurityException se) {
-        LOG.warn("Unexpected SecurityException in Configuration", se);
-      }
-      if (val == null) {
-        val = getRaw(var);
-      }
-      if (val == null) {
-        return eval; // return literal ${var}: var is unbound
-      }
-
-      // prevent recursive resolution
-      //
-      final int dollar = varBounds[SUB_START_IDX] - "${".length();
-      final int afterRightBrace = varBounds[SUB_END_IDX] + "}".length();
-      final String refVar = eval.substring(dollar, afterRightBrace);
-      if (evalSet == null) {
-        evalSet = new HashSet<>();
-      }
-      if (!evalSet.add(refVar)) {
-        return expr; // return original expression if there is a loop
-      }
-
-      // substitute
-      eval = eval.substring(0, dollar)
-             + val
-             + eval.substring(afterRightBrace);
-    }
-    throw new IllegalStateException("Variable substitution depth too large: "
-                                    + MAX_SUBST + " " + expr);
-  }
-
-  String getenv(String name) {
-    return System.getenv(name);
-  }
-
-  String getProperty(String key) {
-    return System.getProperty(key);
-  }
-
-  /**
-   * Get the value of the <code>name</code> property, <code>null</code> if
-   * no such property exists. If the key is deprecated, it returns the value of
-   * the first key which replaces the deprecated key and is not null.
-   *
-   * Values are processed for <a href="#VariableExpansion">variable expansion</a>
-   * before being returned.
-   *
-   * @param name the property name, will be trimmed before get value.
-   * @return the value of the <code>name</code> or its replacing property,
-   *         or null if no such property exists.
-   */
-  public String get(String name) {
-    return substituteVars(getRaw(name));
-  }
-
-  /**
-   * Get the value of the <code>name</code> property as a trimmed <code>String</code>,
-   * <code>null</code> if no such property exists.
-   * If the key is deprecated, it returns the value of
-   * the first key which replaces the deprecated key and is not null
-   *
-   * Values are processed for <a href="#VariableExpansion">variable expansion</a>
-   * before being returned.
-   *
-   * @param name the property name.
-   * @return the value of the <code>name</code> or its replacing property,
-   *         or null if no such property exists.
-   */
-  public String getTrimmed(String name) {
-    String value = get(name);
-
-    if (null == value) {
-      return null;
-    } else {
-      return value.trim();
-    }
-  }
-
-  /**
-   * Get the value of the <code>name</code> property as a trimmed <code>String</code>,
-   * <code>defaultValue</code> if no such property exists.
-   * See @{Configuration#getTrimmed} for more details.
-   *
-   * @param name          the property name.
-   * @param defaultValue  the property default value.
-   * @return              the value of the <code>name</code> or defaultValue
-   *                      if it is not set.
-   */
-  public String getTrimmed(String name, String defaultValue) {
-    String ret = getTrimmed(name);
-    return ret == null ? defaultValue : ret;
-  }
-
-  /**
-   * Get the value of the <code>name</code> property, without doing
-   * <a href="#VariableExpansion">variable expansion</a>.If the key is
-   * deprecated, it returns the value of the first key which replaces
-   * the deprecated key and is not null.
-   *
-   * @param name the property name.
-   * @return the value of the <code>name</code> property or
-   *         its replacing property and null if no such property exists.
-   */
-  public String getRaw(String name) {
-    return getProps().getProperty(name.trim());
-  }
-
-  /**
-   * Set the <code>value</code> of the <code>name</code> property. If
-   * <code>name</code> is deprecated, it also sets the <code>value</code> to
-   * the keys that replace the deprecated key. Name will be trimmed before put
-   * into configuration.
-   *
-   * @param name property name.
-   * @param value property value.
-   * @throws IllegalArgumentException when the value or name is null.
-   */
-  public void set(String name, String value) {
-    Preconditions.checkArgument(name != null, "Property name must not be null");
-    Preconditions.checkArgument(value != null,
-        "The value of property " + name + " must not be null");
-    name = name.trim();
-    getProps();
-
-    getOverlay().setProperty(name, value);
-    getProps().setProperty(name, value);
-  }
-
-  /**
-   * Unset a previously set property.
-   */
-  public synchronized void unset(String name) {
-    getOverlay().remove(name);
-    getProps().remove(name);
-  }
-
-  /**
-   * Sets a property if it is currently unset.
-   * @param name the property name
-   * @param value the new value
-   */
-  public synchronized void setIfUnset(String name, String value) {
-    if (get(name) == null) {
-      set(name, value);
-    }
-  }
-
-  private synchronized Properties getOverlay() {
-    if (overlay == null){
-      overlay = new Properties();
-    }
-    return overlay;
-  }
-
-  /**
-   * Get the value of the <code>name</code>. If the key is deprecated,
-   * it returns the value of the first key which replaces the deprecated key
-   * and is not null.
-   * If no such property exists,
-   * then <code>defaultValue</code> is returned.
-   *
-   * @param name property name, will be trimmed before get value.
-   * @param defaultValue default value.
-   * @return property value, or <code>defaultValue</code> if the property
-   *         doesn't exist.
-   */
-  public String get(String name, String defaultValue) {
-    return substituteVars(getProps().getProperty(name, defaultValue));
-  }
-
-  /**
-   * Get the value of the <code>name</code> property as an <code>int</code>.
-   *
-   * If no such property exists, the provided default value is returned,
-   * or if the specified value is not a valid <code>int</code>,
-   * then an error is thrown.
-   *
-   * @param name property name.
-   * @param defaultValue default value.
-   * @throws NumberFormatException when the value is invalid
-   * @return property value as an <code>int</code>,
-   *         or <code>defaultValue</code>.
-   */
-  public int getInt(String name, int defaultValue) {
-    String valueString = getTrimmed(name);
-    if (valueString == null)
-      return defaultValue;
-    String hexString = getHexDigits(valueString);
-    if (hexString != null) {
-      return Integer.parseInt(hexString, 16);
-    }
-    return Integer.parseInt(valueString);
-  }
-
-  /**
-   * Get the value of the <code>name</code> property as a set of comma-delimited
-   * <code>int</code> values.
-   *
-   * If no such property exists, an empty array is returned.
-   *
-   * @param name property name
-   * @return property value interpreted as an array of comma-delimited
-   *         <code>int</code> values
-   */
-  public int[] getInts(String name) {
-    String[] strings = getTrimmedStrings(name);
-    int[] ints = new int[strings.length];
-    for (int i = 0; i < strings.length; i++) {
-      ints[i] = Integer.parseInt(strings[i]);
-    }
-    return ints;
-  }
-
-  /**
-   * Set the value of the <code>name</code> property to an <code>int</code>.
-   *
-   * @param name property name.
-   * @param value <code>int</code> value of the property.
-   */
-  public void setInt(String name, int value) {
-    set(name, Integer.toString(value));
-  }
-
-
-  /**
-   * Get the value of the <code>name</code> property as a <code>long</code>.
-   * If no such property exists, the provided default value is returned,
-   * or if the specified value is not a valid <code>long</code>,
-   * then an error is thrown.
-   *
-   * @param name property name.
-   * @param defaultValue default value.
-   * @throws NumberFormatException when the value is invalid
-   * @return property value as a <code>long</code>,
-   *         or <code>defaultValue</code>.
-   */
-  public long getLong(String name, long defaultValue) {
-    String valueString = getTrimmed(name);
-    if (valueString == null)
-      return defaultValue;
-    String hexString = getHexDigits(valueString);
-    if (hexString != null) {
-      return Long.parseLong(hexString, 16);
-    }
-    return Long.parseLong(valueString);
-  }
-
-  private String getHexDigits(String value) {
-    boolean negative = false;
-    String str = value;
-    String hexString;
-    if (value.startsWith("-")) {
-      negative = true;
-      str = value.substring(1);
-    }
-    if (str.startsWith("0x") || str.startsWith("0X")) {
-      hexString = str.substring(2);
-      if (negative) {
-        hexString = "-" + hexString;
-      }
-      return hexString;
-    }
-    return null;
-  }
-
-  /**
-   * Set the value of the <code>name</code> property to a <code>long</code>.
-   *
-   * @param name property name.
-   * @param value <code>long</code> value of the property.
-   */
-  public void setLong(String name, long value) {
-    set(name, Long.toString(value));
-  }
-
-  /**
-   * Get the value of the <code>name</code> property as a <code>float</code>.
-   * If no such property exists, the provided default value is returned,
-   * or if the specified value is not a valid <code>float</code>,
-   * then an error is thrown.
-   *
-   * @param name property name.
-   * @param defaultValue default value.
-   * @throws NumberFormatException when the value is invalid
-   * @return property value as a <code>float</code>,
-   *         or <code>defaultValue</code>.
-   */
-  public float getFloat(String name, float defaultValue) {
-    String valueString = getTrimmed(name);
-    if (valueString == null)
-      return defaultValue;
-    return Float.parseFloat(valueString);
-  }
-
-  /**
-   * Set the value of the <code>name</code> property to a <code>float</code>.
-   *
-   * @param name property name.
-   * @param value property value.
-   */
-  public void setFloat(String name, float value) {
-    set(name,Float.toString(value));
-  }
-
-  /**
-   * Get the value of the <code>name</code> property as a <code>double</code>.
-   * If no such property exists, the provided default value is returned,
-   * or if the specified value is not a valid <code>double</code>,
-   * then an error is thrown.
-   *
-   * @param name property name.
-   * @param defaultValue default value.
-   * @throws NumberFormatException when the value is invalid
-   * @return property value as a <code>double</code>,
-   *         or <code>defaultValue</code>.
-   */
-  public double getDouble(String name, double defaultValue) {
-    String valueString = getTrimmed(name);
-    if (valueString == null)
-      return defaultValue;
-    return Double.parseDouble(valueString);
-  }
-
-  /**
-   * Set the value of the <code>name</code> property to a <code>double</code>.
-   *
-   * @param name property name.
-   * @param value property value.
-   */
-  public void setDouble(String name, double value) {
-    set(name,Double.toString(value));
-  }
-
-  /**
-   * Get the value of the <code>name</code> property as a <code>boolean</code>.
-   * If no such property is specified, or if the specified value is not a valid
-   * <code>boolean</code>, then <code>defaultValue</code> is returned.
-   *
-   * @param name property name.
-   * @param defaultValue default value.
-   * @return property value as a <code>boolean</code>,
-   *         or <code>defaultValue</code>.
-   */
-  public boolean getBoolean(String name, boolean defaultValue) {
-    String valueString = getTrimmed(name);
-    return StringUtils.string2boolean(valueString, defaultValue);
-  }
-
-  /**
-   * Set the value of the <code>name</code> property to a <code>boolean</code>.
-   *
-   * @param name property name.
-   * @param value <code>boolean</code> value of the property.
-   */
-  public void setBoolean(String name, boolean value) {
-    set(name, Boolean.toString(value));
-  }
-
-  /**
-   * Set the given property, if it is currently unset.
-   * @param name property name
-   * @param value new value
-   */
-  public void setBooleanIfUnset(String name, boolean value) {
-    setIfUnset(name, Boolean.toString(value));
-  }
-
-  /**
-   * Set the value of the <code>name</code> property to the given type. This
-   * is equivalent to <code>set(&lt;name&gt;, value.toString())</code>.
-   * @param name property name
-   * @param value new value
-   */
-  public <T extends Enum<T>> void setEnum(String name, T value) {
-    set(name, value.toString());
-  }
-
-  /**
-   * Return value matching this enumerated type.
-   * Note that the returned value is trimmed by this method.
-   * @param name Property name
-   * @param defaultValue Value returned if no mapping exists
-   * @throws IllegalArgumentException If mapping is illegal for the type
-   * provided
-   */
-  public <T extends Enum<T>> T getEnum(String name, T defaultValue) {
-    final String val = getTrimmed(name);
-    return null == val
-      ? defaultValue
-      : Enum.valueOf(defaultValue.getDeclaringClass(), val);
-  }
-
-  enum ParsedTimeDuration {
-    NS {
-      TimeUnit unit() { return TimeUnit.NANOSECONDS; }
-      String suffix() { return "ns"; }
-    },
-    US {
-      TimeUnit unit() { return TimeUnit.MICROSECONDS; }
-      String suffix() { return "us"; }
-    },
-    MS {
-      TimeUnit unit() { return TimeUnit.MILLISECONDS; }
-      String suffix() { return "ms"; }
-    },
-    S {
-      TimeUnit unit() { return TimeUnit.SECONDS; }
-      String suffix() { return "s"; }
-    },
-    M {
-      TimeUnit unit() { return TimeUnit.MINUTES; }
-      String suffix() { return "m"; }
-    },
-    H {
-      TimeUnit unit() { return TimeUnit.HOURS; }
-      String suffix() { return "h"; }
-    },
-    D {
-      TimeUnit unit() { return TimeUnit.DAYS; }
-      String suffix() { return "d"; }
-    };
-    abstract TimeUnit unit();
-    abstract String suffix();
-    static ParsedTimeDuration unitFor(String s) {
-      for (ParsedTimeDuration ptd : values()) {
-        // iteration order is in decl order, so SECONDS matched last
-        if (s.endsWith(ptd.suffix())) {
-          return ptd;
-        }
-      }
-      return null;
-    }
-    static ParsedTimeDuration unitFor(TimeUnit unit) {
-      for (ParsedTimeDuration ptd : values()) {
-        if (ptd.unit() == unit) {
-          return ptd;
-        }
-      }
-      return null;
-    }
-  }
-
-  /**
-   * Set the value of <code>name</code> to the given time duration. This
-   * is equivalent to <code>set(&lt;name&gt;, value + &lt;time suffix&gt;)</code>.
-   * @param name Property name
-   * @param value Time duration
-   * @param unit Unit of time
-   */
-  public void setTimeDuration(String name, long value, TimeUnit unit) {
-    set(name, value + ParsedTimeDuration.unitFor(unit).suffix());
-  }
-
-  /**
-   * Return time duration in the given time unit. Valid units are encoded in
-   * properties as suffixes: nanoseconds (ns), microseconds (us), milliseconds
-   * (ms), seconds (s), minutes (m), hours (h), and days (d).
-   * @param name Property name
-   * @param defaultValue Value returned if no mapping exists.
-   * @param unit Unit to convert the stored property, if it exists.
-   * @throws NumberFormatException If the property stripped of its unit is not
-   *         a number
-   */
-  public long getTimeDuration(String name, long defaultValue, TimeUnit unit) {
-    String vStr = get(name);
-    if (null == vStr) {
-      return defaultValue;
-    }
-    vStr = vStr.trim();
-    return getTimeDurationHelper(name, vStr, unit);
-  }
-
-  private long getTimeDurationHelper(String name, String vStr, TimeUnit unit) {
-    ParsedTimeDuration vUnit = ParsedTimeDuration.unitFor(vStr);
-    if (null == vUnit) {
-      LOG.warn("No unit for " + name + "(" + vStr + ") assuming " + unit);
-      vUnit = ParsedTimeDuration.unitFor(unit);
-    } else {
-      vStr = vStr.substring(0, vStr.lastIndexOf(vUnit.suffix()));
-    }
-    return unit.convert(Long.parseLong(vStr), vUnit.unit());
-  }
-
-  public long[] getTimeDurations(String name, TimeUnit unit) {
-    String[] strings = getTrimmedStrings(name);
-    long[] durations = new long[strings.length];
-    for (int i = 0; i < strings.length; i++) {
-      durations[i] = getTimeDurationHelper(name, strings[i], unit);
-    }
-    return durations;
-  }
-
-  /**
-   * Get the value of the <code>name</code> property as a <code>Pattern</code>.
-   * If no such property is specified, or if the specified value is not a valid
-   * <code>Pattern</code>, then <code>DefaultValue</code> is returned.
-   * Note that the returned value is NOT trimmed by this method.
-   *
-   * @param name property name
-   * @param defaultValue default value
-   * @return property value as a compiled Pattern, or defaultValue
-   */
-  public Pattern getPattern(String name, Pattern defaultValue) {
-    String valString = get(name);
-    if (null == valString || valString.isEmpty()) {
-      return defaultValue;
-    }
-    try {
-      return Pattern.compile(valString);
-    } catch (PatternSyntaxException pse) {
-      LOG.warn("Regular expression '" + valString + "' for property '" +
-               name + "' not valid. Using default", pse);
-      return defaultValue;
-    }
-  }
-
-  /**
-   * Set the given property to <code>Pattern</code>.
-   * If the pattern is passed as null, sets the empty pattern which results in
-   * further calls to getPattern(...) returning the default value.
-   *
-   * @param name property name
-   * @param pattern new value
-   */
-  public void setPattern(String name, Pattern pattern) {
-    assert pattern != null : "Pattern cannot be null";
-    set(name, pattern.pattern());
-  }
-
-  /**
-   * A class that represents a set of positive integer ranges. It parses
-   * strings of the form: "2-3,5,7-" where ranges are separated by comma and
-   * the lower/upper bounds are separated by dash. Either the lower or upper
-   * bound may be omitted meaning all values up to or over. So the string
-   * above means 2, 3, 5, and 7, 8, 9, ...
-   */
-  public static class IntegerRanges implements Iterable<Integer>{
-    private static class Range {
-      int start;
-      int end;
-    }
-
-    private static class RangeNumberIterator implements Iterator<Integer> {
-      Iterator<Range> internal;
-      int at;
-      int end;
-
-      public RangeNumberIterator(List<Range> ranges) {
-        if (ranges != null) {
-          internal = ranges.iterator();
-        }
-        at = -1;
-        end = -2;
-      }
-
-      @Override
-      public boolean hasNext() {
-        if (at <= end) {
-          return true;
-        } else if (internal != null){
-          return internal.hasNext();
-        }
-        return false;
-      }
-
-      @Override
-      public Integer next() {
-        if (at <= end) {
-          at++;
-          return at - 1;
-        } else if (internal != null){
-          Range found = internal.next();
-          if (found != null) {
-            at = found.start;
-            end = found.end;
-            at++;
-            return at - 1;
-          }
-        }
-        return null;
-      }
-
-      @Override
-      public void remove() {
-        throw new UnsupportedOperationException();
-      }
-    }
-
-    List<Range> ranges = new ArrayList<>();
-
-    public IntegerRanges() {
-    }
-
-    public IntegerRanges(String newValue) {
-      StringTokenizer itr = new StringTokenizer(newValue, ",");
-      while (itr.hasMoreTokens()) {
-        String rng = itr.nextToken().trim();
-        String[] parts = rng.split("-", 3);
-        if (parts.length < 1 || parts.length > 2) {
-          throw new IllegalArgumentException("integer range badly formed: " +
-                                             rng);
-        }
-        Range r = new Range();
-        r.start = convertToInt(parts[0], 0);
-        if (parts.length == 2) {
-          r.end = convertToInt(parts[1], Integer.MAX_VALUE);
-        } else {
-          r.end = r.start;
-        }
-        if (r.start > r.end) {
-          throw new IllegalArgumentException("IntegerRange from " + r.start +
-                                             " to " + r.end + " is invalid");
-        }
-        ranges.add(r);
-      }
-    }
-
-    /**
-     * Convert a string to an int treating empty strings as the default value.
-     * @param value the string value
-     * @param defaultValue the value for if the string is empty
-     * @return the desired integer
-     */
-    private static int convertToInt(String value, int defaultValue) {
-      String trim = value.trim();
-      if (trim.length() == 0) {
-        return defaultValue;
-      }
-      return Integer.parseInt(trim);
-    }
-
-    /**
-     * Is the given value in the set of ranges
-     * @param value the value to check
-     * @return is the value in the ranges?
-     */
-    public boolean isIncluded(int value) {
-      for(Range r: ranges) {
-        if (r.start <= value && value <= r.end) {
-          return true;
-        }
-      }
-      return false;
-    }
-
-    /**
-     * @return true if there are no values in this range, else false.
-     */
-    public boolean isEmpty() {
-      return ranges == null || ranges.isEmpty();
-    }
-
-    @Override
-    public String toString() {
-      StringBuilder result = new StringBuilder();
-      boolean first = true;
-      for(Range r: ranges) {
-        if (first) {
-          first = false;
-        } else {
-          result.append(',');
-        }
-        result.append(r.start);
-        result.append('-');
-        result.append(r.end);
-      }
-      return result.toString();
-    }
-
-    @Override
-    public Iterator<Integer> iterator() {
-      return new RangeNumberIterator(ranges);
-    }
-
-  }
-
-  /**
-   * Parse the given attribute as a set of integer ranges
-   * @param name the attribute name
-   * @param defaultValue the default value if it is not set
-   * @return a new set of ranges from the configured value
-   */
-  public IntegerRanges getRange(String name, String defaultValue) {
-    return new IntegerRanges(get(name, defaultValue));
-  }
-
-  /**
-   * Get the comma delimited values of the <code>name</code> property as
-   * an array of <code>String</code>s, trimmed of the leading and trailing whitespace.
-   * If no such property is specified then an empty array is returned.
-   *
-   * @param name property name.
-   * @return property value as an array of trimmed <code>String</code>s,
-   *         or empty array.
-   */
-  public String[] getTrimmedStrings(String name) {
-    String valueString = get(name);
-    return StringUtils.getTrimmedStrings(valueString);
-  }
-
-  /**
-   * Load a class by name.
-   *
-   * @param name the class name.
-   * @return the class object.
-   * @throws ClassNotFoundException if the class is not found.
-   */
-  public Class<?> getClassByName(String name) throws ClassNotFoundException {
-    Class<?> ret = getClassByNameOrNull(name);
-    if (ret == null) {
-      throw new ClassNotFoundException("Class " + name + " not found");
-    }
-    return ret;
-  }
-
-  private static final Map<ClassLoader, Map<String, WeakReference<Class<?>>>>
-      CACHE_CLASSES = new WeakHashMap<>();
-
-  /**
-   * Sentinel value to store negative cache results in {@link #CACHE_CLASSES}.
-   */
-  private static final Class<?> NEGATIVE_CACHE_SENTINEL =
-      NegativeCacheSentinel.class;
-
-  /**
-   * Load a class by name, returning null rather than throwing an exception
-   * if it couldn't be loaded. This is to avoid the overhead of creating
-   * an exception.
-   *
-   * @param name the class name
-   * @return the class object, or null if it could not be found.
-   */
-  public Class<?> getClassByNameOrNull(String name) {
-    Map<String, WeakReference<Class<?>>> map;
-
-    synchronized (CACHE_CLASSES) {
-      map = CACHE_CLASSES.get(classLoader);
-      if (map == null) {
-        map = Collections.synchronizedMap(
-          new WeakHashMap<String, WeakReference<Class<?>>>());
-        CACHE_CLASSES.put(classLoader, map);
-      }
-    }
-
-    Class<?> clazz = null;
-    WeakReference<Class<?>> ref = map.get(name);
-    if (ref != null) {
-       clazz = ref.get();
-    }
-
-    if (clazz == null) {
-      try {
-        clazz = Class.forName(name, true, classLoader);
-      } catch (ClassNotFoundException e) {
-        // Leave a marker that the class isn't found
-        map.put(name, new WeakReference<>(NEGATIVE_CACHE_SENTINEL));
-        return null;
-      }
-      // two putters can race here, but they'll put the same class
-      map.put(name, new WeakReference<>(clazz));
-      return clazz;
-    } else if (clazz == NEGATIVE_CACHE_SENTINEL) {
-      return null; // not found
-    } else {
-      // cache hit
-      return clazz;
-    }
-  }
-
-  /**
-   * Get the value of the <code>name</code> property
-   * as an array of <code>Class</code>.
-   * The value of the property specifies a list of comma separated class names.
-   * If no such property is specified, then <code>defaultValue</code> is
-   * returned.
-   *
-   * @param name the property name.
-   * @param defaultValue default value.
-   * @return property value as a <code>Class[]</code>,
-   *         or <code>defaultValue</code>.
-   */
-  public Class<?>[] getClasses(String name, Class<?> ... defaultValue) {
-    String valueString = getRaw(name);
-    if (null == valueString) {
-      return defaultValue;
-    }
-    String[] classnames = getTrimmedStrings(name);
-    try {
-      Class<?>[] classes = new Class<?>[classnames.length];
-      for(int i = 0; i < classnames.length; i++) {
-        classes[i] = getClassByName(classnames[i]);
-      }
-      return classes;
-    } catch (ClassNotFoundException e) {
-      throw new RuntimeException(e);
-    }
-  }
-
-  /**
-   * Get the value of the <code>name</code> property as a <code>Class</code>.
-   * If no such property is specified, then <code>defaultValue</code> is
-   * returned.
-   *
-   * @param name the class name.
-   * @param defaultValue default value.
-   * @return property value as a <code>Class</code>,
-   *         or <code>defaultValue</code>.
-   */
-  public Class<?> getClass(String name, Class<?> defaultValue) {
-    String valueString = getTrimmed(name);
-    if (valueString == null)
-      return defaultValue;
-    try {
-      return getClassByName(valueString);
-    } catch (ClassNotFoundException e) {
-      throw new RuntimeException(e);
-    }
-  }
-
-  /**
-   * Get the value of the <code>name</code> property as a <code>Class</code>
-   * implementing the interface specified by <code>xface</code>.
-   *
-   * If no such property is specified, then <code>defaultValue</code> is
-   * returned.
-   *
-   * An exception is thrown if the returned class does not implement the named
-   * interface.
-   *
-   * @param name the class name.
-   * @param defaultValue default value.
-   * @param xface the interface implemented by the named class.
-   * @return property value as a <code>Class</code>,
-   *         or <code>defaultValue</code>.
-   */
-  public <U> Class<? extends U> getClass(String name,
-                                         Class<? extends U> defaultValue,
-                                         Class<U> xface) {
-    try {
-      Class<?> theClass = getClass(name, defaultValue);
-      if (theClass != null && !xface.isAssignableFrom(theClass))
-        throw new RuntimeException(theClass+" not "+xface.getName());
-      else if (theClass != null)
-        return theClass.asSubclass(xface);
-      else
-        return null;
-    } catch (Exception e) {
-      throw new RuntimeException(e);
-    }
-  }
-
-  /**
-   * Set the value of the <code>name</code> property to the name of a
-   * <code>theClass</code> implementing the given interface <code>xface</code>.
-   *
-   * An exception is thrown if <code>theClass</code> does not implement the
-   * interface <code>xface</code>.
-   *
-   * @param name property name.
-   * @param theClass property value.
-   * @param xface the interface implemented by the named class.
-   */
-  public void setClass(String name, Class<?> theClass, Class<?> xface) {
-    if (!xface.isAssignableFrom(theClass))
-      throw new RuntimeException(theClass+" not "+xface.getName());
-    set(name, theClass.getName());
-  }
-
-  /**
-   * Get the {@link URL} for the named resource.
-   *
-   * @param name resource name.
-   * @return the url for the named resource.
-   */
-  public URL getResource(String name) {
-    return classLoader.getResource(name);
-  }
-
-  protected synchronized Properties getProps() {
-    if (properties == null) {
-      properties = new Properties();
-      Map<String, String[]> backup =
-          new ConcurrentHashMap<>(updatingResource);
-      loadResources(properties, resources);
-
-      if (overlay != null) {
-        properties.putAll(overlay);
-        for (Entry<Object,Object> item: overlay.entrySet()) {
-          String key = (String) item.getKey();
-          String[] source = backup.get(key);
-          if(source != null) {
-            updatingResource.put(key, source);
-          }
-        }
-      }
-    }
-    return properties;
-  }
-
-  /**
-   * Return the number of keys in the configuration.
-   *
-   * @return number of keys in the configuration.
-   */
-  public int size() {
-    return getProps().size();
-  }
-
-  /**
-   * Clears all keys from the configuration.
-   */
-  public void clear() {
-    getProps().clear();
-    getOverlay().clear();
-  }
-
-  private Document parse(DocumentBuilder builder, URL url)
-      throws IOException, SAXException {
-    LOG.debug("parsing URL " + url);
-    if (url == null) {
-      return null;
-    }
-
-    URLConnection connection = url.openConnection();
-    if (connection instanceof JarURLConnection) {
-      // Disable caching for JarURLConnection to avoid sharing JarFile
-      // with other users.
-      connection.setUseCaches(false);
-    }
-    return parse(builder, connection.getInputStream(), url.toString());
-  }
-
-  private Document parse(DocumentBuilder builder, InputStream is,
-      String systemId) throws IOException, SAXException {
-    LOG.debug("parsing input stream " + is);
-    if (is == null) {
-      return null;
-    }
-    try {
-      return (systemId == null) ? builder.parse(is) : builder.parse(is,
-          systemId);
-    } finally {
-      is.close();
-    }
-  }
-
-  private void loadResources(Properties properties,
-                             ArrayList<Resource> resources) {
-    if(loadDefaults) {
-      for (String resource : defaultResources) {
-        loadResource(properties, new Resource(resource));
-      }
-    }
-
-    for (int i = 0; i < resources.size(); i++) {
-      Resource ret = loadResource(properties, resources.get(i));
-      if (ret != null) {
-        resources.set(i, ret);
-      }
-    }
-  }
-
-  private Resource loadResource(Properties properties, Resource wrapper) {
-    String name = UNKNOWN_RESOURCE;
-    try {
-      Object resource = wrapper.getResource();
-      name = wrapper.getName();
-
-      DocumentBuilderFactory docBuilderFactory
-        = DocumentBuilderFactory.newInstance();
-      //ignore all comments inside the xml file
-      docBuilderFactory.setIgnoringComments(true);
-
-      //allow includes in the xml file
-      docBuilderFactory.setNamespaceAware(true);
-      try {
-          docBuilderFactory.setXIncludeAware(true);
-      } catch (UnsupportedOperationException e) {
-        LOG.error("Failed to set setXIncludeAware(true) for parser "
-                + docBuilderFactory
-                + ":" + e,
-                e);
-      }
-      DocumentBuilder builder = docBuilderFactory.newDocumentBuilder();
-      Document doc = null;
-      Element root = null;
-      boolean returnCachedProperties = false;
-
-      if (resource instanceof URL) { // an URL resource
-        doc = parse(builder, (URL) resource);
-      } else if (resource instanceof String) { // a CLASSPATH resource
-        URL url = getResource((String) resource);
-        doc = parse(builder, url);
-      } else if (resource instanceof InputStream) {
-        doc = parse(builder, (InputStream) resource, null);
-        returnCachedProperties = true;
-      } else if (resource instanceof Properties) {
-        overlay(properties, (Properties) resource);
-      } else if (resource instanceof Element) {
-        root = (Element) resource;
-      }
-
-      if (root == null) {
-        if (doc == null) {
-          return null;
-        }
-        root = doc.getDocumentElement();
-      }
-      Properties toAddTo = properties;
-      if(returnCachedProperties) {
-        toAddTo = new Properties();
-      }
-      if (!"configuration".equals(root.getTagName())) {
-        LOG.error("bad conf file: top-level element not <configuration>");
-      }
-      NodeList props = root.getChildNodes();
-      for (int i = 0; i < props.getLength(); i++) {
-        Node propNode = props.item(i);
-        if (!(propNode instanceof Element))
-          continue;
-        Element prop = (Element)propNode;
-        if ("configuration".equals(prop.getTagName())) {
-          loadResource(toAddTo, new Resource(prop, name));
-          continue;
-        }
-        if (!"property".equals(prop.getTagName()))
-          LOG.warn("bad conf file: element not <property>");
-
-        String attr = null;
-        String value = null;
-        boolean finalParameter = false;
-        LinkedList<String> source = new LinkedList<>();
-
-        Attr propAttr = prop.getAttributeNode("name");
-        if (propAttr != null)
-          attr = StringUtils.weakIntern(propAttr.getValue());
-        propAttr = prop.getAttributeNode("value");
-        if (propAttr != null)
-          value = StringUtils.weakIntern(propAttr.getValue());
-        propAttr = prop.getAttributeNode("final");
-        if (propAttr != null)
-          finalParameter = "true".equals(propAttr.getValue());
-        propAttr = prop.getAttributeNode("source");
-        if (propAttr != null)
-          source.add(StringUtils.weakIntern(propAttr.getValue()));
-
-        NodeList fields = prop.getChildNodes();
-        for (int j = 0; j < fields.getLength(); j++) {
-          Node fieldNode = fields.item(j);
-          if (!(fieldNode instanceof Element))
-            continue;
-          Element field = (Element)fieldNode;
-          if ("name".equals(field.getTagName()) && field.hasChildNodes())
-            attr = StringUtils.weakIntern(
-                ((Text)field.getFirstChild()).getData().trim());
-          if ("value".equals(field.getTagName()) && field.hasChildNodes())
-            value = StringUtils.weakIntern(
-                ((Text)field.getFirstChild()).getData());
-          if ("final".equals(field.getTagName()) && field.hasChildNodes())
-            finalParameter = "true".equals(((Text)field.getFirstChild()).getData());
-          if ("source".equals(field.getTagName()) && field.hasChildNodes())
-            source.add(StringUtils.weakIntern(
-                ((Text)field.getFirstChild()).getData()));
-        }
-        source.add(name);
-
-        // Ignore this parameter if it has already been marked as 'final'
-        if (attr != null) {
-          loadProperty(toAddTo, name, attr, value, finalParameter,
-              source.toArray(new String[source.size()]));
-        }
-      }
-
-      if (returnCachedProperties) {
-        overlay(properties, toAddTo);
-        return new Resource(toAddTo, name);
-      }
-      return null;
-    } catch (IOException | DOMException | SAXException |
-        ParserConfigurationException e) {
-      LOG.error("error parsing conf " + name, e);
-      throw new RuntimeException(e);
-    }
-  }
-
-  private void overlay(Properties to, Properties from) {
-    for (Entry<Object, Object> entry: from.entrySet()) {
-      to.put(entry.getKey(), entry.getValue());
-    }
-  }
-
-  private void loadProperty(Properties properties, String name, String attr,
-      String value, boolean finalParameter, String[] source) {
-    if (value != null) {
-      if (!finalParameters.contains(attr)) {
-        properties.setProperty(attr, value);
-        if(source != null) {
-          updatingResource.put(attr, source);
-        }
-      } else if (!value.equals(properties.getProperty(attr))) {
-        LOG.warn(name+":an attempt to override final parameter: "+attr
-            +";  Ignoring.");
-      }
-    }
-    if (finalParameter && attr != null) {
-      finalParameters.add(attr);
-    }
-  }
-
-  /**
-   * Write out the non-default properties in this configuration to the given
-   * {@link OutputStream} using UTF-8 encoding.
-   *
-   * @param out the output stream to write to.
-   */
-  public void writeXml(OutputStream out) throws IOException {
-    writeXml(new OutputStreamWriter(out, "UTF-8"));
-  }
-
-  /**
-   * Write out the non-default properties in this configuration to the given
-   * {@link Writer}.
-   *
-   * @param out the writer to write to.
-   */
-  public void writeXml(Writer out) throws IOException {
-    Document doc = asXmlDocument();
-
-    try {
-      DOMSource source = new DOMSource(doc);
-      StreamResult result = new StreamResult(out);
-      TransformerFactory transFactory = TransformerFactory.newInstance();
-      Transformer transformer = transFactory.newTransformer();
-
-      // Important to not hold Configuration log while writing result, since
-      // 'out' may be an HDFS stream which needs to lock this configuration
-      // from another thread.
-      transformer.transform(source, result);
-    } catch (TransformerException te) {
-      throw new IOException(te);
-    }
-  }
-
-  /**
-   * Return the XML DOM corresponding to this Configuration.
-   */
-  private synchronized Document asXmlDocument() throws IOException {
-    Document doc;
-    try {
-      doc =
-        DocumentBuilderFactory.newInstance().newDocumentBuilder().newDocument();
-    } catch (ParserConfigurationException pe) {
-      throw new IOException(pe);
-    }
-    Element conf = doc.createElement("configuration");
-    doc.appendChild(conf);
-    conf.appendChild(doc.createTextNode("\n"));
-    for (Enumeration<Object> e = properties.keys(); e.hasMoreElements();) {
-      String name = (String)e.nextElement();
-      Object object = properties.get(name);
-      String value;
-      if (object instanceof String) {
-        value = (String) object;
-      }else {
-        continue;
-      }
-      Element propNode = doc.createElement("property");
-      conf.appendChild(propNode);
-
-      Element nameNode = doc.createElement("name");
-      nameNode.appendChild(doc.createTextNode(name));
-      propNode.appendChild(nameNode);
-
-      Element valueNode = doc.createElement("value");
-      valueNode.appendChild(doc.createTextNode(value));
-      propNode.appendChild(valueNode);
-
-      if (updatingResource != null) {
-        String[] sources = updatingResource.get(name);
-        if(sources != null) {
-          for(String s : sources) {
-            Element sourceNode = doc.createElement("source");
-            sourceNode.appendChild(doc.createTextNode(s));
-            propNode.appendChild(sourceNode);
-          }
-        }
-      }
-
-      conf.appendChild(doc.createTextNode("\n"));
-    }
-    return doc;
-  }
-
-  /**
-   * Get the {@link ClassLoader} for this job.
-   *
-   * @return the correct class loader.
-   */
-  public ClassLoader getClassLoader() {
-    return classLoader;
-  }
-
-  /**
-   * Set the class loader that will be used to load the various objects.
-   *
-   * @param classLoader the new class loader.
-   */
-  public void setClassLoader(ClassLoader classLoader) {
-    this.classLoader = classLoader;
-  }
-
-  @Override
-  public String toString() {
-    StringBuilder sb = new StringBuilder();
-    sb.append("Configuration: ");
-    if(loadDefaults) {
-      toString(defaultResources, sb);
-      if(resources.size()>0) {
-        sb.append(", ");
-      }
-    }
-    toString(resources, sb);
-    return sb.toString();
-  }
-
-  private <T> void toString(List<T> resources, StringBuilder sb) {
-    ListIterator<T> i = resources.listIterator();
-    while (i.hasNext()) {
-      if (i.nextIndex() != 0) {
-        sb.append(", ");
-      }
-      sb.append(i.next());
-    }
-  }
-
-  /**
-   * get keys matching the the regex
-   * @return a map with matching keys
-   */
-  public Map<String,String> getValByRegex(String regex) {
-    Pattern p = Pattern.compile(regex);
-
-    Map<String,String> result = new HashMap<>();
-    Matcher m;
-
-    for(Entry<Object,Object> item: getProps().entrySet()) {
-      if (item.getKey() instanceof String &&
-          item.getValue() instanceof String) {
-        m = p.matcher((String)item.getKey());
-        if(m.find()) { // match
-          result.put((String) item.getKey(),
-              substituteVars(getProps().getProperty((String) item.getKey())));
-        }
-      }
-    }
-    return result;
-  }
-
-  /**
-   * A unique class which is used as a sentinel value in the caching
-   * for getClassByName. {@link RaftProperties#getClassByNameOrNull(String)}
-   */
-  private static abstract class NegativeCacheSentinel {}
-}

http://git-wip-us.apache.org/repos/asf/incubator-ratis/blob/7e71a2e0/raft-common/src/main/java/org/apache/raft/io/MD5Hash.java
----------------------------------------------------------------------
diff --git a/raft-common/src/main/java/org/apache/raft/io/MD5Hash.java b/raft-common/src/main/java/org/apache/raft/io/MD5Hash.java
deleted file mode 100644
index 8e95be9..0000000
--- a/raft-common/src/main/java/org/apache/raft/io/MD5Hash.java
+++ /dev/null
@@ -1,205 +0,0 @@
-/**
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- *     http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-
-package org.apache.raft.io;
-
-import java.io.IOException;
-import java.io.DataInput;
-import java.io.DataOutput;
-import java.io.InputStream;
-import java.security.MessageDigest;
-import java.security.NoSuchAlgorithmException;
-import java.util.Arrays;
-
-public class MD5Hash {
-  public static final int MD5_LEN = 16;
-
-  private static final ThreadLocal<MessageDigest> DIGESTER_FACTORY =
-      ThreadLocal.withInitial(() -> {
-    try {
-      return MessageDigest.getInstance("MD5");
-    } catch (NoSuchAlgorithmException e) {
-      throw new RuntimeException(e);
-    }
-  });
-
-  private byte[] digest;
-
-  /** Constructs an MD5Hash. */
-  public MD5Hash() {
-    this.digest = new byte[MD5_LEN];
-  }
-
-  /** Constructs an MD5Hash from a hex string. */
-  public MD5Hash(String hex) {
-    setDigest(hex);
-  }
-
-  /** Constructs an MD5Hash with a specified value. */
-  public MD5Hash(byte[] digest) {
-    if (digest.length != MD5_LEN)
-      throw new IllegalArgumentException("Wrong length: " + digest.length);
-    this.digest = digest;
-  }
-
-  public void readFields(DataInput in) throws IOException {
-    in.readFully(digest);
-  }
-
-  /** Constructs, reads and returns an instance. */
-  public static MD5Hash read(DataInput in) throws IOException {
-    MD5Hash result = new MD5Hash();
-    result.readFields(in);
-    return result;
-  }
-
-  public void write(DataOutput out) throws IOException {
-    out.write(digest);
-  }
-
-  /** Copy the contents of another instance into this instance. */
-  public void set(MD5Hash that) {
-    System.arraycopy(that.digest, 0, this.digest, 0, MD5_LEN);
-  }
-
-  /** Returns the digest bytes. */
-  public byte[] getDigest() { return digest; }
-
-  /** Construct a hash value for a byte array. */
-  public static MD5Hash digest(byte[] data) {
-    return digest(data, 0, data.length);
-  }
-
-  /**
-   * Create a thread local MD5 digester
-   */
-  public static MessageDigest getDigester() {
-    MessageDigest digester = DIGESTER_FACTORY.get();
-    digester.reset();
-    return digester;
-  }
-
-  /** Construct a hash value for the content from the InputStream. */
-  public static MD5Hash digest(InputStream in) throws IOException {
-    final byte[] buffer = new byte[4*1024];
-
-    final MessageDigest digester = getDigester();
-    for(int n; (n = in.read(buffer)) != -1; ) {
-      digester.update(buffer, 0, n);
-    }
-
-    return new MD5Hash(digester.digest());
-  }
-
-  /** Construct a hash value for a byte array. */
-  public static MD5Hash digest(byte[] data, int start, int len) {
-    byte[] digest;
-    MessageDigest digester = getDigester();
-    digester.update(data, start, len);
-    digest = digester.digest();
-    return new MD5Hash(digest);
-  }
-
-  /** Construct a hash value for an array of byte array. */
-  public static MD5Hash digest(byte[][] dataArr, int start, int len) {
-    byte[] digest;
-    MessageDigest digester = getDigester();
-    for (byte[] data : dataArr) {
-      digester.update(data, start, len);
-    }
-    digest = digester.digest();
-    return new MD5Hash(digest);
-  }
-
-  /** Construct a half-sized version of this MD5.  Fits in a long **/
-  public long halfDigest() {
-    long value = 0;
-    for (int i = 0; i < 8; i++)
-      value |= ((digest[i] & 0xffL) << (8*(7-i)));
-    return value;
-  }
-
-  /**
-   * Return a 32-bit digest of the MD5.
-   * @return the first 4 bytes of the md5
-   */
-  public int quarterDigest() {
-    int value = 0;
-    for (int i = 0; i < 4; i++)
-      value |= ((digest[i] & 0xff) << (8*(3-i)));
-    return value;
-  }
-
-  /** Returns true iff <code>o</code> is an MD5Hash whose digest contains the
-   * same values.  */
-  @Override
-  public boolean equals(Object o) {
-    if (!(o instanceof MD5Hash))
-      return false;
-    MD5Hash other = (MD5Hash)o;
-    return Arrays.equals(this.digest, other.digest);
-  }
-
-  /** Returns a hash code value for this object.
-   * Only uses the first 4 bytes, since md5s are evenly distributed.
-   */
-  @Override
-  public int hashCode() {
-    return quarterDigest();
-  }
-
-  private static final char[] HEX_DIGITS =
-      {'0','1','2','3','4','5','6','7','8','9','a','b','c','d','e','f'};
-
-  /** Returns a string representation of this object. */
-  @Override
-  public String toString() {
-    StringBuilder buf = new StringBuilder(MD5_LEN*2);
-    for (int i = 0; i < MD5_LEN; i++) {
-      int b = digest[i];
-      buf.append(HEX_DIGITS[(b >> 4) & 0xf]);
-      buf.append(HEX_DIGITS[b & 0xf]);
-    }
-    return buf.toString();
-  }
-
-  /** Sets the digest value from a hex string. */
-  public void setDigest(String hex) {
-    if (hex.length() != MD5_LEN*2)
-      throw new IllegalArgumentException("Wrong length: " + hex.length());
-    byte[] digest = new byte[MD5_LEN];
-    for (int i = 0; i < MD5_LEN; i++) {
-      int j = i << 1;
-      digest[i] = (byte)(charToNibble(hex.charAt(j)) << 4 |
-          charToNibble(hex.charAt(j+1)));
-    }
-    this.digest = digest;
-  }
-
-  private static int charToNibble(char c) {
-    if (c >= '0' && c <= '9') {
-      return c - '0';
-    } else if (c >= 'a' && c <= 'f') {
-      return 0xa + (c - 'a');
-    } else if (c >= 'A' && c <= 'F') {
-      return 0xA + (c - 'A');
-    } else {
-      throw new RuntimeException("Not a hex character: " + c);
-    }
-  }
-}

http://git-wip-us.apache.org/repos/asf/incubator-ratis/blob/7e71a2e0/raft-common/src/main/java/org/apache/raft/io/nativeio/Errno.java
----------------------------------------------------------------------
diff --git a/raft-common/src/main/java/org/apache/raft/io/nativeio/Errno.java b/raft-common/src/main/java/org/apache/raft/io/nativeio/Errno.java
deleted file mode 100644
index e38595b..0000000
--- a/raft-common/src/main/java/org/apache/raft/io/nativeio/Errno.java
+++ /dev/null
@@ -1,64 +0,0 @@
-/**
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- *     http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-package org.apache.raft.io.nativeio;
-
-/**
- * Enum representing POSIX errno values.
- */
-public enum Errno {
-  EPERM,
-  ENOENT,
-  ESRCH,
-  EINTR,
-  EIO,
-  ENXIO,
-  E2BIG,
-  ENOEXEC,
-  EBADF,
-  ECHILD,
-  EAGAIN,
-  ENOMEM,
-  EACCES,
-  EFAULT,
-  ENOTBLK,
-  EBUSY,
-  EEXIST,
-  EXDEV,
-  ENODEV,
-  ENOTDIR,
-  EISDIR,
-  EINVAL,
-  ENFILE,
-  EMFILE,
-  ENOTTY,
-  ETXTBSY,
-  EFBIG,
-  ENOSPC,
-  ESPIPE,
-  EROFS,
-  EMLINK,
-  EPIPE,
-  EDOM,
-  ERANGE,
-  ELOOP,
-  ENAMETOOLONG,
-  ENOTEMPTY,
-  EOVERFLOW,
-
-  UNKNOWN
-}


[37/54] [abbrv] incubator-ratis git commit: Renamed the packages from raft to ratis in preperation for Apache Incubation - Moved all java packages from org.apache.raft to org.apache.ratis. - Moved native package to org_apache_ratis, and native lib to l

Posted by ji...@apache.org.
http://git-wip-us.apache.org/repos/asf/incubator-ratis/blob/7e71a2e0/raft-server/src/main/java/org/apache/raft/server/impl/RaftServerImpl.java
----------------------------------------------------------------------
diff --git a/raft-server/src/main/java/org/apache/raft/server/impl/RaftServerImpl.java b/raft-server/src/main/java/org/apache/raft/server/impl/RaftServerImpl.java
deleted file mode 100644
index 4b8c442..0000000
--- a/raft-server/src/main/java/org/apache/raft/server/impl/RaftServerImpl.java
+++ /dev/null
@@ -1,813 +0,0 @@
-/**
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- *     http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-package org.apache.raft.server.impl;
-
-import com.google.common.annotations.VisibleForTesting;
-import com.google.common.base.Preconditions;
-import org.apache.raft.conf.RaftProperties;
-import org.apache.raft.protocol.*;
-import org.apache.raft.server.RaftServer;
-import org.apache.raft.server.RaftServerConfigKeys;
-import org.apache.raft.server.RaftServerRpc;
-import org.apache.raft.server.protocol.TermIndex;
-import org.apache.raft.server.storage.FileInfo;
-import org.apache.raft.shaded.proto.RaftProtos.*;
-import org.apache.raft.statemachine.SnapshotInfo;
-import org.apache.raft.statemachine.StateMachine;
-import org.apache.raft.statemachine.TransactionContext;
-import org.apache.raft.util.CodeInjectionForTesting;
-import org.apache.raft.util.LifeCycle;
-import org.apache.raft.util.ProtoUtils;
-import org.apache.raft.util.RaftUtils;
-import org.slf4j.Logger;
-import org.slf4j.LoggerFactory;
-
-import java.io.IOException;
-import java.io.InterruptedIOException;
-import java.util.Arrays;
-import java.util.Collection;
-import java.util.List;
-import java.util.OptionalLong;
-import java.util.concurrent.CompletableFuture;
-import java.util.concurrent.ExecutionException;
-
-import static org.apache.raft.shaded.proto.RaftProtos.AppendEntriesReplyProto.AppendResult.*;
-import static org.apache.raft.util.LifeCycle.State.*;
-
-public class RaftServerImpl implements RaftServer {
-  public static final Logger LOG = LoggerFactory.getLogger(RaftServerImpl.class);
-
-  private static final String CLASS_NAME = RaftServerImpl.class.getSimpleName();
-  static final String REQUEST_VOTE = CLASS_NAME + ".requestVote";
-  static final String APPEND_ENTRIES = CLASS_NAME + ".appendEntries";
-  static final String INSTALL_SNAPSHOT = CLASS_NAME + ".installSnapshot";
-
-
-  /** Role of raft peer */
-  enum Role {
-    LEADER, CANDIDATE, FOLLOWER
-  }
-
-  private final int minTimeoutMs;
-  private final int maxTimeoutMs;
-
-  private final LifeCycle lifeCycle;
-  private final ServerState state;
-  private final StateMachine stateMachine;
-  private final RaftProperties properties;
-  private volatile Role role;
-
-  /** used when the peer is follower, to monitor election timeout */
-  private volatile FollowerState heartbeatMonitor;
-
-  /** used when the peer is candidate, to request votes from other peers */
-  private volatile LeaderElection electionDaemon;
-
-  /** used when the peer is leader */
-  private volatile LeaderState leaderState;
-
-  private RaftServerRpc serverRpc;
-
-  private final LogAppenderFactory appenderFactory;
-
-  public RaftServerImpl(String id, RaftConfiguration raftConf,
-                        RaftProperties properties, StateMachine stateMachine) throws IOException {
-    this.lifeCycle = new LifeCycle(id);
-    minTimeoutMs = properties.getInt(
-        RaftServerConfigKeys.RAFT_SERVER_RPC_TIMEOUT_MIN_MS_KEY,
-        RaftServerConfigKeys.RAFT_SERVER_RPC_TIMEOUT_MIN_MS_DEFAULT);
-    maxTimeoutMs = properties.getInt(
-        RaftServerConfigKeys.RAFT_SERVER_RPC_TIMEOUT_MAX_MS_KEY,
-        RaftServerConfigKeys.RAFT_SERVER_RPC_TIMEOUT_MAX_MS_DEFAULT);
-    Preconditions.checkArgument(maxTimeoutMs > minTimeoutMs,
-        "max timeout: %s, min timeout: %s", maxTimeoutMs, minTimeoutMs);
-    this.properties = properties;
-    this.stateMachine = stateMachine;
-    this.state = new ServerState(id, raftConf, properties, this, stateMachine);
-    appenderFactory = initAppenderFactory();
-  }
-
-  int getMinTimeoutMs() {
-    return minTimeoutMs;
-  }
-
-  int getMaxTimeoutMs() {
-    return maxTimeoutMs;
-  }
-
-  int getRandomTimeoutMs() {
-    return RaftUtils.getRandomBetween(minTimeoutMs, maxTimeoutMs);
-  }
-
-  @Override
-  public StateMachine getStateMachine() {
-    return this.stateMachine;
-  }
-
-  public LogAppenderFactory getLogAppenderFactory() {
-    return appenderFactory;
-  }
-
-  private LogAppenderFactory initAppenderFactory() {
-    Class<? extends LogAppenderFactory> factoryClass = properties.getClass(
-        RaftServerConfigKeys.RAFT_SERVER_LOG_APPENDER_FACTORY_CLASS_KEY,
-        RaftServerConfigKeys.RAFT_SERVER_LOG_APPENDER_FACTORY_CLASS_DEFAULT,
-        LogAppenderFactory.class);
-    return RaftUtils.newInstance(factoryClass);
-  }
-
-  /**
-   * Used by tests to set initial raft configuration with correct port bindings.
-   */
-  @VisibleForTesting
-  public void setInitialConf(RaftConfiguration conf) {
-    this.state.setInitialConf(conf);
-  }
-
-  @Override
-  public void setServerRpc(RaftServerRpc serverRpc) {
-    this.serverRpc = serverRpc;
-    // add peers into rpc service
-    RaftConfiguration conf = getRaftConf();
-    if (conf != null) {
-      serverRpc.addPeers(conf.getPeers());
-    }
-  }
-
-  public RaftServerRpc getServerRpc() {
-    return serverRpc;
-  }
-
-  @Override
-  public void start() {
-    lifeCycle.transition(STARTING);
-    state.start();
-    RaftConfiguration conf = getRaftConf();
-    if (conf != null && conf.contains(getId())) {
-      LOG.debug("{} starts as a follower", getId());
-      startAsFollower();
-    } else {
-      LOG.debug("{} starts with initializing state", getId());
-      startInitializing();
-    }
-  }
-
-  /**
-   * The peer belongs to the current configuration, should start as a follower
-   */
-  private void startAsFollower() {
-    role = Role.FOLLOWER;
-    heartbeatMonitor = new FollowerState(this);
-    heartbeatMonitor.start();
-
-    serverRpc.start();
-    lifeCycle.transition(RUNNING);
-  }
-
-  /**
-   * The peer does not have any configuration (maybe it will later be included
-   * in some configuration). Start still as a follower but will not vote or
-   * start election.
-   */
-  private void startInitializing() {
-    role = Role.FOLLOWER;
-    // do not start heartbeatMonitoring
-    serverRpc.start();
-  }
-
-  public ServerState getState() {
-    return this.state;
-  }
-
-  @Override
-  public String getId() {
-    return getState().getSelfId();
-  }
-
-  RaftConfiguration getRaftConf() {
-    return getState().getRaftConf();
-  }
-
-  @Override
-  public void close() {
-    lifeCycle.checkStateAndClose(() -> {
-      try {
-        shutdownHeartbeatMonitor();
-        shutdownElectionDaemon();
-        shutdownLeaderState();
-
-        serverRpc.close();
-        state.close();
-      } catch (Exception ignored) {
-        LOG.warn("Failed to kill " + state.getSelfId(), ignored);
-      }
-    });
-  }
-
-  @VisibleForTesting
-  public boolean isAlive() {
-    return !lifeCycle.getCurrentState().isOneOf(CLOSING, CLOSED);
-  }
-
-  public boolean isFollower() {
-    return role == Role.FOLLOWER;
-  }
-
-  public boolean isCandidate() {
-    return role == Role.CANDIDATE;
-  }
-
-  public boolean isLeader() {
-    return role == Role.LEADER;
-  }
-
-  /**
-   * Change the server state to Follower if necessary
-   * @param newTerm The new term.
-   * @param sync We will call {@link ServerState#persistMetadata()} if this is
-   *             set to true and term/votedFor get updated.
-   * @return if the term/votedFor should be updated to the new term
-   * @throws IOException if term/votedFor persistence failed.
-   */
-  synchronized boolean changeToFollower(long newTerm, boolean sync)
-      throws IOException {
-    final Role old = role;
-    role = Role.FOLLOWER;
-
-    boolean metadataUpdated = false;
-    if (newTerm > state.getCurrentTerm()) {
-      state.setCurrentTerm(newTerm);
-      state.resetLeaderAndVotedFor();
-      metadataUpdated = true;
-    }
-
-    if (old == Role.LEADER) {
-      assert leaderState != null;
-      shutdownLeaderState();
-    } else if (old == Role.CANDIDATE) {
-      shutdownElectionDaemon();
-    }
-
-    if (old != Role.FOLLOWER) {
-      heartbeatMonitor = new FollowerState(this);
-      heartbeatMonitor.start();
-    }
-
-    if (metadataUpdated && sync) {
-      state.persistMetadata();
-    }
-    return metadataUpdated;
-  }
-
-  private synchronized void shutdownLeaderState() {
-    final LeaderState leader = leaderState;
-    if (leader != null) {
-      leader.stop();
-    }
-    leaderState = null;
-    // TODO: make sure that StateMachineUpdater has applied all transactions that have context
-  }
-
-  private void shutdownElectionDaemon() {
-    final LeaderElection election = electionDaemon;
-    if (election != null) {
-      election.stopRunning();
-      // no need to interrupt the election thread
-    }
-    electionDaemon = null;
-  }
-
-  synchronized void changeToLeader() {
-    Preconditions.checkState(isCandidate());
-    shutdownElectionDaemon();
-    role = Role.LEADER;
-    state.becomeLeader();
-    // start sending AppendEntries RPC to followers
-    leaderState = new LeaderState(this, properties);
-    leaderState.start();
-  }
-
-  private void shutdownHeartbeatMonitor() {
-    final FollowerState hm = heartbeatMonitor;
-    if (hm != null) {
-      hm.stopRunning();
-      hm.interrupt();
-    }
-    heartbeatMonitor = null;
-  }
-
-  synchronized void changeToCandidate() {
-    Preconditions.checkState(isFollower());
-    shutdownHeartbeatMonitor();
-    role = Role.CANDIDATE;
-    // start election
-    electionDaemon = new LeaderElection(this);
-    electionDaemon.start();
-  }
-
-  @Override
-  public String toString() {
-    return role + " " + state + " " + lifeCycle.getCurrentState();
-  }
-
-  /**
-   * @return null if the server is in leader state.
-   */
-  private CompletableFuture<RaftClientReply> checkLeaderState(
-      RaftClientRequest request) {
-    if (!isLeader()) {
-      NotLeaderException exception = generateNotLeaderException();
-      CompletableFuture<RaftClientReply> future = new CompletableFuture<>();
-      future.complete(new RaftClientReply(request, exception));
-      return future;
-    }
-    return null;
-  }
-
-  NotLeaderException generateNotLeaderException() {
-    if (lifeCycle.getCurrentState() != RUNNING) {
-      return new NotLeaderException(getId(), null, null);
-    }
-    String leaderId = state.getLeaderId();
-    if (leaderId == null || leaderId.equals(state.getSelfId())) {
-      // No idea about who is the current leader. Or the peer is the current
-      // leader, but it is about to step down
-      RaftPeer suggestedLeader = state.getRaftConf()
-          .getRandomPeer(state.getSelfId());
-      leaderId = suggestedLeader == null ? null : suggestedLeader.getId();
-    }
-    RaftConfiguration conf = getRaftConf();
-    Collection<RaftPeer> peers = conf.getPeers();
-    return new NotLeaderException(getId(), conf.getPeer(leaderId),
-        peers.toArray(new RaftPeer[peers.size()]));
-  }
-
-  /**
-   * Handle a normal update request from client.
-   */
-  private CompletableFuture<RaftClientReply> appendTransaction(
-      RaftClientRequest request, TransactionContext entry)
-      throws RaftException {
-    LOG.debug("{}: receive client request({})", getId(), request);
-    lifeCycle.assertCurrentState(RUNNING);
-    CompletableFuture<RaftClientReply> reply;
-
-    final PendingRequest pending;
-    synchronized (this) {
-      reply = checkLeaderState(request);
-      if (reply != null) {
-        return reply;
-      }
-
-      // append the message to its local log
-      final long entryIndex;
-      try {
-        entryIndex = state.applyLog(entry);
-      } catch (IOException e) {
-        throw new RaftException(e);
-      }
-
-      // put the request into the pending queue
-      pending = leaderState.addPendingRequest(entryIndex, request, entry);
-      leaderState.notifySenders();
-    }
-    return pending.getFuture();
-  }
-
-  @Override
-  public CompletableFuture<RaftClientReply> submitClientRequestAsync(
-      RaftClientRequest request) throws IOException {
-    // first check the server's leader state
-    CompletableFuture<RaftClientReply> reply = checkLeaderState(request);
-    if (reply != null) {
-      return reply;
-    }
-
-    // let the state machine handle read-only request from client
-    if (request.isReadOnly()) {
-      // TODO: We might not be the leader anymore by the time this completes. See the RAFT paper,
-      // section 8 (last part)
-      return stateMachine.query(request);
-    }
-
-    // TODO: this client request will not be added to pending requests
-    // until later which means that any failure in between will leave partial state in the
-    // state machine. We should call cancelTransaction() for failed requests
-    TransactionContext entry = stateMachine.startTransaction(request);
-    if (entry.getException().isPresent()) {
-      throw RaftUtils.asIOException(entry.getException().get());
-    }
-
-    return appendTransaction(request, entry);
-  }
-
-  @Override
-  public RaftClientReply submitClientRequest(RaftClientRequest request)
-      throws IOException {
-    return waitForReply(getId(), request, submitClientRequestAsync(request));
-  }
-
-  private static RaftClientReply waitForReply(String id, RaftClientRequest request,
-      CompletableFuture<RaftClientReply> future) throws IOException {
-    try {
-      return future.get();
-    } catch (InterruptedException e) {
-      final String s = id + ": Interrupted when waiting for reply, request=" + request;
-      LOG.info(s, e);
-      throw RaftUtils.toInterruptedIOException(s, e);
-    } catch (ExecutionException e) {
-      final Throwable cause = e.getCause();
-      if (cause == null) {
-        throw new IOException(e);
-      }
-      if (cause instanceof NotLeaderException) {
-        return new RaftClientReply(request, (NotLeaderException)cause);
-      } else {
-        throw RaftUtils.asIOException(cause);
-      }
-    }
-  }
-
-  @Override
-  public RaftClientReply setConfiguration(SetConfigurationRequest request)
-      throws IOException {
-    return waitForReply(getId(), request, setConfigurationAsync(request));
-  }
-
-  /**
-   * Handle a raft configuration change request from client.
-   */
-  @Override
-  public CompletableFuture<RaftClientReply> setConfigurationAsync(
-      SetConfigurationRequest request) throws IOException {
-    LOG.debug("{}: receive setConfiguration({})", getId(), request);
-    lifeCycle.assertCurrentState(RUNNING);
-    CompletableFuture<RaftClientReply> reply = checkLeaderState(request);
-    if (reply != null) {
-      return reply;
-    }
-
-    final RaftPeer[] peersInNewConf = request.getPeersInNewConf();
-    final PendingRequest pending;
-    synchronized (this) {
-      reply = checkLeaderState(request);
-      if (reply != null) {
-        return reply;
-      }
-
-      final RaftConfiguration current = getRaftConf();
-      // make sure there is no other raft reconfiguration in progress
-      if (!current.isStable() || leaderState.inStagingState() ||
-          !state.isCurrentConfCommitted()) {
-        throw new ReconfigurationInProgressException(
-            "Reconfiguration is already in progress: " + current);
-      }
-
-      // return true if the new configuration is the same with the current one
-      if (current.hasNoChange(peersInNewConf)) {
-        pending = leaderState.returnNoConfChange(request);
-        return pending.getFuture();
-      }
-
-      // add new peers into the rpc service
-      getServerRpc().addPeers(Arrays.asList(peersInNewConf));
-      // add staging state into the leaderState
-      pending = leaderState.startSetConfiguration(request);
-    }
-    return pending.getFuture();
-  }
-
-  private boolean shouldWithholdVotes() {
-    return isLeader() || (isFollower() && state.hasLeader()
-        && heartbeatMonitor.shouldWithholdVotes());
-  }
-
-  /**
-   * check if the remote peer is not included in the current conf
-   * and should shutdown. should shutdown if all the following stands:
-   * 1. this is a leader
-   * 2. current conf is stable and has been committed
-   * 3. candidate id is not included in conf
-   * 4. candidate's last entry's index < conf's index
-   */
-  private boolean shouldSendShutdown(String candidateId,
-      TermIndex candidateLastEntry) {
-    return isLeader()
-        && getRaftConf().isStable()
-        && getState().isConfCommitted()
-        && !getRaftConf().containsInConf(candidateId)
-        && candidateLastEntry.getIndex() < getRaftConf().getLogEntryIndex()
-        && !leaderState.isBootStrappingPeer(candidateId);
-  }
-
-  @Override
-  public RequestVoteReplyProto requestVote(RequestVoteRequestProto r)
-      throws IOException {
-    final String candidateId = r.getServerRequest().getRequestorId();
-    return requestVote(candidateId, r.getCandidateTerm(),
-        ServerProtoUtils.toTermIndex(r.getCandidateLastEntry()));
-  }
-
-  private RequestVoteReplyProto requestVote(String candidateId,
-      long candidateTerm, TermIndex candidateLastEntry) throws IOException {
-    CodeInjectionForTesting.execute(REQUEST_VOTE, getId(),
-        candidateId, candidateTerm, candidateLastEntry);
-    LOG.debug("{}: receive requestVote({}, {}, {})",
-        getId(), candidateId, candidateTerm, candidateLastEntry);
-    lifeCycle.assertCurrentState(RUNNING);
-
-    boolean voteGranted = false;
-    boolean shouldShutdown = false;
-    final RequestVoteReplyProto reply;
-    synchronized (this) {
-      if (shouldWithholdVotes()) {
-        LOG.info("{} Withhold vote from server {} with term {}. " +
-            "This server:{}, last rpc time from leader {} is {}", getId(),
-            candidateId, candidateTerm, this, this.getState().getLeaderId(),
-            (isFollower() ? heartbeatMonitor.getLastRpcTime() : -1));
-      } else if (state.recognizeCandidate(candidateId, candidateTerm)) {
-        boolean termUpdated = changeToFollower(candidateTerm, false);
-        // see Section 5.4.1 Election restriction
-        if (state.isLogUpToDate(candidateLastEntry)) {
-          heartbeatMonitor.updateLastRpcTime(false);
-          state.grantVote(candidateId);
-          voteGranted = true;
-        }
-        if (termUpdated || voteGranted) {
-          state.persistMetadata(); // sync metafile
-        }
-      }
-      if (!voteGranted && shouldSendShutdown(candidateId, candidateLastEntry)) {
-        shouldShutdown = true;
-      }
-      reply = ServerProtoUtils.toRequestVoteReplyProto(candidateId, getId(),
-          voteGranted, state.getCurrentTerm(), shouldShutdown);
-      if (LOG.isDebugEnabled()) {
-        LOG.debug("{} replies to vote request: {}. Peer's state: {}",
-            getId(), ProtoUtils.toString(reply), state);
-      }
-    }
-    return reply;
-  }
-
-  private void validateEntries(long expectedTerm, TermIndex previous,
-      LogEntryProto... entries) {
-    if (entries != null && entries.length > 0) {
-      final long index0 = entries[0].getIndex();
-
-      if (previous == null || previous.getTerm() == 0) {
-        Preconditions.checkArgument(index0 == 0,
-            "Unexpected Index: previous is null but entries[%s].getIndex()=%s",
-            0, index0);
-      } else {
-        Preconditions.checkArgument(previous.getIndex() == index0 - 1,
-            "Unexpected Index: previous is %s but entries[%s].getIndex()=%s",
-            previous, 0, index0);
-      }
-
-      for (int i = 0; i < entries.length; i++) {
-        final long t = entries[i].getTerm();
-        Preconditions.checkArgument(expectedTerm >= t,
-            "Unexpected Term: entries[%s].getTerm()=%s but expectedTerm=%s",
-            i, t, expectedTerm);
-
-        final long indexi = entries[i].getIndex();
-        Preconditions.checkArgument(indexi == index0 + i,
-            "Unexpected Index: entries[%s].getIndex()=%s but entries[0].getIndex()=%s",
-            i, indexi, index0);
-      }
-    }
-  }
-
-  @Override
-  public AppendEntriesReplyProto appendEntries(AppendEntriesRequestProto r)
-      throws IOException {
-    // TODO avoid converting list to array
-    final LogEntryProto[] entries = r.getEntriesList()
-        .toArray(new LogEntryProto[r.getEntriesCount()]);
-    final TermIndex previous = r.hasPreviousLog() ?
-        ServerProtoUtils.toTermIndex(r.getPreviousLog()) : null;
-    return appendEntries(r.getServerRequest().getRequestorId(),
-        r.getLeaderTerm(), previous, r.getLeaderCommit(), r.getInitializing(),
-        entries);
-  }
-
-  private AppendEntriesReplyProto appendEntries(String leaderId, long leaderTerm,
-      TermIndex previous, long leaderCommit, boolean initializing,
-      LogEntryProto... entries) throws IOException {
-    CodeInjectionForTesting.execute(APPEND_ENTRIES, getId(),
-        leaderId, leaderTerm, previous, leaderCommit, initializing, entries);
-    if (LOG.isDebugEnabled()) {
-      LOG.debug("{}: receive appendEntries({}, {}, {}, {}, {}, {})", getId(),
-          leaderId, leaderTerm, previous, leaderCommit, initializing,
-          ServerProtoUtils.toString(entries));
-    }
-    lifeCycle.assertCurrentState(STARTING, RUNNING);
-
-    try {
-      validateEntries(leaderTerm, previous, entries);
-    } catch (IllegalArgumentException e) {
-      throw new IOException(e);
-    }
-
-    final long currentTerm;
-    long nextIndex = state.getLog().getNextIndex();
-    synchronized (this) {
-      final boolean recognized = state.recognizeLeader(leaderId, leaderTerm);
-      currentTerm = state.getCurrentTerm();
-      if (!recognized) {
-        final AppendEntriesReplyProto reply = ServerProtoUtils.toAppendEntriesReplyProto(
-            leaderId, getId(), currentTerm, nextIndex, NOT_LEADER);
-        if (LOG.isDebugEnabled()) {
-          LOG.debug("{}: do not recognize leader. Reply: {}",
-              getId(), ProtoUtils.toString(reply));
-        }
-        return reply;
-      }
-      changeToFollower(leaderTerm, true);
-      state.setLeader(leaderId);
-
-      if (!initializing && lifeCycle.compareAndTransition(STARTING, RUNNING)) {
-        heartbeatMonitor = new FollowerState(this);
-        heartbeatMonitor.start();
-      }
-      if (lifeCycle.getCurrentState() == RUNNING) {
-        heartbeatMonitor.updateLastRpcTime(true);
-      }
-
-      // We need to check if "previous" is in the local peer. Note that it is
-      // possible that "previous" is covered by the latest snapshot: e.g.,
-      // it's possible there's no log entries outside of the latest snapshot.
-      // However, it is not possible that "previous" index is smaller than the
-      // last index included in snapshot. This is because indices <= snapshot's
-      // last index should have been committed.
-      if (previous != null && !containPrevious(previous)) {
-        final AppendEntriesReplyProto reply =
-            ServerProtoUtils.toAppendEntriesReplyProto(leaderId, getId(),
-                currentTerm, Math.min(nextIndex, previous.getIndex()), INCONSISTENCY);
-        LOG.debug("{}: inconsistency entries. Leader previous:{}, Reply:{}",
-            getId(), previous, ServerProtoUtils.toString(reply));
-        return reply;
-      }
-
-      state.getLog().append(entries);
-      state.updateConfiguration(entries);
-      state.updateStatemachine(leaderCommit, currentTerm);
-    }
-    if (entries != null && entries.length > 0) {
-      try {
-        state.getLog().logSync();
-      } catch (InterruptedException e) {
-        throw new InterruptedIOException("logSync got interrupted");
-      }
-      nextIndex = entries[entries.length - 1].getIndex() + 1;
-    }
-    synchronized (this) {
-      if (lifeCycle.getCurrentState() == RUNNING && isFollower()
-          && getState().getCurrentTerm() == currentTerm) {
-        // reset election timer to avoid punishing the leader for our own
-        // long disk writes
-        heartbeatMonitor.updateLastRpcTime(false);
-      }
-    }
-    final AppendEntriesReplyProto reply = ServerProtoUtils.toAppendEntriesReplyProto(
-        leaderId, getId(), currentTerm, nextIndex, SUCCESS);
-    LOG.debug("{}: succeeded to handle AppendEntries. Reply: {}", getId(),
-        ServerProtoUtils.toString(reply));
-    return reply;
-  }
-
-  private boolean containPrevious(TermIndex previous) {
-    LOG.debug("{}: prev:{}, latestSnapshot:{}, getLatestInstalledSnapshot:{}",
-        getId(), previous, state.getLatestSnapshot(), state.getLatestInstalledSnapshot());
-    return state.getLog().contains(previous)
-        ||  (state.getLatestSnapshot() != null
-             && state.getLatestSnapshot().getTermIndex().equals(previous))
-        || (state.getLatestInstalledSnapshot() != null)
-             && state.getLatestInstalledSnapshot().equals(previous);
-  }
-
-  @Override
-  public InstallSnapshotReplyProto installSnapshot(
-      InstallSnapshotRequestProto request) throws IOException {
-    final String leaderId = request.getServerRequest().getRequestorId();
-    CodeInjectionForTesting.execute(INSTALL_SNAPSHOT, getId(), leaderId, request);
-    LOG.debug("{}: receive installSnapshot({})", getId(), request);
-
-    lifeCycle.assertCurrentState(STARTING, RUNNING);
-
-    final long currentTerm;
-    final long leaderTerm = request.getLeaderTerm();
-    final TermIndex lastTermIndex = ServerProtoUtils.toTermIndex(
-        request.getTermIndex());
-    final long lastIncludedIndex = lastTermIndex.getIndex();
-    synchronized (this) {
-      final boolean recognized = state.recognizeLeader(leaderId, leaderTerm);
-      currentTerm = state.getCurrentTerm();
-      if (!recognized) {
-        final InstallSnapshotReplyProto reply = ServerProtoUtils
-            .toInstallSnapshotReplyProto(leaderId, getId(), currentTerm,
-                request.getRequestIndex(), InstallSnapshotResult.NOT_LEADER);
-        LOG.debug("{}: do not recognize leader for installing snapshot." +
-            " Reply: {}", getId(), reply);
-        return reply;
-      }
-      changeToFollower(leaderTerm, true);
-      state.setLeader(leaderId);
-
-      if (lifeCycle.getCurrentState() == RUNNING) {
-        heartbeatMonitor.updateLastRpcTime(true);
-      }
-
-      // Check and append the snapshot chunk. We simply put this in lock
-      // considering a follower peer requiring a snapshot installation does not
-      // have a lot of requests
-      Preconditions.checkState(
-          state.getLog().getNextIndex() <= lastIncludedIndex,
-          "%s log's next id is %s, last included index in snapshot is %s",
-          getId(),  state.getLog().getNextIndex(), lastIncludedIndex);
-
-      //TODO: We should only update State with installed snapshot once the request is done.
-      state.installSnapshot(request);
-
-      // update the committed index
-      // re-load the state machine if this is the last chunk
-      if (request.getDone()) {
-        state.reloadStateMachine(lastIncludedIndex, leaderTerm);
-      }
-      if (lifeCycle.getCurrentState() == RUNNING) {
-        heartbeatMonitor.updateLastRpcTime(false);
-      }
-    }
-    if (request.getDone()) {
-      LOG.info("{}: successfully install the whole snapshot-{}", getId(),
-          lastIncludedIndex);
-    }
-    return ServerProtoUtils.toInstallSnapshotReplyProto(leaderId, getId(),
-        currentTerm, request.getRequestIndex(), InstallSnapshotResult.SUCCESS);
-  }
-
-  AppendEntriesRequestProto createAppendEntriesRequest(long leaderTerm,
-      String targetId, TermIndex previous, List<LogEntryProto> entries,
-      boolean initializing) {
-    return ServerProtoUtils.toAppendEntriesRequestProto(getId(), targetId,
-        leaderTerm, entries, state.getLog().getLastCommittedIndex(),
-        initializing, previous);
-  }
-
-  synchronized InstallSnapshotRequestProto createInstallSnapshotRequest(
-      String targetId, String requestId, int requestIndex, SnapshotInfo snapshot,
-      List<FileChunkProto> chunks, boolean done) {
-    OptionalLong totalSize = snapshot.getFiles().stream()
-        .mapToLong(FileInfo::getFileSize).reduce(Long::sum);
-    assert totalSize.isPresent();
-    return ServerProtoUtils.toInstallSnapshotRequestProto(getId(), targetId,
-        requestId, requestIndex, state.getCurrentTerm(), snapshot.getTermIndex(),
-        chunks, totalSize.getAsLong(), done);
-  }
-
-  synchronized RequestVoteRequestProto createRequestVoteRequest(String targetId,
-      long term, TermIndex lastEntry) {
-    return ServerProtoUtils.toRequestVoteRequestProto(getId(), targetId, term,
-        lastEntry);
-  }
-
-  public synchronized void submitLocalSyncEvent() {
-    if (isLeader() && leaderState != null) {
-      leaderState.submitUpdateStateEvent(LeaderState.UPDATE_COMMIT_EVENT);
-    }
-  }
-
-  synchronized void replyPendingRequest(long logIndex,
-      CompletableFuture<Message> message) {
-    if (isLeader() && leaderState != null) { // is leader and is running
-      leaderState.replyPendingRequest(logIndex, message);
-    }
-  }
-
-  TransactionContext getTransactionContext(long index) {
-    if (leaderState != null) { // is leader and is running
-      return leaderState.getTransactionContext(index);
-    }
-    return null;
-  }
-
-  public RaftProperties getProperties() {
-    return this.properties;
-  }
-}

http://git-wip-us.apache.org/repos/asf/incubator-ratis/blob/7e71a2e0/raft-server/src/main/java/org/apache/raft/server/impl/ServerProtoUtils.java
----------------------------------------------------------------------
diff --git a/raft-server/src/main/java/org/apache/raft/server/impl/ServerProtoUtils.java b/raft-server/src/main/java/org/apache/raft/server/impl/ServerProtoUtils.java
deleted file mode 100644
index e30b979..0000000
--- a/raft-server/src/main/java/org/apache/raft/server/impl/ServerProtoUtils.java
+++ /dev/null
@@ -1,179 +0,0 @@
-/**
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- *     http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-package org.apache.raft.server.impl;
-
-import org.apache.raft.client.impl.ClientProtoUtils;
-import org.apache.raft.server.protocol.TermIndex;
-import org.apache.raft.shaded.proto.RaftProtos.*;
-import org.apache.raft.util.ProtoUtils;
-
-import java.util.Arrays;
-import java.util.List;
-import java.util.stream.Collectors;
-
-import static org.apache.raft.server.impl.RaftServerConstants.DEFAULT_SEQNUM;
-import static org.apache.raft.shaded.proto.RaftProtos.AppendEntriesReplyProto.AppendResult.SUCCESS;
-
-/** Server proto utilities for internal use. */
-public class ServerProtoUtils {
-  public static TermIndex toTermIndex(TermIndexProto p) {
-    return p == null? null: TermIndex.newTermIndex(p.getTerm(), p.getIndex());
-  }
-
-  public static TermIndexProto toTermIndexProto(TermIndex ti) {
-    return ti == null? null: TermIndexProto.newBuilder()
-        .setTerm(ti.getTerm())
-        .setIndex(ti.getIndex())
-        .build();
-  }
-
-  public static TermIndex toTermIndex(LogEntryProto entry) {
-    return entry == null ? null :
-        TermIndex.newTermIndex(entry.getTerm(), entry.getIndex());
-  }
-
-  public static String toString(LogEntryProto... entries) {
-    return entries == null? "null"
-        : entries.length == 0 ? "[]"
-        : entries.length == 1? "" + toTermIndex(entries[0])
-        : "" + Arrays.stream(entries).map(ServerProtoUtils::toTermIndex)
-            .collect(Collectors.toList());
-  }
-
-  public static String toString(AppendEntriesReplyProto reply) {
-    return toString(reply.getServerReply()) + "," + reply.getResult()
-        + ",nextIndex:" + reply.getNextIndex() + ",term:" + reply.getTerm();
-  }
-
-  private static String toString(RaftRpcReplyProto reply) {
-    return reply.getRequestorId() + "->" + reply.getReplyId() + ","
-        + reply.getSuccess();
-  }
-
-  public static RaftConfigurationProto toRaftConfigurationProto(
-      RaftConfiguration conf) {
-    return RaftConfigurationProto.newBuilder()
-        .addAllPeers(ProtoUtils.toRaftPeerProtos(conf.getPeersInConf()))
-        .addAllOldPeers(ProtoUtils.toRaftPeerProtos(conf.getPeersInOldConf()))
-        .build();
-  }
-
-  public static RaftConfiguration toRaftConfiguration(
-      long index, RaftConfigurationProto proto) {
-    final RaftConfiguration.Builder b = RaftConfiguration.newBuilder()
-        .setConf(ProtoUtils.toRaftPeerArray(proto.getPeersList()))
-        .setLogEntryIndex(index);
-    if (proto.getOldPeersCount() > 0) {
-      b.setOldConf(ProtoUtils.toRaftPeerArray(proto.getOldPeersList()));
-    }
-    return b.build();
-  }
-
-  public static LogEntryProto toLogEntryProto(
-      RaftConfiguration conf, long term, long index) {
-    return LogEntryProto.newBuilder()
-        .setTerm(term)
-        .setIndex(index)
-        .setConfigurationEntry(toRaftConfigurationProto(conf))
-        .build();
-  }
-
-  public static RequestVoteReplyProto toRequestVoteReplyProto(
-      String requestorId, String replyId, boolean success, long term,
-      boolean shouldShutdown) {
-    final RequestVoteReplyProto.Builder b = RequestVoteReplyProto.newBuilder();
-    b.setServerReply(ClientProtoUtils.toRaftRpcReplyProtoBuilder(requestorId, replyId,
-        DEFAULT_SEQNUM, success))
-        .setTerm(term)
-        .setShouldShutdown(shouldShutdown);
-    return b.build();
-  }
-
-  public static RequestVoteRequestProto toRequestVoteRequestProto(
-      String requestorId, String replyId, long term, TermIndex lastEntry) {
-    final RequestVoteRequestProto.Builder b = RequestVoteRequestProto.newBuilder()
-        .setServerRequest(
-            ClientProtoUtils.toRaftRpcRequestProtoBuilder(requestorId, replyId, DEFAULT_SEQNUM))
-        .setCandidateTerm(term);
-    if (lastEntry != null) {
-      b.setCandidateLastEntry(toTermIndexProto(lastEntry));
-    }
-    return b.build();
-  }
-
-  public static InstallSnapshotReplyProto toInstallSnapshotReplyProto(
-      String requestorId, String replyId, long term, int requestIndex,
-      InstallSnapshotResult result) {
-    final RaftRpcReplyProto.Builder rb = ClientProtoUtils.toRaftRpcReplyProtoBuilder(requestorId,
-        replyId, DEFAULT_SEQNUM, result == InstallSnapshotResult.SUCCESS);
-    final InstallSnapshotReplyProto.Builder builder = InstallSnapshotReplyProto
-        .newBuilder().setServerReply(rb).setTerm(term).setResult(result)
-        .setRequestIndex(requestIndex);
-    return builder.build();
-  }
-
-  public static InstallSnapshotRequestProto toInstallSnapshotRequestProto(
-      String requestorId, String replyId, String requestId, int requestIndex,
-      long term, TermIndex lastTermIndex, List<FileChunkProto> chunks,
-      long totalSize, boolean done) {
-    return InstallSnapshotRequestProto.newBuilder()
-        .setServerRequest(
-            ClientProtoUtils.toRaftRpcRequestProtoBuilder(requestorId, replyId, DEFAULT_SEQNUM))
-        .setRequestId(requestId)
-        .setRequestIndex(requestIndex)
-        // .setRaftConfiguration()  TODO: save and pass RaftConfiguration
-        .setLeaderTerm(term)
-        .setTermIndex(toTermIndexProto(lastTermIndex))
-        .addAllFileChunks(chunks)
-        .setTotalSize(totalSize)
-        .setDone(done).build();
-  }
-
-  public static AppendEntriesReplyProto toAppendEntriesReplyProto(
-      String requestorId, String replyId, long term,
-      long nextIndex, AppendEntriesReplyProto.AppendResult appendResult) {
-    RaftRpcReplyProto.Builder rb = ClientProtoUtils.toRaftRpcReplyProtoBuilder(requestorId,
-        replyId, DEFAULT_SEQNUM, appendResult == SUCCESS);
-    final AppendEntriesReplyProto.Builder b = AppendEntriesReplyProto.newBuilder();
-    b.setServerReply(rb).setTerm(term).setNextIndex(nextIndex)
-        .setResult(appendResult);
-    return b.build();
-  }
-
-  public static AppendEntriesRequestProto toAppendEntriesRequestProto(
-      String requestorId, String replyId, long leaderTerm,
-      List<LogEntryProto> entries, long leaderCommit, boolean initializing,
-      TermIndex previous) {
-    final AppendEntriesRequestProto.Builder b = AppendEntriesRequestProto
-        .newBuilder()
-        .setServerRequest(
-            ClientProtoUtils.toRaftRpcRequestProtoBuilder(requestorId, replyId, DEFAULT_SEQNUM))
-        .setLeaderTerm(leaderTerm)
-        .setLeaderCommit(leaderCommit)
-        .setInitializing(initializing);
-    if (entries != null && !entries.isEmpty()) {
-      b.addAllEntries(entries);
-    }
-
-    if (previous != null) {
-      b.setPreviousLog(toTermIndexProto(previous));
-    }
-    return b.build();
-  }
-
-}

http://git-wip-us.apache.org/repos/asf/incubator-ratis/blob/7e71a2e0/raft-server/src/main/java/org/apache/raft/server/impl/ServerState.java
----------------------------------------------------------------------
diff --git a/raft-server/src/main/java/org/apache/raft/server/impl/ServerState.java b/raft-server/src/main/java/org/apache/raft/server/impl/ServerState.java
deleted file mode 100644
index 8611101..0000000
--- a/raft-server/src/main/java/org/apache/raft/server/impl/ServerState.java
+++ /dev/null
@@ -1,345 +0,0 @@
-/**
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- *     http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-package org.apache.raft.server.impl;
-
-import com.google.common.annotations.VisibleForTesting;
-import com.google.common.base.Preconditions;
-import org.apache.raft.conf.RaftProperties;
-import org.apache.raft.server.protocol.TermIndex;
-import org.apache.raft.server.storage.*;
-import org.apache.raft.shaded.proto.RaftProtos.InstallSnapshotRequestProto;
-import org.apache.raft.shaded.proto.RaftProtos.LogEntryProto;
-import org.apache.raft.statemachine.SnapshotInfo;
-import org.apache.raft.statemachine.StateMachine;
-import org.apache.raft.statemachine.TransactionContext;
-import org.apache.raft.util.ProtoUtils;
-
-import java.io.Closeable;
-import java.io.IOException;
-
-import static org.apache.raft.server.RaftServerConfigKeys.RAFT_SERVER_USE_MEMORY_LOG_DEFAULT;
-import static org.apache.raft.server.RaftServerConfigKeys.RAFT_SERVER_USE_MEMORY_LOG_KEY;
-
-/**
- * Common states of a raft peer. Protected by RaftServer's lock.
- */
-public class ServerState implements Closeable {
-  private final String selfId;
-  private final RaftServerImpl server;
-  /** Raft log */
-  private final RaftLog log;
-  /** Raft configuration */
-  private final ConfigurationManager configurationManager;
-  /** The thread that applies committed log entries to the state machine */
-  private final StateMachineUpdater stateMachineUpdater;
-  /** local storage for log and snapshot */
-  private final RaftStorage storage;
-  private final SnapshotManager snapshotManager;
-
-  /**
-   * Latest term server has seen. initialized to 0 on first boot, increases
-   * monotonically.
-   */
-  private long currentTerm;
-  /**
-   * The server ID of the leader for this term. Null means either there is
-   * no leader for this term yet or this server does not know who it is yet.
-   */
-  private String leaderId;
-  /**
-   * Candidate that this peer granted vote for in current term (or null if none)
-   */
-  private String votedFor;
-
-  /**
-   * Latest installed snapshot for this server. This maybe different than StateMachine's latest
-   * snapshot. Once we successfully install a snapshot, the SM may not pick it up immediately.
-   * Further, this will not get updated when SM does snapshots itself.
-   */
-  private TermIndex latestInstalledSnapshot;
-
-  ServerState(String id, RaftConfiguration conf, RaftProperties prop,
-              RaftServerImpl server, StateMachine stateMachine) throws IOException {
-    this.selfId = id;
-    this.server = server;
-    configurationManager = new ConfigurationManager(conf);
-    storage = new RaftStorage(prop, RaftServerConstants.StartupOption.REGULAR);
-    snapshotManager = new SnapshotManager(storage, id);
-
-    long lastApplied = initStatemachine(stateMachine, prop);
-
-    leaderId = null;
-    log = initLog(id, prop, server, lastApplied);
-    RaftLog.Metadata metadata = log.loadMetadata();
-    currentTerm = metadata.getTerm();
-    votedFor = metadata.getVotedFor();
-
-    stateMachineUpdater = new StateMachineUpdater(stateMachine, server, log,
-         lastApplied, prop);
-  }
-
-  /**
-   * Used by tests to set initial raft configuration with correct port bindings.
-   */
-  @VisibleForTesting
-  public void setInitialConf(RaftConfiguration initialConf) {
-    configurationManager.setInitialConf(initialConf);
-  }
-
-  private long initStatemachine(StateMachine sm, RaftProperties properties)
-      throws IOException {
-    sm.initialize(selfId, properties, storage);
-    storage.setStateMachineStorage(sm.getStateMachineStorage());
-    SnapshotInfo snapshot = sm.getLatestSnapshot();
-
-    if (snapshot == null || snapshot.getTermIndex().getIndex() < 0) {
-      return RaftServerConstants.INVALID_LOG_INDEX;
-    }
-
-    // get the raft configuration from the snapshot
-    RaftConfiguration raftConf = sm.getRaftConfiguration();
-    if (raftConf != null) {
-      configurationManager.addConfiguration(raftConf.getLogEntryIndex(),
-          raftConf);
-    }
-    return snapshot.getIndex();
-  }
-
-  void start() {
-    stateMachineUpdater.start();
-  }
-
-  /**
-   * note we do not apply log entries to the state machine here since we do not
-   * know whether they have been committed.
-   */
-  private RaftLog initLog(String id, RaftProperties prop, RaftServerImpl server,
-      long lastIndexInSnapshot) throws IOException {
-    final RaftLog log;
-    if (prop.getBoolean(RAFT_SERVER_USE_MEMORY_LOG_KEY,
-        RAFT_SERVER_USE_MEMORY_LOG_DEFAULT)) {
-      log = new MemoryRaftLog(id);
-    } else {
-      log = new SegmentedRaftLog(id, server, this.storage,
-          lastIndexInSnapshot, prop);
-    }
-    log.open(configurationManager, lastIndexInSnapshot);
-    return log;
-  }
-
-  public RaftConfiguration getRaftConf() {
-    return configurationManager.getCurrent();
-  }
-
-  @VisibleForTesting
-
-  public String getSelfId() {
-    return this.selfId;
-  }
-
-  public long getCurrentTerm() {
-    return currentTerm;
-  }
-
-  void setCurrentTerm(long term) {
-    currentTerm = term;
-  }
-
-  String getLeaderId() {
-    return leaderId;
-  }
-
-  boolean hasLeader() {
-    return leaderId != null;
-  }
-
-  /**
-   * Become a candidate and start leader election
-   */
-  long initElection() {
-    votedFor = selfId;
-    leaderId = null;
-    return ++currentTerm;
-  }
-
-  void persistMetadata() throws IOException {
-    this.log.writeMetadata(currentTerm, votedFor);
-  }
-
-  void resetLeaderAndVotedFor() {
-    votedFor = null;
-    leaderId = null;
-  }
-
-  /**
-   * Vote for a candidate and update the local state.
-   */
-  void grantVote(String candidateId) {
-    votedFor = candidateId;
-    leaderId = null;
-  }
-
-  void setLeader(String leaderId) {
-    this.leaderId = leaderId;
-  }
-
-  void becomeLeader() {
-    leaderId = selfId;
-  }
-
-  public RaftLog getLog() {
-    return log;
-  }
-
-  long applyLog(TransactionContext operation) throws IOException {
-    return log.append(currentTerm, operation);
-  }
-
-  /**
-   * Check if accept the leader selfId and term from the incoming AppendEntries rpc.
-   * If accept, update the current state.
-   * @return true if the check passes
-   */
-  boolean recognizeLeader(String leaderId, long leaderTerm) {
-    if (leaderTerm < currentTerm) {
-      return false;
-    } else if (leaderTerm > currentTerm || this.leaderId == null) {
-      // If the request indicates a term that is greater than the current term
-      // or no leader has been set for the current term, make sure to update
-      // leader and term later
-      return true;
-    }
-    Preconditions.checkArgument(this.leaderId.equals(leaderId),
-        "selfId:%s, this.leaderId:%s, received leaderId:%s",
-        selfId, this.leaderId, leaderId);
-    return true;
-  }
-
-  /**
-   * Check if the candidate's term is acceptable
-   */
-  boolean recognizeCandidate(String candidateId,
-      long candidateTerm) {
-    if (candidateTerm > currentTerm) {
-      return true;
-    } else if (candidateTerm == currentTerm) {
-      // has not voted yet or this is a retry
-      return votedFor == null || votedFor.equals(candidateId);
-    }
-    return false;
-  }
-
-  boolean isLogUpToDate(TermIndex candidateLastEntry) {
-    LogEntryProto lastEntry = log.getLastEntry();
-    // need to take into account snapshot
-    SnapshotInfo snapshot = server.getStateMachine().getLatestSnapshot();
-     if (lastEntry == null && snapshot == null) {
-      return true;
-    } else if (candidateLastEntry == null) {
-      return false;
-    }
-    TermIndex local = ServerProtoUtils.toTermIndex(lastEntry);
-    if (local == null || (snapshot != null && snapshot.getIndex() > lastEntry.getIndex())) {
-      local = snapshot.getTermIndex();
-    }
-    return local.compareTo(candidateLastEntry) <= 0;
-  }
-
-  @Override
-  public String toString() {
-    return selfId + ":t" + currentTerm + ", leader=" + leaderId
-        + ", voted=" + votedFor + ", raftlog=" + log + ", conf=" + getRaftConf();
-  }
-
-  boolean isConfCommitted() {
-    return getLog().getLastCommittedIndex() >=
-        getRaftConf().getLogEntryIndex();
-  }
-
-  public void setRaftConf(long logIndex, RaftConfiguration conf) {
-    configurationManager.addConfiguration(logIndex, conf);
-    RaftServerImpl.LOG.info("{}: successfully update the configuration {}",
-        getSelfId(), conf);
-  }
-
-  void updateConfiguration(LogEntryProto[] entries) {
-    if (entries != null && entries.length > 0) {
-      configurationManager.removeConfigurations(entries[0].getIndex());
-      for (LogEntryProto entry : entries) {
-        if (ProtoUtils.isConfigurationLogEntry(entry)) {
-          final RaftConfiguration conf = ServerProtoUtils.toRaftConfiguration(
-              entry.getIndex(), entry.getConfigurationEntry());
-          configurationManager.addConfiguration(entry.getIndex(), conf);
-          server.getServerRpc().addPeers(conf.getPeers());
-        }
-      }
-    }
-  }
-
-  void updateStatemachine(long majorityIndex, long currentTerm) {
-    log.updateLastCommitted(majorityIndex, currentTerm);
-    stateMachineUpdater.notifyUpdater();
-  }
-
-  void reloadStateMachine(long lastIndexInSnapshot, long currentTerm)
-      throws IOException {
-    log.updateLastCommitted(lastIndexInSnapshot, currentTerm);
-
-    stateMachineUpdater.reloadStateMachine();
-  }
-
-  @Override
-  public void close() throws IOException {
-    stateMachineUpdater.stop();
-    RaftServerImpl.LOG.info("{} closes. The last applied log index is {}",
-        getSelfId(), getLastAppliedIndex());
-    storage.close();
-  }
-
-  @VisibleForTesting
-  public RaftStorage getStorage() {
-    return storage;
-  }
-
-  void installSnapshot(InstallSnapshotRequestProto request) throws IOException {
-    // TODO: verify that we need to install the snapshot
-    StateMachine sm = server.getStateMachine();
-    sm.pause(); // pause the SM to prepare for install snapshot
-    snapshotManager.installSnapshot(sm, request);
-    log.syncWithSnapshot(request.getTermIndex().getIndex());
-    this.latestInstalledSnapshot = ServerProtoUtils.toTermIndex(
-        request.getTermIndex());
-  }
-
-  SnapshotInfo getLatestSnapshot() {
-    return server.getStateMachine().getStateMachineStorage().getLatestSnapshot();
-  }
-
-  public TermIndex getLatestInstalledSnapshot() {
-    return latestInstalledSnapshot;
-  }
-
-  @VisibleForTesting
-  public long getLastAppliedIndex() {
-    return stateMachineUpdater.getLastAppliedIndex();
-  }
-
-  boolean isCurrentConfCommitted() {
-    return getRaftConf().getLogEntryIndex() <= getLog().getLastCommittedIndex();
-  }
-}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/incubator-ratis/blob/7e71a2e0/raft-server/src/main/java/org/apache/raft/server/impl/ServerUtils.java
----------------------------------------------------------------------
diff --git a/raft-server/src/main/java/org/apache/raft/server/impl/ServerUtils.java b/raft-server/src/main/java/org/apache/raft/server/impl/ServerUtils.java
deleted file mode 100644
index ac21386..0000000
--- a/raft-server/src/main/java/org/apache/raft/server/impl/ServerUtils.java
+++ /dev/null
@@ -1,81 +0,0 @@
-/**
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- *     http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-package org.apache.raft.server.impl;
-
-import org.apache.commons.lang.builder.HashCodeBuilder;
-import org.apache.raft.server.protocol.TermIndex;
-
-/** Server utilities for internal use. */
-public class ServerUtils {
-  public static TermIndex newTermIndex(long term, long index) {
-    return new TermIndexImpl(term, index);
-  }
-
-  private static class TermIndexImpl implements TermIndex {
-    private final long term;
-    private final long index; //log index; first index is 1.
-
-    TermIndexImpl(long term, long logIndex) {
-      this.term = term;
-      this.index = logIndex;
-    }
-
-    @Override
-    public long getTerm() {
-      return term;
-    }
-
-    @Override
-    public long getIndex() {
-      return index;
-    }
-
-    @Override
-    public int compareTo(TermIndex that) {
-      final int d = Long.compare(this.getTerm(), that.getTerm());
-      return d != 0 ? d : Long.compare(this.getIndex(), that.getIndex());
-    }
-
-    @Override
-    public boolean equals(Object obj) {
-      if (obj == this) {
-        return true;
-      } else if (obj == null || !(obj instanceof TermIndexImpl)) {
-        return false;
-      }
-
-      final TermIndexImpl that = (TermIndexImpl) obj;
-      return this.getTerm() == that.getTerm()
-          && this.getIndex() == that.getIndex();
-    }
-
-    @Override
-    public int hashCode() {
-      return new HashCodeBuilder().append(term).append(index).hashCode();
-    }
-
-    private static String toString(long n) {
-      return n < 0 ? "~" : "" + n;
-    }
-
-    @Override
-    public String toString() {
-      return "(t:" + toString(term) + ", i:" + toString(index) + ")";
-    }
-  }
-}

http://git-wip-us.apache.org/repos/asf/incubator-ratis/blob/7e71a2e0/raft-server/src/main/java/org/apache/raft/server/impl/StateMachineUpdater.java
----------------------------------------------------------------------
diff --git a/raft-server/src/main/java/org/apache/raft/server/impl/StateMachineUpdater.java b/raft-server/src/main/java/org/apache/raft/server/impl/StateMachineUpdater.java
deleted file mode 100644
index f85639b..0000000
--- a/raft-server/src/main/java/org/apache/raft/server/impl/StateMachineUpdater.java
+++ /dev/null
@@ -1,213 +0,0 @@
-/**
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- *     http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-package org.apache.raft.server.impl;
-
-import com.google.common.base.Preconditions;
-import org.apache.raft.conf.RaftProperties;
-import org.apache.raft.protocol.Message;
-import org.apache.raft.server.RaftServerConfigKeys;
-import org.apache.raft.server.storage.RaftLog;
-import org.apache.raft.server.storage.RaftStorage;
-import org.apache.raft.shaded.proto.RaftProtos.LogEntryProto;
-import org.apache.raft.statemachine.SnapshotInfo;
-import org.apache.raft.statemachine.StateMachine;
-import org.apache.raft.statemachine.TransactionContext;
-import org.apache.raft.util.Daemon;
-import org.apache.raft.util.ExitUtils;
-import org.apache.raft.util.LifeCycle;
-import org.slf4j.Logger;
-import org.slf4j.LoggerFactory;
-
-import java.io.IOException;
-import java.util.concurrent.CompletableFuture;
-
-import static org.apache.raft.shaded.proto.RaftProtos.LogEntryProto.LogEntryBodyCase.CONFIGURATIONENTRY;
-import static org.apache.raft.shaded.proto.RaftProtos.LogEntryProto.LogEntryBodyCase.SMLOGENTRY;
-
-/**
- * This class tracks the log entries that have been committed in a quorum and
- * applies them to the state machine. We let a separate thread do this work
- * asynchronously so that this will not block normal raft protocol.
- *
- * If the auto log compaction is enabled, the state machine updater thread will
- * trigger a snapshot of the state machine by calling
- * {@link StateMachine#takeSnapshot} when the log size exceeds a limit.
- */
-class StateMachineUpdater implements Runnable {
-  static final Logger LOG = LoggerFactory.getLogger(StateMachineUpdater.class);
-
-  enum State {
-    RUNNING, STOP, RELOAD
-  }
-
-  private final RaftProperties properties;
-  private final StateMachine stateMachine;
-  private final RaftServerImpl server;
-  private final RaftLog raftLog;
-
-  private volatile long lastAppliedIndex;
-
-  private final boolean autoSnapshotEnabled;
-  private final long snapshotThreshold;
-  private long lastSnapshotIndex;
-
-  private final Thread updater;
-  private volatile State state = State.RUNNING;
-
-  StateMachineUpdater(StateMachine stateMachine, RaftServerImpl server,
-      RaftLog raftLog, long lastAppliedIndex, RaftProperties properties) {
-    this.properties = properties;
-    this.stateMachine = stateMachine;
-    this.server = server;
-    this.raftLog = raftLog;
-
-    this.lastAppliedIndex = lastAppliedIndex;
-    lastSnapshotIndex = lastAppliedIndex;
-
-    autoSnapshotEnabled = properties.getBoolean(
-        RaftServerConfigKeys.RAFT_SERVER_AUTO_SNAPSHOT_ENABLED_KEY,
-        RaftServerConfigKeys.RAFT_SERVER_AUTO_SNAPSHOT_ENABLED_DEFAULT);
-    snapshotThreshold = properties.getLong(
-        RaftServerConfigKeys.RAFT_SERVER_SNAPSHOT_TRIGGER_THRESHOLD_KEY,
-        RaftServerConfigKeys.RAFT_SERVER_SNAPSHOT_TRIGGER_THRESHOLD_DEFAULT);
-    updater = new Daemon(this);
-  }
-
-  void start() {
-    updater.start();
-  }
-
-  void stop() {
-    state = State.STOP;
-    updater.interrupt();
-    try {
-      stateMachine.close();
-    } catch (IOException ignored) {
-    }
-  }
-
-  void reloadStateMachine() {
-    state = State.RELOAD;
-    notifyUpdater();
-  }
-
-  synchronized void notifyUpdater() {
-    notifyAll();
-  }
-
-  @Override
-  public String toString() {
-    return this.getClass().getSimpleName() + "-" + raftLog.getSelfId();
-  }
-
-  @Override
-  public void run() {
-    final RaftStorage storage = server.getState().getStorage();
-    while (isRunning()) {
-      try {
-        synchronized (this) {
-          // when the peers just start, the committedIndex is initialized as 0
-          // and will be updated only after the leader contacts other peers.
-          // Thus initially lastAppliedIndex can be greater than lastCommitted.
-          while (lastAppliedIndex >= raftLog.getLastCommittedIndex()) {
-            wait();
-          }
-        }
-
-        final long committedIndex = raftLog.getLastCommittedIndex();
-        Preconditions.checkState(lastAppliedIndex < committedIndex);
-
-        if (state == State.RELOAD) {
-          Preconditions.checkState(stateMachine.getLifeCycleState() == LifeCycle.State.PAUSED);
-
-          stateMachine.reinitialize(server.getId(), properties, storage);
-
-          SnapshotInfo snapshot = stateMachine.getLatestSnapshot();
-          Preconditions.checkState(snapshot != null && snapshot.getIndex() > lastAppliedIndex,
-              "Snapshot: %s, lastAppliedIndex: %s", snapshot, lastAppliedIndex);
-
-          lastAppliedIndex = snapshot.getIndex();
-          lastSnapshotIndex = snapshot.getIndex();
-          state = State.RUNNING;
-        }
-
-        while (lastAppliedIndex < committedIndex) {
-          final LogEntryProto next = raftLog.get(lastAppliedIndex + 1);
-          if (next != null) {
-            if (next.getLogEntryBodyCase() == CONFIGURATIONENTRY) {
-              // the reply should have already been set. only need to record
-              // the new conf in the state machine.
-              stateMachine.setRaftConfiguration(
-                  ServerProtoUtils.toRaftConfiguration(next.getIndex(),
-                      next.getConfigurationEntry()));
-            } else if (next.getLogEntryBodyCase() == SMLOGENTRY) {
-              // check whether there is a TransactionContext because we are the leader.
-              TransactionContext trx = server.getTransactionContext(next.getIndex());
-              if (trx == null) {
-                trx = new TransactionContext(stateMachine, next);
-              }
-
-              // Let the StateMachine inject logic for committed transactions in sequential order.
-              trx = stateMachine.applyTransactionSerial(trx);
-
-              // TODO: This step can be parallelized
-              CompletableFuture<Message> messageFuture =
-                  stateMachine.applyTransaction(trx);
-              server.replyPendingRequest(next.getIndex(), messageFuture);
-            }
-            lastAppliedIndex++;
-          } else {
-            LOG.debug("{}: logEntry {} is null. There may be snapshot to load. state:{}",
-                this, lastAppliedIndex + 1, state);
-            break;
-          }
-        }
-
-        // check if need to trigger a snapshot
-        if (shouldTakeSnapshot(lastAppliedIndex)) {
-          stateMachine.takeSnapshot();
-          // TODO purge logs, including log cache. but should keep log for leader's RPCSenders
-          lastSnapshotIndex = lastAppliedIndex;
-        }
-      } catch (InterruptedException e) {
-        if (!isRunning()) {
-          LOG.info("{}: the StateMachineUpdater is interrupted and will exit.", this);
-        } else {
-          final String s = this + ": the StateMachineUpdater is wrongly interrupted";
-          ExitUtils.terminate(1, s, e, LOG);
-        }
-      } catch (Throwable t) {
-        final String s = this + ": the StateMachineUpdater hits Throwable";
-        ExitUtils.terminate(2, s, t, LOG);
-      }
-    }
-  }
-
-  private boolean isRunning() {
-    return state != State.STOP;
-  }
-
-  private boolean shouldTakeSnapshot(long currentAppliedIndex) {
-    return autoSnapshotEnabled && (state != State.RELOAD) &&
-        (currentAppliedIndex - lastSnapshotIndex >= snapshotThreshold);
-  }
-
-  long getLastAppliedIndex() {
-    return lastAppliedIndex;
-  }
-}

http://git-wip-us.apache.org/repos/asf/incubator-ratis/blob/7e71a2e0/raft-server/src/main/java/org/apache/raft/server/protocol/RaftServerProtocol.java
----------------------------------------------------------------------
diff --git a/raft-server/src/main/java/org/apache/raft/server/protocol/RaftServerProtocol.java b/raft-server/src/main/java/org/apache/raft/server/protocol/RaftServerProtocol.java
deleted file mode 100644
index 59e9bba..0000000
--- a/raft-server/src/main/java/org/apache/raft/server/protocol/RaftServerProtocol.java
+++ /dev/null
@@ -1,31 +0,0 @@
-/**
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- *     http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-package org.apache.raft.server.protocol;
-
-import org.apache.raft.shaded.proto.RaftProtos.*;
-
-import java.io.IOException;
-
-public interface RaftServerProtocol {
-
-  RequestVoteReplyProto requestVote(RequestVoteRequestProto request) throws IOException;
-
-  AppendEntriesReplyProto appendEntries(AppendEntriesRequestProto request) throws IOException;
-
-  InstallSnapshotReplyProto installSnapshot(InstallSnapshotRequestProto request) throws IOException;
-}

http://git-wip-us.apache.org/repos/asf/incubator-ratis/blob/7e71a2e0/raft-server/src/main/java/org/apache/raft/server/protocol/TermIndex.java
----------------------------------------------------------------------
diff --git a/raft-server/src/main/java/org/apache/raft/server/protocol/TermIndex.java b/raft-server/src/main/java/org/apache/raft/server/protocol/TermIndex.java
deleted file mode 100644
index df401d6..0000000
--- a/raft-server/src/main/java/org/apache/raft/server/protocol/TermIndex.java
+++ /dev/null
@@ -1,36 +0,0 @@
-/**
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- *     http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-package org.apache.raft.server.protocol;
-
-import org.apache.raft.server.impl.ServerUtils;
-
-/** The term and the log index defined in the Raft consensus algorithm. */
-public interface TermIndex extends Comparable<TermIndex> {
-  /** @return the term. */
-  long getTerm();
-
-  /** @return the index. */
-  long getIndex();
-
-  /** Create a new {@link TermIndex} instance. */
-  static TermIndex newTermIndex(long term, long index) {
-    return ServerUtils.newTermIndex(term, index);
-  }
-}
-
-

http://git-wip-us.apache.org/repos/asf/incubator-ratis/blob/7e71a2e0/raft-server/src/main/java/org/apache/raft/server/storage/BufferedChannelBase.java
----------------------------------------------------------------------
diff --git a/raft-server/src/main/java/org/apache/raft/server/storage/BufferedChannelBase.java b/raft-server/src/main/java/org/apache/raft/server/storage/BufferedChannelBase.java
deleted file mode 100644
index 4440be9..0000000
--- a/raft-server/src/main/java/org/apache/raft/server/storage/BufferedChannelBase.java
+++ /dev/null
@@ -1,52 +0,0 @@
-/**
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- *     http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-package org.apache.raft.server.storage;
-
-import java.io.Closeable;
-import java.io.IOException;
-import java.nio.channels.FileChannel;
-
-public abstract class BufferedChannelBase implements Closeable {
-  protected final FileChannel fileChannel;
-
-  protected BufferedChannelBase(FileChannel fc) {
-    this.fileChannel = fc;
-  }
-
-  protected FileChannel validateAndGetFileChannel() throws IOException {
-    if (!fileChannel.isOpen()) {
-      throw new IOException(
-          "Attempting to access a file channel that has already been closed");
-    }
-    return fileChannel;
-  }
-
-  /**
-   * Get the current size of the underlying FileChannel.
-   */
-  public long size() throws IOException {
-    return validateAndGetFileChannel().size();
-  }
-
-  /**
-   * Get the {@link FileChannel} that this BufferedChannel wraps around.
-   */
-  public FileChannel getFileChannel() {
-    return fileChannel;
-  }
-}

http://git-wip-us.apache.org/repos/asf/incubator-ratis/blob/7e71a2e0/raft-server/src/main/java/org/apache/raft/server/storage/BufferedWriteChannel.java
----------------------------------------------------------------------
diff --git a/raft-server/src/main/java/org/apache/raft/server/storage/BufferedWriteChannel.java b/raft-server/src/main/java/org/apache/raft/server/storage/BufferedWriteChannel.java
deleted file mode 100644
index 6c662d1..0000000
--- a/raft-server/src/main/java/org/apache/raft/server/storage/BufferedWriteChannel.java
+++ /dev/null
@@ -1,159 +0,0 @@
-/**
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- *     http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-package org.apache.raft.server.storage;
-
-import java.io.IOException;
-import java.nio.ByteBuffer;
-import java.nio.channels.FileChannel;
-import java.util.concurrent.atomic.AtomicLong;
-
-/**
- * Provides a buffering layer in front of a FileChannel for writing.
- */
-public class BufferedWriteChannel extends BufferedChannelBase {
-  // The capacity of the write buffer.
-  private final int writeCapacity;
-  // The position of the file channel's write pointer.
-  private AtomicLong writeBufferStartPosition = new AtomicLong(0);
-  // The buffer used to write operations.
-  private final ByteBuffer writeBuffer;
-  // The absolute position of the next write operation.
-  private volatile long position;
-
-  public BufferedWriteChannel(FileChannel fc, int writeCapacity)
-      throws IOException {
-    super(fc);
-    this.writeCapacity = writeCapacity;
-    this.position = fc.position();
-    this.writeBufferStartPosition.set(position);
-    this.writeBuffer = ByteBuffer.allocateDirect(writeCapacity);
-  }
-
-  /**
-   * Write all the data in src to the {@link FileChannel}. Note that this function can
-   * buffer or re-order writes based on the implementation. These writes will be flushed
-   * to the disk only when flush() is invoked.
-   *
-   * @param src The source ByteBuffer which contains the data to be written.
-   * @throws IOException if a write operation fails.
-   */
-  public void write(ByteBuffer src) throws IOException {
-    int copied = 0;
-    while (src.remaining() > 0) {
-      int truncated = 0;
-      if (writeBuffer.remaining() < src.remaining()) {
-        truncated = src.remaining() - writeBuffer.remaining();
-        src.limit(src.limit() - truncated);
-      }
-      copied += src.remaining();
-      writeBuffer.put(src);
-      src.limit(src.limit() + truncated);
-      // if we have run out of buffer space, we should flush to the file
-      if (writeBuffer.remaining() == 0) {
-        flushInternal();
-      }
-    }
-    position += copied;
-  }
-
-  /**
-   * Write the specified byte.
-   * @param b the byte to be written
-   */
-  public void write(int b) throws IOException {
-    writeBuffer.put((byte) b);
-    if (writeBuffer.remaining() == 0) {
-      flushInternal();
-    }
-    position++;
-  }
-
-  public void write(byte[] b) throws IOException {
-    int offset = 0;
-    while (offset < b.length) {
-      int toPut = Math.min(b.length - offset, writeBuffer.remaining());
-      writeBuffer.put(b, offset, toPut);
-      offset += toPut;
-      if (writeBuffer.remaining() == 0) {
-        flushInternal();
-      }
-    }
-    position += b.length;
-  }
-
-  /**
-   * Get the position where the next write operation will begin writing from.
-   */
-  public long position() {
-    return position;
-  }
-
-  /**
-   * Get the position of the file channel's write pointer.
-   */
-  public long getFileChannelPosition() {
-    return writeBufferStartPosition.get();
-  }
-
-
-  /**
-   * Write any data in the buffer to the file. If sync is set to true, force a
-   * sync operation so that data is persisted to the disk.
-   *
-   * @throws IOException if the write or sync operation fails.
-   */
-  public void flush(boolean shouldForceWrite) throws IOException {
-    synchronized (this) {
-      flushInternal();
-    }
-    if (shouldForceWrite) {
-      forceWrite(false);
-    }
-  }
-
-  /**
-   * Write any data in the buffer to the file and advance the writeBufferPosition
-   * Callers are expected to synchronize appropriately
-   *
-   * @throws IOException if the write fails.
-   */
-  private void flushInternal() throws IOException {
-    writeBuffer.flip();
-    do {
-      fileChannel.write(writeBuffer);
-    } while (writeBuffer.hasRemaining());
-    writeBuffer.clear();
-    writeBufferStartPosition.set(fileChannel.position());
-  }
-
-  public long forceWrite(boolean forceMetadata) throws IOException {
-    // This is the point up to which we had flushed to the file system page cache
-    // before issuing this force write hence is guaranteed to be made durable by
-    // the force write, any flush that happens after this may or may
-    // not be flushed
-    long positionForceWrite = writeBufferStartPosition.get();
-    fileChannel.force(forceMetadata);
-    return positionForceWrite;
-  }
-
-  @Override
-  public void close() throws IOException {
-    fileChannel.close();
-    writeBuffer.clear();
-  }
-}

http://git-wip-us.apache.org/repos/asf/incubator-ratis/blob/7e71a2e0/raft-server/src/main/java/org/apache/raft/server/storage/FileInfo.java
----------------------------------------------------------------------
diff --git a/raft-server/src/main/java/org/apache/raft/server/storage/FileInfo.java b/raft-server/src/main/java/org/apache/raft/server/storage/FileInfo.java
deleted file mode 100644
index cdeb622..0000000
--- a/raft-server/src/main/java/org/apache/raft/server/storage/FileInfo.java
+++ /dev/null
@@ -1,59 +0,0 @@
-/**
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- *     http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-package org.apache.raft.server.storage;
-
-import org.apache.raft.io.MD5Hash;
-
-import java.nio.file.Path;
-
-/**
- * Metadata about a file.
- *
- * The objects of this class are immutable.
- */
-public class FileInfo {
-  private final Path path;
-  private final MD5Hash fileDigest;
-  private final long fileSize;
-
-  public FileInfo(Path path, MD5Hash fileDigest) {
-    this.path = path;
-    this.fileDigest = fileDigest;
-    this.fileSize = path.toFile().length();
-  }
-
-  @Override
-  public String toString() {
-    return path.toString();
-  }
-
-  /** @return the path of the file. */
-  public Path getPath() {
-    return path;
-  }
-
-  /** @return the MD5 file digest of the file. */
-  public MD5Hash getFileDigest() {
-    return fileDigest;
-  }
-
-  /** @return the size of the file. */
-  public long getFileSize() {
-    return fileSize;
-  }
-}

http://git-wip-us.apache.org/repos/asf/incubator-ratis/blob/7e71a2e0/raft-server/src/main/java/org/apache/raft/server/storage/LogInputStream.java
----------------------------------------------------------------------
diff --git a/raft-server/src/main/java/org/apache/raft/server/storage/LogInputStream.java b/raft-server/src/main/java/org/apache/raft/server/storage/LogInputStream.java
deleted file mode 100644
index 95597b2..0000000
--- a/raft-server/src/main/java/org/apache/raft/server/storage/LogInputStream.java
+++ /dev/null
@@ -1,258 +0,0 @@
-/**
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- *     http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-package org.apache.raft.server.storage;
-
-import com.google.common.base.Preconditions;
-import com.google.common.base.Throwables;
-import org.apache.raft.shaded.proto.RaftProtos.LogEntryProto;
-import org.apache.raft.util.RaftUtils;
-import org.slf4j.Logger;
-import org.slf4j.LoggerFactory;
-
-import java.io.Closeable;
-import java.io.EOFException;
-import java.io.File;
-import java.io.IOException;
-
-import static org.apache.raft.server.impl.RaftServerConstants.INVALID_LOG_INDEX;
-
-public class LogInputStream implements Closeable {
-  static final Logger LOG = LoggerFactory.getLogger(LogInputStream.class);
-
-  static class LogValidation {
-    private final long validLength;
-    private final long endIndex;
-    private final boolean hasCorruptHeader;
-
-    LogValidation(long validLength, long endIndex, boolean hasCorruptHeader) {
-      this.validLength = validLength;
-      this.endIndex = endIndex;
-      this.hasCorruptHeader = hasCorruptHeader;
-    }
-
-    long getValidLength() {
-      return validLength;
-    }
-
-    long getEndIndex() {
-      return endIndex;
-    }
-
-    boolean hasCorruptHeader() {
-      return hasCorruptHeader;
-    }
-  }
-
-  private enum State {
-    UNINIT,
-    OPEN,
-    CLOSED
-  }
-
-  private final File logFile;
-  private final long startIndex;
-  private final long endIndex;
-  private final boolean isOpen;
-  private State state = State.UNINIT;
-  private LogReader reader;
-
-  public LogInputStream(File log, long startIndex, long endIndex,
-      boolean isOpen) {
-    if (isOpen) {
-      Preconditions.checkArgument(endIndex == INVALID_LOG_INDEX);
-    } else {
-      Preconditions.checkArgument(endIndex >= startIndex);
-    }
-
-    this.logFile = log;
-    this.startIndex = startIndex;
-    this.endIndex = endIndex;
-    this.isOpen = isOpen;
-  }
-
-  private void init() throws IOException {
-    Preconditions.checkState(state == State.UNINIT);
-    try {
-      reader = new LogReader(logFile);
-      // read the log header
-      String header = reader.readLogHeader();
-      Preconditions.checkState(SegmentedRaftLog.HEADER_STR.equals(header),
-          "Corrupted log header: %s", header);
-      state = State.OPEN;
-    } finally {
-      if (reader == null) {
-        state = State.CLOSED;
-      }
-    }
-  }
-
-  long getStartIndex() {
-    return startIndex;
-  }
-
-  long getEndIndex() {
-    return endIndex;
-  }
-
-  String getName() {
-    return logFile.getName();
-  }
-
-  public LogEntryProto nextEntry() throws IOException {
-    LogEntryProto entry = null;
-    switch (state) {
-      case UNINIT:
-        try {
-          init();
-        } catch (Throwable e) {
-          LOG.error("caught exception initializing " + this, e);
-          Throwables.propagateIfPossible(e, IOException.class);
-        }
-        Preconditions.checkState(state != State.UNINIT);
-        return nextEntry();
-      case OPEN:
-        entry = reader.readEntry();
-        if (entry != null) {
-          long index = entry.getIndex();
-          if (!isOpen() && index >= endIndex) {
-            /**
-             * The end index may be derived from the segment recovery
-             * process. It is possible that we still have some uncleaned garbage
-             * in the end. We should skip them.
-             */
-            long skipAmt = logFile.length() - reader.getPos();
-            if (skipAmt > 0) {
-              LOG.debug("skipping {} bytes at the end of log '{}': reached" +
-                  " entry {} out of {}", skipAmt, getName(), index, endIndex);
-              reader.skipFully(skipAmt);
-            }
-          }
-        }
-        break;
-      case CLOSED:
-        break; // return null
-    }
-    return entry;
-  }
-
-  long scanNextEntry() throws IOException {
-    Preconditions.checkState(state == State.OPEN);
-    return reader.scanEntry();
-  }
-
-  long getPosition() {
-    if (state == State.OPEN) {
-      return reader.getPos();
-    } else {
-      return 0;
-    }
-  }
-
-  @Override
-  public void close() throws IOException {
-    if (state == State.OPEN) {
-      reader.close();
-    }
-    state = State.CLOSED;
-  }
-
-  boolean isOpen() {
-    return isOpen;
-  }
-
-  @Override
-  public String toString() {
-    return getName();
-  }
-
-  /**
-   * @param file          File being scanned and validated.
-   * @param maxTxIdToScan Maximum Tx ID to try to scan.
-   *                      The scan returns after reading this or a higher
-   *                      ID. The file portion beyond this ID is
-   *                      potentially being updated.
-   * @return Result of the validation
-   * @throws IOException
-   */
-  static LogValidation scanEditLog(File file, long maxTxIdToScan)
-      throws IOException {
-    LogInputStream in;
-    try {
-      in = new LogInputStream(file, INVALID_LOG_INDEX, INVALID_LOG_INDEX, false);
-      // read the header, initialize the inputstream
-      in.init();
-    } catch (EOFException e) {
-      LOG.warn("Log file " + file + " has no valid header", e);
-      return new LogValidation(0, INVALID_LOG_INDEX, true);
-    }
-
-    try {
-      return scanEditLog(in, maxTxIdToScan);
-    } finally {
-      RaftUtils.cleanup(LOG, in);
-    }
-  }
-
-  /**
-   * Find the last valid entry index in the stream.
-   * If there are invalid or corrupt entries in the middle of the stream,
-   * scanEditLog will skip over them.
-   *
-   * This reads through the stream but does not close it.
-   *
-   * @param maxIndexToScan Maximum entry index to try to scan. The scan returns
-   *                       after reading this or a higher index. The file
-   *                       portion beyond this index is potentially being
-   *                       updated.
-   */
-  static LogValidation scanEditLog(LogInputStream in, long maxIndexToScan) {
-    long lastPos = 0;
-    long end = INVALID_LOG_INDEX;
-    long numValid = 0;
-    boolean hitError = false;
-    while (end < maxIndexToScan) {
-      long index;
-      lastPos = in.getPosition();
-      try {
-        if (hitError) {
-          LogEntryProto entry = in.nextEntry();
-          index = entry != null ? entry.getIndex() : INVALID_LOG_INDEX;
-          LOG.warn("After resync, position is " + in.getPosition());
-        } else {
-          index = in.scanNextEntry();
-        }
-        if (index == INVALID_LOG_INDEX) {
-          break;
-        } else {
-          hitError = false;
-        }
-      } catch (Throwable t) {
-        LOG.warn("Caught exception after scanning through {} ops from {}"
-            + " while determining its valid length. Position was "
-            + lastPos, numValid, in, t);
-        hitError = true;
-        continue;
-      }
-      if (end == INVALID_LOG_INDEX || index > end) {
-        end = index;
-      }
-      numValid++;
-    }
-    return new LogValidation(lastPos, end, false);
-  }
-}


[05/54] [abbrv] incubator-ratis git commit: Move o.a.r.s.* to o.a.r.s.impl.

Posted by ji...@apache.org.
Move o.a.r.s.* to o.a.r.s.impl.


Project: http://git-wip-us.apache.org/repos/asf/incubator-ratis/repo
Commit: http://git-wip-us.apache.org/repos/asf/incubator-ratis/commit/c36810ed
Tree: http://git-wip-us.apache.org/repos/asf/incubator-ratis/tree/c36810ed
Diff: http://git-wip-us.apache.org/repos/asf/incubator-ratis/diff/c36810ed

Branch: refs/heads/master
Commit: c36810ed8dbd0e202f98ba57bd5730e7edff6863
Parents: dd0b631
Author: Tsz-Wo Nicholas Sze <sz...@hortonworks.com>
Authored: Sat Dec 31 17:23:13 2016 +0800
Committer: Tsz-Wo Nicholas Sze <sz...@hortonworks.com>
Committed: Sat Dec 31 17:23:13 2016 +0800

----------------------------------------------------------------------
 .../arithmetic/ArithmeticStateMachine.java      |   2 +-
 .../java/org/apache/raft/TestBatchAppend.java   |   2 +-
 .../org/apache/raft/TestRestartRaftPeer.java    |   2 +-
 .../TestRaftStateMachineException.java          |   2 +-
 .../org/apache/raft/grpc/RaftGRpcService.java   |   6 +-
 .../grpc/client/RaftClientProtocolService.java  |   2 +-
 .../raft/grpc/server/GRpcLogAppender.java       |   8 +-
 .../server/PipelinedLogAppenderFactory.java     |  10 +-
 .../grpc/server/RaftServerProtocolService.java  |   2 +-
 .../raft/grpc/MiniRaftClusterWithGRpc.java      |   8 +-
 .../grpc/TestNotLeaderExceptionWithGrpc.java    |   2 +-
 .../grpc/TestRaftReconfigurationWithGRpc.java   |   4 +-
 .../org/apache/raft/grpc/TestRaftStream.java    |   4 +-
 .../org/apache/raft/grpc/TestRaftWithGrpc.java  |   6 +-
 .../raft/hadooprpc/server/HadoopRpcService.java |   3 +
 .../hadooprpc/MiniRaftClusterWithHadoopRpc.java |   4 +-
 .../TestRaftReconfigurationWithHadoopRpc.java   |   2 +-
 .../raft/hadooprpc/TestRaftWithHadoopRpc.java   |   4 +-
 .../raft/netty/server/NettyRpcService.java      |   6 +-
 .../raft/netty/MiniRaftClusterWithNetty.java    |   7 +-
 .../netty/TestRaftReconfigurationWithNetty.java |   2 +-
 .../apache/raft/netty/TestRaftWithNetty.java    |   4 +-
 .../raft/server/ConfigurationManager.java       |  91 ---
 .../org/apache/raft/server/FollowerInfo.java    | 103 ---
 .../org/apache/raft/server/FollowerState.java   |  91 ---
 .../org/apache/raft/server/LeaderElection.java  | 242 ------
 .../org/apache/raft/server/LeaderState.java     | 581 --------------
 .../org/apache/raft/server/LogAppender.java     | 481 ------------
 .../apache/raft/server/LogAppenderFactory.java  |  31 -
 .../apache/raft/server/PeerConfiguration.java   |  90 ---
 .../org/apache/raft/server/PendingRequest.java  |  87 ---
 .../org/apache/raft/server/PendingRequests.java | 134 ----
 .../apache/raft/server/RaftConfiguration.java   | 261 -------
 .../java/org/apache/raft/server/RaftServer.java | 750 -------------------
 .../raft/server/RaftServerConfigKeys.java       |   1 +
 .../apache/raft/server/RaftServerConstants.java |  46 --
 .../org/apache/raft/server/RaftServerRpc.java   |  44 --
 .../apache/raft/server/RequestDispatcher.java   | 137 ----
 .../main/java/org/apache/raft/server/Role.java  |  25 -
 .../org/apache/raft/server/ServerState.java     | 346 ---------
 .../apache/raft/server/StateMachineUpdater.java | 213 ------
 .../raft/server/impl/ConfigurationManager.java  |  91 +++
 .../apache/raft/server/impl/FollowerInfo.java   | 103 +++
 .../apache/raft/server/impl/FollowerState.java  |  91 +++
 .../apache/raft/server/impl/LeaderElection.java | 241 ++++++
 .../apache/raft/server/impl/LeaderState.java    | 581 ++++++++++++++
 .../apache/raft/server/impl/LogAppender.java    | 480 ++++++++++++
 .../raft/server/impl/LogAppenderFactory.java    |  31 +
 .../raft/server/impl/PeerConfiguration.java     |  90 +++
 .../apache/raft/server/impl/PendingRequest.java |  87 +++
 .../raft/server/impl/PendingRequests.java       | 129 ++++
 .../raft/server/impl/RaftConfiguration.java     | 261 +++++++
 .../org/apache/raft/server/impl/RaftServer.java | 749 ++++++++++++++++++
 .../raft/server/impl/RaftServerConstants.java   |  46 ++
 .../apache/raft/server/impl/RaftServerRpc.java  |  44 ++
 .../raft/server/impl/RequestDispatcher.java     | 137 ++++
 .../java/org/apache/raft/server/impl/Role.java  |  25 +
 .../raft/server/impl/ServerProtoUtils.java      |   3 +-
 .../apache/raft/server/impl/ServerState.java    | 345 +++++++++
 .../raft/server/impl/StateMachineUpdater.java   | 213 ++++++
 .../raft/server/storage/LogInputStream.java     |   2 +-
 .../raft/server/storage/LogOutputStream.java    |   2 +-
 .../apache/raft/server/storage/LogReader.java   |   2 +-
 .../apache/raft/server/storage/LogSegment.java  |   2 +-
 .../raft/server/storage/MemoryRaftLog.java      |   4 +-
 .../org/apache/raft/server/storage/RaftLog.java |   6 +-
 .../raft/server/storage/RaftLogCache.java       |   4 +-
 .../raft/server/storage/RaftLogWorker.java      |   4 +-
 .../apache/raft/server/storage/RaftStorage.java |   2 +-
 .../server/storage/RaftStorageDirectory.java    |   2 +-
 .../raft/server/storage/SegmentedRaftLog.java   |   6 +-
 .../raft/statemachine/BaseStateMachine.java     |   4 +-
 .../statemachine/SimpleStateMachineStorage.java |   2 +-
 .../apache/raft/statemachine/SnapshotInfo.java  |   2 +-
 .../raft/statemachine/SnapshotInfoImpl.java     |   2 +-
 .../apache/raft/statemachine/StateMachine.java  |  12 +-
 .../raft/statemachine/TermIndexTracker.java     |   2 +-
 .../java/org/apache/raft/MiniRaftCluster.java   |   8 +-
 .../java/org/apache/raft/RaftBasicTests.java    |   8 +-
 .../raft/RaftNotLeaderExceptionBaseTest.java    |  10 +-
 .../test/java/org/apache/raft/RaftTestUtil.java |   6 +-
 .../server/BlockRequestHandlingInjection.java   |  84 ---
 .../server/DelayLocalExecutionInjection.java    |  67 --
 .../server/RaftReconfigurationBaseTest.java     | 576 --------------
 .../apache/raft/server/RaftServerTestUtil.java  |  67 --
 .../impl/BlockRequestHandlingInjection.java     |  84 +++
 .../impl/DelayLocalExecutionInjection.java      |  67 ++
 .../impl/RaftReconfigurationBaseTest.java       | 577 ++++++++++++++
 .../raft/server/impl/RaftServerTestUtil.java    |  67 ++
 .../MiniRaftClusterWithSimulatedRpc.java        |   2 +-
 .../server/simulation/SimulatedServerRpc.java   |   6 +-
 ...TestRaftReconfigurationWithSimulatedRpc.java |   2 +-
 .../simulation/TestRaftWithSimulatedRpc.java    |   2 +-
 .../server/storage/TestRaftLogReadWrite.java    |   4 +-
 .../raft/server/storage/TestRaftLogSegment.java |   8 +-
 .../raft/server/storage/TestRaftStorage.java    |   2 +-
 .../server/storage/TestSegmentedRaftLog.java    |   4 +-
 .../raft/statemachine/RaftSnapshotBaseTest.java |   6 +-
 .../SimpleStateMachine4Testing.java             |   2 +-
 .../raft/statemachine/TestStateMachine.java     |   2 +-
 100 files changed, 4651 insertions(+), 4663 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-ratis/blob/c36810ed/raft-examples/src/main/java/org/apache/raft/examples/arithmetic/ArithmeticStateMachine.java
----------------------------------------------------------------------
diff --git a/raft-examples/src/main/java/org/apache/raft/examples/arithmetic/ArithmeticStateMachine.java b/raft-examples/src/main/java/org/apache/raft/examples/arithmetic/ArithmeticStateMachine.java
index 4f038d2..b684669 100644
--- a/raft-examples/src/main/java/org/apache/raft/examples/arithmetic/ArithmeticStateMachine.java
+++ b/raft-examples/src/main/java/org/apache/raft/examples/arithmetic/ArithmeticStateMachine.java
@@ -22,7 +22,7 @@ import org.apache.raft.examples.arithmetic.expression.Expression;
 import org.apache.raft.protocol.Message;
 import org.apache.raft.protocol.RaftClientReply;
 import org.apache.raft.protocol.RaftClientRequest;
-import org.apache.raft.server.RaftServerConstants;
+import org.apache.raft.server.impl.RaftServerConstants;
 import org.apache.raft.server.protocol.TermIndex;
 import org.apache.raft.server.storage.RaftStorage;
 import org.apache.raft.shaded.proto.RaftProtos.LogEntryProto;

http://git-wip-us.apache.org/repos/asf/incubator-ratis/blob/c36810ed/raft-examples/src/test/java/org/apache/raft/TestBatchAppend.java
----------------------------------------------------------------------
diff --git a/raft-examples/src/test/java/org/apache/raft/TestBatchAppend.java b/raft-examples/src/test/java/org/apache/raft/TestBatchAppend.java
index 396bd47..c730245 100644
--- a/raft-examples/src/test/java/org/apache/raft/TestBatchAppend.java
+++ b/raft-examples/src/test/java/org/apache/raft/TestBatchAppend.java
@@ -22,7 +22,7 @@ import org.apache.raft.RaftTestUtil.SimpleMessage;
 import org.apache.raft.client.RaftClient;
 import org.apache.raft.conf.RaftProperties;
 import org.apache.raft.examples.RaftExamplesTestUtil;
-import org.apache.raft.server.RaftServer;
+import org.apache.raft.server.impl.RaftServer;
 import org.apache.raft.server.simulation.RequestHandler;
 import org.apache.raft.statemachine.SimpleStateMachine4Testing;
 import org.apache.raft.statemachine.StateMachine;

http://git-wip-us.apache.org/repos/asf/incubator-ratis/blob/c36810ed/raft-examples/src/test/java/org/apache/raft/TestRestartRaftPeer.java
----------------------------------------------------------------------
diff --git a/raft-examples/src/test/java/org/apache/raft/TestRestartRaftPeer.java b/raft-examples/src/test/java/org/apache/raft/TestRestartRaftPeer.java
index 10741f9..fadd7a8 100644
--- a/raft-examples/src/test/java/org/apache/raft/TestRestartRaftPeer.java
+++ b/raft-examples/src/test/java/org/apache/raft/TestRestartRaftPeer.java
@@ -22,7 +22,7 @@ import org.apache.raft.RaftTestUtil.SimpleMessage;
 import org.apache.raft.client.RaftClient;
 import org.apache.raft.conf.RaftProperties;
 import org.apache.raft.examples.RaftExamplesTestUtil;
-import org.apache.raft.server.RaftServer;
+import org.apache.raft.server.impl.RaftServer;
 import org.apache.raft.server.RaftServerConfigKeys;
 import org.apache.raft.server.simulation.RequestHandler;
 import org.apache.raft.server.storage.RaftLog;

http://git-wip-us.apache.org/repos/asf/incubator-ratis/blob/c36810ed/raft-examples/src/test/java/org/apache/raft/statemachine/TestRaftStateMachineException.java
----------------------------------------------------------------------
diff --git a/raft-examples/src/test/java/org/apache/raft/statemachine/TestRaftStateMachineException.java b/raft-examples/src/test/java/org/apache/raft/statemachine/TestRaftStateMachineException.java
index 61d7989..2fa6e53 100644
--- a/raft-examples/src/test/java/org/apache/raft/statemachine/TestRaftStateMachineException.java
+++ b/raft-examples/src/test/java/org/apache/raft/statemachine/TestRaftStateMachineException.java
@@ -24,7 +24,7 @@ import org.apache.raft.client.RaftClient;
 import org.apache.raft.examples.RaftExamplesTestUtil;
 import org.apache.raft.protocol.Message;
 import org.apache.raft.protocol.StateMachineException;
-import org.apache.raft.server.RaftServer;
+import org.apache.raft.server.impl.RaftServer;
 import org.apache.raft.server.simulation.RequestHandler;
 import org.apache.raft.server.storage.RaftLog;
 import org.apache.raft.util.RaftUtils;

http://git-wip-us.apache.org/repos/asf/incubator-ratis/blob/c36810ed/raft-grpc/src/main/java/org/apache/raft/grpc/RaftGRpcService.java
----------------------------------------------------------------------
diff --git a/raft-grpc/src/main/java/org/apache/raft/grpc/RaftGRpcService.java b/raft-grpc/src/main/java/org/apache/raft/grpc/RaftGRpcService.java
index 8259083..c9a0daf 100644
--- a/raft-grpc/src/main/java/org/apache/raft/grpc/RaftGRpcService.java
+++ b/raft-grpc/src/main/java/org/apache/raft/grpc/RaftGRpcService.java
@@ -23,9 +23,9 @@ import org.apache.raft.grpc.client.RaftClientProtocolService;
 import org.apache.raft.grpc.server.RaftServerProtocolClient;
 import org.apache.raft.grpc.server.RaftServerProtocolService;
 import org.apache.raft.protocol.RaftPeer;
-import org.apache.raft.server.RaftServer;
-import org.apache.raft.server.RaftServerRpc;
-import org.apache.raft.server.RequestDispatcher;
+import org.apache.raft.server.impl.RaftServer;
+import org.apache.raft.server.impl.RaftServerRpc;
+import org.apache.raft.server.impl.RequestDispatcher;
 import org.apache.raft.shaded.io.grpc.Server;
 import org.apache.raft.shaded.io.grpc.ServerBuilder;
 import org.apache.raft.shaded.io.grpc.netty.NettyServerBuilder;

http://git-wip-us.apache.org/repos/asf/incubator-ratis/blob/c36810ed/raft-grpc/src/main/java/org/apache/raft/grpc/client/RaftClientProtocolService.java
----------------------------------------------------------------------
diff --git a/raft-grpc/src/main/java/org/apache/raft/grpc/client/RaftClientProtocolService.java b/raft-grpc/src/main/java/org/apache/raft/grpc/client/RaftClientProtocolService.java
index d8704f6..32dbac7 100644
--- a/raft-grpc/src/main/java/org/apache/raft/grpc/client/RaftClientProtocolService.java
+++ b/raft-grpc/src/main/java/org/apache/raft/grpc/client/RaftClientProtocolService.java
@@ -21,7 +21,7 @@ import com.google.common.base.Preconditions;
 import org.apache.raft.client.impl.ClientProtoUtils;
 import org.apache.raft.grpc.RaftGrpcUtil;
 import org.apache.raft.protocol.RaftClientReply;
-import org.apache.raft.server.RequestDispatcher;
+import org.apache.raft.server.impl.RequestDispatcher;
 import org.apache.raft.shaded.io.grpc.stub.StreamObserver;
 import org.apache.raft.shaded.proto.RaftProtos.RaftClientReplyProto;
 import org.apache.raft.shaded.proto.RaftProtos.RaftClientRequestProto;

http://git-wip-us.apache.org/repos/asf/incubator-ratis/blob/c36810ed/raft-grpc/src/main/java/org/apache/raft/grpc/server/GRpcLogAppender.java
----------------------------------------------------------------------
diff --git a/raft-grpc/src/main/java/org/apache/raft/grpc/server/GRpcLogAppender.java b/raft-grpc/src/main/java/org/apache/raft/grpc/server/GRpcLogAppender.java
index 2f1fc90..b171bed 100644
--- a/raft-grpc/src/main/java/org/apache/raft/grpc/server/GRpcLogAppender.java
+++ b/raft-grpc/src/main/java/org/apache/raft/grpc/server/GRpcLogAppender.java
@@ -20,10 +20,10 @@ package org.apache.raft.grpc.server;
 import com.google.common.base.Preconditions;
 import org.apache.raft.grpc.RaftGRpcService;
 import org.apache.raft.grpc.RaftGrpcConfigKeys;
-import org.apache.raft.server.FollowerInfo;
-import org.apache.raft.server.LeaderState;
-import org.apache.raft.server.LogAppender;
-import org.apache.raft.server.RaftServer;
+import org.apache.raft.server.impl.FollowerInfo;
+import org.apache.raft.server.impl.LeaderState;
+import org.apache.raft.server.impl.LogAppender;
+import org.apache.raft.server.impl.RaftServer;
 import org.apache.raft.shaded.io.grpc.Status;
 import org.apache.raft.shaded.io.grpc.stub.StreamObserver;
 import org.apache.raft.shaded.proto.RaftProtos.AppendEntriesReplyProto;

http://git-wip-us.apache.org/repos/asf/incubator-ratis/blob/c36810ed/raft-grpc/src/main/java/org/apache/raft/grpc/server/PipelinedLogAppenderFactory.java
----------------------------------------------------------------------
diff --git a/raft-grpc/src/main/java/org/apache/raft/grpc/server/PipelinedLogAppenderFactory.java b/raft-grpc/src/main/java/org/apache/raft/grpc/server/PipelinedLogAppenderFactory.java
index 90882df..5f01980 100644
--- a/raft-grpc/src/main/java/org/apache/raft/grpc/server/PipelinedLogAppenderFactory.java
+++ b/raft-grpc/src/main/java/org/apache/raft/grpc/server/PipelinedLogAppenderFactory.java
@@ -17,11 +17,11 @@
  */
 package org.apache.raft.grpc.server;
 
-import org.apache.raft.server.FollowerInfo;
-import org.apache.raft.server.LeaderState;
-import org.apache.raft.server.LogAppender;
-import org.apache.raft.server.LogAppenderFactory;
-import org.apache.raft.server.RaftServer;
+import org.apache.raft.server.impl.FollowerInfo;
+import org.apache.raft.server.impl.LeaderState;
+import org.apache.raft.server.impl.LogAppender;
+import org.apache.raft.server.impl.LogAppenderFactory;
+import org.apache.raft.server.impl.RaftServer;
 
 public class PipelinedLogAppenderFactory implements LogAppenderFactory {
   @Override

http://git-wip-us.apache.org/repos/asf/incubator-ratis/blob/c36810ed/raft-grpc/src/main/java/org/apache/raft/grpc/server/RaftServerProtocolService.java
----------------------------------------------------------------------
diff --git a/raft-grpc/src/main/java/org/apache/raft/grpc/server/RaftServerProtocolService.java b/raft-grpc/src/main/java/org/apache/raft/grpc/server/RaftServerProtocolService.java
index d0fba89..2f06c59 100644
--- a/raft-grpc/src/main/java/org/apache/raft/grpc/server/RaftServerProtocolService.java
+++ b/raft-grpc/src/main/java/org/apache/raft/grpc/server/RaftServerProtocolService.java
@@ -18,7 +18,7 @@
 package org.apache.raft.grpc.server;
 
 import org.apache.raft.grpc.RaftGrpcUtil;
-import org.apache.raft.server.RequestDispatcher;
+import org.apache.raft.server.impl.RequestDispatcher;
 import org.apache.raft.shaded.io.grpc.stub.StreamObserver;
 import org.apache.raft.shaded.proto.RaftProtos.*;
 import org.apache.raft.shaded.proto.grpc.RaftServerProtocolServiceGrpc.RaftServerProtocolServiceImplBase;

http://git-wip-us.apache.org/repos/asf/incubator-ratis/blob/c36810ed/raft-grpc/src/test/java/org/apache/raft/grpc/MiniRaftClusterWithGRpc.java
----------------------------------------------------------------------
diff --git a/raft-grpc/src/test/java/org/apache/raft/grpc/MiniRaftClusterWithGRpc.java b/raft-grpc/src/test/java/org/apache/raft/grpc/MiniRaftClusterWithGRpc.java
index 8316db4..420ee88 100644
--- a/raft-grpc/src/test/java/org/apache/raft/grpc/MiniRaftClusterWithGRpc.java
+++ b/raft-grpc/src/test/java/org/apache/raft/grpc/MiniRaftClusterWithGRpc.java
@@ -25,10 +25,10 @@ import org.apache.raft.conf.RaftProperties;
 import org.apache.raft.grpc.client.RaftClientSenderWithGrpc;
 import org.apache.raft.grpc.server.PipelinedLogAppenderFactory;
 import org.apache.raft.protocol.RaftPeer;
-import org.apache.raft.server.BlockRequestHandlingInjection;
-import org.apache.raft.server.DelayLocalExecutionInjection;
-import org.apache.raft.server.LogAppenderFactory;
-import org.apache.raft.server.RaftServer;
+import org.apache.raft.server.impl.BlockRequestHandlingInjection;
+import org.apache.raft.server.impl.DelayLocalExecutionInjection;
+import org.apache.raft.server.impl.LogAppenderFactory;
+import org.apache.raft.server.impl.RaftServer;
 import org.apache.raft.util.NetUtils;
 
 import java.io.IOException;

http://git-wip-us.apache.org/repos/asf/incubator-ratis/blob/c36810ed/raft-grpc/src/test/java/org/apache/raft/grpc/TestNotLeaderExceptionWithGrpc.java
----------------------------------------------------------------------
diff --git a/raft-grpc/src/test/java/org/apache/raft/grpc/TestNotLeaderExceptionWithGrpc.java b/raft-grpc/src/test/java/org/apache/raft/grpc/TestNotLeaderExceptionWithGrpc.java
index 18f9751..a8357c9 100644
--- a/raft-grpc/src/test/java/org/apache/raft/grpc/TestNotLeaderExceptionWithGrpc.java
+++ b/raft-grpc/src/test/java/org/apache/raft/grpc/TestNotLeaderExceptionWithGrpc.java
@@ -21,7 +21,7 @@ import org.apache.raft.MiniRaftCluster;
 import org.apache.raft.RaftNotLeaderExceptionBaseTest;
 import org.apache.raft.conf.RaftProperties;
 import org.apache.raft.grpc.server.PipelinedLogAppenderFactory;
-import org.apache.raft.server.LogAppenderFactory;
+import org.apache.raft.server.impl.LogAppenderFactory;
 
 import java.io.IOException;
 

http://git-wip-us.apache.org/repos/asf/incubator-ratis/blob/c36810ed/raft-grpc/src/test/java/org/apache/raft/grpc/TestRaftReconfigurationWithGRpc.java
----------------------------------------------------------------------
diff --git a/raft-grpc/src/test/java/org/apache/raft/grpc/TestRaftReconfigurationWithGRpc.java b/raft-grpc/src/test/java/org/apache/raft/grpc/TestRaftReconfigurationWithGRpc.java
index 7912858..83e6c62 100644
--- a/raft-grpc/src/test/java/org/apache/raft/grpc/TestRaftReconfigurationWithGRpc.java
+++ b/raft-grpc/src/test/java/org/apache/raft/grpc/TestRaftReconfigurationWithGRpc.java
@@ -20,8 +20,8 @@ package org.apache.raft.grpc;
 import org.apache.log4j.Level;
 import org.apache.raft.grpc.server.PipelinedLogAppenderFactory;
 import org.apache.raft.grpc.server.RaftServerProtocolService;
-import org.apache.raft.server.LogAppenderFactory;
-import org.apache.raft.server.RaftReconfigurationBaseTest;
+import org.apache.raft.server.impl.LogAppenderFactory;
+import org.apache.raft.server.impl.RaftReconfigurationBaseTest;
 import org.apache.raft.util.RaftUtils;
 import org.junit.BeforeClass;
 

http://git-wip-us.apache.org/repos/asf/incubator-ratis/blob/c36810ed/raft-grpc/src/test/java/org/apache/raft/grpc/TestRaftStream.java
----------------------------------------------------------------------
diff --git a/raft-grpc/src/test/java/org/apache/raft/grpc/TestRaftStream.java b/raft-grpc/src/test/java/org/apache/raft/grpc/TestRaftStream.java
index 981ac5d..0c85854 100644
--- a/raft-grpc/src/test/java/org/apache/raft/grpc/TestRaftStream.java
+++ b/raft-grpc/src/test/java/org/apache/raft/grpc/TestRaftStream.java
@@ -23,8 +23,8 @@ import org.apache.raft.conf.RaftProperties;
 import org.apache.raft.grpc.client.AppendStreamer;
 import org.apache.raft.grpc.client.RaftOutputStream;
 import org.apache.raft.grpc.server.PipelinedLogAppenderFactory;
-import org.apache.raft.server.LogAppenderFactory;
-import org.apache.raft.server.RaftServer;
+import org.apache.raft.server.impl.LogAppenderFactory;
+import org.apache.raft.server.impl.RaftServer;
 import org.apache.raft.server.storage.RaftLog;
 import org.apache.raft.shaded.proto.RaftProtos.LogEntryProto;
 import org.apache.raft.util.RaftUtils;

http://git-wip-us.apache.org/repos/asf/incubator-ratis/blob/c36810ed/raft-grpc/src/test/java/org/apache/raft/grpc/TestRaftWithGrpc.java
----------------------------------------------------------------------
diff --git a/raft-grpc/src/test/java/org/apache/raft/grpc/TestRaftWithGrpc.java b/raft-grpc/src/test/java/org/apache/raft/grpc/TestRaftWithGrpc.java
index 083948a..b2d104b 100644
--- a/raft-grpc/src/test/java/org/apache/raft/grpc/TestRaftWithGrpc.java
+++ b/raft-grpc/src/test/java/org/apache/raft/grpc/TestRaftWithGrpc.java
@@ -20,9 +20,9 @@ package org.apache.raft.grpc;
 import org.apache.log4j.Level;
 import org.apache.raft.RaftBasicTests;
 import org.apache.raft.grpc.server.PipelinedLogAppenderFactory;
-import org.apache.raft.server.BlockRequestHandlingInjection;
-import org.apache.raft.server.LogAppenderFactory;
-import org.apache.raft.server.RaftServer;
+import org.apache.raft.server.impl.BlockRequestHandlingInjection;
+import org.apache.raft.server.impl.LogAppenderFactory;
+import org.apache.raft.server.impl.RaftServer;
 import org.apache.raft.util.RaftUtils;
 import org.junit.Assert;
 import org.junit.BeforeClass;

http://git-wip-us.apache.org/repos/asf/incubator-ratis/blob/c36810ed/raft-hadoop/src/main/java/org/apache/raft/hadooprpc/server/HadoopRpcService.java
----------------------------------------------------------------------
diff --git a/raft-hadoop/src/main/java/org/apache/raft/hadooprpc/server/HadoopRpcService.java b/raft-hadoop/src/main/java/org/apache/raft/hadooprpc/server/HadoopRpcService.java
index 3c22f12..ad4beec 100644
--- a/raft-hadoop/src/main/java/org/apache/raft/hadooprpc/server/HadoopRpcService.java
+++ b/raft-hadoop/src/main/java/org/apache/raft/hadooprpc/server/HadoopRpcService.java
@@ -26,6 +26,9 @@ import org.apache.raft.hadooprpc.client.RaftClientProtocolPB;
 import org.apache.raft.hadooprpc.client.RaftClientProtocolServerSideTranslatorPB;
 import org.apache.raft.protocol.RaftPeer;
 import org.apache.raft.server.*;
+import org.apache.raft.server.impl.RaftServer;
+import org.apache.raft.server.impl.RaftServerRpc;
+import org.apache.raft.server.impl.RequestDispatcher;
 import org.apache.raft.shaded.com.google.protobuf.BlockingService;
 import org.apache.raft.shaded.com.google.protobuf.ServiceException;
 import org.apache.raft.shaded.proto.RaftProtos.*;

http://git-wip-us.apache.org/repos/asf/incubator-ratis/blob/c36810ed/raft-hadoop/src/test/java/org/apache/raft/hadooprpc/MiniRaftClusterWithHadoopRpc.java
----------------------------------------------------------------------
diff --git a/raft-hadoop/src/test/java/org/apache/raft/hadooprpc/MiniRaftClusterWithHadoopRpc.java b/raft-hadoop/src/test/java/org/apache/raft/hadooprpc/MiniRaftClusterWithHadoopRpc.java
index ed3da7f..7f7ef49 100644
--- a/raft-hadoop/src/test/java/org/apache/raft/hadooprpc/MiniRaftClusterWithHadoopRpc.java
+++ b/raft-hadoop/src/test/java/org/apache/raft/hadooprpc/MiniRaftClusterWithHadoopRpc.java
@@ -26,8 +26,8 @@ import org.apache.raft.conf.RaftProperties;
 import org.apache.raft.hadooprpc.client.HadoopClientRequestSender;
 import org.apache.raft.hadooprpc.server.HadoopRpcService;
 import org.apache.raft.protocol.RaftPeer;
-import org.apache.raft.server.DelayLocalExecutionInjection;
-import org.apache.raft.server.RaftServer;
+import org.apache.raft.server.impl.DelayLocalExecutionInjection;
+import org.apache.raft.server.impl.RaftServer;
 import org.apache.raft.server.RaftServerConfigKeys;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;

http://git-wip-us.apache.org/repos/asf/incubator-ratis/blob/c36810ed/raft-hadoop/src/test/java/org/apache/raft/hadooprpc/TestRaftReconfigurationWithHadoopRpc.java
----------------------------------------------------------------------
diff --git a/raft-hadoop/src/test/java/org/apache/raft/hadooprpc/TestRaftReconfigurationWithHadoopRpc.java b/raft-hadoop/src/test/java/org/apache/raft/hadooprpc/TestRaftReconfigurationWithHadoopRpc.java
index fef13d1..0116280 100644
--- a/raft-hadoop/src/test/java/org/apache/raft/hadooprpc/TestRaftReconfigurationWithHadoopRpc.java
+++ b/raft-hadoop/src/test/java/org/apache/raft/hadooprpc/TestRaftReconfigurationWithHadoopRpc.java
@@ -20,7 +20,7 @@ package org.apache.raft.hadooprpc;
 import org.apache.hadoop.conf.Configuration;
 import org.apache.raft.MiniRaftCluster;
 import org.apache.raft.server.RaftServerConfigKeys;
-import org.apache.raft.server.RaftReconfigurationBaseTest;
+import org.apache.raft.server.impl.RaftReconfigurationBaseTest;
 
 import java.io.IOException;
 

http://git-wip-us.apache.org/repos/asf/incubator-ratis/blob/c36810ed/raft-hadoop/src/test/java/org/apache/raft/hadooprpc/TestRaftWithHadoopRpc.java
----------------------------------------------------------------------
diff --git a/raft-hadoop/src/test/java/org/apache/raft/hadooprpc/TestRaftWithHadoopRpc.java b/raft-hadoop/src/test/java/org/apache/raft/hadooprpc/TestRaftWithHadoopRpc.java
index e63b119..3971274 100644
--- a/raft-hadoop/src/test/java/org/apache/raft/hadooprpc/TestRaftWithHadoopRpc.java
+++ b/raft-hadoop/src/test/java/org/apache/raft/hadooprpc/TestRaftWithHadoopRpc.java
@@ -21,8 +21,8 @@ import org.apache.hadoop.conf.Configuration;
 import org.apache.log4j.Level;
 import org.apache.raft.RaftBasicTests;
 import org.apache.raft.client.RaftClient;
-import org.apache.raft.server.BlockRequestHandlingInjection;
-import org.apache.raft.server.RaftServer;
+import org.apache.raft.server.impl.BlockRequestHandlingInjection;
+import org.apache.raft.server.impl.RaftServer;
 import org.apache.raft.server.RaftServerConfigKeys;
 import org.apache.raft.util.RaftUtils;
 import org.junit.Test;

http://git-wip-us.apache.org/repos/asf/incubator-ratis/blob/c36810ed/raft-netty/src/main/java/org/apache/raft/netty/server/NettyRpcService.java
----------------------------------------------------------------------
diff --git a/raft-netty/src/main/java/org/apache/raft/netty/server/NettyRpcService.java b/raft-netty/src/main/java/org/apache/raft/netty/server/NettyRpcService.java
index 9c728ce..c0d751d 100644
--- a/raft-netty/src/main/java/org/apache/raft/netty/server/NettyRpcService.java
+++ b/raft-netty/src/main/java/org/apache/raft/netty/server/NettyRpcService.java
@@ -29,9 +29,9 @@ import org.apache.raft.client.impl.ClientProtoUtils;
 import org.apache.raft.netty.NettyRpcProxy;
 import org.apache.raft.protocol.RaftClientReply;
 import org.apache.raft.protocol.RaftPeer;
-import org.apache.raft.server.RaftServer;
-import org.apache.raft.server.RaftServerRpc;
-import org.apache.raft.server.RequestDispatcher;
+import org.apache.raft.server.impl.RaftServer;
+import org.apache.raft.server.impl.RaftServerRpc;
+import org.apache.raft.server.impl.RequestDispatcher;
 import org.apache.raft.shaded.io.netty.handler.codec.protobuf.ProtobufDecoder;
 import org.apache.raft.shaded.io.netty.handler.codec.protobuf.ProtobufEncoder;
 import org.apache.raft.shaded.io.netty.handler.codec.protobuf.ProtobufVarint32FrameDecoder;

http://git-wip-us.apache.org/repos/asf/incubator-ratis/blob/c36810ed/raft-netty/src/test/java/org/apache/raft/netty/MiniRaftClusterWithNetty.java
----------------------------------------------------------------------
diff --git a/raft-netty/src/test/java/org/apache/raft/netty/MiniRaftClusterWithNetty.java b/raft-netty/src/test/java/org/apache/raft/netty/MiniRaftClusterWithNetty.java
index 3ccb335..4958e9a 100644
--- a/raft-netty/src/test/java/org/apache/raft/netty/MiniRaftClusterWithNetty.java
+++ b/raft-netty/src/test/java/org/apache/raft/netty/MiniRaftClusterWithNetty.java
@@ -24,11 +24,10 @@ import org.apache.raft.conf.RaftProperties;
 import org.apache.raft.netty.client.NettyClientRequestSender;
 import org.apache.raft.netty.server.NettyRpcService;
 import org.apache.raft.protocol.RaftPeer;
-import org.apache.raft.server.DelayLocalExecutionInjection;
-import org.apache.raft.server.RaftConfiguration;
-import org.apache.raft.server.RaftServer;
+import org.apache.raft.server.impl.DelayLocalExecutionInjection;
+import org.apache.raft.server.impl.RaftConfiguration;
+import org.apache.raft.server.impl.RaftServer;
 import org.apache.raft.util.NetUtils;
-import org.apache.raft.util.RaftUtils;
 
 import java.io.IOException;
 import java.util.Collection;

http://git-wip-us.apache.org/repos/asf/incubator-ratis/blob/c36810ed/raft-netty/src/test/java/org/apache/raft/netty/TestRaftReconfigurationWithNetty.java
----------------------------------------------------------------------
diff --git a/raft-netty/src/test/java/org/apache/raft/netty/TestRaftReconfigurationWithNetty.java b/raft-netty/src/test/java/org/apache/raft/netty/TestRaftReconfigurationWithNetty.java
index 17e0a21..c4dd914 100644
--- a/raft-netty/src/test/java/org/apache/raft/netty/TestRaftReconfigurationWithNetty.java
+++ b/raft-netty/src/test/java/org/apache/raft/netty/TestRaftReconfigurationWithNetty.java
@@ -18,7 +18,7 @@
 package org.apache.raft.netty;
 
 import org.apache.raft.MiniRaftCluster;
-import org.apache.raft.server.RaftReconfigurationBaseTest;
+import org.apache.raft.server.impl.RaftReconfigurationBaseTest;
 
 import java.io.IOException;
 

http://git-wip-us.apache.org/repos/asf/incubator-ratis/blob/c36810ed/raft-netty/src/test/java/org/apache/raft/netty/TestRaftWithNetty.java
----------------------------------------------------------------------
diff --git a/raft-netty/src/test/java/org/apache/raft/netty/TestRaftWithNetty.java b/raft-netty/src/test/java/org/apache/raft/netty/TestRaftWithNetty.java
index 916c374..fb75b7b 100644
--- a/raft-netty/src/test/java/org/apache/raft/netty/TestRaftWithNetty.java
+++ b/raft-netty/src/test/java/org/apache/raft/netty/TestRaftWithNetty.java
@@ -20,8 +20,8 @@ package org.apache.raft.netty;
 import org.apache.log4j.Level;
 import org.apache.raft.RaftBasicTests;
 import org.apache.raft.client.RaftClient;
-import org.apache.raft.server.BlockRequestHandlingInjection;
-import org.apache.raft.server.RaftServer;
+import org.apache.raft.server.impl.BlockRequestHandlingInjection;
+import org.apache.raft.server.impl.RaftServer;
 import org.apache.raft.util.RaftUtils;
 import org.junit.Test;
 

http://git-wip-us.apache.org/repos/asf/incubator-ratis/blob/c36810ed/raft-server/src/main/java/org/apache/raft/server/ConfigurationManager.java
----------------------------------------------------------------------
diff --git a/raft-server/src/main/java/org/apache/raft/server/ConfigurationManager.java b/raft-server/src/main/java/org/apache/raft/server/ConfigurationManager.java
deleted file mode 100644
index 405cf2f..0000000
--- a/raft-server/src/main/java/org/apache/raft/server/ConfigurationManager.java
+++ /dev/null
@@ -1,91 +0,0 @@
-/**
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- *     http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-package org.apache.raft.server;
-
-import com.google.common.annotations.VisibleForTesting;
-import com.google.common.base.Preconditions;
-
-import java.util.*;
-
-/**
- * Maintain the mappings between log index and corresponding raft configuration.
- * Initialized when starting the raft peer. The mappings are loaded from the
- * raft log, and updated while appending/truncating configuration related log
- * entries.
- */
-public class ConfigurationManager {
-  private RaftConfiguration initialConf;
-  private final NavigableMap<Long, RaftConfiguration> configurations =
-      new TreeMap<>();
-  /**
-   * The current raft configuration. If configurations is not empty, should be
-   * the last entry of the map. Otherwise is initialConf.
-   */
-  private RaftConfiguration currentConf;
-
-  public ConfigurationManager(RaftConfiguration initialConf) {
-    setInitialConf(initialConf);
-  }
-
-  @VisibleForTesting
-  public synchronized void setInitialConf(RaftConfiguration initialConf) {
-    /**
-     * initialConf should actually be defined as "final". But for tests we want
-     * to change the initial configuration to reflect the correct port binding.
-     */
-    this.initialConf = initialConf;
-    this.currentConf = initialConf;
-  }
-
-  public synchronized void addConfiguration(long logIndex,
-      RaftConfiguration conf) {
-    Preconditions.checkArgument(configurations.isEmpty() ||
-        configurations.lastEntry().getKey() < logIndex);
-    configurations.put(logIndex, conf);
-    this.currentConf = conf;
-  }
-
-  synchronized RaftConfiguration getCurrent() {
-    return currentConf;
-  }
-
-  /**
-   * Remove all the configurations whose log index is >= the given index.
-   * @param index The given index. All the configurations whose log index is >=
-   *              this value will be removed.
-   * @return The configuration with largest log index < the given index.
-   */
-  synchronized RaftConfiguration removeConfigurations(long index) {
-    SortedMap<Long, RaftConfiguration> toRemove = configurations.tailMap(index);
-    for (Iterator<Map.Entry<Long, RaftConfiguration>> iter =
-         toRemove.entrySet().iterator(); iter.hasNext();) {
-      iter.next();
-      iter.remove();
-    }
-    currentConf = configurations.isEmpty() ? initialConf :
-        configurations.lastEntry().getValue();
-    return currentConf;
-  }
-
-  @VisibleForTesting
-  synchronized int numOfConf() {
-    return 1 + configurations.size();
-  }
-
-  // TODO: remove Configuration entries after they are committed
-}

http://git-wip-us.apache.org/repos/asf/incubator-ratis/blob/c36810ed/raft-server/src/main/java/org/apache/raft/server/FollowerInfo.java
----------------------------------------------------------------------
diff --git a/raft-server/src/main/java/org/apache/raft/server/FollowerInfo.java b/raft-server/src/main/java/org/apache/raft/server/FollowerInfo.java
deleted file mode 100644
index 8e5c131..0000000
--- a/raft-server/src/main/java/org/apache/raft/server/FollowerInfo.java
+++ /dev/null
@@ -1,103 +0,0 @@
-/**
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- *     http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-package org.apache.raft.server;
-
-import org.apache.raft.protocol.RaftPeer;
-import org.apache.raft.util.Timestamp;
-
-import java.util.concurrent.atomic.AtomicLong;
-import java.util.concurrent.atomic.AtomicReference;
-
-public class FollowerInfo {
-  private final RaftPeer peer;
-  private final AtomicReference<Timestamp> lastRpcResponseTime;
-  private final AtomicReference<Timestamp> lastRpcSendTime;
-  private long nextIndex;
-  private final AtomicLong matchIndex;
-  private volatile boolean attendVote;
-
-  FollowerInfo(RaftPeer peer, Timestamp lastRpcTime, long nextIndex,
-      boolean attendVote) {
-    this.peer = peer;
-    this.lastRpcResponseTime = new AtomicReference<>(lastRpcTime);
-    this.lastRpcSendTime = new AtomicReference<>(lastRpcTime);
-    this.nextIndex = nextIndex;
-    this.matchIndex = new AtomicLong(0);
-    this.attendVote = attendVote;
-  }
-
-  public void updateMatchIndex(final long matchIndex) {
-    this.matchIndex.set(matchIndex);
-  }
-
-  long getMatchIndex() {
-    return matchIndex.get();
-  }
-
-  public synchronized long getNextIndex() {
-    return nextIndex;
-  }
-
-  public synchronized void updateNextIndex(long i) {
-    nextIndex = i;
-  }
-
-  public synchronized void decreaseNextIndex(long targetIndex) {
-    if (nextIndex > 0) {
-      nextIndex = Math.min(nextIndex - 1, targetIndex);
-    }
-  }
-
-  @Override
-  public String toString() {
-    return peer.getId() + "(next=" + nextIndex + ", match=" + matchIndex + "," +
-        " attendVote=" + attendVote +
-        ", lastRpcSendTime=" + lastRpcSendTime +
-        ", lastRpcResponseTime=" + lastRpcResponseTime + ")";
-  }
-
-  void startAttendVote() {
-    attendVote = true;
-  }
-
-  public boolean isAttendingVote() {
-    return attendVote;
-  }
-
-  public RaftPeer getPeer() {
-    return peer;
-  }
-
-  /** Update lastRpcResponseTime to the current time. */
-  public void updateLastRpcResponseTime() {
-    lastRpcResponseTime.set(new Timestamp());
-  }
-
-  public Timestamp getLastRpcResponseTime() {
-    return lastRpcResponseTime.get();
-  }
-
-  /** Update lastRpcSendTime to the current time. */
-  public void updateLastRpcSendTime() {
-    lastRpcSendTime.set(new Timestamp());
-  }
-
-  public Timestamp getLastRpcTime() {
-    return Timestamp.latest(lastRpcResponseTime.get(), lastRpcSendTime.get());
-  }
-}

http://git-wip-us.apache.org/repos/asf/incubator-ratis/blob/c36810ed/raft-server/src/main/java/org/apache/raft/server/FollowerState.java
----------------------------------------------------------------------
diff --git a/raft-server/src/main/java/org/apache/raft/server/FollowerState.java b/raft-server/src/main/java/org/apache/raft/server/FollowerState.java
deleted file mode 100644
index e5293e4..0000000
--- a/raft-server/src/main/java/org/apache/raft/server/FollowerState.java
+++ /dev/null
@@ -1,91 +0,0 @@
-/**
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- *     http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-package org.apache.raft.server;
-
-import org.apache.raft.util.Daemon;
-import org.apache.raft.util.Timestamp;
-import org.slf4j.Logger;
-
-/**
- * Used when the peer is a follower. Used to track the election timeout.
- */
-class FollowerState extends Daemon {
-  static final Logger LOG = RaftServer.LOG;
-
-  private final RaftServer server;
-
-  private volatile Timestamp lastRpcTime = new Timestamp();
-  private volatile boolean monitorRunning = true;
-  private volatile boolean inLogSync = false;
-
-  FollowerState(RaftServer server) {
-    this.server = server;
-  }
-
-  void updateLastRpcTime(boolean inLogSync) {
-    lastRpcTime = new Timestamp();
-    LOG.trace("{} update last rpc time to {}", server.getId(), lastRpcTime);
-    this.inLogSync = inLogSync;
-  }
-
-  Timestamp getLastRpcTime() {
-    return lastRpcTime;
-  }
-
-  boolean shouldWithholdVotes() {
-    return lastRpcTime.elapsedTimeMs() < server.getMinTimeoutMs();
-  }
-
-  void stopRunning() {
-    this.monitorRunning = false;
-  }
-
-  @Override
-  public  void run() {
-    while (monitorRunning && server.isFollower()) {
-      final long electionTimeout = server.getRandomTimeoutMs();
-      try {
-        Thread.sleep(electionTimeout);
-        if (!monitorRunning || !server.isFollower()) {
-          LOG.info("{} heartbeat monitor quit", server.getId());
-          break;
-        }
-        synchronized (server) {
-          if (!inLogSync && lastRpcTime.elapsedTimeMs() >= electionTimeout) {
-            LOG.info("{} changes to {}, lastRpcTime:{}, electionTimeout:{}",
-                server.getId(), Role.CANDIDATE, lastRpcTime, electionTimeout);
-            // election timeout, should become a candidate
-            server.changeToCandidate();
-            break;
-          }
-        }
-      } catch (InterruptedException e) {
-        LOG.info(this + " was interrupted: " + e);
-        LOG.trace("TRACE", e);
-        return;
-      } catch (Exception e) {
-        LOG.warn(this + " caught an exception", e);
-      }
-    }
-  }
-
-  @Override
-  public String toString() {
-    return server.getId() + ": " + getClass().getSimpleName();
-  }
-}

http://git-wip-us.apache.org/repos/asf/incubator-ratis/blob/c36810ed/raft-server/src/main/java/org/apache/raft/server/LeaderElection.java
----------------------------------------------------------------------
diff --git a/raft-server/src/main/java/org/apache/raft/server/LeaderElection.java b/raft-server/src/main/java/org/apache/raft/server/LeaderElection.java
deleted file mode 100644
index 8a97494..0000000
--- a/raft-server/src/main/java/org/apache/raft/server/LeaderElection.java
+++ /dev/null
@@ -1,242 +0,0 @@
-/**
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- *     http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-package org.apache.raft.server;
-
-import com.google.common.base.Preconditions;
-import com.google.common.util.concurrent.ThreadFactoryBuilder;
-import org.apache.raft.protocol.RaftPeer;
-import org.apache.raft.server.impl.ServerProtoUtils;
-import org.apache.raft.server.protocol.TermIndex;
-import org.apache.raft.shaded.proto.RaftProtos.RequestVoteReplyProto;
-import org.apache.raft.shaded.proto.RaftProtos.RequestVoteRequestProto;
-import org.apache.raft.statemachine.SnapshotInfo;
-import org.apache.raft.util.Daemon;
-import org.apache.raft.util.ProtoUtils;
-import org.apache.raft.util.Timestamp;
-import org.slf4j.Logger;
-import org.slf4j.LoggerFactory;
-
-import java.io.IOException;
-import java.util.ArrayList;
-import java.util.Collection;
-import java.util.List;
-import java.util.concurrent.*;
-import java.util.stream.Collectors;
-
-class LeaderElection extends Daemon {
-  public static final Logger LOG = LoggerFactory.getLogger(LeaderElection.class);
-
-  private ResultAndTerm logAndReturn(Result result,
-      List<RequestVoteReplyProto> responses,
-      List<Exception> exceptions, long newTerm) {
-    LOG.info(server.getId() + ": Election " + result + "; received "
-        + responses.size() + " response(s) "
-        + responses.stream().map(r -> ProtoUtils.toString(r)).collect(Collectors.toList())
-        + " and " + exceptions.size() + " exception(s); " + server.getState());
-    int i = 0;
-    for(Exception e : exceptions) {
-      LOG.info("  " + i++ + ": " + e);
-      LOG.trace("TRACE", e);
-    }
-    return new ResultAndTerm(result, newTerm);
-  }
-
-  enum Result {PASSED, REJECTED, TIMEOUT, DISCOVERED_A_NEW_TERM, SHUTDOWN}
-
-  private static class ResultAndTerm {
-    final Result result;
-    final long term;
-
-    ResultAndTerm(Result result, long term) {
-      this.result = result;
-      this.term = term;
-    }
-  }
-
-  private final RaftServer server;
-  private ExecutorCompletionService<RequestVoteReplyProto> service;
-  private ExecutorService executor;
-  private volatile boolean running;
-  /**
-   * The Raft configuration should not change while the peer is in candidate
-   * state. If the configuration changes, another peer should be acting as a
-   * leader and this LeaderElection session should end.
-   */
-  private final RaftConfiguration conf;
-  private final Collection<RaftPeer> others;
-
-  LeaderElection(RaftServer server) {
-    this.server = server;
-    conf = server.getRaftConf();
-    others = conf.getOtherPeers(server.getId());
-    this.running = true;
-  }
-
-  void stopRunning() {
-    this.running = false;
-  }
-
-  private void initExecutor() {
-    Preconditions.checkState(!others.isEmpty());
-    executor = Executors.newFixedThreadPool(others.size(),
-        new ThreadFactoryBuilder().setDaemon(true).build());
-    service = new ExecutorCompletionService<>(executor);
-  }
-
-  @Override
-  public void run() {
-    try {
-      askForVotes();
-    } catch (InterruptedException e) {
-      // the leader election thread is interrupted. The peer may already step
-      // down to a follower. The leader election should skip.
-      LOG.info("The leader election thread of peer {} is interrupted. " +
-          "Currently role: {}.", server.getId(), server.getRole());
-    } catch (IOException e) {
-      LOG.warn("Failed to persist votedFor/term. Exit the leader election.", e);
-      stopRunning();
-    }
-  }
-
-  /**
-   * After a peer changes its role to candidate, it invokes this method to
-   * send out requestVote rpc to all other peers.
-   */
-  private void askForVotes() throws InterruptedException, IOException {
-    final ServerState state = server.getState();
-    while (running && server.isCandidate()) {
-      // one round of requestVotes
-      final long electionTerm;
-      synchronized (server) {
-        electionTerm = state.initElection();
-        server.getState().persistMetadata();
-      }
-      LOG.info(state.getSelfId() + ": begin an election in Term "
-          + electionTerm);
-
-      TermIndex lastEntry = ServerProtoUtils.toTermIndex(
-          state.getLog().getLastEntry());
-      if (lastEntry == null) {
-        // lastEntry may need to be derived from snapshot
-        SnapshotInfo snapshot = state.getLatestSnapshot();
-        if (snapshot != null) {
-          lastEntry = snapshot.getTermIndex();
-        }
-      }
-
-      final ResultAndTerm r;
-      if (others.isEmpty()) {
-        r = new ResultAndTerm(Result.PASSED, electionTerm);
-      } else {
-        try {
-          initExecutor();
-          int submitted = submitRequests(electionTerm, lastEntry);
-          r = waitForResults(electionTerm, submitted);
-        } finally {
-          if (executor != null) {
-            executor.shutdown();
-          }
-        }
-      }
-
-      synchronized (server) {
-        if (electionTerm != state.getCurrentTerm() || !running ||
-            !server.isCandidate()) {
-          return; // term already passed or no longer a candidate.
-        }
-
-        switch (r.result) {
-          case PASSED:
-            server.changeToLeader();
-            return;
-          case SHUTDOWN:
-            LOG.info("{} received shutdown response when requesting votes.",
-                server.getId());
-            server.close();
-            return;
-          case REJECTED:
-          case DISCOVERED_A_NEW_TERM:
-            final long term = r.term > server.getState().getCurrentTerm() ?
-                r.term : server.getState().getCurrentTerm();
-            server.changeToFollower(term, true);
-            return;
-          case TIMEOUT:
-            // should start another election
-        }
-      }
-    }
-  }
-
-  private int submitRequests(final long electionTerm, final TermIndex lastEntry) {
-    int submitted = 0;
-    for (final RaftPeer peer : others) {
-      final RequestVoteRequestProto r = server.createRequestVoteRequest(
-          peer.getId(), electionTerm, lastEntry);
-      service.submit(
-          () -> server.getServerRpc().sendRequestVote(r));
-      submitted++;
-    }
-    return submitted;
-  }
-
-  private ResultAndTerm waitForResults(final long electionTerm,
-      final int submitted) throws InterruptedException {
-    final Timestamp timeout = new Timestamp().addTimeMs(server.getRandomTimeoutMs());
-    final List<RequestVoteReplyProto> responses = new ArrayList<>();
-    final List<Exception> exceptions = new ArrayList<>();
-    int waitForNum = submitted;
-    Collection<String> votedPeers = new ArrayList<>();
-    while (waitForNum > 0 && running && server.isCandidate()) {
-      final long waitTime = -timeout.elapsedTimeMs();
-      if (waitTime <= 0) {
-        return logAndReturn(Result.TIMEOUT, responses, exceptions, -1);
-      }
-
-      try {
-        final Future<RequestVoteReplyProto> future = service.poll(
-            waitTime, TimeUnit.MILLISECONDS);
-        if (future == null) {
-          continue; // poll timeout, continue to return Result.TIMEOUT
-        }
-
-        final RequestVoteReplyProto r = future.get();
-        responses.add(r);
-        if (r.getShouldShutdown()) {
-          return logAndReturn(Result.SHUTDOWN, responses, exceptions, -1);
-        }
-        if (r.getTerm() > electionTerm) {
-          return logAndReturn(Result.DISCOVERED_A_NEW_TERM, responses,
-              exceptions, r.getTerm());
-        }
-        if (r.getServerReply().getSuccess()) {
-          votedPeers.add(r.getServerReply().getReplyId());
-          if (conf.hasMajority(votedPeers, server.getId())) {
-            return logAndReturn(Result.PASSED, responses, exceptions, -1);
-          }
-        }
-      } catch(ExecutionException e) {
-        LOG.info("Got exception when requesting votes: " + e);
-        LOG.trace("TRACE", e);
-        exceptions.add(e);
-      }
-      waitForNum--;
-    }
-    // received all the responses
-    return logAndReturn(Result.REJECTED, responses, exceptions, -1);
-  }
-}

http://git-wip-us.apache.org/repos/asf/incubator-ratis/blob/c36810ed/raft-server/src/main/java/org/apache/raft/server/LeaderState.java
----------------------------------------------------------------------
diff --git a/raft-server/src/main/java/org/apache/raft/server/LeaderState.java b/raft-server/src/main/java/org/apache/raft/server/LeaderState.java
deleted file mode 100644
index 1576311..0000000
--- a/raft-server/src/main/java/org/apache/raft/server/LeaderState.java
+++ /dev/null
@@ -1,581 +0,0 @@
-/**
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- *     http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-package org.apache.raft.server;
-
-import com.google.common.base.Preconditions;
-import org.apache.raft.conf.RaftProperties;
-import org.apache.raft.protocol.*;
-import org.apache.raft.server.storage.RaftLog;
-import org.apache.raft.shaded.proto.RaftProtos.LeaderNoOp;
-import org.apache.raft.shaded.proto.RaftProtos.LogEntryProto;
-import org.apache.raft.statemachine.TransactionContext;
-import org.apache.raft.util.Daemon;
-import org.apache.raft.util.ProtoUtils;
-import org.apache.raft.util.Timestamp;
-import org.slf4j.Logger;
-
-import java.io.IOException;
-import java.util.*;
-import java.util.concurrent.ArrayBlockingQueue;
-import java.util.concurrent.BlockingQueue;
-import java.util.concurrent.CompletableFuture;
-import java.util.concurrent.TimeUnit;
-import java.util.stream.Collectors;
-
-import static org.apache.raft.server.LeaderState.StateUpdateEventType.*;
-import static org.apache.raft.server.RaftServerConfigKeys.*;
-
-/**
- * States for leader only. It contains three different types of processors:
- * 1. RPC senders: each thread is appending log to a follower
- * 2. EventProcessor: a single thread updating the raft server's state based on
- *                    status of log appending response
- * 3. PendingRequestHandler: a handler sending back responses to clients when
- *                           corresponding log entries are committed
- */
-public class LeaderState {
-  private static final Logger LOG = RaftServer.LOG;
-
-  enum StateUpdateEventType {
-    STEPDOWN, UPDATECOMMIT, STAGINGPROGRESS
-  }
-
-  enum BootStrapProgress {
-    NOPROGRESS, PROGRESSING, CAUGHTUP
-  }
-
-  static class StateUpdateEvent {
-    final StateUpdateEventType type;
-    final long newTerm;
-
-    StateUpdateEvent(StateUpdateEventType type, long newTerm) {
-      this.type = type;
-      this.newTerm = newTerm;
-    }
-  }
-
-  static final StateUpdateEvent UPDATE_COMMIT_EVENT =
-      new StateUpdateEvent(StateUpdateEventType.UPDATECOMMIT, -1);
-  static final StateUpdateEvent STAGING_PROGRESS_EVENT =
-      new StateUpdateEvent(StateUpdateEventType.STAGINGPROGRESS, -1);
-
-  private final RaftServer server;
-  private final RaftLog raftLog;
-  private final long currentTerm;
-  private volatile ConfigurationStagingState stagingState;
-  private List<List<FollowerInfo>> voterLists;
-
-  /**
-   * The list of threads appending entries to followers.
-   * The list is protected by the RaftServer's lock.
-   */
-  private final List<LogAppender> senders;
-  private final BlockingQueue<StateUpdateEvent> eventQ;
-  private final EventProcessor processor;
-  private final PendingRequests pendingRequests;
-  private volatile boolean running = true;
-
-  private final int stagingCatchupGap;
-  private final int snapshotChunkMaxSize;
-  private final int syncInterval;
-
-  LeaderState(RaftServer server, RaftProperties properties) {
-    this.server = server;
-
-    stagingCatchupGap = properties.getInt(
-        RAFT_SERVER_STAGING_CATCHUP_GAP_KEY,
-        RAFT_SERVER_STAGING_CATCHUP_GAP_DEFAULT);
-    snapshotChunkMaxSize = properties.getInt(
-        RAFT_SNAPSHOT_CHUNK_MAX_SIZE_KEY,
-        RAFT_SNAPSHOT_CHUNK_MAX_SIZE_DEFAULT);
-    syncInterval = properties.getInt(
-        RAFT_SERVER_RPC_SLEEP_TIME_MS_KEY,
-        RAFT_SERVER_RPC_SLEEP_TIME_MS_DEFAULT);
-
-    final ServerState state = server.getState();
-    this.raftLog = state.getLog();
-    this.currentTerm = state.getCurrentTerm();
-    eventQ = new ArrayBlockingQueue<>(4096);
-    processor = new EventProcessor();
-    pendingRequests = new PendingRequests(server);
-
-    final RaftConfiguration conf = server.getRaftConf();
-    Collection<RaftPeer> others = conf.getOtherPeers(state.getSelfId());
-    final Timestamp t = new Timestamp().addTimeMs(-server.getMaxTimeoutMs());
-    final long nextIndex = raftLog.getNextIndex();
-    senders = new ArrayList<>(others.size());
-    for (RaftPeer p : others) {
-      FollowerInfo f = new FollowerInfo(p, t, nextIndex, true);
-      senders.add(server.getLogAppenderFactory().getLogAppender(server, this, f));
-    }
-    voterLists = divideFollowers(conf);
-  }
-
-  void start() {
-    // In the beginning of the new term, replicate an empty entry in order
-    // to finally commit entries in the previous term.
-    // Also this message can help identify the last committed index when
-    // the leader peer is just started.
-    final LogEntryProto placeHolder = LogEntryProto.newBuilder()
-        .setTerm(server.getState().getCurrentTerm())
-        .setIndex(raftLog.getNextIndex())
-        .setNoOp(LeaderNoOp.newBuilder()).build();
-    raftLog.append(placeHolder);
-
-    processor.start();
-    startSenders();
-  }
-
-  private void startSenders() {
-    senders.forEach(Thread::start);
-  }
-
-  void stop() {
-    this.running = false;
-    // do not interrupt event processor since it may be in the middle of logSync
-    for (LogAppender sender : senders) {
-      sender.stopSender();
-      sender.interrupt();
-    }
-    try {
-      pendingRequests.sendNotLeaderResponses();
-    } catch (IOException e) {
-      LOG.warn("Caught exception in sendNotLeaderResponses", e);
-    }
-  }
-
-  void notifySenders() {
-    senders.forEach(LogAppender::notifyAppend);
-  }
-
-  boolean inStagingState() {
-    return stagingState != null;
-  }
-
-  ConfigurationStagingState getStagingState() {
-    return stagingState;
-  }
-
-  long getCurrentTerm() {
-    return currentTerm;
-  }
-
-  int getSnapshotChunkMaxSize() {
-    return snapshotChunkMaxSize;
-  }
-
-  int getSyncInterval() {
-    return syncInterval;
-  }
-
-  /**
-   * Start bootstrapping new peers
-   */
-  PendingRequest startSetConfiguration(SetConfigurationRequest request) {
-    Preconditions.checkState(running && !inStagingState());
-
-    RaftPeer[] peersInNewConf = request.getPeersInNewConf();
-    Collection<RaftPeer> peersToBootStrap = RaftConfiguration
-        .computeNewPeers(peersInNewConf, server.getRaftConf());
-
-    // add the request to the pending queue
-    final PendingRequest pending = pendingRequests.addConfRequest(request);
-
-    ConfigurationStagingState stagingState = new ConfigurationStagingState(
-        peersToBootStrap, new PeerConfiguration(Arrays.asList(peersInNewConf)));
-    Collection<RaftPeer> newPeers = stagingState.getNewPeers();
-    // set the staging state
-    this.stagingState = stagingState;
-
-    if (newPeers.isEmpty()) {
-      applyOldNewConf();
-    } else {
-      // update the LeaderState's sender list
-      addSenders(newPeers);
-    }
-    return pending;
-  }
-
-  PendingRequest addPendingRequest(long index, RaftClientRequest request,
-      TransactionContext entry) {
-    return pendingRequests.addPendingRequest(index, request, entry);
-  }
-
-  private void applyOldNewConf() {
-    final ServerState state = server.getState();
-    final RaftConfiguration current = server.getRaftConf();
-    final RaftConfiguration oldNewConf= stagingState.generateOldNewConf(current,
-        state.getLog().getNextIndex());
-    // apply the (old, new) configuration to log, and use it as the current conf
-    long index = state.getLog().append(state.getCurrentTerm(), oldNewConf);
-    updateConfiguration(index, oldNewConf);
-
-    this.stagingState = null;
-    notifySenders();
-  }
-
-  private void updateConfiguration(long logIndex, RaftConfiguration newConf) {
-    voterLists = divideFollowers(newConf);
-    server.getState().setRaftConf(logIndex, newConf);
-  }
-
-  /**
-   * After receiving a setConfiguration request, the leader should update its
-   * RpcSender list.
-   */
-  void addSenders(Collection<RaftPeer> newMembers) {
-    final Timestamp t = new Timestamp().addTimeMs(-server.getMaxTimeoutMs());
-    final long nextIndex = raftLog.getNextIndex();
-    for (RaftPeer peer : newMembers) {
-      FollowerInfo f = new FollowerInfo(peer, t, nextIndex, false);
-      LogAppender sender = server.getLogAppenderFactory()
-          .getLogAppender(server, this, f);
-      senders.add(sender);
-      sender.start();
-    }
-  }
-
-  /**
-   * Update the RpcSender list based on the current configuration
-   */
-  private void updateSenders(RaftConfiguration conf) {
-    Preconditions.checkState(conf.isStable() && !inStagingState());
-    Iterator<LogAppender> iterator = senders.iterator();
-    while (iterator.hasNext()) {
-      LogAppender sender = iterator.next();
-      if (!conf.containsInConf(sender.getFollower().getPeer().getId())) {
-        iterator.remove();
-        sender.stopSender();
-        sender.interrupt();
-      }
-    }
-  }
-
-  void submitUpdateStateEvent(StateUpdateEvent event) {
-    try {
-      eventQ.put(event);
-    } catch (InterruptedException e) {
-      LOG.info("Interrupted when adding event {} into the queue", event);
-    }
-  }
-
-  private void prepare() {
-    synchronized (server) {
-      if (running) {
-        final RaftConfiguration conf = server.getRaftConf();
-        if (conf.isTransitional() && server.getState().isConfCommitted()) {
-          // the configuration is in transitional state, and has been committed
-          // so it is time to generate and replicate (new) conf.
-          replicateNewConf();
-        }
-      }
-    }
-  }
-
-  /**
-   * The processor thread takes the responsibility to update the raft server's
-   * state, such as changing to follower, or updating the committed index.
-   */
-  private class EventProcessor extends Daemon {
-    @Override
-    public void run() {
-      // apply an empty message; check if necessary to replicate (new) conf
-      prepare();
-
-      while (running) {
-        try {
-          StateUpdateEvent event = eventQ.poll(server.getMaxTimeoutMs(),
-              TimeUnit.MILLISECONDS);
-          synchronized (server) {
-            if (running) {
-              handleEvent(event);
-            }
-          }
-          // the updated configuration does not need to be sync'ed here
-        } catch (InterruptedException e) {
-          if (!running) {
-            LOG.info("The LeaderState gets is stopped");
-          } else {
-            LOG.warn("The leader election thread of peer {} is interrupted. "
-                + "Currently role: {}.", server.getId(), server.getRole());
-            throw new RuntimeException(e);
-          }
-        } catch (IOException e) {
-          LOG.warn("Failed to persist new votedFor/term.", e);
-          // the failure should happen while changing the state to follower
-          // thus the in-memory state should have been updated
-          Preconditions.checkState(!running);
-        }
-      }
-    }
-  }
-
-  private void handleEvent(StateUpdateEvent e) throws IOException {
-    if (e == null) {
-      if (inStagingState()) {
-        checkNewPeers();
-      }
-    } else {
-      if (e.type == STEPDOWN) {
-        server.changeToFollower(e.newTerm, true);
-      } else if (e.type == UPDATECOMMIT) {
-        updateLastCommitted();
-      } else if (e.type == STAGINGPROGRESS) {
-        checkNewPeers();
-      }
-    }
-  }
-
-  /**
-   * So far we use a simple implementation for catchup checking:
-   * 1. If the latest rpc time of the remote peer is before 3 * max_timeout,
-   *    the peer made no progress for that long. We should fail the whole
-   *    setConfiguration request.
-   * 2. If the peer's matching index is just behind for a small gap, and the
-   *    peer was updated recently (within max_timeout), declare the peer as
-   *    caught-up.
-   * 3. Otherwise the peer is making progressing. Keep waiting.
-   */
-  private BootStrapProgress checkProgress(FollowerInfo follower,
-      long committed) {
-    Preconditions.checkArgument(!follower.isAttendingVote());
-    final Timestamp progressTime = new Timestamp().addTimeMs(-server.getMaxTimeoutMs());
-    final Timestamp timeoutTime = new Timestamp().addTimeMs(-3*server.getMaxTimeoutMs());
-    if (follower.getLastRpcResponseTime().compareTo(timeoutTime) < 0) {
-      LOG.debug("{} detects a follower {} timeout for bootstrapping," +
-              " timeoutTime: {}", server.getId(), follower, timeoutTime);
-      return BootStrapProgress.NOPROGRESS;
-    } else if (follower.getMatchIndex() + stagingCatchupGap > committed
-        && follower.getLastRpcResponseTime().compareTo(progressTime) > 0) {
-      return BootStrapProgress.CAUGHTUP;
-    } else {
-      return BootStrapProgress.PROGRESSING;
-    }
-  }
-
-  private Collection<BootStrapProgress> checkAllProgress(long committed) {
-    Preconditions.checkState(inStagingState());
-    return senders.stream()
-        .filter(sender -> !sender.getFollower().isAttendingVote())
-        .map(sender -> checkProgress(sender.getFollower(), committed))
-        .collect(Collectors.toCollection(ArrayList::new));
-  }
-
-  private void checkNewPeers() {
-    if (!inStagingState()) {
-      // it is possible that the bootstrapping is done and we still have
-      // remaining STAGINGPROGRESS event to handle.
-      updateLastCommitted();
-    } else {
-      final long committedIndex = server.getState().getLog()
-          .getLastCommittedIndex();
-      Collection<BootStrapProgress> reports = checkAllProgress(committedIndex);
-      if (reports.contains(BootStrapProgress.NOPROGRESS)) {
-        LOG.debug("{} fails the setConfiguration request", server.getId());
-        stagingState.fail();
-      } else if (!reports.contains(BootStrapProgress.PROGRESSING)) {
-        // all caught up!
-        applyOldNewConf();
-        for (LogAppender sender : senders) {
-          sender.getFollower().startAttendVote();
-        }
-      }
-    }
-  }
-
-  boolean isBootStrappingPeer(String peerId) {
-    return inStagingState() && getStagingState().contains(peerId);
-  }
-
-  private void updateLastCommitted() {
-    final String selfId = server.getId();
-    final RaftConfiguration conf = server.getRaftConf();
-    long majorityInNewConf = computeLastCommitted(voterLists.get(0),
-        conf.containsInConf(selfId));
-    final long oldLastCommitted = raftLog.getLastCommittedIndex();
-    final LogEntryProto[] entriesToCommit;
-    if (!conf.isTransitional()) {
-      // copy the entries that may get committed out of the raftlog, to prevent
-      // the possible race that the log gets purged after the statemachine does
-      // a snapshot
-      entriesToCommit = raftLog.getEntries(oldLastCommitted + 1,
-          Math.max(majorityInNewConf, oldLastCommitted) + 1);
-      server.getState().updateStatemachine(majorityInNewConf, currentTerm);
-    } else { // configuration is in transitional state
-      long majorityInOldConf = computeLastCommitted(voterLists.get(1),
-          conf.containsInOldConf(selfId));
-      final long majority = Math.min(majorityInNewConf, majorityInOldConf);
-      entriesToCommit = raftLog.getEntries(oldLastCommitted + 1,
-          Math.max(majority, oldLastCommitted) + 1);
-      server.getState().updateStatemachine(majority, currentTerm);
-    }
-    checkAndUpdateConfiguration(entriesToCommit);
-  }
-
-  private boolean committedConf(LogEntryProto[] entries) {
-    final long currentCommitted = raftLog.getLastCommittedIndex();
-    for (LogEntryProto entry : entries) {
-      if (entry.getIndex() <= currentCommitted &&
-          ProtoUtils.isConfigurationLogEntry(entry)) {
-        return true;
-      }
-    }
-    return false;
-  }
-
-  private void checkAndUpdateConfiguration(LogEntryProto[] entriesToCheck) {
-    final RaftConfiguration conf = server.getRaftConf();
-    if (committedConf(entriesToCheck)) {
-      if (conf.isTransitional()) {
-        replicateNewConf();
-      } else { // the (new) log entry has been committed
-        LOG.debug("{} sends success to setConfiguration request", server.getId());
-        pendingRequests.replySetConfiguration();
-        // if the leader is not included in the current configuration, step down
-        if (!conf.containsInConf(server.getId())) {
-          LOG.info("{} is not included in the new configuration {}. Step down.",
-              server.getId(), conf);
-          try {
-            // leave some time for all RPC senders to send out new conf entry
-            Thread.sleep(server.getMinTimeoutMs());
-          } catch (InterruptedException ignored) {
-          }
-          // the pending request handler will send NotLeaderException for
-          // pending client requests when it stops
-          server.close();
-        }
-      }
-    }
-  }
-
-  /**
-   * when the (old, new) log entry has been committed, should replicate (new):
-   * 1) append (new) to log
-   * 2) update conf to (new)
-   * 3) update RpcSenders list
-   * 4) start replicating the log entry
-   */
-  private void replicateNewConf() {
-    final RaftConfiguration conf = server.getRaftConf();
-    final RaftConfiguration newConf = RaftConfiguration.newBuilder()
-        .setConf(conf)
-        .setLogEntryIndex(raftLog.getNextIndex())
-        .build();
-    // stop the LogAppender if the corresponding follower is no longer in the conf
-    updateSenders(newConf);
-    long index = raftLog.append(server.getState().getCurrentTerm(), newConf);
-    updateConfiguration(index, newConf);
-    notifySenders();
-  }
-
-  private long computeLastCommitted(List<FollowerInfo> followers,
-      boolean includeSelf) {
-    final int length = includeSelf ? followers.size() + 1 : followers.size();
-    final long[] indices = new long[length];
-    for (int i = 0; i < followers.size(); i++) {
-      indices[i] = followers.get(i).getMatchIndex();
-    }
-    if (includeSelf) {
-      // note that we also need to wait for the local disk I/O
-      indices[length - 1] = raftLog.getLatestFlushedIndex();
-    }
-
-    Arrays.sort(indices);
-    return indices[(indices.length - 1) / 2];
-  }
-
-  private List<List<FollowerInfo>> divideFollowers(RaftConfiguration conf) {
-    List<List<FollowerInfo>> lists = new ArrayList<>(2);
-    List<FollowerInfo> listForNew = senders.stream()
-        .filter(sender -> conf.containsInConf(sender.getFollower().getPeer().getId()))
-        .map(LogAppender::getFollower)
-        .collect(Collectors.toList());
-    lists.add(listForNew);
-    if (conf.isTransitional()) {
-      List<FollowerInfo> listForOld = senders.stream()
-          .filter(sender -> conf.containsInOldConf(sender.getFollower().getPeer().getId()))
-          .map(LogAppender::getFollower)
-          .collect(Collectors.toList());
-      lists.add(listForOld);
-    }
-    return lists;
-  }
-
-  PendingRequest returnNoConfChange(SetConfigurationRequest r) {
-    PendingRequest pending = new PendingRequest(r);
-    pending.setSuccessReply(null);
-    return pending;
-  }
-
-  void replyPendingRequest(long logIndex, CompletableFuture<Message> message) {
-    pendingRequests.replyPendingRequest(logIndex, message);
-  }
-
-  TransactionContext getTransactionContext(long index) {
-    return pendingRequests.getTransactionContext(index);
-  }
-
-  private class ConfigurationStagingState {
-    private final Map<String, RaftPeer> newPeers;
-    private final PeerConfiguration newConf;
-
-    ConfigurationStagingState(Collection<RaftPeer> newPeers,
-        PeerConfiguration newConf) {
-      Map<String, RaftPeer> map = new HashMap<>();
-      for (RaftPeer peer : newPeers) {
-        map.put(peer.getId(), peer);
-      }
-      this.newPeers = Collections.unmodifiableMap(map);
-      this.newConf = newConf;
-    }
-
-    RaftConfiguration generateOldNewConf(RaftConfiguration current,
-        long logIndex) {
-      return RaftConfiguration.newBuilder()
-          .setConf(newConf)
-          .setOldConf(current)
-          .setLogEntryIndex(logIndex)
-          .build();
-    }
-
-    Collection<RaftPeer> getNewPeers() {
-      return newPeers.values();
-    }
-
-    boolean contains(String peerId) {
-      return newPeers.containsKey(peerId);
-    }
-
-    void fail() {
-      Iterator<LogAppender> iterator = senders.iterator();
-      while (iterator.hasNext()) {
-        LogAppender sender = iterator.next();
-        if (!sender.getFollower().isAttendingVote()) {
-          iterator.remove();
-          sender.stopSender();
-          sender.interrupt();
-        }
-      }
-      LeaderState.this.stagingState = null;
-      // send back failure response to client's request
-      pendingRequests.failSetConfiguration(
-          new ReconfigurationTimeoutException("Fail to set configuration "
-              + newConf + ". Timeout when bootstrapping new peers."));
-    }
-  }
-}


[02/54] [abbrv] incubator-ratis git commit: Move o.a.r.s.* to o.a.r.s.impl.

Posted by ji...@apache.org.
http://git-wip-us.apache.org/repos/asf/incubator-ratis/blob/c36810ed/raft-server/src/main/java/org/apache/raft/server/impl/PendingRequests.java
----------------------------------------------------------------------
diff --git a/raft-server/src/main/java/org/apache/raft/server/impl/PendingRequests.java b/raft-server/src/main/java/org/apache/raft/server/impl/PendingRequests.java
new file mode 100644
index 0000000..ad6ecef
--- /dev/null
+++ b/raft-server/src/main/java/org/apache/raft/server/impl/PendingRequests.java
@@ -0,0 +1,129 @@
+/**
+ * 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.raft.server.impl;
+
+import com.google.common.base.Preconditions;
+import org.apache.raft.protocol.*;
+import org.apache.raft.statemachine.TransactionContext;
+import org.slf4j.Logger;
+
+import java.io.IOException;
+import java.util.Collection;
+import java.util.concurrent.CompletableFuture;
+import java.util.concurrent.ConcurrentHashMap;
+import java.util.concurrent.ConcurrentMap;
+import java.util.stream.Collectors;
+
+class PendingRequests {
+  private static final Logger LOG = RaftServer.LOG;
+
+  private PendingRequest pendingSetConf;
+  private final RaftServer server;
+  private final ConcurrentMap<Long, PendingRequest> pendingRequests = new ConcurrentHashMap<>();
+  private PendingRequest last = null;
+
+  PendingRequests(RaftServer server) {
+    this.server = server;
+  }
+
+  PendingRequest addPendingRequest(long index, RaftClientRequest request,
+      TransactionContext entry) {
+    // externally synced for now
+    Preconditions.checkArgument(!request.isReadOnly());
+    Preconditions.checkState(last == null || index == last.getIndex() + 1);
+    return add(index, request, entry);
+  }
+
+  private PendingRequest add(long index, RaftClientRequest request,
+      TransactionContext entry) {
+    final PendingRequest pending = new PendingRequest(index, request, entry);
+    pendingRequests.put(index, pending);
+    last = pending;
+    return pending;
+  }
+
+  PendingRequest addConfRequest(SetConfigurationRequest request) {
+    Preconditions.checkState(pendingSetConf == null);
+    pendingSetConf = new PendingRequest(request);
+    return pendingSetConf;
+  }
+
+  void replySetConfiguration() {
+    // we allow the pendingRequest to be null in case that the new leader
+    // commits the new configuration while it has not received the retry
+    // request from the client
+    if (pendingSetConf != null) {
+      // for setConfiguration we do not need to wait for statemachine. send back
+      // reply after it's committed.
+      pendingSetConf.setSuccessReply(null);
+      pendingSetConf = null;
+    }
+  }
+
+  void failSetConfiguration(RaftException e) {
+    Preconditions.checkState(pendingSetConf != null);
+    pendingSetConf.setException(e);
+    pendingSetConf = null;
+  }
+
+  TransactionContext getTransactionContext(long index) {
+    PendingRequest pendingRequest = pendingRequests.get(index);
+    // it is possible that the pendingRequest is null if this peer just becomes
+    // the new leader and commits transactions received by the previous leader
+    return pendingRequest != null ? pendingRequest.getEntry() : null;
+  }
+
+  void replyPendingRequest(long index, CompletableFuture<Message> messageFuture) {
+    final PendingRequest pending = pendingRequests.get(index);
+    if (pending != null) {
+      Preconditions.checkState(pending.getIndex() == index);
+
+      messageFuture.whenComplete((reply, exception) -> {
+        if (exception == null) {
+          pending.setSuccessReply(reply);
+        } else {
+          pending.setException(exception);
+        }
+      });
+    }
+  }
+
+  /**
+   * The leader state is stopped. Send NotLeaderException to all the pending
+   * requests since they have not got applied to the state machine yet.
+   */
+  void sendNotLeaderResponses() throws IOException {
+    LOG.info("{} sends responses before shutting down PendingRequestsHandler",
+        server.getId());
+
+    Collection<TransactionContext> pendingEntries = pendingRequests.values().stream()
+        .map(PendingRequest::getEntry).collect(Collectors.toList());
+    // notify the state machine about stepping down
+    server.getStateMachine().notifyNotLeader(pendingEntries);
+    pendingRequests.values().forEach(this::setNotLeaderException);
+    if (pendingSetConf != null) {
+      setNotLeaderException(pendingSetConf);
+    }
+  }
+
+  private void setNotLeaderException(PendingRequest pending) {
+    RaftClientReply reply = new RaftClientReply(pending.getRequest(),
+        server.generateNotLeaderException());
+    pending.setReply(reply);
+  }
+}

http://git-wip-us.apache.org/repos/asf/incubator-ratis/blob/c36810ed/raft-server/src/main/java/org/apache/raft/server/impl/RaftConfiguration.java
----------------------------------------------------------------------
diff --git a/raft-server/src/main/java/org/apache/raft/server/impl/RaftConfiguration.java b/raft-server/src/main/java/org/apache/raft/server/impl/RaftConfiguration.java
new file mode 100644
index 0000000..28ff330
--- /dev/null
+++ b/raft-server/src/main/java/org/apache/raft/server/impl/RaftConfiguration.java
@@ -0,0 +1,261 @@
+/**
+ * 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.raft.server.impl;
+
+import com.google.common.annotations.VisibleForTesting;
+import com.google.common.base.Preconditions;
+import org.apache.raft.protocol.RaftPeer;
+
+import java.util.*;
+import java.util.concurrent.ThreadLocalRandom;
+
+/**
+ * The configuration of the raft cluster.
+ *
+ * The configuration is stable if there is no on-going peer change. Otherwise,
+ * the configuration is transitional, i.e. in the middle of a peer change.
+ *
+ * The objects of this class are immutable.
+ */
+public class RaftConfiguration {
+  /** Create a {@link Builder}. */
+  public static Builder newBuilder() {
+    return new Builder();
+  }
+
+  /** To build {@link RaftConfiguration} objects. */
+  public static class Builder {
+    private PeerConfiguration oldConf;
+    private PeerConfiguration conf;
+    private long logEntryIndex = RaftServerConstants.INVALID_LOG_INDEX;
+
+    private boolean forceStable = false;
+    private boolean forceTransitional = false;
+
+    private Builder() {}
+
+    public Builder setConf(PeerConfiguration conf) {
+      Preconditions.checkNotNull(conf);
+      Preconditions.checkState(this.conf == null, "conf is already set.");
+      this.conf = conf;
+      return this;
+    }
+
+    public Builder setConf(Iterable<RaftPeer> peers) {
+      return setConf(new PeerConfiguration(peers));
+    }
+
+    public Builder setConf(RaftPeer[] peers) {
+      return setConf(Arrays.asList(peers));
+    }
+
+    Builder setConf(RaftConfiguration transitionalConf) {
+      Preconditions.checkNotNull(transitionalConf);
+      Preconditions.checkState(transitionalConf.isTransitional());
+
+      Preconditions.checkState(!forceTransitional);
+      forceStable = true;
+      return setConf(transitionalConf.conf);
+    }
+
+
+    public Builder setOldConf(PeerConfiguration oldConf) {
+      Preconditions.checkNotNull(oldConf);
+      Preconditions.checkState(this.oldConf == null, "oldConf is already set.");
+      this.oldConf = oldConf;
+      return this;
+    }
+
+    public Builder setOldConf(Iterable<RaftPeer> oldPeers) {
+      return setOldConf(new PeerConfiguration(oldPeers));
+    }
+
+    public Builder setOldConf(RaftPeer[] oldPeers) {
+      return setOldConf(Arrays.asList(oldPeers));
+    }
+
+    Builder setOldConf(RaftConfiguration stableConf) {
+      Preconditions.checkNotNull(stableConf);
+      Preconditions.checkState(stableConf.isStable());
+
+      Preconditions.checkState(!forceStable);
+      forceTransitional = true;
+      return setOldConf(stableConf.conf);
+    }
+
+    public Builder setLogEntryIndex(long logEntryIndex) {
+      Preconditions.checkArgument(
+          logEntryIndex != RaftServerConstants.INVALID_LOG_INDEX);
+      Preconditions.checkState(
+          this.logEntryIndex == RaftServerConstants.INVALID_LOG_INDEX,
+          "logEntryIndex is already set.");
+      this.logEntryIndex = logEntryIndex;
+      return this;
+    }
+
+    /** Build a {@link RaftConfiguration}. */
+    public RaftConfiguration build() {
+      if (forceTransitional) {
+        Preconditions.checkState(oldConf != null);
+      }
+      if (forceStable) {
+        Preconditions.checkState(oldConf == null);
+      }
+      return new RaftConfiguration(conf, oldConf, logEntryIndex);
+    }
+  }
+
+  /** Non-null only if this configuration is transitional. */
+  private final PeerConfiguration oldConf;
+  /**
+   * The current peer configuration while this configuration is stable;
+   * or the new peer configuration while this configuration is transitional.
+   */
+  private final PeerConfiguration conf;
+
+  /** The index of the corresponding log entry for this configuration. */
+  private final long logEntryIndex;
+
+  private RaftConfiguration(PeerConfiguration conf, PeerConfiguration oldConf,
+      long logEntryIndex) {
+    Preconditions.checkNotNull(conf);
+    this.conf = conf;
+    this.oldConf = oldConf;
+    this.logEntryIndex = logEntryIndex;
+  }
+
+  /** Is this configuration transitional, i.e. in the middle of a peer change? */
+  public boolean isTransitional() {
+    return oldConf != null;
+  }
+
+  /** Is this configuration stable, i.e. no on-going peer change? */
+  public boolean isStable() {
+    return oldConf == null;
+  }
+
+  boolean containsInConf(String peerId) {
+    return conf.contains(peerId);
+  }
+
+  boolean containsInOldConf(String peerId) {
+    return oldConf != null && oldConf.contains(peerId);
+  }
+
+  public boolean contains(String peerId) {
+    return containsInConf(peerId) && (oldConf == null || containsInOldConf(peerId));
+  }
+
+  /**
+   * @return the peer corresponding to the given id;
+   *         or return null if the peer is not in this configuration.
+   */
+  public RaftPeer getPeer(String id) {
+    if (id == null) {
+      return null;
+    }
+    RaftPeer peer = conf.getPeer(id);
+    if (peer != null) {
+      return peer;
+    } else if (oldConf != null) {
+      return oldConf.getPeer(id);
+    }
+    return null;
+  }
+
+  /** @return all the peers from the conf, and the old conf if it exists. */
+  public Collection<RaftPeer> getPeers() {
+    final Collection<RaftPeer> peers = new ArrayList<>(conf.getPeers());
+    if (oldConf != null) {
+      oldConf.getPeers().stream().filter(p -> !peers.contains(p))
+          .forEach(peers::add);
+    }
+    return peers;
+  }
+
+  /**
+   * @return all the peers other than the given self id from the conf,
+   *         and the old conf if it exists.
+   */
+  public Collection<RaftPeer> getOtherPeers(String selfId) {
+    Collection<RaftPeer> others = conf.getOtherPeers(selfId);
+    if (oldConf != null) {
+      oldConf.getOtherPeers(selfId).stream()
+          .filter(p -> !others.contains(p))
+          .forEach(others::add);
+    }
+    return others;
+  }
+
+  /** @return true if the self id together with the others are in the majority. */
+  public boolean hasMajority(Collection<String> others, String selfId) {
+    Preconditions.checkArgument(!others.contains(selfId));
+    return conf.hasMajority(others, selfId) &&
+        (oldConf == null || oldConf.hasMajority(others, selfId));
+  }
+
+  @Override
+  public String toString() {
+    return conf + (oldConf != null ? "old:" + oldConf : "");
+  }
+
+  @VisibleForTesting
+  boolean hasNoChange(RaftPeer[] newMembers) {
+    if (!isStable() || conf.size() != newMembers.length) {
+      return false;
+    }
+    for (RaftPeer peer : newMembers) {
+      if (!conf.contains(peer.getId())) {
+        return false;
+      }
+    }
+    return true;
+  }
+
+  long getLogEntryIndex() {
+    return logEntryIndex;
+  }
+
+  static Collection<RaftPeer> computeNewPeers(RaftPeer[] newMembers,
+      RaftConfiguration old) {
+    List<RaftPeer> peers = new ArrayList<>();
+    for (RaftPeer p : newMembers) {
+      if (!old.containsInConf(p.getId())) {
+        peers.add(p);
+      }
+    }
+    return peers;
+  }
+
+  RaftPeer getRandomPeer(String exclusiveId) {
+    final List<RaftPeer> peers = conf.getOtherPeers(exclusiveId);
+    if (peers.isEmpty()) {
+      return null;
+    }
+    final int index = ThreadLocalRandom.current().nextInt(peers.size());
+    return peers.get(index);
+  }
+
+  public Collection<RaftPeer> getPeersInOldConf() {
+    return oldConf != null ? oldConf.getPeers() : Collections.emptyList();
+  }
+
+  public Collection<RaftPeer> getPeersInConf() {
+    return conf.getPeers();
+  }
+}

http://git-wip-us.apache.org/repos/asf/incubator-ratis/blob/c36810ed/raft-server/src/main/java/org/apache/raft/server/impl/RaftServer.java
----------------------------------------------------------------------
diff --git a/raft-server/src/main/java/org/apache/raft/server/impl/RaftServer.java b/raft-server/src/main/java/org/apache/raft/server/impl/RaftServer.java
new file mode 100644
index 0000000..c1bf4a9
--- /dev/null
+++ b/raft-server/src/main/java/org/apache/raft/server/impl/RaftServer.java
@@ -0,0 +1,749 @@
+/**
+ * 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.raft.server.impl;
+
+import com.google.common.annotations.VisibleForTesting;
+import com.google.common.base.Preconditions;
+import org.apache.raft.conf.RaftProperties;
+import org.apache.raft.protocol.*;
+import org.apache.raft.server.RaftServerConfigKeys;
+import org.apache.raft.server.protocol.RaftServerProtocol;
+import org.apache.raft.server.protocol.TermIndex;
+import org.apache.raft.server.storage.FileInfo;
+import org.apache.raft.shaded.proto.RaftProtos.*;
+import org.apache.raft.statemachine.SnapshotInfo;
+import org.apache.raft.statemachine.StateMachine;
+import org.apache.raft.statemachine.TransactionContext;
+import org.apache.raft.util.CodeInjectionForTesting;
+import org.apache.raft.util.LifeCycle;
+import org.apache.raft.util.ProtoUtils;
+import org.apache.raft.util.RaftUtils;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.io.Closeable;
+import java.io.IOException;
+import java.io.InterruptedIOException;
+import java.util.Arrays;
+import java.util.Collection;
+import java.util.List;
+import java.util.OptionalLong;
+import java.util.concurrent.CompletableFuture;
+
+import static org.apache.raft.shaded.proto.RaftProtos.AppendEntriesReplyProto.AppendResult.*;
+import static org.apache.raft.util.LifeCycle.State.*;
+
+public class RaftServer implements RaftServerProtocol, Closeable {
+  public static final Logger LOG = LoggerFactory.getLogger(RaftServer.class);
+
+  private static final String CLASS_NAME = RaftServer.class.getSimpleName();
+  static final String REQUEST_VOTE = CLASS_NAME + ".requestVote";
+  static final String APPEND_ENTRIES = CLASS_NAME + ".appendEntries";
+  static final String INSTALL_SNAPSHOT = CLASS_NAME + ".installSnapshot";
+
+
+  private final int minTimeoutMs;
+  private final int maxTimeoutMs;
+
+  private final LifeCycle lifeCycle;
+  private final ServerState state;
+  private final StateMachine stateMachine;
+  private final RaftProperties properties;
+  private volatile Role role;
+
+  /** used when the peer is follower, to monitor election timeout */
+  private volatile FollowerState heartbeatMonitor;
+
+  /** used when the peer is candidate, to request votes from other peers */
+  private volatile LeaderElection electionDaemon;
+
+  /** used when the peer is leader */
+  private volatile LeaderState leaderState;
+
+  private RaftServerRpc serverRpc;
+
+  private final LogAppenderFactory appenderFactory;
+
+  public RaftServer(String id, RaftConfiguration raftConf,
+      RaftProperties properties, StateMachine stateMachine) throws IOException {
+    this.lifeCycle = new LifeCycle(id);
+    minTimeoutMs = properties.getInt(
+        RaftServerConfigKeys.RAFT_SERVER_RPC_TIMEOUT_MIN_MS_KEY,
+        RaftServerConfigKeys.RAFT_SERVER_RPC_TIMEOUT_MIN_MS_DEFAULT);
+    maxTimeoutMs = properties.getInt(
+        RaftServerConfigKeys.RAFT_SERVER_RPC_TIMEOUT_MAX_MS_KEY,
+        RaftServerConfigKeys.RAFT_SERVER_RPC_TIMEOUT_MAX_MS_DEFAULT);
+    Preconditions.checkArgument(maxTimeoutMs > minTimeoutMs,
+        "max timeout: %s, min timeout: %s", maxTimeoutMs, minTimeoutMs);
+    this.properties = properties;
+    this.stateMachine = stateMachine;
+    this.state = new ServerState(id, raftConf, properties, this, stateMachine);
+    appenderFactory = initAppenderFactory();
+  }
+
+  public int getMinTimeoutMs() {
+    return minTimeoutMs;
+  }
+
+  public int getMaxTimeoutMs() {
+    return maxTimeoutMs;
+  }
+
+  public int getRandomTimeoutMs() {
+    return RaftUtils.getRandomBetween(minTimeoutMs, maxTimeoutMs);
+  }
+
+  public StateMachine getStateMachine() {
+    return this.stateMachine;
+  }
+
+  public LogAppenderFactory getLogAppenderFactory() {
+    return appenderFactory;
+  }
+
+  private LogAppenderFactory initAppenderFactory() {
+    Class<? extends LogAppenderFactory> factoryClass = properties.getClass(
+        RaftServerConfigKeys.RAFT_SERVER_LOG_APPENDER_FACTORY_CLASS_KEY,
+        RaftServerConfigKeys.RAFT_SERVER_LOG_APPENDER_FACTORY_CLASS_DEFAULT,
+        LogAppenderFactory.class);
+    return RaftUtils.newInstance(factoryClass);
+  }
+
+  /**
+   * Used by tests to set initial raft configuration with correct port bindings.
+   */
+  @VisibleForTesting
+  public void setInitialConf(RaftConfiguration conf) {
+    this.state.setInitialConf(conf);
+  }
+
+  public void setServerRpc(RaftServerRpc serverRpc) {
+    this.serverRpc = serverRpc;
+    // add peers into rpc service
+    RaftConfiguration conf = getRaftConf();
+    if (conf != null) {
+      addPeersToRPC(conf.getPeers());
+    }
+  }
+
+  public RaftServerRpc getServerRpc() {
+    return serverRpc;
+  }
+
+  public void start() {
+    lifeCycle.transition(STARTING);
+    state.start();
+    RaftConfiguration conf = getRaftConf();
+    if (conf != null && conf.contains(getId())) {
+      LOG.debug("{} starts as a follower", getId());
+      startAsFollower();
+    } else {
+      LOG.debug("{} starts with initializing state", getId());
+      startInitializing();
+    }
+  }
+
+  /**
+   * The peer belongs to the current configuration, should start as a follower
+   */
+  private void startAsFollower() {
+    role = Role.FOLLOWER;
+    heartbeatMonitor = new FollowerState(this);
+    heartbeatMonitor.start();
+
+    serverRpc.start();
+    lifeCycle.transition(RUNNING);
+  }
+
+  /**
+   * The peer does not have any configuration (maybe it will later be included
+   * in some configuration). Start still as a follower but will not vote or
+   * start election.
+   */
+  private void startInitializing() {
+    role = Role.FOLLOWER;
+    // do not start heartbeatMonitoring
+    serverRpc.start();
+  }
+
+  public ServerState getState() {
+    return this.state;
+  }
+
+  public String getId() {
+    return getState().getSelfId();
+  }
+
+  public RaftConfiguration getRaftConf() {
+    return getState().getRaftConf();
+  }
+
+  @Override
+  public void close() {
+    lifeCycle.checkStateAndClose(() -> {
+      try {
+        shutdownHeartbeatMonitor();
+        shutdownElectionDaemon();
+        shutdownLeaderState();
+
+        serverRpc.shutdown();
+        state.close();
+      } catch (Exception ignored) {
+        LOG.warn("Failed to kill " + state.getSelfId(), ignored);
+      }
+    });
+  }
+
+  public boolean isAlive() {
+    return !lifeCycle.getCurrentState().isOneOf(CLOSING, CLOSED);
+  }
+
+  public boolean isFollower() {
+    return role == Role.FOLLOWER;
+  }
+
+  public boolean isCandidate() {
+    return role == Role.CANDIDATE;
+  }
+
+  public boolean isLeader() {
+    return role == Role.LEADER;
+  }
+
+  Role getRole() {
+    return role;
+  }
+
+  /**
+   * Change the server state to Follower if necessary
+   * @param newTerm The new term.
+   * @param sync We will call {@link ServerState#persistMetadata()} if this is
+   *             set to true and term/votedFor get updated.
+   * @return if the term/votedFor should be updated to the new term
+   * @throws IOException if term/votedFor persistence failed.
+   */
+  synchronized boolean changeToFollower(long newTerm, boolean sync)
+      throws IOException {
+    final Role old = role;
+    role = Role.FOLLOWER;
+
+    boolean metadataUpdated = false;
+    if (newTerm > state.getCurrentTerm()) {
+      state.setCurrentTerm(newTerm);
+      state.resetLeaderAndVotedFor();
+      metadataUpdated = true;
+    }
+
+    if (old == Role.LEADER) {
+      assert leaderState != null;
+      shutdownLeaderState();
+    } else if (old == Role.CANDIDATE) {
+      shutdownElectionDaemon();
+    }
+
+    if (old != Role.FOLLOWER) {
+      heartbeatMonitor = new FollowerState(this);
+      heartbeatMonitor.start();
+    }
+
+    if (metadataUpdated && sync) {
+      state.persistMetadata();
+    }
+    return metadataUpdated;
+  }
+
+  private synchronized void shutdownLeaderState() {
+    final LeaderState leader = leaderState;
+    if (leader != null) {
+      leader.stop();
+    }
+    leaderState = null;
+    // TODO: make sure that StateMachineUpdater has applied all transactions that have context
+  }
+
+  private void shutdownElectionDaemon() {
+    final LeaderElection election = electionDaemon;
+    if (election != null) {
+      election.stopRunning();
+      // no need to interrupt the election thread
+    }
+    electionDaemon = null;
+  }
+
+  synchronized void changeToLeader() {
+    Preconditions.checkState(isCandidate());
+    shutdownElectionDaemon();
+    role = Role.LEADER;
+    state.becomeLeader();
+    // start sending AppendEntries RPC to followers
+    leaderState = new LeaderState(this, properties);
+    leaderState.start();
+  }
+
+  private void shutdownHeartbeatMonitor() {
+    final FollowerState hm = heartbeatMonitor;
+    if (hm != null) {
+      hm.stopRunning();
+      hm.interrupt();
+    }
+    heartbeatMonitor = null;
+  }
+
+  synchronized void changeToCandidate() {
+    Preconditions.checkState(isFollower());
+    shutdownHeartbeatMonitor();
+    role = Role.CANDIDATE;
+    // start election
+    electionDaemon = new LeaderElection(this);
+    electionDaemon.start();
+  }
+
+  @Override
+  public String toString() {
+    return role + " " + state + " " + lifeCycle.getCurrentState();
+  }
+
+  /**
+   * @return null if the server is in leader state.
+   */
+  CompletableFuture<RaftClientReply> checkLeaderState(
+      RaftClientRequest request) {
+    if (!isLeader()) {
+      NotLeaderException exception = generateNotLeaderException();
+      CompletableFuture<RaftClientReply> future = new CompletableFuture<>();
+      future.complete(new RaftClientReply(request, exception));
+      return future;
+    }
+    return null;
+  }
+
+  NotLeaderException generateNotLeaderException() {
+    if (lifeCycle.getCurrentState() != RUNNING) {
+      return new NotLeaderException(getId(), null, null);
+    }
+    String leaderId = state.getLeaderId();
+    if (leaderId == null || leaderId.equals(state.getSelfId())) {
+      // No idea about who is the current leader. Or the peer is the current
+      // leader, but it is about to step down
+      RaftPeer suggestedLeader = state.getRaftConf()
+          .getRandomPeer(state.getSelfId());
+      leaderId = suggestedLeader == null ? null : suggestedLeader.getId();
+    }
+    RaftConfiguration conf = getRaftConf();
+    Collection<RaftPeer> peers = conf.getPeers();
+    return new NotLeaderException(getId(), conf.getPeer(leaderId),
+        peers.toArray(new RaftPeer[peers.size()]));
+  }
+
+  /**
+   * Handle a normal update request from client.
+   */
+  public CompletableFuture<RaftClientReply> appendTransaction(
+      RaftClientRequest request, TransactionContext entry)
+      throws RaftException {
+    LOG.debug("{}: receive client request({})", getId(), request);
+    lifeCycle.assertCurrentState(RUNNING);
+    CompletableFuture<RaftClientReply> reply;
+
+    final PendingRequest pending;
+    synchronized (this) {
+      reply = checkLeaderState(request);
+      if (reply != null) {
+        return reply;
+      }
+
+      // append the message to its local log
+      final long entryIndex;
+      try {
+        entryIndex = state.applyLog(entry);
+      } catch (IOException e) {
+        throw new RaftException(e);
+      }
+
+      // put the request into the pending queue
+      pending = leaderState.addPendingRequest(entryIndex, request, entry);
+      leaderState.notifySenders();
+    }
+    return pending.getFuture();
+  }
+
+  /**
+   * Handle a raft configuration change request from client.
+   */
+  public CompletableFuture<RaftClientReply> setConfiguration(
+      SetConfigurationRequest request) throws IOException {
+    LOG.debug("{}: receive setConfiguration({})", getId(), request);
+    lifeCycle.assertCurrentState(RUNNING);
+    CompletableFuture<RaftClientReply> reply = checkLeaderState(request);
+    if (reply != null) {
+      return reply;
+    }
+
+    final RaftPeer[] peersInNewConf = request.getPeersInNewConf();
+    final PendingRequest pending;
+    synchronized (this) {
+      reply = checkLeaderState(request);
+      if (reply != null) {
+        return reply;
+      }
+
+      final RaftConfiguration current = getRaftConf();
+      // make sure there is no other raft reconfiguration in progress
+      if (!current.isStable() || leaderState.inStagingState() ||
+          !state.isCurrentConfCommitted()) {
+        throw new ReconfigurationInProgressException(
+            "Reconfiguration is already in progress: " + current);
+      }
+
+      // return true if the new configuration is the same with the current one
+      if (current.hasNoChange(peersInNewConf)) {
+        pending = leaderState.returnNoConfChange(request);
+        return pending.getFuture();
+      }
+
+      // add new peers into the rpc service
+      addPeersToRPC(Arrays.asList(peersInNewConf));
+      // add staging state into the leaderState
+      pending = leaderState.startSetConfiguration(request);
+    }
+    return pending.getFuture();
+  }
+
+  private boolean shouldWithholdVotes() {
+    return isLeader() || (isFollower() && state.hasLeader()
+        && heartbeatMonitor.shouldWithholdVotes());
+  }
+
+  /**
+   * check if the remote peer is not included in the current conf
+   * and should shutdown. should shutdown if all the following stands:
+   * 1. this is a leader
+   * 2. current conf is stable and has been committed
+   * 3. candidate id is not included in conf
+   * 4. candidate's last entry's index < conf's index
+   */
+  private boolean shouldSendShutdown(String candidateId,
+      TermIndex candidateLastEntry) {
+    return isLeader()
+        && getRaftConf().isStable()
+        && getState().isConfCommitted()
+        && !getRaftConf().containsInConf(candidateId)
+        && candidateLastEntry.getIndex() < getRaftConf().getLogEntryIndex()
+        && !leaderState.isBootStrappingPeer(candidateId);
+  }
+
+  @Override
+  public RequestVoteReplyProto requestVote(RequestVoteRequestProto r)
+      throws IOException {
+    final String candidateId = r.getServerRequest().getRequestorId();
+    return requestVote(candidateId, r.getCandidateTerm(),
+        ServerProtoUtils.toTermIndex(r.getCandidateLastEntry()));
+  }
+
+  private RequestVoteReplyProto requestVote(String candidateId,
+      long candidateTerm, TermIndex candidateLastEntry) throws IOException {
+    CodeInjectionForTesting.execute(REQUEST_VOTE, getId(),
+        candidateId, candidateTerm, candidateLastEntry);
+    LOG.debug("{}: receive requestVote({}, {}, {})",
+        getId(), candidateId, candidateTerm, candidateLastEntry);
+    lifeCycle.assertCurrentState(RUNNING);
+
+    boolean voteGranted = false;
+    boolean shouldShutdown = false;
+    final RequestVoteReplyProto reply;
+    synchronized (this) {
+      if (shouldWithholdVotes()) {
+        LOG.info("{} Withhold vote from server {} with term {}. " +
+            "This server:{}, last rpc time from leader {} is {}", getId(),
+            candidateId, candidateTerm, this, this.getState().getLeaderId(),
+            (isFollower() ? heartbeatMonitor.getLastRpcTime() : -1));
+      } else if (state.recognizeCandidate(candidateId, candidateTerm)) {
+        boolean termUpdated = changeToFollower(candidateTerm, false);
+        // see Section 5.4.1 Election restriction
+        if (state.isLogUpToDate(candidateLastEntry)) {
+          heartbeatMonitor.updateLastRpcTime(false);
+          state.grantVote(candidateId);
+          voteGranted = true;
+        }
+        if (termUpdated || voteGranted) {
+          state.persistMetadata(); // sync metafile
+        }
+      }
+      if (!voteGranted && shouldSendShutdown(candidateId, candidateLastEntry)) {
+        shouldShutdown = true;
+      }
+      reply = ServerProtoUtils.toRequestVoteReplyProto(candidateId, getId(),
+          voteGranted, state.getCurrentTerm(), shouldShutdown);
+      if (LOG.isDebugEnabled()) {
+        LOG.debug("{} replies to vote request: {}. Peer's state: {}",
+            getId(), ProtoUtils.toString(reply), state);
+      }
+    }
+    return reply;
+  }
+
+  private void validateEntries(long expectedTerm, TermIndex previous,
+      LogEntryProto... entries) {
+    if (entries != null && entries.length > 0) {
+      final long index0 = entries[0].getIndex();
+
+      if (previous == null || previous.getTerm() == 0) {
+        Preconditions.checkArgument(index0 == 0,
+            "Unexpected Index: previous is null but entries[%s].getIndex()=%s",
+            0, index0);
+      } else {
+        Preconditions.checkArgument(previous.getIndex() == index0 - 1,
+            "Unexpected Index: previous is %s but entries[%s].getIndex()=%s",
+            previous, 0, index0);
+      }
+
+      for (int i = 0; i < entries.length; i++) {
+        final long t = entries[i].getTerm();
+        Preconditions.checkArgument(expectedTerm >= t,
+            "Unexpected Term: entries[%s].getTerm()=%s but expectedTerm=%s",
+            i, t, expectedTerm);
+
+        final long indexi = entries[i].getIndex();
+        Preconditions.checkArgument(indexi == index0 + i,
+            "Unexpected Index: entries[%s].getIndex()=%s but entries[0].getIndex()=%s",
+            i, indexi, index0);
+      }
+    }
+  }
+
+  @Override
+  public AppendEntriesReplyProto appendEntries(AppendEntriesRequestProto r)
+      throws IOException {
+    // TODO avoid converting list to array
+    final LogEntryProto[] entries = r.getEntriesList()
+        .toArray(new LogEntryProto[r.getEntriesCount()]);
+    final TermIndex previous = r.hasPreviousLog() ?
+        ServerProtoUtils.toTermIndex(r.getPreviousLog()) : null;
+    return appendEntries(r.getServerRequest().getRequestorId(),
+        r.getLeaderTerm(), previous, r.getLeaderCommit(), r.getInitializing(),
+        entries);
+  }
+
+  private AppendEntriesReplyProto appendEntries(String leaderId, long leaderTerm,
+      TermIndex previous, long leaderCommit, boolean initializing,
+      LogEntryProto... entries) throws IOException {
+    CodeInjectionForTesting.execute(APPEND_ENTRIES, getId(),
+        leaderId, leaderTerm, previous, leaderCommit, initializing, entries);
+    if (LOG.isDebugEnabled()) {
+      LOG.debug("{}: receive appendEntries({}, {}, {}, {}, {}, {})", getId(),
+          leaderId, leaderTerm, previous, leaderCommit, initializing,
+          ServerProtoUtils.toString(entries));
+    }
+    lifeCycle.assertCurrentState(STARTING, RUNNING);
+
+    try {
+      validateEntries(leaderTerm, previous, entries);
+    } catch (IllegalArgumentException e) {
+      throw new IOException(e);
+    }
+
+    final long currentTerm;
+    long nextIndex = state.getLog().getNextIndex();
+    synchronized (this) {
+      final boolean recognized = state.recognizeLeader(leaderId, leaderTerm);
+      currentTerm = state.getCurrentTerm();
+      if (!recognized) {
+        final AppendEntriesReplyProto reply = ServerProtoUtils.toAppendEntriesReplyProto(
+            leaderId, getId(), currentTerm, nextIndex, NOT_LEADER);
+        if (LOG.isDebugEnabled()) {
+          LOG.debug("{}: do not recognize leader. Reply: {}",
+              getId(), ProtoUtils.toString(reply));
+        }
+        return reply;
+      }
+      changeToFollower(leaderTerm, true);
+      state.setLeader(leaderId);
+
+      if (!initializing && lifeCycle.compareAndTransition(STARTING, RUNNING)) {
+        heartbeatMonitor = new FollowerState(this);
+        heartbeatMonitor.start();
+      }
+      if (lifeCycle.getCurrentState() == RUNNING) {
+        heartbeatMonitor.updateLastRpcTime(true);
+      }
+
+      // We need to check if "previous" is in the local peer. Note that it is
+      // possible that "previous" is covered by the latest snapshot: e.g.,
+      // it's possible there's no log entries outside of the latest snapshot.
+      // However, it is not possible that "previous" index is smaller than the
+      // last index included in snapshot. This is because indices <= snapshot's
+      // last index should have been committed.
+      if (previous != null && !containPrevious(previous)) {
+        final AppendEntriesReplyProto reply =
+            ServerProtoUtils.toAppendEntriesReplyProto(leaderId, getId(),
+                currentTerm, Math.min(nextIndex, previous.getIndex()), INCONSISTENCY);
+        LOG.debug("{}: inconsistency entries. Leader previous:{}, Reply:{}",
+            getId(), previous, ServerProtoUtils.toString(reply));
+        return reply;
+      }
+
+      state.getLog().append(entries);
+      state.updateConfiguration(entries);
+      state.updateStatemachine(leaderCommit, currentTerm);
+    }
+    if (entries != null && entries.length > 0) {
+      try {
+        state.getLog().logSync();
+      } catch (InterruptedException e) {
+        throw new InterruptedIOException("logSync got interrupted");
+      }
+      nextIndex = entries[entries.length - 1].getIndex() + 1;
+    }
+    synchronized (this) {
+      if (lifeCycle.getCurrentState() == RUNNING && isFollower()
+          && getState().getCurrentTerm() == currentTerm) {
+        // reset election timer to avoid punishing the leader for our own
+        // long disk writes
+        heartbeatMonitor.updateLastRpcTime(false);
+      }
+    }
+    final AppendEntriesReplyProto reply = ServerProtoUtils.toAppendEntriesReplyProto(
+        leaderId, getId(), currentTerm, nextIndex, SUCCESS);
+    LOG.debug("{}: succeeded to handle AppendEntries. Reply: {}", getId(),
+        ServerProtoUtils.toString(reply));
+    return reply;
+  }
+
+  private boolean containPrevious(TermIndex previous) {
+    LOG.debug("{}: prev:{}, latestSnapshot:{}, getLatestInstalledSnapshot:{}",
+        getId(), previous, state.getLatestSnapshot(), state.getLatestInstalledSnapshot());
+    return state.getLog().contains(previous)
+        ||  (state.getLatestSnapshot() != null
+             && state.getLatestSnapshot().getTermIndex().equals(previous))
+        || (state.getLatestInstalledSnapshot() != null)
+             && state.getLatestInstalledSnapshot().equals(previous);
+  }
+
+  @Override
+  public InstallSnapshotReplyProto installSnapshot(
+      InstallSnapshotRequestProto request) throws IOException {
+    final String leaderId = request.getServerRequest().getRequestorId();
+    CodeInjectionForTesting.execute(INSTALL_SNAPSHOT, getId(), leaderId, request);
+    LOG.debug("{}: receive installSnapshot({})", getId(), request);
+
+    lifeCycle.assertCurrentState(STARTING, RUNNING);
+
+    final long currentTerm;
+    final long leaderTerm = request.getLeaderTerm();
+    final TermIndex lastTermIndex = ServerProtoUtils.toTermIndex(
+        request.getTermIndex());
+    final long lastIncludedIndex = lastTermIndex.getIndex();
+    synchronized (this) {
+      final boolean recognized = state.recognizeLeader(leaderId, leaderTerm);
+      currentTerm = state.getCurrentTerm();
+      if (!recognized) {
+        final InstallSnapshotReplyProto reply = ServerProtoUtils
+            .toInstallSnapshotReplyProto(leaderId, getId(), currentTerm,
+                request.getRequestIndex(), InstallSnapshotResult.NOT_LEADER);
+        LOG.debug("{}: do not recognize leader for installing snapshot." +
+            " Reply: {}", getId(), reply);
+        return reply;
+      }
+      changeToFollower(leaderTerm, true);
+      state.setLeader(leaderId);
+
+      if (lifeCycle.getCurrentState() == RUNNING) {
+        heartbeatMonitor.updateLastRpcTime(true);
+      }
+
+      // Check and append the snapshot chunk. We simply put this in lock
+      // considering a follower peer requiring a snapshot installation does not
+      // have a lot of requests
+      Preconditions.checkState(
+          state.getLog().getNextIndex() <= lastIncludedIndex,
+          "%s log's next id is %s, last included index in snapshot is %s",
+          getId(),  state.getLog().getNextIndex(), lastIncludedIndex);
+
+      //TODO: We should only update State with installed snapshot once the request is done.
+      state.installSnapshot(request);
+
+      // update the committed index
+      // re-load the state machine if this is the last chunk
+      if (request.getDone()) {
+        state.reloadStateMachine(lastIncludedIndex, leaderTerm);
+      }
+      if (lifeCycle.getCurrentState() == RUNNING) {
+        heartbeatMonitor.updateLastRpcTime(false);
+      }
+    }
+    if (request.getDone()) {
+      LOG.info("{}: successfully install the whole snapshot-{}", getId(),
+          lastIncludedIndex);
+    }
+    return ServerProtoUtils.toInstallSnapshotReplyProto(leaderId, getId(),
+        currentTerm, request.getRequestIndex(), InstallSnapshotResult.SUCCESS);
+  }
+
+  AppendEntriesRequestProto createAppendEntriesRequest(long leaderTerm,
+      String targetId, TermIndex previous, List<LogEntryProto> entries,
+      boolean initializing) {
+    return ServerProtoUtils.toAppendEntriesRequestProto(getId(), targetId,
+        leaderTerm, entries, state.getLog().getLastCommittedIndex(),
+        initializing, previous);
+  }
+
+  synchronized InstallSnapshotRequestProto createInstallSnapshotRequest(
+      String targetId, String requestId, int requestIndex, SnapshotInfo snapshot,
+      List<FileChunkProto> chunks, boolean done) {
+    OptionalLong totalSize = snapshot.getFiles().stream()
+        .mapToLong(FileInfo::getFileSize).reduce(Long::sum);
+    assert totalSize.isPresent();
+    return ServerProtoUtils.toInstallSnapshotRequestProto(getId(), targetId,
+        requestId, requestIndex, state.getCurrentTerm(), snapshot.getTermIndex(),
+        chunks, totalSize.getAsLong(), done);
+  }
+
+  synchronized RequestVoteRequestProto createRequestVoteRequest(String targetId,
+      long term, TermIndex lastEntry) {
+    return ServerProtoUtils.toRequestVoteRequestProto(getId(), targetId, term,
+        lastEntry);
+  }
+
+  public synchronized void submitLocalSyncEvent() {
+    if (isLeader() && leaderState != null) {
+      leaderState.submitUpdateStateEvent(LeaderState.UPDATE_COMMIT_EVENT);
+    }
+  }
+
+  public void addPeersToRPC(Iterable<RaftPeer> peers) {
+    serverRpc.addPeers(peers);
+  }
+
+  synchronized void replyPendingRequest(long logIndex,
+      CompletableFuture<Message> message) {
+    if (isLeader() && leaderState != null) { // is leader and is running
+      leaderState.replyPendingRequest(logIndex, message);
+    }
+  }
+
+  TransactionContext getTransactionContext(long index) {
+    if (leaderState != null) { // is leader and is running
+      return leaderState.getTransactionContext(index);
+    }
+    return null;
+  }
+
+  public RaftProperties getProperties() {
+    return this.properties;
+  }
+}

http://git-wip-us.apache.org/repos/asf/incubator-ratis/blob/c36810ed/raft-server/src/main/java/org/apache/raft/server/impl/RaftServerConstants.java
----------------------------------------------------------------------
diff --git a/raft-server/src/main/java/org/apache/raft/server/impl/RaftServerConstants.java b/raft-server/src/main/java/org/apache/raft/server/impl/RaftServerConstants.java
new file mode 100644
index 0000000..6634152
--- /dev/null
+++ b/raft-server/src/main/java/org/apache/raft/server/impl/RaftServerConstants.java
@@ -0,0 +1,46 @@
+/**
+ * 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.raft.server.impl;
+
+import org.apache.raft.client.RaftClient;
+
+public interface RaftServerConstants {
+  long INVALID_LOG_INDEX = -1;
+  byte LOG_TERMINATE_BYTE = 0;
+  long DEFAULT_SEQNUM = RaftClient.DEFAULT_SEQNUM;
+
+  enum StartupOption {
+    FORMAT("format"),
+    REGULAR("regular");
+
+    private final String option;
+
+    StartupOption(String arg) {
+      this.option = arg;
+    }
+
+    public static StartupOption getOption(String arg) {
+      for (StartupOption s : StartupOption.values()) {
+        if (s.option.equals(arg)) {
+          return s;
+        }
+      }
+      return REGULAR;
+    }
+  }
+}

http://git-wip-us.apache.org/repos/asf/incubator-ratis/blob/c36810ed/raft-server/src/main/java/org/apache/raft/server/impl/RaftServerRpc.java
----------------------------------------------------------------------
diff --git a/raft-server/src/main/java/org/apache/raft/server/impl/RaftServerRpc.java b/raft-server/src/main/java/org/apache/raft/server/impl/RaftServerRpc.java
new file mode 100644
index 0000000..fb74a0e
--- /dev/null
+++ b/raft-server/src/main/java/org/apache/raft/server/impl/RaftServerRpc.java
@@ -0,0 +1,44 @@
+/**
+ * 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.raft.server.impl;
+
+import org.apache.raft.protocol.RaftPeer;
+import org.apache.raft.shaded.proto.RaftProtos.*;
+
+import java.io.IOException;
+import java.net.InetSocketAddress;
+
+public interface RaftServerRpc {
+  void start();
+
+  void shutdown();
+
+  InetSocketAddress getInetSocketAddress();
+
+  AppendEntriesReplyProto sendAppendEntries(
+      AppendEntriesRequestProto request) throws IOException;
+
+  InstallSnapshotReplyProto sendInstallSnapshot(
+      InstallSnapshotRequestProto request) throws IOException;
+
+  RequestVoteReplyProto sendRequestVote(RequestVoteRequestProto request)
+      throws IOException;
+
+  /** add information of the given peers */
+  void addPeers(Iterable<RaftPeer> peers);
+}

http://git-wip-us.apache.org/repos/asf/incubator-ratis/blob/c36810ed/raft-server/src/main/java/org/apache/raft/server/impl/RequestDispatcher.java
----------------------------------------------------------------------
diff --git a/raft-server/src/main/java/org/apache/raft/server/impl/RequestDispatcher.java b/raft-server/src/main/java/org/apache/raft/server/impl/RequestDispatcher.java
new file mode 100644
index 0000000..cc8651d
--- /dev/null
+++ b/raft-server/src/main/java/org/apache/raft/server/impl/RequestDispatcher.java
@@ -0,0 +1,137 @@
+/**
+ * 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.raft.server.impl;
+
+import org.apache.raft.protocol.*;
+import org.apache.raft.server.protocol.RaftServerProtocol;
+import org.apache.raft.shaded.proto.RaftProtos.*;
+import org.apache.raft.statemachine.StateMachine;
+import org.apache.raft.statemachine.TransactionContext;
+import org.apache.raft.util.RaftUtils;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.io.IOException;
+import java.util.concurrent.CompletableFuture;
+import java.util.concurrent.ExecutionException;
+
+/**
+ * Each RPC request is first handled by the RequestDispatcher:
+ * 1. A request from another RaftPeer is to be handled by RaftServer.
+ *
+ * If the raft peer is the leader, then:
+ *
+ * 2. A read-only request from client is to be handled by the state machine.
+ * 3. A write request from client is first validated by the state machine. The
+ * state machine returns the content of the raft log entry, which is then passed
+ * to the RaftServer for replication.
+ */
+public class RequestDispatcher implements RaftClientProtocol, RaftServerProtocol {
+  static final Logger LOG = LoggerFactory.getLogger(RequestDispatcher.class);
+
+  private final RaftServer server;
+  private final StateMachine stateMachine;
+
+  public RequestDispatcher(RaftServer server) {
+    this.server = server;
+    this.stateMachine = server.getStateMachine();
+  }
+
+  public CompletableFuture<RaftClientReply> handleClientRequest(
+      RaftClientRequest request) throws IOException {
+    // first check the server's leader state
+    CompletableFuture<RaftClientReply> reply = server.checkLeaderState(request);
+    if (reply != null) {
+      return reply;
+    }
+
+    // let the state machine handle read-only request from client
+    if (request.isReadOnly()) {
+      // TODO: We might not be the leader anymore by the time this completes. See the RAFT paper,
+      // section 8 (last part)
+      return stateMachine.query(request);
+    }
+
+    // TODO: this client request will not be added to pending requests
+    // until later which means that any failure in between will leave partial state in the
+    // state machine. We should call cancelTransaction() for failed requests
+    TransactionContext entry = stateMachine.startTransaction(request);
+    if (entry.getException().isPresent()) {
+      throw RaftUtils.asIOException(entry.getException().get());
+    }
+
+    return server.appendTransaction(request, entry);
+  }
+
+  @Override
+  public RaftClientReply submitClientRequest(RaftClientRequest request)
+      throws IOException {
+    return waitForReply(server.getId(), request, handleClientRequest(request));
+  }
+
+  public CompletableFuture<RaftClientReply> setConfigurationAsync(
+      SetConfigurationRequest request) throws IOException {
+    return server.setConfiguration(request);
+  }
+
+  @Override
+  public RaftClientReply setConfiguration(SetConfigurationRequest request)
+      throws IOException {
+    return waitForReply(server.getId(), request, setConfigurationAsync(request));
+  }
+
+  private static RaftClientReply waitForReply(String serverId,
+      RaftClientRequest request, CompletableFuture<RaftClientReply> future)
+      throws IOException {
+    try {
+      return future.get();
+    } catch (InterruptedException e) {
+      final String s = serverId + ": Interrupted when waiting for reply, request=" + request;
+      LOG.info(s, e);
+      throw RaftUtils.toInterruptedIOException(s, e);
+    } catch (ExecutionException e) {
+      final Throwable cause = e.getCause();
+      if (cause == null) {
+        throw new IOException(e);
+      }
+      if (cause instanceof NotLeaderException) {
+        return new RaftClientReply(request, (NotLeaderException)cause);
+      } else {
+        throw RaftUtils.asIOException(cause);
+      }
+    }
+  }
+
+  @Override
+  public RequestVoteReplyProto requestVote(RequestVoteRequestProto request)
+      throws IOException {
+    return server.requestVote(request);
+  }
+
+  @Override
+  public AppendEntriesReplyProto appendEntries(AppendEntriesRequestProto request)
+      throws IOException {
+    return server.appendEntries(request);
+  }
+
+  @Override
+  public InstallSnapshotReplyProto installSnapshot(
+      InstallSnapshotRequestProto request) throws IOException {
+    return server.installSnapshot(request);
+  }
+}

http://git-wip-us.apache.org/repos/asf/incubator-ratis/blob/c36810ed/raft-server/src/main/java/org/apache/raft/server/impl/Role.java
----------------------------------------------------------------------
diff --git a/raft-server/src/main/java/org/apache/raft/server/impl/Role.java b/raft-server/src/main/java/org/apache/raft/server/impl/Role.java
new file mode 100644
index 0000000..1413961
--- /dev/null
+++ b/raft-server/src/main/java/org/apache/raft/server/impl/Role.java
@@ -0,0 +1,25 @@
+/**
+ * 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.raft.server.impl;
+
+/**
+ * Role of Raft peer
+ */
+public enum Role {
+  LEADER, CANDIDATE, FOLLOWER
+}

http://git-wip-us.apache.org/repos/asf/incubator-ratis/blob/c36810ed/raft-server/src/main/java/org/apache/raft/server/impl/ServerProtoUtils.java
----------------------------------------------------------------------
diff --git a/raft-server/src/main/java/org/apache/raft/server/impl/ServerProtoUtils.java b/raft-server/src/main/java/org/apache/raft/server/impl/ServerProtoUtils.java
index 21fd719..4594666 100644
--- a/raft-server/src/main/java/org/apache/raft/server/impl/ServerProtoUtils.java
+++ b/raft-server/src/main/java/org/apache/raft/server/impl/ServerProtoUtils.java
@@ -18,7 +18,6 @@
 package org.apache.raft.server.impl;
 
 import org.apache.raft.client.impl.ClientProtoUtils;
-import org.apache.raft.server.RaftConfiguration;
 import org.apache.raft.server.protocol.TermIndex;
 import org.apache.raft.shaded.proto.RaftProtos.*;
 import org.apache.raft.util.ProtoUtils;
@@ -27,7 +26,7 @@ import java.util.Arrays;
 import java.util.List;
 import java.util.stream.Collectors;
 
-import static org.apache.raft.server.RaftServerConstants.DEFAULT_SEQNUM;
+import static org.apache.raft.server.impl.RaftServerConstants.DEFAULT_SEQNUM;
 import static org.apache.raft.shaded.proto.RaftProtos.AppendEntriesReplyProto.AppendResult.SUCCESS;
 
 public class ServerProtoUtils {

http://git-wip-us.apache.org/repos/asf/incubator-ratis/blob/c36810ed/raft-server/src/main/java/org/apache/raft/server/impl/ServerState.java
----------------------------------------------------------------------
diff --git a/raft-server/src/main/java/org/apache/raft/server/impl/ServerState.java b/raft-server/src/main/java/org/apache/raft/server/impl/ServerState.java
new file mode 100644
index 0000000..6680175
--- /dev/null
+++ b/raft-server/src/main/java/org/apache/raft/server/impl/ServerState.java
@@ -0,0 +1,345 @@
+/**
+ * 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.raft.server.impl;
+
+import com.google.common.annotations.VisibleForTesting;
+import com.google.common.base.Preconditions;
+import org.apache.raft.conf.RaftProperties;
+import org.apache.raft.server.protocol.TermIndex;
+import org.apache.raft.server.storage.*;
+import org.apache.raft.shaded.proto.RaftProtos.InstallSnapshotRequestProto;
+import org.apache.raft.shaded.proto.RaftProtos.LogEntryProto;
+import org.apache.raft.statemachine.SnapshotInfo;
+import org.apache.raft.statemachine.StateMachine;
+import org.apache.raft.statemachine.TransactionContext;
+import org.apache.raft.util.ProtoUtils;
+
+import java.io.Closeable;
+import java.io.IOException;
+
+import static org.apache.raft.server.RaftServerConfigKeys.RAFT_SERVER_USE_MEMORY_LOG_DEFAULT;
+import static org.apache.raft.server.RaftServerConfigKeys.RAFT_SERVER_USE_MEMORY_LOG_KEY;
+
+/**
+ * Common states of a raft peer. Protected by RaftServer's lock.
+ */
+public class ServerState implements Closeable {
+  private final String selfId;
+  private final RaftServer server;
+  /** Raft log */
+  private final RaftLog log;
+  /** Raft configuration */
+  private final ConfigurationManager configurationManager;
+  /** The thread that applies committed log entries to the state machine */
+  private final StateMachineUpdater stateMachineUpdater;
+  /** local storage for log and snapshot */
+  private final RaftStorage storage;
+  private final SnapshotManager snapshotManager;
+
+  /**
+   * Latest term server has seen. initialized to 0 on first boot, increases
+   * monotonically.
+   */
+  private long currentTerm;
+  /**
+   * The server ID of the leader for this term. Null means either there is
+   * no leader for this term yet or this server does not know who it is yet.
+   */
+  private String leaderId;
+  /**
+   * Candidate that this peer granted vote for in current term (or null if none)
+   */
+  private String votedFor;
+
+  /**
+   * Latest installed snapshot for this server. This maybe different than StateMachine's latest
+   * snapshot. Once we successfully install a snapshot, the SM may not pick it up immediately.
+   * Further, this will not get updated when SM does snapshots itself.
+   */
+  private TermIndex latestInstalledSnapshot;
+
+  ServerState(String id, RaftConfiguration conf, RaftProperties prop,
+      RaftServer server, StateMachine stateMachine) throws IOException {
+    this.selfId = id;
+    this.server = server;
+    configurationManager = new ConfigurationManager(conf);
+    storage = new RaftStorage(prop, RaftServerConstants.StartupOption.REGULAR);
+    snapshotManager = new SnapshotManager(storage, id);
+
+    long lastApplied = initStatemachine(stateMachine, prop);
+
+    leaderId = null;
+    log = initLog(id, prop, server, lastApplied);
+    RaftLog.Metadata metadata = log.loadMetadata();
+    currentTerm = metadata.getTerm();
+    votedFor = metadata.getVotedFor();
+
+    stateMachineUpdater = new StateMachineUpdater(stateMachine, server, log,
+         lastApplied, prop);
+  }
+
+  /**
+   * Used by tests to set initial raft configuration with correct port bindings.
+   */
+  @VisibleForTesting
+  public void setInitialConf(RaftConfiguration initialConf) {
+    configurationManager.setInitialConf(initialConf);
+  }
+
+  private long initStatemachine(StateMachine sm, RaftProperties properties)
+      throws IOException {
+    sm.initialize(selfId, properties, storage);
+    storage.setStateMachineStorage(sm.getStateMachineStorage());
+    SnapshotInfo snapshot = sm.getLatestSnapshot();
+
+    if (snapshot == null || snapshot.getTermIndex().getIndex() < 0) {
+      return RaftServerConstants.INVALID_LOG_INDEX;
+    }
+
+    // get the raft configuration from the snapshot
+    RaftConfiguration raftConf = sm.getRaftConfiguration();
+    if (raftConf != null) {
+      configurationManager.addConfiguration(raftConf.getLogEntryIndex(),
+          raftConf);
+    }
+    return snapshot.getIndex();
+  }
+
+  void start() {
+    stateMachineUpdater.start();
+  }
+
+  /**
+   * note we do not apply log entries to the state machine here since we do not
+   * know whether they have been committed.
+   */
+  private RaftLog initLog(String id, RaftProperties prop, RaftServer server,
+      long lastIndexInSnapshot) throws IOException {
+    final RaftLog log;
+    if (prop.getBoolean(RAFT_SERVER_USE_MEMORY_LOG_KEY,
+        RAFT_SERVER_USE_MEMORY_LOG_DEFAULT)) {
+      log = new MemoryRaftLog(id);
+    } else {
+      log = new SegmentedRaftLog(id, server, this.storage,
+          lastIndexInSnapshot, prop);
+    }
+    log.open(configurationManager, lastIndexInSnapshot);
+    return log;
+  }
+
+  public RaftConfiguration getRaftConf() {
+    return configurationManager.getCurrent();
+  }
+
+  @VisibleForTesting
+
+  public String getSelfId() {
+    return this.selfId;
+  }
+
+  public long getCurrentTerm() {
+    return currentTerm;
+  }
+
+  void setCurrentTerm(long term) {
+    currentTerm = term;
+  }
+
+  String getLeaderId() {
+    return leaderId;
+  }
+
+  boolean hasLeader() {
+    return leaderId != null;
+  }
+
+  /**
+   * Become a candidate and start leader election
+   */
+  long initElection() {
+    votedFor = selfId;
+    leaderId = null;
+    return ++currentTerm;
+  }
+
+  void persistMetadata() throws IOException {
+    this.log.writeMetadata(currentTerm, votedFor);
+  }
+
+  void resetLeaderAndVotedFor() {
+    votedFor = null;
+    leaderId = null;
+  }
+
+  /**
+   * Vote for a candidate and update the local state.
+   */
+  void grantVote(String candidateId) {
+    votedFor = candidateId;
+    leaderId = null;
+  }
+
+  void setLeader(String leaderId) {
+    this.leaderId = leaderId;
+  }
+
+  void becomeLeader() {
+    leaderId = selfId;
+  }
+
+  public RaftLog getLog() {
+    return log;
+  }
+
+  long applyLog(TransactionContext operation) throws IOException {
+    return log.append(currentTerm, operation);
+  }
+
+  /**
+   * Check if accept the leader selfId and term from the incoming AppendEntries rpc.
+   * If accept, update the current state.
+   * @return true if the check passes
+   */
+  boolean recognizeLeader(String leaderId, long leaderTerm) {
+    if (leaderTerm < currentTerm) {
+      return false;
+    } else if (leaderTerm > currentTerm || this.leaderId == null) {
+      // If the request indicates a term that is greater than the current term
+      // or no leader has been set for the current term, make sure to update
+      // leader and term later
+      return true;
+    }
+    Preconditions.checkArgument(this.leaderId.equals(leaderId),
+        "selfId:%s, this.leaderId:%s, received leaderId:%s",
+        selfId, this.leaderId, leaderId);
+    return true;
+  }
+
+  /**
+   * Check if the candidate's term is acceptable
+   */
+  boolean recognizeCandidate(String candidateId,
+      long candidateTerm) {
+    if (candidateTerm > currentTerm) {
+      return true;
+    } else if (candidateTerm == currentTerm) {
+      // has not voted yet or this is a retry
+      return votedFor == null || votedFor.equals(candidateId);
+    }
+    return false;
+  }
+
+  boolean isLogUpToDate(TermIndex candidateLastEntry) {
+    LogEntryProto lastEntry = log.getLastEntry();
+    // need to take into account snapshot
+    SnapshotInfo snapshot = server.getStateMachine().getLatestSnapshot();
+     if (lastEntry == null && snapshot == null) {
+      return true;
+    } else if (candidateLastEntry == null) {
+      return false;
+    }
+    TermIndex local = ServerProtoUtils.toTermIndex(lastEntry);
+    if (local == null || (snapshot != null && snapshot.getIndex() > lastEntry.getIndex())) {
+      local = snapshot.getTermIndex();
+    }
+    return local.compareTo(candidateLastEntry) <= 0;
+  }
+
+  @Override
+  public String toString() {
+    return selfId + ":t" + currentTerm + ", leader=" + leaderId
+        + ", voted=" + votedFor + ", raftlog=" + log + ", conf=" + getRaftConf();
+  }
+
+  boolean isConfCommitted() {
+    return getLog().getLastCommittedIndex() >=
+        getRaftConf().getLogEntryIndex();
+  }
+
+  public void setRaftConf(long logIndex, RaftConfiguration conf) {
+    configurationManager.addConfiguration(logIndex, conf);
+    RaftServer.LOG.info("{}: successfully update the configuration {}",
+        getSelfId(), conf);
+  }
+
+  void updateConfiguration(LogEntryProto[] entries) {
+    if (entries != null && entries.length > 0) {
+      configurationManager.removeConfigurations(entries[0].getIndex());
+      for (LogEntryProto entry : entries) {
+        if (ProtoUtils.isConfigurationLogEntry(entry)) {
+          final RaftConfiguration conf = ServerProtoUtils.toRaftConfiguration(
+              entry.getIndex(), entry.getConfigurationEntry());
+          configurationManager.addConfiguration(entry.getIndex(), conf);
+          server.addPeersToRPC(conf.getPeers());
+        }
+      }
+    }
+  }
+
+  void updateStatemachine(long majorityIndex, long currentTerm) {
+    log.updateLastCommitted(majorityIndex, currentTerm);
+    stateMachineUpdater.notifyUpdater();
+  }
+
+  void reloadStateMachine(long lastIndexInSnapshot, long currentTerm)
+      throws IOException {
+    log.updateLastCommitted(lastIndexInSnapshot, currentTerm);
+
+    stateMachineUpdater.reloadStateMachine();
+  }
+
+  @Override
+  public void close() throws IOException {
+    stateMachineUpdater.stop();
+    RaftServer.LOG.info("{} closes. The last applied log index is {}",
+        getSelfId(), getLastAppliedIndex());
+    storage.close();
+  }
+
+  @VisibleForTesting
+  public RaftStorage getStorage() {
+    return storage;
+  }
+
+  void installSnapshot(InstallSnapshotRequestProto request) throws IOException {
+    // TODO: verify that we need to install the snapshot
+    StateMachine sm = server.getStateMachine();
+    sm.pause(); // pause the SM to prepare for install snapshot
+    snapshotManager.installSnapshot(sm, request);
+    log.syncWithSnapshot(request.getTermIndex().getIndex());
+    this.latestInstalledSnapshot = ServerProtoUtils.toTermIndex(
+        request.getTermIndex());
+  }
+
+  SnapshotInfo getLatestSnapshot() {
+    return server.getStateMachine().getStateMachineStorage().getLatestSnapshot();
+  }
+
+  public TermIndex getLatestInstalledSnapshot() {
+    return latestInstalledSnapshot;
+  }
+
+  @VisibleForTesting
+  public long getLastAppliedIndex() {
+    return stateMachineUpdater.getLastAppliedIndex();
+  }
+
+  boolean isCurrentConfCommitted() {
+    return getRaftConf().getLogEntryIndex() <= getLog().getLastCommittedIndex();
+  }
+}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/incubator-ratis/blob/c36810ed/raft-server/src/main/java/org/apache/raft/server/impl/StateMachineUpdater.java
----------------------------------------------------------------------
diff --git a/raft-server/src/main/java/org/apache/raft/server/impl/StateMachineUpdater.java b/raft-server/src/main/java/org/apache/raft/server/impl/StateMachineUpdater.java
new file mode 100644
index 0000000..06fa221
--- /dev/null
+++ b/raft-server/src/main/java/org/apache/raft/server/impl/StateMachineUpdater.java
@@ -0,0 +1,213 @@
+/**
+ * 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.raft.server.impl;
+
+import com.google.common.base.Preconditions;
+import org.apache.raft.conf.RaftProperties;
+import org.apache.raft.protocol.Message;
+import org.apache.raft.server.RaftServerConfigKeys;
+import org.apache.raft.server.storage.RaftLog;
+import org.apache.raft.server.storage.RaftStorage;
+import org.apache.raft.shaded.proto.RaftProtos.LogEntryProto;
+import org.apache.raft.statemachine.SnapshotInfo;
+import org.apache.raft.statemachine.StateMachine;
+import org.apache.raft.statemachine.TransactionContext;
+import org.apache.raft.util.Daemon;
+import org.apache.raft.util.ExitUtils;
+import org.apache.raft.util.LifeCycle;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.io.IOException;
+import java.util.concurrent.CompletableFuture;
+
+import static org.apache.raft.shaded.proto.RaftProtos.LogEntryProto.LogEntryBodyCase.CONFIGURATIONENTRY;
+import static org.apache.raft.shaded.proto.RaftProtos.LogEntryProto.LogEntryBodyCase.SMLOGENTRY;
+
+/**
+ * This class tracks the log entries that have been committed in a quorum and
+ * applies them to the state machine. We let a separate thread do this work
+ * asynchronously so that this will not block normal raft protocol.
+ *
+ * If the auto log compaction is enabled, the state machine updater thread will
+ * trigger a snapshot of the state machine by calling
+ * {@link StateMachine#takeSnapshot} when the log size exceeds a limit.
+ */
+class StateMachineUpdater implements Runnable {
+  static final Logger LOG = LoggerFactory.getLogger(StateMachineUpdater.class);
+
+  enum State {
+    RUNNING, STOP, RELOAD
+  }
+
+  private final RaftProperties properties;
+  private final StateMachine stateMachine;
+  private final RaftServer server;
+  private final RaftLog raftLog;
+
+  private volatile long lastAppliedIndex;
+
+  private final boolean autoSnapshotEnabled;
+  private final long snapshotThreshold;
+  private long lastSnapshotIndex;
+
+  private final Thread updater;
+  private volatile State state = State.RUNNING;
+
+  StateMachineUpdater(StateMachine stateMachine, RaftServer server,
+      RaftLog raftLog, long lastAppliedIndex, RaftProperties properties) {
+    this.properties = properties;
+    this.stateMachine = stateMachine;
+    this.server = server;
+    this.raftLog = raftLog;
+
+    this.lastAppliedIndex = lastAppliedIndex;
+    lastSnapshotIndex = lastAppliedIndex;
+
+    autoSnapshotEnabled = properties.getBoolean(
+        RaftServerConfigKeys.RAFT_SERVER_AUTO_SNAPSHOT_ENABLED_KEY,
+        RaftServerConfigKeys.RAFT_SERVER_AUTO_SNAPSHOT_ENABLED_DEFAULT);
+    snapshotThreshold = properties.getLong(
+        RaftServerConfigKeys.RAFT_SERVER_SNAPSHOT_TRIGGER_THRESHOLD_KEY,
+        RaftServerConfigKeys.RAFT_SERVER_SNAPSHOT_TRIGGER_THRESHOLD_DEFAULT);
+    updater = new Daemon(this);
+  }
+
+  void start() {
+    updater.start();
+  }
+
+  void stop() {
+    state = State.STOP;
+    updater.interrupt();
+    try {
+      stateMachine.close();
+    } catch (IOException ignored) {
+    }
+  }
+
+  void reloadStateMachine() {
+    state = State.RELOAD;
+    notifyUpdater();
+  }
+
+  synchronized void notifyUpdater() {
+    notifyAll();
+  }
+
+  @Override
+  public String toString() {
+    return this.getClass().getSimpleName() + "-" + raftLog.getSelfId();
+  }
+
+  @Override
+  public void run() {
+    final RaftStorage storage = server.getState().getStorage();
+    while (isRunning()) {
+      try {
+        synchronized (this) {
+          // when the peers just start, the committedIndex is initialized as 0
+          // and will be updated only after the leader contacts other peers.
+          // Thus initially lastAppliedIndex can be greater than lastCommitted.
+          while (lastAppliedIndex >= raftLog.getLastCommittedIndex()) {
+            wait();
+          }
+        }
+
+        final long committedIndex = raftLog.getLastCommittedIndex();
+        Preconditions.checkState(lastAppliedIndex < committedIndex);
+
+        if (state == State.RELOAD) {
+          Preconditions.checkState(stateMachine.getLifeCycleState() == LifeCycle.State.PAUSED);
+
+          stateMachine.reinitialize(server.getId(), properties, storage);
+
+          SnapshotInfo snapshot = stateMachine.getLatestSnapshot();
+          Preconditions.checkState(snapshot != null && snapshot.getIndex() > lastAppliedIndex,
+              "Snapshot: %s, lastAppliedIndex: %s", snapshot, lastAppliedIndex);
+
+          lastAppliedIndex = snapshot.getIndex();
+          lastSnapshotIndex = snapshot.getIndex();
+          state = State.RUNNING;
+        }
+
+        while (lastAppliedIndex < committedIndex) {
+          final LogEntryProto next = raftLog.get(lastAppliedIndex + 1);
+          if (next != null) {
+            if (next.getLogEntryBodyCase() == CONFIGURATIONENTRY) {
+              // the reply should have already been set. only need to record
+              // the new conf in the state machine.
+              stateMachine.setRaftConfiguration(
+                  ServerProtoUtils.toRaftConfiguration(next.getIndex(),
+                      next.getConfigurationEntry()));
+            } else if (next.getLogEntryBodyCase() == SMLOGENTRY) {
+              // check whether there is a TransactionContext because we are the leader.
+              TransactionContext trx = server.getTransactionContext(next.getIndex());
+              if (trx == null) {
+                trx = new TransactionContext(stateMachine, next);
+              }
+
+              // Let the StateMachine inject logic for committed transactions in sequential order.
+              trx = stateMachine.applyTransactionSerial(trx);
+
+              // TODO: This step can be parallelized
+              CompletableFuture<Message> messageFuture =
+                  stateMachine.applyTransaction(trx);
+              server.replyPendingRequest(next.getIndex(), messageFuture);
+            }
+            lastAppliedIndex++;
+          } else {
+            LOG.debug("{}: logEntry {} is null. There may be snapshot to load. state:{}",
+                this, lastAppliedIndex + 1, state);
+            break;
+          }
+        }
+
+        // check if need to trigger a snapshot
+        if (shouldTakeSnapshot(lastAppliedIndex)) {
+          stateMachine.takeSnapshot();
+          // TODO purge logs, including log cache. but should keep log for leader's RPCSenders
+          lastSnapshotIndex = lastAppliedIndex;
+        }
+      } catch (InterruptedException e) {
+        if (!isRunning()) {
+          LOG.info("{}: the StateMachineUpdater is interrupted and will exit.", this);
+        } else {
+          final String s = this + ": the StateMachineUpdater is wrongly interrupted";
+          ExitUtils.terminate(1, s, e, LOG);
+        }
+      } catch (Throwable t) {
+        final String s = this + ": the StateMachineUpdater hits Throwable";
+        ExitUtils.terminate(2, s, t, LOG);
+      }
+    }
+  }
+
+  private boolean isRunning() {
+    return state != State.STOP;
+  }
+
+  private boolean shouldTakeSnapshot(long currentAppliedIndex) {
+    return autoSnapshotEnabled && (state != State.RELOAD) &&
+        (currentAppliedIndex - lastSnapshotIndex >= snapshotThreshold);
+  }
+
+  long getLastAppliedIndex() {
+    return lastAppliedIndex;
+  }
+}

http://git-wip-us.apache.org/repos/asf/incubator-ratis/blob/c36810ed/raft-server/src/main/java/org/apache/raft/server/storage/LogInputStream.java
----------------------------------------------------------------------
diff --git a/raft-server/src/main/java/org/apache/raft/server/storage/LogInputStream.java b/raft-server/src/main/java/org/apache/raft/server/storage/LogInputStream.java
index e058ea3..95597b2 100644
--- a/raft-server/src/main/java/org/apache/raft/server/storage/LogInputStream.java
+++ b/raft-server/src/main/java/org/apache/raft/server/storage/LogInputStream.java
@@ -29,7 +29,7 @@ import java.io.EOFException;
 import java.io.File;
 import java.io.IOException;
 
-import static org.apache.raft.server.RaftServerConstants.INVALID_LOG_INDEX;
+import static org.apache.raft.server.impl.RaftServerConstants.INVALID_LOG_INDEX;
 
 public class LogInputStream implements Closeable {
   static final Logger LOG = LoggerFactory.getLogger(LogInputStream.class);

http://git-wip-us.apache.org/repos/asf/incubator-ratis/blob/c36810ed/raft-server/src/main/java/org/apache/raft/server/storage/LogOutputStream.java
----------------------------------------------------------------------
diff --git a/raft-server/src/main/java/org/apache/raft/server/storage/LogOutputStream.java b/raft-server/src/main/java/org/apache/raft/server/storage/LogOutputStream.java
index 1ecb5d5..0dc8029 100644
--- a/raft-server/src/main/java/org/apache/raft/server/storage/LogOutputStream.java
+++ b/raft-server/src/main/java/org/apache/raft/server/storage/LogOutputStream.java
@@ -18,7 +18,7 @@
 package org.apache.raft.server.storage;
 
 import org.apache.raft.conf.RaftProperties;
-import org.apache.raft.server.RaftServerConstants;
+import org.apache.raft.server.impl.RaftServerConstants;
 import org.apache.raft.shaded.com.google.protobuf.CodedOutputStream;
 import org.apache.raft.shaded.proto.RaftProtos.LogEntryProto;
 import org.apache.raft.util.PureJavaCrc32C;

http://git-wip-us.apache.org/repos/asf/incubator-ratis/blob/c36810ed/raft-server/src/main/java/org/apache/raft/server/storage/LogReader.java
----------------------------------------------------------------------
diff --git a/raft-server/src/main/java/org/apache/raft/server/storage/LogReader.java b/raft-server/src/main/java/org/apache/raft/server/storage/LogReader.java
index 4e37e2a..9523cac 100644
--- a/raft-server/src/main/java/org/apache/raft/server/storage/LogReader.java
+++ b/raft-server/src/main/java/org/apache/raft/server/storage/LogReader.java
@@ -20,7 +20,7 @@ package org.apache.raft.server.storage;
 import com.google.common.base.Preconditions;
 import org.apache.commons.io.Charsets;
 import org.apache.raft.protocol.ChecksumException;
-import org.apache.raft.server.RaftServerConstants;
+import org.apache.raft.server.impl.RaftServerConstants;
 import org.apache.raft.shaded.com.google.protobuf.CodedInputStream;
 import org.apache.raft.shaded.com.google.protobuf.CodedOutputStream;
 import org.apache.raft.shaded.proto.RaftProtos.LogEntryProto;

http://git-wip-us.apache.org/repos/asf/incubator-ratis/blob/c36810ed/raft-server/src/main/java/org/apache/raft/server/storage/LogSegment.java
----------------------------------------------------------------------
diff --git a/raft-server/src/main/java/org/apache/raft/server/storage/LogSegment.java b/raft-server/src/main/java/org/apache/raft/server/storage/LogSegment.java
index b2531f0..987cc6c 100644
--- a/raft-server/src/main/java/org/apache/raft/server/storage/LogSegment.java
+++ b/raft-server/src/main/java/org/apache/raft/server/storage/LogSegment.java
@@ -18,7 +18,7 @@
 package org.apache.raft.server.storage;
 
 import com.google.common.base.Preconditions;
-import org.apache.raft.server.ConfigurationManager;
+import org.apache.raft.server.impl.ConfigurationManager;
 import org.apache.raft.server.impl.ServerProtoUtils;
 import org.apache.raft.shaded.com.google.protobuf.CodedOutputStream;
 import org.apache.raft.shaded.proto.RaftProtos.LogEntryProto;

http://git-wip-us.apache.org/repos/asf/incubator-ratis/blob/c36810ed/raft-server/src/main/java/org/apache/raft/server/storage/MemoryRaftLog.java
----------------------------------------------------------------------
diff --git a/raft-server/src/main/java/org/apache/raft/server/storage/MemoryRaftLog.java b/raft-server/src/main/java/org/apache/raft/server/storage/MemoryRaftLog.java
index d87b0a4..c12e1aa 100644
--- a/raft-server/src/main/java/org/apache/raft/server/storage/MemoryRaftLog.java
+++ b/raft-server/src/main/java/org/apache/raft/server/storage/MemoryRaftLog.java
@@ -18,8 +18,8 @@
 package org.apache.raft.server.storage;
 
 import com.google.common.base.Preconditions;
-import org.apache.raft.server.RaftConfiguration;
-import org.apache.raft.server.RaftServerConstants;
+import org.apache.raft.server.impl.RaftConfiguration;
+import org.apache.raft.server.impl.RaftServerConstants;
 import org.apache.raft.server.impl.ServerProtoUtils;
 import org.apache.raft.shaded.proto.RaftProtos.LogEntryProto;
 import org.apache.raft.util.AutoCloseableLock;

http://git-wip-us.apache.org/repos/asf/incubator-ratis/blob/c36810ed/raft-server/src/main/java/org/apache/raft/server/storage/RaftLog.java
----------------------------------------------------------------------
diff --git a/raft-server/src/main/java/org/apache/raft/server/storage/RaftLog.java b/raft-server/src/main/java/org/apache/raft/server/storage/RaftLog.java
index 42f1391..acd44b0 100644
--- a/raft-server/src/main/java/org/apache/raft/server/storage/RaftLog.java
+++ b/raft-server/src/main/java/org/apache/raft/server/storage/RaftLog.java
@@ -18,9 +18,9 @@
 package org.apache.raft.server.storage;
 
 import com.google.common.base.Preconditions;
-import org.apache.raft.server.ConfigurationManager;
-import org.apache.raft.server.RaftConfiguration;
-import org.apache.raft.server.RaftServerConstants;
+import org.apache.raft.server.impl.ConfigurationManager;
+import org.apache.raft.server.impl.RaftConfiguration;
+import org.apache.raft.server.impl.RaftServerConstants;
 import org.apache.raft.server.impl.ServerProtoUtils;
 import org.apache.raft.server.protocol.TermIndex;
 import org.apache.raft.shaded.proto.RaftProtos.LogEntryProto;

http://git-wip-us.apache.org/repos/asf/incubator-ratis/blob/c36810ed/raft-server/src/main/java/org/apache/raft/server/storage/RaftLogCache.java
----------------------------------------------------------------------
diff --git a/raft-server/src/main/java/org/apache/raft/server/storage/RaftLogCache.java b/raft-server/src/main/java/org/apache/raft/server/storage/RaftLogCache.java
index bdc0675..d022a91 100644
--- a/raft-server/src/main/java/org/apache/raft/server/storage/RaftLogCache.java
+++ b/raft-server/src/main/java/org/apache/raft/server/storage/RaftLogCache.java
@@ -19,14 +19,14 @@ package org.apache.raft.server.storage;
 
 import com.google.common.annotations.VisibleForTesting;
 import com.google.common.base.Preconditions;
-import org.apache.raft.server.RaftServerConstants;
+import org.apache.raft.server.impl.RaftServerConstants;
 import org.apache.raft.server.storage.LogSegment.LogRecord;
 import org.apache.raft.server.storage.LogSegment.SegmentFileInfo;
 import org.apache.raft.shaded.proto.RaftProtos.LogEntryProto;
 
 import java.util.*;
 
-import static org.apache.raft.server.RaftServerConstants.INVALID_LOG_INDEX;
+import static org.apache.raft.server.impl.RaftServerConstants.INVALID_LOG_INDEX;
 
 /**
  * In-memory RaftLog Cache. Currently we provide a simple implementation that

http://git-wip-us.apache.org/repos/asf/incubator-ratis/blob/c36810ed/raft-server/src/main/java/org/apache/raft/server/storage/RaftLogWorker.java
----------------------------------------------------------------------
diff --git a/raft-server/src/main/java/org/apache/raft/server/storage/RaftLogWorker.java b/raft-server/src/main/java/org/apache/raft/server/storage/RaftLogWorker.java
index 090be49..1837e94 100644
--- a/raft-server/src/main/java/org/apache/raft/server/storage/RaftLogWorker.java
+++ b/raft-server/src/main/java/org/apache/raft/server/storage/RaftLogWorker.java
@@ -20,8 +20,8 @@ package org.apache.raft.server.storage;
 import com.google.common.base.Preconditions;
 import org.apache.raft.conf.RaftProperties;
 import org.apache.raft.io.nativeio.NativeIO;
-import org.apache.raft.server.RaftServer;
-import org.apache.raft.server.RaftServerConstants;
+import org.apache.raft.server.impl.RaftServer;
+import org.apache.raft.server.impl.RaftServerConstants;
 import org.apache.raft.server.storage.LogSegment.SegmentFileInfo;
 import org.apache.raft.server.storage.RaftLogCache.TruncationSegments;
 import org.apache.raft.server.storage.SegmentedRaftLog.Task;

http://git-wip-us.apache.org/repos/asf/incubator-ratis/blob/c36810ed/raft-server/src/main/java/org/apache/raft/server/storage/RaftStorage.java
----------------------------------------------------------------------
diff --git a/raft-server/src/main/java/org/apache/raft/server/storage/RaftStorage.java b/raft-server/src/main/java/org/apache/raft/server/storage/RaftStorage.java
index 8646b9a..434f505 100644
--- a/raft-server/src/main/java/org/apache/raft/server/storage/RaftStorage.java
+++ b/raft-server/src/main/java/org/apache/raft/server/storage/RaftStorage.java
@@ -19,7 +19,7 @@ package org.apache.raft.server.storage;
 
 import com.google.common.base.Preconditions;
 import org.apache.raft.conf.RaftProperties;
-import org.apache.raft.server.RaftServerConstants;
+import org.apache.raft.server.impl.RaftServerConstants;
 import org.apache.raft.server.storage.RaftStorageDirectory.StorageState;
 import org.apache.raft.statemachine.SnapshotInfo;
 import org.apache.raft.statemachine.StateMachineStorage;

http://git-wip-us.apache.org/repos/asf/incubator-ratis/blob/c36810ed/raft-server/src/main/java/org/apache/raft/server/storage/RaftStorageDirectory.java
----------------------------------------------------------------------
diff --git a/raft-server/src/main/java/org/apache/raft/server/storage/RaftStorageDirectory.java b/raft-server/src/main/java/org/apache/raft/server/storage/RaftStorageDirectory.java
index e47f3a6..662e4ec 100644
--- a/raft-server/src/main/java/org/apache/raft/server/storage/RaftStorageDirectory.java
+++ b/raft-server/src/main/java/org/apache/raft/server/storage/RaftStorageDirectory.java
@@ -40,7 +40,7 @@ import java.util.regex.Matcher;
 import java.util.regex.Pattern;
 
 import static java.nio.file.Files.newDirectoryStream;
-import static org.apache.raft.server.RaftServerConstants.INVALID_LOG_INDEX;
+import static org.apache.raft.server.impl.RaftServerConstants.INVALID_LOG_INDEX;
 
 public class RaftStorageDirectory {
   static final Logger LOG = LoggerFactory.getLogger(RaftStorageDirectory.class);

http://git-wip-us.apache.org/repos/asf/incubator-ratis/blob/c36810ed/raft-server/src/main/java/org/apache/raft/server/storage/SegmentedRaftLog.java
----------------------------------------------------------------------
diff --git a/raft-server/src/main/java/org/apache/raft/server/storage/SegmentedRaftLog.java b/raft-server/src/main/java/org/apache/raft/server/storage/SegmentedRaftLog.java
index d87fc0a..9c55491 100644
--- a/raft-server/src/main/java/org/apache/raft/server/storage/SegmentedRaftLog.java
+++ b/raft-server/src/main/java/org/apache/raft/server/storage/SegmentedRaftLog.java
@@ -21,9 +21,9 @@ import com.google.common.annotations.VisibleForTesting;
 import com.google.common.base.Preconditions;
 import org.apache.commons.io.Charsets;
 import org.apache.raft.conf.RaftProperties;
-import org.apache.raft.server.ConfigurationManager;
-import org.apache.raft.server.RaftServer;
-import org.apache.raft.server.RaftServerConstants;
+import org.apache.raft.server.impl.ConfigurationManager;
+import org.apache.raft.server.impl.RaftServer;
+import org.apache.raft.server.impl.RaftServerConstants;
 import org.apache.raft.server.storage.RaftStorageDirectory.LogPathAndIndex;
 import org.apache.raft.shaded.proto.RaftProtos.LogEntryProto;
 import org.apache.raft.util.AutoCloseableLock;

http://git-wip-us.apache.org/repos/asf/incubator-ratis/blob/c36810ed/raft-server/src/main/java/org/apache/raft/statemachine/BaseStateMachine.java
----------------------------------------------------------------------
diff --git a/raft-server/src/main/java/org/apache/raft/statemachine/BaseStateMachine.java b/raft-server/src/main/java/org/apache/raft/statemachine/BaseStateMachine.java
index 33fff3f..ccc52c7 100644
--- a/raft-server/src/main/java/org/apache/raft/statemachine/BaseStateMachine.java
+++ b/raft-server/src/main/java/org/apache/raft/statemachine/BaseStateMachine.java
@@ -22,8 +22,8 @@ import org.apache.raft.conf.RaftProperties;
 import org.apache.raft.protocol.Message;
 import org.apache.raft.protocol.RaftClientReply;
 import org.apache.raft.protocol.RaftClientRequest;
-import org.apache.raft.server.RaftConfiguration;
-import org.apache.raft.server.RaftServerConstants;
+import org.apache.raft.server.impl.RaftConfiguration;
+import org.apache.raft.server.impl.RaftServerConstants;
 import org.apache.raft.server.storage.RaftStorage;
 import org.apache.raft.shaded.proto.RaftProtos.SMLogEntryProto;
 import org.apache.raft.util.LifeCycle;

http://git-wip-us.apache.org/repos/asf/incubator-ratis/blob/c36810ed/raft-server/src/main/java/org/apache/raft/statemachine/SimpleStateMachineStorage.java
----------------------------------------------------------------------
diff --git a/raft-server/src/main/java/org/apache/raft/statemachine/SimpleStateMachineStorage.java b/raft-server/src/main/java/org/apache/raft/statemachine/SimpleStateMachineStorage.java
index ad9fee3..bedb5b0 100644
--- a/raft-server/src/main/java/org/apache/raft/statemachine/SimpleStateMachineStorage.java
+++ b/raft-server/src/main/java/org/apache/raft/statemachine/SimpleStateMachineStorage.java
@@ -21,7 +21,7 @@ package org.apache.raft.statemachine;
 import com.google.common.annotations.VisibleForTesting;
 import com.google.common.collect.Lists;
 import org.apache.raft.io.MD5Hash;
-import org.apache.raft.server.RaftConfiguration;
+import org.apache.raft.server.impl.RaftConfiguration;
 import org.apache.raft.server.protocol.TermIndex;
 import org.apache.raft.server.storage.FileInfo;
 import org.apache.raft.server.storage.RaftStorage;



[39/54] [abbrv] incubator-ratis git commit: Renamed the packages from raft to ratis in preperation for Apache Incubation - Moved all java packages from org.apache.raft to org.apache.ratis. - Moved native package to org_apache_ratis, and native lib to l

Posted by ji...@apache.org.
http://git-wip-us.apache.org/repos/asf/incubator-ratis/blob/7e71a2e0/raft-project/pom.xml
----------------------------------------------------------------------
diff --git a/raft-project/pom.xml b/raft-project/pom.xml
deleted file mode 100644
index d25c71b..0000000
--- a/raft-project/pom.xml
+++ /dev/null
@@ -1,409 +0,0 @@
-<?xml version="1.0" encoding="UTF-8"?>
-<!--
-  Licensed 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. See accompanying LICENSE file.
--->
-<project xmlns="http://maven.apache.org/POM/4.0.0"
-         xmlns:xsi="http://www.w3.org/2001/XMLSchema-instance"
-         xsi:schemaLocation="http://maven.apache.org/POM/4.0.0 http://maven.apache.org/xsd/maven-4.0.0.xsd">
-  <modelVersion>4.0.0</modelVersion>
-  <parent>
-    <artifactId>raft-main</artifactId>
-    <groupId>com.hortonworks.raft</groupId>
-    <version>1.0-SNAPSHOT</version>
-  </parent>
-
-  <artifactId>raft-project</artifactId>
-  <groupId>com.hortonworks.raft</groupId>
-  <name>Raft Project</name>
-  <packaging>pom</packaging>
-
-  <modules>
-    <module>../raft-project-dist</module>
-    <module>../raft-common</module>
-    <module>../raft-client</module>
-    <module>../raft-server</module>
-    <module>../raft-hadoop</module>
-    <module>../raft-grpc</module>
-    <module>../raft-netty</module>
-    <module>../raft-examples</module>
-  </modules>
-
-  <properties>
-    <!-- Set the Release year during release -->
-    <release-year>2016</release-year>
-
-    <maven.test.redirectTestOutputToFile>true
-    </maven.test.redirectTestOutputToFile>
-    <test.exclude>_</test.exclude>
-    <test.exclude.pattern>_</test.exclude.pattern>
-
-    <!-- number of threads/forks to use when running tests in parallel, see parallel-tests profile -->
-    <testsThreadCount>4</testsThreadCount>
-
-    <test.build.dir>${project.build.directory}/test-dir</test.build.dir>
-    <test.build.data>${test.build.dir}</test.build.data>
-
-    <findbugs.version>3.0.0</findbugs.version>
-
-    <maven-pdf-plugin.version>1.2</maven-pdf-plugin.version>
-    <build-helper-maven-plugin.version>1.9</build-helper-maven-plugin.version>
-    <exec-maven-plugin.version>1.3.1</exec-maven-plugin.version>
-    <make-maven-plugin.version>1.0-beta-1</make-maven-plugin.version>
-    <native-maven-plugin.version>1.0-alpha-8</native-maven-plugin.version>
-
-    <hadoop.version>3.0.0-alpha1</hadoop.version>
-  </properties>
-
-  <dependencyManagement>
-    <dependencies>
-      <dependency>
-        <artifactId>raft-proto-shaded</artifactId>
-        <groupId>com.hortonworks.raft</groupId>
-        <version>${project.version}</version>
-      </dependency>
-
-      <dependency>
-        <artifactId>raft-common</artifactId>
-        <groupId>com.hortonworks.raft</groupId>
-        <version>${project.version}</version>
-      </dependency>
-      <dependency>
-        <artifactId>raft-common</artifactId>
-        <groupId>com.hortonworks.raft</groupId>
-        <version>${project.version}</version>
-        <type>test-jar</type>
-      </dependency>
-
-      <dependency>
-        <artifactId>raft-client</artifactId>
-        <groupId>com.hortonworks.raft</groupId>
-        <version>${project.version}</version>
-      </dependency>
-      <dependency>
-        <artifactId>raft-client</artifactId>
-        <groupId>com.hortonworks.raft</groupId>
-        <version>${project.version}</version>
-        <type>test-jar</type>
-      </dependency>
-
-      <dependency>
-        <artifactId>raft-hadoop</artifactId>
-        <groupId>com.hortonworks.raft</groupId>
-        <version>${project.version}</version>
-      </dependency>
-      <dependency>
-        <artifactId>raft-hadoop</artifactId>
-        <groupId>com.hortonworks.raft</groupId>
-        <version>${project.version}</version>
-        <type>test-jar</type>
-      </dependency>
-
-      <dependency>
-        <artifactId>raft-grpc</artifactId>
-        <groupId>com.hortonworks.raft</groupId>
-        <version>${project.version}</version>
-      </dependency>
-      <dependency>
-        <artifactId>raft-grpc</artifactId>
-        <groupId>com.hortonworks.raft</groupId>
-        <version>${project.version}</version>
-        <type>test-jar</type>
-      </dependency>
-
-      <dependency>
-        <artifactId>raft-netty</artifactId>
-        <groupId>com.hortonworks.raft</groupId>
-        <version>${project.version}</version>
-      </dependency>
-      <dependency>
-        <artifactId>raft-netty</artifactId>
-        <groupId>com.hortonworks.raft</groupId>
-        <version>${project.version}</version>
-        <type>test-jar</type>
-      </dependency>
-
-      <dependency>
-        <artifactId>raft-server</artifactId>
-        <groupId>com.hortonworks.raft</groupId>
-        <version>${project.version}</version>
-      </dependency>
-      <dependency>
-        <artifactId>raft-server</artifactId>
-        <groupId>com.hortonworks.raft</groupId>
-        <version>${project.version}</version>
-        <type>test-jar</type>
-      </dependency>
-
-      <dependency>
-        <groupId>org.slf4j</groupId>
-        <artifactId>slf4j-api</artifactId>
-        <version>1.7.10</version>
-      </dependency>
-      <dependency>
-        <groupId>org.slf4j</groupId>
-        <artifactId>slf4j-log4j12</artifactId>
-        <version>1.7.10</version>
-      </dependency>
-
-      <dependency>
-        <groupId>com.google.guava</groupId>
-        <artifactId>guava</artifactId>
-        <version>20.0</version>
-      </dependency>
-
-      <dependency>
-        <groupId>io.netty</groupId>
-        <artifactId>netty-all</artifactId>
-        <version>4.1.6.Final</version>
-      </dependency>
-
-      <dependency>
-        <groupId>junit</groupId>
-        <artifactId>junit</artifactId>
-        <version>4.11</version>
-      </dependency>
-      <dependency>
-        <groupId>org.mockito</groupId>
-        <artifactId>mockito-all</artifactId>
-        <version>1.8.5</version>
-      </dependency>
-    </dependencies>
-  </dependencyManagement>
-
-  <build>
-    <pluginManagement>
-      <plugins>
-        <plugin>
-          <artifactId>maven-clean-plugin</artifactId>
-          <version>${maven-clean-plugin.version}</version>
-        </plugin>
-        <plugin>
-          <groupId>org.codehaus.mojo</groupId>
-          <artifactId>build-helper-maven-plugin</artifactId>
-          <version>${build-helper-maven-plugin.version}</version>
-        </plugin>
-        <plugin>
-          <groupId>org.apache.maven.plugins</groupId>
-          <artifactId>maven-install-plugin</artifactId>
-          <version>${maven-install-plugin.version}</version>
-        </plugin>
-        <plugin>
-          <groupId>org.apache.maven.plugins</groupId>
-          <artifactId>maven-jar-plugin</artifactId>
-          <version>${maven-jar-plugin.version}</version>
-        </plugin>
-        <plugin>
-          <groupId>org.apache.maven.plugins</groupId>
-          <artifactId>maven-javadoc-plugin</artifactId>
-          <version>${maven-javadoc-plugin.version}</version>
-          <configuration>
-            <additionalparam>-Xmaxwarns 10000</additionalparam>
-          </configuration>
-        </plugin>
-        <plugin>
-          <groupId>org.codehaus.mojo</groupId>
-          <artifactId>findbugs-maven-plugin</artifactId>
-          <version>${findbugs.version}</version>
-        </plugin>
-        <plugin>
-          <groupId>org.codehaus.mojo</groupId>
-          <artifactId>native-maven-plugin</artifactId>
-          <version>${native-maven-plugin.version}</version>
-        </plugin>
-        <plugin>
-          <groupId>org.codehaus.mojo</groupId>
-          <artifactId>make-maven-plugin</artifactId>
-          <version>${make-maven-plugin.version}</version>
-        </plugin>
-        <plugin>
-          <groupId>org.apache.maven.plugins</groupId>
-          <artifactId>maven-resources-plugin</artifactId>
-          <version>${maven-resources-plugin.version}</version>
-        </plugin>
-        <plugin>
-          <groupId>org.codehaus.mojo</groupId>
-          <artifactId>exec-maven-plugin</artifactId>
-          <version>${exec-maven-plugin.version}</version>
-        </plugin>
-        <plugin>
-          <groupId>org.apache.maven.plugins</groupId>
-          <artifactId>maven-pdf-plugin</artifactId>
-          <version>${maven-pdf-plugin.version}</version>
-        </plugin>
-
-        <plugin>
-          <groupId>org.apache.hadoop</groupId>
-          <artifactId>hadoop-maven-plugins</artifactId>
-          <version>${hadoop.version}</version>
-        </plugin>
-      </plugins>
-    </pluginManagement>
-
-    <plugins>
-      <plugin>
-        <groupId>org.codehaus.mojo</groupId>
-        <artifactId>findbugs-maven-plugin</artifactId>
-      </plugin>
-
-      <plugin>
-        <groupId>org.apache.maven.plugins</groupId>
-        <artifactId>maven-antrun-plugin</artifactId>
-        <executions>
-          <execution>
-            <id>create-testdirs</id>
-            <phase>validate</phase>
-            <goals>
-              <goal>run</goal>
-            </goals>
-            <configuration>
-              <target>
-                <mkdir dir="${test.build.dir}"/>
-                <mkdir dir="${test.build.data}"/>
-              </target>
-            </configuration>
-          </execution>
-        </executions>
-      </plugin>
-
-      <plugin>
-        <groupId>org.apache.maven.plugins</groupId>
-        <artifactId>maven-pdf-plugin</artifactId>
-        <configuration>
-          <outputDirectory>${project.reporting.outputDirectory}
-          </outputDirectory>
-          <includeReports>false</includeReports>
-        </configuration>
-      </plugin>
-    </plugins>
-  </build>
-
-
-  <profiles>
-    <profile>
-      <id>os.linux</id>
-      <activation>
-        <os>
-          <family>!Mac</family>
-        </os>
-      </activation>
-      <properties>
-        <build.platform>${os.name}-${os.arch}-${sun.arch.data.model}
-        </build.platform>
-      </properties>
-    </profile>
-    <profile>
-      <id>os.mac</id>
-      <activation>
-        <os>
-          <family>Mac</family>
-        </os>
-      </activation>
-      <properties>
-        <build.platform>Mac_OS_X-${sun.arch.data.model}</build.platform>
-      </properties>
-    </profile>
-    <profile>
-      <id>native-win</id>
-      <activation>
-        <os>
-          <family>Windows</family>
-        </os>
-      </activation>
-      <properties>
-        <!-- We must use this exact string for egd on Windows, because the -->
-        <!-- JVM will check for an exact string match on this.  If found, it -->
-        <!-- will use a native entropy provider.  This will not really -->
-        <!-- attempt to open a file at this path. -->
-        <java.security.egd>file:/dev/urandom</java.security.egd>
-        <bundle.snappy.in.bin>true</bundle.snappy.in.bin>
-        <bundle.openssl.in.bin>true</bundle.openssl.in.bin>
-      </properties>
-    </profile>
-    <profile>
-      <id>test-patch</id>
-      <activation>
-        <activeByDefault>false</activeByDefault>
-      </activation>
-      <build>
-        <plugins>
-          <plugin>
-            <groupId>org.apache.maven.plugins</groupId>
-            <artifactId>maven-compiler-plugin</artifactId>
-            <configuration>
-              <fork>true</fork>
-              <source>${javac.version}</source>
-              <target>${javac.version}</target>
-              <compilerArguments>
-                <Xlint/>
-                <Xmaxwarns>9999</Xmaxwarns>
-              </compilerArguments>
-            </configuration>
-          </plugin>
-        </plugins>
-      </build>
-    </profile>
-    <profile>
-      <id>dist</id>
-      <build>
-        <plugins>
-          <plugin>
-            <groupId>org.apache.maven.plugins</groupId>
-            <artifactId>maven-javadoc-plugin</artifactId>
-            <executions>
-              <execution>
-                <!-- build javadoc jars per jar for publishing to maven -->
-                <id>module-javadocs</id>
-                <phase>package</phase>
-                <goals>
-                  <goal>jar</goal>
-                </goals>
-                <configuration>
-                  <destDir>${project.build.directory}</destDir>
-                </configuration>
-              </execution>
-            </executions>
-          </plugin>
-          <plugin>
-            <groupId>org.apache.maven.plugins</groupId>
-            <artifactId>maven-source-plugin</artifactId>
-            <executions>
-              <execution>
-                <!-- builds source jars and attaches them to the project for publishing -->
-                <id>raft-java-sources</id>
-                <phase>package</phase>
-                <goals>
-                  <goal>jar-no-fork</goal>
-                  <goal>test-jar-no-fork</goal>
-                </goals>
-              </execution>
-            </executions>
-          </plugin>
-          <plugin>
-            <groupId>org.apache.maven.plugins</groupId>
-            <artifactId>maven-enforcer-plugin</artifactId>
-            <executions>
-              <execution>
-                <id>dist-enforce</id>
-                <phase>package</phase>
-                <goals>
-                  <goal>enforce</goal>
-                </goals>
-              </execution>
-            </executions>
-          </plugin>
-        </plugins>
-      </build>
-    </profile>
-  </profiles>
-
-</project>

http://git-wip-us.apache.org/repos/asf/incubator-ratis/blob/7e71a2e0/raft-proto-shaded/.gitignore
----------------------------------------------------------------------
diff --git a/raft-proto-shaded/.gitignore b/raft-proto-shaded/.gitignore
deleted file mode 100644
index ce50505..0000000
--- a/raft-proto-shaded/.gitignore
+++ /dev/null
@@ -1,2 +0,0 @@
-src/main/java
-dependency-reduced-pom.xml

http://git-wip-us.apache.org/repos/asf/incubator-ratis/blob/7e71a2e0/raft-proto-shaded/README.md
----------------------------------------------------------------------
diff --git a/raft-proto-shaded/README.md b/raft-proto-shaded/README.md
deleted file mode 100644
index 6859aed..0000000
--- a/raft-proto-shaded/README.md
+++ /dev/null
@@ -1,23 +0,0 @@
-# Raft Proto Shaded
-
-This module is to shade protos, protobuf and other libraries such as Netty, gRPC and Hadoop
-so that applications using Raft may use protobuf and other libraries with versions different 
-from the versions used here.
-
-Other modules require the shaded sources for compilation. To generate them,
-run the following command under `raft-proto-shaded/`
-
-- `mvn package -Dcompile-protobuf -DskipTests`
-
-The generated sources are stored in `raft-proto-shaded/src/main/java/`.
-
-## What are shaded?
-
-| Original packages                 | Shaded packages                                          |
-| ----------------------------------|----------------------------------------------------------|
-| `com.google.protobuf`             | `org.apache.raft.shaded.com.google.protobuf`             |
-| `io.grpc`                         | `org.apache.raft.shaded.io.grpc`                         |
-| `io.netty.handler.codec.protobuf` | `org.apache.raft.shaded.io.netty.handler.codec.protobuf` |
-| `org.apache.hadoop.ipc.protobuf`  | `org.apache.raft.shaded.org.apache.hadoop.ipc.protobuf`  |
-
-The protos defined in this project are stored in the `org.apache.raft.shaded.proto` package.

http://git-wip-us.apache.org/repos/asf/incubator-ratis/blob/7e71a2e0/raft-proto-shaded/pom.xml
----------------------------------------------------------------------
diff --git a/raft-proto-shaded/pom.xml b/raft-proto-shaded/pom.xml
deleted file mode 100644
index d431b1a..0000000
--- a/raft-proto-shaded/pom.xml
+++ /dev/null
@@ -1,426 +0,0 @@
-<?xml version="1.0" encoding="UTF-8"?>
-<!--
-  Licensed 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. See accompanying LICENSE file.
--->
-<project xmlns="http://maven.apache.org/POM/4.0.0"
-         xmlns:xsi="http://www.w3.org/2001/XMLSchema-instance"
-         xsi:schemaLocation="http://maven.apache.org/POM/4.0.0 http://maven.apache.org/xsd/maven-4.0.0.xsd">
-  <modelVersion>4.0.0</modelVersion>
-  <parent>
-    <artifactId>raft-main</artifactId>
-    <groupId>com.hortonworks.raft</groupId>
-    <version>1.0-SNAPSHOT</version>
-    <relativePath>..</relativePath>
-  </parent>
-
-  <artifactId>raft-proto-shaded</artifactId>
-  <name>Raft Proto Shaded</name>
-  <properties>
-    <maven.javadoc.skip>true</maven.javadoc.skip>
-    <!--The Default target dir-->
-    <classes.dir>${project.build.directory}/classes</classes.dir>
-    <!--The Default location for sources-->
-    <sources.dir>src/main/java</sources.dir>
-
-    <!--Version of protobuf to be shaded -->
-    <shaded.protobuf.version>3.1.0</shaded.protobuf.version>
-    <!--Version of grpc to be shaded -->
-    <shaded.grpc.version>1.0.1</shaded.grpc.version>
-    <!--Version of Hadoop to be shaded -->
-    <shaded.hadoop.version>3.0.0-alpha1</shaded.hadoop.version>
-  </properties>
-
-  <build>
-    <!--I want to override these in profile so define them
-         with variables up here-->
-    <sourceDirectory>${sources.dir}</sourceDirectory>
-    <outputDirectory>${classes.dir}</outputDirectory>
-    <plugins>
-      <!-- Make a jar and put the sources in the jar -->
-      <plugin>
-        <groupId>org.apache.maven.plugins</groupId>
-        <artifactId>maven-source-plugin</artifactId>
-        <version>${maven-source-plugin.version}</version>
-      </plugin>
-      <plugin>
-        <!--Make it so assembly:single does nothing in here-->
-        <artifactId>maven-assembly-plugin</artifactId>
-        <version>${maven-assembly-plugin.version}</version>
-        <configuration>
-          <skipAssembly>true</skipAssembly>
-        </configuration>
-      </plugin>
-      <plugin>
-        <artifactId>maven-surefire-plugin</artifactId>
-        <!-- Always skip the second part executions
-             since we only run simple unit tests in this module -->
-        <executions>
-          <execution>
-            <id>secondPartTestsExecution</id>
-            <phase>test</phase>
-            <goals>
-              <goal>test</goal>
-            </goals>
-            <configuration>
-              <skip>true</skip>
-            </configuration>
-          </execution>
-        </executions>
-      </plugin>
-    </plugins>
-  </build>
-
-  <dependencies>
-    <dependency>
-      <groupId>com.google.protobuf</groupId>
-      <artifactId>protobuf-java</artifactId>
-      <version>${shaded.protobuf.version}</version>
-    </dependency>
-    <dependency>
-      <groupId>com.google.protobuf.nano</groupId>
-      <artifactId>protobuf-javanano</artifactId>
-      <version>${shaded.protobuf.version}</version>
-    </dependency>
-
-    <dependency>
-      <groupId>io.grpc</groupId>
-      <artifactId>grpc-netty</artifactId>
-      <version>${shaded.grpc.version}</version>
-    </dependency>
-    <dependency>
-      <groupId>io.grpc</groupId>
-      <artifactId>grpc-protobuf</artifactId>
-      <version>${shaded.grpc.version}</version>
-    </dependency>
-    <dependency>
-      <groupId>io.grpc</groupId>
-      <artifactId>grpc-stub</artifactId>
-      <version>${shaded.grpc.version}</version>
-    </dependency>
-
-    <dependency>
-      <groupId>org.apache.hadoop</groupId>
-      <artifactId>hadoop-common</artifactId>
-      <version>${shaded.hadoop.version}</version>
-    </dependency>
-  </dependencies>
-
-  <profiles>
-    <!-- Skip the tests in this module -->
-    <profile>
-      <id>skip-proto-shaded-tests</id>
-      <activation>
-        <property>
-          <name>skip-proto-shaded-tests</name>
-        </property>
-      </activation>
-      <properties>
-        <surefire.skipFirstPart>true</surefire.skipFirstPart>
-      </properties>
-    </profile>
-
-    <profile>
-      <id>compile-protobuf</id>
-      <!--
-         Generate and shade proto files. Drops generated java files
-         under src/main/java. Check in the generated files so available
-         at build time. Run this profile/step everytime you change proto
-         files or update the protobuf version.
-
-         The below does a bunch of ugly stuff. It purges current content
-         of the generated and shaded com.google.protobuf java files first.
-         It does this because later we apply patches later and patches
-         fail they've already been applied. We remove too because we
-         overlay the shaded protobuf and if files have been removed or
-         added, it'll be more plain if we have first done this delete.
-
-         Next up we generate proto, build a jar, shade it (which
-         includes the referenced protobuf), undo it over the src/main/java
-         directory, and then apply patches.
-
-         The result needs to be checked in.
-      -->
-      <activation>
-        <property>
-          <name>compile-protobuf</name>
-        </property>
-      </activation>
-      <properties>
-        <profile.id>compile-protobuf</profile.id>
-        <sources.dir>${project.build.directory}/protoc-generated-sources</sources.dir>
-        <classes.dir>${project.build.directory}/protoc-generated-classes</classes.dir>
-        <!--When the compile for this profile runs, make sure it makes jars that
-             can be related back to this shading profile. Give them a shading prefix.
-         -->
-        <jar.finalName>${profile.id}.${project.artifactId}-${project.version}</jar.finalName>
-      </properties>
-      <build>
-        <finalName>${jar.finalName}</finalName>
-        <plugins>
-          <plugin>
-            <artifactId>maven-clean-plugin</artifactId>
-            <version>${maven-clean-plugin.version}</version>
-            <executions>
-              <execution>
-                <id>pre-compile-protoc</id>
-                <phase>generate-sources</phase>
-                <goals>
-                  <goal>clean</goal>
-                </goals>
-                <configuration>
-                  <filesets>
-                    <fileset>
-                      <directory>
-                        ${basedir}/src/main/java/
-                      </directory>
-                      <followSymlinks>false</followSymlinks>
-                    </fileset>
-                  </filesets>
-                </configuration>
-              </execution>
-            </executions>
-          </plugin>
-
-          <plugin>
-            <groupId>org.xolstice.maven.plugins</groupId>
-            <artifactId>protobuf-maven-plugin</artifactId>
-            <version>${maven-xolstice-plugin.version}</version>
-            <configuration>
-              <protocArtifact>
-                com.google.protobuf:protoc:${shaded.protobuf.version}:exe:${os.detected.classifier}
-              </protocArtifact>
-            </configuration>
-            <executions>
-              <execution>
-                <id>1</id>
-                <goals>
-                  <goal>compile</goal>
-                  <goal>test-compile</goal>
-                </goals>
-              </execution>
-              <execution>
-                <id>2</id>
-                <phase>generate-sources</phase>
-                <goals>
-                  <goal>compile-custom</goal>
-                  <goal>test-compile-custom</goal>
-                </goals>
-                <configuration>
-                  <pluginId>grpc-java</pluginId>
-                  <pluginArtifact>
-                    io.grpc:protoc-gen-grpc-java:${shaded.grpc.version}:exe:${os.detected.classifier}
-                  </pluginArtifact>
-                </configuration>
-              </execution>
-            </executions>
-          </plugin>
-
-          <plugin>
-            <groupId>org.apache.maven.plugins</groupId>
-            <artifactId>maven-shade-plugin</artifactId>
-            <version>2.4.3</version>
-            <executions>
-              <execution>
-                <phase>package</phase>
-                <goals>
-                  <goal>shade</goal>
-                </goals>
-                <configuration>
-                  <shadeSourcesContent>true</shadeSourcesContent>
-                  <createSourcesJar>true</createSourcesJar>
-                  <relocations>
-                    <relocation>
-                      <pattern>com.google.protobuf</pattern>
-                      <shadedPattern>org.apache.raft.shaded.com.google.protobuf</shadedPattern>
-                    </relocation>
-                    <relocation>
-                      <pattern>io.grpc</pattern>
-                      <shadedPattern>org.apache.raft.shaded.io.grpc</shadedPattern>
-                    </relocation>
-                    <relocation>
-                      <pattern>io.netty.handler.codec.protobuf</pattern>
-                      <shadedPattern>org.apache.raft.shaded.io.netty.handler.codec.protobuf</shadedPattern>
-                    </relocation>
-
-                    <relocation>
-                      <pattern>org.apache.hadoop.ipc.protobuf</pattern>
-                      <shadedPattern>org.apache.raft.shaded.org.apache.hadoop.ipc.protobuf</shadedPattern>
-                    </relocation>
-                  </relocations>
-
-                  <filters>
-                    <filter>
-                      <artifact>io.netty:netty-codec</artifact>
-                      <includes>
-                        <include>io/netty/handler/codec/protobuf/**</include>
-                      </includes>
-                    </filter>
-                    <filter>
-                      <artifact>org.apache.hadoop:hadoop-common</artifact>
-                      <includes>
-                        <include>org/apache/hadoop/ipc/protobuf/**</include>
-                      </includes>
-                    </filter>
-                  </filters>
-
-                  <artifactSet>
-                    <excludes>
-                      <exclude>asm:asm</exclude>
-
-                      <exclude>com.google.code.findbugs</exclude>
-                      <exclude>com.google.code.gson:gson</exclude>
-                      <exclude>com.google.guava:guava</exclude>
-		      com.google.re2j:re2j
-
-                      <exclude>com.jamesmurty.utils:java-xmlbuilder</exclude>
-                      <exclude>com.jcraft:jsch</exclude>
-
-                      <exclude>com.sun.jersey:jersey-core</exclude>
-                      <exclude>com.sun.jersey:jersey-json</exclude>
-                      <exclude>com.sun.jersey:jersey-server</exclude>
-                      <exclude>com.sun.jersey:jersey-servlet</exclude>
-                      <exclude>com.sun.xml.bind:jaxb-impl</exclude>
- 
-                      <exclude>com.thoughtworks.paranamer:paranamer</exclude>
-
-                      <exclude>commons-beanutils:commons-beanutils-core</exclude>
-                      <exclude>commons-beanutils:commons-beanutils</exclude>
-                      <exclude>commons-cli:commons-cli</exclude>
-                      <exclude>commons-codec:commons-codec</exclude>
-                      <exclude>commons-collections:commons-collections</exclude>
-                      <exclude>commons-configuration:commons-configuration</exclude>
-                      <exclude>commons-digester:commons-digester</exclude>
-                      <exclude>commons-httpclient:commons-httpclient</exclude>
-                      <exclude>commons-io:commons-io</exclude>
-                      <exclude>commons-lang:commons-lang</exclude>
-                      <exclude>commons-logging:commons-logging</exclude>
-                      <exclude>commons-net:commons-net</exclude>
-
-                      <exclude>io.netty:netty-buffer</exclude>
-                      <exclude>io.netty:netty-codec-http2</exclude>
-                      <exclude>io.netty:netty-codec-http</exclude>
-                      <exclude>io.netty:netty-common</exclude>
-                      <exclude>io.netty:netty-handler</exclude>
-                      <exclude>io.netty:netty-resolver</exclude>
-                      <exclude>io.netty:netty-transport</exclude>
-                      <exclude>io.netty:netty</exclude>
-
-                      <exclude>javax.activation:activation</exclude>
-                      <exclude>javax.servlet.jsp:jsp-api</exclude>
-                      <exclude>javax.servlet:servlet-api</exclude>
-                      <exclude>javax.xml.bind:jaxb-api</exclude>
-                      <exclude>javax.xml.stream:stax-api</exclude>
-		      <exclude>javax.servlet:javax.servlet-api</exclude>
-                      <exclude>javax.ws.rs:jsr311-api</exclude>
-
-                      <exclude>log4j:log4j</exclude>
-                      <exclude>net.java.dev.jets3t:jets3t</exclude>
-
-                      <exclude>org.apache.avro:avro</exclude>
-                      <exclude>org.apache.commons:commons-compress</exclude>
-                      <exclude>org.apache.commons:commons-math3</exclude>
-                      <exclude>org.apache.curator:curator-client</exclude>
-                      <exclude>org.apache.curator:curator-framework</exclude>
-                      <exclude>org.apache.curator:curator-recipes</exclude>
-                      <exclude>org.apache.directory.api:api-asn1-api</exclude>
-                      <exclude>org.apache.directory.api:api-util</exclude>
-                      <exclude>org.apache.directory.server:apacheds-i18n</exclude>
-                      <exclude>org.apache.directory.server:apacheds-kerberos-codec</exclude>
-                      <exclude>org.apache.hadoop:hadoop-annotations</exclude>
-                      <exclude>org.apache.hadoop:hadoop-auth</exclude>
-                      <exclude>org.apache.htrace:htrace-core</exclude>
-                      <exclude>org.apache.httpcomponents:httpclient</exclude>
-                      <exclude>org.apache.httpcomponents:httpcore</exclude>
-                      <exclude>org.apache.zookeeper:zookeeper</exclude>
-
-                      <exclude>org.codehaus.jackson:jackson-core-asl</exclude>
-                      <exclude>org.codehaus.jackson:jackson-jaxrs</exclude>
-                      <exclude>org.codehaus.jackson:jackson-mapper-asl</exclude>
-                      <exclude>org.codehaus.jackson:jackson-xc</exclude>
-                      <exclude>org.codehaus.jettison:jettison</exclude>
-
-                      <exclude>org.mortbay.jetty:jetty-util</exclude>
-                      <exclude>org.mortbay.jetty:jetty</exclude>
-                      <exclude>org.eclipse.jetty:jetty-server</exclude>
-                      <exclude>org.eclipse.jetty:jetty-util</exclude>
-                      <exclude>org.eclipse.jetty:jetty-servlet</exclude>
-                      <exclude>org.eclipse.jetty:jetty-webapp</exclude>
-                      <exclude>org.eclipse.jetty:jetty-util-ajax</exclude>
-		      <exclude>org.mortbay.jetty:jetty-sslengine</exclude>
-
-                      <exclude>org.slf4j:slf4j-api</exclude>
-                      <exclude>org.slf4j:slf4j-log4j12</exclude>
-                      <exclude>org.tukaani:xz</exclude>
-                      <exclude>org.xerial.snappy:snappy-java</exclude>
-                      <exclude>xmlenc:xmlenc</exclude>
-
-                      <exclude>com.nimbusds:nimbus-jose-jwt</exclude>
-                      <exclude>net.jcip:jcip-annotations</exclude>
-                      <exclude>net.minidev:json-smart</exclude>
-
-                      <exclude>org.apache.htrace:htrace-core4</exclude>
-                      <exclude>org.apache.kerby:kerb-simplekdc</exclude>
-                      <exclude>org.apache.kerby:kerby-config</exclude>
-                      <exclude>org.apache.kerby:kerb-core</exclude>
-                      <exclude>org.apache.kerby:kerby-asn1</exclude>
-                      <exclude>org.apache.kerby:kerby-pkix</exclude>
-                      <exclude>org.apache.kerby:kerby-util</exclude>
-                      <exclude>org.apache.kerby:kerb-client</exclude>
-                      <exclude>org.apache.kerby:kerb-common</exclude>
-                      <exclude>org.apache.kerby:kerb-util</exclude>
-                      <exclude>org.apache.kerby:kerb-crypto</exclude>
-                      <exclude>org.apache.kerby:kerb-server</exclude>
-                      <exclude>org.apache.kerby:kerb-identity</exclude>
-                      <exclude>org.apache.kerby:kerb-admin</exclude>
-                    </excludes>
-                  </artifactSet>
-                </configuration>
-              </execution>
-            </executions>
-          </plugin>
-          <!--Now unpack the shaded jar made above so the shaded classes
-             are available to subsequent modules-->
-          <plugin>
-            <groupId>org.apache.maven.plugins</groupId>
-            <artifactId>maven-dependency-plugin</artifactId>
-            <version>${maven-dependency-plugin.version}</version>
-            <executions>
-              <execution>
-                <id>unpack</id>
-                <phase>package</phase>
-                <goals>
-                  <goal>unpack</goal>
-                </goals>
-                <configuration>
-                  <artifactItems>
-                    <artifactItem>
-                      <groupId>${project.groupId}</groupId>
-                      <artifactId>${project.artifactId}</artifactId>
-                      <version>${project.version}</version>
-                      <classifier>sources</classifier>
-                      <type>jar</type>
-                      <overWrite>true</overWrite>
-                      <outputDirectory>${basedir}/src/main/java
-                      </outputDirectory>
-                      <includes>**/*.java</includes>
-                    </artifactItem>
-                  </artifactItems>
-                </configuration>
-              </execution>
-            </executions>
-          </plugin>
-        </plugins>
-      </build>
-    </profile>
-  </profiles>
-</project>

http://git-wip-us.apache.org/repos/asf/incubator-ratis/blob/7e71a2e0/raft-proto-shaded/src/main/proto/GRpc.proto
----------------------------------------------------------------------
diff --git a/raft-proto-shaded/src/main/proto/GRpc.proto b/raft-proto-shaded/src/main/proto/GRpc.proto
deleted file mode 100644
index bc625e6..0000000
--- a/raft-proto-shaded/src/main/proto/GRpc.proto
+++ /dev/null
@@ -1,45 +0,0 @@
-/**
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- *     http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-syntax = "proto3";
-option java_package = "org.apache.raft.shaded.proto.grpc";
-option java_outer_classname = "GRpcProtos";
-option java_generate_equals_and_hash = true;
-package raft.grpc;
-
-import "Raft.proto";
-
-service RaftClientProtocolService {
-  // A client-to-server RPC to set new raft configuration
-  rpc setConfiguration(raft.common.SetConfigurationRequestProto)
-      returns(raft.common.RaftClientReplyProto) {}
-
-  // A client-to-server stream RPC to append data
-  rpc append(stream raft.common.RaftClientRequestProto)
-      returns (stream raft.common.RaftClientReplyProto) {}
-}
-
-service RaftServerProtocolService {
-  rpc requestVote(raft.common.RequestVoteRequestProto)
-      returns(raft.common.RequestVoteReplyProto) {}
-
-  rpc appendEntries(stream raft.common.AppendEntriesRequestProto)
-      returns(stream raft.common.AppendEntriesReplyProto) {}
-
-  rpc installSnapshot(stream raft.common.InstallSnapshotRequestProto)
-      returns(raft.common.InstallSnapshotReplyProto) {}
-}

http://git-wip-us.apache.org/repos/asf/incubator-ratis/blob/7e71a2e0/raft-proto-shaded/src/main/proto/Hadoop.proto
----------------------------------------------------------------------
diff --git a/raft-proto-shaded/src/main/proto/Hadoop.proto b/raft-proto-shaded/src/main/proto/Hadoop.proto
deleted file mode 100644
index 3f8fc00..0000000
--- a/raft-proto-shaded/src/main/proto/Hadoop.proto
+++ /dev/null
@@ -1,44 +0,0 @@
-/**
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- *     http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-syntax = "proto3";
-option java_package = "org.apache.raft.shaded.proto.hadoop";
-option java_outer_classname = "HadoopProtos";
-option java_generic_services = true;
-option java_generate_equals_and_hash = true;
-package raft.hadoop;
-
-import "Raft.proto";
-
-service RaftClientProtocolService {
-  rpc submitClientRequest(raft.common.RaftClientRequestProto)
-      returns(raft.common.RaftClientReplyProto);
-
-  rpc setConfiguration(raft.common.SetConfigurationRequestProto)
-      returns(raft.common.RaftClientReplyProto);
-}
-
-service RaftServerProtocolService {
-  rpc requestVote(raft.common.RequestVoteRequestProto)
-      returns(raft.common.RequestVoteReplyProto);
-
-  rpc appendEntries(raft.common.AppendEntriesRequestProto)
-      returns(raft.common.AppendEntriesReplyProto);
-
-  rpc installSnapshot(raft.common.InstallSnapshotRequestProto)
-      returns(raft.common.InstallSnapshotReplyProto);
-}

http://git-wip-us.apache.org/repos/asf/incubator-ratis/blob/7e71a2e0/raft-proto-shaded/src/main/proto/Netty.proto
----------------------------------------------------------------------
diff --git a/raft-proto-shaded/src/main/proto/Netty.proto b/raft-proto-shaded/src/main/proto/Netty.proto
deleted file mode 100644
index 6fd29e9..0000000
--- a/raft-proto-shaded/src/main/proto/Netty.proto
+++ /dev/null
@@ -1,49 +0,0 @@
-/**
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- *     http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-syntax = "proto3";
-option java_package = "org.apache.raft.shaded.proto.netty";
-option java_outer_classname = "NettyProtos";
-option java_generate_equals_and_hash = true;
-package raft.netty;
-
-import "Raft.proto";
-
-message RaftNettyExceptionReplyProto {
-  raft.common.RaftRpcReplyProto rpcReply = 1;
-  bytes exception = 2;
-}
-
-message RaftNettyServerRequestProto {
-  oneof raftNettyServerRequest {
-    raft.common.RequestVoteRequestProto requestVoteRequest = 1;
-    raft.common.AppendEntriesRequestProto appendEntriesRequest = 2;
-    raft.common.InstallSnapshotRequestProto installSnapshotRequest = 3;
-    raft.common.RaftClientRequestProto raftClientRequest = 4;
-    raft.common.SetConfigurationRequestProto setConfigurationRequest = 5;
-  }
-}
-
-message RaftNettyServerReplyProto {
-  oneof raftNettyServerReply {
-    raft.common.RequestVoteReplyProto requestVoteReply = 1;
-    raft.common.AppendEntriesReplyProto appendEntriesReply = 2;
-    raft.common.InstallSnapshotReplyProto installSnapshotReply = 3;
-    raft.common.RaftClientReplyProto raftClientReply = 4;
-    RaftNettyExceptionReplyProto exceptionReply = 5;
-  }
-}

http://git-wip-us.apache.org/repos/asf/incubator-ratis/blob/7e71a2e0/raft-proto-shaded/src/main/proto/Raft.proto
----------------------------------------------------------------------
diff --git a/raft-proto-shaded/src/main/proto/Raft.proto b/raft-proto-shaded/src/main/proto/Raft.proto
deleted file mode 100644
index b8e82fb..0000000
--- a/raft-proto-shaded/src/main/proto/Raft.proto
+++ /dev/null
@@ -1,165 +0,0 @@
-/**
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- *     http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-syntax = "proto3";
-option java_package = "org.apache.raft.shaded.proto";
-option java_outer_classname = "RaftProtos";
-option java_generate_equals_and_hash = true;
-package raft.common;
-
-message RaftPeerProto {
-  string id = 1;      // id of the peer
-  string address = 2; // e.g. IP address, hostname etc.
-}
-
-message RaftConfigurationProto {
-  repeated RaftPeerProto peers = 1; // the peers in the current or new conf
-  repeated RaftPeerProto oldPeers = 2; // the peers in the old conf
-}
-
-message SMLogEntryProto {
-  // TODO: This is not super efficient if the SM itself uses PB to serialize its own data for a
-  // log entry. Data will be copied twice. We should directly support having any Message from SM
-  bytes data = 1;
-}
-
-message LeaderNoOp {
-  // empty
-}
-
-message LogEntryProto {
-  uint64 term = 1;
-  uint64 index = 2;
-
-  oneof LogEntryBody {
-    SMLogEntryProto smLogEntry = 3;
-    RaftConfigurationProto configurationEntry = 4;
-    LeaderNoOp noOp = 5;
-  }
-}
-
-message TermIndexProto {
-  uint64 term = 1;
-  uint64 index = 2;
-}
-
-message RaftRpcRequestProto {
-  string requestorId = 1;
-  string replyId = 2;
-  uint64 seqNum = 3;
-}
-
-message RaftRpcReplyProto {
-  string requestorId = 1;
-  string replyId = 2;
-  uint64 seqNum = 3;
-  bool success = 4;
-}
-
-message FileChunkProto {
-  string filename = 1; // relative to root
-  uint64 totalSize = 2;
-  bytes fileDigest = 3;
-  uint32 chunkIndex = 4;
-  uint64 offset = 5;
-  bytes data = 6;
-  bool done = 7;
-}
-
-enum InstallSnapshotResult {
-  SUCCESS = 0;
-  NOT_LEADER = 1;
-}
-
-message RequestVoteRequestProto {
-  RaftRpcRequestProto serverRequest = 1;
-  uint64 candidateTerm = 2;
-  TermIndexProto candidateLastEntry = 3;
-}
-
-message RequestVoteReplyProto {
-  RaftRpcReplyProto serverReply = 1;
-  uint64 term = 2;
-  bool shouldShutdown = 3;
-}
-
-message AppendEntriesRequestProto {
-  RaftRpcRequestProto serverRequest = 1;
-  uint64 leaderTerm = 2;
-  TermIndexProto previousLog = 3;
-  repeated LogEntryProto entries = 4;
-  uint64 leaderCommit = 5;
-  bool initializing = 6;
-}
-
-message AppendEntriesReplyProto {
-  enum AppendResult {
-    SUCCESS = 0;
-    NOT_LEADER = 1; // the requester's term is not large enough
-    INCONSISTENCY = 2; // gap between the local log and the entries
-  }
-
-  RaftRpcReplyProto serverReply = 1;
-  uint64 term = 2;
-  uint64 nextIndex = 3;
-  AppendResult result = 4;
-}
-
-message InstallSnapshotRequestProto {
-  RaftRpcRequestProto serverRequest = 1;
-  string requestId = 2; // an identifier for chunked-requests.
-  uint32 requestIndex = 3; // the index for this request chunk. Starts from 0.
-  RaftConfigurationProto raftConfiguration = 4;
-  uint64 leaderTerm = 5;
-  TermIndexProto termIndex = 6;
-  repeated FileChunkProto fileChunks = 7;
-  uint64 totalSize = 8;
-  bool done = 9; // whether this is the final chunk for the same req.
-}
-
-message InstallSnapshotReplyProto {
-  RaftRpcReplyProto serverReply = 1;
-  uint32 requestIndex = 2;
-  uint64 term = 3;
-  InstallSnapshotResult result = 4;
-}
-
-message ClientMessageEntryProto {
-  bytes content = 1;
-}
-
-// normal client request
-message RaftClientRequestProto {
-  RaftRpcRequestProto rpcRequest = 1;
-  ClientMessageEntryProto message = 2;
-  bool readOnly = 3;
-}
-
-message RaftClientReplyProto {
-  RaftRpcReplyProto rpcReply = 1;
-  ClientMessageEntryProto message = 2;
-  // the following 3 fields are used to indicate the server is not leader
-  bool isNotLeader = 3;
-  RaftPeerProto suggestedLeader = 4;
-  repeated RaftPeerProto peersInConf = 5;
-}
-
-// setConfiguration request
-message SetConfigurationRequestProto {
-  RaftRpcRequestProto rpcRequest = 1;
-  repeated RaftPeerProto peers = 2;
-}

http://git-wip-us.apache.org/repos/asf/incubator-ratis/blob/7e71a2e0/raft-proto-shaded/src/main/resources/META-INF/services/org.apache.raft.shaded.io.grpc.ManagedChannelProvider
----------------------------------------------------------------------
diff --git a/raft-proto-shaded/src/main/resources/META-INF/services/org.apache.raft.shaded.io.grpc.ManagedChannelProvider b/raft-proto-shaded/src/main/resources/META-INF/services/org.apache.raft.shaded.io.grpc.ManagedChannelProvider
deleted file mode 100644
index 1b680a2..0000000
--- a/raft-proto-shaded/src/main/resources/META-INF/services/org.apache.raft.shaded.io.grpc.ManagedChannelProvider
+++ /dev/null
@@ -1,16 +0,0 @@
-# Licensed to the Apache Software Foundation (ASF) under one or more
-# contributor license agreements.  See the NOTICE file distributed with
-# this work for additional information regarding copyright ownership.
-# The ASF licenses this file to You under the Apache License, Version 2.0
-# (the "License"); you may not use this file except in compliance with
-# the License.  You may obtain a copy of the License at
-#
-#     http://www.apache.org/licenses/LICENSE-2.0
-#
-# Unless required by applicable law or agreed to in writing, software
-# distributed under the License is distributed on an "AS IS" BASIS,
-# WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
-# See the License for the specific language governing permissions and
-# limitations under the License.
-
-org.apache.raft.shaded.io.grpc.netty.NettyChannelProvider
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/incubator-ratis/blob/7e71a2e0/raft-proto-shaded/src/main/resources/META-INF/services/org.apache.raft.shaded.io.grpc.NameResolverProvider
----------------------------------------------------------------------
diff --git a/raft-proto-shaded/src/main/resources/META-INF/services/org.apache.raft.shaded.io.grpc.NameResolverProvider b/raft-proto-shaded/src/main/resources/META-INF/services/org.apache.raft.shaded.io.grpc.NameResolverProvider
deleted file mode 100644
index 2beedba..0000000
--- a/raft-proto-shaded/src/main/resources/META-INF/services/org.apache.raft.shaded.io.grpc.NameResolverProvider
+++ /dev/null
@@ -1,16 +0,0 @@
-# Licensed to the Apache Software Foundation (ASF) under one or more
-# contributor license agreements.  See the NOTICE file distributed with
-# this work for additional information regarding copyright ownership.
-# The ASF licenses this file to You under the Apache License, Version 2.0
-# (the "License"); you may not use this file except in compliance with
-# the License.  You may obtain a copy of the License at
-#
-#     http://www.apache.org/licenses/LICENSE-2.0
-#
-# Unless required by applicable law or agreed to in writing, software
-# distributed under the License is distributed on an "AS IS" BASIS,
-# WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
-# See the License for the specific language governing permissions and
-# limitations under the License.
-
-org.apache.raft.shaded.io.grpc.internal.DnsNameResolverProvider

http://git-wip-us.apache.org/repos/asf/incubator-ratis/blob/7e71a2e0/raft-proto-shaded/src/main/resources/META-INF/services/org.apache.raft.shaded.io.grpc.ServerProvider
----------------------------------------------------------------------
diff --git a/raft-proto-shaded/src/main/resources/META-INF/services/org.apache.raft.shaded.io.grpc.ServerProvider b/raft-proto-shaded/src/main/resources/META-INF/services/org.apache.raft.shaded.io.grpc.ServerProvider
deleted file mode 100644
index 2180a7f..0000000
--- a/raft-proto-shaded/src/main/resources/META-INF/services/org.apache.raft.shaded.io.grpc.ServerProvider
+++ /dev/null
@@ -1,16 +0,0 @@
-# Licensed to the Apache Software Foundation (ASF) under one or more
-# contributor license agreements.  See the NOTICE file distributed with
-# this work for additional information regarding copyright ownership.
-# The ASF licenses this file to You under the Apache License, Version 2.0
-# (the "License"); you may not use this file except in compliance with
-# the License.  You may obtain a copy of the License at
-#
-#     http://www.apache.org/licenses/LICENSE-2.0
-#
-# Unless required by applicable law or agreed to in writing, software
-# distributed under the License is distributed on an "AS IS" BASIS,
-# WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
-# See the License for the specific language governing permissions and
-# limitations under the License.
-
-org.apache.raft.shaded.io.grpc.netty.NettyServerProvider

http://git-wip-us.apache.org/repos/asf/incubator-ratis/blob/7e71a2e0/raft-server/pom.xml
----------------------------------------------------------------------
diff --git a/raft-server/pom.xml b/raft-server/pom.xml
deleted file mode 100644
index d992306..0000000
--- a/raft-server/pom.xml
+++ /dev/null
@@ -1,80 +0,0 @@
-<?xml version="1.0" encoding="UTF-8"?>
-<!--
-  Licensed 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. See accompanying LICENSE file.
--->
-<project xmlns="http://maven.apache.org/POM/4.0.0"
-         xmlns:xsi="http://www.w3.org/2001/XMLSchema-instance"
-         xsi:schemaLocation="http://maven.apache.org/POM/4.0.0 http://maven.apache.org/xsd/maven-4.0.0.xsd">
-  <modelVersion>4.0.0</modelVersion>
-  <parent>
-    <artifactId>raft-project-dist</artifactId>
-    <groupId>com.hortonworks.raft</groupId>
-    <version>1.0-SNAPSHOT</version>
-    <relativePath>../raft-project-dist</relativePath>
-  </parent>
-
-  <artifactId>raft-server</artifactId>
-  <name>Raft Server</name>
-
-  <dependencies>
-    <dependency>
-      <artifactId>raft-proto-shaded</artifactId>
-      <groupId>com.hortonworks.raft</groupId>
-    </dependency>
-
-    <dependency>
-      <artifactId>raft-common</artifactId>
-      <groupId>com.hortonworks.raft</groupId>
-      <scope>provided</scope>
-    </dependency>
-    <dependency>
-      <artifactId>raft-common</artifactId>
-      <groupId>com.hortonworks.raft</groupId>
-      <scope>test</scope>
-      <type>test-jar</type>
-    </dependency>
-    
-    <dependency>
-      <artifactId>raft-client</artifactId>
-      <groupId>com.hortonworks.raft</groupId>
-      <scope>provided</scope>
-    </dependency>
-    <dependency>
-      <artifactId>raft-client</artifactId>
-      <groupId>com.hortonworks.raft</groupId>
-      <scope>test</scope>
-      <type>test-jar</type>
-    </dependency>
-    
-    <dependency>
-      <groupId>org.slf4j</groupId>
-      <artifactId>slf4j-api</artifactId>
-    </dependency>
-
-    <dependency>
-      <groupId>com.google.guava</groupId>
-      <artifactId>guava</artifactId>
-    </dependency>
-
-    <dependency>
-      <groupId>junit</groupId>
-      <artifactId>junit</artifactId>
-      <scope>test</scope>
-    </dependency>
-    <dependency>
-      <groupId>org.mockito</groupId>
-      <artifactId>mockito-all</artifactId>
-      <scope>test</scope>
-    </dependency>
-  </dependencies>
-</project>

http://git-wip-us.apache.org/repos/asf/incubator-ratis/blob/7e71a2e0/raft-server/src/main/java/org/apache/raft/server/RaftServer.java
----------------------------------------------------------------------
diff --git a/raft-server/src/main/java/org/apache/raft/server/RaftServer.java b/raft-server/src/main/java/org/apache/raft/server/RaftServer.java
deleted file mode 100644
index bb49c5a..0000000
--- a/raft-server/src/main/java/org/apache/raft/server/RaftServer.java
+++ /dev/null
@@ -1,44 +0,0 @@
-/**
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- *     http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-package org.apache.raft.server;
-
-import org.apache.raft.protocol.RaftClientAsynchronousProtocol;
-import org.apache.raft.protocol.RaftClientProtocol;
-import org.apache.raft.server.protocol.RaftServerProtocol;
-import org.apache.raft.statemachine.StateMachine;
-
-import java.io.Closeable;
-
-/** Raft server interface */
-public interface RaftServer extends Closeable, RaftServerProtocol,
-    RaftClientProtocol, RaftClientAsynchronousProtocol {
-  /** @return the server ID. */
-  String getId();
-
-  /** Set server RPC service. */
-  void setServerRpc(RaftServerRpc serverRpc);
-
-  /** Start this server. */
-  void start();
-
-  /**
-   * Returns the StateMachine instance.
-   * @return the StateMachine instance.
-   */
-  StateMachine getStateMachine();
-}

http://git-wip-us.apache.org/repos/asf/incubator-ratis/blob/7e71a2e0/raft-server/src/main/java/org/apache/raft/server/RaftServerConfigKeys.java
----------------------------------------------------------------------
diff --git a/raft-server/src/main/java/org/apache/raft/server/RaftServerConfigKeys.java b/raft-server/src/main/java/org/apache/raft/server/RaftServerConfigKeys.java
deleted file mode 100644
index 2ce0326..0000000
--- a/raft-server/src/main/java/org/apache/raft/server/RaftServerConfigKeys.java
+++ /dev/null
@@ -1,150 +0,0 @@
-/**
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- *     http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-package org.apache.raft.server;
-
-import org.apache.raft.server.impl.LogAppenderFactory;
-import org.apache.raft.util.NetUtils;
-import org.slf4j.Logger;
-import org.slf4j.LoggerFactory;
-
-import java.net.InetSocketAddress;
-
-public interface RaftServerConfigKeys {
-
-  String PREFIX = "raft.server";
-
-  /** IPC server configurations */
-  interface Ipc {
-    String PREFIX  = RaftServerConfigKeys.PREFIX + ".ipc";
-
-    String ADDRESS_KEY = PREFIX + ".address";
-    int    DEFAULT_PORT = 10718;
-    String ADDRESS_DEFAULT = "0.0.0.0:" + DEFAULT_PORT;
-
-    String HANDLERS_KEY = PREFIX + ".handlers";
-    int    HANDLERS_DEFAULT = 10;
-
-    class Getters {
-      private final Get get;
-
-      Getters(Get get) {
-        this.get = get;
-      }
-
-      public int handlers() {
-        return get.getInt(HANDLERS_KEY, HANDLERS_DEFAULT, 1, null);
-      }
-
-      public InetSocketAddress address() {
-        return get.getInetSocketAddress(ADDRESS_KEY, ADDRESS_DEFAULT);
-      }
-    }
-  }
-
-  String RAFT_SERVER_USE_MEMORY_LOG_KEY = "raft.server.use.memory.log";
-  boolean RAFT_SERVER_USE_MEMORY_LOG_DEFAULT = false;
-
-  String RAFT_SERVER_STORAGE_DIR_KEY = "raft.server.storage.dir";
-  String RAFT_SERVER_STORAGE_DIR_DEFAULT = "file:///tmp/raft-server/";
-
-  String RAFT_SERVER_LOG_APPENDER_FACTORY_CLASS_KEY = "raft.server.log.appender.factory.class";
-  Class<? extends LogAppenderFactory> RAFT_SERVER_LOG_APPENDER_FACTORY_CLASS_DEFAULT
-      = LogAppenderFactory.SynchronousLogAppenderFactory.class;
-
-  /** whether trigger snapshot when log size exceeds limit */
-  String RAFT_SERVER_AUTO_SNAPSHOT_ENABLED_KEY = "raft.server.auto.snapshot.enabled";
-  /** by default let the state machine to decide when to do checkpoint */
-  boolean RAFT_SERVER_AUTO_SNAPSHOT_ENABLED_DEFAULT = false;
-
-  /** log size limit (in number of log entries) that triggers the snapshot */
-  String RAFT_SERVER_SNAPSHOT_TRIGGER_THRESHOLD_KEY = "raft.server.snapshot.trigger.threshold";
-  long RAFT_SERVER_SNAPSHOT_TRIGGER_THRESHOLD_DEFAULT = 400000;
-
-  String RAFT_LOG_SEGMENT_MAX_SIZE_KEY = "raft.log.segment.max.size";
-  long RAFT_LOG_SEGMENT_MAX_SIZE_DEFAULT = 1024L * 1024 * 1024 * 2; // 2GB
-
-  String RAFT_LOG_SEGMENT_PREALLOCATED_SIZE_KEY = "raft.log.segment.preallocated.size";
-  int RAFT_LOG_SEGMENT_PREALLOCATED_SIZE_DEFAULT = 1024 * 1024 * 16; // 16MB
-
-  String RAFT_LOG_WRITE_BUFFER_SIZE_KEY = "raft.log.write.buffer.size";
-  int RAFT_LOG_WRITE_BUFFER_SIZE_DEFAULT = 64 * 1024;
-
-  String RAFT_SNAPSHOT_CHUNK_MAX_SIZE_KEY = "raft.snapshot.chunk.max.size";
-  int RAFT_SNAPSHOT_CHUNK_MAX_SIZE_DEFAULT = 1024 * 1024 * 16;
-
-  String RAFT_LOG_FORCE_SYNC_NUM_KEY = "raft.log.force.sync.num";
-  int RAFT_LOG_FORCE_SYNC_NUM_DEFAULT = 128;
-
-  /** server rpc timeout related */
-  String RAFT_SERVER_RPC_TIMEOUT_MIN_MS_KEY = "raft.server.rpc.timeout.min.ms";
-  int RAFT_SERVER_RPC_TIMEOUT_MIN_MS_DEFAULT = 150;
-
-  String RAFT_SERVER_RPC_TIMEOUT_MAX_MS_KEY = "raft.server.rpc.timeout.max.ms";
-  int RAFT_SERVER_RPC_TIMEOUT_MAX_MS_DEFAULT = 300;
-
-  String RAFT_SERVER_RPC_SLEEP_TIME_MS_KEY = "raft.server.rpc.sleep.time.ms";
-  int RAFT_SERVER_RPC_SLEEP_TIME_MS_DEFAULT = 25;
-
-  /**
-   * When bootstrapping a new peer, If the gap between the match index of the
-   * peer and the leader's latest committed index is less than this gap, we
-   * treat the peer as caught-up.
-   */
-  String RAFT_SERVER_STAGING_CATCHUP_GAP_KEY = "raft.server.staging.catchup.gap";
-  int RAFT_SERVER_STAGING_CATCHUP_GAP_DEFAULT = 1000; // increase this number when write throughput is high
-
-  String RAFT_SERVER_LOG_APPENDER_BUFFER_CAPACITY_KEY = "raft.server.log.appender.buffer.capacity";
-  int RAFT_SERVER_LOG_APPENDER_BUFFER_CAPACITY_DEFAULT = 4 * 1024 * 1024; // 4MB
-
-  String RAFT_SERVER_LOG_APPENDER_BATCH_ENABLED_KEY = "raft.server.log.appender.batch.enabled";
-  boolean RAFT_SERVER_LOG_APPENDER_BATCH_ENABLED_DEFAULT = false;
-
-  /** An utility class to get conf values. */
-  abstract class Get {
-    static Logger LOG = LoggerFactory.getLogger(RaftServerConfigKeys.class);
-
-    private final Ipc.Getters ipc = new Ipc.Getters(this);
-
-    protected abstract int getInt(String key, int defaultValue);
-
-    int getInt(String key, int defaultValue, Integer min, Integer max) {
-      final int value = getInt(key, defaultValue);
-      final String s = key + " = " + value;
-      if (min != null && value < min) {
-        throw new IllegalArgumentException(s + " < min = " + min);
-      }
-      if (max != null && value > max) {
-        throw new IllegalArgumentException(s + " > max = " + max);
-      }
-      LOG.info(s);
-      return value;
-    }
-
-    protected abstract String getTrimmed(String key, String defaultValue);
-
-    InetSocketAddress getInetSocketAddress(String key, String defaultValue) {
-      final String address = getTrimmed(key, defaultValue);
-      LOG.info(key + " = " + address);
-      return NetUtils.createSocketAddr(address);
-    }
-
-    public Ipc.Getters ipc() {
-      return ipc;
-    }
-  }
-}

http://git-wip-us.apache.org/repos/asf/incubator-ratis/blob/7e71a2e0/raft-server/src/main/java/org/apache/raft/server/RaftServerRpc.java
----------------------------------------------------------------------
diff --git a/raft-server/src/main/java/org/apache/raft/server/RaftServerRpc.java b/raft-server/src/main/java/org/apache/raft/server/RaftServerRpc.java
deleted file mode 100644
index 6526bea..0000000
--- a/raft-server/src/main/java/org/apache/raft/server/RaftServerRpc.java
+++ /dev/null
@@ -1,39 +0,0 @@
-/**
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- *     http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-package org.apache.raft.server;
-
-import org.apache.raft.protocol.RaftPeer;
-import org.apache.raft.server.protocol.RaftServerProtocol;
-
-import java.io.Closeable;
-import java.net.InetSocketAddress;
-
-/**
- * An server-side interface for supporting different RPC implementations
- * such as Netty, gRPC and Hadoop.
- */
-public interface RaftServerRpc extends RaftServerProtocol, Closeable {
-  /** Start the RPC service. */
-  void start();
-
-  /** @return the address where this RPC server is listening to. */
-  InetSocketAddress getInetSocketAddress();
-
-  /** add information of the given peers */
-  void addPeers(Iterable<RaftPeer> peers);
-}

http://git-wip-us.apache.org/repos/asf/incubator-ratis/blob/7e71a2e0/raft-server/src/main/java/org/apache/raft/server/impl/ConfigurationManager.java
----------------------------------------------------------------------
diff --git a/raft-server/src/main/java/org/apache/raft/server/impl/ConfigurationManager.java b/raft-server/src/main/java/org/apache/raft/server/impl/ConfigurationManager.java
deleted file mode 100644
index b2f077d..0000000
--- a/raft-server/src/main/java/org/apache/raft/server/impl/ConfigurationManager.java
+++ /dev/null
@@ -1,91 +0,0 @@
-/**
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- *     http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-package org.apache.raft.server.impl;
-
-import com.google.common.annotations.VisibleForTesting;
-import com.google.common.base.Preconditions;
-
-import java.util.*;
-
-/**
- * Maintain the mappings between log index and corresponding raft configuration.
- * Initialized when starting the raft peer. The mappings are loaded from the
- * raft log, and updated while appending/truncating configuration related log
- * entries.
- */
-public class ConfigurationManager {
-  private RaftConfiguration initialConf;
-  private final NavigableMap<Long, RaftConfiguration> configurations =
-      new TreeMap<>();
-  /**
-   * The current raft configuration. If configurations is not empty, should be
-   * the last entry of the map. Otherwise is initialConf.
-   */
-  private RaftConfiguration currentConf;
-
-  public ConfigurationManager(RaftConfiguration initialConf) {
-    setInitialConf(initialConf);
-  }
-
-  @VisibleForTesting
-  public synchronized void setInitialConf(RaftConfiguration initialConf) {
-    /**
-     * initialConf should actually be defined as "final". But for tests we want
-     * to change the initial configuration to reflect the correct port binding.
-     */
-    this.initialConf = initialConf;
-    this.currentConf = initialConf;
-  }
-
-  public synchronized void addConfiguration(long logIndex,
-      RaftConfiguration conf) {
-    Preconditions.checkArgument(configurations.isEmpty() ||
-        configurations.lastEntry().getKey() < logIndex);
-    configurations.put(logIndex, conf);
-    this.currentConf = conf;
-  }
-
-  synchronized RaftConfiguration getCurrent() {
-    return currentConf;
-  }
-
-  /**
-   * Remove all the configurations whose log index is >= the given index.
-   * @param index The given index. All the configurations whose log index is >=
-   *              this value will be removed.
-   * @return The configuration with largest log index < the given index.
-   */
-  synchronized RaftConfiguration removeConfigurations(long index) {
-    SortedMap<Long, RaftConfiguration> toRemove = configurations.tailMap(index);
-    for (Iterator<Map.Entry<Long, RaftConfiguration>> iter =
-         toRemove.entrySet().iterator(); iter.hasNext();) {
-      iter.next();
-      iter.remove();
-    }
-    currentConf = configurations.isEmpty() ? initialConf :
-        configurations.lastEntry().getValue();
-    return currentConf;
-  }
-
-  @VisibleForTesting
-  synchronized int numOfConf() {
-    return 1 + configurations.size();
-  }
-
-  // TODO: remove Configuration entries after they are committed
-}

http://git-wip-us.apache.org/repos/asf/incubator-ratis/blob/7e71a2e0/raft-server/src/main/java/org/apache/raft/server/impl/FollowerInfo.java
----------------------------------------------------------------------
diff --git a/raft-server/src/main/java/org/apache/raft/server/impl/FollowerInfo.java b/raft-server/src/main/java/org/apache/raft/server/impl/FollowerInfo.java
deleted file mode 100644
index 683599e..0000000
--- a/raft-server/src/main/java/org/apache/raft/server/impl/FollowerInfo.java
+++ /dev/null
@@ -1,103 +0,0 @@
-/**
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- *     http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-package org.apache.raft.server.impl;
-
-import org.apache.raft.protocol.RaftPeer;
-import org.apache.raft.util.Timestamp;
-
-import java.util.concurrent.atomic.AtomicLong;
-import java.util.concurrent.atomic.AtomicReference;
-
-public class FollowerInfo {
-  private final RaftPeer peer;
-  private final AtomicReference<Timestamp> lastRpcResponseTime;
-  private final AtomicReference<Timestamp> lastRpcSendTime;
-  private long nextIndex;
-  private final AtomicLong matchIndex;
-  private volatile boolean attendVote;
-
-  FollowerInfo(RaftPeer peer, Timestamp lastRpcTime, long nextIndex,
-      boolean attendVote) {
-    this.peer = peer;
-    this.lastRpcResponseTime = new AtomicReference<>(lastRpcTime);
-    this.lastRpcSendTime = new AtomicReference<>(lastRpcTime);
-    this.nextIndex = nextIndex;
-    this.matchIndex = new AtomicLong(0);
-    this.attendVote = attendVote;
-  }
-
-  public void updateMatchIndex(final long matchIndex) {
-    this.matchIndex.set(matchIndex);
-  }
-
-  long getMatchIndex() {
-    return matchIndex.get();
-  }
-
-  public synchronized long getNextIndex() {
-    return nextIndex;
-  }
-
-  public synchronized void updateNextIndex(long i) {
-    nextIndex = i;
-  }
-
-  public synchronized void decreaseNextIndex(long targetIndex) {
-    if (nextIndex > 0) {
-      nextIndex = Math.min(nextIndex - 1, targetIndex);
-    }
-  }
-
-  @Override
-  public String toString() {
-    return peer.getId() + "(next=" + nextIndex + ", match=" + matchIndex + "," +
-        " attendVote=" + attendVote +
-        ", lastRpcSendTime=" + lastRpcSendTime +
-        ", lastRpcResponseTime=" + lastRpcResponseTime + ")";
-  }
-
-  void startAttendVote() {
-    attendVote = true;
-  }
-
-  public boolean isAttendingVote() {
-    return attendVote;
-  }
-
-  public RaftPeer getPeer() {
-    return peer;
-  }
-
-  /** Update lastRpcResponseTime to the current time. */
-  public void updateLastRpcResponseTime() {
-    lastRpcResponseTime.set(new Timestamp());
-  }
-
-  public Timestamp getLastRpcResponseTime() {
-    return lastRpcResponseTime.get();
-  }
-
-  /** Update lastRpcSendTime to the current time. */
-  public void updateLastRpcSendTime() {
-    lastRpcSendTime.set(new Timestamp());
-  }
-
-  public Timestamp getLastRpcTime() {
-    return Timestamp.latest(lastRpcResponseTime.get(), lastRpcSendTime.get());
-  }
-}

http://git-wip-us.apache.org/repos/asf/incubator-ratis/blob/7e71a2e0/raft-server/src/main/java/org/apache/raft/server/impl/FollowerState.java
----------------------------------------------------------------------
diff --git a/raft-server/src/main/java/org/apache/raft/server/impl/FollowerState.java b/raft-server/src/main/java/org/apache/raft/server/impl/FollowerState.java
deleted file mode 100644
index fbef188..0000000
--- a/raft-server/src/main/java/org/apache/raft/server/impl/FollowerState.java
+++ /dev/null
@@ -1,91 +0,0 @@
-/**
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- *     http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-package org.apache.raft.server.impl;
-
-import org.apache.raft.util.Daemon;
-import org.apache.raft.util.Timestamp;
-import org.slf4j.Logger;
-
-/**
- * Used when the peer is a follower. Used to track the election timeout.
- */
-class FollowerState extends Daemon {
-  static final Logger LOG = RaftServerImpl.LOG;
-
-  private final RaftServerImpl server;
-
-  private volatile Timestamp lastRpcTime = new Timestamp();
-  private volatile boolean monitorRunning = true;
-  private volatile boolean inLogSync = false;
-
-  FollowerState(RaftServerImpl server) {
-    this.server = server;
-  }
-
-  void updateLastRpcTime(boolean inLogSync) {
-    lastRpcTime = new Timestamp();
-    LOG.trace("{} update last rpc time to {}", server.getId(), lastRpcTime);
-    this.inLogSync = inLogSync;
-  }
-
-  Timestamp getLastRpcTime() {
-    return lastRpcTime;
-  }
-
-  boolean shouldWithholdVotes() {
-    return lastRpcTime.elapsedTimeMs() < server.getMinTimeoutMs();
-  }
-
-  void stopRunning() {
-    this.monitorRunning = false;
-  }
-
-  @Override
-  public  void run() {
-    while (monitorRunning && server.isFollower()) {
-      final long electionTimeout = server.getRandomTimeoutMs();
-      try {
-        Thread.sleep(electionTimeout);
-        if (!monitorRunning || !server.isFollower()) {
-          LOG.info("{} heartbeat monitor quit", server.getId());
-          break;
-        }
-        synchronized (server) {
-          if (!inLogSync && lastRpcTime.elapsedTimeMs() >= electionTimeout) {
-            LOG.info("{} changes to CANDIDATE, lastRpcTime:{}, electionTimeout:{}ms",
-                server.getId(), lastRpcTime, electionTimeout);
-            // election timeout, should become a candidate
-            server.changeToCandidate();
-            break;
-          }
-        }
-      } catch (InterruptedException e) {
-        LOG.info(this + " was interrupted: " + e);
-        LOG.trace("TRACE", e);
-        return;
-      } catch (Exception e) {
-        LOG.warn(this + " caught an exception", e);
-      }
-    }
-  }
-
-  @Override
-  public String toString() {
-    return server.getId() + ": " + getClass().getSimpleName();
-  }
-}

http://git-wip-us.apache.org/repos/asf/incubator-ratis/blob/7e71a2e0/raft-server/src/main/java/org/apache/raft/server/impl/LeaderElection.java
----------------------------------------------------------------------
diff --git a/raft-server/src/main/java/org/apache/raft/server/impl/LeaderElection.java b/raft-server/src/main/java/org/apache/raft/server/impl/LeaderElection.java
deleted file mode 100644
index a326eb5..0000000
--- a/raft-server/src/main/java/org/apache/raft/server/impl/LeaderElection.java
+++ /dev/null
@@ -1,241 +0,0 @@
-/**
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- *     http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-package org.apache.raft.server.impl;
-
-import com.google.common.base.Preconditions;
-import com.google.common.util.concurrent.ThreadFactoryBuilder;
-import org.apache.raft.protocol.RaftPeer;
-import org.apache.raft.server.protocol.TermIndex;
-import org.apache.raft.shaded.proto.RaftProtos.RequestVoteReplyProto;
-import org.apache.raft.shaded.proto.RaftProtos.RequestVoteRequestProto;
-import org.apache.raft.statemachine.SnapshotInfo;
-import org.apache.raft.util.Daemon;
-import org.apache.raft.util.ProtoUtils;
-import org.apache.raft.util.Timestamp;
-import org.slf4j.Logger;
-import org.slf4j.LoggerFactory;
-
-import java.io.IOException;
-import java.util.ArrayList;
-import java.util.Collection;
-import java.util.List;
-import java.util.concurrent.*;
-import java.util.stream.Collectors;
-
-class LeaderElection extends Daemon {
-  public static final Logger LOG = LoggerFactory.getLogger(LeaderElection.class);
-
-  private ResultAndTerm logAndReturn(Result result,
-      List<RequestVoteReplyProto> responses,
-      List<Exception> exceptions, long newTerm) {
-    LOG.info(server.getId() + ": Election " + result + "; received "
-        + responses.size() + " response(s) "
-        + responses.stream().map(r -> ProtoUtils.toString(r)).collect(Collectors.toList())
-        + " and " + exceptions.size() + " exception(s); " + server.getState());
-    int i = 0;
-    for(Exception e : exceptions) {
-      LOG.info("  " + i++ + ": " + e);
-      LOG.trace("TRACE", e);
-    }
-    return new ResultAndTerm(result, newTerm);
-  }
-
-  enum Result {PASSED, REJECTED, TIMEOUT, DISCOVERED_A_NEW_TERM, SHUTDOWN}
-
-  private static class ResultAndTerm {
-    final Result result;
-    final long term;
-
-    ResultAndTerm(Result result, long term) {
-      this.result = result;
-      this.term = term;
-    }
-  }
-
-  private final RaftServerImpl server;
-  private ExecutorCompletionService<RequestVoteReplyProto> service;
-  private ExecutorService executor;
-  private volatile boolean running;
-  /**
-   * The Raft configuration should not change while the peer is in candidate
-   * state. If the configuration changes, another peer should be acting as a
-   * leader and this LeaderElection session should end.
-   */
-  private final RaftConfiguration conf;
-  private final Collection<RaftPeer> others;
-
-  LeaderElection(RaftServerImpl server) {
-    this.server = server;
-    conf = server.getRaftConf();
-    others = conf.getOtherPeers(server.getId());
-    this.running = true;
-  }
-
-  void stopRunning() {
-    this.running = false;
-  }
-
-  private void initExecutor() {
-    Preconditions.checkState(!others.isEmpty());
-    executor = Executors.newFixedThreadPool(others.size(),
-        new ThreadFactoryBuilder().setDaemon(true).build());
-    service = new ExecutorCompletionService<>(executor);
-  }
-
-  @Override
-  public void run() {
-    try {
-      askForVotes();
-    } catch (InterruptedException e) {
-      // the leader election thread is interrupted. The peer may already step
-      // down to a follower. The leader election should skip.
-      LOG.info(server.getId() + " " + getClass().getSimpleName()
-          + " thread is interrupted gracefully; server=" + server);
-    } catch (IOException e) {
-      LOG.warn("Failed to persist votedFor/term. Exit the leader election.", e);
-      stopRunning();
-    }
-  }
-
-  /**
-   * After a peer changes its role to candidate, it invokes this method to
-   * send out requestVote rpc to all other peers.
-   */
-  private void askForVotes() throws InterruptedException, IOException {
-    final ServerState state = server.getState();
-    while (running && server.isCandidate()) {
-      // one round of requestVotes
-      final long electionTerm;
-      synchronized (server) {
-        electionTerm = state.initElection();
-        server.getState().persistMetadata();
-      }
-      LOG.info(state.getSelfId() + ": begin an election in Term "
-          + electionTerm);
-
-      TermIndex lastEntry = ServerProtoUtils.toTermIndex(
-          state.getLog().getLastEntry());
-      if (lastEntry == null) {
-        // lastEntry may need to be derived from snapshot
-        SnapshotInfo snapshot = state.getLatestSnapshot();
-        if (snapshot != null) {
-          lastEntry = snapshot.getTermIndex();
-        }
-      }
-
-      final ResultAndTerm r;
-      if (others.isEmpty()) {
-        r = new ResultAndTerm(Result.PASSED, electionTerm);
-      } else {
-        try {
-          initExecutor();
-          int submitted = submitRequests(electionTerm, lastEntry);
-          r = waitForResults(electionTerm, submitted);
-        } finally {
-          if (executor != null) {
-            executor.shutdown();
-          }
-        }
-      }
-
-      synchronized (server) {
-        if (electionTerm != state.getCurrentTerm() || !running ||
-            !server.isCandidate()) {
-          return; // term already passed or no longer a candidate.
-        }
-
-        switch (r.result) {
-          case PASSED:
-            server.changeToLeader();
-            return;
-          case SHUTDOWN:
-            LOG.info("{} received shutdown response when requesting votes.",
-                server.getId());
-            server.close();
-            return;
-          case REJECTED:
-          case DISCOVERED_A_NEW_TERM:
-            final long term = r.term > server.getState().getCurrentTerm() ?
-                r.term : server.getState().getCurrentTerm();
-            server.changeToFollower(term, true);
-            return;
-          case TIMEOUT:
-            // should start another election
-        }
-      }
-    }
-  }
-
-  private int submitRequests(final long electionTerm, final TermIndex lastEntry) {
-    int submitted = 0;
-    for (final RaftPeer peer : others) {
-      final RequestVoteRequestProto r = server.createRequestVoteRequest(
-          peer.getId(), electionTerm, lastEntry);
-      service.submit(
-          () -> server.getServerRpc().requestVote(r));
-      submitted++;
-    }
-    return submitted;
-  }
-
-  private ResultAndTerm waitForResults(final long electionTerm,
-      final int submitted) throws InterruptedException {
-    final Timestamp timeout = new Timestamp().addTimeMs(server.getRandomTimeoutMs());
-    final List<RequestVoteReplyProto> responses = new ArrayList<>();
-    final List<Exception> exceptions = new ArrayList<>();
-    int waitForNum = submitted;
-    Collection<String> votedPeers = new ArrayList<>();
-    while (waitForNum > 0 && running && server.isCandidate()) {
-      final long waitTime = -timeout.elapsedTimeMs();
-      if (waitTime <= 0) {
-        return logAndReturn(Result.TIMEOUT, responses, exceptions, -1);
-      }
-
-      try {
-        final Future<RequestVoteReplyProto> future = service.poll(
-            waitTime, TimeUnit.MILLISECONDS);
-        if (future == null) {
-          continue; // poll timeout, continue to return Result.TIMEOUT
-        }
-
-        final RequestVoteReplyProto r = future.get();
-        responses.add(r);
-        if (r.getShouldShutdown()) {
-          return logAndReturn(Result.SHUTDOWN, responses, exceptions, -1);
-        }
-        if (r.getTerm() > electionTerm) {
-          return logAndReturn(Result.DISCOVERED_A_NEW_TERM, responses,
-              exceptions, r.getTerm());
-        }
-        if (r.getServerReply().getSuccess()) {
-          votedPeers.add(r.getServerReply().getReplyId());
-          if (conf.hasMajority(votedPeers, server.getId())) {
-            return logAndReturn(Result.PASSED, responses, exceptions, -1);
-          }
-        }
-      } catch(ExecutionException e) {
-        LOG.info("Got exception when requesting votes: " + e);
-        LOG.trace("TRACE", e);
-        exceptions.add(e);
-      }
-      waitForNum--;
-    }
-    // received all the responses
-    return logAndReturn(Result.REJECTED, responses, exceptions, -1);
-  }
-}


[53/54] [abbrv] incubator-ratis git commit: Add builder classes for RaftServerRpc and its implementations.

Posted by ji...@apache.org.
Add builder classes for RaftServerRpc and its implementations.


Project: http://git-wip-us.apache.org/repos/asf/incubator-ratis/repo
Commit: http://git-wip-us.apache.org/repos/asf/incubator-ratis/commit/d16c5c64
Tree: http://git-wip-us.apache.org/repos/asf/incubator-ratis/tree/d16c5c64
Diff: http://git-wip-us.apache.org/repos/asf/incubator-ratis/diff/d16c5c64

Branch: refs/heads/master
Commit: d16c5c649311c8af3340c3373593821ba67d467f
Parents: 9c44237
Author: Tsz-Wo Nicholas Sze <sz...@hortonworks.com>
Authored: Tue Jan 31 23:54:26 2017 +0800
Committer: Tsz-Wo Nicholas Sze <sz...@hortonworks.com>
Committed: Tue Jan 31 23:54:26 2017 +0800

----------------------------------------------------------------------
 .../org/apache/ratis/grpc/RaftGRpcService.java  | 65 +++++++++++++++-----
 .../ratis/grpc/MiniRaftClusterWithGRpc.java     | 10 ++-
 .../hadooprpc/server/HadoopRpcService.java      | 47 ++++++++++----
 .../hadooprpc/MiniRaftClusterWithHadoopRpc.java | 10 ++-
 .../ratis/netty/server/NettyRpcService.java     | 23 ++++++-
 .../ratis/netty/MiniRaftClusterWithNetty.java   |  2 +-
 .../org/apache/ratis/server/RaftServerRpc.java  | 35 +++++++++++
 7 files changed, 159 insertions(+), 33 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-ratis/blob/d16c5c64/ratis-grpc/src/main/java/org/apache/ratis/grpc/RaftGRpcService.java
----------------------------------------------------------------------
diff --git a/ratis-grpc/src/main/java/org/apache/ratis/grpc/RaftGRpcService.java b/ratis-grpc/src/main/java/org/apache/ratis/grpc/RaftGRpcService.java
index b61e70e..9ea23c3 100644
--- a/ratis-grpc/src/main/java/org/apache/ratis/grpc/RaftGRpcService.java
+++ b/ratis-grpc/src/main/java/org/apache/ratis/grpc/RaftGRpcService.java
@@ -18,11 +18,6 @@
 package org.apache.ratis.grpc;
 
 import com.google.common.base.Preconditions;
-
-import org.apache.ratis.shaded.io.grpc.Server;
-import org.apache.ratis.shaded.io.grpc.ServerBuilder;
-import org.apache.ratis.shaded.io.grpc.netty.NettyServerBuilder;
-import org.apache.ratis.shaded.proto.RaftProtos.*;
 import org.apache.ratis.conf.RaftProperties;
 import org.apache.ratis.grpc.client.RaftClientProtocolService;
 import org.apache.ratis.grpc.server.RaftServerProtocolClient;
@@ -30,36 +25,75 @@ import org.apache.ratis.grpc.server.RaftServerProtocolService;
 import org.apache.ratis.protocol.RaftPeer;
 import org.apache.ratis.server.RaftServer;
 import org.apache.ratis.server.RaftServerRpc;
+import org.apache.ratis.shaded.io.grpc.Server;
+import org.apache.ratis.shaded.io.grpc.ServerBuilder;
+import org.apache.ratis.shaded.io.grpc.netty.NettyServerBuilder;
+import org.apache.ratis.shaded.proto.RaftProtos.*;
 import org.apache.ratis.util.CodeInjectionForTesting;
+import org.apache.ratis.util.ExitUtils;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
-import static org.apache.ratis.grpc.RaftGrpcConfigKeys.RAFT_GRPC_SERVER_PORT_DEFAULT;
-import static org.apache.ratis.grpc.RaftGrpcConfigKeys.RAFT_GRPC_SERVER_PORT_KEY;
-
 import java.io.IOException;
 import java.net.InetSocketAddress;
 import java.util.Collections;
 import java.util.HashMap;
 import java.util.Map;
 
+import static org.apache.ratis.grpc.RaftGrpcConfigKeys.*;
+
+/** A grpc implementation of {@link RaftServerRpc}. */
 public class RaftGRpcService implements RaftServerRpc {
   static final Logger LOG = LoggerFactory.getLogger(RaftGRpcService.class);
   public static final String GRPC_SEND_SERVER_REQUEST =
       RaftGRpcService.class.getSimpleName() + ".sendRequest";
 
+  public static class Builder extends RaftServerRpc.Builder<Builder,RaftGRpcService> {
+    private int maxMessageSize = RAFT_GRPC_MESSAGE_MAXSIZE_DEFAULT;
+
+    private Builder() {
+      super(RAFT_GRPC_SERVER_PORT_DEFAULT);
+    }
+
+    public int getMaxMessageSize() {
+      return maxMessageSize;
+    }
+
+    public Builder setMaxMessageSize(int maxMessageSize) {
+      this.maxMessageSize = maxMessageSize;
+      return this;
+    }
+
+    public Builder setFromRaftProperties(RaftProperties properties) {
+      setPort(properties.getInt(RAFT_GRPC_SERVER_PORT_KEY,
+          RAFT_GRPC_SERVER_PORT_DEFAULT));
+      setMaxMessageSize(properties.getInt(RAFT_GRPC_MESSAGE_MAXSIZE_KEY,
+          RAFT_GRPC_MESSAGE_MAXSIZE_DEFAULT));
+      return this;
+    }
+
+    @Override
+    public Builder getThis() {
+      return this;
+    }
+
+    @Override
+    public RaftGRpcService build() {
+      return new RaftGRpcService(getServer(), getPort(), getMaxMessageSize());
+    }
+  }
+
+  public static Builder newBuilder() {
+    return new Builder();
+  }
+
   private final Server server;
   private final InetSocketAddress address;
   private final Map<String, RaftServerProtocolClient> peers =
       Collections.synchronizedMap(new HashMap<>());
   private final String selfId;
 
-  public RaftGRpcService(RaftServer raftServer, RaftProperties properties) {
-    int port = properties.getInt(RAFT_GRPC_SERVER_PORT_KEY,
-        RAFT_GRPC_SERVER_PORT_DEFAULT);
-    int maxMessageSize = properties.getInt(
-        RaftGrpcConfigKeys.RAFT_GRPC_MESSAGE_MAXSIZE_KEY,
-        RaftGrpcConfigKeys.RAFT_GRPC_MESSAGE_MAXSIZE_DEFAULT);
+  private RaftGRpcService(RaftServer raftServer, int port, int maxMessageSize) {
     ServerBuilder serverBuilder = ServerBuilder.forPort(port);
     selfId = raftServer.getId();
     server = ((NettyServerBuilder) serverBuilder).maxMessageSize(maxMessageSize)
@@ -82,8 +116,7 @@ public class RaftGRpcService implements RaftServerRpc {
     try {
       server.start();
     } catch (IOException e) {
-      LOG.error("Failed to start Grpc server", e);
-      System.exit(1);
+      ExitUtils.terminate(1, "Failed to start Grpc server", e, LOG);
     }
     Runtime.getRuntime().addShutdownHook(new Thread() {
       @Override

http://git-wip-us.apache.org/repos/asf/incubator-ratis/blob/d16c5c64/ratis-grpc/src/test/java/org/apache/ratis/grpc/MiniRaftClusterWithGRpc.java
----------------------------------------------------------------------
diff --git a/ratis-grpc/src/test/java/org/apache/ratis/grpc/MiniRaftClusterWithGRpc.java b/ratis-grpc/src/test/java/org/apache/ratis/grpc/MiniRaftClusterWithGRpc.java
index f5c7b3f..7a996eb 100644
--- a/ratis-grpc/src/test/java/org/apache/ratis/grpc/MiniRaftClusterWithGRpc.java
+++ b/ratis-grpc/src/test/java/org/apache/ratis/grpc/MiniRaftClusterWithGRpc.java
@@ -78,7 +78,10 @@ public class MiniRaftClusterWithGRpc extends MiniRaftCluster.RpcBase {
     final Map<RaftPeer, RaftGRpcService> peerRpcs = new HashMap<>();
 
     for (RaftServerImpl s : servers) {
-      final RaftGRpcService rpc = new RaftGRpcService(s, prop);
+      final RaftGRpcService rpc = RaftGRpcService.newBuilder()
+          .setFromRaftProperties(prop)
+          .setServer(s)
+          .build();
       peerRpcs.put(new RaftPeer(s.getId(), rpc.getInetSocketAddress()), rpc);
     }
     return peerRpcs;
@@ -113,7 +116,10 @@ public class MiniRaftClusterWithGRpc extends MiniRaftCluster.RpcBase {
     int oldPort = properties.getInt(RaftGrpcConfigKeys.RAFT_GRPC_SERVER_PORT_KEY,
         RaftGrpcConfigKeys.RAFT_GRPC_SERVER_PORT_DEFAULT);
     properties.setInt(RaftGrpcConfigKeys.RAFT_GRPC_SERVER_PORT_KEY, port);
-    final RaftGRpcService rpc = new RaftGRpcService(server, properties);
+    final RaftGRpcService rpc = RaftGRpcService.newBuilder()
+        .setFromRaftProperties(properties)
+        .setServer(server)
+        .build();
     Preconditions.checkState(
         rpc.getInetSocketAddress().toString().contains(peer.getAddress()),
         "address in the raft conf: %s, address in rpc server: %s",

http://git-wip-us.apache.org/repos/asf/incubator-ratis/blob/d16c5c64/ratis-hadoop/src/main/java/org/apache/ratis/hadooprpc/server/HadoopRpcService.java
----------------------------------------------------------------------
diff --git a/ratis-hadoop/src/main/java/org/apache/ratis/hadooprpc/server/HadoopRpcService.java b/ratis-hadoop/src/main/java/org/apache/ratis/hadooprpc/server/HadoopRpcService.java
index b7ac64a..4d69797 100644
--- a/ratis-hadoop/src/main/java/org/apache/ratis/hadooprpc/server/HadoopRpcService.java
+++ b/ratis-hadoop/src/main/java/org/apache/ratis/hadooprpc/server/HadoopRpcService.java
@@ -17,9 +17,7 @@
  */
 package org.apache.ratis.hadooprpc.server;
 
-import java.io.IOException;
-import java.net.InetSocketAddress;
-
+import com.google.common.base.Preconditions;
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.ipc.ProtobufRpcEngineShaded;
 import org.apache.hadoop.ipc.RPC;
@@ -34,12 +32,7 @@ import org.apache.ratis.server.RaftServerRpc;
 import org.apache.ratis.server.protocol.RaftServerProtocol;
 import org.apache.ratis.shaded.com.google.protobuf.BlockingService;
 import org.apache.ratis.shaded.com.google.protobuf.ServiceException;
-import org.apache.ratis.shaded.proto.RaftProtos.AppendEntriesReplyProto;
-import org.apache.ratis.shaded.proto.RaftProtos.AppendEntriesRequestProto;
-import org.apache.ratis.shaded.proto.RaftProtos.InstallSnapshotReplyProto;
-import org.apache.ratis.shaded.proto.RaftProtos.InstallSnapshotRequestProto;
-import org.apache.ratis.shaded.proto.RaftProtos.RequestVoteReplyProto;
-import org.apache.ratis.shaded.proto.RaftProtos.RequestVoteRequestProto;
+import org.apache.ratis.shaded.proto.RaftProtos.*;
 import org.apache.ratis.shaded.proto.hadoop.HadoopProtos.RaftClientProtocolService;
 import org.apache.ratis.shaded.proto.hadoop.HadoopProtos.RaftServerProtocolService;
 import org.apache.ratis.util.CodeInjectionForTesting;
@@ -48,7 +41,8 @@ import org.apache.ratis.util.ProtoUtils;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
-import com.google.common.base.Preconditions;
+import java.io.IOException;
+import java.net.InetSocketAddress;
 
 /** Server side Hadoop RPC service. */
 public class HadoopRpcService implements RaftServerRpc {
@@ -56,13 +50,44 @@ public class HadoopRpcService implements RaftServerRpc {
   static final String CLASS_NAME = HadoopRpcService.class.getSimpleName();
   public static final String SEND_SERVER_REQUEST = CLASS_NAME + ".sendServerRequest";
 
+  public static class Builder extends RaftServerRpc.Builder<Builder, HadoopRpcService> {
+    private Configuration conf;
+
+    private Builder() {
+      super(0);
+    }
+
+    public Configuration getConf() {
+      return conf;
+    }
+
+    public Builder setConf(Configuration conf) {
+      this.conf = conf;
+      return this;
+    }
+
+    @Override
+    public Builder getThis() {
+      return this;
+    }
+
+    @Override
+    public HadoopRpcService build() throws IOException {
+      return new HadoopRpcService(getServer(), getConf());
+    }
+  }
+
+  public static Builder newBuilder() {
+    return new Builder();
+  }
+
   private final String id;
   private final RPC.Server ipcServer;
   private final InetSocketAddress ipcServerAddress;
 
   private final PeerProxyMap<Proxy<RaftServerProtocolPB>> proxies;
 
-  public HadoopRpcService(RaftServer server, final Configuration conf)
+  private HadoopRpcService(RaftServer server, final Configuration conf)
       throws IOException {
     this.proxies = new PeerProxyMap<>(
         p -> new Proxy(RaftServerProtocolPB.class, p.getAddress(), conf));

http://git-wip-us.apache.org/repos/asf/incubator-ratis/blob/d16c5c64/ratis-hadoop/src/test/java/org/apache/ratis/hadooprpc/MiniRaftClusterWithHadoopRpc.java
----------------------------------------------------------------------
diff --git a/ratis-hadoop/src/test/java/org/apache/ratis/hadooprpc/MiniRaftClusterWithHadoopRpc.java b/ratis-hadoop/src/test/java/org/apache/ratis/hadooprpc/MiniRaftClusterWithHadoopRpc.java
index 964f3a2..b8c69d5 100644
--- a/ratis-hadoop/src/test/java/org/apache/ratis/hadooprpc/MiniRaftClusterWithHadoopRpc.java
+++ b/ratis-hadoop/src/test/java/org/apache/ratis/hadooprpc/MiniRaftClusterWithHadoopRpc.java
@@ -74,7 +74,10 @@ public class MiniRaftClusterWithHadoopRpc extends MiniRaftCluster.RpcBase {
     final Map<RaftPeer, HadoopRpcService> peerRpcs = new HashMap<>();
 
     for(RaftServerImpl s : servers) {
-      final HadoopRpcService rpc = new HadoopRpcService(s, hadoopConf);
+      final HadoopRpcService rpc = HadoopRpcService.newBuilder()
+          .setServer(s)
+          .setConf(hadoopConf)
+          .build();
       peerRpcs.put(new RaftPeer(s.getId(), rpc.getInetSocketAddress()), rpc);
     }
     return peerRpcs;
@@ -86,7 +89,10 @@ public class MiniRaftClusterWithHadoopRpc extends MiniRaftCluster.RpcBase {
     hconf.set(RaftServerConfigKeys.Ipc.ADDRESS_KEY, peer.getAddress());
 
     RaftServerImpl server = servers.get(peer.getId());
-    final HadoopRpcService rpc = new HadoopRpcService(server, hconf);
+    final HadoopRpcService rpc = HadoopRpcService.newBuilder()
+        .setServer(server)
+        .setConf(hconf)
+        .build();
     Preconditions.checkState(
         rpc.getInetSocketAddress().toString().contains(peer.getAddress()),
         "address in the raft conf: %s, address in rpc server: %s",

http://git-wip-us.apache.org/repos/asf/incubator-ratis/blob/d16c5c64/ratis-netty/src/main/java/org/apache/ratis/netty/server/NettyRpcService.java
----------------------------------------------------------------------
diff --git a/ratis-netty/src/main/java/org/apache/ratis/netty/server/NettyRpcService.java b/ratis-netty/src/main/java/org/apache/ratis/netty/server/NettyRpcService.java
index 153f61e..b3f2efb 100644
--- a/ratis-netty/src/main/java/org/apache/ratis/netty/server/NettyRpcService.java
+++ b/ratis-netty/src/main/java/org/apache/ratis/netty/server/NettyRpcService.java
@@ -55,6 +55,27 @@ public final class NettyRpcService implements RaftServerRpc {
   static final String CLASS_NAME = NettyRpcService.class.getSimpleName();
   public static final String SEND_SERVER_REQUEST = CLASS_NAME + ".sendServerRequest";
 
+  public static class Builder extends RaftServerRpc.Builder<Builder, NettyRpcService> {
+    private Builder() {
+      super(0);
+    }
+
+    @Override
+    public Builder getThis() {
+      return this;
+    }
+
+    @Override
+    public NettyRpcService build() {
+      return new NettyRpcService(getServer(), getPort());
+    }
+  }
+
+  public static Builder newBuilder() {
+    return new Builder();
+  }
+
+
   private final LifeCycle lifeCycle = new LifeCycle(getClass().getSimpleName());
   private final RaftServer server;
   private final String id;
@@ -75,7 +96,7 @@ public final class NettyRpcService implements RaftServerRpc {
   }
 
   /** Constructs a netty server with the given port. */
-  public NettyRpcService(int port, RaftServer server) {
+  private NettyRpcService(RaftServer server, int port) {
     this.server = server;
     this.id = server.getId();
 

http://git-wip-us.apache.org/repos/asf/incubator-ratis/blob/d16c5c64/ratis-netty/src/test/java/org/apache/ratis/netty/MiniRaftClusterWithNetty.java
----------------------------------------------------------------------
diff --git a/ratis-netty/src/test/java/org/apache/ratis/netty/MiniRaftClusterWithNetty.java b/ratis-netty/src/test/java/org/apache/ratis/netty/MiniRaftClusterWithNetty.java
index 92e7722..32ed98b 100644
--- a/ratis-netty/src/test/java/org/apache/ratis/netty/MiniRaftClusterWithNetty.java
+++ b/ratis-netty/src/test/java/org/apache/ratis/netty/MiniRaftClusterWithNetty.java
@@ -72,7 +72,7 @@ public class MiniRaftClusterWithNetty extends MiniRaftCluster.RpcBase {
       RaftServerImpl s, RaftConfiguration conf) {
     final String address = getAddress(s.getId(), conf);
     final int port = NetUtils.newInetSocketAddress(address).getPort();
-    return new NettyRpcService(port, s);
+    return NettyRpcService.newBuilder().setServer(s).setPort(port).build();
   }
 
   private static Map<RaftPeer, NettyRpcService> initRpcServices(

http://git-wip-us.apache.org/repos/asf/incubator-ratis/blob/d16c5c64/ratis-server/src/main/java/org/apache/ratis/server/RaftServerRpc.java
----------------------------------------------------------------------
diff --git a/ratis-server/src/main/java/org/apache/ratis/server/RaftServerRpc.java b/ratis-server/src/main/java/org/apache/ratis/server/RaftServerRpc.java
index 5fecce3..61b3b2e 100644
--- a/ratis-server/src/main/java/org/apache/ratis/server/RaftServerRpc.java
+++ b/ratis-server/src/main/java/org/apache/ratis/server/RaftServerRpc.java
@@ -21,6 +21,7 @@ import org.apache.ratis.protocol.RaftPeer;
 import org.apache.ratis.server.protocol.RaftServerProtocol;
 
 import java.io.Closeable;
+import java.io.IOException;
 import java.net.InetSocketAddress;
 
 /**
@@ -28,6 +29,40 @@ import java.net.InetSocketAddress;
  * such as Netty, gRPC and Hadoop.
  */
 public interface RaftServerRpc extends RaftServerProtocol, Closeable {
+  /** To build {@link RaftServerRpc} objects. */
+  abstract class Builder<B extends Builder, RPC extends RaftServerRpc> {
+    private RaftServer server;
+    private int port;
+
+    /** Construct a builder with the default port. */
+    protected Builder(int defaultPort) {
+      this.port = defaultPort;
+    }
+
+    public RaftServer getServer() {
+      return server;
+    }
+
+    public B setServer(RaftServer server) {
+      this.server = server;
+      return getThis();
+    }
+
+    public int getPort() {
+      return port;
+    }
+
+    /** Set the port for the server to listen to. */
+    public B setPort(int port) {
+      this.port = port;
+      return getThis();
+    }
+
+    protected abstract B getThis();
+
+    public abstract RPC build() throws IOException;
+  }
+
   /** Start the RPC service. */
   void start();
 


[16/54] [abbrv] incubator-ratis git commit: Renamed the packages from raft to ratis in preperation for Apache Incubation - Moved all java packages from org.apache.raft to org.apache.ratis. - Moved native package to org_apache_ratis, and native lib to l

Posted by ji...@apache.org.
http://git-wip-us.apache.org/repos/asf/incubator-ratis/blob/7e71a2e0/ratis-server/src/test/java/org/apache/ratis/server/storage/TestRaftLogCache.java
----------------------------------------------------------------------
diff --git a/ratis-server/src/test/java/org/apache/ratis/server/storage/TestRaftLogCache.java b/ratis-server/src/test/java/org/apache/ratis/server/storage/TestRaftLogCache.java
new file mode 100644
index 0000000..f5a18ac
--- /dev/null
+++ b/ratis-server/src/test/java/org/apache/ratis/server/storage/TestRaftLogCache.java
@@ -0,0 +1,255 @@
+/**
+ * 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.ratis.server.storage;
+
+import java.util.Iterator;
+
+import org.apache.ratis.RaftTestUtil.SimpleOperation;
+import org.apache.ratis.server.storage.RaftLogCache.TruncationSegments;
+import org.apache.ratis.shaded.proto.RaftProtos.LogEntryProto;
+import org.apache.ratis.util.ProtoUtils;
+import org.junit.Assert;
+import org.junit.Before;
+import org.junit.Test;
+
+public class TestRaftLogCache {
+  private RaftLogCache cache;
+
+  @Before
+  public void setup() {
+    cache = new RaftLogCache();
+  }
+
+  private LogSegment prepareLogSegment(long start, long end, boolean isOpen) {
+    LogSegment s = LogSegment.newOpenSegment(start);
+    for (long i = start; i <= end; i++) {
+      SimpleOperation m = new SimpleOperation("m" + i);
+      LogEntryProto entry = ProtoUtils.toLogEntryProto(m.getLogEntryContent(),
+          0, i);
+      s.appendToOpenSegment(entry);
+    }
+    if (!isOpen) {
+      s.close();
+    }
+    return s;
+  }
+
+  private void checkCache(long start, long end, int segmentSize) {
+    Assert.assertEquals(start, cache.getStartIndex());
+    Assert.assertEquals(end, cache.getEndIndex());
+
+    for (long index = start; index <= end; index++) {
+      LogEntryProto entry = cache.getEntry(index);
+      Assert.assertEquals(index, entry.getIndex());
+    }
+
+    long[] offsets = new long[]{start, start + 1, start + (end - start) / 2,
+        end - 1, end};
+    for (long offset : offsets) {
+      checkCacheEntries(offset, (int) (end - offset + 1), end);
+      checkCacheEntries(offset, 1, end);
+      checkCacheEntries(offset, 20, end);
+      checkCacheEntries(offset, segmentSize, end);
+      checkCacheEntries(offset, segmentSize - 1, end);
+    }
+  }
+
+  private void checkCacheEntries(long offset, int size, long end) {
+    LogEntryProto[] entries = cache.getEntries(offset, offset + size);
+    long realEnd = offset + size > end + 1 ? end + 1 : offset + size;
+    Assert.assertEquals(realEnd - offset, entries.length);
+    for (long i = offset; i < realEnd; i++) {
+      Assert.assertEquals(i, entries[(int) (i - offset)].getIndex());
+    }
+  }
+
+  @Test
+  public void testAddSegments() throws Exception {
+    LogSegment s1 = prepareLogSegment(1, 100, false);
+    cache.addSegment(s1);
+    checkCache(1, 100, 100);
+
+    try {
+      LogSegment s = prepareLogSegment(102, 103, true);
+      cache.addSegment(s);
+      Assert.fail("should fail since there is gap between two segments");
+    } catch (IllegalStateException ignored) {
+    }
+
+    LogSegment s2 = prepareLogSegment(101, 200, true);
+    cache.addSegment(s2);
+    checkCache(1, 200, 100);
+
+    try {
+      LogSegment s = prepareLogSegment(201, 202, true);
+      cache.addSegment(s);
+      Assert.fail("should fail since there is still an open segment in cache");
+    } catch (IllegalStateException ignored) {
+    }
+
+    cache.rollOpenSegment(false);
+    checkCache(1, 200, 100);
+
+    try {
+      LogSegment s = prepareLogSegment(202, 203, true);
+      cache.addSegment(s);
+      Assert.fail("should fail since there is gap between two segments");
+    } catch (IllegalStateException ignored) {
+    }
+
+    LogSegment s3 = prepareLogSegment(201, 300, true);
+    cache.addSegment(s3);
+    Assert.assertNotNull(cache.getOpenSegment());
+    checkCache(1, 300, 100);
+
+    cache.rollOpenSegment(true);
+    Assert.assertNotNull(cache.getOpenSegment());
+    checkCache(1, 300, 100);
+  }
+
+  @Test
+  public void testAppendEntry() throws Exception {
+    LogSegment closedSegment = prepareLogSegment(0, 99, false);
+    cache.addSegment(closedSegment);
+
+    final SimpleOperation m = new SimpleOperation("m");
+    try {
+      LogEntryProto entry = ProtoUtils.toLogEntryProto(m.getLogEntryContent(),
+          0, 0);
+      cache.appendEntry(entry);
+      Assert.fail("the open segment is null");
+    } catch (IllegalStateException ignored) {
+    }
+
+    LogSegment openSegment = prepareLogSegment(100, 100, true);
+    cache.addSegment(openSegment);
+    for (long index = 101; index < 200; index++) {
+      LogEntryProto entry = ProtoUtils.toLogEntryProto(m.getLogEntryContent(),
+          0, index);
+      cache.appendEntry(entry);
+    }
+
+    Assert.assertNotNull(cache.getOpenSegment());
+    checkCache(0, 199, 100);
+  }
+
+  @Test
+  public void testTruncate() throws Exception {
+    long start = 0;
+    for (int i = 0; i < 5; i++) { // 5 closed segments
+      LogSegment s = prepareLogSegment(start, start + 99, false);
+      cache.addSegment(s);
+      start += 100;
+    }
+    // add another open segment
+    LogSegment s = prepareLogSegment(start, start + 99, true);
+    cache.addSegment(s);
+
+    long end = cache.getEndIndex();
+    Assert.assertEquals(599, end);
+    int numOfSegments = 6;
+    // start truncation
+    for (int i = 0; i < 10; i++) { // truncate 10 times
+      // each time truncate 37 entries
+      end -= 37;
+      TruncationSegments ts = cache.truncate(end + 1);
+      checkCache(0, end, 100);
+
+      // check TruncationSegments
+      int currentNum= (int) (end / 100 + 1);
+      if (currentNum < numOfSegments) {
+        Assert.assertEquals(1, ts.toDelete.length);
+        numOfSegments = currentNum;
+      } else {
+        Assert.assertEquals(0, ts.toDelete.length);
+      }
+    }
+
+    // 230 entries remaining. truncate at the segment boundary
+    TruncationSegments ts = cache.truncate(200);
+    checkCache(0, 199, 100);
+    Assert.assertEquals(1, ts.toDelete.length);
+    Assert.assertEquals(200, ts.toDelete[0].startIndex);
+    Assert.assertEquals(229, ts.toDelete[0].endIndex);
+    Assert.assertEquals(0, ts.toDelete[0].targetLength);
+    Assert.assertFalse(ts.toDelete[0].isOpen);
+    Assert.assertNull(ts.toTruncate);
+
+    // add another open segment and truncate it as a whole
+    LogSegment newOpen = prepareLogSegment(200, 249, true);
+    cache.addSegment(newOpen);
+    ts = cache.truncate(200);
+    checkCache(0, 199, 100);
+    Assert.assertEquals(1, ts.toDelete.length);
+    Assert.assertEquals(200, ts.toDelete[0].startIndex);
+    Assert.assertEquals(249, ts.toDelete[0].endIndex);
+    Assert.assertEquals(0, ts.toDelete[0].targetLength);
+    Assert.assertTrue(ts.toDelete[0].isOpen);
+    Assert.assertNull(ts.toTruncate);
+
+    // add another open segment and truncate part of it
+    newOpen = prepareLogSegment(200, 249, true);
+    cache.addSegment(newOpen);
+    ts = cache.truncate(220);
+    checkCache(0, 219, 100);
+    Assert.assertNull(cache.getOpenSegment());
+    Assert.assertEquals(0, ts.toDelete.length);
+    Assert.assertTrue(ts.toTruncate.isOpen);
+    Assert.assertEquals(219, ts.toTruncate.newEndIndex);
+    Assert.assertEquals(200, ts.toTruncate.startIndex);
+    Assert.assertEquals(249, ts.toTruncate.endIndex);
+  }
+
+  private void testIterator(long startIndex) {
+    Iterator<LogEntryProto> iterator = cache.iterator(startIndex);
+    LogEntryProto prev = null;
+    while (iterator.hasNext()) {
+      LogEntryProto entry = iterator.next();
+      Assert.assertEquals(cache.getEntry(entry.getIndex()), entry);
+      if (prev != null) {
+        Assert.assertEquals(prev.getIndex() + 1, entry.getIndex());
+      }
+      prev = entry;
+    }
+    if (startIndex <= cache.getEndIndex()) {
+      Assert.assertNotNull(prev);
+      Assert.assertEquals(cache.getEndIndex(), prev.getIndex());
+    }
+  }
+
+  @Test
+  public void testIterator() throws Exception {
+    long start = 0;
+    for (int i = 0; i < 2; i++) { // 2 closed segments
+      LogSegment s = prepareLogSegment(start, start + 99, false);
+      cache.addSegment(s);
+      start += 100;
+    }
+    // add another open segment
+    LogSegment s = prepareLogSegment(start, start + 99, true);
+    cache.addSegment(s);
+
+    for (long startIndex = 0; startIndex < 300; startIndex += 50) {
+      testIterator(startIndex);
+    }
+    testIterator(299);
+
+    Iterator<LogEntryProto> iterator = cache.iterator(300);
+    Assert.assertFalse(iterator.hasNext());
+  }
+}

http://git-wip-us.apache.org/repos/asf/incubator-ratis/blob/7e71a2e0/ratis-server/src/test/java/org/apache/ratis/server/storage/TestRaftLogReadWrite.java
----------------------------------------------------------------------
diff --git a/ratis-server/src/test/java/org/apache/ratis/server/storage/TestRaftLogReadWrite.java b/ratis-server/src/test/java/org/apache/ratis/server/storage/TestRaftLogReadWrite.java
new file mode 100644
index 0000000..bcdb958
--- /dev/null
+++ b/ratis-server/src/test/java/org/apache/ratis/server/storage/TestRaftLogReadWrite.java
@@ -0,0 +1,269 @@
+/**
+ * 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.ratis.server.storage;
+
+import static org.apache.ratis.server.RaftServerConfigKeys.RAFT_LOG_SEGMENT_MAX_SIZE_KEY;
+import static org.apache.ratis.server.RaftServerConfigKeys.RAFT_LOG_SEGMENT_PREALLOCATED_SIZE_DEFAULT;
+import static org.apache.ratis.server.RaftServerConfigKeys.RAFT_LOG_SEGMENT_PREALLOCATED_SIZE_KEY;
+import static org.apache.ratis.server.RaftServerConfigKeys.RAFT_SERVER_STORAGE_DIR_KEY;
+
+import java.io.File;
+import java.io.FileOutputStream;
+import java.io.IOException;
+import java.io.RandomAccessFile;
+import java.nio.ByteBuffer;
+import java.util.ArrayList;
+import java.util.List;
+
+import org.apache.ratis.RaftTestUtil;
+import org.apache.ratis.RaftTestUtil.SimpleOperation;
+import org.apache.ratis.conf.RaftProperties;
+import org.apache.ratis.protocol.ChecksumException;
+import org.apache.ratis.server.impl.RaftServerConstants;
+import org.apache.ratis.server.impl.RaftServerConstants.StartupOption;
+import org.apache.ratis.shaded.com.google.protobuf.CodedOutputStream;
+import org.apache.ratis.shaded.proto.RaftProtos.LogEntryProto;
+import org.apache.ratis.util.FileUtils;
+import org.apache.ratis.util.ProtoUtils;
+import org.junit.After;
+import org.junit.Assert;
+import org.junit.Before;
+import org.junit.Test;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+/**
+ * Test basic functionality of LogReader, LogInputStream, and LogOutputStream.
+ */
+public class TestRaftLogReadWrite {
+  private static final Logger LOG = LoggerFactory.getLogger(TestRaftLogReadWrite.class);
+
+  private File storageDir;
+  private RaftProperties properties;
+  private int segmentMaxSize;
+
+  @Before
+  public void setup() throws Exception {
+    storageDir = RaftTestUtil.getTestDir(TestRaftLogReadWrite.class);
+    properties = new RaftProperties();
+    properties.set(RAFT_SERVER_STORAGE_DIR_KEY,
+        FileUtils.fileAsURI(storageDir).toString());
+  }
+
+  @After
+  public void tearDown() throws Exception {
+    if (storageDir != null) {
+      FileUtils.fullyDelete(storageDir.getParentFile());
+    }
+  }
+
+  private LogEntryProto[] readLog(File file, long startIndex, long endIndex,
+      boolean isOpen) throws IOException {
+    List<LogEntryProto> list = new ArrayList<>();
+    try (LogInputStream in =
+             new LogInputStream(file, startIndex, endIndex, isOpen)) {
+      LogEntryProto entry;
+      while ((entry = in.nextEntry()) != null) {
+        list.add(entry);
+      }
+    }
+    return list.toArray(new LogEntryProto[list.size()]);
+  }
+
+  private long writeMessages(LogEntryProto[] entries, LogOutputStream out)
+      throws IOException {
+    long size = 0;
+    for (int i = 0; i < entries.length; i++) {
+      SimpleOperation m = new SimpleOperation("m" + i);
+      entries[i] = ProtoUtils.toLogEntryProto(m.getLogEntryContent(), 0, i);
+      final int s = entries[i].getSerializedSize();
+      size += CodedOutputStream.computeUInt32SizeNoTag(s) + s + 4;
+      out.write(entries[i]);
+    }
+    return size;
+  }
+
+  /**
+   * Test basic functionality: write several log entries, then read
+   */
+  @Test
+  public void testReadWriteLog() throws IOException {
+    RaftStorage storage = new RaftStorage(properties, StartupOption.REGULAR);
+    File openSegment = storage.getStorageDir().getOpenLogFile(0);
+    long size = SegmentedRaftLog.HEADER_BYTES.length;
+
+    final LogEntryProto[] entries = new LogEntryProto[100];
+    try (LogOutputStream out =
+             new LogOutputStream(openSegment, false, properties)) {
+      size += writeMessages(entries, out);
+    } finally {
+      storage.close();
+    }
+
+    Assert.assertEquals(size, openSegment.length());
+
+    LogEntryProto[] readEntries = readLog(openSegment, 0,
+        RaftServerConstants.INVALID_LOG_INDEX, true);
+    Assert.assertArrayEquals(entries, readEntries);
+  }
+
+  @Test
+  public void testAppendLog() throws IOException {
+    RaftStorage storage = new RaftStorage(properties, StartupOption.REGULAR);
+    File openSegment = storage.getStorageDir().getOpenLogFile(0);
+    LogEntryProto[] entries = new LogEntryProto[200];
+    try (LogOutputStream out =
+             new LogOutputStream(openSegment, false, properties)) {
+      for (int i = 0; i < 100; i++) {
+        SimpleOperation m = new SimpleOperation("m" + i);
+        entries[i] = ProtoUtils.toLogEntryProto(m.getLogEntryContent(), 0, i);
+        out.write(entries[i]);
+      }
+    }
+
+    try (LogOutputStream out =
+             new LogOutputStream(openSegment, true, properties)) {
+      for (int i = 100; i < 200; i++) {
+        SimpleOperation m = new SimpleOperation("m" + i);
+        entries[i] = ProtoUtils.toLogEntryProto(m.getLogEntryContent(), 0, i);
+        out.write(entries[i]);
+      }
+    }
+
+    LogEntryProto[] readEntries = readLog(openSegment, 0,
+        RaftServerConstants.INVALID_LOG_INDEX, true);
+    Assert.assertArrayEquals(entries, readEntries);
+
+    storage.close();
+  }
+
+  /**
+   * Simulate the scenario that the peer is shutdown without truncating
+   * log segment file padding. Make sure the reader can correctly handle this.
+   */
+  @Test
+  public void testReadWithPadding() throws IOException {
+    RaftStorage storage = new RaftStorage(properties, StartupOption.REGULAR);
+    File openSegment = storage.getStorageDir().getOpenLogFile(0);
+    long size = SegmentedRaftLog.HEADER_BYTES.length;
+
+    LogEntryProto[] entries = new LogEntryProto[100];
+    LogOutputStream out = new LogOutputStream(openSegment, false, properties);
+    size += writeMessages(entries, out);
+    out.flush();
+
+    // make sure the file contains padding
+    Assert.assertEquals(RAFT_LOG_SEGMENT_PREALLOCATED_SIZE_DEFAULT,
+        openSegment.length());
+
+    // check if the reader can correctly read the log file
+    LogEntryProto[] readEntries = readLog(openSegment, 0,
+        RaftServerConstants.INVALID_LOG_INDEX, true);
+    Assert.assertArrayEquals(entries, readEntries);
+
+    out.close();
+    Assert.assertEquals(size, openSegment.length());
+  }
+
+  /**
+   * corrupt the padding by inserting non-zero bytes. Make sure the reader
+   * throws exception.
+   */
+  @Test
+  public void testReadWithCorruptPadding() throws IOException {
+    properties.setLong(RAFT_LOG_SEGMENT_PREALLOCATED_SIZE_KEY, 4 * 1024 * 1024);
+    properties.setLong(RAFT_LOG_SEGMENT_MAX_SIZE_KEY, 16 * 1024 * 1024);
+
+    RaftStorage storage = new RaftStorage(properties, StartupOption.REGULAR);
+    File openSegment = storage.getStorageDir().getOpenLogFile(0);
+
+    LogEntryProto[] entries = new LogEntryProto[10];
+    LogOutputStream out = new LogOutputStream(openSegment, false, properties);
+    for (int i = 0; i < 10; i++) {
+      SimpleOperation m = new SimpleOperation("m" + i);
+      entries[i] = ProtoUtils.toLogEntryProto(m.getLogEntryContent(), 0, i);
+      out.write(entries[i]);
+    }
+    out.flush();
+
+    // make sure the file contains padding
+    Assert.assertEquals(4 * 1024 * 1024, openSegment.length());
+
+    try (FileOutputStream fout = new FileOutputStream(openSegment, true)) {
+      ByteBuffer byteBuffer = ByteBuffer.wrap(new byte[]{-1, 1});
+      fout.getChannel()
+          .write(byteBuffer, 16 * 1024 * 1024 - 10);
+    }
+
+    List<LogEntryProto> list = new ArrayList<>();
+    try (LogInputStream in = new LogInputStream(openSegment, 0,
+        RaftServerConstants.INVALID_LOG_INDEX, true)) {
+      LogEntryProto entry;
+      while ((entry = in.nextEntry()) != null) {
+        list.add(entry);
+      }
+      Assert.fail("should fail since we corrupt the padding");
+    } catch (IOException e) {
+      boolean findVerifyTerminator = false;
+      for (StackTraceElement s : e.getStackTrace()) {
+        if (s.getMethodName().equals("verifyTerminator")) {
+          findVerifyTerminator = true;
+          break;
+        }
+      }
+      Assert.assertTrue(findVerifyTerminator);
+    }
+    Assert.assertArrayEquals(entries,
+        list.toArray(new LogEntryProto[list.size()]));
+  }
+
+  /**
+   * Test the log reader to make sure it can detect the checksum mismatch.
+   */
+  @Test
+  public void testReadWithEntryCorruption() throws IOException {
+    RaftStorage storage = new RaftStorage(properties, StartupOption.REGULAR);
+    File openSegment = storage.getStorageDir().getOpenLogFile(0);
+    try (LogOutputStream out =
+             new LogOutputStream(openSegment, false, properties)) {
+      for (int i = 0; i < 100; i++) {
+        LogEntryProto entry = ProtoUtils.toLogEntryProto(
+            new SimpleOperation("m" + i).getLogEntryContent(), 0, i);
+        out.write(entry);
+      }
+    } finally {
+      storage.close();
+    }
+
+    // corrupt the log file
+    try (RandomAccessFile raf = new RandomAccessFile(openSegment.getCanonicalFile(),
+        "rw")) {
+      raf.seek(100);
+      int correctValue = raf.read();
+      raf.seek(100);
+      raf.write(correctValue + 1);
+    }
+
+    try {
+      readLog(openSegment, 0, RaftServerConstants.INVALID_LOG_INDEX, true);
+      Assert.fail("The read of corrupted log file should fail");
+    } catch (ChecksumException e) {
+      LOG.info("Caught ChecksumException as expected", e);
+    }
+  }
+}

http://git-wip-us.apache.org/repos/asf/incubator-ratis/blob/7e71a2e0/ratis-server/src/test/java/org/apache/ratis/server/storage/TestRaftLogSegment.java
----------------------------------------------------------------------
diff --git a/ratis-server/src/test/java/org/apache/ratis/server/storage/TestRaftLogSegment.java b/ratis-server/src/test/java/org/apache/ratis/server/storage/TestRaftLogSegment.java
new file mode 100644
index 0000000..3092a21
--- /dev/null
+++ b/ratis-server/src/test/java/org/apache/ratis/server/storage/TestRaftLogSegment.java
@@ -0,0 +1,305 @@
+/**
+ * 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.ratis.server.storage;
+
+import static org.apache.ratis.server.RaftServerConfigKeys.RAFT_LOG_SEGMENT_MAX_SIZE_KEY;
+import static org.apache.ratis.server.RaftServerConfigKeys.RAFT_LOG_SEGMENT_PREALLOCATED_SIZE_KEY;
+import static org.apache.ratis.server.RaftServerConfigKeys.RAFT_LOG_WRITE_BUFFER_SIZE_KEY;
+import static org.apache.ratis.server.impl.RaftServerConstants.INVALID_LOG_INDEX;
+import static org.apache.ratis.server.storage.LogSegment.getEntrySize;
+
+import java.io.File;
+import java.io.IOException;
+import java.util.ArrayList;
+import java.util.Arrays;
+import java.util.List;
+
+import org.apache.ratis.RaftTestUtil;
+import org.apache.ratis.RaftTestUtil.SimpleOperation;
+import org.apache.ratis.conf.RaftProperties;
+import org.apache.ratis.server.RaftServerConfigKeys;
+import org.apache.ratis.server.impl.RaftServerConstants.StartupOption;
+import org.apache.ratis.shaded.proto.RaftProtos.LogEntryProto;
+import org.apache.ratis.shaded.proto.RaftProtos.SMLogEntryProto;
+import org.apache.ratis.util.FileUtils;
+import org.apache.ratis.util.ProtoUtils;
+import org.junit.After;
+import org.junit.Assert;
+import org.junit.Before;
+import org.junit.Test;
+
+/**
+ * Test basic functionality of {@link LogSegment}
+ */
+public class TestRaftLogSegment {
+  private File storageDir;
+  private final RaftProperties properties = new RaftProperties();
+
+  @Before
+  public void setup() throws Exception {
+    storageDir = RaftTestUtil.getTestDir(TestRaftLogSegment.class);
+    properties.set(RaftServerConfigKeys.RAFT_SERVER_STORAGE_DIR_KEY,
+        storageDir.getCanonicalPath());
+  }
+
+  @After
+  public void tearDown() throws Exception {
+    if (storageDir != null) {
+      FileUtils.fullyDelete(storageDir.getParentFile());
+    }
+  }
+
+  private File prepareLog(boolean isOpen, long start, int size, long term)
+      throws IOException {
+    RaftStorage storage = new RaftStorage(properties, StartupOption.REGULAR);
+    File file = isOpen ? storage.getStorageDir().getOpenLogFile(start) :
+        storage.getStorageDir().getClosedLogFile(start, start + size - 1);
+
+    LogEntryProto[] entries = new LogEntryProto[size];
+    try (LogOutputStream out = new LogOutputStream(file, false, properties)) {
+      for (int i = 0; i < size; i++) {
+        SimpleOperation op = new SimpleOperation("m" + i);
+        entries[i] = ProtoUtils.toLogEntryProto(op.getLogEntryContent(),
+            term, i + start);
+        out.write(entries[i]);
+      }
+    }
+    storage.close();
+    return file;
+  }
+
+  private void checkLogSegment(LogSegment segment, long start, long end,
+      boolean isOpen, long totalSize, long term) {
+    Assert.assertEquals(start, segment.getStartIndex());
+    Assert.assertEquals(end, segment.getEndIndex());
+    Assert.assertEquals(isOpen, segment.isOpen());
+    Assert.assertEquals(totalSize, segment.getTotalSize());
+
+    long offset = SegmentedRaftLog.HEADER_BYTES.length;
+    for (long i = start; i <= end; i++) {
+      LogSegment.LogRecord record = segment.getLogRecord(i);
+      Assert.assertEquals(i, record.entry.getIndex());
+      Assert.assertEquals(term, record.entry.getTerm());
+      Assert.assertEquals(offset, record.offset);
+
+      offset += getEntrySize(record.entry);
+    }
+  }
+
+  @Test
+  public void testLoadLogSegment() throws Exception {
+    // load an open segment
+    File openSegmentFile = prepareLog(true, 0, 100, 0);
+    LogSegment openSegment = LogSegment.loadSegment(openSegmentFile, 0,
+        INVALID_LOG_INDEX, true, null);
+    checkLogSegment(openSegment, 0, 99, true, openSegmentFile.length(), 0);
+
+    // load a closed segment (1000-1099)
+    File closedSegmentFile = prepareLog(false, 1000, 100, 1);
+    LogSegment closedSegment = LogSegment.loadSegment(closedSegmentFile, 1000,
+        1099, false, null);
+    checkLogSegment(closedSegment, 1000, 1099, false,
+        closedSegment.getTotalSize(), 1);
+  }
+
+  @Test
+  public void testAppendEntries() throws Exception {
+    final long start = 1000;
+    LogSegment segment = LogSegment.newOpenSegment(start);
+    long size = SegmentedRaftLog.HEADER_BYTES.length;
+    final long max = 8 * 1024 * 1024;
+    checkLogSegment(segment, start, start - 1, true, size, 0);
+
+    // append till full
+    long term = 0;
+    int i = 0;
+    List<LogEntryProto> list = new ArrayList<>();
+    while (size < max) {
+      SimpleOperation op = new SimpleOperation("m" + i);
+      LogEntryProto entry = ProtoUtils.toLogEntryProto(op.getLogEntryContent(),
+          term, i++ + start);
+      size += getEntrySize(entry);
+      list.add(entry);
+    }
+
+    segment.appendToOpenSegment(list.toArray(new LogEntryProto[list.size()]));
+    Assert.assertTrue(segment.getTotalSize() >= max);
+    checkLogSegment(segment, start, i - 1 + start, true, size, term);
+  }
+
+  @Test
+  public void testAppendWithGap() throws Exception {
+    LogSegment segment = LogSegment.newOpenSegment(1000);
+    SimpleOperation op = new SimpleOperation("m");
+    final SMLogEntryProto m = op.getLogEntryContent();
+    try {
+      LogEntryProto entry = ProtoUtils.toLogEntryProto(m, 0, 1001);
+      segment.appendToOpenSegment(entry);
+      Assert.fail("should fail since the entry's index needs to be 1000");
+    } catch (Exception e) {
+      Assert.assertTrue(e instanceof IllegalArgumentException);
+    }
+
+    LogEntryProto entry = ProtoUtils.toLogEntryProto(m, 0, 1000);
+    segment.appendToOpenSegment(entry);
+
+    try {
+      entry = ProtoUtils.toLogEntryProto(m, 0, 1002);
+      segment.appendToOpenSegment(entry);
+      Assert.fail("should fail since the entry's index needs to be 1001");
+    } catch (Exception e) {
+      Assert.assertTrue(e instanceof IllegalArgumentException);
+    }
+
+    LogEntryProto[] entries = new LogEntryProto[2];
+    for (int i = 0; i < 2; i++) {
+      entries[i] = ProtoUtils.toLogEntryProto(m, 0, 1001 + i * 2);
+    }
+    try {
+      segment.appendToOpenSegment(entries);
+      Assert.fail("should fail since there is gap between entries");
+    } catch (Exception e) {
+      Assert.assertTrue(e instanceof IllegalArgumentException);
+    }
+  }
+
+  @Test
+  public void testTruncate() throws Exception {
+    final long term = 1;
+    final long start = 1000;
+    LogSegment segment = LogSegment.newOpenSegment(start);
+    for (int i = 0; i < 100; i++) {
+      LogEntryProto entry = ProtoUtils.toLogEntryProto(
+          new SimpleOperation("m" + i).getLogEntryContent(), term, i + start);
+      segment.appendToOpenSegment(entry);
+    }
+
+    // truncate an open segment (remove 1080~1099)
+    long newSize = segment.getLogRecord(start + 80).offset;
+    segment.truncate(start + 80);
+    Assert.assertEquals(80, segment.numOfEntries());
+    checkLogSegment(segment, start, start + 79, false, newSize, term);
+
+    // truncate a closed segment (remove 1050~1079)
+    newSize = segment.getLogRecord(start + 50).offset;
+    segment.truncate(start + 50);
+    Assert.assertEquals(50, segment.numOfEntries());
+    checkLogSegment(segment, start, start + 49, false, newSize, term);
+
+    // truncate all the remaining entries
+    segment.truncate(start);
+    Assert.assertEquals(0, segment.numOfEntries());
+    checkLogSegment(segment, start, start - 1, false,
+        SegmentedRaftLog.HEADER_BYTES.length, term);
+  }
+
+  private RaftProperties getProperties(long maxSegmentSize,
+      long preallocatedSize) {
+    RaftProperties p = new RaftProperties();
+    p.setLong(RAFT_LOG_SEGMENT_MAX_SIZE_KEY,
+        maxSegmentSize);
+    p.setLong(RaftServerConfigKeys.RAFT_LOG_SEGMENT_PREALLOCATED_SIZE_KEY,
+        preallocatedSize);
+    return p;
+  }
+
+  @Test
+  public void testPreallocateSegment() throws Exception {
+    RaftStorage storage = new RaftStorage(properties, StartupOption.REGULAR);
+    final File file = storage.getStorageDir().getOpenLogFile(0);
+    final int[] maxSizes = new int[]{1024, 1025, 1024 * 1024 - 1, 1024 * 1024,
+        1024 * 1024 + 1, 2 * 1024 * 1024 - 1, 2 * 1024 * 1024,
+        2 * 1024 * 1024 + 1, 8 * 1024 * 1024};
+    final int[] preallocated = new int[]{512, 1024, 1025, 1024 * 1024,
+        1024 * 1024 + 1, 2 * 1024 * 1024};
+
+    // make sure preallocation is correct with different max/pre-allocated size
+    for (int max : maxSizes) {
+      for (int a : preallocated) {
+        try (LogOutputStream ignored =
+                 new LogOutputStream(file, false, getProperties(max, a))) {
+          Assert.assertEquals(file.length(), Math.min(max, a));
+        }
+        try (LogInputStream in =
+                 new LogInputStream(file, 0, INVALID_LOG_INDEX, true)) {
+          LogEntryProto entry = in.nextEntry();
+          Assert.assertNull(entry);
+        }
+      }
+    }
+
+    // test the scenario where an entry's size is larger than the max size
+    final byte[] content = new byte[1024 * 2];
+    Arrays.fill(content, (byte) 1);
+    final long size;
+    try (LogOutputStream out = new LogOutputStream(file, false,
+        getProperties(1024, 1024))) {
+      SimpleOperation op = new SimpleOperation(new String(content));
+      LogEntryProto entry = ProtoUtils.toLogEntryProto(op.getLogEntryContent(),
+          0, 0);
+      size = LogSegment.getEntrySize(entry);
+      out.write(entry);
+    }
+    Assert.assertEquals(file.length(),
+        size + SegmentedRaftLog.HEADER_BYTES.length);
+    try (LogInputStream in = new LogInputStream(file, 0,
+        INVALID_LOG_INDEX, true)) {
+      LogEntryProto entry = in.nextEntry();
+      Assert.assertArrayEquals(content,
+          entry.getSmLogEntry().getData().toByteArray());
+      Assert.assertNull(in.nextEntry());
+    }
+  }
+
+  /**
+   * Keep appending and check if pre-allocation is correct
+   */
+  @Test
+  public void testPreallocationAndAppend() throws Exception {
+    final long max = 2 * 1024 * 1024;
+    properties.setLong(RAFT_LOG_SEGMENT_MAX_SIZE_KEY, max);
+    properties.setLong(RAFT_LOG_SEGMENT_PREALLOCATED_SIZE_KEY, 16 * 1024);
+    properties.setLong(RAFT_LOG_WRITE_BUFFER_SIZE_KEY, 10 * 1024);
+    RaftStorage storage = new RaftStorage(properties, StartupOption.REGULAR);
+    final File file = storage.getStorageDir().getOpenLogFile(0);
+
+    final byte[] content = new byte[1024];
+    Arrays.fill(content, (byte) 1);
+    SimpleOperation op = new SimpleOperation(new String(content));
+    LogEntryProto entry = ProtoUtils.toLogEntryProto(op.getLogEntryContent(),
+        0, 0);
+    final long entrySize = LogSegment.getEntrySize(entry);
+
+    long totalSize = SegmentedRaftLog.HEADER_BYTES.length;
+    long preallocated = 16 * 1024;
+    try (LogOutputStream out = new LogOutputStream(file, false, properties)) {
+      Assert.assertEquals(preallocated, file.length());
+      while (totalSize + entrySize < max) {
+        totalSize += entrySize;
+        out.write(entry);
+        if (totalSize > preallocated) {
+          Assert.assertEquals("totalSize==" + totalSize,
+              preallocated + 16 * 1024, file.length());
+          preallocated += 16 * 1024;
+        }
+      }
+    }
+
+    Assert.assertEquals(totalSize, file.length());
+  }
+}

http://git-wip-us.apache.org/repos/asf/incubator-ratis/blob/7e71a2e0/ratis-server/src/test/java/org/apache/ratis/server/storage/TestRaftStorage.java
----------------------------------------------------------------------
diff --git a/ratis-server/src/test/java/org/apache/ratis/server/storage/TestRaftStorage.java b/ratis-server/src/test/java/org/apache/ratis/server/storage/TestRaftStorage.java
new file mode 100644
index 0000000..a51e933
--- /dev/null
+++ b/ratis-server/src/test/java/org/apache/ratis/server/storage/TestRaftStorage.java
@@ -0,0 +1,215 @@
+/**
+ * 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.ratis.server.storage;
+
+import org.apache.ratis.RaftTestUtil;
+import org.apache.ratis.conf.RaftProperties;
+import org.apache.ratis.io.nativeio.NativeIO;
+import org.apache.ratis.server.RaftServerConfigKeys;
+import org.apache.ratis.server.impl.RaftServerConstants.StartupOption;
+import org.apache.ratis.server.protocol.TermIndex;
+import org.apache.ratis.server.storage.MetaFile;
+import org.apache.ratis.server.storage.RaftStorage;
+import org.apache.ratis.server.storage.RaftStorageDirectory;
+import org.apache.ratis.server.storage.RaftStorageDirectory.StorageState;
+import org.apache.ratis.statemachine.SimpleStateMachineStorage;
+import org.apache.ratis.util.FileUtils;
+import org.junit.After;
+import org.junit.Assert;
+import org.junit.Before;
+import org.junit.Test;
+import org.mockito.internal.util.reflection.Whitebox;
+
+import java.io.File;
+import java.io.IOException;
+import java.util.concurrent.ThreadLocalRandom;
+
+/**
+ * Test RaftStorage and RaftStorageDirectory
+ */
+public class TestRaftStorage {
+  private File storageDir;
+  private final RaftProperties properties = new RaftProperties();
+
+  @Before
+  public void setup() throws Exception {
+    storageDir = RaftTestUtil.getTestDir(TestRaftStorage.class);
+    properties.set(RaftServerConfigKeys.RAFT_SERVER_STORAGE_DIR_KEY,
+        storageDir.getCanonicalPath());
+  }
+
+  @After
+  public void tearDown() throws Exception {
+    if (storageDir != null) {
+      FileUtils.fullyDelete(storageDir.getParentFile());
+    }
+  }
+
+  @Test
+  public void testNotExistent() throws IOException {
+    FileUtils.fullyDelete(storageDir);
+
+    // we will format the empty directory
+    RaftStorage storage = new RaftStorage(properties, StartupOption.REGULAR);
+    Assert.assertEquals(StorageState.NORMAL, storage.getState());
+
+    try {
+      new RaftStorage(properties, StartupOption.FORMAT).close();
+      Assert.fail("the format should fail since the storage is still locked");
+    } catch (IOException e) {
+      Assert.assertTrue(e.getMessage().contains("directory is already locked"));
+    }
+
+    storage.close();
+    FileUtils.fullyDelete(storageDir);
+    Assert.assertTrue(storageDir.createNewFile());
+    try {
+      new RaftStorage(properties, StartupOption.REGULAR);
+      Assert.fail();
+    } catch (IOException e) {
+      Assert.assertTrue(
+          e.getMessage().contains(StorageState.NON_EXISTENT.name()));
+    }
+  }
+
+  /**
+   * make sure the RaftStorage format works
+   */
+  @Test
+  public void testStorage() throws Exception {
+    RaftStorageDirectory sd = new RaftStorageDirectory(storageDir);
+    try {
+      StorageState state = sd.analyzeStorage(true);
+      Assert.assertEquals(StorageState.NOT_FORMATTED, state);
+      Assert.assertTrue(sd.isCurrentEmpty());
+    } finally {
+      sd.unlock();
+    }
+
+    RaftStorage storage = new RaftStorage(properties, StartupOption.REGULAR);
+    Assert.assertEquals(StorageState.NORMAL, storage.getState());
+    storage.close();
+
+    Assert.assertEquals(StorageState.NORMAL, sd.analyzeStorage(false));
+    File m = sd.getMetaFile();
+    Assert.assertTrue(m.exists());
+    MetaFile metaFile = new MetaFile(m);
+    Assert.assertEquals(MetaFile.DEFAULT_TERM, metaFile.getTerm());
+    Assert.assertEquals(MetaFile.EMPTY_VOTEFOR, metaFile.getVotedFor());
+
+    metaFile.set(123, "peer1");
+    metaFile.readFile();
+    Assert.assertEquals(123, metaFile.getTerm());
+    Assert.assertEquals("peer1", metaFile.getVotedFor());
+
+    MetaFile metaFile2 = new MetaFile(m);
+    Assert.assertFalse((Boolean) Whitebox.getInternalState(metaFile2, "loaded"));
+    Assert.assertEquals(123, metaFile.getTerm());
+    Assert.assertEquals("peer1", metaFile.getVotedFor());
+
+    // test format
+    storage = new RaftStorage(properties, StartupOption.FORMAT);
+    Assert.assertEquals(StorageState.NORMAL, storage.getState());
+    metaFile = new MetaFile(sd.getMetaFile());
+    Assert.assertEquals(MetaFile.DEFAULT_TERM, metaFile.getTerm());
+    Assert.assertEquals(MetaFile.EMPTY_VOTEFOR, metaFile.getVotedFor());
+    storage.close();
+  }
+
+  @Test
+  public void testMetaFile() throws Exception {
+    RaftStorage storage = new RaftStorage(properties, StartupOption.FORMAT);
+    File m = storage.getStorageDir().getMetaFile();
+    Assert.assertTrue(m.exists());
+    MetaFile metaFile = new MetaFile(m);
+    Assert.assertEquals(MetaFile.DEFAULT_TERM, metaFile.getTerm());
+    Assert.assertEquals(MetaFile.EMPTY_VOTEFOR, metaFile.getVotedFor());
+
+    metaFile.set(123, "peer1");
+    metaFile.readFile();
+    Assert.assertEquals(123, metaFile.getTerm());
+    Assert.assertEquals("peer1", metaFile.getVotedFor());
+
+    MetaFile metaFile2 = new MetaFile(m);
+    Assert.assertFalse((Boolean) Whitebox.getInternalState(metaFile2, "loaded"));
+    Assert.assertEquals(123, metaFile.getTerm());
+    Assert.assertEquals("peer1", metaFile.getVotedFor());
+
+    storage.close();
+  }
+
+  /**
+   * check if RaftStorage deletes tmp metafile when startup
+   */
+  @Test
+  public void testCleanMetaTmpFile() throws Exception {
+    RaftStorage storage = new RaftStorage(properties, StartupOption.REGULAR);
+    Assert.assertEquals(StorageState.NORMAL, storage.getState());
+    storage.close();
+
+    RaftStorageDirectory sd = new RaftStorageDirectory(storageDir);
+    File metaFile = sd.getMetaFile();
+    NativeIO.renameTo(metaFile, sd.getMetaTmpFile());
+
+    Assert.assertEquals(StorageState.NOT_FORMATTED, sd.analyzeStorage(false));
+
+    try {
+      new RaftStorage(properties, StartupOption.REGULAR);
+      Assert.fail("should throw IOException since storage dir is not formatted");
+    } catch (IOException e) {
+      Assert.assertTrue(
+          e.getMessage().contains(StorageState.NOT_FORMATTED.name()));
+    }
+
+    // let the storage dir contain both raft-meta and raft-meta.tmp
+    new RaftStorage(properties, StartupOption.FORMAT).close();
+    Assert.assertTrue(sd.getMetaFile().exists());
+    Assert.assertTrue(sd.getMetaTmpFile().createNewFile());
+    Assert.assertTrue(sd.getMetaTmpFile().exists());
+    try {
+      storage = new RaftStorage(properties, StartupOption.REGULAR);
+      Assert.assertEquals(StorageState.NORMAL, storage.getState());
+      Assert.assertFalse(sd.getMetaTmpFile().exists());
+      Assert.assertTrue(sd.getMetaFile().exists());
+    } finally {
+      storage.close();
+    }
+  }
+
+  @Test
+  public void testSnapshotFileName() throws Exception {
+    final long term = ThreadLocalRandom.current().nextLong(Long.MAX_VALUE);
+    final long index = ThreadLocalRandom.current().nextLong(Long.MAX_VALUE);
+    final String name = SimpleStateMachineStorage.getSnapshotFileName(term, index);
+    System.out.println("name = " + name);
+    final File file = new File(storageDir, name);
+    final TermIndex ti = SimpleStateMachineStorage.getTermIndexFromSnapshotFile(file);
+    System.out.println("file = " + file);
+    Assert.assertEquals(term, ti.getTerm());
+    Assert.assertEquals(index, ti.getIndex());
+    System.out.println("ti = " + ti);
+
+    final File foo = new File(storageDir, "foo");
+    try {
+      SimpleStateMachineStorage.getTermIndexFromSnapshotFile(foo);
+      Assert.fail();
+    } catch(IllegalArgumentException iae) {
+      System.out.println("Good " + iae);
+    }
+  }
+}

http://git-wip-us.apache.org/repos/asf/incubator-ratis/blob/7e71a2e0/ratis-server/src/test/java/org/apache/ratis/server/storage/TestSegmentedRaftLog.java
----------------------------------------------------------------------
diff --git a/ratis-server/src/test/java/org/apache/ratis/server/storage/TestSegmentedRaftLog.java b/ratis-server/src/test/java/org/apache/ratis/server/storage/TestSegmentedRaftLog.java
new file mode 100644
index 0000000..405a1a5
--- /dev/null
+++ b/ratis-server/src/test/java/org/apache/ratis/server/storage/TestSegmentedRaftLog.java
@@ -0,0 +1,329 @@
+/**
+ * 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.ratis.server.storage;
+
+import static org.apache.ratis.server.RaftServerConfigKeys.RAFT_LOG_SEGMENT_MAX_SIZE_KEY;
+import static org.apache.ratis.server.RaftServerConfigKeys.RAFT_LOG_SEGMENT_PREALLOCATED_SIZE_KEY;
+
+import java.io.File;
+import java.io.IOException;
+import java.util.ArrayList;
+import java.util.Arrays;
+import java.util.Collections;
+import java.util.List;
+import java.util.function.Supplier;
+
+import org.apache.log4j.Level;
+import org.apache.ratis.MiniRaftCluster;
+import org.apache.ratis.RaftTestUtil;
+import org.apache.ratis.RaftTestUtil.SimpleOperation;
+import org.apache.ratis.conf.RaftProperties;
+import org.apache.ratis.server.RaftServerConfigKeys;
+import org.apache.ratis.server.impl.ConfigurationManager;
+import org.apache.ratis.server.impl.RaftServerConstants;
+import org.apache.ratis.shaded.proto.RaftProtos.LogEntryProto;
+import org.apache.ratis.util.FileUtils;
+import org.apache.ratis.util.ProtoUtils;
+import org.apache.ratis.util.RaftUtils;
+import org.junit.After;
+import org.junit.Assert;
+import org.junit.Before;
+import org.junit.Test;
+
+public class TestSegmentedRaftLog {
+  static {
+    RaftUtils.setLogLevel(RaftLogWorker.LOG, Level.DEBUG);
+  }
+
+  private static final String peerId = "s0";
+
+  private static class SegmentRange {
+    final long start;
+    final long end;
+    final long term;
+    final boolean isOpen;
+
+    SegmentRange(long s, long e, long term, boolean isOpen) {
+      this.start = s;
+      this.end = e;
+      this.term = term;
+      this.isOpen = isOpen;
+    }
+  }
+
+  private File storageDir;
+  private RaftProperties properties;
+  private RaftStorage storage;
+  private final ConfigurationManager cm = new ConfigurationManager(
+      MiniRaftCluster.initConfiguration(MiniRaftCluster.generateIds(3, 0)));
+
+  @Before
+  public void setup() throws Exception {
+    storageDir = RaftTestUtil.getTestDir(TestSegmentedRaftLog.class);
+    properties = new RaftProperties();
+    properties.set(RaftServerConfigKeys.RAFT_SERVER_STORAGE_DIR_KEY,
+        storageDir.getCanonicalPath());
+    storage = new RaftStorage(properties, RaftServerConstants.StartupOption.REGULAR);
+  }
+
+  @After
+  public void tearDown() throws Exception {
+    if (storageDir != null) {
+      FileUtils.fullyDelete(storageDir.getParentFile());
+    }
+  }
+
+  private LogEntryProto[] prepareLog(List<SegmentRange> list) throws IOException {
+    List<LogEntryProto> entryList = new ArrayList<>();
+    for (SegmentRange range : list) {
+      File file = range.isOpen ?
+          storage.getStorageDir().getOpenLogFile(range.start) :
+          storage.getStorageDir().getClosedLogFile(range.start, range.end);
+
+      final int size = (int) (range.end - range.start + 1);
+      LogEntryProto[] entries = new LogEntryProto[size];
+      try (LogOutputStream out = new LogOutputStream(file, false, properties)) {
+        for (int i = 0; i < size; i++) {
+          SimpleOperation m = new SimpleOperation("m" + (i + range.start));
+          entries[i] = ProtoUtils.toLogEntryProto(m.getLogEntryContent(),
+              range.term, i + range.start);
+          out.write(entries[i]);
+        }
+      }
+      Collections.addAll(entryList, entries);
+    }
+    return entryList.toArray(new LogEntryProto[entryList.size()]);
+  }
+
+  private List<SegmentRange> prepareRanges(int number, int segmentSize,
+      long startIndex) {
+    List<SegmentRange> list = new ArrayList<>(number);
+    for (int i = 0; i < number; i++) {
+      list.add(new SegmentRange(startIndex, startIndex + segmentSize - 1, i,
+          i == number - 1));
+      startIndex += segmentSize;
+    }
+    return list;
+  }
+
+  @Test
+  public void testLoadLogSegments() throws Exception {
+    // first generate log files
+    List<SegmentRange> ranges = prepareRanges(5, 100, 0);
+    LogEntryProto[] entries = prepareLog(ranges);
+
+    // create RaftLog object and load log file
+    try (SegmentedRaftLog raftLog =
+             new SegmentedRaftLog(peerId, null, storage, -1, properties)) {
+      raftLog.open(cm, RaftServerConstants.INVALID_LOG_INDEX);
+      // check if log entries are loaded correctly
+      for (LogEntryProto e : entries) {
+        LogEntryProto entry = raftLog.get(e.getIndex());
+        Assert.assertEquals(e, entry);
+      }
+
+      Assert.assertArrayEquals(entries, raftLog.getEntries(0, 500));
+      Assert.assertEquals(entries[entries.length - 1], raftLog.getLastEntry());
+    }
+  }
+
+  List<LogEntryProto> prepareLogEntries(List<SegmentRange> slist,
+      Supplier<String> stringSupplier) {
+    List<LogEntryProto> eList = new ArrayList<>();
+    for (SegmentRange range : slist) {
+      for (long index = range.start; index <= range.end; index++) {
+        SimpleOperation m = stringSupplier == null ?
+            new SimpleOperation("m" + index) :
+            new SimpleOperation(stringSupplier.get());
+        eList.add(ProtoUtils.toLogEntryProto(m.getLogEntryContent(),
+            range.term, index));
+      }
+    }
+    return eList;
+  }
+
+  /**
+   * Append entry one by one and check if log state is correct.
+   */
+  @Test
+  public void testAppendEntry() throws Exception {
+    List<SegmentRange> ranges = prepareRanges(5, 200, 0);
+    List<LogEntryProto> entries = prepareLogEntries(ranges, null);
+
+    try (SegmentedRaftLog raftLog =
+             new SegmentedRaftLog(peerId, null, storage, -1, properties)) {
+      raftLog.open(cm, RaftServerConstants.INVALID_LOG_INDEX);
+      // append entries to the raftlog
+      entries.forEach(raftLog::appendEntry);
+      raftLog.logSync();
+    }
+
+    try (SegmentedRaftLog raftLog =
+             new SegmentedRaftLog(peerId, null, storage, -1, properties)) {
+      raftLog.open(cm, RaftServerConstants.INVALID_LOG_INDEX);
+      // check if the raft log is correct
+      checkEntries(raftLog, entries, 0, entries.size());
+    }
+  }
+
+  /**
+   * Keep appending entries, make sure the rolling is correct.
+   */
+  @Test
+  public void testAppendAndRoll() throws Exception {
+    properties.setLong(RAFT_LOG_SEGMENT_PREALLOCATED_SIZE_KEY, 16 * 1024);
+    properties.setLong(RAFT_LOG_SEGMENT_MAX_SIZE_KEY, 128 * 1024);
+
+    List<SegmentRange> ranges = prepareRanges(1, 1024, 0);
+    final byte[] content = new byte[1024];
+    List<LogEntryProto> entries = prepareLogEntries(ranges,
+        () -> new String(content));
+
+    try (SegmentedRaftLog raftLog =
+             new SegmentedRaftLog(peerId, null, storage, -1, properties)) {
+      raftLog.open(cm, RaftServerConstants.INVALID_LOG_INDEX);
+      // append entries to the raftlog
+      entries.forEach(raftLog::appendEntry);
+      raftLog.logSync();
+    }
+
+    try (SegmentedRaftLog raftLog =
+             new SegmentedRaftLog(peerId, null, storage, -1, properties)) {
+      raftLog.open(cm, RaftServerConstants.INVALID_LOG_INDEX);
+      // check if the raft log is correct
+      checkEntries(raftLog, entries, 0, entries.size());
+      Assert.assertEquals(9, raftLog.getRaftLogCache().getNumOfSegments());
+    }
+  }
+
+  @Test
+  public void testTruncate() throws Exception {
+    // prepare the log for truncation
+    List<SegmentRange> ranges = prepareRanges(5, 200, 0);
+    List<LogEntryProto> entries = prepareLogEntries(ranges, null);
+
+    try (SegmentedRaftLog raftLog =
+             new SegmentedRaftLog(peerId, null, storage, -1, properties)) {
+      raftLog.open(cm, RaftServerConstants.INVALID_LOG_INDEX);
+      // append entries to the raftlog
+      entries.forEach(raftLog::appendEntry);
+      raftLog.logSync();
+    }
+
+    for (long fromIndex = 900; fromIndex >= 0; fromIndex -= 150) {
+      testTruncate(entries, fromIndex);
+    }
+  }
+
+  private void testTruncate(List<LogEntryProto> entries, long fromIndex)
+      throws Exception {
+    try (SegmentedRaftLog raftLog =
+             new SegmentedRaftLog(peerId, null, storage, -1, properties)) {
+      raftLog.open(cm, RaftServerConstants.INVALID_LOG_INDEX);
+      // truncate the log
+      raftLog.truncate(fromIndex);
+      raftLog.logSync();
+
+      checkEntries(raftLog, entries, 0, (int) fromIndex);
+    }
+
+    try (SegmentedRaftLog raftLog =
+             new SegmentedRaftLog(peerId, null, storage, -1, properties)) {
+      raftLog.open(cm, RaftServerConstants.INVALID_LOG_INDEX);
+      // check if the raft log is correct
+      if (fromIndex > 0) {
+        Assert.assertEquals(entries.get((int) (fromIndex - 1)),
+            raftLog.getLastEntry());
+      } else {
+        Assert.assertNull(raftLog.getLastEntry());
+      }
+      checkEntries(raftLog, entries, 0, (int) fromIndex);
+    }
+  }
+
+  private void checkEntries(RaftLog raftLog, List<LogEntryProto> expected,
+      int offset, int size) {
+    if (size > 0) {
+      for (int i = offset; i < size + offset; i++) {
+        LogEntryProto entry = raftLog.get(expected.get(i).getIndex());
+        Assert.assertEquals(expected.get(i), entry);
+      }
+      LogEntryProto[] entriesFromLog = raftLog.getEntries(
+          expected.get(offset).getIndex(),
+          expected.get(offset + size - 1).getIndex() + 1);
+      LogEntryProto[] expectedArray = expected.subList(offset, offset + size)
+          .toArray(SegmentedRaftLog.EMPTY_LOGENTRY_ARRAY);
+      Assert.assertArrayEquals(expectedArray, entriesFromLog);
+    }
+  }
+
+  /**
+   * Test append with inconsistent entries
+   */
+  @Test
+  public void testAppendEntriesWithInconsistency() throws Exception {
+    // prepare the log for truncation
+    List<SegmentRange> ranges = prepareRanges(5, 200, 0);
+    List<LogEntryProto> entries = prepareLogEntries(ranges, null);
+
+    try (SegmentedRaftLog raftLog =
+             new SegmentedRaftLog(peerId, null, storage, -1, properties)) {
+      raftLog.open(cm, RaftServerConstants.INVALID_LOG_INDEX);
+      // append entries to the raftlog
+      entries.forEach(raftLog::appendEntry);
+      raftLog.logSync();
+    }
+
+    // append entries whose first 100 entries are the same with existing log,
+    // and the next 100 are with different term
+    SegmentRange r1 = new SegmentRange(550, 599, 2, false);
+    SegmentRange r2 = new SegmentRange(600, 649, 3, false);
+    SegmentRange r3 = new SegmentRange(650, 749, 10, false);
+    List<LogEntryProto> newEntries = prepareLogEntries(
+        Arrays.asList(r1, r2, r3), null);
+
+    try (SegmentedRaftLog raftLog =
+             new SegmentedRaftLog(peerId, null, storage, -1, properties)) {
+      raftLog.open(cm, RaftServerConstants.INVALID_LOG_INDEX);
+      raftLog.append(newEntries.toArray(new LogEntryProto[newEntries.size()]));
+      raftLog.logSync();
+
+      checkEntries(raftLog, entries, 0, 650);
+      checkEntries(raftLog, newEntries, 100, 100);
+      Assert.assertEquals(newEntries.get(newEntries.size() - 1),
+          raftLog.getLastEntry());
+      Assert.assertEquals(newEntries.get(newEntries.size() - 1).getIndex(),
+          raftLog.getLatestFlushedIndex());
+    }
+
+    // load the raftlog again and check
+    try (SegmentedRaftLog raftLog =
+             new SegmentedRaftLog(peerId, null, storage, -1, properties)) {
+      raftLog.open(cm, RaftServerConstants.INVALID_LOG_INDEX);
+      checkEntries(raftLog, entries, 0, 650);
+      checkEntries(raftLog, newEntries, 100, 100);
+      Assert.assertEquals(newEntries.get(newEntries.size() - 1),
+          raftLog.getLastEntry());
+      Assert.assertEquals(newEntries.get(newEntries.size() - 1).getIndex(),
+          raftLog.getLatestFlushedIndex());
+
+      RaftLogCache cache = raftLog.getRaftLogCache();
+      Assert.assertEquals(5, cache.getNumOfSegments());
+    }
+  }
+}

http://git-wip-us.apache.org/repos/asf/incubator-ratis/blob/7e71a2e0/ratis-server/src/test/java/org/apache/ratis/statemachine/RaftSnapshotBaseTest.java
----------------------------------------------------------------------
diff --git a/ratis-server/src/test/java/org/apache/ratis/statemachine/RaftSnapshotBaseTest.java b/ratis-server/src/test/java/org/apache/ratis/statemachine/RaftSnapshotBaseTest.java
new file mode 100644
index 0000000..6854b42
--- /dev/null
+++ b/ratis-server/src/test/java/org/apache/ratis/statemachine/RaftSnapshotBaseTest.java
@@ -0,0 +1,214 @@
+/**
+ * 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.ratis.statemachine;
+
+import static org.apache.ratis.server.RaftServerConfigKeys.RAFT_SERVER_AUTO_SNAPSHOT_ENABLED_KEY;
+import static org.apache.ratis.server.RaftServerConfigKeys.RAFT_SERVER_SNAPSHOT_TRIGGER_THRESHOLD_KEY;
+import static org.apache.ratis.server.impl.RaftServerConstants.DEFAULT_SEQNUM;
+
+import java.io.File;
+import java.io.IOException;
+import java.util.ArrayList;
+import java.util.List;
+
+import org.apache.log4j.Level;
+import org.apache.ratis.MiniRaftCluster;
+import org.apache.ratis.RaftTestUtil;
+import org.apache.ratis.RaftTestUtil.SimpleMessage;
+import org.apache.ratis.client.RaftClient;
+import org.apache.ratis.conf.RaftProperties;
+import org.apache.ratis.protocol.RaftClientReply;
+import org.apache.ratis.protocol.SetConfigurationRequest;
+import org.apache.ratis.server.impl.RaftServerImpl;
+import org.apache.ratis.server.impl.RaftServerTestUtil;
+import org.apache.ratis.server.simulation.RequestHandler;
+import org.apache.ratis.server.storage.RaftLog;
+import org.apache.ratis.server.storage.RaftStorageDirectory;
+import org.apache.ratis.server.storage.RaftStorageDirectory.LogPathAndIndex;
+import org.apache.ratis.shaded.proto.RaftProtos.LogEntryProto;
+import org.apache.ratis.util.FileUtils;
+import org.apache.ratis.util.RaftUtils;
+import org.junit.After;
+import org.junit.Assert;
+import org.junit.Before;
+import org.junit.Test;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+public abstract class RaftSnapshotBaseTest {
+  static {
+    RaftUtils.setLogLevel(RaftServerImpl.LOG, Level.DEBUG);
+    RaftUtils.setLogLevel(RaftLog.LOG, Level.DEBUG);
+    RaftUtils.setLogLevel(RequestHandler.LOG, Level.DEBUG);
+    RaftUtils.setLogLevel(RaftClient.LOG, Level.DEBUG);
+  }
+
+  static final Logger LOG = LoggerFactory.getLogger(RaftSnapshotBaseTest.class);
+  private static final int SNAPSHOT_TRIGGER_THRESHOLD = 10;
+
+  static File getSnapshotFile(MiniRaftCluster cluster, int i) {
+    final RaftServerImpl leader = cluster.getLeader();
+    final SimpleStateMachine4Testing sm = SimpleStateMachine4Testing.get(leader);
+    return sm.getStateMachineStorage().getSnapshotFile(
+        leader.getState().getCurrentTerm(), i);
+  }
+
+  static void assertLeaderContent(MiniRaftCluster cluster)
+      throws InterruptedException {
+    final RaftServerImpl leader = RaftTestUtil.waitForLeader(cluster);
+    Assert.assertEquals(SNAPSHOT_TRIGGER_THRESHOLD * 2,
+        leader.getState().getLog().getLastCommittedIndex());
+    final LogEntryProto[] entries = SimpleStateMachine4Testing.get(leader).getContent();
+
+    for (int i = 1; i < SNAPSHOT_TRIGGER_THRESHOLD * 2 - 1; i++) {
+      Assert.assertEquals(i+1, entries[i].getIndex());
+      Assert.assertArrayEquals(
+          new SimpleMessage("m" + i).getContent().toByteArray(),
+          entries[i].getSmLogEntry().getData().toByteArray());
+    }
+  }
+
+  private MiniRaftCluster cluster;
+
+  public abstract MiniRaftCluster initCluster(int numServer, RaftProperties prop)
+      throws IOException;
+
+  @Before
+  public void setup() throws IOException {
+    final RaftProperties prop = new RaftProperties();
+    prop.setClass(MiniRaftCluster.STATEMACHINE_CLASS_KEY,
+        SimpleStateMachine4Testing.class, StateMachine.class);
+    prop.setLong(RAFT_SERVER_SNAPSHOT_TRIGGER_THRESHOLD_KEY,
+        SNAPSHOT_TRIGGER_THRESHOLD);
+    prop.setBoolean(RAFT_SERVER_AUTO_SNAPSHOT_ENABLED_KEY, true);
+    this.cluster = initCluster(1, prop);
+    cluster.start();
+  }
+
+  @After
+  public void tearDown() {
+    if (cluster != null) {
+      cluster.shutdown();
+    }
+  }
+
+  /**
+   * Keep generating writing traffic and make sure snapshots are taken.
+   * We then restart the whole raft peer and check if it can correctly load
+   * snapshots + raft log.
+   */
+  @Test
+  public void testRestartPeer() throws Exception {
+    RaftTestUtil.waitForLeader(cluster);
+    final String leaderId = cluster.getLeader().getId();
+    int i = 0;
+    try(final RaftClient client = cluster.createClient("client", leaderId)) {
+      for (; i < SNAPSHOT_TRIGGER_THRESHOLD * 2 - 1; i++) {
+        RaftClientReply reply = client.send(new SimpleMessage("m" + i));
+        Assert.assertTrue(reply.isSuccess());
+      }
+    }
+
+    // wait for the snapshot to be done
+    final File snapshotFile = getSnapshotFile(cluster, i);
+
+    int retries = 0;
+    do {
+      Thread.sleep(1000);
+    } while (!snapshotFile.exists() && retries++ < 10);
+
+    Assert.assertTrue(snapshotFile + " does not exist", snapshotFile.exists());
+
+    // restart the peer and check if it can correctly load snapshot
+    cluster.restart(false);
+    try {
+      // 200 messages + two leader elections --> last committed = 201
+      assertLeaderContent(cluster);
+    } finally {
+      cluster.shutdown();
+    }
+  }
+
+  /**
+   * Basic test for install snapshot: start a one node cluster and let it
+   * generate a snapshot. Then delete the log and restart the node, and add more
+   * nodes as followers.
+   */
+  @Test
+  public void testBasicInstallSnapshot() throws Exception {
+    List<LogPathAndIndex> logs = new ArrayList<>();
+    try {
+      RaftTestUtil.waitForLeader(cluster);
+      final String leaderId = cluster.getLeader().getId();
+
+      int i = 0;
+      try(final RaftClient client = cluster.createClient("client", leaderId)) {
+        for (; i < SNAPSHOT_TRIGGER_THRESHOLD * 2 - 1; i++) {
+          RaftClientReply reply = client.send(new SimpleMessage("m" + i));
+          Assert.assertTrue(reply.isSuccess());
+        }
+      }
+
+      // wait for the snapshot to be done
+      RaftStorageDirectory storageDirectory = cluster.getLeader().getState()
+          .getStorage().getStorageDir();
+      final File snapshotFile = getSnapshotFile(cluster, i);
+      logs = storageDirectory.getLogSegmentFiles();
+
+      int retries = 0;
+      do {
+        Thread.sleep(1000);
+      } while (!snapshotFile.exists() && retries++ < 10);
+
+      Assert.assertTrue(snapshotFile + " does not exist", snapshotFile.exists());
+    } finally {
+      cluster.shutdown();
+    }
+
+    // delete the log segments from the leader
+    for (LogPathAndIndex path : logs) {
+      FileUtils.deleteFile(path.path.toFile());
+    }
+
+    // restart the peer
+    LOG.info("Restarting the cluster");
+    cluster.restart(false);
+    try {
+      assertLeaderContent(cluster);
+
+      // generate some more traffic
+      try(final RaftClient client = cluster.createClient("client",
+          cluster.getLeader().getId())) {
+        Assert.assertTrue(client.send(new SimpleMessage("test")).isSuccess());
+      }
+
+      // add two more peers
+      MiniRaftCluster.PeerChanges change = cluster.addNewPeers(
+          new String[]{"s3", "s4"}, true);
+      // trigger setConfiguration
+      SetConfigurationRequest request = new SetConfigurationRequest("client",
+          cluster.getLeader().getId(), DEFAULT_SEQNUM, change.allPeersInNewConf);
+      LOG.info("Start changing the configuration: {}", request);
+      cluster.getLeader().setConfiguration(request);
+
+      RaftServerTestUtil.waitAndCheckNewConf(cluster, change.allPeersInNewConf, 0, null);
+    } finally {
+      cluster.shutdown();
+    }
+  }
+}

http://git-wip-us.apache.org/repos/asf/incubator-ratis/blob/7e71a2e0/ratis-server/src/test/java/org/apache/ratis/statemachine/SimpleStateMachine4Testing.java
----------------------------------------------------------------------
diff --git a/ratis-server/src/test/java/org/apache/ratis/statemachine/SimpleStateMachine4Testing.java b/ratis-server/src/test/java/org/apache/ratis/statemachine/SimpleStateMachine4Testing.java
new file mode 100644
index 0000000..cc82371
--- /dev/null
+++ b/ratis-server/src/test/java/org/apache/ratis/statemachine/SimpleStateMachine4Testing.java
@@ -0,0 +1,246 @@
+/**
+ * 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.ratis.statemachine;
+
+import java.io.File;
+import java.io.IOException;
+import java.util.ArrayList;
+import java.util.Collection;
+import java.util.Collections;
+import java.util.List;
+import java.util.concurrent.CompletableFuture;
+
+import org.apache.ratis.RaftTestUtil.SimpleMessage;
+import org.apache.ratis.conf.RaftProperties;
+import org.apache.ratis.io.MD5Hash;
+import org.apache.ratis.protocol.Message;
+import org.apache.ratis.protocol.RaftClientReply;
+import org.apache.ratis.protocol.RaftClientRequest;
+import org.apache.ratis.server.impl.RaftServerConstants;
+import org.apache.ratis.server.impl.RaftServerImpl;
+import org.apache.ratis.server.impl.RaftServerTestUtil;
+import org.apache.ratis.server.impl.ServerProtoUtils;
+import org.apache.ratis.server.protocol.TermIndex;
+import org.apache.ratis.server.storage.LogInputStream;
+import org.apache.ratis.server.storage.LogOutputStream;
+import org.apache.ratis.server.storage.RaftStorage;
+import org.apache.ratis.shaded.proto.RaftProtos.LogEntryProto;
+import org.apache.ratis.shaded.proto.RaftProtos.SMLogEntryProto;
+import org.apache.ratis.util.Daemon;
+import org.apache.ratis.util.LifeCycle;
+import org.apache.ratis.util.MD5FileUtil;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import com.google.common.base.Preconditions;
+
+/**
+ * A {@link StateMachine} implementation example that simply stores all the log
+ * entries in a list. Mainly used for test.
+ *
+ * For snapshot it simply merges all the log segments together.
+ */
+public class SimpleStateMachine4Testing extends BaseStateMachine {
+  static volatile int SNAPSHOT_THRESHOLD = 100;
+  static final Logger LOG = LoggerFactory.getLogger(SimpleStateMachine4Testing.class);
+  public static final String RAFT_TEST_SIMPLE_STATE_MACHINE_TAKE_SNAPSHOT_KEY
+      = "raft.test.simple.state.machine.take.snapshot";
+  public static final boolean RAFT_TEST_SIMPLE_STATE_MACHINE_TAKE_SNAPSHOT_DEFAULT = false;
+
+  public static SimpleStateMachine4Testing get(RaftServerImpl s) {
+    return (SimpleStateMachine4Testing)RaftServerTestUtil.getStateMachine(s);
+  }
+
+  private final List<LogEntryProto> list =
+      Collections.synchronizedList(new ArrayList<>());
+  private final Daemon checkpointer;
+  private final SimpleStateMachineStorage storage = new SimpleStateMachineStorage();
+  private final TermIndexTracker termIndexTracker = new TermIndexTracker();
+  private final RaftProperties properties = new RaftProperties();
+
+  private volatile boolean running = true;
+  private long endIndexLastCkpt = RaftServerConstants.INVALID_LOG_INDEX;
+
+  SimpleStateMachine4Testing() {
+    checkpointer = new Daemon(() -> {
+      while (running) {
+        try {
+          if (list.get(list.size() - 1).getIndex() - endIndexLastCkpt >=
+              SNAPSHOT_THRESHOLD) {
+            endIndexLastCkpt = takeSnapshot();
+          }
+          try {
+            Thread.sleep(1000);
+          } catch (InterruptedException ignored) {
+          }
+        } catch (IOException ioe) {
+          LOG.warn("Received IOException in Checkpointer", ioe);
+        }
+      }
+    });
+  }
+
+  @Override
+  public synchronized void initialize(String id, RaftProperties properties,
+      RaftStorage raftStorage) throws IOException {
+    LOG.info("Initializing " + getClass().getSimpleName() + ":" + id);
+    lifeCycle.startAndTransition(() -> {
+      super.initialize(id, properties, raftStorage);
+      storage.init(raftStorage);
+      loadSnapshot(storage.findLatestSnapshot());
+
+      if (properties.getBoolean(
+          RAFT_TEST_SIMPLE_STATE_MACHINE_TAKE_SNAPSHOT_KEY,
+          RAFT_TEST_SIMPLE_STATE_MACHINE_TAKE_SNAPSHOT_DEFAULT)) {
+        checkpointer.start();
+      }
+    });
+  }
+
+  @Override
+  public synchronized void pause() {
+    lifeCycle.transition(LifeCycle.State.PAUSING);
+    lifeCycle.transition(LifeCycle.State.PAUSED);
+  }
+
+  @Override
+  public synchronized void reinitialize(String id, RaftProperties properties,
+      RaftStorage storage) throws IOException {
+    LOG.info("Reinitializing " + getClass().getSimpleName() + ":" + id);
+    initialize(id, properties, storage);
+  }
+
+  @Override
+  public CompletableFuture<Message> applyTransaction(TransactionContext trx) {
+    LogEntryProto entry = trx.getLogEntry().get();
+    Preconditions.checkNotNull(entry);
+    list.add(entry);
+    termIndexTracker.update(ServerProtoUtils.toTermIndex(entry));
+    return CompletableFuture.completedFuture(
+        new SimpleMessage(entry.getIndex() + " OK"));
+  }
+
+  @Override
+  public long takeSnapshot() throws IOException {
+    TermIndex termIndex = termIndexTracker.getLatestTermIndex();
+    if (termIndex.getTerm() <= 0 || termIndex.getIndex() <= 0) {
+      return RaftServerConstants.INVALID_LOG_INDEX;
+    }
+    final long endIndex = termIndex.getIndex();
+
+    // TODO: snapshot should be written to a tmp file, then renamed
+    File snapshotFile = storage.getSnapshotFile(termIndex.getTerm(),
+        termIndex.getIndex());
+    LOG.debug("Taking a snapshot with t:{}, i:{}, file:{}", termIndex.getTerm(),
+        termIndex.getIndex(), snapshotFile);
+    try (LogOutputStream out = new LogOutputStream(snapshotFile, false, properties)) {
+      for (final LogEntryProto entry : list) {
+        if (entry.getIndex() > endIndex) {
+          break;
+        } else {
+          out.write(entry);
+        }
+      }
+      out.flush();
+    } catch (IOException e) {
+      LOG.warn("Failed to take snapshot", e);
+    }
+
+    try {
+      final MD5Hash digest = MD5FileUtil.computeMd5ForFile(snapshotFile);
+      MD5FileUtil.saveMD5File(snapshotFile, digest);
+    } catch (IOException e) {
+      LOG.warn("Hit IOException when computing MD5 for snapshot file "
+          + snapshotFile, e);
+    }
+
+    try {
+      this.storage.loadLatestSnapshot();
+    } catch (IOException e) {
+      LOG.warn("Hit IOException when loading latest snapshot for snapshot file "
+          + snapshotFile, e);
+    }
+    // TODO: purge log segments
+    return endIndex;
+  }
+
+  @Override
+  public SimpleStateMachineStorage getStateMachineStorage() {
+    return storage;
+  }
+
+  public synchronized long loadSnapshot(SingleFileSnapshotInfo snapshot)
+      throws IOException {
+    if (snapshot == null || !snapshot.getFile().getPath().toFile().exists()) {
+      LOG.info("The snapshot file {} does not exist",
+          snapshot == null ? null : snapshot.getFile());
+      return RaftServerConstants.INVALID_LOG_INDEX;
+    } else {
+      LOG.info("Loading snapshot with t:{}, i:{}, file:{}", snapshot.getTerm(),
+          snapshot.getIndex(), snapshot.getFile().getPath());
+      final long endIndex = snapshot.getIndex();
+      try (LogInputStream in = new LogInputStream(
+          snapshot.getFile().getPath().toFile(), 0, endIndex, false)) {
+        LogEntryProto entry;
+        while ((entry = in.nextEntry()) != null) {
+          list.add(entry);
+          termIndexTracker.update(ServerProtoUtils.toTermIndex(entry));
+        }
+      }
+      Preconditions.checkState(
+          !list.isEmpty() && endIndex == list.get(list.size() - 1).getIndex(),
+          "endIndex=%s, list=%s", endIndex, list);
+      this.endIndexLastCkpt = endIndex;
+      termIndexTracker.init(snapshot.getTermIndex());
+      this.storage.loadLatestSnapshot();
+      return endIndex;
+    }
+  }
+
+  @Override
+  public CompletableFuture<RaftClientReply> query(
+      RaftClientRequest request) {
+    return CompletableFuture.completedFuture(
+        new RaftClientReply(request, new SimpleMessage("query success")));
+  }
+
+  @Override
+  public TransactionContext startTransaction(RaftClientRequest request)
+      throws IOException {
+    return new TransactionContext(this, request, SMLogEntryProto.newBuilder()
+        .setData(request.getMessage().getContent())
+        .build());
+  }
+
+  @Override
+  public void notifyNotLeader(Collection<TransactionContext> pendingEntries) {
+    // do nothing
+  }
+
+  @Override
+  public void close() {
+    lifeCycle.checkStateAndClose(() -> {
+      running = false;
+      checkpointer.interrupt();
+    });
+  }
+
+  public LogEntryProto[] getContent() {
+    return list.toArray(new LogEntryProto[list.size()]);
+  }
+}

http://git-wip-us.apache.org/repos/asf/incubator-ratis/blob/7e71a2e0/ratis-server/src/test/java/org/apache/ratis/statemachine/TermIndexTracker.java
----------------------------------------------------------------------
diff --git a/ratis-server/src/test/java/org/apache/ratis/statemachine/TermIndexTracker.java b/ratis-server/src/test/java/org/apache/ratis/statemachine/TermIndexTracker.java
new file mode 100644
index 0000000..31768e8
--- /dev/null
+++ b/ratis-server/src/test/java/org/apache/ratis/statemachine/TermIndexTracker.java
@@ -0,0 +1,67 @@
+/**
+ * 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.ratis.statemachine;
+
+import static org.apache.ratis.server.impl.RaftServerConstants.INVALID_LOG_INDEX;
+
+import org.apache.ratis.server.protocol.TermIndex;
+
+import com.google.common.base.Preconditions;
+
+/**
+ * Tracks the term index that is applied to the StateMachine for simple state machines with
+ * no concurrent snapshoting capabilities.
+ */
+class TermIndexTracker {
+  static final TermIndex INIT_TERMINDEX =
+      TermIndex.newTermIndex(INVALID_LOG_INDEX, INVALID_LOG_INDEX);
+
+  private TermIndex current = INIT_TERMINDEX;
+
+  //TODO: developer note: everything is synchronized for now for convenience.
+
+  /**
+   * Initialize the tracker with a term index (likely from a snapshot).
+   */
+  public synchronized void init(TermIndex termIndex) {
+    this.current = termIndex;
+  }
+
+  public synchronized void reset() {
+    init(INIT_TERMINDEX);
+  }
+
+  /**
+   * Update the tracker with a new TermIndex. It means that the StateMachine has
+   * this index in memory.
+   */
+  public synchronized void update(TermIndex termIndex) {
+    Preconditions.checkArgument(termIndex != null &&
+        termIndex.compareTo(current) >= 0);
+    this.current = termIndex;
+  }
+
+  /**
+   * Return latest term and index that is inserted to this tracker as an atomic
+   * entity.
+   */
+  public synchronized TermIndex getLatestTermIndex() {
+    return current;
+  }
+
+}

http://git-wip-us.apache.org/repos/asf/incubator-ratis/blob/7e71a2e0/ratis-server/src/test/java/org/apache/ratis/statemachine/TestStateMachine.java
----------------------------------------------------------------------
diff --git a/ratis-server/src/test/java/org/apache/ratis/statemachine/TestStateMachine.java b/ratis-server/src/test/java/org/apache/ratis/statemachine/TestStateMachine.java
new file mode 100644
index 0000000..cdce568
--- /dev/null
+++ b/ratis-server/src/test/java/org/apache/ratis/statemachine/TestStateMachine.java
@@ -0,0 +1,192 @@
+/**
+ * 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.ratis.statemachine;
+
+import static org.junit.Assert.assertEquals;
+import static org.junit.Assert.assertFalse;
+import static org.junit.Assert.assertTrue;
+
+import java.io.IOException;
+import java.util.Collections;
+import java.util.List;
+import java.util.Optional;
+import java.util.concurrent.CompletableFuture;
+import java.util.concurrent.ConcurrentLinkedQueue;
+import java.util.concurrent.atomic.AtomicBoolean;
+import java.util.concurrent.atomic.AtomicLong;
+import java.util.concurrent.atomic.AtomicReference;
+import java.util.stream.Collectors;
+
+import org.apache.log4j.Level;
+import org.apache.ratis.MiniRaftCluster;
+import org.apache.ratis.RaftTestUtil;
+import org.apache.ratis.client.RaftClient;
+import org.apache.ratis.conf.RaftProperties;
+import org.apache.ratis.protocol.Message;
+import org.apache.ratis.protocol.RaftClientRequest;
+import org.apache.ratis.server.RaftServerConfigKeys;
+import org.apache.ratis.server.impl.RaftServerImpl;
+import org.apache.ratis.server.impl.RaftServerTestUtil;
+import org.apache.ratis.server.simulation.MiniRaftClusterWithSimulatedRpc;
+import org.apache.ratis.shaded.proto.RaftProtos.SMLogEntryProto;
+import org.apache.ratis.util.RaftUtils;
+import org.junit.After;
+import org.junit.Assert;
+import org.junit.Before;
+import org.junit.Rule;
+import org.junit.Test;
+import org.junit.rules.Timeout;
+
+/**
+ * Test StateMachine related functionality
+ */
+public class TestStateMachine {
+  static {
+    RaftUtils.setLogLevel(RaftServerImpl.LOG, Level.DEBUG);
+    RaftUtils.setLogLevel(RaftClient.LOG, Level.DEBUG);
+  }
+
+  public static final int NUM_SERVERS = 5;
+
+  private final RaftProperties properties = new RaftProperties();
+  {
+    // TODO: fix and run with in-memory log. It fails with NPE
+    properties.setBoolean(RaftServerConfigKeys.RAFT_SERVER_USE_MEMORY_LOG_KEY, false);
+  }
+
+  private MiniRaftClusterWithSimulatedRpc cluster;
+
+  @Rule
+  public Timeout globalTimeout = new Timeout(60 * 1000);
+
+  @Before
+  public void setup() throws IOException {
+  }
+
+  private void startCluster() {
+    cluster = new MiniRaftClusterWithSimulatedRpc(NUM_SERVERS, properties);
+    Assert.assertNull(getCluster().getLeader());
+    getCluster().start();
+  }
+
+  @After
+  public void tearDown() {
+    final MiniRaftCluster cluster = getCluster();
+    if (cluster != null) {
+      cluster.shutdown();
+    }
+  }
+
+  public MiniRaftClusterWithSimulatedRpc getCluster() {
+    return cluster;
+  }
+
+  public RaftProperties getProperties() {
+    return properties;
+  }
+
+  static class SMTransactionContext extends SimpleStateMachine4Testing {
+    public static SMTransactionContext get(RaftServerImpl s) {
+      return (SMTransactionContext)RaftServerTestUtil.getStateMachine(s);
+    }
+
+    AtomicReference<Throwable> throwable = new AtomicReference<>(null);
+    AtomicLong transactions = new AtomicLong(0);
+    AtomicBoolean isLeader = new AtomicBoolean(false);
+    AtomicLong numApplied = new AtomicLong(0);
+    ConcurrentLinkedQueue<Long> applied = new ConcurrentLinkedQueue<>();
+
+    @Override
+    public TransactionContext startTransaction(RaftClientRequest request) throws IOException {
+      // only leader will get this call
+      isLeader.set(true);
+      // send the next transaction id as the "context" from SM
+      return new TransactionContext(this, request, SMLogEntryProto.newBuilder()
+          .setData(request.getMessage().getContent())
+          .build(), transactions.incrementAndGet());
+    }
+
+    @Override
+    public CompletableFuture<Message> applyTransaction(TransactionContext trx) {
+      try {
+        assertTrue(trx.getLogEntry().isPresent());
+        assertTrue(trx.getSMLogEntry().isPresent());
+        Optional<Object> context = trx.getStateMachineContext();
+        if (isLeader.get()) {
+          assertTrue(trx.getClientRequest().isPresent());
+          assertTrue(context.isPresent());
+          assertTrue(context.get() instanceof Long);
+          Long val = (Long)context.get();
+          assertTrue(val <= transactions.get());
+          applied.add(val);
+        } else {
+          assertFalse(trx.getClientRequest().isPresent());
+          assertFalse(context.isPresent());
+        }
+        numApplied.incrementAndGet();
+      } catch (Throwable t) {
+        throwable.set(t);
+      }
+      return CompletableFuture.completedFuture(null);
+    }
+
+    void rethrowIfException() throws Throwable {
+      Throwable t = throwable.get();
+      if (t != null) {
+        throw t;
+      }
+    }
+  }
+
+  @Test
+  public void testTransactionContextIsPassedBack() throws Throwable {
+    // tests that the TrxContext set by the StateMachine in Leader is passed back to the SM
+    properties.setClass(
+        MiniRaftCluster.STATEMACHINE_CLASS_KEY,
+        SMTransactionContext.class, StateMachine.class);
+    startCluster();
+
+    int numTrx = 100;
+    final RaftTestUtil.SimpleMessage[] messages = RaftTestUtil.SimpleMessage.create(numTrx);
+    try(final RaftClient client = cluster.createClient("client", null)) {
+      for (RaftTestUtil.SimpleMessage message : messages) {
+        client.send(message);
+      }
+    }
+
+    // TODO: there eshould be a better way to ensure all data is replicated and applied
+    Thread.sleep(cluster.getMaxTimeout() + 100);
+
+    for (RaftServerImpl raftServer : cluster.getServers()) {
+      final SMTransactionContext sm = SMTransactionContext.get(raftServer);
+      sm.rethrowIfException();
+      assertEquals(numTrx, sm.numApplied.get());
+    }
+
+    // check leader
+    RaftServerImpl raftServer = cluster.getLeader();
+    // assert every transaction has obtained context in leader
+    final SMTransactionContext sm = SMTransactionContext.get(raftServer);
+    List<Long> ll = sm.applied.stream().collect(Collectors.toList());
+    Collections.sort(ll);
+    assertEquals(ll.toString(), ll.size(), numTrx);
+    for (int i=0; i < numTrx; i++) {
+      assertEquals(ll.toString(), Long.valueOf(i+1), ll.get(i));
+    }
+  }
+}

http://git-wip-us.apache.org/repos/asf/incubator-ratis/blob/7e71a2e0/ratis-server/src/test/resources/log4j.properties
----------------------------------------------------------------------
diff --git a/ratis-server/src/test/resources/log4j.properties b/ratis-server/src/test/resources/log4j.properties
new file mode 100644
index 0000000..ced0687
--- /dev/null
+++ b/ratis-server/src/test/resources/log4j.properties
@@ -0,0 +1,18 @@
+#   Licensed 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.
+# log4j configuration used during build and unit tests
+
+log4j.rootLogger=info,stdout
+log4j.threshold=ALL
+log4j.appender.stdout=org.apache.log4j.ConsoleAppender
+log4j.appender.stdout.layout=org.apache.log4j.PatternLayout
+log4j.appender.stdout.layout.ConversionPattern=%d{ISO8601} %-5p %c{2} (%F:%M(%L)) - %m%n


[12/54] [abbrv] incubator-ratis git commit: Move TermIndexTracker to test.

Posted by ji...@apache.org.
Move TermIndexTracker to test.


Project: http://git-wip-us.apache.org/repos/asf/incubator-ratis/repo
Commit: http://git-wip-us.apache.org/repos/asf/incubator-ratis/commit/8dbb64fb
Tree: http://git-wip-us.apache.org/repos/asf/incubator-ratis/tree/8dbb64fb
Diff: http://git-wip-us.apache.org/repos/asf/incubator-ratis/diff/8dbb64fb

Branch: refs/heads/master
Commit: 8dbb64fbae8f85170b4e1494cc9b1d4fcd98b513
Parents: eaadf8e
Author: Tsz-Wo Nicholas Sze <sz...@hortonworks.com>
Authored: Thu Jan 5 20:38:48 2017 +0800
Committer: Tsz-Wo Nicholas Sze <sz...@hortonworks.com>
Committed: Thu Jan 5 20:38:48 2017 +0800

----------------------------------------------------------------------
 .../arithmetic/ArithmeticStateMachine.java      | 20 ++++--
 .../raft/statemachine/TermIndexTracker.java     | 66 --------------------
 .../SimpleStateMachine4Testing.java             | 11 ++--
 .../raft/statemachine/TermIndexTracker.java     | 66 ++++++++++++++++++++
 4 files changed, 86 insertions(+), 77 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-ratis/blob/8dbb64fb/raft-examples/src/main/java/org/apache/raft/examples/arithmetic/ArithmeticStateMachine.java
----------------------------------------------------------------------
diff --git a/raft-examples/src/main/java/org/apache/raft/examples/arithmetic/ArithmeticStateMachine.java b/raft-examples/src/main/java/org/apache/raft/examples/arithmetic/ArithmeticStateMachine.java
index b684669..91776eb 100644
--- a/raft-examples/src/main/java/org/apache/raft/examples/arithmetic/ArithmeticStateMachine.java
+++ b/raft-examples/src/main/java/org/apache/raft/examples/arithmetic/ArithmeticStateMachine.java
@@ -17,6 +17,7 @@
  */
 package org.apache.raft.examples.arithmetic;
 
+import com.google.common.base.Preconditions;
 import org.apache.raft.conf.RaftProperties;
 import org.apache.raft.examples.arithmetic.expression.Expression;
 import org.apache.raft.protocol.Message;
@@ -37,6 +38,7 @@ import java.util.HashMap;
 import java.util.Map;
 import java.util.concurrent.CompletableFuture;
 import java.util.concurrent.ConcurrentHashMap;
+import java.util.concurrent.atomic.AtomicReference;
 import java.util.concurrent.locks.ReentrantReadWriteLock;
 
 public class ArithmeticStateMachine extends BaseStateMachine {
@@ -45,7 +47,7 @@ public class ArithmeticStateMachine extends BaseStateMachine {
   private final Map<String, Double> variables = new ConcurrentHashMap<>();
 
   private final SimpleStateMachineStorage storage = new SimpleStateMachineStorage();
-  private final TermIndexTracker termIndexTracker = new TermIndexTracker();
+  private final AtomicReference<TermIndex> latestTermIndex = new AtomicReference<>();
 
   private final ReentrantReadWriteLock lock = new ReentrantReadWriteLock(true);
 
@@ -59,7 +61,7 @@ public class ArithmeticStateMachine extends BaseStateMachine {
 
   void reset() {
     variables.clear();
-    termIndexTracker.reset();
+    latestTermIndex.set(null);
   }
 
   @Override
@@ -83,7 +85,7 @@ public class ArithmeticStateMachine extends BaseStateMachine {
     final TermIndex last;
     try(final AutoCloseableLock readLock = readLock()) {
       copy = new HashMap<>(variables);
-      last = termIndexTracker.getLatestTermIndex();
+      last = latestTermIndex.get();
     }
 
     File snapshotFile =  new File(SimpleStateMachineStorage.getSnapshotFileName(
@@ -118,7 +120,7 @@ public class ArithmeticStateMachine extends BaseStateMachine {
       if (reload) {
         reset();
       }
-      termIndexTracker.init(last);
+      latestTermIndex.set(last);
       variables.putAll((Map<String, Double>) in.readObject());
     } catch (ClassNotFoundException e) {
       throw new IllegalStateException(e);
@@ -162,10 +164,18 @@ public class ArithmeticStateMachine extends BaseStateMachine {
     final Double result;
     try(final AutoCloseableLock writeLock = writeLock()) {
       result = assignment.evaluate(variables);
-      termIndexTracker.update(new TermIndex(entry.getTerm(), index));
+      updateLatestTermIndex(entry.getTerm(), index);
     }
     final Expression r = Expression.Utils.double2Expression(result);
     LOG.debug("{}: {} = {}, variables={}", index, assignment, r, variables);
     return CompletableFuture.completedFuture(Expression.Utils.toMessage(r));
   }
+
+  private void updateLatestTermIndex(long term, long index) {
+    final TermIndex newTI = new TermIndex(term, index);
+    final TermIndex oldTI = latestTermIndex.getAndSet(newTI);
+    if (oldTI != null) {
+      Preconditions.checkArgument(newTI.compareTo(oldTI) >= 0);
+    }
+  }
 }

http://git-wip-us.apache.org/repos/asf/incubator-ratis/blob/8dbb64fb/raft-server/src/main/java/org/apache/raft/statemachine/TermIndexTracker.java
----------------------------------------------------------------------
diff --git a/raft-server/src/main/java/org/apache/raft/statemachine/TermIndexTracker.java b/raft-server/src/main/java/org/apache/raft/statemachine/TermIndexTracker.java
deleted file mode 100644
index 694eef4..0000000
--- a/raft-server/src/main/java/org/apache/raft/statemachine/TermIndexTracker.java
+++ /dev/null
@@ -1,66 +0,0 @@
-/**
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- *     http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-package org.apache.raft.statemachine;
-
-import com.google.common.base.Preconditions;
-import org.apache.raft.server.protocol.TermIndex;
-
-import static org.apache.raft.server.impl.RaftServerConstants.INVALID_LOG_INDEX;
-
-/**
- * Tracks the term index that is applied to the StateMachine for simple state machines with
- * no concurrent snapshoting capabilities.
- */
-public class TermIndexTracker {
-  static final TermIndex INIT_TERMINDEX =
-      new TermIndex(INVALID_LOG_INDEX, INVALID_LOG_INDEX);
-
-  private TermIndex current = INIT_TERMINDEX;
-
-  //TODO: developer note: everything is synchronized for now for convenience.
-
-  /**
-   * Initialize the tracker with a term index (likely from a snapshot).
-   */
-  public synchronized void init(TermIndex termIndex) {
-    this.current = termIndex;
-  }
-
-  public synchronized void reset() {
-    init(INIT_TERMINDEX);
-  }
-
-  /**
-   * Update the tracker with a new TermIndex. It means that the StateMachine has
-   * this index in memory.
-   */
-  public synchronized void update(TermIndex termIndex) {
-    Preconditions.checkArgument(termIndex != null &&
-        termIndex.compareTo(current) >= 0);
-    this.current = termIndex;
-  }
-
-  /**
-   * Return latest term and index that is inserted to this tracker as an atomic
-   * entity.
-   */
-  public synchronized TermIndex getLatestTermIndex() {
-    return current;
-  }
-
-}

http://git-wip-us.apache.org/repos/asf/incubator-ratis/blob/8dbb64fb/raft-server/src/test/java/org/apache/raft/statemachine/SimpleStateMachine4Testing.java
----------------------------------------------------------------------
diff --git a/raft-server/src/test/java/org/apache/raft/statemachine/SimpleStateMachine4Testing.java b/raft-server/src/test/java/org/apache/raft/statemachine/SimpleStateMachine4Testing.java
index 3e0ae15..d11bfd4 100644
--- a/raft-server/src/test/java/org/apache/raft/statemachine/SimpleStateMachine4Testing.java
+++ b/raft-server/src/test/java/org/apache/raft/statemachine/SimpleStateMachine4Testing.java
@@ -68,15 +68,14 @@ public class SimpleStateMachine4Testing extends BaseStateMachine {
   private final List<LogEntryProto> list =
       Collections.synchronizedList(new ArrayList<>());
   private final Daemon checkpointer;
+  private final SimpleStateMachineStorage storage = new SimpleStateMachineStorage();
+  private final TermIndexTracker termIndexTracker = new TermIndexTracker();
+  private final RaftProperties properties = new RaftProperties();
+
   private volatile boolean running = true;
   private long endIndexLastCkpt = RaftServerConstants.INVALID_LOG_INDEX;
-  private SimpleStateMachineStorage storage;
-  private TermIndexTracker termIndexTracker;
-  private final RaftProperties properties = new RaftProperties();
 
-  public SimpleStateMachine4Testing() {
-    this.storage  = new SimpleStateMachineStorage();
-    this.termIndexTracker = new TermIndexTracker();
+  SimpleStateMachine4Testing() {
     checkpointer = new Daemon(() -> {
       while (running) {
         try {

http://git-wip-us.apache.org/repos/asf/incubator-ratis/blob/8dbb64fb/raft-server/src/test/java/org/apache/raft/statemachine/TermIndexTracker.java
----------------------------------------------------------------------
diff --git a/raft-server/src/test/java/org/apache/raft/statemachine/TermIndexTracker.java b/raft-server/src/test/java/org/apache/raft/statemachine/TermIndexTracker.java
new file mode 100644
index 0000000..fa9c130
--- /dev/null
+++ b/raft-server/src/test/java/org/apache/raft/statemachine/TermIndexTracker.java
@@ -0,0 +1,66 @@
+/**
+ * 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.raft.statemachine;
+
+import com.google.common.base.Preconditions;
+import org.apache.raft.server.protocol.TermIndex;
+
+import static org.apache.raft.server.impl.RaftServerConstants.INVALID_LOG_INDEX;
+
+/**
+ * Tracks the term index that is applied to the StateMachine for simple state machines with
+ * no concurrent snapshoting capabilities.
+ */
+class TermIndexTracker {
+  static final TermIndex INIT_TERMINDEX =
+      new TermIndex(INVALID_LOG_INDEX, INVALID_LOG_INDEX);
+
+  private TermIndex current = INIT_TERMINDEX;
+
+  //TODO: developer note: everything is synchronized for now for convenience.
+
+  /**
+   * Initialize the tracker with a term index (likely from a snapshot).
+   */
+  public synchronized void init(TermIndex termIndex) {
+    this.current = termIndex;
+  }
+
+  public synchronized void reset() {
+    init(INIT_TERMINDEX);
+  }
+
+  /**
+   * Update the tracker with a new TermIndex. It means that the StateMachine has
+   * this index in memory.
+   */
+  public synchronized void update(TermIndex termIndex) {
+    Preconditions.checkArgument(termIndex != null &&
+        termIndex.compareTo(current) >= 0);
+    this.current = termIndex;
+  }
+
+  /**
+   * Return latest term and index that is inserted to this tracker as an atomic
+   * entity.
+   */
+  public synchronized TermIndex getLatestTermIndex() {
+    return current;
+  }
+
+}


[04/54] [abbrv] incubator-ratis git commit: Move o.a.r.s.* to o.a.r.s.impl.

Posted by ji...@apache.org.
http://git-wip-us.apache.org/repos/asf/incubator-ratis/blob/c36810ed/raft-server/src/main/java/org/apache/raft/server/LogAppender.java
----------------------------------------------------------------------
diff --git a/raft-server/src/main/java/org/apache/raft/server/LogAppender.java b/raft-server/src/main/java/org/apache/raft/server/LogAppender.java
deleted file mode 100644
index 1d2983b..0000000
--- a/raft-server/src/main/java/org/apache/raft/server/LogAppender.java
+++ /dev/null
@@ -1,481 +0,0 @@
-/**
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- *     http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-package org.apache.raft.server;
-
-import com.google.common.base.Preconditions;
-import com.google.common.collect.Lists;
-import org.apache.commons.io.IOUtils;
-import org.apache.raft.server.LeaderState.StateUpdateEventType;
-import org.apache.raft.server.impl.ServerProtoUtils;
-import org.apache.raft.server.protocol.TermIndex;
-import org.apache.raft.server.storage.FileInfo;
-import org.apache.raft.server.storage.RaftLog;
-import org.apache.raft.shaded.com.google.protobuf.ByteString;
-import org.apache.raft.shaded.proto.RaftProtos.*;
-import org.apache.raft.statemachine.SnapshotInfo;
-import org.apache.raft.util.Daemon;
-import org.apache.raft.util.ProtoUtils;
-import org.apache.raft.util.Timestamp;
-import org.slf4j.Logger;
-
-import java.io.File;
-import java.io.FileInputStream;
-import java.io.IOException;
-import java.io.InterruptedIOException;
-import java.nio.file.Path;
-import java.util.*;
-
-import static org.apache.raft.server.RaftServerConfigKeys.*;
-import static org.apache.raft.server.RaftServerConstants.INVALID_LOG_INDEX;
-
-/**
- * A daemon thread appending log entries to a follower peer.
- */
-public class LogAppender extends Daemon {
-  public static final Logger LOG = RaftServer.LOG;
-
-  protected final RaftServer server;
-  private final LeaderState leaderState;
-  protected final RaftLog raftLog;
-  protected final FollowerInfo follower;
-  private final int maxBufferSize;
-  private final boolean batchSending;
-  private final LogEntryBuffer buffer;
-  private final long leaderTerm;
-
-  private volatile boolean sending = true;
-
-  public LogAppender(RaftServer server, LeaderState leaderState, FollowerInfo f) {
-    this.follower = f;
-    this.server = server;
-    this.leaderState = leaderState;
-    this.raftLog = server.getState().getLog();
-    this.maxBufferSize = server.getProperties().getInt(
-        RAFT_SERVER_LOG_APPENDER_BUFFER_CAPACITY_KEY,
-        RAFT_SERVER_LOG_APPENDER_BUFFER_CAPACITY_DEFAULT);
-    this.batchSending = server.getProperties().getBoolean(
-        RAFT_SERVER_LOG_APPENDER_BATCH_ENABLED_KEY,
-        RAFT_SERVER_LOG_APPENDER_BATCH_ENABLED_DEFAULT);
-    this.buffer = new LogEntryBuffer();
-    this.leaderTerm = server.getState().getCurrentTerm();
-  }
-
-  @Override
-  public String toString() {
-    return getClass().getSimpleName() + "(" + server.getId() + " -> " +
-        follower.getPeer().getId() + ")";
-  }
-
-  @Override
-  public void run() {
-    try {
-      checkAndSendAppendEntries();
-    } catch (InterruptedException | InterruptedIOException e) {
-      LOG.info(this + " was interrupted: " + e);
-    }
-  }
-
-  protected boolean isAppenderRunning() {
-    return sending;
-  }
-
-  public void stopSender() {
-    this.sending = false;
-  }
-
-  public FollowerInfo getFollower() {
-    return follower;
-  }
-
-  /**
-   * A buffer for log entries with size limitation.
-   */
-  private class LogEntryBuffer {
-    private final List<LogEntryProto> buf = new ArrayList<>();
-    private int totalSize = 0;
-
-    void addEntry(LogEntryProto entry) {
-      buf.add(entry);
-      totalSize += entry.getSerializedSize();
-    }
-
-    boolean isFull() {
-      return totalSize >= maxBufferSize;
-    }
-
-    boolean isEmpty() {
-      return buf.isEmpty();
-    }
-
-    AppendEntriesRequestProto getAppendRequest(TermIndex previous) {
-      final AppendEntriesRequestProto request = server
-          .createAppendEntriesRequest(leaderTerm, follower.getPeer().getId(),
-              previous, buf, !follower.isAttendingVote());
-      buf.clear();
-      totalSize = 0;
-      return request;
-    }
-
-    int getPendingEntryNum() {
-      return buf.size();
-    }
-  }
-
-  private TermIndex getPrevious() {
-    TermIndex previous = ServerProtoUtils.toTermIndex(
-        raftLog.get(follower.getNextIndex() - 1));
-    if (previous == null) {
-      // if previous is null, nextIndex must be equal to the log start
-      // index (otherwise we will install snapshot).
-      Preconditions.checkState(follower.getNextIndex() == raftLog.getStartIndex(),
-          "follower's next index %s, local log start index %s",
-          follower.getNextIndex(), raftLog.getStartIndex());
-      SnapshotInfo snapshot = server.getState().getLatestSnapshot();
-      previous = snapshot == null ? null : snapshot.getTermIndex();
-    }
-    return previous;
-  }
-
-  protected AppendEntriesRequestProto createRequest() {
-    final TermIndex previous = getPrevious();
-    final long leaderNext = raftLog.getNextIndex();
-    long next = follower.getNextIndex() + buffer.getPendingEntryNum();
-    boolean toSend = false;
-
-    if (leaderNext == next && !buffer.isEmpty()) {
-      // no new entries, then send out the entries in the buffer
-      toSend = true;
-    } else if (leaderNext > next) {
-      while (leaderNext > next && !buffer.isFull()) {
-        // stop adding entry once the buffer size is >= the max size
-        buffer.addEntry(raftLog.get(next++));
-      }
-      if (buffer.isFull() || !batchSending) {
-        // buffer is full or batch sending is disabled, send out a request
-        toSend = true;
-      }
-    }
-
-    if (toSend || shouldHeartbeat()) {
-      return buffer.getAppendRequest(previous);
-    }
-    return null;
-  }
-
-  /** Send an appendEntries RPC; retry indefinitely. */
-  private AppendEntriesReplyProto sendAppendEntriesWithRetries()
-      throws InterruptedException, InterruptedIOException {
-    int retry = 0;
-    AppendEntriesRequestProto request = null;
-    while (isAppenderRunning()) { // keep retrying for IOException
-      try {
-        if (request == null || request.getEntriesCount() == 0) {
-          request = createRequest();
-        }
-
-        if (request == null) {
-          LOG.trace("{} need not send AppendEntries now." +
-              " Wait for more entries.", server.getId());
-          return null;
-        } else if (!isAppenderRunning()) {
-          LOG.debug("LogAppender {} has been stopped. Skip the request.", this);
-          return null;
-        }
-
-        follower.updateLastRpcSendTime();
-        final AppendEntriesReplyProto r = server.getServerRpc()
-            .sendAppendEntries(request);
-        follower.updateLastRpcResponseTime();
-
-        return r;
-      } catch (InterruptedIOException iioe) {
-        throw iioe;
-      } catch (IOException ioe) {
-        LOG.debug(this + ": failed to send appendEntries; retry " + retry++, ioe);
-      }
-      if (isAppenderRunning()) {
-        Thread.sleep(leaderState.getSyncInterval());
-      }
-    }
-    return null;
-  }
-
-  protected class SnapshotRequestIter
-      implements Iterable<InstallSnapshotRequestProto> {
-    private final SnapshotInfo snapshot;
-    private final List<FileInfo> files;
-    private FileInputStream in;
-    private int fileIndex = 0;
-
-    private FileInfo currentFileInfo;
-    private byte[] currentBuf;
-    private long currentFileSize;
-    private long currentOffset = 0;
-    private int chunkIndex = 0;
-
-    private final String requestId;
-    private int requestIndex = 0;
-
-    public SnapshotRequestIter(SnapshotInfo snapshot, String requestId)
-        throws IOException {
-      this.snapshot = snapshot;
-      this.requestId = requestId;
-      this.files = snapshot.getFiles();
-      if (files.size() > 0) {
-        startReadFile();
-      }
-    }
-
-    private void startReadFile() throws IOException {
-      currentFileInfo = files.get(fileIndex);
-      File snapshotFile = currentFileInfo.getPath().toFile();
-      currentFileSize = snapshotFile.length();
-      final int bufLength =
-          (int) Math.min(leaderState.getSnapshotChunkMaxSize(), currentFileSize);
-      currentBuf = new byte[bufLength];
-      currentOffset = 0;
-      chunkIndex = 0;
-      in = new FileInputStream(snapshotFile);
-    }
-
-    @Override
-    public Iterator<InstallSnapshotRequestProto> iterator() {
-      return new Iterator<InstallSnapshotRequestProto>() {
-        @Override
-        public boolean hasNext() {
-          return fileIndex < files.size();
-        }
-
-        @Override
-        public InstallSnapshotRequestProto next() {
-          if (fileIndex >= files.size()) {
-            throw new NoSuchElementException();
-          }
-          int targetLength = (int) Math.min(currentFileSize - currentOffset,
-              leaderState.getSnapshotChunkMaxSize());
-          FileChunkProto chunk;
-          try {
-            chunk = readFileChunk(currentFileInfo, in, currentBuf,
-                targetLength, currentOffset, chunkIndex);
-            boolean done = (fileIndex == files.size() - 1) &&
-                chunk.getDone();
-            InstallSnapshotRequestProto request =
-                server.createInstallSnapshotRequest(follower.getPeer().getId(),
-                    requestId, requestIndex++, snapshot,
-                    Lists.newArrayList(chunk), done);
-            currentOffset += targetLength;
-            chunkIndex++;
-
-            if (currentOffset >= currentFileSize) {
-              in.close();
-              fileIndex++;
-              if (fileIndex < files.size()) {
-                startReadFile();
-              }
-            }
-
-            return request;
-          } catch (IOException e) {
-            if (in != null) {
-              try {
-                in.close();
-              } catch (IOException ignored) {
-              }
-            }
-            LOG.warn("Got exception when preparing InstallSnapshot request", e);
-            throw new RuntimeException(e);
-          }
-        }
-      };
-    }
-  }
-
-  private FileChunkProto readFileChunk(FileInfo fileInfo,
-      FileInputStream in, byte[] buf, int length, long offset, int chunkIndex)
-      throws IOException {
-    FileChunkProto.Builder builder = FileChunkProto.newBuilder()
-        .setOffset(offset).setChunkIndex(chunkIndex);
-    IOUtils.readFully(in, buf, 0, length);
-    Path relativePath = server.getState().getStorage().getStorageDir()
-        .relativizeToRoot(fileInfo.getPath());
-    builder.setFilename(relativePath.toString());
-    builder.setDone(offset + length == fileInfo.getFileSize());
-    builder.setFileDigest(
-        ByteString.copyFrom(fileInfo.getFileDigest().getDigest()));
-    builder.setData(ByteString.copyFrom(buf, 0, length));
-    return builder.build();
-  }
-
-  private InstallSnapshotReplyProto installSnapshot(SnapshotInfo snapshot)
-      throws InterruptedException, InterruptedIOException {
-    String requestId = UUID.randomUUID().toString();
-    InstallSnapshotReplyProto reply = null;
-    try {
-      for (InstallSnapshotRequestProto request :
-          new SnapshotRequestIter(snapshot, requestId)) {
-        follower.updateLastRpcSendTime();
-        reply = server.getServerRpc().sendInstallSnapshot(request);
-        follower.updateLastRpcResponseTime();
-
-        if (!reply.getServerReply().getSuccess()) {
-          return reply;
-        }
-      }
-    } catch (InterruptedIOException iioe) {
-      throw iioe;
-    } catch (Exception ioe) {
-      LOG.warn(this + ": failed to install SnapshotInfo " + snapshot.getFiles(),
-          ioe);
-      return null;
-    }
-
-    if (reply != null) {
-      follower.updateMatchIndex(snapshot.getTermIndex().getIndex());
-      follower.updateNextIndex(snapshot.getTermIndex().getIndex() + 1);
-      LOG.info("{}: install snapshot-{} successfully on follower {}",
-          server.getId(), snapshot.getTermIndex().getIndex(), follower.getPeer());
-    }
-    return reply;
-  }
-
-  protected SnapshotInfo shouldInstallSnapshot() {
-    final long logStartIndex = raftLog.getStartIndex();
-    // we should install snapshot if the follower needs to catch up and:
-    // 1. there is no local log entry but there is snapshot
-    // 2. or the follower's next index is smaller than the log start index
-    if (follower.getNextIndex() < raftLog.getNextIndex()) {
-      SnapshotInfo snapshot = server.getState().getLatestSnapshot();
-      if (follower.getNextIndex() < logStartIndex ||
-          (logStartIndex == INVALID_LOG_INDEX && snapshot != null)) {
-        return snapshot;
-      }
-    }
-    return null;
-  }
-
-  /** Check and send appendEntries RPC */
-  private void checkAndSendAppendEntries()
-      throws InterruptedException, InterruptedIOException {
-    while (isAppenderRunning()) {
-      if (shouldSendRequest()) {
-        SnapshotInfo snapshot = shouldInstallSnapshot();
-        if (snapshot != null) {
-          LOG.info("{}: follower {}'s next index is {}," +
-              " log's start index is {}, need to install snapshot",
-              server.getId(), follower.getPeer(), follower.getNextIndex(),
-              raftLog.getStartIndex());
-
-          final InstallSnapshotReplyProto r = installSnapshot(snapshot);
-          if (r != null && r.getResult() == InstallSnapshotResult.NOT_LEADER) {
-            checkResponseTerm(r.getTerm());
-          } // otherwise if r is null, retry the snapshot installation
-        } else {
-          final AppendEntriesReplyProto r = sendAppendEntriesWithRetries();
-          if (r != null) {
-            handleReply(r);
-          }
-        }
-      }
-      if (isAppenderRunning() && !shouldAppendEntries(
-          follower.getNextIndex() + buffer.getPendingEntryNum())) {
-        final long waitTime = getHeartbeatRemainingTime(
-            follower.getLastRpcTime());
-        if (waitTime > 0) {
-          synchronized (this) {
-            wait(waitTime);
-          }
-        }
-      }
-    }
-  }
-
-  private void handleReply(AppendEntriesReplyProto reply) {
-    if (reply != null) {
-      switch (reply.getResult()) {
-        case SUCCESS:
-          final long oldNextIndex = follower.getNextIndex();
-          final long nextIndex = reply.getNextIndex();
-          if (nextIndex < oldNextIndex) {
-            throw new IllegalStateException("nextIndex=" + nextIndex
-                + " < oldNextIndex=" + oldNextIndex
-                + ", reply=" + ProtoUtils.toString(reply));
-          }
-
-          if (nextIndex > oldNextIndex) {
-            follower.updateMatchIndex(nextIndex - 1);
-            follower.updateNextIndex(nextIndex);
-            submitEventOnSuccessAppend();
-          }
-          break;
-        case NOT_LEADER:
-          // check if should step down
-          checkResponseTerm(reply.getTerm());
-          break;
-        case INCONSISTENCY:
-          follower.decreaseNextIndex(reply.getNextIndex());
-          break;
-        case UNRECOGNIZED:
-          LOG.warn("{} received UNRECOGNIZED AppendResult from {}",
-              server.getId(), follower.getPeer().getId());
-          break;
-      }
-    }
-  }
-
-  protected void submitEventOnSuccessAppend() {
-    LeaderState.StateUpdateEvent e = follower.isAttendingVote() ?
-        LeaderState.UPDATE_COMMIT_EVENT :
-        LeaderState.STAGING_PROGRESS_EVENT;
-    leaderState.submitUpdateStateEvent(e);
-  }
-
-  public synchronized void notifyAppend() {
-    this.notify();
-  }
-
-  /** Should the leader send appendEntries RPC to this follower? */
-  protected boolean shouldSendRequest() {
-    return shouldAppendEntries(follower.getNextIndex()) || shouldHeartbeat();
-  }
-
-  private boolean shouldAppendEntries(long followerIndex) {
-    return followerIndex < raftLog.getNextIndex();
-  }
-
-  private boolean shouldHeartbeat() {
-    return getHeartbeatRemainingTime(follower.getLastRpcTime()) <= 0;
-  }
-
-  /**
-   * @return the time in milliseconds that the leader should send a heartbeat.
-   */
-  protected long getHeartbeatRemainingTime(Timestamp lastTime) {
-    return server.getMinTimeoutMs() / 2 - lastTime.elapsedTimeMs();
-  }
-
-  protected void checkResponseTerm(long responseTerm) {
-    synchronized (server) {
-      if (isAppenderRunning() && follower.isAttendingVote()
-          && responseTerm > leaderState.getCurrentTerm()) {
-        leaderState.submitUpdateStateEvent(
-            new LeaderState.StateUpdateEvent(StateUpdateEventType.STEPDOWN,
-                responseTerm));
-      }
-    }
-  }
-}

http://git-wip-us.apache.org/repos/asf/incubator-ratis/blob/c36810ed/raft-server/src/main/java/org/apache/raft/server/LogAppenderFactory.java
----------------------------------------------------------------------
diff --git a/raft-server/src/main/java/org/apache/raft/server/LogAppenderFactory.java b/raft-server/src/main/java/org/apache/raft/server/LogAppenderFactory.java
deleted file mode 100644
index 3cb2b06..0000000
--- a/raft-server/src/main/java/org/apache/raft/server/LogAppenderFactory.java
+++ /dev/null
@@ -1,31 +0,0 @@
-/**
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- *     http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-package org.apache.raft.server;
-
-public interface LogAppenderFactory {
-  LogAppender getLogAppender(RaftServer server, LeaderState state,
-      FollowerInfo f);
-
-  class SynchronousLogAppenderFactory implements LogAppenderFactory {
-    @Override
-    public LogAppender getLogAppender(RaftServer server, LeaderState state,
-        FollowerInfo f) {
-      return new LogAppender(server, state, f);
-    }
-  }
-}

http://git-wip-us.apache.org/repos/asf/incubator-ratis/blob/c36810ed/raft-server/src/main/java/org/apache/raft/server/PeerConfiguration.java
----------------------------------------------------------------------
diff --git a/raft-server/src/main/java/org/apache/raft/server/PeerConfiguration.java b/raft-server/src/main/java/org/apache/raft/server/PeerConfiguration.java
deleted file mode 100644
index 9f01390..0000000
--- a/raft-server/src/main/java/org/apache/raft/server/PeerConfiguration.java
+++ /dev/null
@@ -1,90 +0,0 @@
-/**
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- *     http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-package org.apache.raft.server;
-
-import com.google.common.base.Preconditions;
-import org.apache.raft.protocol.RaftPeer;
-
-import java.util.*;
-
-/**
- * The peer configuration of a raft cluster.
- *
- * The objects of this class are immutable.
- */
-class PeerConfiguration {
-  private final Map<String, RaftPeer> peers;
-
-  PeerConfiguration(Iterable<RaftPeer> peers) {
-    Preconditions.checkNotNull(peers);
-    Map<String, RaftPeer> map = new HashMap<>();
-    for(RaftPeer p : peers) {
-      map.put(p.getId(), p);
-    }
-    this.peers = Collections.unmodifiableMap(map);
-    Preconditions.checkState(!this.peers.isEmpty());
-  }
-
-  Collection<RaftPeer> getPeers() {
-    return Collections.unmodifiableCollection(peers.values());
-  }
-
-  int size() {
-    return peers.size();
-  }
-
-  @Override
-  public String toString() {
-    return peers.values().toString();
-  }
-
-  RaftPeer getPeer(String id) {
-    return peers.get(id);
-  }
-
-  boolean contains(String id) {
-    return peers.containsKey(id);
-  }
-
-  List<RaftPeer> getOtherPeers(String selfId) {
-    List<RaftPeer> others = new ArrayList<>();
-    for (Map.Entry<String, RaftPeer> entry : peers.entrySet()) {
-      if (!selfId.equals(entry.getValue().getId())) {
-        others.add(entry.getValue());
-      }
-    }
-    return others;
-  }
-
-  boolean hasMajority(Collection<String> others, String selfId) {
-    Preconditions.checkArgument(!others.contains(selfId));
-    int num = 0;
-    if (contains(selfId)) {
-      num++;
-    }
-    for (String other : others) {
-      if (contains(other)) {
-        num++;
-      }
-      if (num > size() / 2) {
-        return true;
-      }
-    }
-    return false;
-  }
-}

http://git-wip-us.apache.org/repos/asf/incubator-ratis/blob/c36810ed/raft-server/src/main/java/org/apache/raft/server/PendingRequest.java
----------------------------------------------------------------------
diff --git a/raft-server/src/main/java/org/apache/raft/server/PendingRequest.java b/raft-server/src/main/java/org/apache/raft/server/PendingRequest.java
deleted file mode 100644
index 3598349..0000000
--- a/raft-server/src/main/java/org/apache/raft/server/PendingRequest.java
+++ /dev/null
@@ -1,87 +0,0 @@
-/**
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- * <p>
- * http://www.apache.org/licenses/LICENSE-2.0
- * <p>
- * 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.raft.server;
-
-import com.google.common.base.Preconditions;
-import org.apache.raft.protocol.Message;
-import org.apache.raft.protocol.RaftClientReply;
-import org.apache.raft.protocol.RaftClientRequest;
-import org.apache.raft.protocol.SetConfigurationRequest;
-import org.apache.raft.statemachine.TransactionContext;
-
-import java.util.concurrent.CompletableFuture;
-
-public class PendingRequest implements Comparable<PendingRequest> {
-  private final Long index;
-  private final RaftClientRequest request;
-  private final TransactionContext entry;
-  private final CompletableFuture<RaftClientReply> future;
-
-  PendingRequest(long index, RaftClientRequest request,
-                 TransactionContext entry) {
-    this.index = index;
-    this.request = request;
-    this.entry = entry;
-    this.future = new CompletableFuture<>();
-  }
-
-  PendingRequest(SetConfigurationRequest request) {
-    this(RaftServerConstants.INVALID_LOG_INDEX, request, null);
-  }
-
-  long getIndex() {
-    return index;
-  }
-
-  RaftClientRequest getRequest() {
-    return request;
-  }
-
-  public CompletableFuture<RaftClientReply> getFuture() {
-    return future;
-  }
-
-  TransactionContext getEntry() {
-    return entry;
-  }
-
-  synchronized void setException(Throwable e) {
-    Preconditions.checkArgument(e != null);
-    future.completeExceptionally(e);
-  }
-
-  synchronized void setReply(RaftClientReply r) {
-    Preconditions.checkArgument(r != null);
-    future.complete(r);
-  }
-
-  void setSuccessReply(Message message) {
-    setReply(new RaftClientReply(getRequest(), message));
-  }
-
-  @Override
-  public int compareTo(PendingRequest that) {
-    return Long.compare(this.index, that.index);
-  }
-
-  @Override
-  public String toString() {
-    return getClass().getSimpleName() + "(index=" + index
-        + ", request=" + request;
-  }
-}

http://git-wip-us.apache.org/repos/asf/incubator-ratis/blob/c36810ed/raft-server/src/main/java/org/apache/raft/server/PendingRequests.java
----------------------------------------------------------------------
diff --git a/raft-server/src/main/java/org/apache/raft/server/PendingRequests.java b/raft-server/src/main/java/org/apache/raft/server/PendingRequests.java
deleted file mode 100644
index a5731fd..0000000
--- a/raft-server/src/main/java/org/apache/raft/server/PendingRequests.java
+++ /dev/null
@@ -1,134 +0,0 @@
-/**
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- *     http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-package org.apache.raft.server;
-
-import java.io.IOException;
-import java.util.Collection;
-import java.util.concurrent.CompletableFuture;
-import java.util.concurrent.ConcurrentHashMap;
-import java.util.concurrent.ConcurrentMap;
-import java.util.stream.Collectors;
-
-import org.apache.raft.protocol.Message;
-import org.apache.raft.protocol.RaftClientReply;
-import org.apache.raft.protocol.RaftClientRequest;
-import org.apache.raft.protocol.RaftException;
-import org.apache.raft.protocol.SetConfigurationRequest;
-import org.apache.raft.statemachine.TransactionContext;
-import org.slf4j.Logger;
-
-import com.google.common.base.Preconditions;
-
-class PendingRequests {
-  private static final Logger LOG = RaftServer.LOG;
-
-  private PendingRequest pendingSetConf;
-  private final RaftServer server;
-  private final ConcurrentMap<Long, PendingRequest> pendingRequests = new ConcurrentHashMap<>();
-  private PendingRequest last = null;
-
-  PendingRequests(RaftServer server) {
-    this.server = server;
-  }
-
-  PendingRequest addPendingRequest(long index, RaftClientRequest request,
-      TransactionContext entry) {
-    // externally synced for now
-    Preconditions.checkArgument(!request.isReadOnly());
-    Preconditions.checkState(last == null || index == last.getIndex() + 1);
-    return add(index, request, entry);
-  }
-
-  private PendingRequest add(long index, RaftClientRequest request,
-      TransactionContext entry) {
-    final PendingRequest pending = new PendingRequest(index, request, entry);
-    pendingRequests.put(index, pending);
-    last = pending;
-    return pending;
-  }
-
-  PendingRequest addConfRequest(SetConfigurationRequest request) {
-    Preconditions.checkState(pendingSetConf == null);
-    pendingSetConf = new PendingRequest(request);
-    return pendingSetConf;
-  }
-
-  void replySetConfiguration() {
-    // we allow the pendingRequest to be null in case that the new leader
-    // commits the new configuration while it has not received the retry
-    // request from the client
-    if (pendingSetConf != null) {
-      // for setConfiguration we do not need to wait for statemachine. send back
-      // reply after it's committed.
-      pendingSetConf.setSuccessReply(null);
-      pendingSetConf = null;
-    }
-  }
-
-  void failSetConfiguration(RaftException e) {
-    Preconditions.checkState(pendingSetConf != null);
-    pendingSetConf.setException(e);
-    pendingSetConf = null;
-  }
-
-  TransactionContext getTransactionContext(long index) {
-    PendingRequest pendingRequest = pendingRequests.get(index);
-    // it is possible that the pendingRequest is null if this peer just becomes
-    // the new leader and commits transactions received by the previous leader
-    return pendingRequest != null ? pendingRequest.getEntry() : null;
-  }
-
-  void replyPendingRequest(long index, CompletableFuture<Message> messageFuture) {
-    final PendingRequest pending = pendingRequests.get(index);
-    if (pending != null) {
-      Preconditions.checkState(pending.getIndex() == index);
-
-      messageFuture.whenComplete((reply, exception) -> {
-        if (exception == null) {
-          pending.setSuccessReply(reply);
-        } else {
-          pending.setException(exception);
-        }
-      });
-    }
-  }
-
-  /**
-   * The leader state is stopped. Send NotLeaderException to all the pending
-   * requests since they have not got applied to the state machine yet.
-   */
-  void sendNotLeaderResponses() throws IOException {
-    LOG.info("{} sends responses before shutting down PendingRequestsHandler",
-        server.getId());
-
-    Collection<TransactionContext> pendingEntries = pendingRequests.values().stream()
-        .map(PendingRequest::getEntry).collect(Collectors.toList());
-    // notify the state machine about stepping down
-    server.getStateMachine().notifyNotLeader(pendingEntries);
-    pendingRequests.values().forEach(this::setNotLeaderException);
-    if (pendingSetConf != null) {
-      setNotLeaderException(pendingSetConf);
-    }
-  }
-
-  private void setNotLeaderException(PendingRequest pending) {
-    RaftClientReply reply = new RaftClientReply(pending.getRequest(),
-        server.generateNotLeaderException());
-    pending.setReply(reply);
-  }
-}

http://git-wip-us.apache.org/repos/asf/incubator-ratis/blob/c36810ed/raft-server/src/main/java/org/apache/raft/server/RaftConfiguration.java
----------------------------------------------------------------------
diff --git a/raft-server/src/main/java/org/apache/raft/server/RaftConfiguration.java b/raft-server/src/main/java/org/apache/raft/server/RaftConfiguration.java
deleted file mode 100644
index 54ed9d6..0000000
--- a/raft-server/src/main/java/org/apache/raft/server/RaftConfiguration.java
+++ /dev/null
@@ -1,261 +0,0 @@
-/**
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- *     http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-package org.apache.raft.server;
-
-import com.google.common.annotations.VisibleForTesting;
-import com.google.common.base.Preconditions;
-import org.apache.raft.protocol.RaftPeer;
-
-import java.util.*;
-import java.util.concurrent.ThreadLocalRandom;
-
-/**
- * The configuration of the raft cluster.
- *
- * The configuration is stable if there is no on-going peer change. Otherwise,
- * the configuration is transitional, i.e. in the middle of a peer change.
- *
- * The objects of this class are immutable.
- */
-public class RaftConfiguration {
-  /** Create a {@link Builder}. */
-  public static Builder newBuilder() {
-    return new Builder();
-  }
-
-  /** To build {@link RaftConfiguration} objects. */
-  public static class Builder {
-    private PeerConfiguration oldConf;
-    private PeerConfiguration conf;
-    private long logEntryIndex = RaftServerConstants.INVALID_LOG_INDEX;
-
-    private boolean forceStable = false;
-    private boolean forceTransitional = false;
-
-    private Builder() {}
-
-    public Builder setConf(PeerConfiguration conf) {
-      Preconditions.checkNotNull(conf);
-      Preconditions.checkState(this.conf == null, "conf is already set.");
-      this.conf = conf;
-      return this;
-    }
-
-    public Builder setConf(Iterable<RaftPeer> peers) {
-      return setConf(new PeerConfiguration(peers));
-    }
-
-    public Builder setConf(RaftPeer[] peers) {
-      return setConf(Arrays.asList(peers));
-    }
-
-    Builder setConf(RaftConfiguration transitionalConf) {
-      Preconditions.checkNotNull(transitionalConf);
-      Preconditions.checkState(transitionalConf.isTransitional());
-
-      Preconditions.checkState(!forceTransitional);
-      forceStable = true;
-      return setConf(transitionalConf.conf);
-    }
-
-
-    public Builder setOldConf(PeerConfiguration oldConf) {
-      Preconditions.checkNotNull(oldConf);
-      Preconditions.checkState(this.oldConf == null, "oldConf is already set.");
-      this.oldConf = oldConf;
-      return this;
-    }
-
-    public Builder setOldConf(Iterable<RaftPeer> oldPeers) {
-      return setOldConf(new PeerConfiguration(oldPeers));
-    }
-
-    public Builder setOldConf(RaftPeer[] oldPeers) {
-      return setOldConf(Arrays.asList(oldPeers));
-    }
-
-    Builder setOldConf(RaftConfiguration stableConf) {
-      Preconditions.checkNotNull(stableConf);
-      Preconditions.checkState(stableConf.isStable());
-
-      Preconditions.checkState(!forceStable);
-      forceTransitional = true;
-      return setOldConf(stableConf.conf);
-    }
-
-    public Builder setLogEntryIndex(long logEntryIndex) {
-      Preconditions.checkArgument(
-          logEntryIndex != RaftServerConstants.INVALID_LOG_INDEX);
-      Preconditions.checkState(
-          this.logEntryIndex == RaftServerConstants.INVALID_LOG_INDEX,
-          "logEntryIndex is already set.");
-      this.logEntryIndex = logEntryIndex;
-      return this;
-    }
-
-    /** Build a {@link RaftConfiguration}. */
-    public RaftConfiguration build() {
-      if (forceTransitional) {
-        Preconditions.checkState(oldConf != null);
-      }
-      if (forceStable) {
-        Preconditions.checkState(oldConf == null);
-      }
-      return new RaftConfiguration(conf, oldConf, logEntryIndex);
-    }
-  }
-
-  /** Non-null only if this configuration is transitional. */
-  private final PeerConfiguration oldConf;
-  /**
-   * The current peer configuration while this configuration is stable;
-   * or the new peer configuration while this configuration is transitional.
-   */
-  private final PeerConfiguration conf;
-
-  /** The index of the corresponding log entry for this configuration. */
-  private final long logEntryIndex;
-
-  private RaftConfiguration(PeerConfiguration conf, PeerConfiguration oldConf,
-      long logEntryIndex) {
-    Preconditions.checkNotNull(conf);
-    this.conf = conf;
-    this.oldConf = oldConf;
-    this.logEntryIndex = logEntryIndex;
-  }
-
-  /** Is this configuration transitional, i.e. in the middle of a peer change? */
-  public boolean isTransitional() {
-    return oldConf != null;
-  }
-
-  /** Is this configuration stable, i.e. no on-going peer change? */
-  public boolean isStable() {
-    return oldConf == null;
-  }
-
-  boolean containsInConf(String peerId) {
-    return conf.contains(peerId);
-  }
-
-  boolean containsInOldConf(String peerId) {
-    return oldConf != null && oldConf.contains(peerId);
-  }
-
-  public boolean contains(String peerId) {
-    return containsInConf(peerId) && (oldConf == null || containsInOldConf(peerId));
-  }
-
-  /**
-   * @return the peer corresponding to the given id;
-   *         or return null if the peer is not in this configuration.
-   */
-  public RaftPeer getPeer(String id) {
-    if (id == null) {
-      return null;
-    }
-    RaftPeer peer = conf.getPeer(id);
-    if (peer != null) {
-      return peer;
-    } else if (oldConf != null) {
-      return oldConf.getPeer(id);
-    }
-    return null;
-  }
-
-  /** @return all the peers from the conf, and the old conf if it exists. */
-  public Collection<RaftPeer> getPeers() {
-    final Collection<RaftPeer> peers = new ArrayList<>(conf.getPeers());
-    if (oldConf != null) {
-      oldConf.getPeers().stream().filter(p -> !peers.contains(p))
-          .forEach(peers::add);
-    }
-    return peers;
-  }
-
-  /**
-   * @return all the peers other than the given self id from the conf,
-   *         and the old conf if it exists.
-   */
-  public Collection<RaftPeer> getOtherPeers(String selfId) {
-    Collection<RaftPeer> others = conf.getOtherPeers(selfId);
-    if (oldConf != null) {
-      oldConf.getOtherPeers(selfId).stream()
-          .filter(p -> !others.contains(p))
-          .forEach(others::add);
-    }
-    return others;
-  }
-
-  /** @return true if the self id together with the others are in the majority. */
-  public boolean hasMajority(Collection<String> others, String selfId) {
-    Preconditions.checkArgument(!others.contains(selfId));
-    return conf.hasMajority(others, selfId) &&
-        (oldConf == null || oldConf.hasMajority(others, selfId));
-  }
-
-  @Override
-  public String toString() {
-    return conf + (oldConf != null ? "old:" + oldConf : "");
-  }
-
-  @VisibleForTesting
-  boolean hasNoChange(RaftPeer[] newMembers) {
-    if (!isStable() || conf.size() != newMembers.length) {
-      return false;
-    }
-    for (RaftPeer peer : newMembers) {
-      if (!conf.contains(peer.getId())) {
-        return false;
-      }
-    }
-    return true;
-  }
-
-  long getLogEntryIndex() {
-    return logEntryIndex;
-  }
-
-  static Collection<RaftPeer> computeNewPeers(RaftPeer[] newMembers,
-      RaftConfiguration old) {
-    List<RaftPeer> peers = new ArrayList<>();
-    for (RaftPeer p : newMembers) {
-      if (!old.containsInConf(p.getId())) {
-        peers.add(p);
-      }
-    }
-    return peers;
-  }
-
-  RaftPeer getRandomPeer(String exclusiveId) {
-    final List<RaftPeer> peers = conf.getOtherPeers(exclusiveId);
-    if (peers.isEmpty()) {
-      return null;
-    }
-    final int index = ThreadLocalRandom.current().nextInt(peers.size());
-    return peers.get(index);
-  }
-
-  public Collection<RaftPeer> getPeersInOldConf() {
-    return oldConf != null ? oldConf.getPeers() : Collections.emptyList();
-  }
-
-  public Collection<RaftPeer> getPeersInConf() {
-    return conf.getPeers();
-  }
-}

http://git-wip-us.apache.org/repos/asf/incubator-ratis/blob/c36810ed/raft-server/src/main/java/org/apache/raft/server/RaftServer.java
----------------------------------------------------------------------
diff --git a/raft-server/src/main/java/org/apache/raft/server/RaftServer.java b/raft-server/src/main/java/org/apache/raft/server/RaftServer.java
deleted file mode 100644
index c7ea6b2..0000000
--- a/raft-server/src/main/java/org/apache/raft/server/RaftServer.java
+++ /dev/null
@@ -1,750 +0,0 @@
-/**
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- *     http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-package org.apache.raft.server;
-
-import com.google.common.annotations.VisibleForTesting;
-import com.google.common.base.Preconditions;
-import org.apache.raft.conf.RaftProperties;
-import org.apache.raft.protocol.*;
-import org.apache.raft.server.protocol.RaftServerProtocol;
-import org.apache.raft.server.impl.ServerProtoUtils;
-import org.apache.raft.server.protocol.TermIndex;
-import org.apache.raft.server.storage.FileInfo;
-import org.apache.raft.shaded.proto.RaftProtos.*;
-import org.apache.raft.statemachine.SnapshotInfo;
-import org.apache.raft.statemachine.StateMachine;
-import org.apache.raft.statemachine.TransactionContext;
-import org.apache.raft.util.CodeInjectionForTesting;
-import org.apache.raft.util.LifeCycle;
-import org.apache.raft.util.ProtoUtils;
-import org.apache.raft.util.RaftUtils;
-import org.slf4j.Logger;
-import org.slf4j.LoggerFactory;
-
-import java.io.Closeable;
-import java.io.IOException;
-import java.io.InterruptedIOException;
-import java.util.Arrays;
-import java.util.Collection;
-import java.util.List;
-import java.util.OptionalLong;
-import java.util.concurrent.CompletableFuture;
-
-import static org.apache.raft.server.LeaderState.UPDATE_COMMIT_EVENT;
-import static org.apache.raft.shaded.proto.RaftProtos.AppendEntriesReplyProto.AppendResult.*;
-import static org.apache.raft.util.LifeCycle.State.*;
-
-public class RaftServer implements RaftServerProtocol, Closeable {
-  public static final Logger LOG = LoggerFactory.getLogger(RaftServer.class);
-
-  private static final String CLASS_NAME = RaftServer.class.getSimpleName();
-  static final String REQUEST_VOTE = CLASS_NAME + ".requestVote";
-  static final String APPEND_ENTRIES = CLASS_NAME + ".appendEntries";
-  static final String INSTALL_SNAPSHOT = CLASS_NAME + ".installSnapshot";
-
-
-  private final int minTimeoutMs;
-  private final int maxTimeoutMs;
-
-  private final LifeCycle lifeCycle;
-  private final ServerState state;
-  private final StateMachine stateMachine;
-  private final RaftProperties properties;
-  private volatile Role role;
-
-  /** used when the peer is follower, to monitor election timeout */
-  private volatile FollowerState heartbeatMonitor;
-
-  /** used when the peer is candidate, to request votes from other peers */
-  private volatile LeaderElection electionDaemon;
-
-  /** used when the peer is leader */
-  private volatile LeaderState leaderState;
-
-  private RaftServerRpc serverRpc;
-
-  private final LogAppenderFactory appenderFactory;
-
-  public RaftServer(String id, RaftConfiguration raftConf,
-      RaftProperties properties, StateMachine stateMachine) throws IOException {
-    this.lifeCycle = new LifeCycle(id);
-    minTimeoutMs = properties.getInt(
-        RaftServerConfigKeys.RAFT_SERVER_RPC_TIMEOUT_MIN_MS_KEY,
-        RaftServerConfigKeys.RAFT_SERVER_RPC_TIMEOUT_MIN_MS_DEFAULT);
-    maxTimeoutMs = properties.getInt(
-        RaftServerConfigKeys.RAFT_SERVER_RPC_TIMEOUT_MAX_MS_KEY,
-        RaftServerConfigKeys.RAFT_SERVER_RPC_TIMEOUT_MAX_MS_DEFAULT);
-    Preconditions.checkArgument(maxTimeoutMs > minTimeoutMs,
-        "max timeout: %s, min timeout: %s", maxTimeoutMs, minTimeoutMs);
-    this.properties = properties;
-    this.stateMachine = stateMachine;
-    this.state = new ServerState(id, raftConf, properties, this, stateMachine);
-    appenderFactory = initAppenderFactory();
-  }
-
-  public int getMinTimeoutMs() {
-    return minTimeoutMs;
-  }
-
-  public int getMaxTimeoutMs() {
-    return maxTimeoutMs;
-  }
-
-  public int getRandomTimeoutMs() {
-    return RaftUtils.getRandomBetween(minTimeoutMs, maxTimeoutMs);
-  }
-
-  public StateMachine getStateMachine() {
-    return this.stateMachine;
-  }
-
-  public LogAppenderFactory getLogAppenderFactory() {
-    return appenderFactory;
-  }
-
-  private LogAppenderFactory initAppenderFactory() {
-    Class<? extends LogAppenderFactory> factoryClass = properties.getClass(
-        RaftServerConfigKeys.RAFT_SERVER_LOG_APPENDER_FACTORY_CLASS_KEY,
-        RaftServerConfigKeys.RAFT_SERVER_LOG_APPENDER_FACTORY_CLASS_DEFAULT,
-        LogAppenderFactory.class);
-    return RaftUtils.newInstance(factoryClass);
-  }
-
-  /**
-   * Used by tests to set initial raft configuration with correct port bindings.
-   */
-  @VisibleForTesting
-  public void setInitialConf(RaftConfiguration conf) {
-    this.state.setInitialConf(conf);
-  }
-
-  public void setServerRpc(RaftServerRpc serverRpc) {
-    this.serverRpc = serverRpc;
-    // add peers into rpc service
-    RaftConfiguration conf = getRaftConf();
-    if (conf != null) {
-      addPeersToRPC(conf.getPeers());
-    }
-  }
-
-  public RaftServerRpc getServerRpc() {
-    return serverRpc;
-  }
-
-  public void start() {
-    lifeCycle.transition(STARTING);
-    state.start();
-    RaftConfiguration conf = getRaftConf();
-    if (conf != null && conf.contains(getId())) {
-      LOG.debug("{} starts as a follower", getId());
-      startAsFollower();
-    } else {
-      LOG.debug("{} starts with initializing state", getId());
-      startInitializing();
-    }
-  }
-
-  /**
-   * The peer belongs to the current configuration, should start as a follower
-   */
-  private void startAsFollower() {
-    role = Role.FOLLOWER;
-    heartbeatMonitor = new FollowerState(this);
-    heartbeatMonitor.start();
-
-    serverRpc.start();
-    lifeCycle.transition(RUNNING);
-  }
-
-  /**
-   * The peer does not have any configuration (maybe it will later be included
-   * in some configuration). Start still as a follower but will not vote or
-   * start election.
-   */
-  private void startInitializing() {
-    role = Role.FOLLOWER;
-    // do not start heartbeatMonitoring
-    serverRpc.start();
-  }
-
-  public ServerState getState() {
-    return this.state;
-  }
-
-  public String getId() {
-    return getState().getSelfId();
-  }
-
-  public RaftConfiguration getRaftConf() {
-    return getState().getRaftConf();
-  }
-
-  @Override
-  public void close() {
-    lifeCycle.checkStateAndClose(() -> {
-      try {
-        shutdownHeartbeatMonitor();
-        shutdownElectionDaemon();
-        shutdownLeaderState();
-
-        serverRpc.shutdown();
-        state.close();
-      } catch (Exception ignored) {
-        LOG.warn("Failed to kill " + state.getSelfId(), ignored);
-      }
-    });
-  }
-
-  public boolean isAlive() {
-    return !lifeCycle.getCurrentState().isOneOf(CLOSING, CLOSED);
-  }
-
-  public boolean isFollower() {
-    return role == Role.FOLLOWER;
-  }
-
-  public boolean isCandidate() {
-    return role == Role.CANDIDATE;
-  }
-
-  public boolean isLeader() {
-    return role == Role.LEADER;
-  }
-
-  Role getRole() {
-    return role;
-  }
-
-  /**
-   * Change the server state to Follower if necessary
-   * @param newTerm The new term.
-   * @param sync We will call {@link ServerState#persistMetadata()} if this is
-   *             set to true and term/votedFor get updated.
-   * @return if the term/votedFor should be updated to the new term
-   * @throws IOException if term/votedFor persistence failed.
-   */
-  synchronized boolean changeToFollower(long newTerm, boolean sync)
-      throws IOException {
-    final Role old = role;
-    role = Role.FOLLOWER;
-
-    boolean metadataUpdated = false;
-    if (newTerm > state.getCurrentTerm()) {
-      state.setCurrentTerm(newTerm);
-      state.resetLeaderAndVotedFor();
-      metadataUpdated = true;
-    }
-
-    if (old == Role.LEADER) {
-      assert leaderState != null;
-      shutdownLeaderState();
-    } else if (old == Role.CANDIDATE) {
-      shutdownElectionDaemon();
-    }
-
-    if (old != Role.FOLLOWER) {
-      heartbeatMonitor = new FollowerState(this);
-      heartbeatMonitor.start();
-    }
-
-    if (metadataUpdated && sync) {
-      state.persistMetadata();
-    }
-    return metadataUpdated;
-  }
-
-  private synchronized void shutdownLeaderState() {
-    final LeaderState leader = leaderState;
-    if (leader != null) {
-      leader.stop();
-    }
-    leaderState = null;
-    // TODO: make sure that StateMachineUpdater has applied all transactions that have context
-  }
-
-  private void shutdownElectionDaemon() {
-    final LeaderElection election = electionDaemon;
-    if (election != null) {
-      election.stopRunning();
-      // no need to interrupt the election thread
-    }
-    electionDaemon = null;
-  }
-
-  synchronized void changeToLeader() {
-    Preconditions.checkState(isCandidate());
-    shutdownElectionDaemon();
-    role = Role.LEADER;
-    state.becomeLeader();
-    // start sending AppendEntries RPC to followers
-    leaderState = new LeaderState(this, properties);
-    leaderState.start();
-  }
-
-  private void shutdownHeartbeatMonitor() {
-    final FollowerState hm = heartbeatMonitor;
-    if (hm != null) {
-      hm.stopRunning();
-      hm.interrupt();
-    }
-    heartbeatMonitor = null;
-  }
-
-  synchronized void changeToCandidate() {
-    Preconditions.checkState(isFollower());
-    shutdownHeartbeatMonitor();
-    role = Role.CANDIDATE;
-    // start election
-    electionDaemon = new LeaderElection(this);
-    electionDaemon.start();
-  }
-
-  @Override
-  public String toString() {
-    return role + " " + state + " " + lifeCycle.getCurrentState();
-  }
-
-  /**
-   * @return null if the server is in leader state.
-   */
-  CompletableFuture<RaftClientReply> checkLeaderState(
-      RaftClientRequest request) {
-    if (!isLeader()) {
-      NotLeaderException exception = generateNotLeaderException();
-      CompletableFuture<RaftClientReply> future = new CompletableFuture<>();
-      future.complete(new RaftClientReply(request, exception));
-      return future;
-    }
-    return null;
-  }
-
-  NotLeaderException generateNotLeaderException() {
-    if (lifeCycle.getCurrentState() != RUNNING) {
-      return new NotLeaderException(getId(), null, null);
-    }
-    String leaderId = state.getLeaderId();
-    if (leaderId == null || leaderId.equals(state.getSelfId())) {
-      // No idea about who is the current leader. Or the peer is the current
-      // leader, but it is about to step down
-      RaftPeer suggestedLeader = state.getRaftConf()
-          .getRandomPeer(state.getSelfId());
-      leaderId = suggestedLeader == null ? null : suggestedLeader.getId();
-    }
-    RaftConfiguration conf = getRaftConf();
-    Collection<RaftPeer> peers = conf.getPeers();
-    return new NotLeaderException(getId(), conf.getPeer(leaderId),
-        peers.toArray(new RaftPeer[peers.size()]));
-  }
-
-  /**
-   * Handle a normal update request from client.
-   */
-  public CompletableFuture<RaftClientReply> appendTransaction(
-      RaftClientRequest request, TransactionContext entry)
-      throws RaftException {
-    LOG.debug("{}: receive client request({})", getId(), request);
-    lifeCycle.assertCurrentState(RUNNING);
-    CompletableFuture<RaftClientReply> reply;
-
-    final PendingRequest pending;
-    synchronized (this) {
-      reply = checkLeaderState(request);
-      if (reply != null) {
-        return reply;
-      }
-
-      // append the message to its local log
-      final long entryIndex;
-      try {
-        entryIndex = state.applyLog(entry);
-      } catch (IOException e) {
-        throw new RaftException(e);
-      }
-
-      // put the request into the pending queue
-      pending = leaderState.addPendingRequest(entryIndex, request, entry);
-      leaderState.notifySenders();
-    }
-    return pending.getFuture();
-  }
-
-  /**
-   * Handle a raft configuration change request from client.
-   */
-  public CompletableFuture<RaftClientReply> setConfiguration(
-      SetConfigurationRequest request) throws IOException {
-    LOG.debug("{}: receive setConfiguration({})", getId(), request);
-    lifeCycle.assertCurrentState(RUNNING);
-    CompletableFuture<RaftClientReply> reply = checkLeaderState(request);
-    if (reply != null) {
-      return reply;
-    }
-
-    final RaftPeer[] peersInNewConf = request.getPeersInNewConf();
-    final PendingRequest pending;
-    synchronized (this) {
-      reply = checkLeaderState(request);
-      if (reply != null) {
-        return reply;
-      }
-
-      final RaftConfiguration current = getRaftConf();
-      // make sure there is no other raft reconfiguration in progress
-      if (!current.isStable() || leaderState.inStagingState() ||
-          !state.isCurrentConfCommitted()) {
-        throw new ReconfigurationInProgressException(
-            "Reconfiguration is already in progress: " + current);
-      }
-
-      // return true if the new configuration is the same with the current one
-      if (current.hasNoChange(peersInNewConf)) {
-        pending = leaderState.returnNoConfChange(request);
-        return pending.getFuture();
-      }
-
-      // add new peers into the rpc service
-      addPeersToRPC(Arrays.asList(peersInNewConf));
-      // add staging state into the leaderState
-      pending = leaderState.startSetConfiguration(request);
-    }
-    return pending.getFuture();
-  }
-
-  private boolean shouldWithholdVotes() {
-    return isLeader() || (isFollower() && state.hasLeader()
-        && heartbeatMonitor.shouldWithholdVotes());
-  }
-
-  /**
-   * check if the remote peer is not included in the current conf
-   * and should shutdown. should shutdown if all the following stands:
-   * 1. this is a leader
-   * 2. current conf is stable and has been committed
-   * 3. candidate id is not included in conf
-   * 4. candidate's last entry's index < conf's index
-   */
-  private boolean shouldSendShutdown(String candidateId,
-      TermIndex candidateLastEntry) {
-    return isLeader()
-        && getRaftConf().isStable()
-        && getState().isConfCommitted()
-        && !getRaftConf().containsInConf(candidateId)
-        && candidateLastEntry.getIndex() < getRaftConf().getLogEntryIndex()
-        && !leaderState.isBootStrappingPeer(candidateId);
-  }
-
-  @Override
-  public RequestVoteReplyProto requestVote(RequestVoteRequestProto r)
-      throws IOException {
-    final String candidateId = r.getServerRequest().getRequestorId();
-    return requestVote(candidateId, r.getCandidateTerm(),
-        ServerProtoUtils.toTermIndex(r.getCandidateLastEntry()));
-  }
-
-  private RequestVoteReplyProto requestVote(String candidateId,
-      long candidateTerm, TermIndex candidateLastEntry) throws IOException {
-    CodeInjectionForTesting.execute(REQUEST_VOTE, getId(),
-        candidateId, candidateTerm, candidateLastEntry);
-    LOG.debug("{}: receive requestVote({}, {}, {})",
-        getId(), candidateId, candidateTerm, candidateLastEntry);
-    lifeCycle.assertCurrentState(RUNNING);
-
-    boolean voteGranted = false;
-    boolean shouldShutdown = false;
-    final RequestVoteReplyProto reply;
-    synchronized (this) {
-      if (shouldWithholdVotes()) {
-        LOG.info("{} Withhold vote from server {} with term {}. " +
-            "This server:{}, last rpc time from leader {} is {}", getId(),
-            candidateId, candidateTerm, this, this.getState().getLeaderId(),
-            (isFollower() ? heartbeatMonitor.getLastRpcTime() : -1));
-      } else if (state.recognizeCandidate(candidateId, candidateTerm)) {
-        boolean termUpdated = changeToFollower(candidateTerm, false);
-        // see Section 5.4.1 Election restriction
-        if (state.isLogUpToDate(candidateLastEntry)) {
-          heartbeatMonitor.updateLastRpcTime(false);
-          state.grantVote(candidateId);
-          voteGranted = true;
-        }
-        if (termUpdated || voteGranted) {
-          state.persistMetadata(); // sync metafile
-        }
-      }
-      if (!voteGranted && shouldSendShutdown(candidateId, candidateLastEntry)) {
-        shouldShutdown = true;
-      }
-      reply = ServerProtoUtils.toRequestVoteReplyProto(candidateId, getId(),
-          voteGranted, state.getCurrentTerm(), shouldShutdown);
-      if (LOG.isDebugEnabled()) {
-        LOG.debug("{} replies to vote request: {}. Peer's state: {}",
-            getId(), ProtoUtils.toString(reply), state);
-      }
-    }
-    return reply;
-  }
-
-  private void validateEntries(long expectedTerm, TermIndex previous,
-      LogEntryProto... entries) {
-    if (entries != null && entries.length > 0) {
-      final long index0 = entries[0].getIndex();
-
-      if (previous == null || previous.getTerm() == 0) {
-        Preconditions.checkArgument(index0 == 0,
-            "Unexpected Index: previous is null but entries[%s].getIndex()=%s",
-            0, index0);
-      } else {
-        Preconditions.checkArgument(previous.getIndex() == index0 - 1,
-            "Unexpected Index: previous is %s but entries[%s].getIndex()=%s",
-            previous, 0, index0);
-      }
-
-      for (int i = 0; i < entries.length; i++) {
-        final long t = entries[i].getTerm();
-        Preconditions.checkArgument(expectedTerm >= t,
-            "Unexpected Term: entries[%s].getTerm()=%s but expectedTerm=%s",
-            i, t, expectedTerm);
-
-        final long indexi = entries[i].getIndex();
-        Preconditions.checkArgument(indexi == index0 + i,
-            "Unexpected Index: entries[%s].getIndex()=%s but entries[0].getIndex()=%s",
-            i, indexi, index0);
-      }
-    }
-  }
-
-  @Override
-  public AppendEntriesReplyProto appendEntries(AppendEntriesRequestProto r)
-      throws IOException {
-    // TODO avoid converting list to array
-    final LogEntryProto[] entries = r.getEntriesList()
-        .toArray(new LogEntryProto[r.getEntriesCount()]);
-    final TermIndex previous = r.hasPreviousLog() ?
-        ServerProtoUtils.toTermIndex(r.getPreviousLog()) : null;
-    return appendEntries(r.getServerRequest().getRequestorId(),
-        r.getLeaderTerm(), previous, r.getLeaderCommit(), r.getInitializing(),
-        entries);
-  }
-
-  private AppendEntriesReplyProto appendEntries(String leaderId, long leaderTerm,
-      TermIndex previous, long leaderCommit, boolean initializing,
-      LogEntryProto... entries) throws IOException {
-    CodeInjectionForTesting.execute(APPEND_ENTRIES, getId(),
-        leaderId, leaderTerm, previous, leaderCommit, initializing, entries);
-    if (LOG.isDebugEnabled()) {
-      LOG.debug("{}: receive appendEntries({}, {}, {}, {}, {}, {})", getId(),
-          leaderId, leaderTerm, previous, leaderCommit, initializing,
-          ServerProtoUtils.toString(entries));
-    }
-    lifeCycle.assertCurrentState(STARTING, RUNNING);
-
-    try {
-      validateEntries(leaderTerm, previous, entries);
-    } catch (IllegalArgumentException e) {
-      throw new IOException(e);
-    }
-
-    final long currentTerm;
-    long nextIndex = state.getLog().getNextIndex();
-    synchronized (this) {
-      final boolean recognized = state.recognizeLeader(leaderId, leaderTerm);
-      currentTerm = state.getCurrentTerm();
-      if (!recognized) {
-        final AppendEntriesReplyProto reply = ServerProtoUtils.toAppendEntriesReplyProto(
-            leaderId, getId(), currentTerm, nextIndex, NOT_LEADER);
-        if (LOG.isDebugEnabled()) {
-          LOG.debug("{}: do not recognize leader. Reply: {}",
-              getId(), ProtoUtils.toString(reply));
-        }
-        return reply;
-      }
-      changeToFollower(leaderTerm, true);
-      state.setLeader(leaderId);
-
-      if (!initializing && lifeCycle.compareAndTransition(STARTING, RUNNING)) {
-        heartbeatMonitor = new FollowerState(this);
-        heartbeatMonitor.start();
-      }
-      if (lifeCycle.getCurrentState() == RUNNING) {
-        heartbeatMonitor.updateLastRpcTime(true);
-      }
-
-      // We need to check if "previous" is in the local peer. Note that it is
-      // possible that "previous" is covered by the latest snapshot: e.g.,
-      // it's possible there's no log entries outside of the latest snapshot.
-      // However, it is not possible that "previous" index is smaller than the
-      // last index included in snapshot. This is because indices <= snapshot's
-      // last index should have been committed.
-      if (previous != null && !containPrevious(previous)) {
-        final AppendEntriesReplyProto reply =
-            ServerProtoUtils.toAppendEntriesReplyProto(leaderId, getId(),
-                currentTerm, Math.min(nextIndex, previous.getIndex()), INCONSISTENCY);
-        LOG.debug("{}: inconsistency entries. Leader previous:{}, Reply:{}",
-            getId(), previous, ServerProtoUtils.toString(reply));
-        return reply;
-      }
-
-      state.getLog().append(entries);
-      state.updateConfiguration(entries);
-      state.updateStatemachine(leaderCommit, currentTerm);
-    }
-    if (entries != null && entries.length > 0) {
-      try {
-        state.getLog().logSync();
-      } catch (InterruptedException e) {
-        throw new InterruptedIOException("logSync got interrupted");
-      }
-      nextIndex = entries[entries.length - 1].getIndex() + 1;
-    }
-    synchronized (this) {
-      if (lifeCycle.getCurrentState() == RUNNING && isFollower()
-          && getState().getCurrentTerm() == currentTerm) {
-        // reset election timer to avoid punishing the leader for our own
-        // long disk writes
-        heartbeatMonitor.updateLastRpcTime(false);
-      }
-    }
-    final AppendEntriesReplyProto reply = ServerProtoUtils.toAppendEntriesReplyProto(
-        leaderId, getId(), currentTerm, nextIndex, SUCCESS);
-    LOG.debug("{}: succeeded to handle AppendEntries. Reply: {}", getId(),
-        ServerProtoUtils.toString(reply));
-    return reply;
-  }
-
-  private boolean containPrevious(TermIndex previous) {
-    LOG.debug("{}: prev:{}, latestSnapshot:{}, getLatestInstalledSnapshot:{}",
-        getId(), previous, state.getLatestSnapshot(), state.getLatestInstalledSnapshot());
-    return state.getLog().contains(previous)
-        ||  (state.getLatestSnapshot() != null
-             && state.getLatestSnapshot().getTermIndex().equals(previous))
-        || (state.getLatestInstalledSnapshot() != null)
-             && state.getLatestInstalledSnapshot().equals(previous);
-  }
-
-  @Override
-  public InstallSnapshotReplyProto installSnapshot(
-      InstallSnapshotRequestProto request) throws IOException {
-    final String leaderId = request.getServerRequest().getRequestorId();
-    CodeInjectionForTesting.execute(INSTALL_SNAPSHOT, getId(), leaderId, request);
-    LOG.debug("{}: receive installSnapshot({})", getId(), request);
-
-    lifeCycle.assertCurrentState(STARTING, RUNNING);
-
-    final long currentTerm;
-    final long leaderTerm = request.getLeaderTerm();
-    final TermIndex lastTermIndex = ServerProtoUtils.toTermIndex(
-        request.getTermIndex());
-    final long lastIncludedIndex = lastTermIndex.getIndex();
-    synchronized (this) {
-      final boolean recognized = state.recognizeLeader(leaderId, leaderTerm);
-      currentTerm = state.getCurrentTerm();
-      if (!recognized) {
-        final InstallSnapshotReplyProto reply = ServerProtoUtils
-            .toInstallSnapshotReplyProto(leaderId, getId(), currentTerm,
-                request.getRequestIndex(), InstallSnapshotResult.NOT_LEADER);
-        LOG.debug("{}: do not recognize leader for installing snapshot." +
-            " Reply: {}", getId(), reply);
-        return reply;
-      }
-      changeToFollower(leaderTerm, true);
-      state.setLeader(leaderId);
-
-      if (lifeCycle.getCurrentState() == RUNNING) {
-        heartbeatMonitor.updateLastRpcTime(true);
-      }
-
-      // Check and append the snapshot chunk. We simply put this in lock
-      // considering a follower peer requiring a snapshot installation does not
-      // have a lot of requests
-      Preconditions.checkState(
-          state.getLog().getNextIndex() <= lastIncludedIndex,
-          "%s log's next id is %s, last included index in snapshot is %s",
-          getId(),  state.getLog().getNextIndex(), lastIncludedIndex);
-
-      //TODO: We should only update State with installed snapshot once the request is done.
-      state.installSnapshot(request);
-
-      // update the committed index
-      // re-load the state machine if this is the last chunk
-      if (request.getDone()) {
-        state.reloadStateMachine(lastIncludedIndex, leaderTerm);
-      }
-      if (lifeCycle.getCurrentState() == RUNNING) {
-        heartbeatMonitor.updateLastRpcTime(false);
-      }
-    }
-    if (request.getDone()) {
-      LOG.info("{}: successfully install the whole snapshot-{}", getId(),
-          lastIncludedIndex);
-    }
-    return ServerProtoUtils.toInstallSnapshotReplyProto(leaderId, getId(),
-        currentTerm, request.getRequestIndex(), InstallSnapshotResult.SUCCESS);
-  }
-
-  AppendEntriesRequestProto createAppendEntriesRequest(long leaderTerm,
-      String targetId, TermIndex previous, List<LogEntryProto> entries,
-      boolean initializing) {
-    return ServerProtoUtils.toAppendEntriesRequestProto(getId(), targetId,
-        leaderTerm, entries, state.getLog().getLastCommittedIndex(),
-        initializing, previous);
-  }
-
-  synchronized InstallSnapshotRequestProto createInstallSnapshotRequest(
-      String targetId, String requestId, int requestIndex, SnapshotInfo snapshot,
-      List<FileChunkProto> chunks, boolean done) {
-    OptionalLong totalSize = snapshot.getFiles().stream()
-        .mapToLong(FileInfo::getFileSize).reduce(Long::sum);
-    assert totalSize.isPresent();
-    return ServerProtoUtils.toInstallSnapshotRequestProto(getId(), targetId,
-        requestId, requestIndex, state.getCurrentTerm(), snapshot.getTermIndex(),
-        chunks, totalSize.getAsLong(), done);
-  }
-
-  synchronized RequestVoteRequestProto createRequestVoteRequest(String targetId,
-      long term, TermIndex lastEntry) {
-    return ServerProtoUtils.toRequestVoteRequestProto(getId(), targetId, term,
-        lastEntry);
-  }
-
-  public synchronized void submitLocalSyncEvent() {
-    if (isLeader() && leaderState != null) {
-      leaderState.submitUpdateStateEvent(UPDATE_COMMIT_EVENT);
-    }
-  }
-
-  public void addPeersToRPC(Iterable<RaftPeer> peers) {
-    serverRpc.addPeers(peers);
-  }
-
-  synchronized void replyPendingRequest(long logIndex,
-      CompletableFuture<Message> message) {
-    if (isLeader() && leaderState != null) { // is leader and is running
-      leaderState.replyPendingRequest(logIndex, message);
-    }
-  }
-
-  TransactionContext getTransactionContext(long index) {
-    if (leaderState != null) { // is leader and is running
-      return leaderState.getTransactionContext(index);
-    }
-    return null;
-  }
-
-  public RaftProperties getProperties() {
-    return this.properties;
-  }
-}

http://git-wip-us.apache.org/repos/asf/incubator-ratis/blob/c36810ed/raft-server/src/main/java/org/apache/raft/server/RaftServerConfigKeys.java
----------------------------------------------------------------------
diff --git a/raft-server/src/main/java/org/apache/raft/server/RaftServerConfigKeys.java b/raft-server/src/main/java/org/apache/raft/server/RaftServerConfigKeys.java
index 837b53b..2ce0326 100644
--- a/raft-server/src/main/java/org/apache/raft/server/RaftServerConfigKeys.java
+++ b/raft-server/src/main/java/org/apache/raft/server/RaftServerConfigKeys.java
@@ -17,6 +17,7 @@
  */
 package org.apache.raft.server;
 
+import org.apache.raft.server.impl.LogAppenderFactory;
 import org.apache.raft.util.NetUtils;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;

http://git-wip-us.apache.org/repos/asf/incubator-ratis/blob/c36810ed/raft-server/src/main/java/org/apache/raft/server/RaftServerConstants.java
----------------------------------------------------------------------
diff --git a/raft-server/src/main/java/org/apache/raft/server/RaftServerConstants.java b/raft-server/src/main/java/org/apache/raft/server/RaftServerConstants.java
deleted file mode 100644
index f6781f3..0000000
--- a/raft-server/src/main/java/org/apache/raft/server/RaftServerConstants.java
+++ /dev/null
@@ -1,46 +0,0 @@
-/**
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- *     http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-package org.apache.raft.server;
-
-import org.apache.raft.client.RaftClient;
-
-public interface RaftServerConstants {
-  long INVALID_LOG_INDEX = -1;
-  byte LOG_TERMINATE_BYTE = 0;
-  long DEFAULT_SEQNUM = RaftClient.DEFAULT_SEQNUM;
-
-  enum StartupOption {
-    FORMAT("format"),
-    REGULAR("regular");
-
-    private final String option;
-
-    StartupOption(String arg) {
-      this.option = arg;
-    }
-
-    public static StartupOption getOption(String arg) {
-      for (StartupOption s : StartupOption.values()) {
-        if (s.option.equals(arg)) {
-          return s;
-        }
-      }
-      return REGULAR;
-    }
-  }
-}

http://git-wip-us.apache.org/repos/asf/incubator-ratis/blob/c36810ed/raft-server/src/main/java/org/apache/raft/server/RaftServerRpc.java
----------------------------------------------------------------------
diff --git a/raft-server/src/main/java/org/apache/raft/server/RaftServerRpc.java b/raft-server/src/main/java/org/apache/raft/server/RaftServerRpc.java
deleted file mode 100644
index de81ec2..0000000
--- a/raft-server/src/main/java/org/apache/raft/server/RaftServerRpc.java
+++ /dev/null
@@ -1,44 +0,0 @@
-/**
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- *     http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-package org.apache.raft.server;
-
-import org.apache.raft.protocol.RaftPeer;
-import org.apache.raft.shaded.proto.RaftProtos.*;
-
-import java.io.IOException;
-import java.net.InetSocketAddress;
-
-public interface RaftServerRpc {
-  void start();
-
-  void shutdown();
-
-  InetSocketAddress getInetSocketAddress();
-
-  AppendEntriesReplyProto sendAppendEntries(
-      AppendEntriesRequestProto request) throws IOException;
-
-  InstallSnapshotReplyProto sendInstallSnapshot(
-      InstallSnapshotRequestProto request) throws IOException;
-
-  RequestVoteReplyProto sendRequestVote(RequestVoteRequestProto request)
-      throws IOException;
-
-  /** add information of the given peers */
-  void addPeers(Iterable<RaftPeer> peers);
-}

http://git-wip-us.apache.org/repos/asf/incubator-ratis/blob/c36810ed/raft-server/src/main/java/org/apache/raft/server/RequestDispatcher.java
----------------------------------------------------------------------
diff --git a/raft-server/src/main/java/org/apache/raft/server/RequestDispatcher.java b/raft-server/src/main/java/org/apache/raft/server/RequestDispatcher.java
deleted file mode 100644
index e281bfa..0000000
--- a/raft-server/src/main/java/org/apache/raft/server/RequestDispatcher.java
+++ /dev/null
@@ -1,137 +0,0 @@
-/**
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- *     http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-package org.apache.raft.server;
-
-import org.apache.raft.protocol.*;
-import org.apache.raft.server.protocol.RaftServerProtocol;
-import org.apache.raft.shaded.proto.RaftProtos.*;
-import org.apache.raft.statemachine.StateMachine;
-import org.apache.raft.statemachine.TransactionContext;
-import org.apache.raft.util.RaftUtils;
-import org.slf4j.Logger;
-import org.slf4j.LoggerFactory;
-
-import java.io.IOException;
-import java.util.concurrent.CompletableFuture;
-import java.util.concurrent.ExecutionException;
-
-/**
- * Each RPC request is first handled by the RequestDispatcher:
- * 1. A request from another RaftPeer is to be handled by RaftServer.
- *
- * If the raft peer is the leader, then:
- *
- * 2. A read-only request from client is to be handled by the state machine.
- * 3. A write request from client is first validated by the state machine. The
- * state machine returns the content of the raft log entry, which is then passed
- * to the RaftServer for replication.
- */
-public class RequestDispatcher implements RaftClientProtocol, RaftServerProtocol {
-  static final Logger LOG = LoggerFactory.getLogger(RequestDispatcher.class);
-
-  private final RaftServer server;
-  private final StateMachine stateMachine;
-
-  public RequestDispatcher(RaftServer server) {
-    this.server = server;
-    this.stateMachine = server.getStateMachine();
-  }
-
-  public CompletableFuture<RaftClientReply> handleClientRequest(
-      RaftClientRequest request) throws IOException {
-    // first check the server's leader state
-    CompletableFuture<RaftClientReply> reply = server.checkLeaderState(request);
-    if (reply != null) {
-      return reply;
-    }
-
-    // let the state machine handle read-only request from client
-    if (request.isReadOnly()) {
-      // TODO: We might not be the leader anymore by the time this completes. See the RAFT paper,
-      // section 8 (last part)
-      return stateMachine.query(request);
-    }
-
-    // TODO: this client request will not be added to pending requests
-    // until later which means that any failure in between will leave partial state in the
-    // state machine. We should call cancelTransaction() for failed requests
-    TransactionContext entry = stateMachine.startTransaction(request);
-    if (entry.getException().isPresent()) {
-      throw RaftUtils.asIOException(entry.getException().get());
-    }
-
-    return server.appendTransaction(request, entry);
-  }
-
-  @Override
-  public RaftClientReply submitClientRequest(RaftClientRequest request)
-      throws IOException {
-    return waitForReply(server.getId(), request, handleClientRequest(request));
-  }
-
-  public CompletableFuture<RaftClientReply> setConfigurationAsync(
-      SetConfigurationRequest request) throws IOException {
-    return server.setConfiguration(request);
-  }
-
-  @Override
-  public RaftClientReply setConfiguration(SetConfigurationRequest request)
-      throws IOException {
-    return waitForReply(server.getId(), request, setConfigurationAsync(request));
-  }
-
-  private static RaftClientReply waitForReply(String serverId,
-      RaftClientRequest request, CompletableFuture<RaftClientReply> future)
-      throws IOException {
-    try {
-      return future.get();
-    } catch (InterruptedException e) {
-      final String s = serverId + ": Interrupted when waiting for reply, request=" + request;
-      LOG.info(s, e);
-      throw RaftUtils.toInterruptedIOException(s, e);
-    } catch (ExecutionException e) {
-      final Throwable cause = e.getCause();
-      if (cause == null) {
-        throw new IOException(e);
-      }
-      if (cause instanceof NotLeaderException) {
-        return new RaftClientReply(request, (NotLeaderException)cause);
-      } else {
-        throw RaftUtils.asIOException(cause);
-      }
-    }
-  }
-
-  @Override
-  public RequestVoteReplyProto requestVote(RequestVoteRequestProto request)
-      throws IOException {
-    return server.requestVote(request);
-  }
-
-  @Override
-  public AppendEntriesReplyProto appendEntries(AppendEntriesRequestProto request)
-      throws IOException {
-    return server.appendEntries(request);
-  }
-
-  @Override
-  public InstallSnapshotReplyProto installSnapshot(
-      InstallSnapshotRequestProto request) throws IOException {
-    return server.installSnapshot(request);
-  }
-}

http://git-wip-us.apache.org/repos/asf/incubator-ratis/blob/c36810ed/raft-server/src/main/java/org/apache/raft/server/Role.java
----------------------------------------------------------------------
diff --git a/raft-server/src/main/java/org/apache/raft/server/Role.java b/raft-server/src/main/java/org/apache/raft/server/Role.java
deleted file mode 100644
index a7e2f4c..0000000
--- a/raft-server/src/main/java/org/apache/raft/server/Role.java
+++ /dev/null
@@ -1,25 +0,0 @@
-/**
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- *     http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-package org.apache.raft.server;
-
-/**
- * Role of Raft peer
- */
-public enum Role {
-  LEADER, CANDIDATE, FOLLOWER
-}


[27/54] [abbrv] incubator-ratis git commit: Renamed the packages from raft to ratis in preperation for Apache Incubation - Moved all java packages from org.apache.raft to org.apache.ratis. - Moved native package to org_apache_ratis, and native lib to l

Posted by ji...@apache.org.
http://git-wip-us.apache.org/repos/asf/incubator-ratis/blob/7e71a2e0/ratis-common/src/main/native/src/org/apache/ratis/io/nativeio/errno_enum.c
----------------------------------------------------------------------
diff --git a/ratis-common/src/main/native/src/org/apache/ratis/io/nativeio/errno_enum.c b/ratis-common/src/main/native/src/org/apache/ratis/io/nativeio/errno_enum.c
new file mode 100644
index 0000000..ff186a9
--- /dev/null
+++ b/ratis-common/src/main/native/src/org/apache/ratis/io/nativeio/errno_enum.c
@@ -0,0 +1,123 @@
+/*
+ * 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.
+ */
+ #include <assert.h>
+ #include <errno.h>
+ #include <jni.h>
+
+#include "org_apache_ratis.h"
+
+typedef struct errno_mapping {
+  int errno_val;
+  char *errno_str;
+} errno_mapping_t;
+
+// Macro to define structs like {FOO, "FOO"} for each errno value
+#define MAPPING(x) {x, #x}
+static errno_mapping_t ERRNO_MAPPINGS[] = {
+  MAPPING(EPERM),
+  MAPPING(ENOENT),
+  MAPPING(ESRCH),
+  MAPPING(EINTR),
+  MAPPING(EIO),
+  MAPPING(ENXIO),
+  MAPPING(E2BIG),
+  MAPPING(ENOEXEC),
+  MAPPING(EBADF),
+  MAPPING(ECHILD),
+  MAPPING(EAGAIN),
+  MAPPING(ENOMEM),
+  MAPPING(EACCES),
+  MAPPING(EFAULT),
+  MAPPING(ENOTBLK),
+  MAPPING(EBUSY),
+  MAPPING(EEXIST),
+  MAPPING(EXDEV),
+  MAPPING(ENODEV),
+  MAPPING(ENOTDIR),
+  MAPPING(EISDIR),
+  MAPPING(EINVAL),
+  MAPPING(ENFILE),
+  MAPPING(EMFILE),
+  MAPPING(ENOTTY),
+  MAPPING(ETXTBSY),
+  MAPPING(EFBIG),
+  MAPPING(ENOSPC),
+  MAPPING(ESPIPE),
+  MAPPING(EROFS),
+  MAPPING(EMLINK),
+  MAPPING(EPIPE),
+  MAPPING(EDOM),
+  MAPPING(ERANGE),
+  MAPPING(ELOOP),
+  MAPPING(ENAMETOOLONG),
+  MAPPING(ENOTEMPTY),
+  MAPPING(EOVERFLOW),
+  {-1, NULL}
+};
+
+static jclass enum_class;
+static jmethodID enum_valueOf;
+static jclass errno_class;
+
+void errno_enum_init(JNIEnv *env) {
+  if (enum_class != NULL) return;
+
+  enum_class = (*env)->FindClass(env, "java/lang/Enum");
+  PASS_EXCEPTIONS(env);
+  enum_class = (*env)->NewGlobalRef(env, enum_class);
+  enum_valueOf = (*env)->GetStaticMethodID(env, enum_class,
+    "valueOf", "(Ljava/lang/Class;Ljava/lang/String;)Ljava/lang/Enum;");
+  PASS_EXCEPTIONS(env);
+
+  errno_class = (*env)->FindClass(env, "org/apache/ratis/io/nativeio/Errno");
+  PASS_EXCEPTIONS(env);
+  errno_class = (*env)->NewGlobalRef(env, errno_class);
+}
+
+void errno_enum_deinit(JNIEnv *env) {
+  if (enum_class != NULL) {
+    (*env)->DeleteGlobalRef(env, enum_class);
+    enum_class = NULL;
+  }
+  if (errno_class != NULL) {
+    (*env)->DeleteGlobalRef(env, errno_class);
+    errno_class = NULL;
+  }
+  enum_valueOf = NULL;
+}
+
+
+static char *errno_to_string(int errnum) {
+  int i;
+  for (i = 0; ERRNO_MAPPINGS[i].errno_str != NULL; i++) {
+    if (ERRNO_MAPPINGS[i].errno_val == errnum)
+      return ERRNO_MAPPINGS[i].errno_str;
+  }
+  return "UNKNOWN";
+}
+
+jobject errno_to_enum(JNIEnv *env, int errnum) {
+  char *str = errno_to_string(errnum);
+  assert(str != NULL);
+
+  jstring jstr = (*env)->NewStringUTF(env, str);
+  PASS_EXCEPTIONS_RET(env, NULL);
+
+  return (*env)->CallStaticObjectMethod(
+    env, enum_class, enum_valueOf, errno_class, jstr);
+}

http://git-wip-us.apache.org/repos/asf/incubator-ratis/blob/7e71a2e0/ratis-common/src/main/native/src/org/apache/ratis/io/nativeio/errno_enum.h
----------------------------------------------------------------------
diff --git a/ratis-common/src/main/native/src/org/apache/ratis/io/nativeio/errno_enum.h b/ratis-common/src/main/native/src/org/apache/ratis/io/nativeio/errno_enum.h
new file mode 100644
index 0000000..1eee11a
--- /dev/null
+++ b/ratis-common/src/main/native/src/org/apache/ratis/io/nativeio/errno_enum.h
@@ -0,0 +1,27 @@
+/*
+ * 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.
+ */
+#ifndef ERRNO_ENUM_H
+#define ERRNO_ENUM_H
+
+#include <jni.h>
+
+void errno_enum_init(JNIEnv *env);
+void errno_enum_deinit(JNIEnv *env);
+jobject errno_to_enum(JNIEnv *env, int errnum);
+
+#endif

http://git-wip-us.apache.org/repos/asf/incubator-ratis/blob/7e71a2e0/ratis-common/src/main/native/src/org/apache/ratis/io/nativeio/file_descriptor.c
----------------------------------------------------------------------
diff --git a/ratis-common/src/main/native/src/org/apache/ratis/io/nativeio/file_descriptor.c b/ratis-common/src/main/native/src/org/apache/ratis/io/nativeio/file_descriptor.c
new file mode 100644
index 0000000..4ef7308
--- /dev/null
+++ b/ratis-common/src/main/native/src/org/apache/ratis/io/nativeio/file_descriptor.c
@@ -0,0 +1,115 @@
+/*
+ *  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.
+ */
+
+#include <jni.h>
+#include "file_descriptor.h"
+#include "org_apache_ratis.h"
+
+// class of java.io.FileDescriptor
+static jclass fd_class;
+// the internal field for the integer fd
+static jfieldID fd_descriptor;
+// the no-argument constructor
+static jmethodID fd_constructor;
+
+#ifdef WINDOWS
+// the internal field for the long handle
+static jfieldID fd_handle;
+#endif
+
+void fd_init(JNIEnv* env)
+{
+  if (fd_class != NULL) return; // already initted
+
+  fd_class = (*env)->FindClass(env, "java/io/FileDescriptor");
+  PASS_EXCEPTIONS(env);
+  fd_class = (*env)->NewGlobalRef(env, fd_class);
+
+  fd_descriptor = (*env)->GetFieldID(env, fd_class, "fd", "I");
+  PASS_EXCEPTIONS(env);
+
+#ifdef WINDOWS
+  fd_handle = (*env)->GetFieldID(env, fd_class, "handle", "J");
+  PASS_EXCEPTIONS(env);
+#endif
+
+  fd_constructor = (*env)->GetMethodID(env, fd_class, "<init>", "()V");
+}
+
+void fd_deinit(JNIEnv *env) {
+  if (fd_class != NULL) {
+    (*env)->DeleteGlobalRef(env, fd_class);
+    fd_class = NULL;
+  }
+  fd_descriptor = NULL;
+#ifdef WINDOWS
+  fd_handle = NULL;
+#endif
+  fd_constructor = NULL;
+}
+
+#ifdef UNIX
+/*
+ * Given an instance 'obj' of java.io.FileDescriptor, return the
+ * underlying fd, or throw if unavailable
+ */
+int fd_get(JNIEnv* env, jobject obj) {
+  if (obj == NULL) {
+    THROW(env, "java/lang/NullPointerException",
+          "FileDescriptor object is null");
+    return -1;
+  }
+  return (*env)->GetIntField(env, obj, fd_descriptor);
+}
+
+/*
+ * Create a FileDescriptor object corresponding to the given int fd
+ */
+jobject fd_create(JNIEnv *env, int fd) {
+  jobject obj = (*env)->NewObject(env, fd_class, fd_constructor);
+  PASS_EXCEPTIONS_RET(env, NULL);
+
+  (*env)->SetIntField(env, obj, fd_descriptor, fd);
+  return obj;
+}
+#endif
+
+#ifdef WINDOWS
+/*
+ * Given an instance 'obj' of java.io.FileDescriptor, return the
+ * underlying fd, or throw if unavailable
+ */
+long fd_get(JNIEnv* env, jobject obj) {
+  if (obj == NULL) {
+    THROW(env, "java/lang/NullPointerException",
+          "FileDescriptor object is null");
+    return -1;
+  }
+  return (long) (*env)->GetLongField(env, obj, fd_handle);
+}
+
+/*
+ * Create a FileDescriptor object corresponding to the given int fd
+ */
+jobject fd_create(JNIEnv *env, long fd) {
+  jobject obj = (*env)->NewObject(env, fd_class, fd_constructor);
+  PASS_EXCEPTIONS_RET(env, (jobject) NULL);
+
+  (*env)->SetLongField(env, obj, fd_handle, fd);
+  return obj;
+}
+#endif

http://git-wip-us.apache.org/repos/asf/incubator-ratis/blob/7e71a2e0/ratis-common/src/main/native/src/org/apache/ratis/io/nativeio/file_descriptor.h
----------------------------------------------------------------------
diff --git a/ratis-common/src/main/native/src/org/apache/ratis/io/nativeio/file_descriptor.h b/ratis-common/src/main/native/src/org/apache/ratis/io/nativeio/file_descriptor.h
new file mode 100644
index 0000000..565b4e1
--- /dev/null
+++ b/ratis-common/src/main/native/src/org/apache/ratis/io/nativeio/file_descriptor.h
@@ -0,0 +1,36 @@
+/*
+ *  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.
+ */
+#ifndef FILE_DESCRIPTOR_H
+#define FILE_DESCRIPTOR_H
+
+#include <jni.h>
+#include "org_apache_ratis.h"
+
+void fd_init(JNIEnv *env);
+void fd_deinit(JNIEnv *env);
+
+#ifdef UNIX
+int fd_get(JNIEnv* env, jobject obj);
+jobject fd_create(JNIEnv *env, int fd);
+#endif
+
+#ifdef WINDOWS
+long fd_get(JNIEnv* env, jobject obj);
+jobject fd_create(JNIEnv *env, long fd);
+#endif
+
+#endif

http://git-wip-us.apache.org/repos/asf/incubator-ratis/blob/7e71a2e0/ratis-common/src/main/native/src/org/apache/ratis/util/NativeCodeLoader.c
----------------------------------------------------------------------
diff --git a/ratis-common/src/main/native/src/org/apache/ratis/util/NativeCodeLoader.c b/ratis-common/src/main/native/src/org/apache/ratis/util/NativeCodeLoader.c
new file mode 100644
index 0000000..16b8980
--- /dev/null
+++ b/ratis-common/src/main/native/src/org/apache/ratis/util/NativeCodeLoader.c
@@ -0,0 +1,56 @@
+/*
+ * 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.
+ */
+
+#include "org_apache_ratis.h"
+
+#ifdef UNIX
+#include <dlfcn.h>
+#include "config.h"
+#endif // UNIX
+
+#ifdef WINDOWS
+#include "winutils.h"
+#endif
+
+#include <jni.h>
+
+JNIEXPORT jstring JNICALL Java_org_apache_ratis_util_NativeCodeLoader_getLibraryName
+  (JNIEnv *env, jclass clazz)
+{
+#ifdef UNIX
+  Dl_info dl_info;
+  int ret = dladdr(
+      Java_org_apache_ratis_util_NativeCodeLoader_getLibraryName,
+      &dl_info);
+  return (*env)->NewStringUTF(env, ret==0 ? "Unavailable" : dl_info.dli_fname);
+#endif
+
+#ifdef WINDOWS
+  LPWSTR filename = NULL;
+  GetLibraryName(Java_org_apache_ratis_util_NativeCodeLoader_getLibraryName,
+    &filename);
+  if (filename != NULL)
+  {
+    return (*env)->NewString(env, filename, (jsize) wcslen(filename));
+  }
+  else
+  {
+    return (*env)->NewStringUTF(env, "Unavailable");
+  }
+#endif
+}

http://git-wip-us.apache.org/repos/asf/incubator-ratis/blob/7e71a2e0/ratis-common/src/main/native/src/org/apache/ratis/util/NativeCrc32.c
----------------------------------------------------------------------
diff --git a/ratis-common/src/main/native/src/org/apache/ratis/util/NativeCrc32.c b/ratis-common/src/main/native/src/org/apache/ratis/util/NativeCrc32.c
new file mode 100644
index 0000000..2ed9ab1
--- /dev/null
+++ b/ratis-common/src/main/native/src/org/apache/ratis/util/NativeCrc32.c
@@ -0,0 +1,276 @@
+/*
+ * 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.
+ */
+
+#include "org_apache_ratis.h"
+#include "org_apache_ratis_util_NativeCrc32.h"
+
+#include <assert.h>
+#include <stdlib.h>
+#include <stdint.h>
+#include <string.h>
+
+#ifdef UNIX
+#include <inttypes.h>
+#include <arpa/inet.h>
+#include <unistd.h>
+#include "config.h"
+#include "gcc_optimizations.h"
+#endif // UNIX
+
+#include "bulk_crc32.h"
+
+#define MBYTE 1048576
+#define MIN(X,Y) ((X) < (Y) ? (X) : (Y))
+#define MAX(X,Y) ((X) > (Y) ? (X) : (Y))
+
+static void throw_checksum_exception(JNIEnv *env,
+    uint32_t got_crc, uint32_t expected_crc,
+    jstring j_filename, jlong pos) {
+  char message[1024];
+  jstring jstr_message;
+  char *filename;
+  jclass checksum_exception_clazz;
+  jmethodID checksum_exception_ctor;
+  jthrowable obj;
+
+  // Get filename as C string, or "null" if not provided
+  if (j_filename == NULL) {
+    filename = strdup("null");
+  } else {
+    const char *c_filename = (*env)->GetStringUTFChars(env, j_filename, NULL);
+    if (c_filename == NULL) {
+      return; // OOME already thrown
+    }
+    filename = strdup(c_filename);
+    (*env)->ReleaseStringUTFChars(env, j_filename, c_filename);
+  }
+
+  // Format error message
+#ifdef WINDOWS
+  _snprintf_s(
+	message,
+	sizeof(message),
+	_TRUNCATE,
+    "Checksum error: %s at %I64d exp: %d got: %d",
+    filename, pos, expected_crc, got_crc);
+#else
+  snprintf(message, sizeof(message),
+    "Checksum error: %s at %"PRId64" exp: %"PRId32" got: %"PRId32,
+    filename, pos, expected_crc, got_crc);
+#endif // WINDOWS
+
+  if ((jstr_message = (*env)->NewStringUTF(env, message)) == NULL) {
+    goto cleanup;
+  }
+ 
+  // Throw exception
+  checksum_exception_clazz = (*env)->FindClass(
+    env, "org/apache/ratis/protocol/ChecksumException");
+  if (checksum_exception_clazz == NULL) {
+    goto cleanup;
+  }
+
+  checksum_exception_ctor = (*env)->GetMethodID(env,
+    checksum_exception_clazz, "<init>",
+    "(Ljava/lang/String;J)V");
+  if (checksum_exception_ctor == NULL) {
+    goto cleanup;
+  }
+
+  obj = (jthrowable)(*env)->NewObject(env, checksum_exception_clazz,
+    checksum_exception_ctor, jstr_message, pos);
+  if (obj == NULL) goto cleanup;
+
+  (*env)->Throw(env, obj);
+
+cleanup:
+  if (filename != NULL) {
+    free(filename);
+  }
+}
+
+static int convert_java_crc_type(JNIEnv *env, jint crc_type) {
+  switch (crc_type) {
+    case org_apache_ratis_util_NativeCrc32_CHECKSUM_CRC32:
+      return CRC32_ZLIB_POLYNOMIAL;
+    case org_apache_ratis_util_NativeCrc32_CHECKSUM_CRC32C:
+      return CRC32C_POLYNOMIAL;
+    default:
+      THROW(env, "java/lang/IllegalArgumentException",
+        "Invalid checksum type");
+      return -1;
+  }
+}
+
+JNIEXPORT void JNICALL Java_org_apache_ratis_util_NativeCrc32_nativeComputeChunkedSums
+  (JNIEnv *env, jclass clazz,
+    jint bytes_per_checksum, jint j_crc_type,
+    jobject j_sums, jint sums_offset,
+    jobject j_data, jint data_offset, jint data_len,
+    jstring j_filename, jlong base_pos, jboolean verify)
+{
+  uint8_t *sums_addr;
+  uint8_t *data_addr;
+  uint32_t *sums;
+  uint8_t *data;
+  int crc_type;
+  crc32_error_t error_data;
+  int ret;
+
+  if (unlikely(!j_sums || !j_data)) {
+    THROW(env, "java/lang/NullPointerException",
+      "input ByteBuffers must not be null");
+    return;
+  }
+
+  // Convert direct byte buffers to C pointers
+  sums_addr = (*env)->GetDirectBufferAddress(env, j_sums);
+  data_addr = (*env)->GetDirectBufferAddress(env, j_data);
+
+  if (unlikely(!sums_addr || !data_addr)) {
+    THROW(env, "java/lang/IllegalArgumentException",
+      "input ByteBuffers must be direct buffers");
+    return;
+  }
+  if (unlikely(sums_offset < 0 || data_offset < 0 || data_len < 0)) {
+    THROW(env, "java/lang/IllegalArgumentException",
+      "bad offsets or lengths");
+    return;
+  }
+  if (unlikely(bytes_per_checksum) <= 0) {
+    THROW(env, "java/lang/IllegalArgumentException",
+      "invalid bytes_per_checksum");
+    return;
+  }
+
+  sums = (uint32_t *)(sums_addr + sums_offset);
+  data = data_addr + data_offset;
+
+  // Convert to correct internal C constant for CRC type
+  crc_type = convert_java_crc_type(env, j_crc_type);
+  if (crc_type == -1) return; // exception already thrown
+
+  // Setup complete. Actually verify checksums.
+  ret = bulk_crc(data, data_len, sums, crc_type,
+                            bytes_per_checksum, verify ? &error_data : NULL);
+  if (likely((verify && ret == CHECKSUMS_VALID) || (!verify && ret == 0))) {
+    return;
+  } else if (unlikely(verify && ret == INVALID_CHECKSUM_DETECTED)) {
+    long pos = base_pos + (error_data.bad_data - data);
+    throw_checksum_exception(
+      env, error_data.got_crc, error_data.expected_crc,
+      j_filename, pos);
+  } else {
+    THROW(env, "java/lang/AssertionError",
+      "Bad response code from native bulk_crc");
+  }
+}
+
+JNIEXPORT void JNICALL Java_org_apache_ratis_util_NativeCrc32_nativeVerifyChunkedSums
+  (JNIEnv *env, jclass clazz,
+    jint bytes_per_checksum, jint j_crc_type,
+    jobject j_sums, jint sums_offset,
+    jobject j_data, jint data_offset, jint data_len,
+    jstring j_filename, jlong base_pos)
+{
+  Java_org_apache_ratis_util_NativeCrc32_nativeComputeChunkedSums(env, clazz,
+    bytes_per_checksum, j_crc_type, j_sums, sums_offset, j_data, data_offset,
+    data_len, j_filename, base_pos, JNI_TRUE);
+}
+
+JNIEXPORT void JNICALL Java_org_apache_ratis_util_NativeCrc32_nativeComputeChunkedSumsByteArray
+  (JNIEnv *env, jclass clazz,
+    jint bytes_per_checksum, jint j_crc_type,
+    jarray j_sums, jint sums_offset,
+    jarray j_data, jint data_offset, jint data_len,
+    jstring j_filename, jlong base_pos, jboolean verify)
+{
+  uint8_t *sums_addr;
+  uint8_t *data_addr;
+  uint32_t *sums;
+  uint8_t *data;
+  int crc_type;
+  crc32_error_t error_data;
+  int ret;
+  int numChecksumsPerIter;
+  int checksumNum;
+
+  if (unlikely(!j_sums || !j_data)) {
+    THROW(env, "java/lang/NullPointerException",
+      "input byte arrays must not be null");
+    return;
+  }
+  if (unlikely(sums_offset < 0 || data_offset < 0 || data_len < 0)) {
+    THROW(env, "java/lang/IllegalArgumentException",
+      "bad offsets or lengths");
+    return;
+  }
+  if (unlikely(bytes_per_checksum) <= 0) {
+    THROW(env, "java/lang/IllegalArgumentException",
+      "invalid bytes_per_checksum");
+    return;
+  }
+
+  // Convert to correct internal C constant for CRC type
+  crc_type = convert_java_crc_type(env, j_crc_type);
+  if (crc_type == -1) return; // exception already thrown
+
+  numChecksumsPerIter = MAX(1, MBYTE / bytes_per_checksum);
+  checksumNum = 0;
+  while (checksumNum * bytes_per_checksum < data_len) {
+    // Convert byte arrays to C pointers
+    sums_addr = (*env)->GetPrimitiveArrayCritical(env, j_sums, NULL);
+    data_addr = (*env)->GetPrimitiveArrayCritical(env, j_data, NULL);
+
+    if (unlikely(!sums_addr || !data_addr)) {
+      if (data_addr) (*env)->ReleasePrimitiveArrayCritical(env, j_data, data_addr, 0);
+      if (sums_addr) (*env)->ReleasePrimitiveArrayCritical(env, j_sums, sums_addr, 0);
+      THROW(env, "java/lang/OutOfMemoryError",
+        "not enough memory for byte arrays in JNI code");
+      return;
+    }
+
+    sums = (uint32_t *)(sums_addr + sums_offset) + checksumNum;
+    data = data_addr + data_offset + checksumNum * bytes_per_checksum;
+
+    // Setup complete. Actually verify checksums.
+    ret = bulk_crc(data, MIN(numChecksumsPerIter * bytes_per_checksum,
+                             data_len - checksumNum * bytes_per_checksum),
+                   sums, crc_type, bytes_per_checksum, verify ? &error_data : NULL);
+    (*env)->ReleasePrimitiveArrayCritical(env, j_data, data_addr, 0);
+    (*env)->ReleasePrimitiveArrayCritical(env, j_sums, sums_addr, 0);
+    if (unlikely(verify && ret == INVALID_CHECKSUM_DETECTED)) {
+      long pos = base_pos + (error_data.bad_data - data) + checksumNum *
+        bytes_per_checksum;
+      throw_checksum_exception(
+        env, error_data.got_crc, error_data.expected_crc,
+        j_filename, pos);
+      return;
+    } else if (unlikely((verify && ret != CHECKSUMS_VALID) || (!verify && ret != 0))) {
+      THROW(env, "java/lang/AssertionError",
+        "Bad response code from native bulk_crc");
+      return;
+    }
+    checksumNum += numChecksumsPerIter;
+  }
+
+}
+
+/**
+ * vim: sw=2: ts=2: et:
+ */

http://git-wip-us.apache.org/repos/asf/incubator-ratis/blob/7e71a2e0/ratis-common/src/main/native/src/org/apache/ratis/util/bulk_crc32.c
----------------------------------------------------------------------
diff --git a/ratis-common/src/main/native/src/org/apache/ratis/util/bulk_crc32.c b/ratis-common/src/main/native/src/org/apache/ratis/util/bulk_crc32.c
new file mode 100644
index 0000000..5620487
--- /dev/null
+++ b/ratis-common/src/main/native/src/org/apache/ratis/util/bulk_crc32.c
@@ -0,0 +1,244 @@
+/*
+ * 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.
+ *
+ * Portions of this file are from http://www.evanjones.ca/crc32c.html under
+ * the BSD license:
+ *   Copyright 2008,2009,2010 Massachusetts Institute of Technology.
+ *   All rights reserved. Use of this source code is governed by a
+ *   BSD-style license that can be found in the LICENSE file.
+ */
+
+#include "org_apache_ratis.h"
+
+#include <assert.h>
+#include <errno.h>
+#include <stdint.h>
+
+#ifdef UNIX
+#include <arpa/inet.h>
+#include <unistd.h>
+#endif // UNIX
+
+#include "crc32_zlib_polynomial_tables.h"
+#include "crc32c_tables.h"
+#include "bulk_crc32.h"
+#include "gcc_optimizations.h"
+
+#define CRC_INITIAL_VAL 0xffffffff
+
+static uint32_t crc_val(uint32_t crc);
+
+typedef void (*crc_pipelined_func_t)(uint32_t *, uint32_t *, uint32_t *, const uint8_t *, size_t, int);
+
+// The software versions of pipelined crc
+static void pipelined_crc32c_sb8(uint32_t *crc1, uint32_t *crc2, uint32_t *crc3,
+                                 const uint8_t *p_buf, size_t block_size, int num_blocks);
+static void pipelined_crc32_zlib_sb8(uint32_t *crc1, uint32_t *crc2, uint32_t *crc3,
+                                 const uint8_t *p_buf, size_t block_size, int num_blocks);
+
+// Satically initialise the function pointers to the software versions
+// If a HW implementation is available they will subsequently be initialised in the dynamic
+// initialisers to point to the HW routines.
+crc_pipelined_func_t pipelined_crc32c_func = pipelined_crc32c_sb8;
+crc_pipelined_func_t pipelined_crc32_zlib_func = pipelined_crc32_zlib_sb8;
+
+static inline int store_or_verify(uint32_t *sums, uint32_t crc,
+                                   int is_verify) {
+  if (!is_verify) {
+    *sums = crc;
+    return 1;
+  } else {
+    return crc == *sums;
+  }
+}
+
+int bulk_crc(const uint8_t *data, size_t data_len,
+                    uint32_t *sums, int checksum_type,
+                    int bytes_per_checksum,
+                    crc32_error_t *error_info) {
+
+  int is_verify = error_info != NULL;
+
+  uint32_t crc1, crc2, crc3;
+  int n_blocks = data_len / bytes_per_checksum;
+  int remainder = data_len % bytes_per_checksum;
+  uint32_t crc;
+  crc_pipelined_func_t crc_pipelined_func;
+  switch (checksum_type) {
+    case CRC32_ZLIB_POLYNOMIAL:
+      crc_pipelined_func = pipelined_crc32_zlib_func;
+      break;
+    case CRC32C_POLYNOMIAL:
+      crc_pipelined_func = pipelined_crc32c_func;
+      break;
+    default:
+      return is_verify ? INVALID_CHECKSUM_TYPE : -EINVAL;
+  }
+
+  /* Process three blocks at a time */
+  while (likely(n_blocks >= 3)) {
+    crc1 = crc2 = crc3 = CRC_INITIAL_VAL;
+    crc_pipelined_func(&crc1, &crc2, &crc3, data, bytes_per_checksum, 3);
+
+    if (unlikely(!store_or_verify(sums, (crc = ntohl(crc_val(crc1))), is_verify)))
+      goto return_crc_error;
+    sums++;
+    data += bytes_per_checksum;
+    if (unlikely(!store_or_verify(sums, (crc = ntohl(crc_val(crc2))), is_verify)))
+      goto return_crc_error;
+    sums++;
+    data += bytes_per_checksum;
+    if (unlikely(!store_or_verify(sums, (crc = ntohl(crc_val(crc3))), is_verify)))
+      goto return_crc_error;
+    sums++;
+    data += bytes_per_checksum;
+    n_blocks -= 3;
+  }
+
+  /* One or two blocks */
+  if (n_blocks) {
+    crc1 = crc2 = crc3 = CRC_INITIAL_VAL;
+    crc_pipelined_func(&crc1, &crc2, &crc3, data, bytes_per_checksum, n_blocks);
+
+    if (unlikely(!store_or_verify(sums, (crc = ntohl(crc_val(crc1))), is_verify)))
+      goto return_crc_error;
+    data += bytes_per_checksum;
+    sums++;
+    if (n_blocks == 2) {
+      if (unlikely(!store_or_verify(sums, (crc = ntohl(crc_val(crc2))), is_verify)))
+        goto return_crc_error;
+      sums++;
+      data += bytes_per_checksum;
+    }
+  }
+
+  /* For something smaller than a block */
+  if (remainder) {
+    crc1 = crc2 = crc3 = CRC_INITIAL_VAL;
+    crc_pipelined_func(&crc1, &crc2, &crc3, data, remainder, 1);
+
+    if (unlikely(!store_or_verify(sums, (crc = ntohl(crc_val(crc1))), is_verify)))
+      goto return_crc_error;
+  }
+  return is_verify ? CHECKSUMS_VALID : 0;
+
+return_crc_error:
+  if (error_info != NULL) {
+    error_info->got_crc = crc;
+    error_info->expected_crc = *sums;
+    error_info->bad_data = data;
+  }
+  return INVALID_CHECKSUM_DETECTED;
+}
+
+/**
+ * Extract the final result of a CRC
+ */
+static uint32_t crc_val(uint32_t crc) {
+  return ~crc;
+}
+
+/**
+ * Computes the CRC32c checksum for the specified buffer using the slicing by 8 
+ * algorithm over 64 bit quantities.
+ */
+static uint32_t crc32c_sb8(uint32_t crc, const uint8_t *buf, size_t length) {
+  uint32_t running_length = ((length)/8)*8;
+  uint32_t end_bytes = length - running_length; 
+  int li;
+  for (li=0; li < running_length/8; li++) {
+	uint32_t term1;
+	uint32_t term2;
+    crc ^= *(uint32_t *)buf;
+    buf += 4;
+    term1 = CRC32C_T8_7[crc & 0x000000FF] ^
+        CRC32C_T8_6[(crc >> 8) & 0x000000FF];
+    term2 = crc >> 16;
+    crc = term1 ^
+        CRC32C_T8_5[term2 & 0x000000FF] ^ 
+        CRC32C_T8_4[(term2 >> 8) & 0x000000FF];
+    term1 = CRC32C_T8_3[(*(uint32_t *)buf) & 0x000000FF] ^
+        CRC32C_T8_2[((*(uint32_t *)buf) >> 8) & 0x000000FF];
+    
+    term2 = (*(uint32_t *)buf) >> 16;
+    crc =  crc ^ 
+        term1 ^    
+        CRC32C_T8_1[term2  & 0x000000FF] ^  
+        CRC32C_T8_0[(term2 >> 8) & 0x000000FF];  
+    buf += 4;
+  }
+  for (li=0; li < end_bytes; li++) {
+    crc = CRC32C_T8_0[(crc ^ *buf++) & 0x000000FF] ^ (crc >> 8);
+  }
+  return crc;    
+}
+
+static void pipelined_crc32c_sb8(uint32_t *crc1, uint32_t *crc2, uint32_t *crc3,
+                                 const uint8_t *p_buf, size_t block_size, int num_blocks) {
+  assert(num_blocks >= 1 && num_blocks <=3 && "invalid num_blocks");
+  *crc1 = crc32c_sb8(*crc1, p_buf, block_size);
+  if (num_blocks >= 2)
+    *crc2 = crc32c_sb8(*crc2, p_buf+block_size, block_size);
+  if (num_blocks >= 3)
+    *crc3 = crc32c_sb8(*crc3, p_buf+2*block_size, block_size);
+}
+
+/**
+ * Update a CRC using the "zlib" polynomial -- what Raft calls CHECKSUM_CRC32
+ * using slicing-by-8
+ */
+static uint32_t crc32_zlib_sb8(
+    uint32_t crc, const uint8_t *buf, size_t length) {
+  uint32_t running_length = ((length)/8)*8;
+  uint32_t end_bytes = length - running_length; 
+  int li;
+  for (li=0; li < running_length/8; li++) {
+	uint32_t term1;
+	uint32_t term2;
+    crc ^= *(uint32_t *)buf;
+    buf += 4;
+    term1 = CRC32_T8_7[crc & 0x000000FF] ^
+        CRC32_T8_6[(crc >> 8) & 0x000000FF];
+    term2 = crc >> 16;
+    crc = term1 ^
+        CRC32_T8_5[term2 & 0x000000FF] ^ 
+        CRC32_T8_4[(term2 >> 8) & 0x000000FF];
+    term1 = CRC32_T8_3[(*(uint32_t *)buf) & 0x000000FF] ^
+        CRC32_T8_2[((*(uint32_t *)buf) >> 8) & 0x000000FF];
+    
+    term2 = (*(uint32_t *)buf) >> 16;
+    crc =  crc ^ 
+        term1 ^    
+        CRC32_T8_1[term2  & 0x000000FF] ^  
+        CRC32_T8_0[(term2 >> 8) & 0x000000FF];  
+    buf += 4;
+  }
+  for (li=0; li < end_bytes; li++) {
+    crc = CRC32_T8_0[(crc ^ *buf++) & 0x000000FF] ^ (crc >> 8);
+  }
+  return crc;    
+}
+
+static void pipelined_crc32_zlib_sb8(uint32_t *crc1, uint32_t *crc2, uint32_t *crc3,
+                                     const uint8_t *p_buf, size_t block_size, int num_blocks) {
+  assert(num_blocks >= 1 && num_blocks <=3 && "invalid num_blocks");
+  *crc1 = crc32_zlib_sb8(*crc1, p_buf, block_size);
+  if (num_blocks >= 2)
+    *crc2 = crc32_zlib_sb8(*crc2, p_buf+block_size, block_size);
+  if (num_blocks >= 3)
+    *crc3 = crc32_zlib_sb8(*crc3, p_buf+2*block_size, block_size);
+}

http://git-wip-us.apache.org/repos/asf/incubator-ratis/blob/7e71a2e0/ratis-common/src/main/native/src/org/apache/ratis/util/bulk_crc32.h
----------------------------------------------------------------------
diff --git a/ratis-common/src/main/native/src/org/apache/ratis/util/bulk_crc32.h b/ratis-common/src/main/native/src/org/apache/ratis/util/bulk_crc32.h
new file mode 100644
index 0000000..b38a65a
--- /dev/null
+++ b/ratis-common/src/main/native/src/org/apache/ratis/util/bulk_crc32.h
@@ -0,0 +1,73 @@
+/*
+ * 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.
+ */
+#ifndef BULK_CRC32_H_INCLUDED
+#define BULK_CRC32_H_INCLUDED
+
+#include <stdint.h>
+
+#ifdef UNIX
+#include <unistd.h> /* for size_t */
+#endif // UNIX
+
+// Constants for different CRC algorithms
+#define CRC32C_POLYNOMIAL 1
+#define CRC32_ZLIB_POLYNOMIAL 2
+
+// Return codes for bulk_verify_crc
+#define CHECKSUMS_VALID 0
+#define INVALID_CHECKSUM_DETECTED -1
+#define INVALID_CHECKSUM_TYPE -2
+
+// Return type for bulk verification when verification fails
+typedef struct crc32_error {
+  uint32_t got_crc;
+  uint32_t expected_crc;
+  const uint8_t *bad_data; // pointer to start of data chunk with error
+} crc32_error_t;
+
+
+/**
+ * Either calculates checksums for or verifies a buffer of data.
+ * Checksums performed in chunks of bytes_per_checksum bytes. The checksums
+ * are each 32 bits and are stored in sequential indexes of the 'sums' array.
+ * Verification is done (sums is assumed to already contain the checksums)
+ * if error_info is non-null; otherwise calculation is done and checksums
+ * are stored into sums.
+ *
+ * @param data                  The data to checksum
+ * @param dataLen               Length of the data buffer
+ * @param sums                  (out param) buffer to write checksums into or
+ *                              where checksums are already stored.
+ *                              It must contain at least
+ *                              ((dataLen - 1) / bytes_per_checksum + 1) * 4 bytes.
+ * @param checksum_type         One of the CRC32 algorithm constants defined 
+ *                              above
+ * @param bytes_per_checksum    How many bytes of data to process per checksum.
+ * @param error_info            If non-NULL, verification will be performed and
+ *                              it will be filled in if an error
+ *                              is detected. Otherwise calculation is performed.
+ *
+ * @return                      0 for success, non-zero for an error, result codes
+ *                              for verification are defined above
+ */
+extern int bulk_crc(const uint8_t *data, size_t data_len,
+    uint32_t *sums, int checksum_type,
+    int bytes_per_checksum,
+    crc32_error_t *error_info);
+
+#endif

http://git-wip-us.apache.org/repos/asf/incubator-ratis/blob/7e71a2e0/ratis-common/src/main/native/src/org/apache/ratis/util/bulk_crc32_aarch64.c
----------------------------------------------------------------------
diff --git a/ratis-common/src/main/native/src/org/apache/ratis/util/bulk_crc32_aarch64.c b/ratis-common/src/main/native/src/org/apache/ratis/util/bulk_crc32_aarch64.c
new file mode 100644
index 0000000..ab4690b
--- /dev/null
+++ b/ratis-common/src/main/native/src/org/apache/ratis/util/bulk_crc32_aarch64.c
@@ -0,0 +1,362 @@
+/*
+ * 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.
+ */
+
+#include <assert.h>
+#include <stddef.h>    // for size_t
+
+#include  "bulk_crc32.h"
+#include "gcc_optimizations.h"
+
+/**
+ * Hardware-accelerated CRC32 calculation using the 64-bit instructions.
+ * 2 variants:-
+ *   pipelined_crc32c uses the Castagnoli polynomial 0x1EDC6F41
+ *   pipelined_crc32_zlib uses the Zlib polynomial 0x04C11DB7
+ */
+
+// gcc doesn't know how to vectorize a 128 bit load, so use the following to tell it
+#define LDP(x,y,p) asm("ldp %x[a], %x[b], [%x[c]], #16" : [a]"=r"(x),[b]"=r"(y),[c]"+r"(p))
+
+#define CRC32CX(crc,value) asm("crc32cx %w[c], %w[c], %x[v]" : [c]"+r"(*&crc) : [v]"r"(+value))
+#define CRC32CW(crc,value) asm("crc32cw %w[c], %w[c], %w[v]" : [c]"+r"(*&crc) : [v]"r"(+value))
+#define CRC32CH(crc,value) asm("crc32ch %w[c], %w[c], %w[v]" : [c]"+r"(*&crc) : [v]"r"(+value))
+#define CRC32CB(crc,value) asm("crc32cb %w[c], %w[c], %w[v]" : [c]"+r"(*&crc) : [v]"r"(+value))
+
+#define CRC32ZX(crc,value) asm("crc32x %w[c], %w[c], %x[v]" : [c]"+r"(crc) : [v]"r"(value))
+#define CRC32ZW(crc,value) asm("crc32w %w[c], %w[c], %w[v]" : [c]"+r"(crc) : [v]"r"(value))
+#define CRC32ZH(crc,value) asm("crc32h %w[c], %w[c], %w[v]" : [c]"+r"(crc) : [v]"r"(value))
+#define CRC32ZB(crc,value) asm("crc32b %w[c], %w[c], %w[v]" : [c]"+r"(crc) : [v]"r"(value))
+
+/**
+ * Pipelined version of hardware-accelerated CRC32 calculation using
+ * the 64 bit crc32 instructions. 
+ * One crc32 instruction takes three cycles, but two more with no data
+ * dependency can be in the pipeline to achieve something close to single 
+ * instruction/cycle. Here we feed three blocks in RR.
+ *
+ * 2 variants:-
+ *   pipelined_crc32c uses the Castagnoli polynomial 0x1EDC6F41
+ *   pipelined_crc32_zlib uses the Zlib polynomial 0x04C11DB7
+ *
+ *   crc1, crc2, crc3 : Store initial checksum for each block before
+ *           calling. When it returns, updated checksums are stored.
+ *   p_buf : The base address of the data buffer. The buffer should be
+ *           at least as big as block_size * num_blocks.
+ *   block_size : The size of each block in bytes.
+ *   num_blocks : The number of blocks to work on. Min = 1, Max = 3
+ */
+static void pipelined_crc32c(uint32_t *crc1, uint32_t *crc2, uint32_t *crc3, const uint8_t *p_buf1, size_t block_size, int num_blocks) {
+  uint64_t c1 = *crc1;
+  uint64_t c2 = *crc2;
+  uint64_t c3 = *crc3;
+  const uint8_t *p_buf2 = p_buf1 + block_size;
+  const uint8_t *p_buf3 = p_buf1 + block_size * 2;
+  uint64_t x1, y1, x2, y2, x3, y3;
+  long len = block_size;
+
+  /* We do switch here because the loop has to be tight in order
+   * to fill the pipeline. Any other statement inside the loop
+   * or inbetween crc32 instruction can slow things down.
+   *
+   * Do verify that this code generates the expected assembler
+   * by disassembling test_bulk_crc32
+   */
+
+  asm(".cpu generic+crc");	// Allow crc instructions in asm
+  switch (num_blocks) {
+    case 3:
+      /* Do three blocks */
+      while ((len -= 2*sizeof(uint64_t)) >= 0) {
+        LDP(x1,y1,p_buf1);
+        LDP(x2,y2,p_buf2);
+        LDP(x3,y3,p_buf3);
+        CRC32CX(c1, x1);
+        CRC32CX(c2, x2);
+        CRC32CX(c3, x3);
+        CRC32CX(c1, y1);
+        CRC32CX(c2, y2);
+        CRC32CX(c3, y3);
+      }
+
+      if (unlikely(len & sizeof(uint64_t))) {
+        x1 = *(uint64_t*)p_buf1; p_buf1 += sizeof(uint64_t);
+        x2 = *(uint64_t*)p_buf2; p_buf2 += sizeof(uint64_t);
+        x3 = *(uint64_t*)p_buf3; p_buf3 += sizeof(uint64_t);
+        CRC32CX(c1, x1);
+        CRC32CX(c2, x2);
+        CRC32CX(c3, x3);
+      }
+      if (unlikely(len & sizeof(uint32_t))) {
+        x1 = *(uint32_t*)p_buf1; p_buf1 += sizeof(uint32_t);
+        x2 = *(uint32_t*)p_buf2; p_buf2 += sizeof(uint32_t);
+        x3 = *(uint32_t*)p_buf3; p_buf3 += sizeof(uint32_t);
+        CRC32CW(c1, x1);
+        CRC32CW(c2, x2);
+        CRC32CW(c3, x3);
+      }
+      if (unlikely(len & sizeof(uint16_t))) {
+        x1 = *(uint16_t*)p_buf1; p_buf1 += sizeof(uint16_t);
+        x2 = *(uint16_t*)p_buf2; p_buf2 += sizeof(uint16_t);
+        x3 = *(uint16_t*)p_buf3; p_buf3 += sizeof(uint16_t);
+        CRC32CH(c1, x1);
+        CRC32CH(c2, x2);
+        CRC32CH(c3, x3);
+      }
+      if (unlikely(len & sizeof(uint8_t))) {
+        x1 = *p_buf1;
+        x2 = *p_buf2;
+        x3 = *p_buf3;
+        CRC32CB(c1, x1);
+        CRC32CB(c2, x2);
+        CRC32CB(c3, x3);
+      }
+      break;
+    case 2:
+      /* Do two blocks */
+      while ((len -= 2*sizeof(uint64_t)) >= 0) {
+        LDP(x1,y1,p_buf1);
+        LDP(x2,y2,p_buf2);
+        CRC32CX(c1, x1);
+        CRC32CX(c2, x2);
+        CRC32CX(c1, y1);
+        CRC32CX(c2, y2);
+      }
+
+      if (unlikely(len & sizeof(uint64_t))) {
+        x1 = *(uint64_t*)p_buf1; p_buf1 += sizeof(uint64_t);
+        x2 = *(uint64_t*)p_buf2; p_buf2 += sizeof(uint64_t);
+        CRC32CX(c1, x1);
+        CRC32CX(c2, x2);
+      }
+      if (unlikely(len & sizeof(uint32_t))) {
+        x1 = *(uint32_t*)p_buf1; p_buf1 += sizeof(uint32_t);
+        x2 = *(uint32_t*)p_buf2; p_buf2 += sizeof(uint32_t);
+        CRC32CW(c1, x1);
+        CRC32CW(c2, x2);
+      }
+      if (unlikely(len & sizeof(uint16_t))) {
+        x1 = *(uint16_t*)p_buf1; p_buf1 += sizeof(uint16_t);
+        x2 = *(uint16_t*)p_buf2; p_buf2 += sizeof(uint16_t);
+        CRC32CH(c1, x1);
+        CRC32CH(c2, x2);
+      }
+      if (unlikely(len & sizeof(uint8_t))) {
+        x1 = *p_buf1;
+        x2 = *p_buf2;
+        CRC32CB(c1, x1);
+        CRC32CB(c2, x2);
+      }
+      break;
+    case 1:
+      /* single block */
+      while ((len -= 2*sizeof(uint64_t)) >= 0) {
+        LDP(x1,y1,p_buf1);
+        CRC32CX(c1, x1);
+        CRC32CX(c1, y1);
+      }
+
+      if (unlikely(len & sizeof(uint64_t))) {
+        x1 = *(uint64_t*)p_buf1; p_buf1 += sizeof(uint64_t);
+        CRC32CX(c1, x1);
+      }
+      if (unlikely(len & sizeof(uint32_t))) {
+        x1 = *(uint32_t*)p_buf1; p_buf1 += sizeof(uint32_t);
+        CRC32CW(c1, x1);
+      }
+      if (unlikely(len & sizeof(uint16_t))) {
+        x1 = *(uint16_t*)p_buf1; p_buf1 += sizeof(uint16_t);
+        CRC32CH(c1, x1);
+      }
+      if (unlikely(len & sizeof(uint8_t))) {
+        x1 = *p_buf1;
+        CRC32CB(c1, x1);
+      }
+      break;
+    case 0:
+      return;
+    default:
+      assert(0 && "BUG: Invalid number of checksum blocks");
+  }
+
+  *crc1 = c1;
+  *crc2 = c2;
+  *crc3 = c3;
+  return;
+}
+
+static void pipelined_crc32_zlib(uint32_t *crc1, uint32_t *crc2, uint32_t *crc3, const uint8_t *p_buf1, size_t block_size, int num_blocks) {
+  uint64_t c1 = *crc1;
+  uint64_t c2 = *crc2;
+  uint64_t c3 = *crc3;
+  const uint8_t *p_buf2 = p_buf1 + block_size;
+  const uint8_t *p_buf3 = p_buf1 + block_size * 2;
+  uint64_t x1, y1, x2, y2, x3, y3;
+  long len = block_size;
+
+  /* We do switch here because the loop has to be tight in order
+   * to fill the pipeline. Any other statement inside the loop
+   * or inbetween crc32 instruction can slow things down.
+   *
+   * Do verify that this code generates the expected assembler
+   * by disassembling test_bulk_crc32
+   */
+
+  asm(".cpu generic+crc");	// Allow crc instructions in asm
+  switch (num_blocks) {
+    case 3:
+      /* Do three blocks */
+      while ((len -= 2*sizeof(uint64_t)) >= 0) {
+        LDP(x1,y1,p_buf1);
+        LDP(x2,y2,p_buf2);
+        LDP(x3,y3,p_buf3);
+        CRC32ZX(c1, x1);
+        CRC32ZX(c2, x2);
+        CRC32ZX(c3, x3);
+        CRC32ZX(c1, y1);
+        CRC32ZX(c2, y2);
+        CRC32ZX(c3, y3);
+      }
+
+      if (unlikely(len & sizeof(uint64_t))) {
+        x1 = *(uint64_t*)p_buf1; p_buf1 += sizeof(uint64_t);
+        x2 = *(uint64_t*)p_buf2; p_buf2 += sizeof(uint64_t);
+        x3 = *(uint64_t*)p_buf3; p_buf3 += sizeof(uint64_t);
+        CRC32ZX(c1, x1);
+        CRC32ZX(c2, x2);
+        CRC32ZX(c3, x3);
+      }
+      if (unlikely(len & sizeof(uint32_t))) {
+        x1 = *(uint32_t*)p_buf1; p_buf1 += sizeof(uint32_t);
+        x2 = *(uint32_t*)p_buf2; p_buf2 += sizeof(uint32_t);
+        x3 = *(uint32_t*)p_buf3; p_buf3 += sizeof(uint32_t);
+        CRC32ZW(c1, x1);
+        CRC32ZW(c2, x2);
+        CRC32ZW(c3, x3);
+      }
+      if (unlikely(len & sizeof(uint16_t))) {
+        x1 = *(uint16_t*)p_buf1; p_buf1 += sizeof(uint16_t);
+        x2 = *(uint16_t*)p_buf2; p_buf2 += sizeof(uint16_t);
+        x3 = *(uint16_t*)p_buf3; p_buf3 += sizeof(uint16_t);
+        CRC32ZH(c1, x1);
+        CRC32ZH(c2, x2);
+        CRC32ZH(c3, x3);
+      }
+      if (unlikely(len & sizeof(uint8_t))) {
+        x1 = *p_buf1;
+        x2 = *p_buf2;
+        x3 = *p_buf3;
+        CRC32ZB(c1, x1);
+        CRC32ZB(c2, x2);
+        CRC32ZB(c3, x3);
+      }
+      break;
+    case 2:
+      /* Do two blocks */
+      while ((len -= 2*sizeof(uint64_t)) >= 0) {
+        LDP(x1,y1,p_buf1);
+        LDP(x2,y2,p_buf2);
+        CRC32ZX(c1, x1);
+        CRC32ZX(c2, x2);
+        CRC32ZX(c1, y1);
+        CRC32ZX(c2, y2);
+      }
+
+      if (unlikely(len & sizeof(uint64_t))) {
+        x1 = *(uint64_t*)p_buf1; p_buf1 += sizeof(uint64_t);
+        x2 = *(uint64_t*)p_buf2; p_buf2 += sizeof(uint64_t);
+        CRC32ZX(c1, x1);
+        CRC32ZX(c2, x2);
+      }
+      if (unlikely(len & sizeof(uint32_t))) {
+        x1 = *(uint32_t*)p_buf1; p_buf1 += sizeof(uint32_t);
+        x2 = *(uint32_t*)p_buf2; p_buf2 += sizeof(uint32_t);
+        CRC32ZW(c1, x1);
+        CRC32ZW(c2, x2);
+      }
+      if (unlikely(len & sizeof(uint16_t))) {
+        x1 = *(uint16_t*)p_buf1; p_buf1 += sizeof(uint16_t);
+        x2 = *(uint16_t*)p_buf2; p_buf2 += sizeof(uint16_t);
+        CRC32ZH(c1, x1);
+        CRC32ZH(c2, x2);
+      }
+      if (unlikely(len & sizeof(uint8_t))) {
+        x1 = *p_buf1;
+        x2 = *p_buf2;
+        CRC32ZB(c1, x1);
+        CRC32ZB(c2, x2);
+      }
+      break;
+    case 1:
+      /* single block */
+      while ((len -= 2*sizeof(uint64_t)) >= 0) {
+        LDP(x1,y1,p_buf1);
+        CRC32ZX(c1, x1);
+        CRC32ZX(c1, y1);
+      }
+
+      if (unlikely(len & sizeof(uint64_t))) {
+        x1 = *(uint64_t*)p_buf1; p_buf1 += sizeof(uint64_t);
+        CRC32ZX(c1, x1);
+      }
+      if (unlikely(len & sizeof(uint32_t))) {
+        x1 = *(uint32_t*)p_buf1; p_buf1 += sizeof(uint32_t);
+        CRC32ZW(c1, x1);
+      }
+      if (unlikely(len & sizeof(uint16_t))) {
+        x1 = *(uint16_t*)p_buf1; p_buf1 += sizeof(uint16_t);
+        CRC32ZH(c1, x1);
+      }
+      if (unlikely(len & sizeof(uint8_t))) {
+        x1 = *p_buf1;
+        CRC32ZB(c1, x1);
+      }
+      break;
+    case 0:
+      return;
+    default:
+      assert(0 && "BUG: Invalid number of checksum blocks");
+  }
+
+  *crc1 = c1;
+  *crc2 = c2;
+  *crc3 = c3;
+  return;
+}
+
+typedef void (*crc_pipelined_func_t)(uint32_t *, uint32_t *, uint32_t *, const uint8_t *, size_t, int);
+extern crc_pipelined_func_t pipelined_crc32c_func;
+extern crc_pipelined_func_t pipelined_crc32_zlib_func;
+
+#include <sys/auxv.h>
+#include <asm/hwcap.h>
+
+#ifndef HWCAP_CRC32
+#define HWCAP_CRC32 (1 << 7)
+#endif
+
+/**
+ * On library load, determine what sort of crc we are going to do
+ * and set crc function pointers appropriately.
+ */
+void __attribute__ ((constructor)) init_cpu_support_flag(void) {
+  unsigned long auxv = getauxval(AT_HWCAP);
+  if (auxv & HWCAP_CRC32) {
+    pipelined_crc32c_func = pipelined_crc32c;
+    pipelined_crc32_zlib_func = pipelined_crc32_zlib;
+  }
+}

http://git-wip-us.apache.org/repos/asf/incubator-ratis/blob/7e71a2e0/ratis-common/src/main/native/src/org/apache/ratis/util/bulk_crc32_x86.c
----------------------------------------------------------------------
diff --git a/ratis-common/src/main/native/src/org/apache/ratis/util/bulk_crc32_x86.c b/ratis-common/src/main/native/src/org/apache/ratis/util/bulk_crc32_x86.c
new file mode 100644
index 0000000..290b8a6
--- /dev/null
+++ b/ratis-common/src/main/native/src/org/apache/ratis/util/bulk_crc32_x86.c
@@ -0,0 +1,345 @@
+/*
+ * 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.
+ *
+ * Portions of this file are from http://www.evanjones.ca/crc32c.html under
+ * the BSD license:
+ *   Copyright 2008,2009,2010 Massachusetts Institute of Technology.
+ *   All rights reserved. Use of this source code is governed by a
+ *   BSD-style license that can be found in the LICENSE file.
+ */
+
+#include <assert.h>
+#include <stddef.h>    // for size_t
+
+#include  "bulk_crc32.h"
+#include "gcc_optimizations.h"
+#include "gcc_optimizations.h"
+
+///////////////////////////////////////////////////////////////////////////
+// Begin code for SSE4.2 specific hardware support of CRC32C
+///////////////////////////////////////////////////////////////////////////
+
+#  define SSE42_FEATURE_BIT (1 << 20)
+#  define CPUID_FEATURES 1
+/**
+ * Call the cpuid instruction to determine CPU feature flags.
+ */
+static uint32_t cpuid(uint32_t eax_in) {
+  uint32_t eax, ebx, ecx, edx;
+#  if defined(__PIC__) && !defined(__LP64__)
+// 32-bit PIC code uses the ebx register for the base offset --
+// have to save and restore it on the stack
+  asm("pushl %%ebx\n\t"
+      "cpuid\n\t"
+      "movl %%ebx, %[ebx]\n\t"
+      "popl %%ebx" : "=a" (eax), [ebx] "=r"(ebx),  "=c"(ecx), "=d"(edx) : "a" (eax_in)
+      : "cc");
+#  else
+  asm("cpuid" : "=a" (eax), "=b"(ebx), "=c"(ecx), "=d"(edx) : "a"(eax_in)
+      : "cc");
+#  endif
+
+  return ecx;
+}
+
+//
+// Definitions of the SSE4.2 crc32 operations. Using these instead of
+// the GCC __builtin_* intrinsics allows this code to compile without
+// -msse4.2, since we do dynamic CPU detection at runtime.
+//
+
+#  ifdef __LP64__
+inline uint64_t _mm_crc32_u64(uint64_t crc, uint64_t value) {
+  asm("crc32q %[value], %[crc]\n" : [crc] "+r" (crc) : [value] "rm" (value));
+  return crc;
+}
+#  endif
+
+inline uint32_t _mm_crc32_u32(uint32_t crc, uint32_t value) {
+  asm("crc32l %[value], %[crc]\n" : [crc] "+r" (crc) : [value] "rm" (value));
+  return crc;
+}
+
+inline uint32_t _mm_crc32_u16(uint32_t crc, uint16_t value) {
+  asm("crc32w %[value], %[crc]\n" : [crc] "+r" (crc) : [value] "rm" (value));
+  return crc;
+}
+
+inline uint32_t _mm_crc32_u8(uint32_t crc, uint8_t value) {
+  asm("crc32b %[value], %[crc]\n" : [crc] "+r" (crc) : [value] "rm" (value));
+  return crc;
+}
+
+#  ifdef __LP64__
+/**
+ * Pipelined version of hardware-accelerated CRC32C calculation using
+ * the 64 bit crc32q instruction. 
+ * One crc32c instruction takes three cycles, but two more with no data
+ * dependency can be in the pipeline to achieve something close to single 
+ * instruction/cycle. Here we feed three blocks in RR.
+ *
+ *   crc1, crc2, crc3 : Store initial checksum for each block before
+ *           calling. When it returns, updated checksums are stored.
+ *   p_buf : The base address of the data buffer. The buffer should be
+ *           at least as big as block_size * num_blocks.
+ *   block_size : The size of each block in bytes.
+ *   num_blocks : The number of blocks to work on. Min = 1, Max = 3
+ */
+static void pipelined_crc32c(uint32_t *crc1, uint32_t *crc2, uint32_t *crc3, const uint8_t *p_buf, size_t block_size, int num_blocks) {
+  uint64_t c1 = *crc1;
+  uint64_t c2 = *crc2;
+  uint64_t c3 = *crc3;
+  uint64_t *data = (uint64_t*)p_buf;
+  int counter = block_size / sizeof(uint64_t);
+  int remainder = block_size % sizeof(uint64_t);
+  uint8_t *bdata;
+
+  /* We do switch here because the loop has to be tight in order
+   * to fill the pipeline. Any other statement inside the loop
+   * or inbetween crc32 instruction can slow things down. Calling
+   * individual crc32 instructions three times from C also causes
+   * gcc to insert other instructions inbetween.
+   *
+   * Do not rearrange the following code unless you have verified
+   * the generated machine code is as efficient as before.
+   */
+  switch (num_blocks) {
+    case 3:
+      /* Do three blocks */
+      while (likely(counter)) {
+        __asm__ __volatile__(
+        "crc32q (%7), %0;\n\t"
+        "crc32q (%7,%6,1), %1;\n\t"
+        "crc32q (%7,%6,2), %2;\n\t"
+         : "=r"(c1), "=r"(c2), "=r"(c3)
+         : "0"(c1), "1"(c2), "2"(c3), "r"(block_size), "r"(data)
+        );
+        data++;
+        counter--;
+      }
+
+      /* Take care of the remainder. They are only up to seven bytes,
+       * so performing byte-level crc32 won't take much time.
+       */
+      bdata = (uint8_t*)data;
+      while (likely(remainder)) {
+        __asm__ __volatile__(
+        "crc32b (%7), %0;\n\t"
+        "crc32b (%7,%6,1), %1;\n\t"
+        "crc32b (%7,%6,2), %2;\n\t"
+         : "=r"(c1), "=r"(c2), "=r"(c3)
+         : "0"(c1), "1"(c2), "2"(c3), "r"(block_size), "r"(bdata)
+        );
+        bdata++;
+        remainder--;
+      }
+      break;
+    case 2:
+      /* Do two blocks */
+      while (likely(counter)) {
+        __asm__ __volatile__(
+        "crc32q (%5), %0;\n\t"
+        "crc32q (%5,%4,1), %1;\n\t"
+         : "=r"(c1), "=r"(c2) 
+         : "0"(c1), "1"(c2), "r"(block_size), "r"(data)
+        );
+        data++;
+        counter--;
+      }
+
+      bdata = (uint8_t*)data;
+      while (likely(remainder)) {
+        __asm__ __volatile__(
+        "crc32b (%5), %0;\n\t"
+        "crc32b (%5,%4,1), %1;\n\t"
+         : "=r"(c1), "=r"(c2) 
+         : "0"(c1), "1"(c2), "r"(block_size), "r"(bdata)
+        );
+        bdata++;
+        remainder--;
+      }
+      break;
+    case 1:
+      /* single block */
+      while (likely(counter)) {
+        __asm__ __volatile__(
+        "crc32q (%2), %0;\n\t"
+         : "=r"(c1) 
+         : "0"(c1), "r"(data)
+        );
+        data++;
+        counter--;
+      }
+      bdata = (uint8_t*)data;
+      while (likely(remainder)) {
+        __asm__ __volatile__(
+        "crc32b (%2), %0;\n\t"
+         : "=r"(c1) 
+         : "0"(c1), "r"(bdata)
+        );
+        bdata++;
+        remainder--;
+      }
+      break;
+    case 0:
+      return;
+    default:
+      assert(0 && "BUG: Invalid number of checksum blocks");
+  }
+
+  *crc1 = c1;
+  *crc2 = c2;
+  *crc3 = c3;
+  return;
+}
+
+# else  // 32-bit
+
+/**
+ * Pipelined version of hardware-accelerated CRC32C calculation using
+ * the 32 bit crc32l instruction. 
+ * One crc32c instruction takes three cycles, but two more with no data
+ * dependency can be in the pipeline to achieve something close to single 
+ * instruction/cycle. Here we feed three blocks in RR.
+ *
+ *   crc1, crc2, crc3 : Store initial checksum for each block before
+ *                calling. When it returns, updated checksums are stored.
+ *   data       : The base address of the data buffer. The buffer should be
+ *                at least as big as block_size * num_blocks.
+ *   block_size : The size of each block in bytes. 
+ *   num_blocks : The number of blocks to work on. Min = 1, Max = 3
+ */
+static void pipelined_crc32c(uint32_t *crc1, uint32_t *crc2, uint32_t *crc3, const uint8_t *p_buf, size_t block_size, int num_blocks) {
+  uint32_t c1 = *crc1;
+  uint32_t c2 = *crc2;
+  uint32_t c3 = *crc3;
+  int counter = block_size / sizeof(uint32_t);
+  int remainder = block_size % sizeof(uint32_t);
+  uint32_t *data = (uint32_t*)p_buf;
+  uint8_t *bdata;
+
+  /* We do switch here because the loop has to be tight in order
+   * to fill the pipeline. Any other statement inside the loop
+   * or inbetween crc32 instruction can slow things down. Calling
+   * individual crc32 instructions three times from C also causes
+   * gcc to insert other instructions inbetween.
+   *
+   * Do not rearrange the following code unless you have verified
+   * the generated machine code is as efficient as before.
+   */
+  switch (num_blocks) {
+    case 3:
+      /* Do three blocks */
+      while (likely(counter)) {
+        __asm__ __volatile__(
+        "crc32l (%7), %0;\n\t"
+        "crc32l (%7,%6,1), %1;\n\t"
+        "crc32l (%7,%6,2), %2;\n\t"
+         : "=r"(c1), "=r"(c2), "=r"(c3)
+         : "r"(c1), "r"(c2), "r"(c3), "r"(block_size), "r"(data)
+        );
+        data++;
+        counter--;
+      }
+      /* Take care of the remainder. They are only up to three bytes,
+       * so performing byte-level crc32 won't take much time.
+       */
+      bdata = (uint8_t*)data;
+      while (likely(remainder)) {
+        __asm__ __volatile__(
+        "crc32b (%7), %0;\n\t"
+        "crc32b (%7,%6,1), %1;\n\t"
+        "crc32b (%7,%6,2), %2;\n\t"
+         : "=r"(c1), "=r"(c2), "=r"(c3)
+         : "r"(c1), "r"(c2), "r"(c3), "r"(block_size), "r"(bdata)
+        );
+        bdata++;
+        remainder--;
+      }
+      break;
+    case 2:
+      /* Do two blocks */
+      while (likely(counter)) {
+        __asm__ __volatile__(
+        "crc32l (%5), %0;\n\t"
+        "crc32l (%5,%4,1), %1;\n\t"
+         : "=r"(c1), "=r"(c2) 
+         : "r"(c1), "r"(c2), "r"(block_size), "r"(data)
+        );
+        data++;
+        counter--;
+      }
+
+      bdata = (uint8_t*)data;
+      while (likely(remainder)) {
+        __asm__ __volatile__(
+        "crc32b (%5), %0;\n\t"
+        "crc32b (%5,%4,1), %1;\n\t"
+         : "=r"(c1), "=r"(c2) 
+         : "r"(c1), "r"(c2), "r"(block_size), "r"(bdata)
+        );
+        bdata++;
+        remainder--;
+      }
+      break;
+    case 1:
+      /* single block */
+      while (likely(counter)) {
+        __asm__ __volatile__(
+        "crc32l (%2), %0;\n\t"
+         : "=r"(c1) 
+         : "r"(c1), "r"(data)
+        );
+        data++;
+        counter--;
+      }
+      bdata = (uint8_t*)data;
+      while (likely(remainder)) {
+        __asm__ __volatile__(
+        "crc32b (%2), %0;\n\t"
+         : "=r"(c1) 
+         : "r"(c1), "r"(bdata)
+        );
+        bdata++;
+        remainder--;
+      }
+      break;
+    case 0:
+       return;
+    default:
+      assert(0 && "BUG: Invalid number of checksum blocks");
+  }
+
+  *crc1 = c1;
+  *crc2 = c2;
+  *crc3 = c3;
+  return;
+}
+
+# endif // 64-bit vs 32-bit
+
+/**
+ * On library load, initiailize the cached function pointer
+ * if cpu supports SSE4.2's crc32 instruction.
+ */
+typedef void (*crc_pipelined_func_t)(uint32_t *, uint32_t *, uint32_t *, const uint8_t *, size_t, int);
+extern crc_pipelined_func_t pipelined_crc32c_func;
+
+void __attribute__ ((constructor)) init_cpu_support_flag(void) {
+  uint32_t ecx = cpuid(CPUID_FEATURES);
+  if (ecx & SSE42_FEATURE_BIT) pipelined_crc32c_func = pipelined_crc32c;
+}

http://git-wip-us.apache.org/repos/asf/incubator-ratis/blob/7e71a2e0/ratis-common/src/main/native/src/org/apache/ratis/util/crc32_zlib_polynomial_tables.h
----------------------------------------------------------------------
diff --git a/ratis-common/src/main/native/src/org/apache/ratis/util/crc32_zlib_polynomial_tables.h b/ratis-common/src/main/native/src/org/apache/ratis/util/crc32_zlib_polynomial_tables.h
new file mode 100644
index 0000000..59d8f4d
--- /dev/null
+++ b/ratis-common/src/main/native/src/org/apache/ratis/util/crc32_zlib_polynomial_tables.h
@@ -0,0 +1,552 @@
+/*
+ * 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.
+ */
+
+/*
+ * CRC-32 lookup tables generated by the polynomial 0xEDB88320.
+ * See also TestPureJavaCrc32.Table.
+ */
+const uint32_t CRC32_T8_0[] = {
+  0x00000000, 0x77073096, 0xEE0E612C, 0x990951BA, 
+  0x076DC419, 0x706AF48F, 0xE963A535, 0x9E6495A3, 
+  0x0EDB8832, 0x79DCB8A4, 0xE0D5E91E, 0x97D2D988, 
+  0x09B64C2B, 0x7EB17CBD, 0xE7B82D07, 0x90BF1D91, 
+  0x1DB71064, 0x6AB020F2, 0xF3B97148, 0x84BE41DE, 
+  0x1ADAD47D, 0x6DDDE4EB, 0xF4D4B551, 0x83D385C7, 
+  0x136C9856, 0x646BA8C0, 0xFD62F97A, 0x8A65C9EC, 
+  0x14015C4F, 0x63066CD9, 0xFA0F3D63, 0x8D080DF5, 
+  0x3B6E20C8, 0x4C69105E, 0xD56041E4, 0xA2677172, 
+  0x3C03E4D1, 0x4B04D447, 0xD20D85FD, 0xA50AB56B, 
+  0x35B5A8FA, 0x42B2986C, 0xDBBBC9D6, 0xACBCF940, 
+  0x32D86CE3, 0x45DF5C75, 0xDCD60DCF, 0xABD13D59, 
+  0x26D930AC, 0x51DE003A, 0xC8D75180, 0xBFD06116, 
+  0x21B4F4B5, 0x56B3C423, 0xCFBA9599, 0xB8BDA50F, 
+  0x2802B89E, 0x5F058808, 0xC60CD9B2, 0xB10BE924, 
+  0x2F6F7C87, 0x58684C11, 0xC1611DAB, 0xB6662D3D, 
+  0x76DC4190, 0x01DB7106, 0x98D220BC, 0xEFD5102A, 
+  0x71B18589, 0x06B6B51F, 0x9FBFE4A5, 0xE8B8D433, 
+  0x7807C9A2, 0x0F00F934, 0x9609A88E, 0xE10E9818, 
+  0x7F6A0DBB, 0x086D3D2D, 0x91646C97, 0xE6635C01, 
+  0x6B6B51F4, 0x1C6C6162, 0x856530D8, 0xF262004E, 
+  0x6C0695ED, 0x1B01A57B, 0x8208F4C1, 0xF50FC457, 
+  0x65B0D9C6, 0x12B7E950, 0x8BBEB8EA, 0xFCB9887C, 
+  0x62DD1DDF, 0x15DA2D49, 0x8CD37CF3, 0xFBD44C65, 
+  0x4DB26158, 0x3AB551CE, 0xA3BC0074, 0xD4BB30E2, 
+  0x4ADFA541, 0x3DD895D7, 0xA4D1C46D, 0xD3D6F4FB, 
+  0x4369E96A, 0x346ED9FC, 0xAD678846, 0xDA60B8D0, 
+  0x44042D73, 0x33031DE5, 0xAA0A4C5F, 0xDD0D7CC9, 
+  0x5005713C, 0x270241AA, 0xBE0B1010, 0xC90C2086, 
+  0x5768B525, 0x206F85B3, 0xB966D409, 0xCE61E49F, 
+  0x5EDEF90E, 0x29D9C998, 0xB0D09822, 0xC7D7A8B4, 
+  0x59B33D17, 0x2EB40D81, 0xB7BD5C3B, 0xC0BA6CAD, 
+  0xEDB88320, 0x9ABFB3B6, 0x03B6E20C, 0x74B1D29A, 
+  0xEAD54739, 0x9DD277AF, 0x04DB2615, 0x73DC1683, 
+  0xE3630B12, 0x94643B84, 0x0D6D6A3E, 0x7A6A5AA8, 
+  0xE40ECF0B, 0x9309FF9D, 0x0A00AE27, 0x7D079EB1, 
+  0xF00F9344, 0x8708A3D2, 0x1E01F268, 0x6906C2FE, 
+  0xF762575D, 0x806567CB, 0x196C3671, 0x6E6B06E7, 
+  0xFED41B76, 0x89D32BE0, 0x10DA7A5A, 0x67DD4ACC, 
+  0xF9B9DF6F, 0x8EBEEFF9, 0x17B7BE43, 0x60B08ED5, 
+  0xD6D6A3E8, 0xA1D1937E, 0x38D8C2C4, 0x4FDFF252, 
+  0xD1BB67F1, 0xA6BC5767, 0x3FB506DD, 0x48B2364B, 
+  0xD80D2BDA, 0xAF0A1B4C, 0x36034AF6, 0x41047A60, 
+  0xDF60EFC3, 0xA867DF55, 0x316E8EEF, 0x4669BE79, 
+  0xCB61B38C, 0xBC66831A, 0x256FD2A0, 0x5268E236, 
+  0xCC0C7795, 0xBB0B4703, 0x220216B9, 0x5505262F, 
+  0xC5BA3BBE, 0xB2BD0B28, 0x2BB45A92, 0x5CB36A04, 
+  0xC2D7FFA7, 0xB5D0CF31, 0x2CD99E8B, 0x5BDEAE1D, 
+  0x9B64C2B0, 0xEC63F226, 0x756AA39C, 0x026D930A, 
+  0x9C0906A9, 0xEB0E363F, 0x72076785, 0x05005713, 
+  0x95BF4A82, 0xE2B87A14, 0x7BB12BAE, 0x0CB61B38, 
+  0x92D28E9B, 0xE5D5BE0D, 0x7CDCEFB7, 0x0BDBDF21, 
+  0x86D3D2D4, 0xF1D4E242, 0x68DDB3F8, 0x1FDA836E, 
+  0x81BE16CD, 0xF6B9265B, 0x6FB077E1, 0x18B74777, 
+  0x88085AE6, 0xFF0F6A70, 0x66063BCA, 0x11010B5C, 
+  0x8F659EFF, 0xF862AE69, 0x616BFFD3, 0x166CCF45, 
+  0xA00AE278, 0xD70DD2EE, 0x4E048354, 0x3903B3C2, 
+  0xA7672661, 0xD06016F7, 0x4969474D, 0x3E6E77DB, 
+  0xAED16A4A, 0xD9D65ADC, 0x40DF0B66, 0x37D83BF0, 
+  0xA9BCAE53, 0xDEBB9EC5, 0x47B2CF7F, 0x30B5FFE9, 
+  0xBDBDF21C, 0xCABAC28A, 0x53B39330, 0x24B4A3A6, 
+  0xBAD03605, 0xCDD70693, 0x54DE5729, 0x23D967BF, 
+  0xB3667A2E, 0xC4614AB8, 0x5D681B02, 0x2A6F2B94, 
+  0xB40BBE37, 0xC30C8EA1, 0x5A05DF1B, 0x2D02EF8D
+};
+const uint32_t CRC32_T8_1[] = {
+  0x00000000, 0x191B3141, 0x32366282, 0x2B2D53C3, 
+  0x646CC504, 0x7D77F445, 0x565AA786, 0x4F4196C7, 
+  0xC8D98A08, 0xD1C2BB49, 0xFAEFE88A, 0xE3F4D9CB, 
+  0xACB54F0C, 0xB5AE7E4D, 0x9E832D8E, 0x87981CCF, 
+  0x4AC21251, 0x53D92310, 0x78F470D3, 0x61EF4192, 
+  0x2EAED755, 0x37B5E614, 0x1C98B5D7, 0x05838496, 
+  0x821B9859, 0x9B00A918, 0xB02DFADB, 0xA936CB9A, 
+  0xE6775D5D, 0xFF6C6C1C, 0xD4413FDF, 0xCD5A0E9E, 
+  0x958424A2, 0x8C9F15E3, 0xA7B24620, 0xBEA97761, 
+  0xF1E8E1A6, 0xE8F3D0E7, 0xC3DE8324, 0xDAC5B265, 
+  0x5D5DAEAA, 0x44469FEB, 0x6F6BCC28, 0x7670FD69, 
+  0x39316BAE, 0x202A5AEF, 0x0B07092C, 0x121C386D, 
+  0xDF4636F3, 0xC65D07B2, 0xED705471, 0xF46B6530, 
+  0xBB2AF3F7, 0xA231C2B6, 0x891C9175, 0x9007A034, 
+  0x179FBCFB, 0x0E848DBA, 0x25A9DE79, 0x3CB2EF38, 
+  0x73F379FF, 0x6AE848BE, 0x41C51B7D, 0x58DE2A3C, 
+  0xF0794F05, 0xE9627E44, 0xC24F2D87, 0xDB541CC6, 
+  0x94158A01, 0x8D0EBB40, 0xA623E883, 0xBF38D9C2, 
+  0x38A0C50D, 0x21BBF44C, 0x0A96A78F, 0x138D96CE, 
+  0x5CCC0009, 0x45D73148, 0x6EFA628B, 0x77E153CA, 
+  0xBABB5D54, 0xA3A06C15, 0x888D3FD6, 0x91960E97, 
+  0xDED79850, 0xC7CCA911, 0xECE1FAD2, 0xF5FACB93, 
+  0x7262D75C, 0x6B79E61D, 0x4054B5DE, 0x594F849F, 
+  0x160E1258, 0x0F152319, 0x243870DA, 0x3D23419B, 
+  0x65FD6BA7, 0x7CE65AE6, 0x57CB0925, 0x4ED03864, 
+  0x0191AEA3, 0x188A9FE2, 0x33A7CC21, 0x2ABCFD60, 
+  0xAD24E1AF, 0xB43FD0EE, 0x9F12832D, 0x8609B26C, 
+  0xC94824AB, 0xD05315EA, 0xFB7E4629, 0xE2657768, 
+  0x2F3F79F6, 0x362448B7, 0x1D091B74, 0x04122A35, 
+  0x4B53BCF2, 0x52488DB3, 0x7965DE70, 0x607EEF31, 
+  0xE7E6F3FE, 0xFEFDC2BF, 0xD5D0917C, 0xCCCBA03D, 
+  0x838A36FA, 0x9A9107BB, 0xB1BC5478, 0xA8A76539, 
+  0x3B83984B, 0x2298A90A, 0x09B5FAC9, 0x10AECB88, 
+  0x5FEF5D4F, 0x46F46C0E, 0x6DD93FCD, 0x74C20E8C, 
+  0xF35A1243, 0xEA412302, 0xC16C70C1, 0xD8774180, 
+  0x9736D747, 0x8E2DE606, 0xA500B5C5, 0xBC1B8484, 
+  0x71418A1A, 0x685ABB5B, 0x4377E898, 0x5A6CD9D9, 
+  0x152D4F1E, 0x0C367E5F, 0x271B2D9C, 0x3E001CDD, 
+  0xB9980012, 0xA0833153, 0x8BAE6290, 0x92B553D1, 
+  0xDDF4C516, 0xC4EFF457, 0xEFC2A794, 0xF6D996D5, 
+  0xAE07BCE9, 0xB71C8DA8, 0x9C31DE6B, 0x852AEF2A, 
+  0xCA6B79ED, 0xD37048AC, 0xF85D1B6F, 0xE1462A2E, 
+  0x66DE36E1, 0x7FC507A0, 0x54E85463, 0x4DF36522, 
+  0x02B2F3E5, 0x1BA9C2A4, 0x30849167, 0x299FA026, 
+  0xE4C5AEB8, 0xFDDE9FF9, 0xD6F3CC3A, 0xCFE8FD7B, 
+  0x80A96BBC, 0x99B25AFD, 0xB29F093E, 0xAB84387F, 
+  0x2C1C24B0, 0x350715F1, 0x1E2A4632, 0x07317773, 
+  0x4870E1B4, 0x516BD0F5, 0x7A468336, 0x635DB277, 
+  0xCBFAD74E, 0xD2E1E60F, 0xF9CCB5CC, 0xE0D7848D, 
+  0xAF96124A, 0xB68D230B, 0x9DA070C8, 0x84BB4189, 
+  0x03235D46, 0x1A386C07, 0x31153FC4, 0x280E0E85, 
+  0x674F9842, 0x7E54A903, 0x5579FAC0, 0x4C62CB81, 
+  0x8138C51F, 0x9823F45E, 0xB30EA79D, 0xAA1596DC, 
+  0xE554001B, 0xFC4F315A, 0xD7626299, 0xCE7953D8, 
+  0x49E14F17, 0x50FA7E56, 0x7BD72D95, 0x62CC1CD4, 
+  0x2D8D8A13, 0x3496BB52, 0x1FBBE891, 0x06A0D9D0, 
+  0x5E7EF3EC, 0x4765C2AD, 0x6C48916E, 0x7553A02F, 
+  0x3A1236E8, 0x230907A9, 0x0824546A, 0x113F652B, 
+  0x96A779E4, 0x8FBC48A5, 0xA4911B66, 0xBD8A2A27, 
+  0xF2CBBCE0, 0xEBD08DA1, 0xC0FDDE62, 0xD9E6EF23, 
+  0x14BCE1BD, 0x0DA7D0FC, 0x268A833F, 0x3F91B27E, 
+  0x70D024B9, 0x69CB15F8, 0x42E6463B, 0x5BFD777A, 
+  0xDC656BB5, 0xC57E5AF4, 0xEE530937, 0xF7483876, 
+  0xB809AEB1, 0xA1129FF0, 0x8A3FCC33, 0x9324FD72
+};
+const uint32_t CRC32_T8_2[] = {
+  0x00000000, 0x01C26A37, 0x0384D46E, 0x0246BE59, 
+  0x0709A8DC, 0x06CBC2EB, 0x048D7CB2, 0x054F1685, 
+  0x0E1351B8, 0x0FD13B8F, 0x0D9785D6, 0x0C55EFE1, 
+  0x091AF964, 0x08D89353, 0x0A9E2D0A, 0x0B5C473D, 
+  0x1C26A370, 0x1DE4C947, 0x1FA2771E, 0x1E601D29, 
+  0x1B2F0BAC, 0x1AED619B, 0x18ABDFC2, 0x1969B5F5, 
+  0x1235F2C8, 0x13F798FF, 0x11B126A6, 0x10734C91, 
+  0x153C5A14, 0x14FE3023, 0x16B88E7A, 0x177AE44D, 
+  0x384D46E0, 0x398F2CD7, 0x3BC9928E, 0x3A0BF8B9, 
+  0x3F44EE3C, 0x3E86840B, 0x3CC03A52, 0x3D025065, 
+  0x365E1758, 0x379C7D6F, 0x35DAC336, 0x3418A901, 
+  0x3157BF84, 0x3095D5B3, 0x32D36BEA, 0x331101DD, 
+  0x246BE590, 0x25A98FA7, 0x27EF31FE, 0x262D5BC9, 
+  0x23624D4C, 0x22A0277B, 0x20E69922, 0x2124F315, 
+  0x2A78B428, 0x2BBADE1F, 0x29FC6046, 0x283E0A71, 
+  0x2D711CF4, 0x2CB376C3, 0x2EF5C89A, 0x2F37A2AD, 
+  0x709A8DC0, 0x7158E7F7, 0x731E59AE, 0x72DC3399, 
+  0x7793251C, 0x76514F2B, 0x7417F172, 0x75D59B45, 
+  0x7E89DC78, 0x7F4BB64F, 0x7D0D0816, 0x7CCF6221, 
+  0x798074A4, 0x78421E93, 0x7A04A0CA, 0x7BC6CAFD, 
+  0x6CBC2EB0, 0x6D7E4487, 0x6F38FADE, 0x6EFA90E9, 
+  0x6BB5866C, 0x6A77EC5B, 0x68315202, 0x69F33835, 
+  0x62AF7F08, 0x636D153F, 0x612BAB66, 0x60E9C151, 
+  0x65A6D7D4, 0x6464BDE3, 0x662203BA, 0x67E0698D, 
+  0x48D7CB20, 0x4915A117, 0x4B531F4E, 0x4A917579, 
+  0x4FDE63FC, 0x4E1C09CB, 0x4C5AB792, 0x4D98DDA5, 
+  0x46C49A98, 0x4706F0AF, 0x45404EF6, 0x448224C1, 
+  0x41CD3244, 0x400F5873, 0x4249E62A, 0x438B8C1D, 
+  0x54F16850, 0x55330267, 0x5775BC3E, 0x56B7D609, 
+  0x53F8C08C, 0x523AAABB, 0x507C14E2, 0x51BE7ED5, 
+  0x5AE239E8, 0x5B2053DF, 0x5966ED86, 0x58A487B1, 
+  0x5DEB9134, 0x5C29FB03, 0x5E6F455A, 0x5FAD2F6D, 
+  0xE1351B80, 0xE0F771B7, 0xE2B1CFEE, 0xE373A5D9, 
+  0xE63CB35C, 0xE7FED96B, 0xE5B86732, 0xE47A0D05, 
+  0xEF264A38, 0xEEE4200F, 0xECA29E56, 0xED60F461, 
+  0xE82FE2E4, 0xE9ED88D3, 0xEBAB368A, 0xEA695CBD, 
+  0xFD13B8F0, 0xFCD1D2C7, 0xFE976C9E, 0xFF5506A9, 
+  0xFA1A102C, 0xFBD87A1B, 0xF99EC442, 0xF85CAE75, 
+  0xF300E948, 0xF2C2837F, 0xF0843D26, 0xF1465711, 
+  0xF4094194, 0xF5CB2BA3, 0xF78D95FA, 0xF64FFFCD, 
+  0xD9785D60, 0xD8BA3757, 0xDAFC890E, 0xDB3EE339, 
+  0xDE71F5BC, 0xDFB39F8B, 0xDDF521D2, 0xDC374BE5, 
+  0xD76B0CD8, 0xD6A966EF, 0xD4EFD8B6, 0xD52DB281, 
+  0xD062A404, 0xD1A0CE33, 0xD3E6706A, 0xD2241A5D, 
+  0xC55EFE10, 0xC49C9427, 0xC6DA2A7E, 0xC7184049, 
+  0xC25756CC, 0xC3953CFB, 0xC1D382A2, 0xC011E895, 
+  0xCB4DAFA8, 0xCA8FC59F, 0xC8C97BC6, 0xC90B11F1, 
+  0xCC440774, 0xCD866D43, 0xCFC0D31A, 0xCE02B92D, 
+  0x91AF9640, 0x906DFC77, 0x922B422E, 0x93E92819, 
+  0x96A63E9C, 0x976454AB, 0x9522EAF2, 0x94E080C5, 
+  0x9FBCC7F8, 0x9E7EADCF, 0x9C381396, 0x9DFA79A1, 
+  0x98B56F24, 0x99770513, 0x9B31BB4A, 0x9AF3D17D, 
+  0x8D893530, 0x8C4B5F07, 0x8E0DE15E, 0x8FCF8B69, 
+  0x8A809DEC, 0x8B42F7DB, 0x89044982, 0x88C623B5, 
+  0x839A6488, 0x82580EBF, 0x801EB0E6, 0x81DCDAD1, 
+  0x8493CC54, 0x8551A663, 0x8717183A, 0x86D5720D, 
+  0xA9E2D0A0, 0xA820BA97, 0xAA6604CE, 0xABA46EF9, 
+  0xAEEB787C, 0xAF29124B, 0xAD6FAC12, 0xACADC625, 
+  0xA7F18118, 0xA633EB2F, 0xA4755576, 0xA5B73F41, 
+  0xA0F829C4, 0xA13A43F3, 0xA37CFDAA, 0xA2BE979D, 
+  0xB5C473D0, 0xB40619E7, 0xB640A7BE, 0xB782CD89, 
+  0xB2CDDB0C, 0xB30FB13B, 0xB1490F62, 0xB08B6555, 
+  0xBBD72268, 0xBA15485F, 0xB853F606, 0xB9919C31, 
+  0xBCDE8AB4, 0xBD1CE083, 0xBF5A5EDA, 0xBE9834ED
+};
+const uint32_t CRC32_T8_3[] = {
+  0x00000000, 0xB8BC6765, 0xAA09C88B, 0x12B5AFEE, 
+  0x8F629757, 0x37DEF032, 0x256B5FDC, 0x9DD738B9, 
+  0xC5B428EF, 0x7D084F8A, 0x6FBDE064, 0xD7018701, 
+  0x4AD6BFB8, 0xF26AD8DD, 0xE0DF7733, 0x58631056, 
+  0x5019579F, 0xE8A530FA, 0xFA109F14, 0x42ACF871, 
+  0xDF7BC0C8, 0x67C7A7AD, 0x75720843, 0xCDCE6F26, 
+  0x95AD7F70, 0x2D111815, 0x3FA4B7FB, 0x8718D09E, 
+  0x1ACFE827, 0xA2738F42, 0xB0C620AC, 0x087A47C9, 
+  0xA032AF3E, 0x188EC85B, 0x0A3B67B5, 0xB28700D0, 
+  0x2F503869, 0x97EC5F0C, 0x8559F0E2, 0x3DE59787, 
+  0x658687D1, 0xDD3AE0B4, 0xCF8F4F5A, 0x7733283F, 
+  0xEAE41086, 0x525877E3, 0x40EDD80D, 0xF851BF68, 
+  0xF02BF8A1, 0x48979FC4, 0x5A22302A, 0xE29E574F, 
+  0x7F496FF6, 0xC7F50893, 0xD540A77D, 0x6DFCC018, 
+  0x359FD04E, 0x8D23B72B, 0x9F9618C5, 0x272A7FA0, 
+  0xBAFD4719, 0x0241207C, 0x10F48F92, 0xA848E8F7, 
+  0x9B14583D, 0x23A83F58, 0x311D90B6, 0x89A1F7D3, 
+  0x1476CF6A, 0xACCAA80F, 0xBE7F07E1, 0x06C36084, 
+  0x5EA070D2, 0xE61C17B7, 0xF4A9B859, 0x4C15DF3C, 
+  0xD1C2E785, 0x697E80E0, 0x7BCB2F0E, 0xC377486B, 
+  0xCB0D0FA2, 0x73B168C7, 0x6104C729, 0xD9B8A04C, 
+  0x446F98F5, 0xFCD3FF90, 0xEE66507E, 0x56DA371B, 
+  0x0EB9274D, 0xB6054028, 0xA4B0EFC6, 0x1C0C88A3, 
+  0x81DBB01A, 0x3967D77F, 0x2BD27891, 0x936E1FF4, 
+  0x3B26F703, 0x839A9066, 0x912F3F88, 0x299358ED, 
+  0xB4446054, 0x0CF80731, 0x1E4DA8DF, 0xA6F1CFBA, 
+  0xFE92DFEC, 0x462EB889, 0x549B1767, 0xEC277002, 
+  0x71F048BB, 0xC94C2FDE, 0xDBF98030, 0x6345E755, 
+  0x6B3FA09C, 0xD383C7F9, 0xC1366817, 0x798A0F72, 
+  0xE45D37CB, 0x5CE150AE, 0x4E54FF40, 0xF6E89825, 
+  0xAE8B8873, 0x1637EF16, 0x048240F8, 0xBC3E279D, 
+  0x21E91F24, 0x99557841, 0x8BE0D7AF, 0x335CB0CA, 
+  0xED59B63B, 0x55E5D15E, 0x47507EB0, 0xFFEC19D5, 
+  0x623B216C, 0xDA874609, 0xC832E9E7, 0x708E8E82, 
+  0x28ED9ED4, 0x9051F9B1, 0x82E4565F, 0x3A58313A, 
+  0xA78F0983, 0x1F336EE6, 0x0D86C108, 0xB53AA66D, 
+  0xBD40E1A4, 0x05FC86C1, 0x1749292F, 0xAFF54E4A, 
+  0x322276F3, 0x8A9E1196, 0x982BBE78, 0x2097D91D, 
+  0x78F4C94B, 0xC048AE2E, 0xD2FD01C0, 0x6A4166A5, 
+  0xF7965E1C, 0x4F2A3979, 0x5D9F9697, 0xE523F1F2, 
+  0x4D6B1905, 0xF5D77E60, 0xE762D18E, 0x5FDEB6EB, 
+  0xC2098E52, 0x7AB5E937, 0x680046D9, 0xD0BC21BC, 
+  0x88DF31EA, 0x3063568F, 0x22D6F961, 0x9A6A9E04, 
+  0x07BDA6BD, 0xBF01C1D8, 0xADB46E36, 0x15080953, 
+  0x1D724E9A, 0xA5CE29FF, 0xB77B8611, 0x0FC7E174, 
+  0x9210D9CD, 0x2AACBEA8, 0x38191146, 0x80A57623, 
+  0xD8C66675, 0x607A0110, 0x72CFAEFE, 0xCA73C99B, 
+  0x57A4F122, 0xEF189647, 0xFDAD39A9, 0x45115ECC, 
+  0x764DEE06, 0xCEF18963, 0xDC44268D, 0x64F841E8, 
+  0xF92F7951, 0x41931E34, 0x5326B1DA, 0xEB9AD6BF, 
+  0xB3F9C6E9, 0x0B45A18C, 0x19F00E62, 0xA14C6907, 
+  0x3C9B51BE, 0x842736DB, 0x96929935, 0x2E2EFE50, 
+  0x2654B999, 0x9EE8DEFC, 0x8C5D7112, 0x34E11677, 
+  0xA9362ECE, 0x118A49AB, 0x033FE645, 0xBB838120, 
+  0xE3E09176, 0x5B5CF613, 0x49E959FD, 0xF1553E98, 
+  0x6C820621, 0xD43E6144, 0xC68BCEAA, 0x7E37A9CF, 
+  0xD67F4138, 0x6EC3265D, 0x7C7689B3, 0xC4CAEED6, 
+  0x591DD66F, 0xE1A1B10A, 0xF3141EE4, 0x4BA87981, 
+  0x13CB69D7, 0xAB770EB2, 0xB9C2A15C, 0x017EC639, 
+  0x9CA9FE80, 0x241599E5, 0x36A0360B, 0x8E1C516E, 
+  0x866616A7, 0x3EDA71C2, 0x2C6FDE2C, 0x94D3B949, 
+  0x090481F0, 0xB1B8E695, 0xA30D497B, 0x1BB12E1E, 
+  0x43D23E48, 0xFB6E592D, 0xE9DBF6C3, 0x516791A6, 
+  0xCCB0A91F, 0x740CCE7A, 0x66B96194, 0xDE0506F1
+};
+const uint32_t CRC32_T8_4[] = {
+  0x00000000, 0x3D6029B0, 0x7AC05360, 0x47A07AD0, 
+  0xF580A6C0, 0xC8E08F70, 0x8F40F5A0, 0xB220DC10, 
+  0x30704BC1, 0x0D106271, 0x4AB018A1, 0x77D03111, 
+  0xC5F0ED01, 0xF890C4B1, 0xBF30BE61, 0x825097D1, 
+  0x60E09782, 0x5D80BE32, 0x1A20C4E2, 0x2740ED52, 
+  0x95603142, 0xA80018F2, 0xEFA06222, 0xD2C04B92, 
+  0x5090DC43, 0x6DF0F5F3, 0x2A508F23, 0x1730A693, 
+  0xA5107A83, 0x98705333, 0xDFD029E3, 0xE2B00053, 
+  0xC1C12F04, 0xFCA106B4, 0xBB017C64, 0x866155D4, 
+  0x344189C4, 0x0921A074, 0x4E81DAA4, 0x73E1F314, 
+  0xF1B164C5, 0xCCD14D75, 0x8B7137A5, 0xB6111E15, 
+  0x0431C205, 0x3951EBB5, 0x7EF19165, 0x4391B8D5, 
+  0xA121B886, 0x9C419136, 0xDBE1EBE6, 0xE681C256, 
+  0x54A11E46, 0x69C137F6, 0x2E614D26, 0x13016496, 
+  0x9151F347, 0xAC31DAF7, 0xEB91A027, 0xD6F18997, 
+  0x64D15587, 0x59B17C37, 0x1E1106E7, 0x23712F57, 
+  0x58F35849, 0x659371F9, 0x22330B29, 0x1F532299, 
+  0xAD73FE89, 0x9013D739, 0xD7B3ADE9, 0xEAD38459, 
+  0x68831388, 0x55E33A38, 0x124340E8, 0x2F236958, 
+  0x9D03B548, 0xA0639CF8, 0xE7C3E628, 0xDAA3CF98, 
+  0x3813CFCB, 0x0573E67B, 0x42D39CAB, 0x7FB3B51B, 
+  0xCD93690B, 0xF0F340BB, 0xB7533A6B, 0x8A3313DB, 
+  0x0863840A, 0x3503ADBA, 0x72A3D76A, 0x4FC3FEDA, 
+  0xFDE322CA, 0xC0830B7A, 0x872371AA, 0xBA43581A, 
+  0x9932774D, 0xA4525EFD, 0xE3F2242D, 0xDE920D9D, 
+  0x6CB2D18D, 0x51D2F83D, 0x167282ED, 0x2B12AB5D, 
+  0xA9423C8C, 0x9422153C, 0xD3826FEC, 0xEEE2465C, 
+  0x5CC29A4C, 0x61A2B3FC, 0x2602C92C, 0x1B62E09C, 
+  0xF9D2E0CF, 0xC4B2C97F, 0x8312B3AF, 0xBE729A1F, 
+  0x0C52460F, 0x31326FBF, 0x7692156F, 0x4BF23CDF, 
+  0xC9A2AB0E, 0xF4C282BE, 0xB362F86E, 0x8E02D1DE, 
+  0x3C220DCE, 0x0142247E, 0x46E25EAE, 0x7B82771E, 
+  0xB1E6B092, 0x8C869922, 0xCB26E3F2, 0xF646CA42, 
+  0x44661652, 0x79063FE2, 0x3EA64532, 0x03C66C82, 
+  0x8196FB53, 0xBCF6D2E3, 0xFB56A833, 0xC6368183, 
+  0x74165D93, 0x49767423, 0x0ED60EF3, 0x33B62743, 
+  0xD1062710, 0xEC660EA0, 0xABC67470, 0x96A65DC0, 
+  0x248681D0, 0x19E6A860, 0x5E46D2B0, 0x6326FB00, 
+  0xE1766CD1, 0xDC164561, 0x9BB63FB1, 0xA6D61601, 
+  0x14F6CA11, 0x2996E3A1, 0x6E369971, 0x5356B0C1, 
+  0x70279F96, 0x4D47B626, 0x0AE7CCF6, 0x3787E546, 
+  0x85A73956, 0xB8C710E6, 0xFF676A36, 0xC2074386, 
+  0x4057D457, 0x7D37FDE7, 0x3A978737, 0x07F7AE87, 
+  0xB5D77297, 0x88B75B27, 0xCF1721F7, 0xF2770847, 
+  0x10C70814, 0x2DA721A4, 0x6A075B74, 0x576772C4, 
+  0xE547AED4, 0xD8278764, 0x9F87FDB4, 0xA2E7D404, 
+  0x20B743D5, 0x1DD76A65, 0x5A7710B5, 0x67173905, 
+  0xD537E515, 0xE857CCA5, 0xAFF7B675, 0x92979FC5, 
+  0xE915E8DB, 0xD475C16B, 0x93D5BBBB, 0xAEB5920B, 
+  0x1C954E1B, 0x21F567AB, 0x66551D7B, 0x5B3534CB, 
+  0xD965A31A, 0xE4058AAA, 0xA3A5F07A, 0x9EC5D9CA, 
+  0x2CE505DA, 0x11852C6A, 0x562556BA, 0x6B457F0A, 
+  0x89F57F59, 0xB49556E9, 0xF3352C39, 0xCE550589, 
+  0x7C75D999, 0x4115F029, 0x06B58AF9, 0x3BD5A349, 
+  0xB9853498, 0x84E51D28, 0xC34567F8, 0xFE254E48, 
+  0x4C059258, 0x7165BBE8, 0x36C5C138, 0x0BA5E888, 
+  0x28D4C7DF, 0x15B4EE6F, 0x521494BF, 0x6F74BD0F, 
+  0xDD54611F, 0xE03448AF, 0xA794327F, 0x9AF41BCF, 
+  0x18A48C1E, 0x25C4A5AE, 0x6264DF7E, 0x5F04F6CE, 
+  0xED242ADE, 0xD044036E, 0x97E479BE, 0xAA84500E, 
+  0x4834505D, 0x755479ED, 0x32F4033D, 0x0F942A8D, 
+  0xBDB4F69D, 0x80D4DF2D, 0xC774A5FD, 0xFA148C4D, 
+  0x78441B9C, 0x4524322C, 0x028448FC, 0x3FE4614C, 
+  0x8DC4BD5C, 0xB0A494EC, 0xF704EE3C, 0xCA64C78C
+};
+const uint32_t CRC32_T8_5[] = {
+  0x00000000, 0xCB5CD3A5, 0x4DC8A10B, 0x869472AE, 
+  0x9B914216, 0x50CD91B3, 0xD659E31D, 0x1D0530B8, 
+  0xEC53826D, 0x270F51C8, 0xA19B2366, 0x6AC7F0C3, 
+  0x77C2C07B, 0xBC9E13DE, 0x3A0A6170, 0xF156B2D5, 
+  0x03D6029B, 0xC88AD13E, 0x4E1EA390, 0x85427035, 
+  0x9847408D, 0x531B9328, 0xD58FE186, 0x1ED33223, 
+  0xEF8580F6, 0x24D95353, 0xA24D21FD, 0x6911F258, 
+  0x7414C2E0, 0xBF481145, 0x39DC63EB, 0xF280B04E, 
+  0x07AC0536, 0xCCF0D693, 0x4A64A43D, 0x81387798, 
+  0x9C3D4720, 0x57619485, 0xD1F5E62B, 0x1AA9358E, 
+  0xEBFF875B, 0x20A354FE, 0xA6372650, 0x6D6BF5F5, 
+  0x706EC54D, 0xBB3216E8, 0x3DA66446, 0xF6FAB7E3, 
+  0x047A07AD, 0xCF26D408, 0x49B2A6A6, 0x82EE7503, 
+  0x9FEB45BB, 0x54B7961E, 0xD223E4B0, 0x197F3715, 
+  0xE82985C0, 0x23755665, 0xA5E124CB, 0x6EBDF76E, 
+  0x73B8C7D6, 0xB8E41473, 0x3E7066DD, 0xF52CB578, 
+  0x0F580A6C, 0xC404D9C9, 0x4290AB67, 0x89CC78C2, 
+  0x94C9487A, 0x5F959BDF, 0xD901E971, 0x125D3AD4, 
+  0xE30B8801, 0x28575BA4, 0xAEC3290A, 0x659FFAAF, 
+  0x789ACA17, 0xB3C619B2, 0x35526B1C, 0xFE0EB8B9, 
+  0x0C8E08F7, 0xC7D2DB52, 0x4146A9FC, 0x8A1A7A59, 
+  0x971F4AE1, 0x5C439944, 0xDAD7EBEA, 0x118B384F, 
+  0xE0DD8A9A, 0x2B81593F, 0xAD152B91, 0x6649F834, 
+  0x7B4CC88C, 0xB0101B29, 0x36846987, 0xFDD8BA22, 
+  0x08F40F5A, 0xC3A8DCFF, 0x453CAE51, 0x8E607DF4, 
+  0x93654D4C, 0x58399EE9, 0xDEADEC47, 0x15F13FE2, 
+  0xE4A78D37, 0x2FFB5E92, 0xA96F2C3C, 0x6233FF99, 
+  0x7F36CF21, 0xB46A1C84, 0x32FE6E2A, 0xF9A2BD8F, 
+  0x0B220DC1, 0xC07EDE64, 0x46EAACCA, 0x8DB67F6F, 
+  0x90B34FD7, 0x5BEF9C72, 0xDD7BEEDC, 0x16273D79, 
+  0xE7718FAC, 0x2C2D5C09, 0xAAB92EA7, 0x61E5FD02, 
+  0x7CE0CDBA, 0xB7BC1E1F, 0x31286CB1, 0xFA74BF14, 
+  0x1EB014D8, 0xD5ECC77D, 0x5378B5D3, 0x98246676, 
+  0x852156CE, 0x4E7D856B, 0xC8E9F7C5, 0x03B52460, 
+  0xF2E396B5, 0x39BF4510, 0xBF2B37BE, 0x7477E41B, 
+  0x6972D4A3, 0xA22E0706, 0x24BA75A8, 0xEFE6A60D, 
+  0x1D661643, 0xD63AC5E6, 0x50AEB748, 0x9BF264ED, 
+  0x86F75455, 0x4DAB87F0, 0xCB3FF55E, 0x006326FB, 
+  0xF135942E, 0x3A69478B, 0xBCFD3525, 0x77A1E680, 
+  0x6AA4D638, 0xA1F8059D, 0x276C7733, 0xEC30A496, 
+  0x191C11EE, 0xD240C24B, 0x54D4B0E5, 0x9F886340, 
+  0x828D53F8, 0x49D1805D, 0xCF45F2F3, 0x04192156, 
+  0xF54F9383, 0x3E134026, 0xB8873288, 0x73DBE12D, 
+  0x6EDED195, 0xA5820230, 0x2316709E, 0xE84AA33B, 
+  0x1ACA1375, 0xD196C0D0, 0x5702B27E, 0x9C5E61DB, 
+  0x815B5163, 0x4A0782C6, 0xCC93F068, 0x07CF23CD, 
+  0xF6999118, 0x3DC542BD, 0xBB513013, 0x700DE3B6, 
+  0x6D08D30E, 0xA65400AB, 0x20C07205, 0xEB9CA1A0, 
+  0x11E81EB4, 0xDAB4CD11, 0x5C20BFBF, 0x977C6C1A, 
+  0x8A795CA2, 0x41258F07, 0xC7B1FDA9, 0x0CED2E0C, 
+  0xFDBB9CD9, 0x36E74F7C, 0xB0733DD2, 0x7B2FEE77, 
+  0x662ADECF, 0xAD760D6A, 0x2BE27FC4, 0xE0BEAC61, 
+  0x123E1C2F, 0xD962CF8A, 0x5FF6BD24, 0x94AA6E81, 
+  0x89AF5E39, 0x42F38D9C, 0xC467FF32, 0x0F3B2C97, 
+  0xFE6D9E42, 0x35314DE7, 0xB3A53F49, 0x78F9ECEC, 
+  0x65FCDC54, 0xAEA00FF1, 0x28347D5F, 0xE368AEFA, 
+  0x16441B82, 0xDD18C827, 0x5B8CBA89, 0x90D0692C, 
+  0x8DD55994, 0x46898A31, 0xC01DF89F, 0x0B412B3A, 
+  0xFA1799EF, 0x314B4A4A, 0xB7DF38E4, 0x7C83EB41, 
+  0x6186DBF9, 0xAADA085C, 0x2C4E7AF2, 0xE712A957, 
+  0x15921919, 0xDECECABC, 0x585AB812, 0x93066BB7, 
+  0x8E035B0F, 0x455F88AA, 0xC3CBFA04, 0x089729A1, 
+  0xF9C19B74, 0x329D48D1, 0xB4093A7F, 0x7F55E9DA, 
+  0x6250D962, 0xA90C0AC7, 0x2F987869, 0xE4C4ABCC
+};
+const uint32_t CRC32_T8_6[] = {
+  0x00000000, 0xA6770BB4, 0x979F1129, 0x31E81A9D, 
+  0xF44F2413, 0x52382FA7, 0x63D0353A, 0xC5A73E8E, 
+  0x33EF4E67, 0x959845D3, 0xA4705F4E, 0x020754FA, 
+  0xC7A06A74, 0x61D761C0, 0x503F7B5D, 0xF64870E9, 
+  0x67DE9CCE, 0xC1A9977A, 0xF0418DE7, 0x56368653, 
+  0x9391B8DD, 0x35E6B369, 0x040EA9F4, 0xA279A240, 
+  0x5431D2A9, 0xF246D91D, 0xC3AEC380, 0x65D9C834, 
+  0xA07EF6BA, 0x0609FD0E, 0x37E1E793, 0x9196EC27, 
+  0xCFBD399C, 0x69CA3228, 0x582228B5, 0xFE552301, 
+  0x3BF21D8F, 0x9D85163B, 0xAC6D0CA6, 0x0A1A0712, 
+  0xFC5277FB, 0x5A257C4F, 0x6BCD66D2, 0xCDBA6D66, 
+  0x081D53E8, 0xAE6A585C, 0x9F8242C1, 0x39F54975, 
+  0xA863A552, 0x0E14AEE6, 0x3FFCB47B, 0x998BBFCF, 
+  0x5C2C8141, 0xFA5B8AF5, 0xCBB39068, 0x6DC49BDC, 
+  0x9B8CEB35, 0x3DFBE081, 0x0C13FA1C, 0xAA64F1A8, 
+  0x6FC3CF26, 0xC9B4C492, 0xF85CDE0F, 0x5E2BD5BB, 
+  0x440B7579, 0xE27C7ECD, 0xD3946450, 0x75E36FE4, 
+  0xB044516A, 0x16335ADE, 0x27DB4043, 0x81AC4BF7, 
+  0x77E43B1E, 0xD19330AA, 0xE07B2A37, 0x460C2183, 
+  0x83AB1F0D, 0x25DC14B9, 0x14340E24, 0xB2430590, 
+  0x23D5E9B7, 0x85A2E203, 0xB44AF89E, 0x123DF32A, 
+  0xD79ACDA4, 0x71EDC610, 0x4005DC8D, 0xE672D739, 
+  0x103AA7D0, 0xB64DAC64, 0x87A5B6F9, 0x21D2BD4D, 
+  0xE47583C3, 0x42028877, 0x73EA92EA, 0xD59D995E, 
+  0x8BB64CE5, 0x2DC14751, 0x1C295DCC, 0xBA5E5678, 
+  0x7FF968F6, 0xD98E6342, 0xE86679DF, 0x4E11726B, 
+  0xB8590282, 0x1E2E0936, 0x2FC613AB, 0x89B1181F, 
+  0x4C162691, 0xEA612D25, 0xDB8937B8, 0x7DFE3C0C, 
+  0xEC68D02B, 0x4A1FDB9F, 0x7BF7C102, 0xDD80CAB6, 
+  0x1827F438, 0xBE50FF8C, 0x8FB8E511, 0x29CFEEA5, 
+  0xDF879E4C, 0x79F095F8, 0x48188F65, 0xEE6F84D1, 
+  0x2BC8BA5F, 0x8DBFB1EB, 0xBC57AB76, 0x1A20A0C2, 
+  0x8816EAF2, 0x2E61E146, 0x1F89FBDB, 0xB9FEF06F, 
+  0x7C59CEE1, 0xDA2EC555, 0xEBC6DFC8, 0x4DB1D47C, 
+  0xBBF9A495, 0x1D8EAF21, 0x2C66B5BC, 0x8A11BE08, 
+  0x4FB68086, 0xE9C18B32, 0xD82991AF, 0x7E5E9A1B, 
+  0xEFC8763C, 0x49BF7D88, 0x78576715, 0xDE206CA1, 
+  0x1B87522F, 0xBDF0599B, 0x8C184306, 0x2A6F48B2, 
+  0xDC27385B, 0x7A5033EF, 0x4BB82972, 0xEDCF22C6, 
+  0x28681C48, 0x8E1F17FC, 0xBFF70D61, 0x198006D5, 
+  0x47ABD36E, 0xE1DCD8DA, 0xD034C247, 0x7643C9F3, 
+  0xB3E4F77D, 0x1593FCC9, 0x247BE654, 0x820CEDE0, 
+  0x74449D09, 0xD23396BD, 0xE3DB8C20, 0x45AC8794, 
+  0x800BB91A, 0x267CB2AE, 0x1794A833, 0xB1E3A387, 
+  0x20754FA0, 0x86024414, 0xB7EA5E89, 0x119D553D, 
+  0xD43A6BB3, 0x724D6007, 0x43A57A9A, 0xE5D2712E, 
+  0x139A01C7, 0xB5ED0A73, 0x840510EE, 0x22721B5A, 
+  0xE7D525D4, 0x41A22E60, 0x704A34FD, 0xD63D3F49, 
+  0xCC1D9F8B, 0x6A6A943F, 0x5B828EA2, 0xFDF58516, 
+  0x3852BB98, 0x9E25B02C, 0xAFCDAAB1, 0x09BAA105, 
+  0xFFF2D1EC, 0x5985DA58, 0x686DC0C5, 0xCE1ACB71, 
+  0x0BBDF5FF, 0xADCAFE4B, 0x9C22E4D6, 0x3A55EF62, 
+  0xABC30345, 0x0DB408F1, 0x3C5C126C, 0x9A2B19D8, 
+  0x5F8C2756, 0xF9FB2CE2, 0xC813367F, 0x6E643DCB, 
+  0x982C4D22, 0x3E5B4696, 0x0FB35C0B, 0xA9C457BF, 
+  0x6C636931, 0xCA146285, 0xFBFC7818, 0x5D8B73AC, 
+  0x03A0A617, 0xA5D7ADA3, 0x943FB73E, 0x3248BC8A, 
+  0xF7EF8204, 0x519889B0, 0x6070932D, 0xC6079899, 
+  0x304FE870, 0x9638E3C4, 0xA7D0F959, 0x01A7F2ED, 
+  0xC400CC63, 0x6277C7D7, 0x539FDD4A, 0xF5E8D6FE, 
+  0x647E3AD9, 0xC209316D, 0xF3E12BF0, 0x55962044, 
+  0x90311ECA, 0x3646157E, 0x07AE0FE3, 0xA1D90457, 
+  0x579174BE, 0xF1E67F0A, 0xC00E6597, 0x66796E23, 
+  0xA3DE50AD, 0x05A95B19, 0x34414184, 0x92364A30
+};
+const uint32_t CRC32_T8_7[] = {
+  0x00000000, 0xCCAA009E, 0x4225077D, 0x8E8F07E3, 
+  0x844A0EFA, 0x48E00E64, 0xC66F0987, 0x0AC50919, 
+  0xD3E51BB5, 0x1F4F1B2B, 0x91C01CC8, 0x5D6A1C56, 
+  0x57AF154F, 0x9B0515D1, 0x158A1232, 0xD92012AC, 
+  0x7CBB312B, 0xB01131B5, 0x3E9E3656, 0xF23436C8, 
+  0xF8F13FD1, 0x345B3F4F, 0xBAD438AC, 0x767E3832, 
+  0xAF5E2A9E, 0x63F42A00, 0xED7B2DE3, 0x21D12D7D, 
+  0x2B142464, 0xE7BE24FA, 0x69312319, 0xA59B2387, 
+  0xF9766256, 0x35DC62C8, 0xBB53652B, 0x77F965B5, 
+  0x7D3C6CAC, 0xB1966C32, 0x3F196BD1, 0xF3B36B4F, 
+  0x2A9379E3, 0xE639797D, 0x68B67E9E, 0xA41C7E00, 
+  0xAED97719, 0x62737787, 0xECFC7064, 0x205670FA, 
+  0x85CD537D, 0x496753E3, 0xC7E85400, 0x0B42549E, 
+  0x01875D87, 0xCD2D5D19, 0x43A25AFA, 0x8F085A64, 
+  0x562848C8, 0x9A824856, 0x140D4FB5, 0xD8A74F2B, 
+  0xD2624632, 0x1EC846AC, 0x9047414F, 0x5CED41D1, 
+  0x299DC2ED, 0xE537C273, 0x6BB8C590, 0xA712C50E, 
+  0xADD7CC17, 0x617DCC89, 0xEFF2CB6A, 0x2358CBF4, 
+  0xFA78D958, 0x36D2D9C6, 0xB85DDE25, 0x74F7DEBB, 
+  0x7E32D7A2, 0xB298D73C, 0x3C17D0DF, 0xF0BDD041, 
+  0x5526F3C6, 0x998CF358, 0x1703F4BB, 0xDBA9F425, 
+  0xD16CFD3C, 0x1DC6FDA2, 0x9349FA41, 0x5FE3FADF, 
+  0x86C3E873, 0x4A69E8ED, 0xC4E6EF0E, 0x084CEF90, 
+  0x0289E689, 0xCE23E617, 0x40ACE1F4, 0x8C06E16A, 
+  0xD0EBA0BB, 0x1C41A025, 0x92CEA7C6, 0x5E64A758, 
+  0x54A1AE41, 0x980BAEDF, 0x1684A93C, 0xDA2EA9A2, 
+  0x030EBB0E, 0xCFA4BB90, 0x412BBC73, 0x8D81BCED, 
+  0x8744B5F4, 0x4BEEB56A, 0xC561B289, 0x09CBB217, 
+  0xAC509190, 0x60FA910E, 0xEE7596ED, 0x22DF9673, 
+  0x281A9F6A, 0xE4B09FF4, 0x6A3F9817, 0xA6959889, 
+  0x7FB58A25, 0xB31F8ABB, 0x3D908D58, 0xF13A8DC6, 
+  0xFBFF84DF, 0x37558441, 0xB9DA83A2, 0x7570833C, 
+  0x533B85DA, 0x9F918544, 0x111E82A7, 0xDDB48239, 
+  0xD7718B20, 0x1BDB8BBE, 0x95548C5D, 0x59FE8CC3, 
+  0x80DE9E6F, 0x4C749EF1, 0xC2FB9912, 0x0E51998C, 
+  0x04949095, 0xC83E900B, 0x46B197E8, 0x8A1B9776, 
+  0x2F80B4F1, 0xE32AB46F, 0x6DA5B38C, 0xA10FB312, 
+  0xABCABA0B, 0x6760BA95, 0xE9EFBD76, 0x2545BDE8, 
+  0xFC65AF44, 0x30CFAFDA, 0xBE40A839, 0x72EAA8A7, 
+  0x782FA1BE, 0xB485A120, 0x3A0AA6C3, 0xF6A0A65D, 
+  0xAA4DE78C, 0x66E7E712, 0xE868E0F1, 0x24C2E06F, 
+  0x2E07E976, 0xE2ADE9E8, 0x6C22EE0B, 0xA088EE95, 
+  0x79A8FC39, 0xB502FCA7, 0x3B8DFB44, 0xF727FBDA, 
+  0xFDE2F2C3, 0x3148F25D, 0xBFC7F5BE, 0x736DF520, 
+  0xD6F6D6A7, 0x1A5CD639, 0x94D3D1DA, 0x5879D144, 
+  0x52BCD85D, 0x9E16D8C3, 0x1099DF20, 0xDC33DFBE, 
+  0x0513CD12, 0xC9B9CD8C, 0x4736CA6F, 0x8B9CCAF1, 
+  0x8159C3E8, 0x4DF3C376, 0xC37CC495, 0x0FD6C40B, 
+  0x7AA64737, 0xB60C47A9, 0x3883404A, 0xF42940D4, 
+  0xFEEC49CD, 0x32464953, 0xBCC94EB0, 0x70634E2E, 
+  0xA9435C82, 0x65E95C1C, 0xEB665BFF, 0x27CC5B61, 
+  0x2D095278, 0xE1A352E6, 0x6F2C5505, 0xA386559B, 
+  0x061D761C, 0xCAB77682, 0x44387161, 0x889271FF, 
+  0x825778E6, 0x4EFD7878, 0xC0727F9B, 0x0CD87F05, 
+  0xD5F86DA9, 0x19526D37, 0x97DD6AD4, 0x5B776A4A, 
+  0x51B26353, 0x9D1863CD, 0x1397642E, 0xDF3D64B0, 
+  0x83D02561, 0x4F7A25FF, 0xC1F5221C, 0x0D5F2282, 
+  0x079A2B9B, 0xCB302B05, 0x45BF2CE6, 0x89152C78, 
+  0x50353ED4, 0x9C9F3E4A, 0x121039A9, 0xDEBA3937, 
+  0xD47F302E, 0x18D530B0, 0x965A3753, 0x5AF037CD, 
+  0xFF6B144A, 0x33C114D4, 0xBD4E1337, 0x71E413A9, 
+  0x7B211AB0, 0xB78B1A2E, 0x39041DCD, 0xF5AE1D53, 
+  0x2C8E0FFF, 0xE0240F61, 0x6EAB0882, 0xA201081C, 
+  0xA8C40105, 0x646E019B, 0xEAE10678, 0x264B06E6
+};
+
+



[51/54] [abbrv] incubator-ratis git commit: Add a Builder for RaftServer.

Posted by ji...@apache.org.
Add a Builder for RaftServer.


Project: http://git-wip-us.apache.org/repos/asf/incubator-ratis/repo
Commit: http://git-wip-us.apache.org/repos/asf/incubator-ratis/commit/0fed3078
Tree: http://git-wip-us.apache.org/repos/asf/incubator-ratis/tree/0fed3078
Diff: http://git-wip-us.apache.org/repos/asf/incubator-ratis/diff/0fed3078

Branch: refs/heads/master
Commit: 0fed3078dd04c799067e856fb191fb0244a0f544
Parents: 336874f
Author: Tsz-Wo Nicholas Sze <sz...@hortonworks.com>
Authored: Fri Jan 13 15:47:28 2017 +0800
Committer: Tsz-Wo Nicholas Sze <sz...@hortonworks.com>
Committed: Fri Jan 13 15:47:28 2017 +0800

----------------------------------------------------------------------
 .../ratis/client/impl/ClientImplUtils.java      |   2 +-
 .../ratis/examples/RaftExamplesTestUtil.java    |   9 ++
 .../org/apache/ratis/server/RaftServer.java     |  54 ++++++++++
 .../ratis/server/impl/ConfigurationManager.java |   2 +-
 .../ratis/server/impl/RaftServerImpl.java       |  10 +-
 .../ratis/server/impl/ServerImplUtils.java      | 101 +++++++++++++++++++
 .../apache/ratis/server/impl/ServerUtils.java   |  81 ---------------
 .../apache/ratis/server/protocol/TermIndex.java |   4 +-
 .../java/org/apache/ratis/MiniRaftCluster.java  |  19 ++--
 .../ratis/server/impl/RaftServerTestUtil.java   |   5 +-
 .../server/storage/TestSegmentedRaftLog.java    |   5 +-
 .../SimpleStateMachine4Testing.java             |   2 +-
 .../ratis/statemachine/TestStateMachine.java    |   2 +-
 13 files changed, 192 insertions(+), 104 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-ratis/blob/0fed3078/ratis-client/src/main/java/org/apache/ratis/client/impl/ClientImplUtils.java
----------------------------------------------------------------------
diff --git a/ratis-client/src/main/java/org/apache/ratis/client/impl/ClientImplUtils.java b/ratis-client/src/main/java/org/apache/ratis/client/impl/ClientImplUtils.java
index 472c8b4..542f600 100644
--- a/ratis-client/src/main/java/org/apache/ratis/client/impl/ClientImplUtils.java
+++ b/ratis-client/src/main/java/org/apache/ratis/client/impl/ClientImplUtils.java
@@ -23,7 +23,7 @@ import org.apache.ratis.protocol.RaftPeer;
 
 import java.util.Collection;
 
-/** Utilities for the client implementation. */
+/** Client utilities for internal use. */
 public class ClientImplUtils {
   public static RaftClient newRaftClient(
       String clientId, Collection< RaftPeer > peers, String leaderId,

http://git-wip-us.apache.org/repos/asf/incubator-ratis/blob/0fed3078/ratis-examples/src/test/java/org/apache/ratis/examples/RaftExamplesTestUtil.java
----------------------------------------------------------------------
diff --git a/ratis-examples/src/test/java/org/apache/ratis/examples/RaftExamplesTestUtil.java b/ratis-examples/src/test/java/org/apache/ratis/examples/RaftExamplesTestUtil.java
index ff936bd..873c0da 100644
--- a/ratis-examples/src/test/java/org/apache/ratis/examples/RaftExamplesTestUtil.java
+++ b/ratis-examples/src/test/java/org/apache/ratis/examples/RaftExamplesTestUtil.java
@@ -24,11 +24,16 @@ import org.apache.ratis.hadooprpc.MiniRaftClusterWithHadoopRpc;
 import org.apache.ratis.netty.MiniRaftClusterWithNetty;
 import org.apache.ratis.server.simulation.MiniRaftClusterWithSimulatedRpc;
 import org.apache.ratis.statemachine.StateMachine;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
 
 import java.io.IOException;
 import java.util.*;
+import java.util.stream.Collectors;
 
 public class RaftExamplesTestUtil {
+  public static final Logger LOG = LoggerFactory.getLogger(RaftExamplesTestUtil.class);
+
   private static void add(
       Collection<Object[]> clusters, MiniRaftCluster.Factory factory,
       String[] ids, RaftProperties properties)
@@ -68,6 +73,10 @@ public class RaftExamplesTestUtil {
     if (isAll || classes.contains(MiniRaftClusterWithGRpc.class)) {
       add(clusters, MiniRaftClusterWithGRpc.FACTORY, ids.next(), prop);
     }
+    for(int i = 0; i < clusters.size(); i++) {
+      LOG.info(i + ": " + clusters.get(i)[0].getClass().getSimpleName());
+    }
+    LOG.info("#clusters = " + clusters.size());
     return clusters;
   }
 

http://git-wip-us.apache.org/repos/asf/incubator-ratis/blob/0fed3078/ratis-server/src/main/java/org/apache/ratis/server/RaftServer.java
----------------------------------------------------------------------
diff --git a/ratis-server/src/main/java/org/apache/ratis/server/RaftServer.java b/ratis-server/src/main/java/org/apache/ratis/server/RaftServer.java
index 06967ce..f3f63ae 100644
--- a/ratis-server/src/main/java/org/apache/ratis/server/RaftServer.java
+++ b/ratis-server/src/main/java/org/apache/ratis/server/RaftServer.java
@@ -17,12 +17,18 @@
  */
 package org.apache.ratis.server;
 
+import com.google.common.base.Preconditions;
+import org.apache.ratis.conf.RaftProperties;
 import org.apache.ratis.protocol.RaftClientAsynchronousProtocol;
 import org.apache.ratis.protocol.RaftClientProtocol;
+import org.apache.ratis.protocol.RaftPeer;
+import org.apache.ratis.server.impl.ServerImplUtils;
 import org.apache.ratis.server.protocol.RaftServerProtocol;
 import org.apache.ratis.statemachine.StateMachine;
 
 import java.io.Closeable;
+import java.io.IOException;
+import java.util.concurrent.atomic.AtomicInteger;
 
 /** Raft server interface */
 public interface RaftServer extends Closeable, RaftServerProtocol,
@@ -41,4 +47,52 @@ public interface RaftServer extends Closeable, RaftServerProtocol,
    * @return the StateMachine instance.
    */
   StateMachine getStateMachine();
+
+  /** @return a {@link Builder}. */
+  static Builder newBuilder() {
+    return new Builder();
+  }
+
+  /** To build {@link RaftServer} objects. */
+  class Builder {
+    private static final AtomicInteger COUNT = new AtomicInteger();
+
+    private String serverId = RaftServer.class.getSimpleName() + COUNT.incrementAndGet();
+    private StateMachine stateMachine;
+    private Iterable<RaftPeer> peers;
+    private RaftProperties properties;
+
+    /** @return a {@link RaftServer} object. */
+    public RaftServer build() throws IOException {
+      Preconditions.checkNotNull(stateMachine);
+      Preconditions.checkNotNull(peers);
+      Preconditions.checkNotNull(properties);
+
+      return ServerImplUtils.newRaftServer(serverId, stateMachine, peers, properties);
+    }
+
+    /** Set the server ID. */
+    public Builder setServerId(String serverId) {
+      this.serverId = serverId;
+      return this;
+    }
+
+    /** Set the {@link StateMachine} of the server. */
+    public Builder setStateMachine(StateMachine stateMachine) {
+      this.stateMachine = stateMachine;
+      return this;
+    }
+
+    /** Set all the peers (including the server being built) in the Raft cluster. */
+    public Builder setPeers(Iterable<RaftPeer> peers) {
+      this.peers = peers;
+      return this;
+    }
+
+    /** Set {@link RaftProperties}. */
+    public Builder setProperties(RaftProperties properties) {
+      this.properties = properties;
+      return this;
+    }
+  }
 }

http://git-wip-us.apache.org/repos/asf/incubator-ratis/blob/0fed3078/ratis-server/src/main/java/org/apache/ratis/server/impl/ConfigurationManager.java
----------------------------------------------------------------------
diff --git a/ratis-server/src/main/java/org/apache/ratis/server/impl/ConfigurationManager.java b/ratis-server/src/main/java/org/apache/ratis/server/impl/ConfigurationManager.java
index f495c28..7769de1 100644
--- a/ratis-server/src/main/java/org/apache/ratis/server/impl/ConfigurationManager.java
+++ b/ratis-server/src/main/java/org/apache/ratis/server/impl/ConfigurationManager.java
@@ -38,7 +38,7 @@ public class ConfigurationManager {
    */
   private RaftConfiguration currentConf;
 
-  public ConfigurationManager(RaftConfiguration initialConf) {
+  ConfigurationManager(RaftConfiguration initialConf) {
     setInitialConf(initialConf);
   }
 

http://git-wip-us.apache.org/repos/asf/incubator-ratis/blob/0fed3078/ratis-server/src/main/java/org/apache/ratis/server/impl/RaftServerImpl.java
----------------------------------------------------------------------
diff --git a/ratis-server/src/main/java/org/apache/ratis/server/impl/RaftServerImpl.java b/ratis-server/src/main/java/org/apache/ratis/server/impl/RaftServerImpl.java
index d079abf..135ae68 100644
--- a/ratis-server/src/main/java/org/apache/ratis/server/impl/RaftServerImpl.java
+++ b/ratis-server/src/main/java/org/apache/ratis/server/impl/RaftServerImpl.java
@@ -25,10 +25,7 @@ import static org.apache.ratis.util.LifeCycle.State.STARTING;
 import static org.apache.ratis.shaded.proto.RaftProtos.AppendEntriesReplyProto.AppendResult.*;
 import java.io.IOException;
 import java.io.InterruptedIOException;
-import java.util.Arrays;
-import java.util.Collection;
-import java.util.List;
-import java.util.OptionalLong;
+import java.util.*;
 import java.util.concurrent.CompletableFuture;
 import java.util.concurrent.ExecutionException;
 
@@ -104,8 +101,9 @@ public class RaftServerImpl implements RaftServer {
 
   private final LogAppenderFactory appenderFactory;
 
-  public RaftServerImpl(String id, RaftConfiguration raftConf,
-                        RaftProperties properties, StateMachine stateMachine) throws IOException {
+  RaftServerImpl(String id, StateMachine stateMachine,
+                 RaftConfiguration raftConf, RaftProperties properties)
+      throws IOException {
     this.lifeCycle = new LifeCycle(id);
     minTimeoutMs = properties.getInt(
         RaftServerConfigKeys.RAFT_SERVER_RPC_TIMEOUT_MIN_MS_KEY,

http://git-wip-us.apache.org/repos/asf/incubator-ratis/blob/0fed3078/ratis-server/src/main/java/org/apache/ratis/server/impl/ServerImplUtils.java
----------------------------------------------------------------------
diff --git a/ratis-server/src/main/java/org/apache/ratis/server/impl/ServerImplUtils.java b/ratis-server/src/main/java/org/apache/ratis/server/impl/ServerImplUtils.java
new file mode 100644
index 0000000..c6d650f
--- /dev/null
+++ b/ratis-server/src/main/java/org/apache/ratis/server/impl/ServerImplUtils.java
@@ -0,0 +1,101 @@
+/**
+ * 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.ratis.server.impl;
+
+import org.apache.commons.lang.builder.HashCodeBuilder;
+import org.apache.ratis.conf.RaftProperties;
+import org.apache.ratis.protocol.RaftPeer;
+import org.apache.ratis.server.RaftServer;
+import org.apache.ratis.server.protocol.TermIndex;
+import org.apache.ratis.statemachine.StateMachine;
+
+import java.io.IOException;
+
+/** Server utilities for internal use. */
+public class ServerImplUtils {
+  public static RaftServer newRaftServer(
+      String id, StateMachine stateMachine,
+      Iterable<RaftPeer> peers, RaftProperties properties) throws IOException {
+    return newRaftServer(id, stateMachine,
+        RaftConfiguration.newBuilder().setConf(peers).build(),
+        properties);
+  }
+
+  public static RaftServerImpl newRaftServer(
+      String id, StateMachine stateMachine,
+      RaftConfiguration conf, RaftProperties properties) throws IOException {
+    return new RaftServerImpl(id, stateMachine, conf, properties);
+  }
+
+  public static TermIndex newTermIndex(long term, long index) {
+    return new TermIndexImpl(term, index);
+  }
+
+  private static class TermIndexImpl implements TermIndex {
+    private final long term;
+    private final long index; //log index; first index is 1.
+
+    TermIndexImpl(long term, long logIndex) {
+      this.term = term;
+      this.index = logIndex;
+    }
+
+    @Override
+    public long getTerm() {
+      return term;
+    }
+
+    @Override
+    public long getIndex() {
+      return index;
+    }
+
+    @Override
+    public int compareTo(TermIndex that) {
+      final int d = Long.compare(this.getTerm(), that.getTerm());
+      return d != 0 ? d : Long.compare(this.getIndex(), that.getIndex());
+    }
+
+    @Override
+    public boolean equals(Object obj) {
+      if (obj == this) {
+        return true;
+      } else if (obj == null || !(obj instanceof TermIndexImpl)) {
+        return false;
+      }
+
+      final TermIndexImpl that = (TermIndexImpl) obj;
+      return this.getTerm() == that.getTerm()
+          && this.getIndex() == that.getIndex();
+    }
+
+    @Override
+    public int hashCode() {
+      return new HashCodeBuilder().append(term).append(index).hashCode();
+    }
+
+    private static String toString(long n) {
+      return n < 0 ? "~" : "" + n;
+    }
+
+    @Override
+    public String toString() {
+      return "(t:" + toString(term) + ", i:" + toString(index) + ")";
+    }
+  }
+}

http://git-wip-us.apache.org/repos/asf/incubator-ratis/blob/0fed3078/ratis-server/src/main/java/org/apache/ratis/server/impl/ServerUtils.java
----------------------------------------------------------------------
diff --git a/ratis-server/src/main/java/org/apache/ratis/server/impl/ServerUtils.java b/ratis-server/src/main/java/org/apache/ratis/server/impl/ServerUtils.java
deleted file mode 100644
index 3dc9ab4..0000000
--- a/ratis-server/src/main/java/org/apache/ratis/server/impl/ServerUtils.java
+++ /dev/null
@@ -1,81 +0,0 @@
-/**
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- *     http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-package org.apache.ratis.server.impl;
-
-import org.apache.commons.lang.builder.HashCodeBuilder;
-import org.apache.ratis.server.protocol.TermIndex;
-
-/** Server utilities for internal use. */
-public class ServerUtils {
-  public static TermIndex newTermIndex(long term, long index) {
-    return new TermIndexImpl(term, index);
-  }
-
-  private static class TermIndexImpl implements TermIndex {
-    private final long term;
-    private final long index; //log index; first index is 1.
-
-    TermIndexImpl(long term, long logIndex) {
-      this.term = term;
-      this.index = logIndex;
-    }
-
-    @Override
-    public long getTerm() {
-      return term;
-    }
-
-    @Override
-    public long getIndex() {
-      return index;
-    }
-
-    @Override
-    public int compareTo(TermIndex that) {
-      final int d = Long.compare(this.getTerm(), that.getTerm());
-      return d != 0 ? d : Long.compare(this.getIndex(), that.getIndex());
-    }
-
-    @Override
-    public boolean equals(Object obj) {
-      if (obj == this) {
-        return true;
-      } else if (obj == null || !(obj instanceof TermIndexImpl)) {
-        return false;
-      }
-
-      final TermIndexImpl that = (TermIndexImpl) obj;
-      return this.getTerm() == that.getTerm()
-          && this.getIndex() == that.getIndex();
-    }
-
-    @Override
-    public int hashCode() {
-      return new HashCodeBuilder().append(term).append(index).hashCode();
-    }
-
-    private static String toString(long n) {
-      return n < 0 ? "~" : "" + n;
-    }
-
-    @Override
-    public String toString() {
-      return "(t:" + toString(term) + ", i:" + toString(index) + ")";
-    }
-  }
-}

http://git-wip-us.apache.org/repos/asf/incubator-ratis/blob/0fed3078/ratis-server/src/main/java/org/apache/ratis/server/protocol/TermIndex.java
----------------------------------------------------------------------
diff --git a/ratis-server/src/main/java/org/apache/ratis/server/protocol/TermIndex.java b/ratis-server/src/main/java/org/apache/ratis/server/protocol/TermIndex.java
index 665f5d5..477b70c 100644
--- a/ratis-server/src/main/java/org/apache/ratis/server/protocol/TermIndex.java
+++ b/ratis-server/src/main/java/org/apache/ratis/server/protocol/TermIndex.java
@@ -17,7 +17,7 @@
  */
 package org.apache.ratis.server.protocol;
 
-import org.apache.ratis.server.impl.ServerUtils;
+import org.apache.ratis.server.impl.ServerImplUtils;
 
 /** The term and the log index defined in the Raft consensus algorithm. */
 public interface TermIndex extends Comparable<TermIndex> {
@@ -29,7 +29,7 @@ public interface TermIndex extends Comparable<TermIndex> {
 
   /** Create a new {@link TermIndex} instance. */
   static TermIndex newTermIndex(long term, long index) {
-    return ServerUtils.newTermIndex(term, index);
+    return ServerImplUtils.newTermIndex(term, index);
   }
 }
 

http://git-wip-us.apache.org/repos/asf/incubator-ratis/blob/0fed3078/ratis-server/src/test/java/org/apache/ratis/MiniRaftCluster.java
----------------------------------------------------------------------
diff --git a/ratis-server/src/test/java/org/apache/ratis/MiniRaftCluster.java b/ratis-server/src/test/java/org/apache/ratis/MiniRaftCluster.java
index 7eb6ddb..d824014 100644
--- a/ratis-server/src/test/java/org/apache/ratis/MiniRaftCluster.java
+++ b/ratis-server/src/test/java/org/apache/ratis/MiniRaftCluster.java
@@ -27,6 +27,7 @@ import org.apache.ratis.server.RaftServerRpc;
 import org.apache.ratis.server.impl.DelayLocalExecutionInjection;
 import org.apache.ratis.server.impl.RaftConfiguration;
 import org.apache.ratis.server.impl.RaftServerImpl;
+import org.apache.ratis.server.impl.ServerImplUtils;
 import org.apache.ratis.server.storage.MemoryRaftLog;
 import org.apache.ratis.server.storage.RaftLog;
 import org.apache.ratis.statemachine.BaseStateMachine;
@@ -104,6 +105,10 @@ public abstract class MiniRaftCluster {
     }
   }
 
+  public static RaftConfiguration initConfiguration(int numServers) {
+    return initConfiguration(generateIds(numServers, 0));
+  }
+
   public static RaftConfiguration initConfiguration(String[] ids) {
     return RaftConfiguration.newBuilder()
         .setConf(Arrays.stream(ids).map(RaftPeer::new).collect(Collectors.toList()))
@@ -142,7 +147,7 @@ public abstract class MiniRaftCluster {
     this.testBaseDir = getBaseDirectory();
 
     conf.getPeers().forEach(
-        p -> servers.put(p.getId(), newRaftServer(p.getId(), conf, formatted)));
+        p -> servers.put(p.getId(), newRaftServer(p.getId(), formatted)));
 
     ExitUtils.disableSystemExit();
   }
@@ -168,7 +173,7 @@ public abstract class MiniRaftCluster {
   public void restartServer(String id, boolean format) throws IOException {
     killServer(id);
     servers.remove(id);
-    servers.put(id, newRaftServer(id, conf, format));
+    servers.put(id, newRaftServer(id, format));
   }
 
   public final void restart(boolean format) throws IOException {
@@ -177,7 +182,7 @@ public abstract class MiniRaftCluster {
     List<String> idList = new ArrayList<>(servers.keySet());
     for (String id : idList) {
       servers.remove(id);
-      servers.put(id, newRaftServer(id, conf, format));
+      servers.put(id, newRaftServer(id, format));
     }
 
     setPeerRpc();
@@ -196,8 +201,7 @@ public abstract class MiniRaftCluster {
     return conf;
   }
 
-  private RaftServerImpl newRaftServer(String id, RaftConfiguration conf,
-                                       boolean format) {
+  private RaftServerImpl newRaftServer(String id, boolean format) {
     final RaftServerImpl s;
     try {
       final String dirStr = testBaseDir + id;
@@ -205,7 +209,8 @@ public abstract class MiniRaftCluster {
         formatDir(dirStr);
       }
       properties.set(RaftServerConfigKeys.RAFT_SERVER_STORAGE_DIR_KEY, dirStr);
-      s = new RaftServerImpl(id, conf, properties, getStateMachine4Test(properties));
+      final StateMachine stateMachine = getStateMachine4Test(properties);
+      s = ServerImplUtils.newRaftServer(id, stateMachine, conf, properties);
     } catch (IOException e) {
       throw new RuntimeException(e);
     }
@@ -255,7 +260,7 @@ public abstract class MiniRaftCluster {
     // create and add new RaftServers
     final List<RaftServerImpl> newServers = new ArrayList<>(ids.length);
     for (RaftPeer p : newPeers) {
-      RaftServerImpl newServer = newRaftServer(p.getId(), conf, true);
+      RaftServerImpl newServer = newRaftServer(p.getId(), true);
       Preconditions.checkArgument(!servers.containsKey(p.getId()));
       servers.put(p.getId(), newServer);
       newServers.add(newServer);

http://git-wip-us.apache.org/repos/asf/incubator-ratis/blob/0fed3078/ratis-server/src/test/java/org/apache/ratis/server/impl/RaftServerTestUtil.java
----------------------------------------------------------------------
diff --git a/ratis-server/src/test/java/org/apache/ratis/server/impl/RaftServerTestUtil.java b/ratis-server/src/test/java/org/apache/ratis/server/impl/RaftServerTestUtil.java
index dc10bd3..3cf2ef6 100644
--- a/ratis-server/src/test/java/org/apache/ratis/server/impl/RaftServerTestUtil.java
+++ b/ratis-server/src/test/java/org/apache/ratis/server/impl/RaftServerTestUtil.java
@@ -69,7 +69,8 @@ public class RaftServerTestUtil {
     Assert.assertEquals(peers.length, numIncluded + deadIncluded);
   }
 
-  public static StateMachine getStateMachine(RaftServerImpl s) {
-    return s.getStateMachine();
+  public static ConfigurationManager newConfigurationManager(
+      RaftConfiguration initialConf) {
+    return new ConfigurationManager(initialConf);
   }
 }

http://git-wip-us.apache.org/repos/asf/incubator-ratis/blob/0fed3078/ratis-server/src/test/java/org/apache/ratis/server/storage/TestSegmentedRaftLog.java
----------------------------------------------------------------------
diff --git a/ratis-server/src/test/java/org/apache/ratis/server/storage/TestSegmentedRaftLog.java b/ratis-server/src/test/java/org/apache/ratis/server/storage/TestSegmentedRaftLog.java
index 405a1a5..ce595c5 100644
--- a/ratis-server/src/test/java/org/apache/ratis/server/storage/TestSegmentedRaftLog.java
+++ b/ratis-server/src/test/java/org/apache/ratis/server/storage/TestSegmentedRaftLog.java
@@ -36,6 +36,7 @@ import org.apache.ratis.conf.RaftProperties;
 import org.apache.ratis.server.RaftServerConfigKeys;
 import org.apache.ratis.server.impl.ConfigurationManager;
 import org.apache.ratis.server.impl.RaftServerConstants;
+import org.apache.ratis.server.impl.RaftServerTestUtil;
 import org.apache.ratis.shaded.proto.RaftProtos.LogEntryProto;
 import org.apache.ratis.util.FileUtils;
 import org.apache.ratis.util.ProtoUtils;
@@ -69,8 +70,8 @@ public class TestSegmentedRaftLog {
   private File storageDir;
   private RaftProperties properties;
   private RaftStorage storage;
-  private final ConfigurationManager cm = new ConfigurationManager(
-      MiniRaftCluster.initConfiguration(MiniRaftCluster.generateIds(3, 0)));
+  private final ConfigurationManager cm = RaftServerTestUtil.newConfigurationManager(
+      MiniRaftCluster.initConfiguration(3));
 
   @Before
   public void setup() throws Exception {

http://git-wip-us.apache.org/repos/asf/incubator-ratis/blob/0fed3078/ratis-server/src/test/java/org/apache/ratis/statemachine/SimpleStateMachine4Testing.java
----------------------------------------------------------------------
diff --git a/ratis-server/src/test/java/org/apache/ratis/statemachine/SimpleStateMachine4Testing.java b/ratis-server/src/test/java/org/apache/ratis/statemachine/SimpleStateMachine4Testing.java
index cc82371..952c40b 100644
--- a/ratis-server/src/test/java/org/apache/ratis/statemachine/SimpleStateMachine4Testing.java
+++ b/ratis-server/src/test/java/org/apache/ratis/statemachine/SimpleStateMachine4Testing.java
@@ -63,7 +63,7 @@ public class SimpleStateMachine4Testing extends BaseStateMachine {
   public static final boolean RAFT_TEST_SIMPLE_STATE_MACHINE_TAKE_SNAPSHOT_DEFAULT = false;
 
   public static SimpleStateMachine4Testing get(RaftServerImpl s) {
-    return (SimpleStateMachine4Testing)RaftServerTestUtil.getStateMachine(s);
+    return (SimpleStateMachine4Testing)s.getStateMachine();
   }
 
   private final List<LogEntryProto> list =

http://git-wip-us.apache.org/repos/asf/incubator-ratis/blob/0fed3078/ratis-server/src/test/java/org/apache/ratis/statemachine/TestStateMachine.java
----------------------------------------------------------------------
diff --git a/ratis-server/src/test/java/org/apache/ratis/statemachine/TestStateMachine.java b/ratis-server/src/test/java/org/apache/ratis/statemachine/TestStateMachine.java
index cdce568..bac2c38 100644
--- a/ratis-server/src/test/java/org/apache/ratis/statemachine/TestStateMachine.java
+++ b/ratis-server/src/test/java/org/apache/ratis/statemachine/TestStateMachine.java
@@ -102,7 +102,7 @@ public class TestStateMachine {
 
   static class SMTransactionContext extends SimpleStateMachine4Testing {
     public static SMTransactionContext get(RaftServerImpl s) {
-      return (SMTransactionContext)RaftServerTestUtil.getStateMachine(s);
+      return (SMTransactionContext)s.getStateMachine();
     }
 
     AtomicReference<Throwable> throwable = new AtomicReference<>(null);


[35/54] [abbrv] incubator-ratis git commit: Renamed the packages from raft to ratis in preperation for Apache Incubation - Moved all java packages from org.apache.raft to org.apache.ratis. - Moved native package to org_apache_ratis, and native lib to l

Posted by ji...@apache.org.
http://git-wip-us.apache.org/repos/asf/incubator-ratis/blob/7e71a2e0/raft-server/src/main/java/org/apache/raft/server/storage/SegmentedRaftLog.java
----------------------------------------------------------------------
diff --git a/raft-server/src/main/java/org/apache/raft/server/storage/SegmentedRaftLog.java b/raft-server/src/main/java/org/apache/raft/server/storage/SegmentedRaftLog.java
deleted file mode 100644
index 293e1a4..0000000
--- a/raft-server/src/main/java/org/apache/raft/server/storage/SegmentedRaftLog.java
+++ /dev/null
@@ -1,327 +0,0 @@
-/**
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- *     http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-package org.apache.raft.server.storage;
-
-import com.google.common.annotations.VisibleForTesting;
-import com.google.common.base.Preconditions;
-import org.apache.commons.io.Charsets;
-import org.apache.raft.conf.RaftProperties;
-import org.apache.raft.server.impl.ConfigurationManager;
-import org.apache.raft.server.impl.RaftServerImpl;
-import org.apache.raft.server.impl.RaftServerConstants;
-import org.apache.raft.server.storage.RaftStorageDirectory.LogPathAndIndex;
-import org.apache.raft.shaded.proto.RaftProtos.LogEntryProto;
-import org.apache.raft.util.AutoCloseableLock;
-import org.apache.raft.util.CodeInjectionForTesting;
-
-import java.io.File;
-import java.io.IOException;
-import java.util.Iterator;
-import java.util.List;
-
-import static org.apache.raft.server.RaftServerConfigKeys.RAFT_LOG_SEGMENT_MAX_SIZE_DEFAULT;
-import static org.apache.raft.server.RaftServerConfigKeys.RAFT_LOG_SEGMENT_MAX_SIZE_KEY;
-
-/**
- * The RaftLog implementation that writes log entries into segmented files in
- * local disk.
- *
- * The max log segment size is 8MB. The real log segment size may not be
- * exactly equal to this limit. If a log entry's size exceeds 8MB, this entry
- * will be stored in a single segment.
- *
- * There are two types of segments: closed segment and open segment. The former
- * is named as "log_startindex-endindex", the later is named as
- * "log_inprogress_startindex".
- *
- * There can be multiple closed segments but there is at most one open segment.
- * When the open segment reaches the size limit, or the log term increases, we
- * close the open segment and start a new open segment. A closed segment cannot
- * be appended anymore, but it can be truncated in case that a follower's log is
- * inconsistent with the current leader.
- *
- * Every closed segment should be non-empty, i.e., it should contain at least
- * one entry.
- *
- * There should not be any gap between segments. The first segment may not start
- * from index 0 since there may be snapshots as log compaction. The last index
- * in segments should be no smaller than the last index of snapshot, otherwise
- * we may have hole when append further log.
- */
-public class SegmentedRaftLog extends RaftLog {
-  static final String HEADER_STR = "RAFTLOG1";
-  static final byte[] HEADER_BYTES = HEADER_STR.getBytes(Charsets.UTF_8);
-
-  /**
-   * I/O task definitions.
-   */
-  static abstract class Task {
-    private boolean done = false;
-
-    synchronized void done() {
-      done = true;
-      notifyAll();
-    }
-
-    synchronized void waitForDone() throws InterruptedException {
-      while (!done) {
-        wait();
-      }
-    }
-
-    abstract void execute() throws IOException;
-
-    abstract long getEndIndex();
-
-    @Override
-    public String toString() {
-      return getClass().getSimpleName() + "-" + getEndIndex();
-    }
-  }
-  private static final ThreadLocal<Task> myTask = new ThreadLocal<>();
-
-  private final RaftStorage storage;
-  private final RaftLogCache cache;
-  private final RaftLogWorker fileLogWorker;
-  private final long segmentMaxSize;
-
-  public SegmentedRaftLog(String selfId, RaftServerImpl server, RaftStorage storage,
-                          long lastIndexInSnapshot, RaftProperties properties) throws IOException {
-    super(selfId);
-    this.storage = storage;
-    this.segmentMaxSize = properties.getLong(RAFT_LOG_SEGMENT_MAX_SIZE_KEY,
-        RAFT_LOG_SEGMENT_MAX_SIZE_DEFAULT);
-    cache = new RaftLogCache();
-    fileLogWorker = new RaftLogWorker(server, storage, properties);
-    lastCommitted.set(lastIndexInSnapshot);
-  }
-
-  @Override
-  public void open(ConfigurationManager confManager, long lastIndexInSnapshot)
-      throws IOException {
-    loadLogSegments(confManager, lastIndexInSnapshot);
-    File openSegmentFile = null;
-    if (cache.getOpenSegment() != null) {
-      openSegmentFile = storage.getStorageDir()
-          .getOpenLogFile(cache.getOpenSegment().getStartIndex());
-    }
-    fileLogWorker.start(Math.max(cache.getEndIndex(), lastIndexInSnapshot),
-        openSegmentFile);
-    super.open(confManager, lastIndexInSnapshot);
-  }
-
-  @Override
-  public long getStartIndex() {
-    return cache.getStartIndex();
-  }
-
-  private void loadLogSegments(ConfigurationManager confManager,
-      long lastIndexInSnapshot) throws IOException {
-    try(AutoCloseableLock writeLock = writeLock()) {
-      List<LogPathAndIndex> paths = storage.getStorageDir().getLogSegmentFiles();
-      for (LogPathAndIndex pi : paths) {
-        LogSegment logSegment = parseLogSegment(pi, confManager);
-        cache.addSegment(logSegment);
-      }
-
-      // if the largest index is smaller than the last index in snapshot, we do
-      // not load the log to avoid holes between log segments. This may happen
-      // when the local I/O worker is too slow to persist log (slower than
-      // committing the log and taking snapshot)
-      if (!cache.isEmpty() && cache.getEndIndex() < lastIndexInSnapshot) {
-        LOG.warn("End log index {} is smaller than last index in snapshot {}",
-            cache.getEndIndex(), lastIndexInSnapshot);
-        cache.clear();
-        // TODO purge all segment files
-      }
-    }
-  }
-
-  private LogSegment parseLogSegment(LogPathAndIndex pi,
-      ConfigurationManager confManager) throws IOException {
-    final boolean isOpen = pi.endIndex == RaftServerConstants.INVALID_LOG_INDEX;
-    return LogSegment.loadSegment(pi.path.toFile(), pi.startIndex, pi.endIndex,
-        isOpen, confManager);
-  }
-
-  @Override
-  public LogEntryProto get(long index) {
-    checkLogState();
-    try(AutoCloseableLock readLock = readLock()) {
-      return cache.getEntry(index);
-    }
-  }
-
-  @Override
-  public LogEntryProto[] getEntries(long startIndex, long endIndex) {
-    checkLogState();
-    try(AutoCloseableLock readLock = readLock()) {
-      return cache.getEntries(startIndex, endIndex);
-    }
-  }
-
-  @Override
-  public LogEntryProto getLastEntry() {
-    checkLogState();
-    try(AutoCloseableLock readLock = readLock()) {
-      return cache.getLastEntry();
-    }
-  }
-
-  /**
-   * The method, along with {@link #appendEntry} and
-   * {@link #append(LogEntryProto...)} need protection of RaftServer's lock.
-   */
-  @Override
-  void truncate(long index) {
-    checkLogState();
-    try(AutoCloseableLock writeLock = writeLock()) {
-      RaftLogCache.TruncationSegments ts = cache.truncate(index);
-      if (ts != null) {
-        Task task = fileLogWorker.truncate(ts);
-        myTask.set(task);
-      }
-    }
-  }
-
-  @Override
-  void appendEntry(LogEntryProto entry) {
-    checkLogState();
-    try(AutoCloseableLock writeLock = writeLock()) {
-      final LogSegment currentOpenSegment = cache.getOpenSegment();
-      if (currentOpenSegment == null) {
-        cache.addSegment(LogSegment.newOpenSegment(entry.getIndex()));
-        fileLogWorker.startLogSegment(getNextIndex());
-      } else if (isSegmentFull(currentOpenSegment, entry)) {
-        cache.rollOpenSegment(true);
-        fileLogWorker.rollLogSegment(currentOpenSegment);
-      } else if (currentOpenSegment.numOfEntries() > 0 &&
-          currentOpenSegment.getLastRecord().entry.getTerm() != entry.getTerm()) {
-        // the term changes
-        final long currentTerm = currentOpenSegment.getLastRecord().entry
-            .getTerm();
-        Preconditions.checkState(currentTerm < entry.getTerm(),
-            "open segment's term %s is larger than the new entry's term %s",
-            currentTerm, entry.getTerm());
-        cache.rollOpenSegment(true);
-        fileLogWorker.rollLogSegment(currentOpenSegment);
-      }
-
-      cache.appendEntry(entry);
-      myTask.set(fileLogWorker.writeLogEntry(entry));
-    }
-  }
-
-  private boolean isSegmentFull(LogSegment segment, LogEntryProto entry) {
-    if (segment.getTotalSize() >= segmentMaxSize) {
-      return true;
-    } else {
-      final long entrySize = LogSegment.getEntrySize(entry);
-      // if entry size is greater than the max segment size, write it directly
-      // into the current segment
-      return entrySize <= segmentMaxSize &&
-          segment.getTotalSize() + entrySize > segmentMaxSize;
-    }
-  }
-
-  @Override
-  public void append(LogEntryProto... entries) {
-    checkLogState();
-    if (entries == null || entries.length == 0) {
-      return;
-    }
-    try(AutoCloseableLock writeLock = writeLock()) {
-      Iterator<LogEntryProto> iter = cache.iterator(entries[0].getIndex());
-      int index = 0;
-      long truncateIndex = -1;
-      for (; iter.hasNext() && index < entries.length; index++) {
-        LogEntryProto storedEntry = iter.next();
-        Preconditions.checkState(
-            storedEntry.getIndex() == entries[index].getIndex(),
-            "The stored entry's index %s is not consistent with" +
-                " the received entries[%s]'s index %s", storedEntry.getIndex(),
-            index, entries[index].getIndex());
-
-        if (storedEntry.getTerm() != entries[index].getTerm()) {
-          // we should truncate from the storedEntry's index
-          truncateIndex = storedEntry.getIndex();
-          break;
-        }
-      }
-      if (truncateIndex != -1) {
-        // truncate from truncateIndex
-        truncate(truncateIndex);
-      }
-      // append from entries[index]
-      for (int i = index; i < entries.length; i++) {
-        appendEntry(entries[i]);
-      }
-    }
-  }
-
-  @Override
-  public void logSync() throws InterruptedException {
-    CodeInjectionForTesting.execute(LOG_SYNC, getSelfId(), null);
-    final Task task = myTask.get();
-    if (task != null) {
-      task.waitForDone();
-    }
-  }
-
-  @Override
-  public long getLatestFlushedIndex() {
-    return fileLogWorker.getFlushedIndex();
-  }
-
-  /**
-   * {@inheritDoc}
-   *
-   * This operation is protected by the RaftServer's lock
-   */
-  @Override
-  public void writeMetadata(long term, String votedFor) throws IOException {
-    storage.getMetaFile().set(term, votedFor);
-  }
-
-  @Override
-  public Metadata loadMetadata() throws IOException {
-    return new Metadata(storage.getMetaFile().getVotedFor(),
-        storage.getMetaFile().getTerm());
-  }
-
-  @Override
-  public void syncWithSnapshot(long lastSnapshotIndex) {
-    fileLogWorker.syncWithSnapshot(lastSnapshotIndex);
-    // TODO purge log files and normal/tmp/corrupt snapshot files
-    // if the last index in snapshot is larger than the index of the last
-    // log entry, we should delete all the log entries and their cache to avoid
-    // gaps between log segments.
-  }
-
-  @Override
-  public void close() throws IOException {
-    super.close();
-    fileLogWorker.close();
-    storage.close();
-  }
-
-  @VisibleForTesting
-  RaftLogCache getRaftLogCache() {
-    return cache;
-  }
-}

http://git-wip-us.apache.org/repos/asf/incubator-ratis/blob/7e71a2e0/raft-server/src/main/java/org/apache/raft/server/storage/SnapshotManager.java
----------------------------------------------------------------------
diff --git a/raft-server/src/main/java/org/apache/raft/server/storage/SnapshotManager.java b/raft-server/src/main/java/org/apache/raft/server/storage/SnapshotManager.java
deleted file mode 100644
index 8ab2833..0000000
--- a/raft-server/src/main/java/org/apache/raft/server/storage/SnapshotManager.java
+++ /dev/null
@@ -1,133 +0,0 @@
-/**
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- *     http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-package org.apache.raft.server.storage;
-
-import com.google.common.base.Preconditions;
-import org.apache.raft.io.MD5Hash;
-import org.apache.raft.shaded.proto.RaftProtos.FileChunkProto;
-import org.apache.raft.shaded.proto.RaftProtos.InstallSnapshotRequestProto;
-import org.apache.raft.statemachine.SnapshotInfo;
-import org.apache.raft.statemachine.StateMachine;
-import org.apache.raft.util.FileUtils;
-import org.apache.raft.util.MD5FileUtil;
-import org.apache.raft.util.RaftUtils;
-import org.slf4j.Logger;
-import org.slf4j.LoggerFactory;
-
-import java.io.File;
-import java.io.FileOutputStream;
-import java.io.IOException;
-import java.nio.channels.FileChannel;
-
-/**
- * Manage snapshots of a raft peer.
- * TODO: snapshot should be treated as compaction log thus can be merged into
- *       RaftLog. In this way we can have a unified getLastTermIndex interface.
- */
-public class SnapshotManager {
-  private static final Logger LOG = LoggerFactory.getLogger(SnapshotManager.class);
-
-  private final RaftStorage storage;
-  private final String selfId;
-
-  public SnapshotManager(RaftStorage storage, String selfId)
-      throws IOException {
-    this.storage = storage;
-    this.selfId = selfId;
-  }
-
-  public void installSnapshot(StateMachine stateMachine,
-      InstallSnapshotRequestProto request) throws IOException {
-    final long lastIncludedIndex = request.getTermIndex().getIndex();
-    final RaftStorageDirectory dir = storage.getStorageDir();
-
-    File tmpDir = dir.getNewTempDir();
-    tmpDir.mkdirs();
-    tmpDir.deleteOnExit();
-
-    LOG.info("Installing snapshot:{}, to tmp dir:{}", request, tmpDir);
-
-    // TODO: Make sure that subsequent requests for the same installSnapshot are coming in order,
-    // and are not lost when whole request cycle is done. Check requestId and requestIndex here
-
-    for (FileChunkProto chunk : request.getFileChunksList()) {
-      SnapshotInfo pi = stateMachine.getLatestSnapshot();
-      if (pi != null && pi.getTermIndex().getIndex() >= lastIncludedIndex) {
-        throw new IOException("There exists snapshot file "
-            + pi.getFiles() + " in " + selfId
-            + " with endIndex >= lastIncludedIndex " + lastIncludedIndex);
-      }
-
-      String fileName = chunk.getFilename(); // this is relative to the root dir
-      // TODO: assumes flat layout inside SM dir
-      File tmpSnapshotFile = new File(tmpDir,
-          new File(dir.getRoot(), fileName).getName());
-
-      FileOutputStream out = null;
-      try {
-        // if offset is 0, delete any existing temp snapshot file if it has the
-        // same last index.
-        if (chunk.getOffset() == 0) {
-          if (tmpSnapshotFile.exists()) {
-            FileUtils.fullyDelete(tmpSnapshotFile);
-          }
-          // create the temp snapshot file and put padding inside
-          out = new FileOutputStream(tmpSnapshotFile);
-        } else {
-          Preconditions.checkState(tmpSnapshotFile.exists());
-          out = new FileOutputStream(tmpSnapshotFile, true);
-          FileChannel fc = out.getChannel();
-          fc.position(chunk.getOffset());
-        }
-
-        // write data to the file
-        out.write(chunk.getData().toByteArray());
-      } finally {
-        RaftUtils.cleanup(null, out);
-      }
-
-      // rename the temp snapshot file if this is the last chunk. also verify
-      // the md5 digest and create the md5 meta-file.
-      if (chunk.getDone()) {
-        final MD5Hash expectedDigest =
-            new MD5Hash(chunk.getFileDigest().toByteArray());
-        // calculate the checksum of the snapshot file and compare it with the
-        // file digest in the request
-        MD5Hash digest = MD5FileUtil.computeMd5ForFile(tmpSnapshotFile);
-        if (!digest.equals(expectedDigest)) {
-          LOG.warn("The snapshot md5 digest {} does not match expected {}",
-              digest, expectedDigest);
-          // rename the temp snapshot file to .corrupt
-//          NativeIO.renameTo(tmpSnapshotFile, // TODO:
-//              dir.getCorruptSnapshotFile(lastIncludedTerm, lastIncludedIndex));
-          throw new IOException("MD5 mismatch for snapshot-" + lastIncludedIndex
-              + " installation");
-        } else {
-          MD5FileUtil.saveMD5File(tmpSnapshotFile, digest);
-        }
-      }
-    }
-
-    if (request.getDone()) {
-      LOG.info("Install snapshot is done, renaming tnp dir:{} to:{}",
-          tmpDir, dir.getStateMachineDir());
-      dir.getStateMachineDir().delete();
-      tmpDir.renameTo(dir.getStateMachineDir());
-    }
-  }
-}

http://git-wip-us.apache.org/repos/asf/incubator-ratis/blob/7e71a2e0/raft-server/src/main/java/org/apache/raft/statemachine/BaseStateMachine.java
----------------------------------------------------------------------
diff --git a/raft-server/src/main/java/org/apache/raft/statemachine/BaseStateMachine.java b/raft-server/src/main/java/org/apache/raft/statemachine/BaseStateMachine.java
deleted file mode 100644
index ccc52c7..0000000
--- a/raft-server/src/main/java/org/apache/raft/statemachine/BaseStateMachine.java
+++ /dev/null
@@ -1,150 +0,0 @@
-/**
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- *     http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-
-package org.apache.raft.statemachine;
-
-import org.apache.raft.conf.RaftProperties;
-import org.apache.raft.protocol.Message;
-import org.apache.raft.protocol.RaftClientReply;
-import org.apache.raft.protocol.RaftClientRequest;
-import org.apache.raft.server.impl.RaftConfiguration;
-import org.apache.raft.server.impl.RaftServerConstants;
-import org.apache.raft.server.storage.RaftStorage;
-import org.apache.raft.shaded.proto.RaftProtos.SMLogEntryProto;
-import org.apache.raft.util.LifeCycle;
-
-import java.io.IOException;
-import java.util.Collection;
-import java.util.concurrent.CompletableFuture;
-
-/**
- * Base implementation for StateMachines.
- */
-public class BaseStateMachine implements StateMachine {
-
-  protected RaftProperties properties;
-  protected RaftStorage storage;
-  protected RaftConfiguration raftConf;
-  protected final LifeCycle lifeCycle = new LifeCycle(getClass().getSimpleName());
-
-  @Override
-  public LifeCycle.State getLifeCycleState() {
-    return lifeCycle.getCurrentState();
-  }
-
-  @Override
-  public void initialize(String id, RaftProperties properties, RaftStorage storage)
-      throws IOException {
-    lifeCycle.setName(getClass().getSimpleName() + ":" + id);
-    this.properties = properties;
-    this.storage = storage;
-  }
-
-  @Override
-  public void setRaftConfiguration(RaftConfiguration conf) {
-    this.raftConf = conf;
-  }
-
-  @Override
-  public RaftConfiguration getRaftConfiguration() {
-    return this.raftConf;
-  }
-
-  @Override
-  public SnapshotInfo getLatestSnapshot() {
-    return getStateMachineStorage().getLatestSnapshot();
-  }
-
-  @Override
-  public void notifyNotLeader(Collection<TransactionContext> pendingEntries) throws IOException {
-    // do nothing
-  }
-
-  public void pause() {
-  }
-
-  @Override
-  public void reinitialize(String id, RaftProperties properties, RaftStorage storage)
-      throws IOException {
-  }
-
-  @Override
-  public TransactionContext applyTransactionSerial(TransactionContext trx) throws IOException {
-    return trx;
-  }
-
-  @Override
-  public CompletableFuture<Message> applyTransaction(TransactionContext trx) throws IOException {
-    // return the same message contained in the entry
-    Message msg = () -> trx.getLogEntry().get().getSmLogEntry().getData();
-    return CompletableFuture.completedFuture(msg);
-  }
-
-  @Override
-  public long takeSnapshot() throws IOException {
-    return RaftServerConstants.INVALID_LOG_INDEX;
-  }
-
-  @Override
-  public StateMachineStorage getStateMachineStorage() {
-    return new StateMachineStorage() {
-      @Override
-      public void init(RaftStorage raftStorage) throws IOException {
-      }
-
-      @Override
-      public SnapshotInfo getLatestSnapshot() {
-        return null;
-      }
-
-      @Override
-      public void format() throws IOException {
-      }
-    };
-  }
-
-  @Override
-  public CompletableFuture<RaftClientReply> query(
-      RaftClientRequest request) {
-    return null;
-  }
-
-  @Override
-  public TransactionContext startTransaction(RaftClientRequest request)
-      throws IOException {
-    return new TransactionContext(this, request,
-        SMLogEntryProto.newBuilder()
-            .setData(request.getMessage().getContent())
-            .build());
-  }
-
-  @Override
-  public TransactionContext cancelTransaction(TransactionContext trx) throws IOException {
-    return trx;
-  }
-
-  @Override
-  public TransactionContext preAppendTransaction(TransactionContext trx) throws IOException {
-    return trx;
-  }
-
-  @Override
-  public void close() throws IOException {
-    // do nothing
-  }
-}

http://git-wip-us.apache.org/repos/asf/incubator-ratis/blob/7e71a2e0/raft-server/src/main/java/org/apache/raft/statemachine/FileListSnapshotInfo.java
----------------------------------------------------------------------
diff --git a/raft-server/src/main/java/org/apache/raft/statemachine/FileListSnapshotInfo.java b/raft-server/src/main/java/org/apache/raft/statemachine/FileListSnapshotInfo.java
deleted file mode 100644
index b65fc13..0000000
--- a/raft-server/src/main/java/org/apache/raft/statemachine/FileListSnapshotInfo.java
+++ /dev/null
@@ -1,64 +0,0 @@
-/**
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- *     http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-package org.apache.raft.statemachine;
-
-import org.apache.raft.server.protocol.TermIndex;
-import org.apache.raft.server.storage.FileInfo;
-
-import java.util.Collections;
-import java.util.List;
-
-/**
- * Each snapshot has a list of files.
- *
- * The objects of this class are immutable.
- */
-public class FileListSnapshotInfo implements SnapshotInfo {
-  private final TermIndex termIndex;
-  private final List<FileInfo> files;
-
-  public FileListSnapshotInfo(List<FileInfo> files, long term, long index) {
-    this.termIndex = TermIndex.newTermIndex(term, index);
-    this.files = Collections.unmodifiableList(files);
-  }
-
-  @Override
-  public TermIndex getTermIndex() {
-    return termIndex;
-  }
-
-  @Override
-  public long getTerm() {
-    return termIndex.getTerm();
-  }
-
-  @Override
-  public long getIndex() {
-    return termIndex.getIndex();
-  }
-
-  @Override
-  public List<FileInfo> getFiles() {
-    return files;
-  }
-
-  @Override
-  public String toString() {
-    return termIndex + ":" + files;
-  }
-}

http://git-wip-us.apache.org/repos/asf/incubator-ratis/blob/7e71a2e0/raft-server/src/main/java/org/apache/raft/statemachine/SimpleStateMachineStorage.java
----------------------------------------------------------------------
diff --git a/raft-server/src/main/java/org/apache/raft/statemachine/SimpleStateMachineStorage.java b/raft-server/src/main/java/org/apache/raft/statemachine/SimpleStateMachineStorage.java
deleted file mode 100644
index a779f98..0000000
--- a/raft-server/src/main/java/org/apache/raft/statemachine/SimpleStateMachineStorage.java
+++ /dev/null
@@ -1,134 +0,0 @@
-/**
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- *     http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-package org.apache.raft.statemachine;
-
-
-import com.google.common.annotations.VisibleForTesting;
-import org.apache.raft.io.MD5Hash;
-import org.apache.raft.server.protocol.TermIndex;
-import org.apache.raft.server.storage.FileInfo;
-import org.apache.raft.server.storage.RaftStorage;
-import org.apache.raft.util.AtomicFileOutputStream;
-import org.apache.raft.util.MD5FileUtil;
-import org.slf4j.Logger;
-import org.slf4j.LoggerFactory;
-
-import java.io.File;
-import java.io.IOException;
-import java.nio.file.DirectoryStream;
-import java.nio.file.Files;
-import java.nio.file.Path;
-import java.util.regex.Matcher;
-import java.util.regex.Pattern;
-
-/**
- * A StateMachineStorage that stores the snapshot in a single file.
- */
-public class SimpleStateMachineStorage implements StateMachineStorage {
-
-  private static final Logger LOG = LoggerFactory.getLogger(SimpleStateMachineStorage.class);
-
-  static final String SNAPSHOT_FILE_PREFIX = "snapshot";
-  static final String CORRUPT_SNAPSHOT_FILE_SUFFIX = ".corrupt";
-  /** snapshot.term_index */
-  static final Pattern SNAPSHOT_REGEX =
-      Pattern.compile(SNAPSHOT_FILE_PREFIX + "\\.(\\d+)_(\\d+)");
-
-  private RaftStorage raftStorage;
-  private File smDir = null;
-
-  private volatile SingleFileSnapshotInfo currentSnapshot = null;
-
-  public void init(RaftStorage raftStorage) throws IOException {
-    this.raftStorage = raftStorage;
-    this.smDir = raftStorage.getStorageDir().getStateMachineDir();
-    loadLatestSnapshot();
-  }
-
-  @Override
-  public void format() throws IOException {
-    // TODO
-  }
-
-  @VisibleForTesting
-  public static TermIndex getTermIndexFromSnapshotFile(File file) {
-    final String name = file.getName();
-    final Matcher m = SNAPSHOT_REGEX.matcher(name);
-    if (!m.matches()) {
-      throw new IllegalArgumentException("File \"" + file
-          + "\" does not match snapshot file name pattern \""
-          + SNAPSHOT_REGEX + "\"");
-    }
-    final long term = Long.parseLong(m.group(1));
-    final long index = Long.parseLong(m.group(2));
-    return TermIndex.newTermIndex(term, index);
-  }
-
-  protected static String getTmpSnapshotFileName(long term, long endIndex) {
-    return getSnapshotFileName(term, endIndex) + AtomicFileOutputStream.TMP_EXTENSION;
-  }
-
-  protected static String getCorruptSnapshotFileName(long term, long endIndex) {
-    return getSnapshotFileName(term, endIndex) + CORRUPT_SNAPSHOT_FILE_SUFFIX;
-  }
-
-  public File getSnapshotFile(long term, long endIndex) {
-    return new File(smDir, getSnapshotFileName(term, endIndex));
-  }
-
-  protected File getTmpSnapshotFile(long term, long endIndex) {
-    return new File(smDir, getTmpSnapshotFileName(term, endIndex));
-  }
-
-  protected File getCorruptSnapshotFile(long term, long endIndex) {
-    return new File(smDir, getCorruptSnapshotFileName(term, endIndex));
-  }
-
-  public SingleFileSnapshotInfo findLatestSnapshot() throws IOException {
-    SingleFileSnapshotInfo latest = null;
-    try (DirectoryStream<Path> stream =
-             Files.newDirectoryStream(smDir.toPath())) {
-      for (Path path : stream) {
-        Matcher matcher = SNAPSHOT_REGEX.matcher(path.getFileName().toString());
-        if (matcher.matches()) {
-          final long endIndex = Long.parseLong(matcher.group(2));
-          if (latest == null || endIndex > latest.getIndex()) {
-            final long term = Long.parseLong(matcher.group(1));
-            MD5Hash fileDigest = MD5FileUtil.readStoredMd5ForFile(path.toFile());
-            final FileInfo fileInfo = new FileInfo(path, fileDigest);
-            latest = new SingleFileSnapshotInfo(fileInfo, term, endIndex);
-          }
-        }
-      }
-    }
-    return latest;
-  }
-
-  public void loadLatestSnapshot() throws IOException {
-    this.currentSnapshot = findLatestSnapshot();
-  }
-
-  public static String getSnapshotFileName(long term, long endIndex) {
-    return SNAPSHOT_FILE_PREFIX + "." + term + "_" + endIndex;
-  }
-
-  @Override
-  public SingleFileSnapshotInfo getLatestSnapshot() {
-    return currentSnapshot;
-  }
-}

http://git-wip-us.apache.org/repos/asf/incubator-ratis/blob/7e71a2e0/raft-server/src/main/java/org/apache/raft/statemachine/SingleFileSnapshotInfo.java
----------------------------------------------------------------------
diff --git a/raft-server/src/main/java/org/apache/raft/statemachine/SingleFileSnapshotInfo.java b/raft-server/src/main/java/org/apache/raft/statemachine/SingleFileSnapshotInfo.java
deleted file mode 100644
index 6b01e17..0000000
--- a/raft-server/src/main/java/org/apache/raft/statemachine/SingleFileSnapshotInfo.java
+++ /dev/null
@@ -1,38 +0,0 @@
-/**
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- *     http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-package org.apache.raft.statemachine;
-
-import org.apache.raft.server.storage.FileInfo;
-
-import java.util.Arrays;
-
-/**
- * Each snapshot only has a single file.
- *
- * The objects of this class are immutable.
- */
-public class SingleFileSnapshotInfo extends FileListSnapshotInfo {
-  public SingleFileSnapshotInfo(FileInfo fileInfo, long term, long endIndex) {
-    super(Arrays.asList(fileInfo), term, endIndex);
-  }
-
-  /** @return the file associated with the snapshot. */
-  public FileInfo getFile() {
-    return getFiles().get(0);
-  }
-}

http://git-wip-us.apache.org/repos/asf/incubator-ratis/blob/7e71a2e0/raft-server/src/main/java/org/apache/raft/statemachine/SnapshotInfo.java
----------------------------------------------------------------------
diff --git a/raft-server/src/main/java/org/apache/raft/statemachine/SnapshotInfo.java b/raft-server/src/main/java/org/apache/raft/statemachine/SnapshotInfo.java
deleted file mode 100644
index 0fdcbc3..0000000
--- a/raft-server/src/main/java/org/apache/raft/statemachine/SnapshotInfo.java
+++ /dev/null
@@ -1,58 +0,0 @@
-/**
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- *     http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-package org.apache.raft.statemachine;
-
-import org.apache.raft.server.protocol.TermIndex;
-import org.apache.raft.server.storage.FileInfo;
-
-import java.util.List;
-
-/**
- * SnapshotInfo represents a durable state by the state machine. The state machine implementation is
- * responsible for the layout of the snapshot files as well as making the data durable. Latest term,
- * latest index, and the raft configuration must be saved together with any data files in the
- * snapshot.
- */
-public interface SnapshotInfo {
-
-  /**
-   * Returns the term and index corresponding to this snapshot.
-   * @return The term and index corresponding to this snapshot.
-   */
-  TermIndex getTermIndex();
-
-  /**
-   * Returns the term corresponding to this snapshot.
-   * @return The term corresponding to this snapshot.
-   */
-  long getTerm();
-
-  /**
-   * Returns the index corresponding to this snapshot.
-   * @return The index corresponding to this snapshot.
-   */
-  long getIndex();
-
-  /**
-   * Returns a list of files corresponding to this snapshot. This list should include all
-   * the files that the state machine keeps in its data directory. This list of files will be
-   * copied as to other replicas in install snapshot RPCs.
-   * @return a list of Files corresponding to the this snapshot.
-   */
-  List<FileInfo> getFiles();
-}

http://git-wip-us.apache.org/repos/asf/incubator-ratis/blob/7e71a2e0/raft-server/src/main/java/org/apache/raft/statemachine/StateMachine.java
----------------------------------------------------------------------
diff --git a/raft-server/src/main/java/org/apache/raft/statemachine/StateMachine.java b/raft-server/src/main/java/org/apache/raft/statemachine/StateMachine.java
deleted file mode 100644
index 3dedf88..0000000
--- a/raft-server/src/main/java/org/apache/raft/statemachine/StateMachine.java
+++ /dev/null
@@ -1,168 +0,0 @@
-/**
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- *     http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-package org.apache.raft.statemachine;
-
-import org.apache.raft.conf.RaftProperties;
-import org.apache.raft.protocol.Message;
-import org.apache.raft.protocol.RaftClientReply;
-import org.apache.raft.protocol.RaftClientRequest;
-import org.apache.raft.server.RaftServerConfigKeys;
-import org.apache.raft.server.impl.RaftConfiguration;
-import org.apache.raft.server.storage.RaftStorage;
-import org.apache.raft.util.LifeCycle;
-
-import java.io.Closeable;
-import java.io.IOException;
-import java.util.Collection;
-import java.util.concurrent.CompletableFuture;
-
-/**
- * StateMachine is the entry point for the custom implementation of replicated state as defined in
- * the "State Machine Approach" in the literature
- * (see https://en.wikipedia.org/wiki/State_machine_replication).
- */
-public interface StateMachine extends Closeable {
-  /**
-   * Initializes the State Machine with the given properties and storage. The state machine is
-   * responsible reading the latest snapshot from the file system (if any) and initialize itself
-   * with the latest term and index there including all the edits.
-   */
-  void initialize(String id, RaftProperties properties, RaftStorage storage)
-      throws IOException;
-
-  /**
-   * Returns the lifecycle state for this StateMachine.
-   * @return the lifecycle state.
-   */
-  LifeCycle.State getLifeCycleState();
-
-  /**
-   * Pauses the state machine. On return, the state machine should have closed all open files so
-   * that a new snapshot can be installed.
-   */
-  void pause();
-
-  /**
-   * Re-initializes the State Machine in PAUSED state with the given properties and storage. The
-   * state machine is responsible reading the latest snapshot from the file system (if any) and
-   * initialize itself with the latest term and index there including all the edits.
-   */
-  void reinitialize(String id, RaftProperties properties, RaftStorage storage)
-      throws IOException;
-
-  /**
-   * Dump the in-memory state into a snapshot file in the RaftStorage. The
-   * StateMachine implementation can decide 1) its own snapshot format, 2) when
-   * a snapshot is taken, and 3) how the snapshot is taken (e.g., whether the
-   * snapshot blocks the state machine, and whether to purge log entries after
-   * a snapshot is done).
-   *
-   * In the meanwhile, when the size of raft log outside of the latest snapshot
-   * exceeds certain threshold, the RaftServer may choose to trigger a snapshot
-   * if {@link RaftServerConfigKeys#RAFT_SERVER_AUTO_SNAPSHOT_ENABLED_KEY} is
-   * enabled.
-   *
-   * The snapshot should include the latest raft configuration.
-   *
-   * @return the largest index of the log entry that has been applied to the
-   *         state machine and also included in the snapshot. Note the log purge
-   *         should be handled separately.
-   */
-  // TODO: refactor this
-  long takeSnapshot() throws IOException;
-
-  /**
-   * Record the RaftConfiguration in the state machine. The RaftConfiguration
-   * should also be stored in the snapshot.
-   */
-  void setRaftConfiguration(RaftConfiguration conf);
-
-  /**
-   * @return the latest raft configuration recorded in the state machine.
-   */
-  RaftConfiguration getRaftConfiguration();
-
-  /**
-   * @return StateMachineStorage to interact with the durability guarantees provided by the
-   * state machine.
-   */
-  StateMachineStorage getStateMachineStorage();
-
-  /**
-   * Returns the information for the latest durable snapshot.
-   */
-  SnapshotInfo getLatestSnapshot();
-
-  /**
-   * Query the state machine. The request must be read-only.
-   * TODO: extend RaftClientRequest to have a read-only request subclass.
-   */
-  CompletableFuture<RaftClientReply> query(RaftClientRequest request);
-
-  /**
-   * Validate/pre-process the incoming update request in the state machine.
-   * @return the content to be written to the log entry. Null means the request
-   * should be rejected.
-   * @throws IOException thrown by the state machine while validation
-   */
-  TransactionContext startTransaction(RaftClientRequest request)
-      throws IOException;
-
-  /**
-   * This is called before the transaction passed from the StateMachine is appended to the raft log.
-   * This method will be called from log append and having the same strict serial order that the
-   * transactions will have in the RAFT log. Since this is called serially in the critical path of
-   * log append, it is important to do only required operations here.
-   * @return The Transaction context.
-   */
-  TransactionContext preAppendTransaction(TransactionContext trx) throws IOException;
-
-  /**
-   * Called to notify the state machine that the Transaction passed cannot be appended (or synced).
-   * The exception field will indicate whether there was an exception or not.
-   * @param trx the transaction to cancel
-   * @return cancelled transaction
-   */
-  TransactionContext cancelTransaction(TransactionContext trx) throws IOException;
-
-  /**
-   * Called for transactions that have been committed to the RAFT log. This step is called
-   * sequentially in strict serial order that the transactions have been committed in the log.
-   * The SM is expected to do only necessary work, and leave the actual apply operation to the
-   * applyTransaction calls that can happen concurrently.
-   * @param trx the transaction state including the log entry that has been committed to a quorum
-   *            of the raft peers
-   * @return The Transaction context.
-   */
-  TransactionContext applyTransactionSerial(TransactionContext trx) throws IOException;
-
-  /**
-   * Apply a committed log entry to the state machine. This method can be called concurrently with
-   * the other calls, and there is no guarantee that the calls will be ordered according to the
-   * log commit order.
-   * @param trx the transaction state including the log entry that has been committed to a quorum
-   *            of the raft peers
-   */
-  // TODO: We do not need to return CompletableFuture
-  CompletableFuture<Message> applyTransaction(TransactionContext trx) throws IOException;
-
-  /**
-   * Notify the state machine that the raft peer is no longer leader.
-   */
-  void notifyNotLeader(Collection<TransactionContext> pendingEntries) throws IOException;
-}

http://git-wip-us.apache.org/repos/asf/incubator-ratis/blob/7e71a2e0/raft-server/src/main/java/org/apache/raft/statemachine/StateMachineStorage.java
----------------------------------------------------------------------
diff --git a/raft-server/src/main/java/org/apache/raft/statemachine/StateMachineStorage.java b/raft-server/src/main/java/org/apache/raft/statemachine/StateMachineStorage.java
deleted file mode 100644
index 30005f9..0000000
--- a/raft-server/src/main/java/org/apache/raft/statemachine/StateMachineStorage.java
+++ /dev/null
@@ -1,39 +0,0 @@
-/**
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- *     http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-package org.apache.raft.statemachine;
-
-import org.apache.raft.server.storage.RaftStorage;
-import java.io.IOException;
-
-public interface StateMachineStorage {
-
-  void init(RaftStorage raftStorage) throws IOException;
-
-  /**
-   * Returns the information for the latest durable snapshot.
-   */
-  SnapshotInfo getLatestSnapshot();
-
-  // TODO: StateMachine can decide to compact the files independently of concurrent install snapshot
-  // etc requests. We should have ref counting for the SnapshotInfo with a release mechanism
-  // so that raft server will release the files after the snapshot file copy in case a compaction
-  // is waiting for deleting these files.
-
-  void format() throws IOException;
-
-}

http://git-wip-us.apache.org/repos/asf/incubator-ratis/blob/7e71a2e0/raft-server/src/main/java/org/apache/raft/statemachine/TransactionContext.java
----------------------------------------------------------------------
diff --git a/raft-server/src/main/java/org/apache/raft/statemachine/TransactionContext.java b/raft-server/src/main/java/org/apache/raft/statemachine/TransactionContext.java
deleted file mode 100644
index 675ada9..0000000
--- a/raft-server/src/main/java/org/apache/raft/statemachine/TransactionContext.java
+++ /dev/null
@@ -1,210 +0,0 @@
-/**
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- *     http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-package org.apache.raft.statemachine;
-
-import org.apache.raft.protocol.RaftClientRequest;
-import org.apache.raft.shaded.proto.RaftProtos.LogEntryProto;
-import org.apache.raft.shaded.proto.RaftProtos.SMLogEntryProto;
-
-import java.io.IOException;
-import java.util.Collection;
-import java.util.Optional;
-
-/**
- * Context for a transaction.
- * The transaction might have originated from a client request, or it
- * maybe coming from another replica of the state machine through the RAFT log.
- * {@link TransactionContext} can be created from
- * either the {@link StateMachine} or the state machine updater.
- *
- * In the first case, the {@link StateMachine} is a leader. When it receives
- * a {@link StateMachine#startTransaction(RaftClientRequest)} request, it returns
- * a {@link TransactionContext} with the changes from the {@link StateMachine}.
- * The same context will be passed back to the {@link StateMachine}
- * via the {@link StateMachine#applyTransaction(TransactionContext)} call
- * or the {@link StateMachine#notifyNotLeader(Collection)} call.
- *
- * In the second case, the {@link StateMachine} is a follower.
- * The {@link TransactionContext} will be a committed entry coming from
- * the RAFT log from the leader.
- */
-public class TransactionContext {
-
-  /** The {@link StateMachine} that originated the transaction. */
-  private final StateMachine stateMachine;
-
-  /** Original request from the client */
-  private Optional<RaftClientRequest> clientRequest = Optional.empty();
-
-  /** Exception from the {@link StateMachine} or from the log */
-  private Optional<Exception> exception = Optional.empty();
-
-  /** Data from the {@link StateMachine} */
-  private Optional<SMLogEntryProto> smLogEntryProto = Optional.empty();
-
-  /**
-   * Context specific to the state machine.
-   * The {@link StateMachine} can use this object to carry state between
-   * {@link StateMachine#startTransaction(RaftClientRequest)} and
-   * {@link StateMachine#applyTransaction(TransactionContext)}.
-   */
-  private Optional<Object> stateMachineContext = Optional.empty();
-
-  /**
-   * Whether to commit the transaction to the RAFT Log.
-   * In some cases the {@link StateMachine} may want to indicate
-   * that the transaction should not be committed
-   */
-  private boolean shouldCommit = true;
-
-  /** Committed LogEntry. */
-  private Optional<LogEntryProto> logEntry = Optional.empty();
-
-  private TransactionContext(StateMachine stateMachine) {
-    this.stateMachine = stateMachine;
-  }
-
-  /** The same as this(stateMachine, clientRequest, smLogEntryProto, null). */
-  public TransactionContext(
-      StateMachine stateMachine, RaftClientRequest clientRequest,
-      SMLogEntryProto smLogEntryProto) {
-    this(stateMachine, clientRequest, smLogEntryProto, null);
-  }
-
-  /**
-   * Construct a {@link TransactionContext} from a client request.
-   * Used by the state machine to start a transaction
-   * and send the Log entry representing the transaction data
-   * to be applied to the raft log.
-   */
-  public TransactionContext(
-      StateMachine stateMachine, RaftClientRequest clientRequest,
-      SMLogEntryProto smLogEntryProto, Object stateMachineContext) {
-    this(stateMachine);
-    this.clientRequest = Optional.of(clientRequest);
-    this.smLogEntryProto = Optional.ofNullable(smLogEntryProto);
-    this.stateMachineContext = Optional.ofNullable(stateMachineContext);
-  }
-
-  /** The same as this(stateMachine, clientRequest, exception, null). */
-  public TransactionContext(
-      StateMachine stateMachine, RaftClientRequest clientRequest,
-      Exception exception) {
-    this(stateMachine, clientRequest, exception, null);
-  }
-
-  /**
-   * Construct a {@link TransactionContext} from a client request to signal
-   * an exception so that the RAFT server will fail the request on behalf
-   * of the {@link StateMachine}.
-   */
-  public TransactionContext(
-      StateMachine stateMachine, RaftClientRequest clientRequest,
-      Exception exception, Object stateMachineContext) {
-    this(stateMachine);
-    this.clientRequest = Optional.of(clientRequest);
-    this.exception = Optional.of(exception);
-    this.stateMachineContext = Optional.ofNullable(stateMachineContext);
-  }
-
-  /**
-   * Construct a {@link TransactionContext} from a {@link LogEntryProto}.
-   * Used by followers for applying committed entries to the state machine.
-   * @param logEntry the log entry to be applied
-   */
-  public TransactionContext(StateMachine stateMachine, LogEntryProto logEntry) {
-    this(stateMachine);
-    this.smLogEntryProto = Optional.of(logEntry.getSmLogEntry());
-    this.logEntry = Optional.of(logEntry);
-  }
-
-  public Optional<RaftClientRequest> getClientRequest() {
-    return this.clientRequest;
-  }
-
-  public Optional<SMLogEntryProto> getSMLogEntry() {
-    return this.smLogEntryProto;
-  }
-
-  public Optional<Exception> getException() {
-    return this.exception;
-  }
-
-  public TransactionContext setStateMachineContext(Object stateMachineContext) {
-    this.stateMachineContext = Optional.ofNullable(stateMachineContext);
-    return this;
-  }
-
-  public Optional<Object> getStateMachineContext() {
-    return stateMachineContext;
-  }
-
-  public TransactionContext setLogEntry(LogEntryProto logEntry) {
-    this.logEntry = Optional.of(logEntry);
-    return this;
-  }
-
-  public TransactionContext setSmLogEntryProto(SMLogEntryProto smLogEntryProto) {
-    this.smLogEntryProto = Optional.of(smLogEntryProto);
-    return this;
-  }
-
-  public Optional<LogEntryProto> getLogEntry() {
-    return logEntry;
-  }
-
-  private TransactionContext setException(IOException ioe) {
-    assert !this.exception.isPresent();
-    this.exception = Optional.of(ioe);
-    return this;
-  }
-
-  public TransactionContext setShouldCommit(boolean shouldCommit) {
-    this.shouldCommit = shouldCommit;
-    return this;
-  }
-
-  public boolean shouldCommit() {
-    // TODO: Hook this up in the server to bypass the RAFT Log and send back a response to client
-    return this.shouldCommit;
-  }
-
-  // proxy StateMachine methods. We do not want to expose the SM to the RaftLog
-
-  /**
-   * This is called before the transaction passed from the StateMachine is appended to the raft log.
-   * This method will be called from log append and having the same strict serial order that the
-   * Transactions will have in the RAFT log. Since this is called serially in the critical path of
-   * log append, it is important to do only required operations here.
-   * @return The Transaction context.
-   */
-  public TransactionContext preAppendTransaction() throws IOException {
-    return stateMachine.preAppendTransaction(this);
-  }
-
-  /**
-   * Called to notify the state machine that the Transaction passed cannot be appended (or synced).
-   * The exception field will indicate whether there was an exception or not.
-   * @return cancelled transaction
-   */
-  public TransactionContext cancelTransaction() throws IOException {
-    // TODO: This is not called from Raft server / log yet. When an IOException happens, we should
-    // call this to let the SM know that Transaction cannot be synced
-    return stateMachine.cancelTransaction(this);
-  }
-}

http://git-wip-us.apache.org/repos/asf/incubator-ratis/blob/7e71a2e0/raft-server/src/test/java/org/apache/raft/MiniRaftCluster.java
----------------------------------------------------------------------
diff --git a/raft-server/src/test/java/org/apache/raft/MiniRaftCluster.java b/raft-server/src/test/java/org/apache/raft/MiniRaftCluster.java
deleted file mode 100644
index c66ef8f..0000000
--- a/raft-server/src/test/java/org/apache/raft/MiniRaftCluster.java
+++ /dev/null
@@ -1,444 +0,0 @@
-/**
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- *     http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-package org.apache.raft;
-
-import com.google.common.base.Preconditions;
-import org.apache.raft.client.RaftClient;
-import org.apache.raft.client.RaftClientRequestSender;
-import org.apache.raft.client.impl.RaftClientImpl;
-import org.apache.raft.conf.RaftProperties;
-import org.apache.raft.protocol.RaftPeer;
-import org.apache.raft.server.RaftServerConfigKeys;
-import org.apache.raft.server.impl.DelayLocalExecutionInjection;
-import org.apache.raft.server.impl.RaftConfiguration;
-import org.apache.raft.server.impl.RaftServerImpl;
-import org.apache.raft.server.RaftServerRpc;
-import org.apache.raft.server.storage.MemoryRaftLog;
-import org.apache.raft.server.storage.RaftLog;
-import org.apache.raft.statemachine.BaseStateMachine;
-import org.apache.raft.statemachine.StateMachine;
-import org.apache.raft.util.ExitUtils;
-import org.apache.raft.util.FileUtils;
-import org.apache.raft.util.RaftUtils;
-import org.junit.Assert;
-import org.slf4j.Logger;
-import org.slf4j.LoggerFactory;
-
-import java.io.File;
-import java.io.IOException;
-import java.util.*;
-import java.util.stream.Collectors;
-
-import static org.apache.raft.server.RaftServerConfigKeys.RAFT_SERVER_RPC_TIMEOUT_MIN_MS_DEFAULT;
-
-public abstract class MiniRaftCluster {
-  public static final Logger LOG = LoggerFactory.getLogger(MiniRaftCluster.class);
-  public static final DelayLocalExecutionInjection logSyncDelay =
-      new DelayLocalExecutionInjection(RaftLog.LOG_SYNC);
-
-  public static final String CLASS_NAME = MiniRaftCluster.class.getSimpleName();
-  public static final String STATEMACHINE_CLASS_KEY = CLASS_NAME + ".statemachine.class";
-  public static final Class<? extends StateMachine> STATEMACHINE_CLASS_DEFAULT = BaseStateMachine.class;
-
-  public static abstract class Factory<CLUSTER extends MiniRaftCluster> {
-    public abstract CLUSTER newCluster(
-        String[] ids, RaftProperties prop, boolean formatted)
-        throws IOException;
-
-    public CLUSTER newCluster(
-        int numServer, RaftProperties prop, boolean formatted)
-        throws IOException {
-      return newCluster(generateIds(numServer, 0), prop, formatted);
-    }
-  }
-
-  public static abstract class RpcBase extends MiniRaftCluster {
-    public RpcBase(String[] ids, RaftProperties properties, boolean formatted) {
-      super(ids, properties, formatted);
-    }
-
-    protected abstract RaftServerImpl setPeerRpc(RaftPeer peer) throws IOException;
-
-    @Override
-    protected void setPeerRpc() throws IOException {
-      for (RaftPeer p : conf.getPeers()) {
-        setPeerRpc(p);
-      }
-    }
-
-    @Override
-    public void restartServer(String id, boolean format) throws IOException {
-      super.restartServer(id, format);
-      setPeerRpc(conf.getPeer(id)).start();
-    }
-
-    @Override
-    public void setBlockRequestsFrom(String src, boolean block) {
-      RaftTestUtil.setBlockRequestsFrom(src, block);
-    }
-  }
-
-  public static class PeerChanges {
-    public final RaftPeer[] allPeersInNewConf;
-    public final RaftPeer[] newPeers;
-    public final RaftPeer[] removedPeers;
-
-    public PeerChanges(RaftPeer[] all, RaftPeer[] newPeers, RaftPeer[] removed) {
-      this.allPeersInNewConf = all;
-      this.newPeers = newPeers;
-      this.removedPeers = removed;
-    }
-  }
-
-  public static RaftConfiguration initConfiguration(String[] ids) {
-    return RaftConfiguration.newBuilder()
-        .setConf(Arrays.stream(ids).map(RaftPeer::new).collect(Collectors.toList()))
-        .build();
-  }
-
-  private static String getBaseDirectory() {
-    return System.getProperty("test.build.data", "target/test/data") + "/raft/";
-  }
-
-  private static void formatDir(String dirStr) {
-    final File serverDir = new File(dirStr);
-    Preconditions.checkState(FileUtils.fullyDelete(serverDir),
-        "Failed to format directory %s", dirStr);
-    LOG.info("Formatted directory {}", dirStr);
-  }
-
-  public static String[] generateIds(int numServers, int base) {
-    String[] ids = new String[numServers];
-    for (int i = 0; i < numServers; i++) {
-      ids[i] = "s" + (i + base);
-    }
-    return ids;
-  }
-
-  protected RaftConfiguration conf;
-  protected final RaftProperties properties;
-  private final String testBaseDir;
-  protected final Map<String, RaftServerImpl> servers =
-      Collections.synchronizedMap(new LinkedHashMap<>());
-
-  public MiniRaftCluster(String[] ids, RaftProperties properties,
-      boolean formatted) {
-    this.conf = initConfiguration(ids);
-    this.properties = new RaftProperties(properties);
-    this.testBaseDir = getBaseDirectory();
-
-    conf.getPeers().forEach(
-        p -> servers.put(p.getId(), newRaftServer(p.getId(), conf, formatted)));
-
-    ExitUtils.disableSystemExit();
-  }
-
-  protected <RPC extends  RaftServerRpc> void init(Map<RaftPeer, RPC> peers) {
-    LOG.info("peers = " + peers.keySet());
-    conf = RaftConfiguration.newBuilder().setConf(peers.keySet()).build();
-    for (Map.Entry<RaftPeer, RPC> entry : peers.entrySet()) {
-      final RaftServerImpl server = servers.get(entry.getKey().getId());
-      server.setInitialConf(conf);
-      server.setServerRpc(entry.getValue());
-    }
-  }
-
-  public void start() {
-    LOG.info("Starting " + getClass().getSimpleName());
-    servers.values().forEach(RaftServerImpl::start);
-  }
-
-  /**
-   * start a stopped server again.
-   */
-  public void restartServer(String id, boolean format) throws IOException {
-    killServer(id);
-    servers.remove(id);
-    servers.put(id, newRaftServer(id, conf, format));
-  }
-
-  public final void restart(boolean format) throws IOException {
-    servers.values().stream().filter(RaftServerImpl::isAlive)
-        .forEach(RaftServerImpl::close);
-    List<String> idList = new ArrayList<>(servers.keySet());
-    for (String id : idList) {
-      servers.remove(id);
-      servers.put(id, newRaftServer(id, conf, format));
-    }
-
-    setPeerRpc();
-    start();
-  }
-
-  protected abstract void setPeerRpc() throws IOException;
-
-  public int getMaxTimeout() {
-    return properties.getInt(
-        RaftServerConfigKeys.RAFT_SERVER_RPC_TIMEOUT_MAX_MS_KEY,
-        RaftServerConfigKeys.RAFT_SERVER_RPC_TIMEOUT_MAX_MS_DEFAULT);
-  }
-
-  public RaftConfiguration getConf() {
-    return conf;
-  }
-
-  private RaftServerImpl newRaftServer(String id, RaftConfiguration conf,
-                                       boolean format) {
-    final RaftServerImpl s;
-    try {
-      final String dirStr = testBaseDir + id;
-      if (format) {
-        formatDir(dirStr);
-      }
-      properties.set(RaftServerConfigKeys.RAFT_SERVER_STORAGE_DIR_KEY, dirStr);
-      s = new RaftServerImpl(id, conf, properties, getStateMachine4Test(properties));
-    } catch (IOException e) {
-      throw new RuntimeException(e);
-    }
-    return s;
-  }
-
-  static StateMachine getStateMachine4Test(RaftProperties properties) {
-    final Class<? extends StateMachine> smClass = properties.getClass(
-        STATEMACHINE_CLASS_KEY,
-        STATEMACHINE_CLASS_DEFAULT,
-        StateMachine.class);
-    return RaftUtils.newInstance(smClass);
-  }
-
-  public abstract RaftClientRequestSender getRaftClientRequestSender();
-
-  protected <RPC extends RaftServerRpc> Collection<RaftPeer> addNewPeers(
-      Map<RaftPeer, RPC> newPeers, Collection<RaftServerImpl> newServers,
-      boolean startService) throws IOException {
-    for (Map.Entry<RaftPeer, RPC> entry : newPeers.entrySet()) {
-      RaftServerImpl server = servers.get(entry.getKey().getId());
-      server.setServerRpc(entry.getValue());
-    }
-    if (startService) {
-      newServers.forEach(RaftServerImpl::start);
-    }
-    return new ArrayList<>(newPeers.keySet());
-  }
-
-  protected abstract Collection<RaftPeer> addNewPeers(
-      Collection<RaftPeer> newPeers, Collection<RaftServerImpl> newServers,
-      boolean startService) throws IOException;
-
-  public PeerChanges addNewPeers(int number, boolean startNewPeer)
-      throws IOException {
-    return addNewPeers(generateIds(number, servers.size()), startNewPeer);
-  }
-
-  public PeerChanges addNewPeers(String[] ids,
-      boolean startNewPeer) throws IOException {
-    LOG.info("Add new peers {}", Arrays.asList(ids));
-    Collection<RaftPeer> newPeers = new ArrayList<>(ids.length);
-    for (String id : ids) {
-      newPeers.add(new RaftPeer(id));
-    }
-
-    // create and add new RaftServers
-    final List<RaftServerImpl> newServers = new ArrayList<>(ids.length);
-    for (RaftPeer p : newPeers) {
-      RaftServerImpl newServer = newRaftServer(p.getId(), conf, true);
-      Preconditions.checkArgument(!servers.containsKey(p.getId()));
-      servers.put(p.getId(), newServer);
-      newServers.add(newServer);
-    }
-
-    // for hadoop-rpc-enabled peer, we assign inetsocketaddress here
-    newPeers = addNewPeers(newPeers, newServers, startNewPeer);
-
-    final RaftPeer[] np = newPeers.toArray(new RaftPeer[newPeers.size()]);
-    newPeers.addAll(conf.getPeers());
-    conf = RaftConfiguration.newBuilder().setConf(newPeers).setLogEntryIndex(0).build();
-    RaftPeer[] p = newPeers.toArray(new RaftPeer[newPeers.size()]);
-    return new PeerChanges(p, np, new RaftPeer[0]);
-  }
-
-  public void startServer(String id) {
-    RaftServerImpl server = servers.get(id);
-    assert server != null;
-    server.start();
-  }
-
-  private RaftPeer getPeer(RaftServerImpl s) {
-    return new RaftPeer(s.getId(), s.getServerRpc().getInetSocketAddress());
-  }
-
-  /**
-   * prepare the peer list when removing some peers from the conf
-   */
-  public PeerChanges removePeers(int number, boolean removeLeader,
-      Collection<RaftPeer> excluded) {
-    Collection<RaftPeer> peers = new ArrayList<>(conf.getPeers());
-    List<RaftPeer> removedPeers = new ArrayList<>(number);
-    if (removeLeader) {
-      final RaftPeer leader = getPeer(getLeader());
-      assert !excluded.contains(leader);
-      peers.remove(leader);
-      removedPeers.add(leader);
-    }
-    List<RaftServerImpl> followers = getFollowers();
-    for (int i = 0, removed = 0; i < followers.size() &&
-        removed < (removeLeader ? number - 1 : number); i++) {
-      RaftPeer toRemove = getPeer(followers.get(i));
-      if (!excluded.contains(toRemove)) {
-        peers.remove(toRemove);
-        removedPeers.add(toRemove);
-        removed++;
-      }
-    }
-    conf = RaftConfiguration.newBuilder().setConf(peers).setLogEntryIndex(0).build();
-    RaftPeer[] p = peers.toArray(new RaftPeer[peers.size()]);
-    return new PeerChanges(p, new RaftPeer[0],
-        removedPeers.toArray(new RaftPeer[removedPeers.size()]));
-  }
-
-  public void killServer(String id) {
-    servers.get(id).close();
-  }
-
-  public String printServers() {
-    StringBuilder b = new StringBuilder("\n#servers = " + servers.size() + "\n");
-    for (RaftServerImpl s : servers.values()) {
-      b.append("  ");
-      b.append(s).append("\n");
-    }
-    return b.toString();
-  }
-
-  public String printAllLogs() {
-    StringBuilder b = new StringBuilder("\n#servers = " + servers.size() + "\n");
-    for (RaftServerImpl s : servers.values()) {
-      b.append("  ");
-      b.append(s).append("\n");
-
-      final RaftLog log = s.getState().getLog();
-      if (log instanceof MemoryRaftLog) {
-        b.append("    ");
-        b.append(((MemoryRaftLog) log).getEntryString());
-      }
-    }
-    return b.toString();
-  }
-
-  public RaftServerImpl getLeader() {
-    final List<RaftServerImpl> leaders = new ArrayList<>();
-    servers.values().stream()
-        .filter(s -> s.isAlive() && s.isLeader())
-        .forEach(s -> {
-      if (leaders.isEmpty()) {
-        leaders.add(s);
-      } else {
-        final long leaderTerm = leaders.get(0).getState().getCurrentTerm();
-        final long term = s.getState().getCurrentTerm();
-        if (term >= leaderTerm) {
-          if (term > leaderTerm) {
-            leaders.clear();
-          }
-          leaders.add(s);
-        }
-      }
-    });
-    if (leaders.isEmpty()) {
-      return null;
-    } else if (leaders.size() != 1) {
-      Assert.fail(printServers() + leaders.toString()
-          + "leaders.size() = " + leaders.size() + " != 1");
-    }
-    return leaders.get(0);
-  }
-
-  public boolean isLeader(String leaderId) throws InterruptedException {
-    final RaftServerImpl leader = getLeader();
-    return leader != null && leader.getId().equals(leaderId);
-  }
-
-  public List<RaftServerImpl> getFollowers() {
-    return servers.values().stream()
-        .filter(s -> s.isAlive() && s.isFollower())
-        .collect(Collectors.toList());
-  }
-
-  public Collection<RaftServerImpl> getServers() {
-    return servers.values();
-  }
-
-  public RaftServerImpl getServer(String id) {
-    return servers.get(id);
-  }
-
-  public Collection<RaftPeer> getPeers() {
-    return getServers().stream().map(s ->
-        new RaftPeer(s.getId(), s.getServerRpc().getInetSocketAddress()))
-        .collect(Collectors.toList());
-  }
-
-  public RaftClient createClient(String clientId, String leaderId) {
-    return new RaftClientImpl(clientId, conf.getPeers(),
-        getRaftClientRequestSender(), leaderId, properties);
-  }
-
-  public void shutdown() {
-    LOG.info("Stopping " + getClass().getSimpleName());
-    servers.values().stream().filter(RaftServerImpl::isAlive)
-        .forEach(RaftServerImpl::close);
-
-    if (ExitUtils.isTerminated()) {
-      LOG.error("Test resulted in an unexpected exit",
-          ExitUtils.getFirstExitException());
-      throw new AssertionError("Test resulted in an unexpected exit");
-    }
-  }
-
-  /**
-   * Block all the incoming requests for the peer with leaderId. Also delay
-   * outgoing or incoming msg for all other peers.
-   */
-  protected abstract void blockQueueAndSetDelay(String leaderId, int delayMs)
-      throws InterruptedException;
-
-  /**
-   * Try to enforce the leader of the cluster.
-   * @param leaderId ID of the targeted leader server.
-   * @return true if server has been successfully enforced to the leader, false
-   *         otherwise.
-   */
-  public boolean tryEnforceLeader(String leaderId) throws InterruptedException {
-    // do nothing and see if the given id is already a leader.
-    if (isLeader(leaderId)) {
-      return true;
-    }
-
-    // Blocking all other server's RPC read process to make sure a read takes at
-    // least ELECTION_TIMEOUT_MIN. In this way when the target leader request a
-    // vote, all non-leader servers can grant the vote.
-    // Disable the target leader server RPC so that it can request a vote.
-    blockQueueAndSetDelay(leaderId, RAFT_SERVER_RPC_TIMEOUT_MIN_MS_DEFAULT);
-
-    // Reopen queues so that the vote can make progress.
-    blockQueueAndSetDelay(leaderId, 0);
-
-    return isLeader(leaderId);
-  }
-
-  /** Block/unblock the requests sent from the given source. */
-  public abstract void setBlockRequestsFrom(String src, boolean block);
-}

http://git-wip-us.apache.org/repos/asf/incubator-ratis/blob/7e71a2e0/raft-server/src/test/java/org/apache/raft/RaftBasicTests.java
----------------------------------------------------------------------
diff --git a/raft-server/src/test/java/org/apache/raft/RaftBasicTests.java b/raft-server/src/test/java/org/apache/raft/RaftBasicTests.java
deleted file mode 100644
index ed40bde..0000000
--- a/raft-server/src/test/java/org/apache/raft/RaftBasicTests.java
+++ /dev/null
@@ -1,199 +0,0 @@
-/**
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- *     http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-package org.apache.raft;
-
-import org.apache.raft.RaftTestUtil.SimpleMessage;
-import org.apache.raft.client.RaftClient;
-import org.apache.raft.conf.RaftProperties;
-import org.apache.raft.server.impl.RaftServerImpl;
-import org.junit.*;
-import org.junit.rules.Timeout;
-import org.slf4j.Logger;
-import org.slf4j.LoggerFactory;
-
-import java.io.IOException;
-import java.util.List;
-import java.util.concurrent.ThreadLocalRandom;
-import java.util.concurrent.atomic.AtomicInteger;
-import java.util.stream.Collectors;
-import java.util.stream.Stream;
-
-import static org.apache.raft.RaftTestUtil.waitAndKillLeader;
-import static org.apache.raft.RaftTestUtil.waitForLeader;
-
-public abstract class RaftBasicTests {
-  public static final Logger LOG = LoggerFactory.getLogger(RaftBasicTests.class);
-
-  public static final int NUM_SERVERS = 5;
-
-  protected static final RaftProperties properties = new RaftProperties();
-
-  public abstract MiniRaftCluster getCluster();
-
-  public RaftProperties getProperties() {
-    return properties;
-  }
-
-  @Rule
-  public Timeout globalTimeout = new Timeout(120 * 1000);
-
-  @Before
-  public void setup() throws IOException {
-    Assert.assertNull(getCluster().getLeader());
-    getCluster().start();
-  }
-
-  @After
-  public void tearDown() {
-    final MiniRaftCluster cluster = getCluster();
-    if (cluster != null) {
-      cluster.shutdown();
-    }
-  }
-
-  @Test
-  public void testBasicLeaderElection() throws Exception {
-    LOG.info("Running testBasicLeaderElection");
-    final MiniRaftCluster cluster = getCluster();
-    waitAndKillLeader(cluster, true);
-    waitAndKillLeader(cluster, true);
-    waitAndKillLeader(cluster, true);
-    waitAndKillLeader(cluster, false);
-  }
-
-  @Test
-  public void testBasicAppendEntries() throws Exception {
-    LOG.info("Running testBasicAppendEntries");
-    final MiniRaftCluster cluster = getCluster();
-    RaftServerImpl leader = waitForLeader(cluster);
-    final long term = leader.getState().getCurrentTerm();
-    final String killed = cluster.getFollowers().get(3).getId();
-    cluster.killServer(killed);
-    LOG.info(cluster.printServers());
-
-    final SimpleMessage[] messages = SimpleMessage.create(10);
-    try(final RaftClient client = cluster.createClient("client", null)) {
-      for (SimpleMessage message : messages) {
-        client.send(message);
-      }
-    }
-
-    Thread.sleep(cluster.getMaxTimeout() + 100);
-    LOG.info(cluster.printAllLogs());
-
-    cluster.getServers().stream().filter(RaftServerImpl::isAlive)
-        .map(s -> s.getState().getLog().getEntries(1, Long.MAX_VALUE))
-        .forEach(e -> RaftTestUtil.assertLogEntries(e, 1, term, messages));
-  }
-
-  @Test
-  public void testEnforceLeader() throws Exception {
-    LOG.info("Running testEnforceLeader");
-    final String leader = "s" + ThreadLocalRandom.current().nextInt(NUM_SERVERS);
-    LOG.info("enforce leader to " + leader);
-    final MiniRaftCluster cluster = getCluster();
-    waitForLeader(cluster);
-    waitForLeader(cluster, leader);
-  }
-
-  static class Client4TestWithLoad extends Thread {
-    final RaftClient client;
-    final SimpleMessage[] messages;
-
-    final AtomicInteger step = new AtomicInteger();
-    volatile Exception exceptionInClientThread;
-
-    Client4TestWithLoad(RaftClient client, int numMessages) {
-      this.client = client;
-      this.messages = SimpleMessage.create(numMessages, client.getId());
-    }
-
-    boolean isRunning() {
-      return step.get() < messages.length && exceptionInClientThread == null;
-    }
-
-    @Override
-    public void run() {
-      try {
-        for (; isRunning(); ) {
-          client.send(messages[step.getAndIncrement()]);
-        }
-        client.close();
-      } catch (IOException ioe) {
-        exceptionInClientThread = ioe;
-      }
-    }
-  }
-
-  @Test
-  public void testWithLoad() throws Exception {
-    testWithLoad(10, 500);
-  }
-
-  private void testWithLoad(final int numClients, final int numMessages)
-      throws Exception {
-    LOG.info("Running testWithLoad: numClients=" + numClients
-        + ", numMessages=" + numMessages);
-
-    final MiniRaftCluster cluster = getCluster();
-    LOG.info(cluster.printServers());
-
-    final List<Client4TestWithLoad> clients
-        = Stream.iterate(0, i -> i+1).limit(numClients)
-        .map(i -> cluster.createClient(String.valueOf((char)('a' + i)), null))
-        .map(c -> new Client4TestWithLoad(c, numMessages))
-        .collect(Collectors.toList());
-    clients.forEach(Thread::start);
-
-    int count = 0;
-    for(int lastStep = 0;; ) {
-      if (clients.stream().filter(Client4TestWithLoad::isRunning).count() == 0) {
-        break;
-      }
-
-      final int n = clients.stream().mapToInt(c -> c.step.get()).sum();
-      if (n - lastStep < 50 * numClients) { // Change leader at least 50 steps.
-        Thread.sleep(10);
-        continue;
-      }
-      lastStep = n;
-      count++;
-
-      RaftServerImpl leader = cluster.getLeader();
-      if (leader != null) {
-        final String oldLeader = leader.getId();
-        LOG.info("Block all requests sent by leader " + oldLeader);
-        String newLeader = RaftTestUtil.changeLeader(cluster, oldLeader);
-        LOG.info("Changed leader from " + oldLeader + " to " + newLeader);
-        Assert.assertFalse(newLeader.equals(oldLeader));
-      }
-    }
-
-    for(Client4TestWithLoad c : clients) {
-      c.join();
-    }
-    for(Client4TestWithLoad c : clients) {
-      if (c.exceptionInClientThread != null) {
-        throw new AssertionError(c.exceptionInClientThread);
-      }
-      RaftTestUtil.assertLogEntries(cluster.getServers(), c.messages);
-    }
-
-    LOG.info("Leader change count=" + count + cluster.printAllLogs());
-  }
-}

http://git-wip-us.apache.org/repos/asf/incubator-ratis/blob/7e71a2e0/raft-server/src/test/java/org/apache/raft/RaftNotLeaderExceptionBaseTest.java
----------------------------------------------------------------------
diff --git a/raft-server/src/test/java/org/apache/raft/RaftNotLeaderExceptionBaseTest.java b/raft-server/src/test/java/org/apache/raft/RaftNotLeaderExceptionBaseTest.java
deleted file mode 100644
index 195cbec..0000000
--- a/raft-server/src/test/java/org/apache/raft/RaftNotLeaderExceptionBaseTest.java
+++ /dev/null
@@ -1,162 +0,0 @@
-/**
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- *     http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-package org.apache.raft;
-
-import org.apache.log4j.Level;
-import org.apache.raft.RaftTestUtil.SimpleMessage;
-import org.apache.raft.client.RaftClient;
-import org.apache.raft.client.RaftClientRequestSender;
-import org.apache.raft.client.impl.RaftClientImpl;
-import org.apache.raft.protocol.RaftClientReply;
-import org.apache.raft.protocol.RaftClientRequest;
-import org.apache.raft.protocol.RaftPeer;
-import org.apache.raft.server.impl.RaftServerImpl;
-import org.apache.raft.server.simulation.RequestHandler;
-import org.apache.raft.server.storage.RaftLog;
-import org.apache.raft.util.RaftUtils;
-import org.junit.*;
-import org.junit.rules.Timeout;
-import org.slf4j.Logger;
-import org.slf4j.LoggerFactory;
-
-import java.io.IOException;
-import java.util.Arrays;
-import java.util.Collection;
-
-import static org.apache.raft.server.impl.RaftServerConstants.DEFAULT_SEQNUM;
-
-public abstract class RaftNotLeaderExceptionBaseTest {
-  static {
-    RaftUtils.setLogLevel(RaftServerImpl.LOG, Level.DEBUG);
-    RaftUtils.setLogLevel(RaftLog.LOG, Level.DEBUG);
-    RaftUtils.setLogLevel(RequestHandler.LOG, Level.DEBUG);
-    RaftUtils.setLogLevel(RaftClient.LOG, Level.DEBUG);
-  }
-
-  public static final Logger LOG =
-      LoggerFactory.getLogger(RaftNotLeaderExceptionBaseTest.class);
-  public static final int NUM_PEERS = 3;
-
-  @Rule
-  public Timeout globalTimeout = new Timeout(60 * 1000);
-
-  private MiniRaftCluster cluster;
-
-  public abstract MiniRaftCluster initCluster() throws IOException;
-
-  @Before
-  public void setup() throws IOException {
-    this.cluster = initCluster();
-    cluster.start();
-  }
-
-  @After
-  public void tearDown() {
-    if (cluster != null) {
-      cluster.shutdown();
-    }
-  }
-
-  @Test
-  public void testHandleNotLeaderException() throws Exception {
-    RaftTestUtil.waitForLeader(cluster);
-    final String leaderId = cluster.getLeader().getId();
-    final RaftClient client = cluster.createClient("client", leaderId);
-
-    RaftClientReply reply = client.send(new SimpleMessage("m1"));
-    Assert.assertTrue(reply.isSuccess());
-
-    // enforce leader change
-    String newLeader = RaftTestUtil.changeLeader(cluster, leaderId);
-    Assert.assertNotEquals(leaderId, newLeader);
-
-    RaftClientRequestSender rpc = ((RaftClientImpl)client).getRequestSender();
-    reply= null;
-    for (int i = 0; reply == null && i < 10; i++) {
-      try {
-        reply = rpc.sendRequest(
-            new RaftClientRequest("client", leaderId, DEFAULT_SEQNUM,
-                new SimpleMessage("m2")));
-      } catch (IOException ignored) {
-        Thread.sleep(1000);
-      }
-    }
-    Assert.assertNotNull(reply);
-    Assert.assertFalse(reply.isSuccess());
-    Assert.assertTrue(reply.isNotLeader());
-    Assert.assertEquals(newLeader,
-        reply.getNotLeaderException().getSuggestedLeader().getId());
-
-    reply = client.send(new SimpleMessage("m3"));
-    Assert.assertTrue(reply.isSuccess());
-    client.close();
-  }
-
-  @Test
-  public void testNotLeaderExceptionWithReconf() throws Exception {
-    Assert.assertNotNull(RaftTestUtil.waitForLeader(cluster));
-
-    final String leaderId = cluster.getLeader().getId();
-    final RaftClient client = cluster.createClient("client", leaderId);
-
-    // enforce leader change
-    String newLeader = RaftTestUtil.changeLeader(cluster, leaderId);
-    Assert.assertNotEquals(leaderId, newLeader);
-
-    // also add two new peers
-    // add two more peers
-    MiniRaftCluster.PeerChanges change = cluster.addNewPeers(
-        new String[]{"ss1", "ss2"}, true);
-    // trigger setConfiguration
-    LOG.info("Start changing the configuration: {}",
-        Arrays.asList(change.allPeersInNewConf));
-    try(final RaftClient c2 = cluster.createClient("client2", newLeader)) {
-      RaftClientReply reply = c2.setConfiguration(change.allPeersInNewConf);
-      Assert.assertTrue(reply.isSuccess());
-    }
-    LOG.info(cluster.printServers());
-
-    RaftClientRequestSender rpc = ((RaftClientImpl)client).getRequestSender();
-    RaftClientReply reply = null;
-    // it is possible that the remote peer's rpc server is not ready. need retry
-    for (int i = 0; reply == null && i < 10; i++) {
-      try {
-        reply = rpc.sendRequest(
-            new RaftClientRequest("client", leaderId, DEFAULT_SEQNUM,
-                new SimpleMessage("m1")));
-      } catch (IOException ignored) {
-        Thread.sleep(1000);
-      }
-    }
-    Assert.assertNotNull(reply);
-    Assert.assertFalse(reply.isSuccess());
-    Assert.assertTrue(reply.isNotLeader());
-    Assert.assertEquals(newLeader,
-        reply.getNotLeaderException().getSuggestedLeader().getId());
-    Collection<RaftPeer> peers = cluster.getPeers();
-    RaftPeer[] peersFromReply = reply.getNotLeaderException().getPeers();
-    Assert.assertEquals(peers.size(), peersFromReply.length);
-    for (RaftPeer p : peersFromReply) {
-      Assert.assertTrue(peers.contains(p));
-    }
-
-    reply = client.send(new SimpleMessage("m2"));
-    Assert.assertTrue(reply.isSuccess());
-    client.close();
-  }
-}


[30/54] [abbrv] incubator-ratis git commit: Renamed the packages from raft to ratis in preperation for Apache Incubation - Moved all java packages from org.apache.raft to org.apache.ratis. - Moved native package to org_apache_ratis, and native lib to l

Posted by ji...@apache.org.
http://git-wip-us.apache.org/repos/asf/incubator-ratis/blob/7e71a2e0/ratis-common/src/main/java/org/apache/ratis/io/nativeio/NativeIO.java
----------------------------------------------------------------------
diff --git a/ratis-common/src/main/java/org/apache/ratis/io/nativeio/NativeIO.java b/ratis-common/src/main/java/org/apache/ratis/io/nativeio/NativeIO.java
new file mode 100644
index 0000000..9d20682
--- /dev/null
+++ b/ratis-common/src/main/java/org/apache/ratis/io/nativeio/NativeIO.java
@@ -0,0 +1,805 @@
+/**
+ * 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.ratis.io.nativeio;
+
+import com.google.common.annotations.VisibleForTesting;
+
+import org.apache.ratis.protocol.AlreadyExistsException;
+import org.apache.ratis.util.NativeCodeLoader;
+import org.apache.ratis.util.RaftUtils;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+import sun.misc.Unsafe;
+
+import java.io.File;
+import java.io.FileDescriptor;
+import java.io.FileInputStream;
+import java.io.FileOutputStream;
+import java.io.IOException;
+import java.io.RandomAccessFile;
+import java.lang.reflect.Field;
+import java.nio.ByteBuffer;
+import java.nio.MappedByteBuffer;
+import java.nio.channels.FileChannel;
+
+/**
+ * JNI wrappers for various native IO-related calls not available in Java.
+ * These functions should generally be used alongside a fallback to another
+ * more portable mechanism.
+ */
+public class NativeIO {
+  private static final Logger LOG = LoggerFactory.getLogger(NativeIO.class);
+
+  public static class POSIX {
+    // Flags for open() call from bits/fcntl.h - Set by JNI
+    public static int O_RDONLY = -1;
+    public static int O_WRONLY = -1;
+    public static int O_RDWR = -1;
+    public static int O_CREAT = -1;
+    public static int O_EXCL = -1;
+    public static int O_NOCTTY = -1;
+    public static int O_TRUNC = -1;
+    public static int O_APPEND = -1;
+    public static int O_NONBLOCK = -1;
+    public static int O_SYNC = -1;
+
+    // Flags for posix_fadvise() from bits/fcntl.h - Set by JNI
+    /* No further special treatment.  */
+    public static int POSIX_FADV_NORMAL = -1;
+    /* Expect random page references.  */
+    public static int POSIX_FADV_RANDOM = -1;
+    /* Expect sequential page references.  */
+    public static int POSIX_FADV_SEQUENTIAL = -1;
+    /* Will need these pages.  */
+    public static int POSIX_FADV_WILLNEED = -1;
+    /* Don't need these pages.  */
+    public static int POSIX_FADV_DONTNEED = -1;
+    /* Data will be accessed once.  */
+    public static int POSIX_FADV_NOREUSE = -1;
+
+
+    // Updated by JNI when supported by glibc.  Leave defaults in case kernel
+    // supports sync_file_range, but glibc does not.
+    /* Wait upon writeout of all pages
+       in the range before performing the
+       write.  */
+    public static int SYNC_FILE_RANGE_WAIT_BEFORE = 1;
+    /* Initiate writeout of all those
+       dirty pages in the range which are
+       not presently under writeback.  */
+    public static int SYNC_FILE_RANGE_WRITE = 2;
+    /* Wait upon writeout of all pages in
+       the range after performing the
+       write.  */
+    public static int SYNC_FILE_RANGE_WAIT_AFTER = 4;
+
+    // Set to true via JNI if possible
+    public static boolean fadvisePossible = false;
+
+    private static boolean nativeLoaded = false;
+    private static boolean syncFileRangePossible = true;
+
+    private static long cacheTimeout = -1;
+
+    private static CacheManipulator cacheManipulator = new CacheManipulator();
+
+    public static CacheManipulator getCacheManipulator() {
+      return cacheManipulator;
+    }
+
+    public static void setCacheManipulator(CacheManipulator cacheManipulator) {
+      POSIX.cacheManipulator = cacheManipulator;
+    }
+
+    /**
+     * Used to manipulate the operating system cache.
+     */
+    @VisibleForTesting
+    public static class CacheManipulator {
+      public void mlock(String identifier, ByteBuffer buffer,
+          long len) throws IOException {
+        POSIX.mlock(buffer, len);
+      }
+
+      public long getMemlockLimit() {
+        return NativeIO.getMemlockLimit();
+      }
+
+      public long getOperatingSystemPageSize() {
+        return NativeIO.getOperatingSystemPageSize();
+      }
+
+      public void posixFadviseIfPossible(String identifier,
+        FileDescriptor fd, long offset, long len, int flags)
+            throws NativeIOException {
+        NativeIO.POSIX.posixFadviseIfPossible(identifier, fd, offset,
+            len, flags);
+      }
+
+      public boolean verifyCanMlock() {
+        return NativeIO.isAvailable();
+      }
+    }
+
+    /**
+     * A CacheManipulator used for testing which does not actually call mlock.
+     * This allows many tests to be run even when the operating system does not
+     * allow mlock, or only allows limited mlocking.
+     */
+    @VisibleForTesting
+    public static class NoMlockCacheManipulator extends CacheManipulator {
+      public void mlock(String identifier, ByteBuffer buffer,
+          long len) throws IOException {
+        LOG.info("mlocking " + identifier);
+      }
+
+      public long getMemlockLimit() {
+        return 1125899906842624L;
+      }
+
+      public long getOperatingSystemPageSize() {
+        return 4096;
+      }
+
+      public boolean verifyCanMlock() {
+        return true;
+      }
+    }
+
+    static {
+      initNativeLib();
+    }
+
+    /**
+     * Return true if the JNI-based native IO extensions are available.
+     */
+    public static boolean isAvailable() {
+      return NativeCodeLoader.isNativeCodeLoaded() && nativeLoaded;
+    }
+
+    private static void assertCodeLoaded() throws IOException {
+      if (!isAvailable()) {
+        throw new IOException("NativeIO was not loaded");
+      }
+    }
+
+    /** Wrapper around open(2) */
+    public static native FileDescriptor open(String path, int flags, int mode) throws IOException;
+    /** Wrapper around fstat(2) */
+    private static native Stat fstat(FileDescriptor fd) throws IOException;
+
+    /** Native chmod implementation. On UNIX, it is a wrapper around chmod(2) */
+    private static native void chmodImpl(String path, int mode) throws IOException;
+
+    public static void chmod(String path, int mode) throws IOException {
+      if (!RaftUtils.WINDOWS) {
+        chmodImpl(path, mode);
+      } else {
+        try {
+          chmodImpl(path, mode);
+        } catch (NativeIOException nioe) {
+          if (nioe.getErrorCode() == 3) {
+            throw new NativeIOException("No such file or directory",
+                Errno.ENOENT);
+          } else {
+            LOG.warn(String.format("NativeIO.chmod error (%d): %s",
+                nioe.getErrorCode(), nioe.getMessage()));
+            throw new NativeIOException("Unknown error", Errno.UNKNOWN);
+          }
+        }
+      }
+    }
+
+    /** Wrapper around posix_fadvise(2) */
+    static native void posix_fadvise(
+      FileDescriptor fd, long offset, long len, int flags) throws NativeIOException;
+
+    /** Wrapper around sync_file_range(2) */
+    static native void sync_file_range(
+      FileDescriptor fd, long offset, long nbytes, int flags) throws NativeIOException;
+
+    /**
+     * Call posix_fadvise on the given file descriptor. See the manpage
+     * for this syscall for more information. On systems where this
+     * call is not available, does nothing.
+     *
+     * @throws NativeIOException if there is an error with the syscall
+     */
+    static void posixFadviseIfPossible(String identifier,
+        FileDescriptor fd, long offset, long len, int flags)
+        throws NativeIOException {
+      if (nativeLoaded && fadvisePossible) {
+        try {
+          posix_fadvise(fd, offset, len, flags);
+        } catch (UnsatisfiedLinkError ule) {
+          fadvisePossible = false;
+        }
+      }
+    }
+
+    /**
+     * Call sync_file_range on the given file descriptor. See the manpage
+     * for this syscall for more information. On systems where this
+     * call is not available, does nothing.
+     *
+     * @throws NativeIOException if there is an error with the syscall
+     */
+    public static void syncFileRangeIfPossible(
+        FileDescriptor fd, long offset, long nbytes, int flags)
+        throws NativeIOException {
+      if (nativeLoaded && syncFileRangePossible) {
+        try {
+          sync_file_range(fd, offset, nbytes, flags);
+        } catch (UnsupportedOperationException | UnsatisfiedLinkError uoe) {
+          syncFileRangePossible = false;
+        }
+      }
+    }
+
+    static native void mlock_native(
+        ByteBuffer buffer, long len) throws NativeIOException;
+
+    /**
+     * Locks the provided direct ByteBuffer into memory, preventing it from
+     * swapping out. After a buffer is locked, future accesses will not incur
+     * a page fault.
+     *
+     * See the mlock(2) man page for more information.
+     */
+    static void mlock(ByteBuffer buffer, long len)
+        throws IOException {
+      assertCodeLoaded();
+      if (!buffer.isDirect()) {
+        throw new IOException("Cannot mlock a non-direct ByteBuffer");
+      }
+      mlock_native(buffer, len);
+    }
+
+    /**
+     * Unmaps the block from memory. See munmap(2).
+     *
+     * There isn't any portable way to unmap a memory region in Java.
+     * So we use the sun.nio method here.
+     * Note that unmapping a memory region could cause crashes if code
+     * continues to reference the unmapped code.  However, if we don't
+     * manually unmap the memory, we are dependent on the finalizer to
+     * do it, and we have no idea when the finalizer will run.
+     *
+     * @param buffer    The buffer to unmap.
+     */
+    public static void munmap(MappedByteBuffer buffer) {
+      if (buffer instanceof sun.nio.ch.DirectBuffer) {
+        sun.misc.Cleaner cleaner =
+            ((sun.nio.ch.DirectBuffer)buffer).cleaner();
+        cleaner.clean();
+      }
+    }
+
+    /** Linux only methods used for getOwner() implementation */
+    private static native long getUIDforFDOwnerforOwner(FileDescriptor fd) throws IOException;
+    private static native String getUserName(long uid) throws IOException;
+
+    /**
+     * Result type of the fstat call
+     */
+    public static class Stat {
+      private int ownerId, groupId;
+      private String owner, group;
+      private int mode;
+
+      // Mode constants - Set by JNI
+      public static int S_IFMT = -1;    /* type of file */
+      public static int S_IFIFO  = -1;  /* named pipe (fifo) */
+      public static int S_IFCHR  = -1;  /* character special */
+      public static int S_IFDIR  = -1;  /* directory */
+      public static int S_IFBLK  = -1;  /* block special */
+      public static int S_IFREG  = -1;  /* regular */
+      public static int S_IFLNK  = -1;  /* symbolic link */
+      public static int S_IFSOCK = -1;  /* socket */
+      public static int S_ISUID = -1;  /* set user id on execution */
+      public static int S_ISGID = -1;  /* set group id on execution */
+      public static int S_ISVTX = -1;  /* save swapped text even after use */
+      public static int S_IRUSR = -1;  /* read permission, owner */
+      public static int S_IWUSR = -1;  /* write permission, owner */
+      public static int S_IXUSR = -1;  /* execute/search permission, owner */
+
+      Stat(int ownerId, int groupId, int mode) {
+        this.ownerId = ownerId;
+        this.groupId = groupId;
+        this.mode = mode;
+      }
+
+      Stat(String owner, String group, int mode) {
+        if (!RaftUtils.WINDOWS) {
+          this.owner = owner;
+        } else {
+          this.owner = stripDomain(owner);
+        }
+        if (!RaftUtils.WINDOWS) {
+          this.group = group;
+        } else {
+          this.group = stripDomain(group);
+        }
+        this.mode = mode;
+      }
+
+      @Override
+      public String toString() {
+        return "Stat(owner='" + owner + "', group='" + group + "'" +
+          ", mode=" + mode + ")";
+      }
+
+      public String getOwner() {
+        return owner;
+      }
+      public String getGroup() {
+        return group;
+      }
+      public int getMode() {
+        return mode;
+      }
+    }
+
+    private static class CachedName {
+      final long timestamp;
+      final String name;
+
+      public CachedName(String name, long timestamp) {
+        this.name = name;
+        this.timestamp = timestamp;
+      }
+    }
+
+    public final static int MMAP_PROT_READ = 0x1;
+    public final static int MMAP_PROT_WRITE = 0x2;
+    public final static int MMAP_PROT_EXEC = 0x4;
+
+    public static native long mmap(FileDescriptor fd, int prot,
+        boolean shared, long length) throws IOException;
+
+    public static native void munmap(long addr, long length)
+        throws IOException;
+  }
+
+  private static boolean workaroundNonThreadSafePasswdCalls = false;
+
+
+  public static class Windows {
+    // Flags for CreateFile() call on Windows
+    public static final long GENERIC_READ = 0x80000000L;
+    public static final long GENERIC_WRITE = 0x40000000L;
+
+    public static final long FILE_SHARE_READ = 0x00000001L;
+    public static final long FILE_SHARE_WRITE = 0x00000002L;
+    public static final long FILE_SHARE_DELETE = 0x00000004L;
+
+    public static final long CREATE_NEW = 1;
+    public static final long CREATE_ALWAYS = 2;
+    public static final long OPEN_EXISTING = 3;
+    public static final long OPEN_ALWAYS = 4;
+    public static final long TRUNCATE_EXISTING = 5;
+
+    public static final long FILE_BEGIN = 0;
+    public static final long FILE_CURRENT = 1;
+    public static final long FILE_END = 2;
+
+    public static final long FILE_ATTRIBUTE_NORMAL = 0x00000080L;
+
+    /**
+     * Create a directory with permissions set to the specified mode.  By setting
+     * permissions at creation time, we avoid issues related to the user lacking
+     * WRITE_DAC rights on subsequent chmod calls.  One example where this can
+     * occur is writing to an SMB share where the user does not have Full Control
+     * rights, and therefore WRITE_DAC is denied.
+     *
+     * @param path directory to create
+     * @param mode permissions of new directory
+     * @throws IOException if there is an I/O error
+     */
+    public static void createDirectoryWithMode(File path, int mode)
+        throws IOException {
+      createDirectoryWithMode0(path.getAbsolutePath(), mode);
+    }
+
+    /** Wrapper around CreateDirectory() on Windows */
+    private static native void createDirectoryWithMode0(String path, int mode)
+        throws NativeIOException;
+
+    /** Wrapper around CreateFile() on Windows */
+    public static native FileDescriptor createFile(String path,
+        long desiredAccess, long shareMode, long creationDisposition)
+        throws IOException;
+
+    /**
+     * Create a file for write with permissions set to the specified mode.  By
+     * setting permissions at creation time, we avoid issues related to the user
+     * lacking WRITE_DAC rights on subsequent chmod calls.  One example where
+     * this can occur is writing to an SMB share where the user does not have
+     * Full Control rights, and therefore WRITE_DAC is denied.
+     *
+     * This method mimics the semantics implemented by the JDK in
+     * {@link FileOutputStream}.  The file is opened for truncate or
+     * append, the sharing mode allows other readers and writers, and paths
+     * longer than MAX_PATH are supported.  (See io_util_md.c in the JDK.)
+     *
+     * @param path file to create
+     * @param append if true, then open file for append
+     * @param mode permissions of new directory
+     * @return FileOutputStream of opened file
+     * @throws IOException if there is an I/O error
+     */
+    public static FileOutputStream createFileOutputStreamWithMode(File path,
+        boolean append, int mode) throws IOException {
+      long shareMode = FILE_SHARE_READ | FILE_SHARE_WRITE;
+      long creationDisposition = append ? OPEN_ALWAYS : CREATE_ALWAYS;
+      return new FileOutputStream(createFileWithMode0(path.getAbsolutePath(),
+          GENERIC_WRITE, shareMode, creationDisposition, mode));
+    }
+
+    /** Wrapper around CreateFile() with security descriptor on Windows */
+    private static native FileDescriptor createFileWithMode0(String path,
+        long desiredAccess, long shareMode, long creationDisposition, int mode)
+        throws NativeIOException;
+
+    /** Wrapper around SetFilePointer() on Windows */
+    public static native long setFilePointer(FileDescriptor fd,
+        long distanceToMove, long moveMethod) throws IOException;
+
+    /** Windows only methods used for getOwner() implementation */
+    private static native String getOwner(FileDescriptor fd) throws IOException;
+
+    /** Supported list of Windows access right flags */
+    public enum AccessRight {
+      ACCESS_READ (0x0001),      // FILE_READ_DATA
+      ACCESS_WRITE (0x0002),     // FILE_WRITE_DATA
+      ACCESS_EXECUTE (0x0020);   // FILE_EXECUTE
+
+      private final int accessRight;
+      AccessRight(int access) {
+        accessRight = access;
+      }
+
+      public int accessRight() {
+        return accessRight;
+      }
+    }
+
+    /** Windows only method used to check if the current process has requested
+     *  access rights on the given path. */
+    private static native boolean access0(String path, int requestedAccess);
+
+    /**
+     * Checks whether the current process has desired access rights on
+     * the given path.
+     *
+     * Longer term this native function can be substituted with JDK7
+     * function Files#isReadable, isWritable, isExecutable.
+     *
+     * @param path input path
+     * @param desiredAccess ACCESS_READ, ACCESS_WRITE or ACCESS_EXECUTE
+     * @return true if access is allowed
+     * @throws IOException I/O exception on error
+     */
+    public static boolean access(String path, AccessRight desiredAccess)
+        throws IOException {
+      return access0(path, desiredAccess.accessRight());
+    }
+
+    /**
+     * Extends both the minimum and maximum working set size of the current
+     * process.  This method gets the current minimum and maximum working set
+     * size, adds the requested amount to each and then sets the minimum and
+     * maximum working set size to the new values.  Controlling the working set
+     * size of the process also controls the amount of memory it can lock.
+     *
+     * @param delta amount to increment minimum and maximum working set size
+     * @throws IOException for any error
+     * @see POSIX#mlock(ByteBuffer, long)
+     */
+    public static native void extendWorkingSetSize(long delta) throws IOException;
+
+    static {
+      initNativeLib();
+    }
+  }
+
+  private static boolean nativeLoaded = false;
+
+  static {
+    initNativeLib();
+  }
+
+  private static void initNativeLib() {
+    if (NativeCodeLoader.isNativeCodeLoaded()) {
+      try {
+        initNative();
+        nativeLoaded = true;
+      } catch (Throwable t) {
+        LOG.debug("Unable to initialize NativeIO libraries", t);
+      }
+    }
+  }
+
+  /**
+   * Return true if the JNI-based native IO extensions are available.
+   */
+  public static boolean isAvailable() {
+    return NativeCodeLoader.isNativeCodeLoaded() && nativeLoaded;
+  }
+
+  /** Initialize the JNI method ID and class ID cache */
+  private static native void initNative();
+
+  /**
+   * Get the maximum number of bytes that can be locked into memory at any
+   * given point.
+   *
+   * @return 0 if no bytes can be locked into memory;
+   *         Long.MAX_VALUE if there is no limit;
+   *         The number of bytes that can be locked into memory otherwise.
+   */
+  static long getMemlockLimit() {
+    return isAvailable() ? getMemlockLimit0() : 0;
+  }
+
+  private static native long getMemlockLimit0();
+
+  /**
+   * @return the operating system's page size.
+   */
+  static long getOperatingSystemPageSize() {
+    try {
+      Field f = Unsafe.class.getDeclaredField("theUnsafe");
+      f.setAccessible(true);
+      Unsafe unsafe = (Unsafe)f.get(null);
+      return unsafe.pageSize();
+    } catch (Throwable e) {
+      LOG.warn("Unable to get operating system page size.  Guessing 4096.", e);
+      return 4096;
+    }
+  }
+
+  private static class CachedUid {
+    final long timestamp;
+    final String username;
+    public CachedUid(String username, long timestamp) {
+      this.timestamp = timestamp;
+      this.username = username;
+    }
+  }
+
+  private static boolean initialized = false;
+
+  /**
+   * The Windows logon name has two part, NetBIOS domain name and
+   * user account name, of the format DOMAIN\UserName. This method
+   * will remove the domain part of the full logon name.
+   *
+   * @param name full principal name containing the domain
+   * @return name with domain removed
+   */
+  private static String stripDomain(String name) {
+    int i = name.indexOf('\\');
+    if (i != -1)
+      name = name.substring(i + 1);
+    return name;
+  }
+
+  /**
+   * Create a FileInputStream that shares delete permission on the
+   * file opened, i.e. other process can delete the file the
+   * FileInputStream is reading. Only Windows implementation uses
+   * the native interface.
+   */
+  public static FileInputStream getShareDeleteFileInputStream(File f)
+      throws IOException {
+    if (!RaftUtils.WINDOWS) {
+      // On Linux the default FileInputStream shares delete permission
+      // on the file opened.
+      //
+      return new FileInputStream(f);
+    } else {
+      // Use Windows native interface to create a FileInputStream that
+      // shares delete permission on the file opened.
+      //
+      FileDescriptor fd = Windows.createFile(
+          f.getAbsolutePath(),
+          Windows.GENERIC_READ,
+          Windows.FILE_SHARE_READ |
+              Windows.FILE_SHARE_WRITE |
+              Windows.FILE_SHARE_DELETE,
+          Windows.OPEN_EXISTING);
+      return new FileInputStream(fd);
+    }
+  }
+
+  /**
+   * Create a FileInputStream that shares delete permission on the
+   * file opened at a given offset, i.e. other process can delete
+   * the file the FileInputStream is reading. Only Windows implementation
+   * uses the native interface.
+   */
+  public static FileInputStream getShareDeleteFileInputStream(File f, long seekOffset)
+      throws IOException {
+    if (!RaftUtils.WINDOWS) {
+      RandomAccessFile rf = new RandomAccessFile(f, "r");
+      if (seekOffset > 0) {
+        rf.seek(seekOffset);
+      }
+      return new FileInputStream(rf.getFD());
+    } else {
+      // Use Windows native interface to create a FileInputStream that
+      // shares delete permission on the file opened, and set it to the
+      // given offset.
+      //
+      FileDescriptor fd = NativeIO.Windows.createFile(
+          f.getAbsolutePath(),
+          NativeIO.Windows.GENERIC_READ,
+          NativeIO.Windows.FILE_SHARE_READ |
+              NativeIO.Windows.FILE_SHARE_WRITE |
+              NativeIO.Windows.FILE_SHARE_DELETE,
+          NativeIO.Windows.OPEN_EXISTING);
+      if (seekOffset > 0)
+        NativeIO.Windows.setFilePointer(fd, seekOffset, NativeIO.Windows.FILE_BEGIN);
+      return new FileInputStream(fd);
+    }
+  }
+
+  /**
+   * Create the specified File for write access, ensuring that it does not exist.
+   * @param f the file that we want to create
+   * @param permissions we want to have on the file (if security is enabled)
+   *
+   * @throws AlreadyExistsException if the file already exists
+   * @throws IOException if any other error occurred
+   */
+  public static FileOutputStream getCreateForWriteFileOutputStream(File f, int permissions)
+      throws IOException {
+    if (!RaftUtils.WINDOWS) {
+      // Use the native wrapper around open(2)
+      try {
+        FileDescriptor fd = NativeIO.POSIX.open(f.getAbsolutePath(),
+            NativeIO.POSIX.O_WRONLY | NativeIO.POSIX.O_CREAT
+                | NativeIO.POSIX.O_EXCL, permissions);
+        return new FileOutputStream(fd);
+      } catch (NativeIOException nioe) {
+        if (nioe.getErrno() == Errno.EEXIST) {
+          throw new AlreadyExistsException(nioe);
+        }
+        throw nioe;
+      }
+    } else {
+      // Use the Windows native APIs to create equivalent FileOutputStream
+      try {
+        FileDescriptor fd = NativeIO.Windows.createFile(f.getCanonicalPath(),
+            NativeIO.Windows.GENERIC_WRITE,
+            NativeIO.Windows.FILE_SHARE_DELETE
+                | NativeIO.Windows.FILE_SHARE_READ
+                | NativeIO.Windows.FILE_SHARE_WRITE,
+            NativeIO.Windows.CREATE_NEW);
+        NativeIO.POSIX.chmod(f.getCanonicalPath(), permissions);
+        return new FileOutputStream(fd);
+      } catch (NativeIOException nioe) {
+        if (nioe.getErrorCode() == 80) {
+          // ERROR_FILE_EXISTS
+          // 80 (0x50)
+          // The file exists
+          throw new AlreadyExistsException(nioe);
+        }
+        throw nioe;
+      }
+    }
+  }
+  
+  /**
+   * A version of renameTo that throws a descriptive exception when it fails.
+   *
+   * @param src                  The source path
+   * @param dst                  The destination path
+   * 
+   * @throws NativeIOException   On failure.
+   */
+  public static void renameTo(File src, File dst)
+      throws IOException {
+    if (!nativeLoaded) {
+      if (!src.renameTo(dst)) {
+        throw new IOException("renameTo(src=" + src + ", dst=" +
+          dst + ") failed.");
+      }
+    } else {
+      renameTo0(src.getAbsolutePath(), dst.getAbsolutePath());
+    }
+  }
+
+  /**
+   * A version of renameTo that throws a descriptive exception when it fails.
+   *
+   * @param src                  The source path
+   * @param dst                  The destination path
+   * 
+   * @throws NativeIOException   On failure.
+   */
+  private static native void renameTo0(String src, String dst)
+      throws NativeIOException;
+
+  private static native void link0(String src, String dst)
+      throws NativeIOException;
+
+  /**
+   * Unbuffered file copy from src to dst without tainting OS buffer cache
+   *
+   * In POSIX platform:
+   * It uses FileChannel#transferTo() which internally attempts
+   * unbuffered IO on OS with native sendfile64() support and falls back to
+   * buffered IO otherwise.
+   *
+   * It minimizes the number of FileChannel#transferTo call by passing the the
+   * src file size directly instead of a smaller size as the 3rd parameter.
+   * This saves the number of sendfile64() system call when native sendfile64()
+   * is supported. In the two fall back cases where sendfile is not supported,
+   * FileChannle#transferTo already has its own batching of size 8 MB and 8 KB,
+   * respectively.
+   *
+   * In Windows Platform:
+   * It uses its own native wrapper of CopyFileEx with COPY_FILE_NO_BUFFERING
+   * flag, which is supported on Windows Server 2008 and above.
+   *
+   * Ideally, we should use FileChannel#transferTo() across both POSIX and Windows
+   * platform. Unfortunately, the wrapper(Java_sun_nio_ch_FileChannelImpl_transferTo0)
+   * used by FileChannel#transferTo for unbuffered IO is not implemented on Windows.
+   * Based on OpenJDK 6/7/8 source code, Java_sun_nio_ch_FileChannelImpl_transferTo0
+   * on Windows simply returns IOS_UNSUPPORTED.
+   *
+   * Note: This simple native wrapper does minimal parameter checking before copy and
+   * consistency check (e.g., size) after copy.
+   * It is recommended to use wrapper function like
+   * the Storage#nativeCopyFileUnbuffered() function with pre/post copy checks.
+   *
+   * @param src                  The source path
+   * @param dst                  The destination path
+   */
+  public static void copyFileUnbuffered(File src, File dst) throws IOException {
+    if (nativeLoaded && RaftUtils.WINDOWS) {
+      copyFileUnbuffered0(src.getAbsolutePath(), dst.getAbsolutePath());
+    } else {
+      FileInputStream fis = null;
+      FileOutputStream fos = null;
+      FileChannel input = null;
+      FileChannel output = null;
+      try {
+        fis = new FileInputStream(src);
+        fos = new FileOutputStream(dst);
+        input = fis.getChannel();
+        output = fos.getChannel();
+        long remaining = input.size();
+        long position = 0;
+        long transferred;
+        while (remaining > 0) {
+          transferred = input.transferTo(position, remaining, output);
+          remaining -= transferred;
+          position += transferred;
+        }
+      } finally {
+        RaftUtils.cleanup(LOG, output, fos, input, fis);
+      }
+    }
+  }
+
+  private static native void copyFileUnbuffered0(String src, String dst)
+      throws NativeIOException;
+}

http://git-wip-us.apache.org/repos/asf/incubator-ratis/blob/7e71a2e0/ratis-common/src/main/java/org/apache/ratis/io/nativeio/NativeIOException.java
----------------------------------------------------------------------
diff --git a/ratis-common/src/main/java/org/apache/ratis/io/nativeio/NativeIOException.java b/ratis-common/src/main/java/org/apache/ratis/io/nativeio/NativeIOException.java
new file mode 100644
index 0000000..58b83e7
--- /dev/null
+++ b/ratis-common/src/main/java/org/apache/ratis/io/nativeio/NativeIOException.java
@@ -0,0 +1,70 @@
+/**
+ * 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.ratis.io.nativeio;
+
+import java.io.IOException;
+
+import org.apache.ratis.util.RaftUtils;
+
+
+/**
+ * An exception generated by a call to the native IO code.
+ *
+ * These exceptions simply wrap <i>errno</i> result codes on Linux,
+ * or the System Error Code on Windows.
+ */
+public class NativeIOException extends IOException {
+  private static final long serialVersionUID = 1L;
+
+  private Errno errno;
+
+  // Java has no unsigned primitive error code. Use a signed 32-bit
+  // integer to hold the unsigned 32-bit integer.
+  private int errorCode;
+
+  public NativeIOException(String msg, Errno errno) {
+    super(msg);
+    this.errno = errno;
+    // Windows error code is always set to ERROR_SUCCESS on Linux,
+    // i.e. no failure on Windows
+    this.errorCode = 0;
+  }
+
+  public NativeIOException(String msg, int errorCode) {
+    super(msg);
+    this.errorCode = errorCode;
+    this.errno = Errno.UNKNOWN;
+  }
+
+  public long getErrorCode() {
+    return errorCode;
+  }
+
+  public Errno getErrno() {
+    return errno;
+  }
+
+  @Override
+  public String toString() {
+    if (RaftUtils.WINDOWS)
+      return errorCode + ": " + super.getMessage();
+    else
+      return errno.toString() + ": " + super.getMessage();
+  }
+}
+

http://git-wip-us.apache.org/repos/asf/incubator-ratis/blob/7e71a2e0/ratis-common/src/main/java/org/apache/ratis/protocol/AlreadyExistsException.java
----------------------------------------------------------------------
diff --git a/ratis-common/src/main/java/org/apache/ratis/protocol/AlreadyExistsException.java b/ratis-common/src/main/java/org/apache/ratis/protocol/AlreadyExistsException.java
new file mode 100644
index 0000000..cc441f2
--- /dev/null
+++ b/ratis-common/src/main/java/org/apache/ratis/protocol/AlreadyExistsException.java
@@ -0,0 +1,36 @@
+/**
+ * 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.ratis.protocol;
+
+import java.io.IOException;
+
+/**
+ * Signals that an attempt to create a file at a given pathname has failed
+ * because another file already existed at that path.
+ */
+public class AlreadyExistsException extends IOException {
+  private static final long serialVersionUID = 1L;
+
+  public AlreadyExistsException(String msg) {
+    super(msg);
+  }
+
+  public AlreadyExistsException(Throwable cause) {
+    super(cause);
+  }
+}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/incubator-ratis/blob/7e71a2e0/ratis-common/src/main/java/org/apache/ratis/protocol/ChecksumException.java
----------------------------------------------------------------------
diff --git a/ratis-common/src/main/java/org/apache/ratis/protocol/ChecksumException.java b/ratis-common/src/main/java/org/apache/ratis/protocol/ChecksumException.java
new file mode 100644
index 0000000..1742c24
--- /dev/null
+++ b/ratis-common/src/main/java/org/apache/ratis/protocol/ChecksumException.java
@@ -0,0 +1,35 @@
+/**
+ * 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.ratis.protocol;
+
+import java.io.IOException;
+
+/** Thrown for checksum errors. */
+public class ChecksumException extends IOException {
+  private static final long serialVersionUID = 1L;
+  private long pos;
+  public ChecksumException(String description, long pos) {
+    super(description);
+    this.pos = pos;
+  }
+
+  public long getPos() {
+    return pos;
+  }
+}

http://git-wip-us.apache.org/repos/asf/incubator-ratis/blob/7e71a2e0/ratis-common/src/main/java/org/apache/ratis/protocol/Message.java
----------------------------------------------------------------------
diff --git a/ratis-common/src/main/java/org/apache/ratis/protocol/Message.java b/ratis-common/src/main/java/org/apache/ratis/protocol/Message.java
new file mode 100644
index 0000000..77ef267
--- /dev/null
+++ b/ratis-common/src/main/java/org/apache/ratis/protocol/Message.java
@@ -0,0 +1,30 @@
+/**
+ * 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.ratis.protocol;
+
+import org.apache.ratis.shaded.com.google.protobuf.ByteString;
+
+/**
+ * The information clients append to the raft ring.
+ */
+public interface Message {
+  /**
+   * @return the content of the message
+   */
+  ByteString getContent();
+}

http://git-wip-us.apache.org/repos/asf/incubator-ratis/blob/7e71a2e0/ratis-common/src/main/java/org/apache/ratis/protocol/NotLeaderException.java
----------------------------------------------------------------------
diff --git a/ratis-common/src/main/java/org/apache/ratis/protocol/NotLeaderException.java b/ratis-common/src/main/java/org/apache/ratis/protocol/NotLeaderException.java
new file mode 100644
index 0000000..1306290
--- /dev/null
+++ b/ratis-common/src/main/java/org/apache/ratis/protocol/NotLeaderException.java
@@ -0,0 +1,40 @@
+/**
+ * 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.ratis.protocol;
+
+public class NotLeaderException extends RaftException {
+  private final RaftPeer suggestedLeader;
+  /** the client may need to update its RaftPeer list */
+  private final RaftPeer[] peers;
+
+  public NotLeaderException(String id, RaftPeer suggestedLeader,
+      RaftPeer[] peers) {
+    super("Server " + id + " is not the leader (" + suggestedLeader
+        + "). Request must be sent to leader.");
+    this.suggestedLeader = suggestedLeader;
+    this.peers = peers == null ? RaftPeer.EMPTY_PEERS : peers;
+  }
+
+  public RaftPeer getSuggestedLeader() {
+    return suggestedLeader;
+  }
+
+  public RaftPeer[] getPeers() {
+    return peers;
+  }
+}

http://git-wip-us.apache.org/repos/asf/incubator-ratis/blob/7e71a2e0/ratis-common/src/main/java/org/apache/ratis/protocol/RaftClientAsynchronousProtocol.java
----------------------------------------------------------------------
diff --git a/ratis-common/src/main/java/org/apache/ratis/protocol/RaftClientAsynchronousProtocol.java b/ratis-common/src/main/java/org/apache/ratis/protocol/RaftClientAsynchronousProtocol.java
new file mode 100644
index 0000000..3298431
--- /dev/null
+++ b/ratis-common/src/main/java/org/apache/ratis/protocol/RaftClientAsynchronousProtocol.java
@@ -0,0 +1,30 @@
+/**
+ * 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.ratis.protocol;
+
+import java.io.IOException;
+import java.util.concurrent.CompletableFuture;
+
+/** Asynchronous version of {@link RaftClientProtocol}. */
+public interface RaftClientAsynchronousProtocol {
+  CompletableFuture<RaftClientReply> submitClientRequestAsync(
+      RaftClientRequest request) throws IOException;
+
+  CompletableFuture<RaftClientReply> setConfigurationAsync(
+      SetConfigurationRequest request) throws IOException;
+}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/incubator-ratis/blob/7e71a2e0/ratis-common/src/main/java/org/apache/ratis/protocol/RaftClientProtocol.java
----------------------------------------------------------------------
diff --git a/ratis-common/src/main/java/org/apache/ratis/protocol/RaftClientProtocol.java b/ratis-common/src/main/java/org/apache/ratis/protocol/RaftClientProtocol.java
new file mode 100644
index 0000000..b3cbcc3
--- /dev/null
+++ b/ratis-common/src/main/java/org/apache/ratis/protocol/RaftClientProtocol.java
@@ -0,0 +1,26 @@
+/**
+ * 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.ratis.protocol;
+
+import java.io.IOException;
+
+public interface RaftClientProtocol {
+  RaftClientReply submitClientRequest(RaftClientRequest request) throws IOException;
+
+  RaftClientReply setConfiguration(SetConfigurationRequest request) throws IOException;
+}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/incubator-ratis/blob/7e71a2e0/ratis-common/src/main/java/org/apache/ratis/protocol/RaftClientReply.java
----------------------------------------------------------------------
diff --git a/ratis-common/src/main/java/org/apache/ratis/protocol/RaftClientReply.java b/ratis-common/src/main/java/org/apache/ratis/protocol/RaftClientReply.java
new file mode 100644
index 0000000..8c5cd75
--- /dev/null
+++ b/ratis-common/src/main/java/org/apache/ratis/protocol/RaftClientReply.java
@@ -0,0 +1,91 @@
+/**
+ * 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.ratis.protocol;
+
+public class RaftClientReply extends RaftRpcMessage {
+  private final String requestorId;
+  private final String replierId;
+  private final boolean success;
+  private final long seqNum;
+
+  /** non-null if the server is not leader */
+  private final NotLeaderException notLeaderException;
+  private final Message message;
+
+  public RaftClientReply(String requestorId, String replierId, long seqNum,
+      boolean success, Message message, NotLeaderException notLeaderException) {
+    this.requestorId = requestorId;
+    this.replierId = replierId;
+    this.success = success;
+    this.seqNum = seqNum;
+    this.message = message;
+    this.notLeaderException = notLeaderException;
+  }
+
+  public RaftClientReply(RaftClientRequest request,
+      NotLeaderException notLeaderException) {
+    this(request.getRequestorId(), request.getReplierId(), request.getSeqNum(),
+        false, null, notLeaderException);
+  }
+
+  public RaftClientReply(RaftClientRequest request, Message message) {
+    this(request.getRequestorId(), request.getReplierId(), request.getSeqNum(),
+        true, message, null);
+  }
+
+  @Override
+  public final boolean isRequest() {
+    return false;
+  }
+
+  @Override
+  public String getRequestorId() {
+    return requestorId;
+  }
+
+  @Override
+  public String getReplierId() {
+    return replierId;
+  }
+
+  public long getSeqNum() {
+    return seqNum;
+  }
+
+  @Override
+  public String toString() {
+    return super.toString() + ", seqNum: " + getSeqNum()
+        + ", success: " + isSuccess();
+  }
+
+  public boolean isSuccess() {
+    return success;
+  }
+
+  public Message getMessage() {
+    return message;
+  }
+
+  public NotLeaderException getNotLeaderException() {
+    return notLeaderException;
+  }
+
+  public boolean isNotLeader() {
+    return notLeaderException != null;
+  }
+}

http://git-wip-us.apache.org/repos/asf/incubator-ratis/blob/7e71a2e0/ratis-common/src/main/java/org/apache/ratis/protocol/RaftClientRequest.java
----------------------------------------------------------------------
diff --git a/ratis-common/src/main/java/org/apache/ratis/protocol/RaftClientRequest.java b/ratis-common/src/main/java/org/apache/ratis/protocol/RaftClientRequest.java
new file mode 100644
index 0000000..90b648a
--- /dev/null
+++ b/ratis-common/src/main/java/org/apache/ratis/protocol/RaftClientRequest.java
@@ -0,0 +1,73 @@
+/**
+ * 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.ratis.protocol;
+
+public class RaftClientRequest extends RaftRpcMessage {
+  private final String requestorId;
+  private final String replierId;
+  private final long seqNum;
+  private final Message message;
+  private final boolean readOnly;
+
+  public RaftClientRequest(String  requestorId, String replierId, long seqNum,
+                           Message message) {
+    this(requestorId, replierId, seqNum, message, false);
+  }
+
+  public RaftClientRequest(String requestorId, String replierId, long seqNum,
+       Message message, boolean readOnly) {
+    this.requestorId = requestorId;
+    this.replierId = replierId;
+    this.seqNum = seqNum;
+    this.message = message;
+    this.readOnly = readOnly;
+  }
+
+  @Override
+  public final boolean isRequest() {
+    return true;
+  }
+
+  @Override
+  public String getRequestorId() {
+    return requestorId;
+  }
+
+  @Override
+  public String getReplierId() {
+    return replierId;
+  }
+
+  public long getSeqNum() {
+    return seqNum;
+  }
+
+  public Message getMessage() {
+    return message;
+  }
+
+  public boolean isReadOnly() {
+    return readOnly;
+  }
+
+  @Override
+  public String toString() {
+    return super.toString() + ", seqNum: " + seqNum + ", "
+        + (isReadOnly()? "RO": "RW");
+  }
+}

http://git-wip-us.apache.org/repos/asf/incubator-ratis/blob/7e71a2e0/ratis-common/src/main/java/org/apache/ratis/protocol/RaftException.java
----------------------------------------------------------------------
diff --git a/ratis-common/src/main/java/org/apache/ratis/protocol/RaftException.java b/ratis-common/src/main/java/org/apache/ratis/protocol/RaftException.java
new file mode 100644
index 0000000..11aac90
--- /dev/null
+++ b/ratis-common/src/main/java/org/apache/ratis/protocol/RaftException.java
@@ -0,0 +1,36 @@
+/**
+ * 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.ratis.protocol;
+
+import java.io.IOException;
+
+public class RaftException extends IOException {
+  private static final long serialVersionUID = 1L;
+
+  public RaftException(String message) {
+    super(message);
+  }
+
+  public RaftException(Throwable cause) {
+    super(cause);
+  }
+
+  public RaftException(String message, Throwable cause) {
+    super(message, cause);
+  }
+}

http://git-wip-us.apache.org/repos/asf/incubator-ratis/blob/7e71a2e0/ratis-common/src/main/java/org/apache/ratis/protocol/RaftPeer.java
----------------------------------------------------------------------
diff --git a/ratis-common/src/main/java/org/apache/ratis/protocol/RaftPeer.java b/ratis-common/src/main/java/org/apache/ratis/protocol/RaftPeer.java
new file mode 100644
index 0000000..a32aaa0
--- /dev/null
+++ b/ratis-common/src/main/java/org/apache/ratis/protocol/RaftPeer.java
@@ -0,0 +1,79 @@
+/**
+ * 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.ratis.protocol;
+
+import com.google.common.net.HostAndPort;
+
+import java.net.InetSocketAddress;
+
+/**
+ * A {@link RaftPeer} is a server in a Raft cluster.
+ *
+ * The objects of this class are immutable.
+ */
+public class RaftPeer {
+  public static final RaftPeer[] EMPTY_PEERS = {};
+
+  /** The id of the peer. */
+  private final String id;
+  /** The address of the peer. */
+  private final String address;
+
+  /** Construct a peer with the given id and a null address. */
+  public RaftPeer(String id) {
+    this(id, (String)null);
+  }
+
+  /** Construct a peer with the given id and address. */
+  public RaftPeer(String id, InetSocketAddress address) {
+    this(id, address == null ? null :
+        HostAndPort.fromParts(address.getAddress().getHostAddress(),
+            address.getPort()).toString());
+  }
+
+  /** Construct a peer with the given id and address. */
+  public RaftPeer(String id, String address) {
+    this.id = id;
+    this.address = address;
+  }
+
+  /** @return The id of the peer. */
+  public String getId() {
+    return id;
+  }
+
+  /** @return The address of the peer. */
+  public String getAddress() {
+    return address;
+  }
+
+  @Override
+  public String toString() {
+    return id + ":" + address;
+  }
+
+  @Override
+  public boolean equals(Object o) {
+    return (o instanceof RaftPeer) && id.equals(((RaftPeer) o).getId());
+  }
+
+  @Override
+  public int hashCode() {
+    return id.hashCode();
+  }
+}

http://git-wip-us.apache.org/repos/asf/incubator-ratis/blob/7e71a2e0/ratis-common/src/main/java/org/apache/ratis/protocol/RaftRpcMessage.java
----------------------------------------------------------------------
diff --git a/ratis-common/src/main/java/org/apache/ratis/protocol/RaftRpcMessage.java b/ratis-common/src/main/java/org/apache/ratis/protocol/RaftRpcMessage.java
new file mode 100644
index 0000000..82f1ebb
--- /dev/null
+++ b/ratis-common/src/main/java/org/apache/ratis/protocol/RaftRpcMessage.java
@@ -0,0 +1,33 @@
+/**
+ * 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.ratis.protocol;
+
+public abstract class RaftRpcMessage {
+
+  public abstract boolean isRequest();
+
+  public abstract String getRequestorId();
+
+  public abstract String getReplierId();
+
+  @Override
+  public String toString() {
+    return getClass().getSimpleName() + "(" + getRequestorId()
+        + (isRequest()? "->": "<-") + getReplierId() + ")";
+  }
+}

http://git-wip-us.apache.org/repos/asf/incubator-ratis/blob/7e71a2e0/ratis-common/src/main/java/org/apache/ratis/protocol/ReconfigurationInProgressException.java
----------------------------------------------------------------------
diff --git a/ratis-common/src/main/java/org/apache/ratis/protocol/ReconfigurationInProgressException.java b/ratis-common/src/main/java/org/apache/ratis/protocol/ReconfigurationInProgressException.java
new file mode 100644
index 0000000..3d0f093
--- /dev/null
+++ b/ratis-common/src/main/java/org/apache/ratis/protocol/ReconfigurationInProgressException.java
@@ -0,0 +1,24 @@
+/**
+ * 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.ratis.protocol;
+
+public class ReconfigurationInProgressException extends RaftException {
+  public ReconfigurationInProgressException(String message) {
+    super(message);
+  }
+}

http://git-wip-us.apache.org/repos/asf/incubator-ratis/blob/7e71a2e0/ratis-common/src/main/java/org/apache/ratis/protocol/ReconfigurationTimeoutException.java
----------------------------------------------------------------------
diff --git a/ratis-common/src/main/java/org/apache/ratis/protocol/ReconfigurationTimeoutException.java b/ratis-common/src/main/java/org/apache/ratis/protocol/ReconfigurationTimeoutException.java
new file mode 100644
index 0000000..69a2e51
--- /dev/null
+++ b/ratis-common/src/main/java/org/apache/ratis/protocol/ReconfigurationTimeoutException.java
@@ -0,0 +1,24 @@
+/**
+ * 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.ratis.protocol;
+
+public class ReconfigurationTimeoutException extends RaftException {
+  public ReconfigurationTimeoutException(String message) {
+    super(message);
+  }
+}

http://git-wip-us.apache.org/repos/asf/incubator-ratis/blob/7e71a2e0/ratis-common/src/main/java/org/apache/ratis/protocol/SetConfigurationRequest.java
----------------------------------------------------------------------
diff --git a/ratis-common/src/main/java/org/apache/ratis/protocol/SetConfigurationRequest.java b/ratis-common/src/main/java/org/apache/ratis/protocol/SetConfigurationRequest.java
new file mode 100644
index 0000000..84449d4
--- /dev/null
+++ b/ratis-common/src/main/java/org/apache/ratis/protocol/SetConfigurationRequest.java
@@ -0,0 +1,39 @@
+/**
+ * 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.ratis.protocol;
+
+import java.util.Arrays;
+
+public class SetConfigurationRequest extends RaftClientRequest {
+  private final RaftPeer[] peers;
+
+  public SetConfigurationRequest(String requestorId, String replierId,
+      long seqNum, RaftPeer[] peers) {
+    super(requestorId, replierId, seqNum, null);
+    this.peers = peers;
+  }
+
+  public RaftPeer[] getPeersInNewConf() {
+    return peers;
+  }
+
+  @Override
+  public String toString() {
+    return super.toString() + ", peers:" + Arrays.asList(getPeersInNewConf());
+  }
+}

http://git-wip-us.apache.org/repos/asf/incubator-ratis/blob/7e71a2e0/ratis-common/src/main/java/org/apache/ratis/protocol/StateMachineException.java
----------------------------------------------------------------------
diff --git a/ratis-common/src/main/java/org/apache/ratis/protocol/StateMachineException.java b/ratis-common/src/main/java/org/apache/ratis/protocol/StateMachineException.java
new file mode 100644
index 0000000..099133d
--- /dev/null
+++ b/ratis-common/src/main/java/org/apache/ratis/protocol/StateMachineException.java
@@ -0,0 +1,28 @@
+/**
+ * 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.ratis.protocol;
+
+public class StateMachineException extends RaftException {
+  public StateMachineException(String serverId, Exception cause) {
+    super(cause.getClass().getName() + " from Server " + serverId, cause);
+  }
+
+  public StateMachineException(String msg) {
+    super(msg);
+  }
+}

http://git-wip-us.apache.org/repos/asf/incubator-ratis/blob/7e71a2e0/ratis-common/src/main/java/org/apache/ratis/util/AtomicFileOutputStream.java
----------------------------------------------------------------------
diff --git a/ratis-common/src/main/java/org/apache/ratis/util/AtomicFileOutputStream.java b/ratis-common/src/main/java/org/apache/ratis/util/AtomicFileOutputStream.java
new file mode 100644
index 0000000..e8e267e
--- /dev/null
+++ b/ratis-common/src/main/java/org/apache/ratis/util/AtomicFileOutputStream.java
@@ -0,0 +1,112 @@
+/**
+ * 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.ratis.util;
+
+import org.apache.ratis.io.nativeio.NativeIO;
+import org.apache.ratis.io.nativeio.NativeIOException;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.io.*;
+
+/**
+ * A FileOutputStream that has the property that it will only show
+ * up at its destination once it has been entirely written and flushed
+ * to disk. While being written, it will use a .tmp suffix.
+ *
+ * When the output stream is closed, it is flushed, fsynced, and
+ * will be moved into place, overwriting any file that already
+ * exists at that location.
+ *
+ * <b>NOTE</b>: on Windows platforms, it will not atomically
+ * replace the target file - instead the target file is deleted
+ * before this one is moved into place.
+ */
+public class AtomicFileOutputStream extends FilterOutputStream {
+
+  public static final String TMP_EXTENSION = ".tmp";
+
+  public static final Logger LOG = LoggerFactory.getLogger(AtomicFileOutputStream.class);
+
+  private final File origFile;
+  private final File tmpFile;
+
+  public AtomicFileOutputStream(File f) throws FileNotFoundException {
+    // Code unfortunately must be duplicated below since we can't assign anything
+    // before calling super
+    super(new FileOutputStream(new File(f.getParentFile(), f.getName() + TMP_EXTENSION)));
+    origFile = f.getAbsoluteFile();
+    tmpFile = new File(f.getParentFile(), f.getName() + TMP_EXTENSION).getAbsoluteFile();
+  }
+
+  @Override
+  public void close() throws IOException {
+    boolean triedToClose = false, success = false;
+    try {
+      flush();
+      ((FileOutputStream)out).getChannel().force(true);
+
+      triedToClose = true;
+      super.close();
+      success = true;
+    } finally {
+      if (success) {
+        boolean renamed = tmpFile.renameTo(origFile);
+        if (!renamed) {
+          // On windows, renameTo does not replace.
+          if (origFile.exists() && !origFile.delete()) {
+            throw new IOException("Could not delete original file " + origFile);
+          }
+          try {
+            NativeIO.renameTo(tmpFile, origFile);
+          } catch (NativeIOException e) {
+            throw new IOException("Could not rename temporary file " + tmpFile
+                + " to " + origFile + " due to failure in native rename. "
+                + e.toString());
+          }
+        }
+      } else {
+        if (!triedToClose) {
+          // If we failed when flushing, try to close it to not leak an FD
+          RaftUtils.cleanup(LOG, out);
+        }
+        // close wasn't successful, try to delete the tmp file
+        if (!tmpFile.delete()) {
+          LOG.warn("Unable to delete tmp file " + tmpFile);
+        }
+      }
+    }
+  }
+
+  /**
+   * Close the atomic file, but do not "commit" the temporary file
+   * on top of the destination. This should be used if there is a failure
+   * in writing.
+   */
+  public void abort() {
+    try {
+      super.close();
+    } catch (IOException ioe) {
+      LOG.warn("Unable to abort file " + tmpFile, ioe);
+    }
+    if (!tmpFile.delete()) {
+      LOG.warn("Unable to delete tmp file during abort " + tmpFile);
+    }
+  }
+
+}

http://git-wip-us.apache.org/repos/asf/incubator-ratis/blob/7e71a2e0/ratis-common/src/main/java/org/apache/ratis/util/AutoCloseableLock.java
----------------------------------------------------------------------
diff --git a/ratis-common/src/main/java/org/apache/ratis/util/AutoCloseableLock.java b/ratis-common/src/main/java/org/apache/ratis/util/AutoCloseableLock.java
new file mode 100644
index 0000000..489b5cd
--- /dev/null
+++ b/ratis-common/src/main/java/org/apache/ratis/util/AutoCloseableLock.java
@@ -0,0 +1,57 @@
+/**
+ * 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.ratis.util;
+
+import java.util.concurrent.atomic.AtomicBoolean;
+import java.util.concurrent.locks.Lock;
+
+/**
+ * Wrap a lock with the {@link AutoCloseable} interface
+ * so that the {@link #close()} method will unlock the lock.
+ */
+public class AutoCloseableLock implements AutoCloseable {
+  /**
+   * Acquire the given lock and then wrap it with {@link AutoCloseableLock}
+   * so that the given lock can be released by calling {@link #close()},
+   * or by using a {@code try}-with-resources statement as shown below.
+   *
+   * <pre> {@code
+   * try(AutoCloseableLock acl = AutoCloseableLock.acquire(lock)) {
+   *   ...
+   * }}</pre>
+   */
+  public static AutoCloseableLock acquire(final Lock lock) {
+    lock.lock();
+    return new AutoCloseableLock(lock);
+  }
+
+  private final Lock underlying;
+  private final AtomicBoolean closed = new AtomicBoolean(false);
+
+  private AutoCloseableLock(Lock underlying) {
+    this.underlying = underlying;
+  }
+
+  /** Unlock the underlying lock.  This method is idempotent. */
+  @Override
+  public void close() {
+    if (closed.compareAndSet(false, true)) {
+      underlying.unlock();
+    }
+  }
+}

http://git-wip-us.apache.org/repos/asf/incubator-ratis/blob/7e71a2e0/ratis-common/src/main/java/org/apache/ratis/util/CheckedFunction.java
----------------------------------------------------------------------
diff --git a/ratis-common/src/main/java/org/apache/ratis/util/CheckedFunction.java b/ratis-common/src/main/java/org/apache/ratis/util/CheckedFunction.java
new file mode 100644
index 0000000..4badc66
--- /dev/null
+++ b/ratis-common/src/main/java/org/apache/ratis/util/CheckedFunction.java
@@ -0,0 +1,30 @@
+/**
+ * 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.ratis.util;
+
+import java.util.function.Function;
+
+/** Function with a throws-clause. */
+@FunctionalInterface
+public interface CheckedFunction<INPUT, OUTPUT, THROWABLE extends Throwable> {
+  /**
+   * The same as {@link Function#apply(Object)}
+   * except that this method is declared with a throws-clause.
+   */
+  OUTPUT apply(INPUT input) throws THROWABLE;
+}

http://git-wip-us.apache.org/repos/asf/incubator-ratis/blob/7e71a2e0/ratis-common/src/main/java/org/apache/ratis/util/CheckedRunnable.java
----------------------------------------------------------------------
diff --git a/ratis-common/src/main/java/org/apache/ratis/util/CheckedRunnable.java b/ratis-common/src/main/java/org/apache/ratis/util/CheckedRunnable.java
new file mode 100644
index 0000000..b6e90b9
--- /dev/null
+++ b/ratis-common/src/main/java/org/apache/ratis/util/CheckedRunnable.java
@@ -0,0 +1,28 @@
+/**
+ * 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.ratis.util;
+
+/** Runnable with a throws-clause. */
+@FunctionalInterface
+public interface CheckedRunnable<THROWABLE extends Throwable> {
+  /**
+   * The same as {@link Runnable#run()}
+   * except that this method is declared with a throws-clause.
+   */
+  void run() throws THROWABLE;
+}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/incubator-ratis/blob/7e71a2e0/ratis-common/src/main/java/org/apache/ratis/util/CodeInjectionForTesting.java
----------------------------------------------------------------------
diff --git a/ratis-common/src/main/java/org/apache/ratis/util/CodeInjectionForTesting.java b/ratis-common/src/main/java/org/apache/ratis/util/CodeInjectionForTesting.java
new file mode 100644
index 0000000..60b5daf
--- /dev/null
+++ b/ratis-common/src/main/java/org/apache/ratis/util/CodeInjectionForTesting.java
@@ -0,0 +1,67 @@
+/**
+ * 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.ratis.util;
+
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.util.Arrays;
+import java.util.Map;
+import java.util.concurrent.ConcurrentHashMap;
+
+/** Inject code for testing. */
+public class CodeInjectionForTesting {
+  public static final Logger LOG = LoggerFactory.getLogger(CodeInjectionForTesting.class);
+
+  /** Code to be injected. */
+  public interface Code {
+    Logger LOG = CodeInjectionForTesting.LOG;
+
+    /**
+     * Execute the injected code for testing.
+     * @param localId the id of the local peer
+     * @param remoteId the id of the remote peer if handling a request
+     * @param args other possible args
+     * @return if the injected code is executed
+     */
+    boolean execute(String localId, String remoteId, Object... args);
+  }
+
+  private static final Map<String, Code> INJECTION_POINTS
+      = new ConcurrentHashMap<>();
+
+  /** Put an injection point. */
+  public static void put(String injectionPoint, Code code) {
+    LOG.debug("put: {}, {}", injectionPoint, code);
+    INJECTION_POINTS.put(injectionPoint, code);
+  }
+
+  /** Execute the injected code, if there is any. */
+  public static boolean execute(String injectionPoint, String localId,
+      String remoteId, Object... args) {
+    final Code code = INJECTION_POINTS.get(injectionPoint);
+    if (code == null) {
+      return false;
+    }
+    if (LOG.isDebugEnabled()) {
+      LOG.debug("execute: {}, {}, localId={}, remoteId={}, args={}",
+          injectionPoint, code, localId, remoteId, Arrays.toString(args));
+    }
+    return code.execute(localId, remoteId, args);
+  }
+}

http://git-wip-us.apache.org/repos/asf/incubator-ratis/blob/7e71a2e0/ratis-common/src/main/java/org/apache/ratis/util/Daemon.java
----------------------------------------------------------------------
diff --git a/ratis-common/src/main/java/org/apache/ratis/util/Daemon.java b/ratis-common/src/main/java/org/apache/ratis/util/Daemon.java
new file mode 100644
index 0000000..1ef95ae
--- /dev/null
+++ b/ratis-common/src/main/java/org/apache/ratis/util/Daemon.java
@@ -0,0 +1,35 @@
+/**
+ * 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.ratis.util;
+
+public class Daemon extends Thread {
+  {
+    setDaemon(true);
+  }
+
+  /** Construct a daemon thread. */
+  public Daemon() {
+    super();
+  }
+
+  /** Construct a daemon thread with the given runnable. */
+  public Daemon(Runnable runnable) {
+    super(runnable);
+    this.setName(runnable.toString());
+  }
+}

http://git-wip-us.apache.org/repos/asf/incubator-ratis/blob/7e71a2e0/ratis-common/src/main/java/org/apache/ratis/util/ExitUtils.java
----------------------------------------------------------------------
diff --git a/ratis-common/src/main/java/org/apache/ratis/util/ExitUtils.java b/ratis-common/src/main/java/org/apache/ratis/util/ExitUtils.java
new file mode 100644
index 0000000..4404344
--- /dev/null
+++ b/ratis-common/src/main/java/org/apache/ratis/util/ExitUtils.java
@@ -0,0 +1,98 @@
+/**
+ * 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.ratis.util;
+
+import org.slf4j.Logger;
+
+/** Facilitates hooking process termination for tests and debugging. */
+public class ExitUtils {
+  public static class ExitException extends RuntimeException {
+    private static final long serialVersionUID = 1L;
+
+    public final int status;
+
+    public ExitException(int status, String message, Throwable throwable) {
+      super(message, throwable);
+      this.status = status;
+    }
+  }
+
+  private static volatile boolean systemExitDisabled = false;
+  private static volatile ExitException firstExitException;
+
+  /**
+   * @return the first {@link ExitException} thrown, or null if none thrown yet.
+   */
+  public static ExitException getFirstExitException() {
+    return firstExitException;
+  }
+
+  /**
+   * Reset the tracking of process termination.
+   * This is useful when some tests expect an exit but the others do not.
+   */
+  public static void resetFirstExitException() {
+    firstExitException = null;
+  }
+
+  /** @return true if {@link #terminate(int, String, Throwable, Logger)} has been invoked. */
+  public static boolean isTerminated() {
+    // Either this member is set or System.exit is actually invoked.
+    return firstExitException != null;
+  }
+
+  /** Disable the use of {@link System#exit(int)} for testing. */
+  public static void disableSystemExit() {
+    systemExitDisabled = true;
+  }
+
+  /**
+   * Terminate the current process. Note that terminate is the *only* method
+   * that should be used to terminate the daemon processes.
+   *
+   * @param status Exit status
+   * @param message message used to create the {@code ExitException}
+   * @throws ExitException if System.exit is disabled for test purposes
+   */
+  public static void terminate(
+      int status, String message, Throwable throwable, Logger log)
+      throws ExitException {
+    if (log != null) {
+      final String s = "Terminating with exit status " + status + ": " + message;
+      if (status == 0) {
+        log.info(s, throwable);
+      } else {
+        log.error(s, throwable);
+      }
+    }
+
+    if (!systemExitDisabled) {
+      System.exit(status);
+    }
+
+    final ExitException ee = new ExitException(status, message, throwable);
+    if (firstExitException == null) {
+      firstExitException = ee;
+    }
+    throw ee;
+  }
+
+  public static void terminate(int status, String message, Logger log) {
+    terminate(status, message, null, log);
+  }
+}

http://git-wip-us.apache.org/repos/asf/incubator-ratis/blob/7e71a2e0/ratis-common/src/main/java/org/apache/ratis/util/FileUtils.java
----------------------------------------------------------------------
diff --git a/ratis-common/src/main/java/org/apache/ratis/util/FileUtils.java b/ratis-common/src/main/java/org/apache/ratis/util/FileUtils.java
new file mode 100644
index 0000000..5dc509d
--- /dev/null
+++ b/ratis-common/src/main/java/org/apache/ratis/util/FileUtils.java
@@ -0,0 +1,207 @@
+/**
+ * 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.ratis.util;
+
+import org.apache.ratis.io.nativeio.NativeIO;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.io.File;
+import java.io.FileOutputStream;
+import java.io.IOException;
+import java.net.URI;
+import java.net.URISyntaxException;
+import java.nio.file.Files;
+
+public class FileUtils {
+  public static final Logger LOG = LoggerFactory.getLogger(FileUtils.class);
+
+  public static void truncateFile(File f, long target) throws IOException {
+    try (FileOutputStream out = new FileOutputStream(f, true)) {
+      out.getChannel().truncate(target);
+    }
+  }
+
+  public static void deleteFile(File f) throws IOException {
+    try {
+      Files.delete(f.toPath());
+    } catch (IOException e) {
+      LOG.warn("Could not delete " + f);
+      throw e;
+    }
+  }
+
+  /**
+   * Delete a directory and all its contents.  If
+   * we return false, the directory may be partially-deleted.
+   * (1) If dir is symlink to a file, the symlink is deleted. The file pointed
+   *     to by the symlink is not deleted.
+   * (2) If dir is symlink to a directory, symlink is deleted. The directory
+   *     pointed to by symlink is not deleted.
+   * (3) If dir is a normal file, it is deleted.
+   * (4) If dir is a normal directory, then dir and all its contents recursively
+   *     are deleted.
+   */
+  public static boolean fullyDelete(final File dir) {
+    if (deleteImpl(dir, false)) {
+      // dir is (a) normal file, (b) symlink to a file, (c) empty directory or
+      // (d) symlink to a directory
+      return true;
+    }
+    // handle nonempty directory deletion
+    return fullyDeleteContents(dir) && deleteImpl(dir, true);
+  }
+
+  private static boolean deleteImpl(final File f, final boolean doLog) {
+    if (f == null) {
+      LOG.warn("null file argument.");
+      return false;
+    }
+    final boolean wasDeleted = f.delete();
+    if (wasDeleted) {
+      return true;
+    }
+    final boolean ex = f.exists();
+    if (doLog && ex) {
+      LOG.warn("Failed to delete file or dir ["
+          + f.getAbsolutePath() + "]: it still exists.");
+    }
+    return !ex;
+  }
+
+  /**
+   * Delete the contents of a directory, not the directory itself.  If
+   * we return false, the directory may be partially-deleted.
+   * If dir is a symlink to a directory, all the contents of the actual
+   * directory pointed to by dir will be deleted.
+   */
+  private static boolean fullyDeleteContents(final File dir) {
+    boolean deletionSucceeded = true;
+    final File[] contents = dir.listFiles();
+    if (contents != null) {
+      for (File content : contents) {
+        if (content.isFile()) {
+          if (!deleteImpl(content, true)) {
+            deletionSucceeded = false;
+          }
+        } else {
+          // Either directory or symlink to another directory.
+          // Try deleting the directory as this might be a symlink
+          if (deleteImpl(content, false)) {
+            // this was indeed a symlink or an empty directory
+            continue;
+          }
+          // if not an empty directory or symlink let
+          // fullyDelete handle it.
+          if (!fullyDelete(content)) {
+            deletionSucceeded = false;
+            // continue deletion of other files/dirs under dir
+          }
+        }
+      }
+    }
+    return deletionSucceeded;
+  }
+
+  /**
+   * Interprets the passed string as a URI. In case of error it
+   * assumes the specified string is a file.
+   *
+   * @param s the string to interpret
+   * @return the resulting URI
+   */
+  public static URI stringAsURI(String s) throws IOException {
+    URI u = null;
+    // try to make a URI
+    try {
+      u = new URI(s);
+    } catch (URISyntaxException e){
+      LOG.error("Syntax error in URI " + s
+          + ". Please check hdfs configuration.", e);
+    }
+
+    // if URI is null or scheme is undefined, then assume it's file://
+    if(u == null || u.getScheme() == null){
+      LOG.warn("Path " + s + " should be specified as a URI "
+          + "in configuration files. Please update configuration.");
+      u = fileAsURI(new File(s));
+    }
+    return u;
+  }
+
+  /**
+   * Converts the passed File to a URI. This method trims the trailing slash if
+   * one is appended because the underlying file is in fact a directory that
+   * exists.
+   *
+   * @param f the file to convert
+   * @return the resulting URI
+   */
+  public static URI fileAsURI(File f) throws IOException {
+    URI u = f.getCanonicalFile().toURI();
+
+    // trim the trailing slash, if it's present
+    if (u.getPath().endsWith("/")) {
+      String uriAsString = u.toString();
+      try {
+        u = new URI(uriAsString.substring(0, uriAsString.length() - 1));
+      } catch (URISyntaxException e) {
+        throw new IOException(e);
+      }
+    }
+    return u;
+  }
+
+  /**
+   * A wrapper for {@link File#listFiles()}. This java.io API returns null
+   * when a dir is not a directory or for any I/O error. Instead of having
+   * null check everywhere File#listFiles() is used, we will add utility API
+   * to get around this problem. For the majority of cases where we prefer
+   * an IOException to be thrown.
+   * @param dir directory for which listing should be performed
+   * @return list of files or empty list
+   * @exception IOException for invalid directory or for a bad disk.
+   */
+  public static File[] listFiles(File dir) throws IOException {
+    File[] files = dir.listFiles();
+    if(files == null) {
+      throw new IOException("Invalid directory or I/O error occurred for dir: "
+          + dir.toString());
+    }
+    return files;
+  }
+
+  /**
+   * Platform independent implementation for {@link File#canWrite()}
+   * @param f input file
+   * @return On Unix, same as {@link File#canWrite()}
+   *         On Windows, true if process has write access on the path
+   */
+  public static boolean canWrite(File f) {
+    if (RaftUtils.WINDOWS) {
+      try {
+        return NativeIO.Windows.access(f.getCanonicalPath(),
+            NativeIO.Windows.AccessRight.ACCESS_WRITE);
+      } catch (IOException e) {
+        return false;
+      }
+    } else {
+      return f.canWrite();
+    }
+  }
+}



[29/54] [abbrv] incubator-ratis git commit: Renamed the packages from raft to ratis in preperation for Apache Incubation - Moved all java packages from org.apache.raft to org.apache.ratis. - Moved native package to org_apache_ratis, and native lib to l

Posted by ji...@apache.org.
http://git-wip-us.apache.org/repos/asf/incubator-ratis/blob/7e71a2e0/ratis-common/src/main/java/org/apache/ratis/util/LifeCycle.java
----------------------------------------------------------------------
diff --git a/ratis-common/src/main/java/org/apache/ratis/util/LifeCycle.java b/ratis-common/src/main/java/org/apache/ratis/util/LifeCycle.java
new file mode 100644
index 0000000..b1f3fa5
--- /dev/null
+++ b/ratis-common/src/main/java/org/apache/ratis/util/LifeCycle.java
@@ -0,0 +1,210 @@
+/**
+ * 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.ratis.util;
+
+import com.google.common.base.Preconditions;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.util.*;
+import java.util.concurrent.atomic.AtomicReference;
+
+/**
+ * The life cycle of a machine.
+ * <pre>
+ *   -------------------------------------------------
+ *  |        --------------------------------         |
+ *  |       |     ------------------------   |        |
+ *  |       |    |                        |  |        |
+ *  |       |  PAUSED <---- PAUSING----   |  |        |
+ *  |       |    |          ^     |    |  |  |        |
+ *  |       |    V          |     |    V  V  V        V
+ * NEW --> STARTING --> RUNNING --|--> CLOSING --> [CLOSED]
+ *  ^       |    |          |     |       ^
+ *  |       |    |          V     V       |
+ *   -------      -------> EXCEPTION -----
+ * </pre>
+ * Note that there is no transition from PAUSING to CLOSING.
+ */
+public class LifeCycle {
+  public static final Logger LOG = LoggerFactory.getLogger(LifeCycle.class);
+
+  /** The states in the life cycle. */
+  public enum State {
+    /** The machine is newly created and holds zero resource. */
+    NEW,
+    /** The machine is starting and does not yet provide any service. */
+    STARTING,
+    /** The machine is running and providing service. */
+    RUNNING,
+    /** The machine is pausing and stopping providing service. */
+    PAUSING,
+    /** The machine is paused and does not provide any service. */
+    PAUSED,
+    /** The machine catches an internal exception so that it must be closed. */
+    EXCEPTION,
+    /** The machine is closing, stopping providing service and releasing resources. */
+    CLOSING,
+    /** The machine is closed, a final state. */
+    CLOSED;
+
+    private static final Map<State, List<State>> PREDECESSORS;
+
+    /** Does this object equal to one of the given states? */
+    public boolean isOneOf(State... states) {
+      for(State e : states) {
+        if (e == this) {
+          return true;
+        }
+      }
+      return false;
+    }
+
+    static void put(State key, Map<State, List<State>> map, State... values) {
+      map.put(key, Collections.unmodifiableList(Arrays.asList(values)));
+    }
+
+    static {
+      final Map<State, List<State>> predecessors = new EnumMap<>(State.class);
+      put(NEW,       predecessors, STARTING);
+      put(STARTING,  predecessors, NEW, PAUSED);
+      put(RUNNING,   predecessors, STARTING);
+      put(PAUSING,   predecessors, RUNNING);
+      put(PAUSED,    predecessors, PAUSING);
+      put(EXCEPTION, predecessors, STARTING, PAUSING, RUNNING);
+      put(CLOSING,   predecessors, STARTING, RUNNING, PAUSING, PAUSED, EXCEPTION);
+      put(CLOSED,    predecessors, NEW, CLOSING);
+
+      PREDECESSORS = Collections.unmodifiableMap(predecessors);
+    }
+
+    /** Is the given transition valid? */
+    static boolean isValid(State from, State to) {
+      return PREDECESSORS.get(to).contains(from);
+    }
+
+    /** Validate the given transition. */
+    static void validate(Object name, State from, State to) {
+      LOG.debug("{}: {} -> {}", name, from, to);
+      if (LOG.isTraceEnabled()) {
+        LOG.trace("TRACE", new Throwable());
+      }
+
+      Preconditions.checkState(isValid(from, to),
+          "ILLEGAL TRANSITION: In %s, %s -> %s", name, from, to);
+    }
+  }
+
+  private volatile String name;
+  private final AtomicReference<State> current = new AtomicReference<>(State.NEW);
+
+  public LifeCycle(Object name) {
+    this.name = name.toString();
+    LOG.debug("{}: {}", name, current);
+  }
+
+  public void setName(String name) {
+    this.name = name;
+  }
+
+  /** Transition from the current state to the given state. */
+  public void transition(final State to) {
+    final State from = current.getAndSet(to);
+    State.validate(name, from, to);
+  }
+
+  /**
+   * If the current state is equal to the specified from state,
+   * then transition to the give to state; otherwise, make no change.
+   *
+   * @return true iff the current state is equal to the specified from state.
+   */
+  public boolean compareAndTransition(final State from, final State to) {
+    if (current.compareAndSet(from, to)) {
+      State.validate(name, from, to);
+      return true;
+    }
+    return false;
+  }
+
+  /** @return the current state. */
+  public State getCurrentState() {
+    return current.get();
+  }
+
+  /** Assert if the current state equals to one of the expected states. */
+  public void assertCurrentState(State... expected) {
+    final State c = getCurrentState();
+    if (!c.isOneOf(expected)) {
+      throw new IllegalStateException("STATE MISMATCHED: In " + name
+          + ", current state " + c + " is not one of the expected states "
+          + Arrays.toString(expected));
+    }
+  }
+
+  @Override
+  public String toString() {
+    return name + ":" + getCurrentState();
+  }
+
+  /** Run the given start method and transition the current state accordingly. */
+  public <T extends Throwable> void startAndTransition(
+      CheckedRunnable<T> startImpl, Class<? extends Throwable>... exceptionClasses)
+      throws T {
+    transition(State.STARTING);
+    try {
+      startImpl.run();
+      transition(State.RUNNING);
+    } catch (Throwable t) {
+      transition(RaftUtils.isInstance(t, exceptionClasses)?
+          State.NEW: State.EXCEPTION);
+      throw t;
+    }
+  }
+
+
+  /**
+   * Check the current state and, if applicable, run the given close method.
+   * This method can be called multiple times
+   * while the given close method will only be executed at most once.
+   */
+  public <T extends Throwable> void checkStateAndClose(
+      CheckedRunnable<T> closeImpl) throws T {
+    if (compareAndTransition(State.NEW, State.CLOSED)) {
+      return;
+    }
+
+    for(;;) {
+      final State c = getCurrentState();
+      if (c.isOneOf(State.CLOSING, State.CLOSED)) {
+        return; //already closing or closed.
+      }
+
+      if (compareAndTransition(c, State.CLOSING)) {
+        try {
+          closeImpl.run();
+        } finally {
+          transition(State.CLOSED);
+        }
+        return;
+      }
+
+      // lifecycle state is changed, retry.
+    }
+  }
+}

http://git-wip-us.apache.org/repos/asf/incubator-ratis/blob/7e71a2e0/ratis-common/src/main/java/org/apache/ratis/util/MD5FileUtil.java
----------------------------------------------------------------------
diff --git a/ratis-common/src/main/java/org/apache/ratis/util/MD5FileUtil.java b/ratis-common/src/main/java/org/apache/ratis/util/MD5FileUtil.java
new file mode 100644
index 0000000..7218790
--- /dev/null
+++ b/ratis-common/src/main/java/org/apache/ratis/util/MD5FileUtil.java
@@ -0,0 +1,177 @@
+/**
+ * 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.ratis.util;
+
+import com.google.common.base.Charsets;
+
+import org.apache.ratis.io.MD5Hash;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.io.*;
+import java.security.DigestInputStream;
+import java.security.MessageDigest;
+import java.util.regex.Matcher;
+import java.util.regex.Pattern;
+
+public abstract class MD5FileUtil {
+  public static final Logger LOG = LoggerFactory.getLogger(MD5FileUtil.class);
+
+  // TODO: we should provide something like Hadoop's checksum fs for the local filesystem
+  // so that individual state machines do not have to deal with checksumming/corruption prevention.
+  // Keep the checksum and data in the same block format instead of individual files.
+
+  public static final String MD5_SUFFIX = ".md5";
+  private static final Pattern LINE_REGEX =
+      Pattern.compile("([0-9a-f]{32}) [ *](.+)");
+
+  /**
+   * Verify that the previously saved md5 for the given file matches
+   * expectedMd5.
+   */
+  public static void verifySavedMD5(File dataFile, MD5Hash expectedMD5)
+      throws IOException {
+    MD5Hash storedHash = readStoredMd5ForFile(dataFile);
+    // Check the hash itself
+    if (!expectedMD5.equals(storedHash)) {
+      throw new IOException(
+          "File " + dataFile + " did not match stored MD5 checksum " +
+              " (stored: " + storedHash + ", computed: " + expectedMD5);
+    }
+  }
+
+  /**
+   * Read the md5 file stored alongside the given data file and match the md5
+   * file content.
+   * @param md5File the file containing data
+   * @return a matcher with two matched groups where group(1) is the md5 string
+   *         and group(2) is the data file path.
+   */
+  private static Matcher readStoredMd5(File md5File) throws IOException {
+    BufferedReader reader =
+        new BufferedReader(new InputStreamReader(new FileInputStream(
+            md5File), Charsets.UTF_8));
+    String md5Line;
+    try {
+      md5Line = reader.readLine();
+      if (md5Line == null) { md5Line = ""; }
+      md5Line = md5Line.trim();
+    } catch (IOException ioe) {
+      throw new IOException("Error reading md5 file at " + md5File, ioe);
+    } finally {
+      RaftUtils.cleanup(LOG, reader);
+    }
+
+    Matcher matcher = LINE_REGEX.matcher(md5Line);
+    if (!matcher.matches()) {
+      throw new IOException("Invalid MD5 file " + md5File + ": the content \""
+          + md5Line + "\" does not match the expected pattern.");
+    }
+    return matcher;
+  }
+
+  /**
+   * Read the md5 checksum stored alongside the given data file.
+   * @param dataFile the file containing data
+   * @return the checksum stored in dataFile.md5
+   */
+  public static MD5Hash readStoredMd5ForFile(File dataFile) throws IOException {
+    final File md5File = getDigestFileForFile(dataFile);
+    if (!md5File.exists()) {
+      return null;
+    }
+
+    final Matcher matcher = readStoredMd5(md5File);
+    String storedHash = matcher.group(1);
+    File referencedFile = new File(matcher.group(2));
+
+    // Sanity check: Make sure that the file referenced in the .md5 file at
+    // least has the same name as the file we expect
+    if (!referencedFile.getName().equals(dataFile.getName())) {
+      throw new IOException(
+          "MD5 file at " + md5File + " references file named " +
+              referencedFile.getName() + " but we expected it to reference " +
+              dataFile);
+    }
+    return new MD5Hash(storedHash);
+  }
+
+  /**
+   * Read dataFile and compute its MD5 checksum.
+   */
+  public static MD5Hash computeMd5ForFile(File dataFile) throws IOException {
+    InputStream in = new FileInputStream(dataFile);
+    try {
+      MessageDigest digester = MD5Hash.getDigester();
+      DigestInputStream dis = new DigestInputStream(in, digester);
+      RaftUtils.readFully(dis, 128*1024);
+
+      return new MD5Hash(digester.digest());
+    } finally {
+      RaftUtils.cleanup(LOG, in);
+    }
+  }
+
+  /**
+   * Save the ".md5" file that lists the md5sum of another file.
+   * @param dataFile the original file whose md5 was computed
+   * @param digest the computed digest
+   */
+  public static void saveMD5File(File dataFile, MD5Hash digest)
+      throws IOException {
+    final String digestString = StringUtils.bytes2HexString(digest.getDigest());
+    saveMD5File(dataFile, digestString);
+  }
+
+  private static void saveMD5File(File dataFile, String digestString)
+      throws IOException {
+    File md5File = getDigestFileForFile(dataFile);
+    String md5Line = digestString + " *" + dataFile.getName() + "\n";
+
+    AtomicFileOutputStream afos = new AtomicFileOutputStream(md5File);
+    afos.write(md5Line.getBytes(Charsets.UTF_8));
+    afos.close();
+
+    if (LOG.isDebugEnabled()) {
+      LOG.debug("Saved MD5 " + digestString + " to " + md5File);
+    }
+  }
+
+  public static void renameMD5File(File oldDataFile, File newDataFile)
+      throws IOException {
+    final File fromFile = getDigestFileForFile(oldDataFile);
+    if (!fromFile.exists()) {
+      throw new FileNotFoundException(fromFile + " does not exist.");
+    }
+
+    final String digestString = readStoredMd5(fromFile).group(1);
+    saveMD5File(newDataFile, digestString);
+
+    if (!fromFile.delete()) {
+      LOG.warn("deleting  " + fromFile.getAbsolutePath() + " FAILED");
+    }
+  }
+
+  /**
+   * @return a reference to the file with .md5 suffix that will
+   * contain the md5 checksum for the given data file.
+   */
+  public static File getDigestFileForFile(File file) {
+    return new File(file.getParentFile(), file.getName() + MD5_SUFFIX);
+  }
+}

http://git-wip-us.apache.org/repos/asf/incubator-ratis/blob/7e71a2e0/ratis-common/src/main/java/org/apache/ratis/util/NativeCodeLoader.java
----------------------------------------------------------------------
diff --git a/ratis-common/src/main/java/org/apache/ratis/util/NativeCodeLoader.java b/ratis-common/src/main/java/org/apache/ratis/util/NativeCodeLoader.java
new file mode 100644
index 0000000..94d4e07
--- /dev/null
+++ b/ratis-common/src/main/java/org/apache/ratis/util/NativeCodeLoader.java
@@ -0,0 +1,70 @@
+/**
+ * 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.ratis.util;
+
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+/**
+ * A helper to load the native ratis code i.e. libratis.so.
+ * This handles the fallback to either the bundled libratis-Linux-i386-32.so
+ * or the default java implementations where appropriate.
+ */
+public final class NativeCodeLoader {
+
+  private static final Logger LOG = LoggerFactory.getLogger(NativeCodeLoader.class);
+
+  private static boolean nativeCodeLoaded = false;
+
+  static {
+    // Try to load native ratis library and set fallback flag appropriately
+    LOG.debug("Trying to load the custom-built native-ratis library...");
+    try {
+      System.loadLibrary("ratis");
+      LOG.debug("Loaded the native-ratis library");
+      nativeCodeLoaded = true;
+    } catch (Throwable t) {
+      // Ignore failure to load
+      if(LOG.isDebugEnabled()) {
+        LOG.debug("Failed to load native-ratis with error: " + t);
+        LOG.debug("java.library.path=" +
+            System.getProperty("java.library.path"));
+      }
+    }
+
+    if (!nativeCodeLoaded) {
+      LOG.warn("Unable to load native-ratis library for your platform... " +
+               "using builtin-java classes where applicable");
+    }
+  }
+
+  private NativeCodeLoader() {}
+
+  /**
+   * Check if native-ratis code is loaded for this platform.
+   *
+   * @return <code>true</code> if native-ratis is loaded,
+   *         else <code>false</code>
+   */
+  public static boolean isNativeCodeLoaded() {
+    return nativeCodeLoaded;
+  }
+
+  public static native String getLibraryName();
+}

http://git-wip-us.apache.org/repos/asf/incubator-ratis/blob/7e71a2e0/ratis-common/src/main/java/org/apache/ratis/util/NativeCrc32.java
----------------------------------------------------------------------
diff --git a/ratis-common/src/main/java/org/apache/ratis/util/NativeCrc32.java b/ratis-common/src/main/java/org/apache/ratis/util/NativeCrc32.java
new file mode 100644
index 0000000..902d0a7
--- /dev/null
+++ b/ratis-common/src/main/java/org/apache/ratis/util/NativeCrc32.java
@@ -0,0 +1,143 @@
+/**
+ * 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.ratis.util;
+
+import com.google.common.annotations.VisibleForTesting;
+
+import java.nio.ByteBuffer;
+
+import org.apache.ratis.protocol.ChecksumException;
+
+/**
+ * Wrapper around JNI support code to do checksum computation
+ * natively.
+ */
+class NativeCrc32 {
+  
+  /**
+   * Return true if the JNI-based native CRC extensions are available.
+   */
+  public static boolean isAvailable() {
+    if (System.getProperty("os.arch").toLowerCase().startsWith("sparc")) {
+      return false;
+    } else {
+      return NativeCodeLoader.isNativeCodeLoaded();
+    }
+  }
+
+  /**
+   * Verify the given buffers of data and checksums, and throw an exception
+   * if any checksum is invalid. The buffers given to this function should
+   * have their position initially at the start of the data, and their limit
+   * set at the end of the data. The position, limit, and mark are not
+   * modified.
+   * 
+   * @param bytesPerSum the chunk size (eg 512 bytes)
+   * @param checksumType the DataChecksum type constant (NULL is not supported)
+   * @param sums the DirectByteBuffer pointing at the beginning of the
+   *             stored checksums
+   * @param data the DirectByteBuffer pointing at the beginning of the
+   *             data to check
+   * @param basePos the position in the file where the data buffer starts 
+   * @param fileName the name of the file being verified
+   * @throws ChecksumException if there is an invalid checksum
+   */
+  public static void verifyChunkedSums(int bytesPerSum, int checksumType,
+      ByteBuffer sums, ByteBuffer data, String fileName, long basePos)
+      throws ChecksumException {
+    nativeComputeChunkedSums(bytesPerSum, checksumType,
+        sums, sums.position(),
+        data, data.position(), data.remaining(),
+        fileName, basePos, true);
+  }
+
+  public static void verifyChunkedSumsByteArray(int bytesPerSum,
+      int checksumType, byte[] sums, int sumsOffset, byte[] data,
+      int dataOffset, int dataLength, String fileName, long basePos)
+      throws ChecksumException {
+    nativeComputeChunkedSumsByteArray(bytesPerSum, checksumType,
+        sums, sumsOffset,
+        data, dataOffset, dataLength,
+        fileName, basePos, true);
+  }
+
+  public static void calculateChunkedSums(int bytesPerSum, int checksumType,
+      ByteBuffer sums, ByteBuffer data) {
+    nativeComputeChunkedSums(bytesPerSum, checksumType,
+        sums, sums.position(),
+        data, data.position(), data.remaining(),
+        "", 0, false);
+  }
+
+  public static void calculateChunkedSumsByteArray(int bytesPerSum,
+      int checksumType, byte[] sums, int sumsOffset, byte[] data,
+      int dataOffset, int dataLength) {
+    nativeComputeChunkedSumsByteArray(bytesPerSum, checksumType,
+        sums, sumsOffset,
+        data, dataOffset, dataLength,
+        "", 0, false);
+  }
+
+  /**
+   * Verify the given buffers of data and checksums, and throw an exception
+   * if any checksum is invalid. The buffers given to this function should
+   * have their position initially at the start of the data, and their limit
+   * set at the end of the data. The position, limit, and mark are not
+   * modified.  This method is retained only for backwards-compatibility with
+   * prior jar versions that need the corresponding JNI function.
+   *
+   * @param bytesPerSum the chunk size (eg 512 bytes)
+   * @param checksumType the DataChecksum type constant
+   * @param sums the DirectByteBuffer pointing at the beginning of the
+   *             stored checksums
+   * @param sumsOffset start offset in sums buffer
+   * @param data the DirectByteBuffer pointing at the beginning of the
+   *             data to check
+   * @param dataOffset start offset in data buffer
+   * @param dataLength length of data buffer
+   * @param fileName the name of the file being verified
+   * @param basePos the position in the file where the data buffer starts 
+   * @throws ChecksumException if there is an invalid checksum
+   * @deprecated use {@link #nativeComputeChunkedSums(int, int, ByteBuffer, int,
+   *   ByteBuffer, int, int, String, long, boolean)} instead
+   */
+  @Deprecated
+  @VisibleForTesting
+  static native void nativeVerifyChunkedSums(
+      int bytesPerSum, int checksumType,
+      ByteBuffer sums, int sumsOffset,
+      ByteBuffer data, int dataOffset, int dataLength,
+      String fileName, long basePos) throws ChecksumException;
+  
+  private static native void nativeComputeChunkedSums(
+      int bytesPerSum, int checksumType,
+      ByteBuffer sums, int sumsOffset,
+      ByteBuffer data, int dataOffset, int dataLength,
+      String fileName, long basePos, boolean verify);
+
+  private static native void nativeComputeChunkedSumsByteArray(
+      int bytesPerSum, int checksumType,
+      byte[] sums, int sumsOffset,
+      byte[] data, int dataOffset, int dataLength,
+      String fileName, long basePos, boolean verify);
+
+  // Copy the constants over from DataChecksum so that javah will pick them up
+  // and make them available in the native code header.
+  public static final int CHECKSUM_CRC32 = 1;  //DataChecksum.CHECKSUM_CRC32
+  public static final int CHECKSUM_CRC32C = 2; //DataChecksum.CHECKSUM_CRC32C
+}

http://git-wip-us.apache.org/repos/asf/incubator-ratis/blob/7e71a2e0/ratis-common/src/main/java/org/apache/ratis/util/NativeLibraryChecker.java
----------------------------------------------------------------------
diff --git a/ratis-common/src/main/java/org/apache/ratis/util/NativeLibraryChecker.java b/ratis-common/src/main/java/org/apache/ratis/util/NativeLibraryChecker.java
new file mode 100644
index 0000000..b32702b
--- /dev/null
+++ b/ratis-common/src/main/java/org/apache/ratis/util/NativeLibraryChecker.java
@@ -0,0 +1,64 @@
+/**
+ * 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.ratis.util;
+
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+public class NativeLibraryChecker {
+  public static final Logger LOG = LoggerFactory.getLogger(NativeLibraryChecker.class);
+
+  /**
+   * A tool to test native library availability,
+   */
+  public static void main(String[] args) {
+    String usage = "NativeLibraryChecker [-a|-h]\n"
+        + "  -a  use -a to check all libraries are available\n"
+        + "      by default just check ratis library (and\n"
+        + "      winutils.exe on Windows OS) is available\n"
+        + "      exit with error code 1 if check failed\n"
+        + "  -h  print this message\n";
+    if (args.length > 1 ||
+        (args.length == 1 &&
+            !(args[0].equals("-a") || args[0].equals("-h")))) {
+      System.err.println(usage);
+      ExitUtils.terminate(1, "Illegal arguments.", LOG);
+    }
+    if (args.length == 1) {
+      if (args[0].equals("-h")) {
+        System.out.println(usage);
+        return;
+      }
+    }
+    boolean nativeRatisLoaded = NativeCodeLoader.isNativeCodeLoaded();
+    String raftLibraryName = "";
+
+    if (nativeRatisLoaded) {
+      raftLibraryName = NativeCodeLoader.getLibraryName();
+    }
+
+    System.out.println("Native library checking:");
+    System.out.printf("raft:  %b %s%n", nativeRatisLoaded, raftLibraryName);
+
+    if (!nativeRatisLoaded) {
+      // return 1 to indicated check failed
+      ExitUtils.terminate(1, "Failed to load native library.", LOG);
+    }
+  }
+}

http://git-wip-us.apache.org/repos/asf/incubator-ratis/blob/7e71a2e0/ratis-common/src/main/java/org/apache/ratis/util/NetUtils.java
----------------------------------------------------------------------
diff --git a/ratis-common/src/main/java/org/apache/ratis/util/NetUtils.java b/ratis-common/src/main/java/org/apache/ratis/util/NetUtils.java
new file mode 100644
index 0000000..b6634b6
--- /dev/null
+++ b/ratis-common/src/main/java/org/apache/ratis/util/NetUtils.java
@@ -0,0 +1,151 @@
+/**
+ * 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.ratis.util;
+
+import com.google.common.base.Preconditions;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.net.InetAddress;
+import java.net.InetSocketAddress;
+import java.net.URI;
+import java.net.UnknownHostException;
+import java.util.Map;
+import java.util.concurrent.ConcurrentHashMap;
+
+public abstract class NetUtils {
+  public static final Logger LOG = LoggerFactory.getLogger(NetUtils.class);
+
+  public static abstract class StaticResolution {
+    /** Host -> resolved name */
+    private static final Map<String, String> hostToResolved = new ConcurrentHashMap<>();
+
+    /** Adds a static resolution for host. */
+    public static void put(String host, String resolvedName) {
+      hostToResolved.put(host, resolvedName);
+    }
+
+    /** @return the resolved name, or null if the host is not found. */
+    public static String get(String host) {
+      return hostToResolved.get(host);
+    }
+  }
+
+  public static InetSocketAddress newInetSocketAddress(String address) {
+    if (address.charAt(0) == '/') {
+      address = address.substring(1);
+    }
+    try {
+      return createSocketAddr(address);
+    } catch (Exception e) {
+      LOG.trace("", e);
+      return null;
+    }
+  }
+
+  /**
+   * Util method to build socket addr from either:
+   *   <host>:<port>
+   *   <fs>://<host>:<port>/<path>
+   */
+  public static InetSocketAddress createSocketAddr(String target) {
+    return createSocketAddr(target, -1);
+  }
+
+  /**
+   * Util method to build socket addr from either:
+   *   <host>
+   *   <host>:<port>
+   *   <fs>://<host>:<port>/<path>
+   */
+  public static InetSocketAddress createSocketAddr(String target, int defaultPort) {
+    return createSocketAddr(target, defaultPort, null);
+  }
+
+  /**
+   * Create an InetSocketAddress from the given target string and
+   * default port. If the string cannot be parsed correctly, the
+   * <code>configName</code> parameter is used as part of the
+   * exception message, allowing the user to better diagnose
+   * the misconfiguration.
+   *
+   * @param target a string of either "host" or "host:port"
+   * @param defaultPort the default port if <code>target</code> does not
+   *                    include a port number
+   * @param propertyName the name of the configuration from which
+   *                   <code>target</code> was loaded. This is used in the
+   *                   exception message in the case that parsing fails.
+   */
+  public static InetSocketAddress createSocketAddr(
+      String target, int defaultPort, String propertyName) {
+    final String helpText = propertyName == null? ""
+        : " (property '" + propertyName + "')";
+    Preconditions.checkNotNull(target, "Target address cannot be null.%s", helpText);
+
+    target = target.trim();
+    boolean hasScheme = target.contains("://");
+    final URI uri;
+    try {
+      uri = hasScheme ? URI.create(target) : URI.create("dummyscheme://"+target);
+    } catch (IllegalArgumentException e) {
+      throw new IllegalArgumentException(
+          "Invalid host:port authority: " + target + helpText, e);
+    }
+
+    final String host = uri.getHost();
+    int port = uri.getPort();
+    if (port == -1) {
+      port = defaultPort;
+    }
+    final String path = uri.getPath();
+
+    if (host == null || port < 0
+        || (!hasScheme && path != null && !path.isEmpty())) {
+      throw new IllegalArgumentException(
+          "Invalid host:port authority: " + target + helpText);
+    }
+    return createSocketAddrForHost(host, port);
+  }
+
+  /**
+   * Create a socket address with the given host and port.  The hostname
+   * might be replaced with another host that was set via
+   * {@link StaticResolution#put(String, String)}.
+   * @param host the hostname or IP use to instantiate the object
+   * @param port the port number
+   * @return InetSocketAddress
+   */
+  public static InetSocketAddress createSocketAddrForHost(String host, int port) {
+    String staticHost = StaticResolution.get(host);
+    String resolveHost = (staticHost != null) ? staticHost : host;
+
+    InetSocketAddress addr;
+    try {
+      InetAddress iaddr = InetAddress.getByName(resolveHost);
+      // if there is a static entry for the host, make the returned
+      // address look like the original given host
+      if (staticHost != null) {
+        iaddr = InetAddress.getByAddress(host, iaddr.getAddress());
+      }
+      addr = new InetSocketAddress(iaddr, port);
+    } catch (UnknownHostException e) {
+      addr = InetSocketAddress.createUnresolved(host, port);
+    }
+    return addr;
+  }
+}

http://git-wip-us.apache.org/repos/asf/incubator-ratis/blob/7e71a2e0/ratis-common/src/main/java/org/apache/ratis/util/PeerProxyMap.java
----------------------------------------------------------------------
diff --git a/ratis-common/src/main/java/org/apache/ratis/util/PeerProxyMap.java b/ratis-common/src/main/java/org/apache/ratis/util/PeerProxyMap.java
new file mode 100644
index 0000000..9466719
--- /dev/null
+++ b/ratis-common/src/main/java/org/apache/ratis/util/PeerProxyMap.java
@@ -0,0 +1,128 @@
+/**
+ * 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.ratis.util;
+
+import com.google.common.base.Preconditions;
+
+import org.apache.ratis.protocol.RaftPeer;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.io.Closeable;
+import java.io.IOException;
+import java.util.Map;
+import java.util.concurrent.ConcurrentHashMap;
+
+/** A map from peer id to peer and its proxy. */
+public class PeerProxyMap<PROXY extends Closeable> implements Closeable {
+  public static final Logger LOG = LoggerFactory.getLogger(PeerProxyMap.class);
+
+  /** Peer and its proxy. */
+  private class PeerAndProxy implements Closeable {
+    private final RaftPeer peer;
+    private volatile PROXY proxy = null;
+    private final LifeCycle lifeCycle;
+
+    PeerAndProxy(RaftPeer peer) {
+      this.peer = peer;
+      this.lifeCycle = new LifeCycle(peer);
+    }
+
+    RaftPeer getPeer() {
+      return peer;
+    }
+
+    PROXY getProxy() throws IOException {
+      if (proxy == null) {
+        synchronized (this) {
+          if (proxy == null) {
+            lifeCycle.startAndTransition(
+                () -> proxy = createProxy.apply(peer), IOException.class);
+          }
+        }
+      }
+      return proxy;
+    }
+
+    @Override
+    public synchronized void close() {
+      lifeCycle.checkStateAndClose(() -> {
+        if (proxy != null) {
+          try {
+            proxy.close();
+          } catch (IOException e) {
+            LOG.warn("Failed to close proxy for peer {}, proxy class: ",
+                peer, proxy.getClass());
+          }
+        }
+      });
+    }
+  }
+
+  private final Map<String, PeerAndProxy> peers = new ConcurrentHashMap<>();
+  private final Object resetLock = new Object();
+
+  private final CheckedFunction<RaftPeer, PROXY, IOException> createProxy;
+
+  public PeerProxyMap(CheckedFunction<RaftPeer, PROXY, IOException> createProxy) {
+    this.createProxy = createProxy;
+  }
+  public PeerProxyMap() {
+    this.createProxy = this::createProxyImpl;
+  }
+
+  public PROXY getProxy(String id) throws IOException {
+    PeerAndProxy p = peers.get(id);
+    if (p == null) {
+      synchronized (resetLock) {
+        p = peers.get(id);
+      }
+    }
+    Preconditions.checkNotNull(p, "Server %s not found; peers=%s",
+        id, peers.keySet());
+    return p.getProxy();
+  }
+
+  public void addPeers(Iterable<RaftPeer> newPeers) {
+    for(RaftPeer p : newPeers) {
+      peers.put(p.getId(), new PeerAndProxy(p));
+    }
+  }
+
+  public void putIfAbsent(RaftPeer p) {
+    peers.putIfAbsent(p.getId(), new PeerAndProxy(p));
+  }
+
+  public void resetProxy(String id) {
+    synchronized (resetLock) {
+      final PeerAndProxy pp = peers.remove(id);
+      final RaftPeer peer = pp.getPeer();
+      pp.close();
+      peers.put(id, new PeerAndProxy(peer));
+    }
+  }
+
+  public PROXY createProxyImpl(RaftPeer peer) throws IOException {
+    throw new UnsupportedOperationException();
+  }
+
+  @Override
+  public void close() {
+    peers.values().forEach(PeerAndProxy::close);
+  }
+}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/incubator-ratis/blob/7e71a2e0/ratis-common/src/main/java/org/apache/ratis/util/ProtoUtils.java
----------------------------------------------------------------------
diff --git a/ratis-common/src/main/java/org/apache/ratis/util/ProtoUtils.java b/ratis-common/src/main/java/org/apache/ratis/util/ProtoUtils.java
new file mode 100644
index 0000000..8dc822b
--- /dev/null
+++ b/ratis-common/src/main/java/org/apache/ratis/util/ProtoUtils.java
@@ -0,0 +1,147 @@
+/**
+ * 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.ratis.util;
+
+import java.io.IOException;
+import java.io.ObjectInputStream;
+import java.io.ObjectOutputStream;
+import java.util.Collection;
+import java.util.Iterator;
+import java.util.List;
+
+import org.apache.ratis.protocol.RaftPeer;
+import org.apache.ratis.shaded.com.google.protobuf.ByteString;
+import org.apache.ratis.shaded.com.google.protobuf.ServiceException;
+import org.apache.ratis.shaded.proto.RaftProtos.AppendEntriesReplyProto;
+import org.apache.ratis.shaded.proto.RaftProtos.LogEntryProto;
+import org.apache.ratis.shaded.proto.RaftProtos.RaftPeerProto;
+import org.apache.ratis.shaded.proto.RaftProtos.RaftRpcReplyProto;
+import org.apache.ratis.shaded.proto.RaftProtos.RaftRpcRequestProto;
+import org.apache.ratis.shaded.proto.RaftProtos.RequestVoteReplyProto;
+import org.apache.ratis.shaded.proto.RaftProtos.SMLogEntryProto;
+
+public class ProtoUtils {
+  public static ByteString toByteString(Object obj) {
+    final ByteString.Output byteOut = ByteString.newOutput();
+    try(final ObjectOutputStream objOut = new ObjectOutputStream(byteOut)) {
+      objOut.writeObject(obj);
+    } catch (IOException e) {
+      throw new IllegalStateException(
+          "Unexpected IOException when writing an object to a ByteString.", e);
+    }
+    return byteOut.toByteString();
+  }
+
+  public static Object toObject(ByteString bytes) {
+    try(final ObjectInputStream in = new ObjectInputStream(bytes.newInput())) {
+      return in.readObject();
+    } catch (IOException e) {
+      throw new IllegalStateException(
+          "Unexpected IOException when reading an object from a ByteString.", e);
+    } catch (ClassNotFoundException e) {
+      throw new IllegalStateException(e);
+    }
+  }
+
+  public static ByteString toByteString(byte[] bytes) {
+    return toByteString(bytes, 0, bytes.length);
+  }
+
+  public static ByteString toByteString(byte[] bytes, int offset, int size) {
+    // return singleton to reduce object allocation
+    return bytes.length == 0 ?
+        ByteString.EMPTY : ByteString.copyFrom(bytes, offset, size);
+  }
+
+  public static RaftPeerProto toRaftPeerProto(RaftPeer peer) {
+    RaftPeerProto.Builder builder = RaftPeerProto.newBuilder()
+        .setId(peer.getId());
+    if (peer.getAddress() != null) {
+      builder.setAddress(peer.getAddress());
+    }
+    return builder.build();
+  }
+
+  public static RaftPeer toRaftPeer(RaftPeerProto p) {
+    return new RaftPeer(p.getId(), p.getAddress());
+  }
+
+  public static RaftPeer[] toRaftPeerArray(List<RaftPeerProto> protos) {
+    final RaftPeer[] peers = new RaftPeer[protos.size()];
+    for (int i = 0; i < peers.length; i++) {
+      peers[i] = toRaftPeer(protos.get(i));
+    }
+    return peers;
+  }
+
+  public static Iterable<RaftPeerProto> toRaftPeerProtos(
+      final Collection<RaftPeer> peers) {
+    return () -> new Iterator<RaftPeerProto>() {
+      final Iterator<RaftPeer> i = peers.iterator();
+
+      @Override
+      public boolean hasNext() {
+        return i.hasNext();
+      }
+
+      @Override
+      public RaftPeerProto next() {
+        return toRaftPeerProto(i.next());
+      }
+    };
+  }
+
+  public static boolean isConfigurationLogEntry(LogEntryProto entry) {
+    return entry.getLogEntryBodyCase() ==
+        LogEntryProto.LogEntryBodyCase.CONFIGURATIONENTRY;
+  }
+
+  public static LogEntryProto toLogEntryProto(
+      SMLogEntryProto operation, long term, long index) {
+    return LogEntryProto.newBuilder().setTerm(term).setIndex(index)
+        .setSmLogEntry(operation)
+        .build();
+  }
+
+  public static IOException toIOException(ServiceException se) {
+    final Throwable t = se.getCause();
+    if (t == null) {
+      return new IOException(se);
+    }
+    return t instanceof IOException? (IOException)t : new IOException(se);
+  }
+
+  public static String toString(RaftRpcRequestProto proto) {
+    return proto.getRequestorId() + "->" + proto.getReplyId()
+        + "#" + proto.getSeqNum();
+  }
+
+  public static String toString(RaftRpcReplyProto proto) {
+    return proto.getRequestorId() + "<-" + proto.getReplyId()
+        + "#" + proto.getSeqNum() + ":"
+        + (proto.getSuccess()? "OK": "FAIL");
+  }
+  public static String toString(RequestVoteReplyProto proto) {
+    return toString(proto.getServerReply()) + "-t" + proto.getTerm();
+  }
+  public static String toString(AppendEntriesReplyProto proto) {
+    return toString(proto.getServerReply()) + "-t" + proto.getTerm()
+        + ", nextIndex=" + proto.getNextIndex()
+        + ", result: " + proto.getResult();
+  }
+}

http://git-wip-us.apache.org/repos/asf/incubator-ratis/blob/7e71a2e0/ratis-common/src/main/java/org/apache/ratis/util/PureJavaCrc32.java
----------------------------------------------------------------------
diff --git a/ratis-common/src/main/java/org/apache/ratis/util/PureJavaCrc32.java b/ratis-common/src/main/java/org/apache/ratis/util/PureJavaCrc32.java
new file mode 100644
index 0000000..d21b56a
--- /dev/null
+++ b/ratis-common/src/main/java/org/apache/ratis/util/PureJavaCrc32.java
@@ -0,0 +1,619 @@
+/**
+ * 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.ratis.util;
+
+import java.util.zip.Checksum;
+
+/**
+ * A pure-java implementation of the CRC32 checksum that uses
+ * the same polynomial as the built-in native CRC32.
+ *
+ * This is to avoid the JNI overhead for certain uses of Checksumming
+ * where many small pieces of data are checksummed in succession.
+ *
+ * The current version is ~10x to 1.8x as fast as Sun's native
+ * java.util.zip.CRC32 in Java 1.6
+ *
+ * @see java.util.zip.CRC32
+ */
+public class PureJavaCrc32 implements Checksum {
+
+  /** the current CRC value, bit-flipped */
+  private int crc;
+
+  /** Create a new PureJavaCrc32 object. */
+  public PureJavaCrc32() {
+    reset();
+  }
+
+  @Override
+  public long getValue() {
+    return (~crc) & 0xffffffffL;
+  }
+
+  @Override
+  public void reset() {
+    crc = 0xffffffff;
+  }
+
+  @Override
+  public void update(final byte[] b, final int offset, final int len) {
+    int localCrc = crc;
+
+    final int remainder = len & 0x7;
+    int i = offset;
+    for(final int end = offset + len - remainder; i < end; i += 8) {
+      final int x = localCrc
+          ^ ((((b[i  ] << 24) >>> 24) + ((b[i+1] << 24) >>> 16))
+           + (((b[i+2] << 24) >>> 8 ) +  (b[i+3] << 24)));
+
+      localCrc = ((T[((x << 24) >>> 24) + 0x700] ^ T[((x << 16) >>> 24) + 0x600])
+                ^ (T[((x <<  8) >>> 24) + 0x500] ^ T[ (x        >>> 24) + 0x400]))
+               ^ ((T[((b[i+4] << 24) >>> 24) + 0x300] ^ T[((b[i+5] << 24) >>> 24) + 0x200])
+                ^ (T[((b[i+6] << 24) >>> 24) + 0x100] ^ T[((b[i+7] << 24) >>> 24)]));
+    }
+
+    /* loop unroll - duff's device style */
+    switch(remainder) {
+      case 7: localCrc = (localCrc >>> 8) ^ T[((localCrc ^ b[i++]) << 24) >>> 24];
+      case 6: localCrc = (localCrc >>> 8) ^ T[((localCrc ^ b[i++]) << 24) >>> 24];
+      case 5: localCrc = (localCrc >>> 8) ^ T[((localCrc ^ b[i++]) << 24) >>> 24];
+      case 4: localCrc = (localCrc >>> 8) ^ T[((localCrc ^ b[i++]) << 24) >>> 24];
+      case 3: localCrc = (localCrc >>> 8) ^ T[((localCrc ^ b[i++]) << 24) >>> 24];
+      case 2: localCrc = (localCrc >>> 8) ^ T[((localCrc ^ b[i++]) << 24) >>> 24];
+      case 1: localCrc = (localCrc >>> 8) ^ T[((localCrc ^ b[i++]) << 24) >>> 24];
+      default:
+        /* nothing */
+    }
+    
+    // Publish crc out to object
+    crc = localCrc;
+  }
+
+  @Override
+  final public void update(int b) {
+    crc = (crc >>> 8) ^ T[(((crc ^ b) << 24) >>> 24)];
+  }
+
+  /*
+   * CRC-32 lookup tables generated by the polynomial 0xEDB88320.
+   * See also TestPureJavaCrc32.Table.
+   */
+  private static final int[] T = new int[] {
+    /* T8_0 */
+    0x00000000, 0x77073096, 0xEE0E612C, 0x990951BA, 
+    0x076DC419, 0x706AF48F, 0xE963A535, 0x9E6495A3, 
+    0x0EDB8832, 0x79DCB8A4, 0xE0D5E91E, 0x97D2D988, 
+    0x09B64C2B, 0x7EB17CBD, 0xE7B82D07, 0x90BF1D91, 
+    0x1DB71064, 0x6AB020F2, 0xF3B97148, 0x84BE41DE, 
+    0x1ADAD47D, 0x6DDDE4EB, 0xF4D4B551, 0x83D385C7, 
+    0x136C9856, 0x646BA8C0, 0xFD62F97A, 0x8A65C9EC, 
+    0x14015C4F, 0x63066CD9, 0xFA0F3D63, 0x8D080DF5, 
+    0x3B6E20C8, 0x4C69105E, 0xD56041E4, 0xA2677172, 
+    0x3C03E4D1, 0x4B04D447, 0xD20D85FD, 0xA50AB56B, 
+    0x35B5A8FA, 0x42B2986C, 0xDBBBC9D6, 0xACBCF940, 
+    0x32D86CE3, 0x45DF5C75, 0xDCD60DCF, 0xABD13D59, 
+    0x26D930AC, 0x51DE003A, 0xC8D75180, 0xBFD06116, 
+    0x21B4F4B5, 0x56B3C423, 0xCFBA9599, 0xB8BDA50F, 
+    0x2802B89E, 0x5F058808, 0xC60CD9B2, 0xB10BE924, 
+    0x2F6F7C87, 0x58684C11, 0xC1611DAB, 0xB6662D3D, 
+    0x76DC4190, 0x01DB7106, 0x98D220BC, 0xEFD5102A, 
+    0x71B18589, 0x06B6B51F, 0x9FBFE4A5, 0xE8B8D433, 
+    0x7807C9A2, 0x0F00F934, 0x9609A88E, 0xE10E9818, 
+    0x7F6A0DBB, 0x086D3D2D, 0x91646C97, 0xE6635C01, 
+    0x6B6B51F4, 0x1C6C6162, 0x856530D8, 0xF262004E, 
+    0x6C0695ED, 0x1B01A57B, 0x8208F4C1, 0xF50FC457, 
+    0x65B0D9C6, 0x12B7E950, 0x8BBEB8EA, 0xFCB9887C, 
+    0x62DD1DDF, 0x15DA2D49, 0x8CD37CF3, 0xFBD44C65, 
+    0x4DB26158, 0x3AB551CE, 0xA3BC0074, 0xD4BB30E2, 
+    0x4ADFA541, 0x3DD895D7, 0xA4D1C46D, 0xD3D6F4FB, 
+    0x4369E96A, 0x346ED9FC, 0xAD678846, 0xDA60B8D0, 
+    0x44042D73, 0x33031DE5, 0xAA0A4C5F, 0xDD0D7CC9, 
+    0x5005713C, 0x270241AA, 0xBE0B1010, 0xC90C2086, 
+    0x5768B525, 0x206F85B3, 0xB966D409, 0xCE61E49F, 
+    0x5EDEF90E, 0x29D9C998, 0xB0D09822, 0xC7D7A8B4, 
+    0x59B33D17, 0x2EB40D81, 0xB7BD5C3B, 0xC0BA6CAD, 
+    0xEDB88320, 0x9ABFB3B6, 0x03B6E20C, 0x74B1D29A, 
+    0xEAD54739, 0x9DD277AF, 0x04DB2615, 0x73DC1683, 
+    0xE3630B12, 0x94643B84, 0x0D6D6A3E, 0x7A6A5AA8, 
+    0xE40ECF0B, 0x9309FF9D, 0x0A00AE27, 0x7D079EB1, 
+    0xF00F9344, 0x8708A3D2, 0x1E01F268, 0x6906C2FE, 
+    0xF762575D, 0x806567CB, 0x196C3671, 0x6E6B06E7, 
+    0xFED41B76, 0x89D32BE0, 0x10DA7A5A, 0x67DD4ACC, 
+    0xF9B9DF6F, 0x8EBEEFF9, 0x17B7BE43, 0x60B08ED5, 
+    0xD6D6A3E8, 0xA1D1937E, 0x38D8C2C4, 0x4FDFF252, 
+    0xD1BB67F1, 0xA6BC5767, 0x3FB506DD, 0x48B2364B, 
+    0xD80D2BDA, 0xAF0A1B4C, 0x36034AF6, 0x41047A60, 
+    0xDF60EFC3, 0xA867DF55, 0x316E8EEF, 0x4669BE79, 
+    0xCB61B38C, 0xBC66831A, 0x256FD2A0, 0x5268E236, 
+    0xCC0C7795, 0xBB0B4703, 0x220216B9, 0x5505262F, 
+    0xC5BA3BBE, 0xB2BD0B28, 0x2BB45A92, 0x5CB36A04, 
+    0xC2D7FFA7, 0xB5D0CF31, 0x2CD99E8B, 0x5BDEAE1D, 
+    0x9B64C2B0, 0xEC63F226, 0x756AA39C, 0x026D930A, 
+    0x9C0906A9, 0xEB0E363F, 0x72076785, 0x05005713, 
+    0x95BF4A82, 0xE2B87A14, 0x7BB12BAE, 0x0CB61B38, 
+    0x92D28E9B, 0xE5D5BE0D, 0x7CDCEFB7, 0x0BDBDF21, 
+    0x86D3D2D4, 0xF1D4E242, 0x68DDB3F8, 0x1FDA836E, 
+    0x81BE16CD, 0xF6B9265B, 0x6FB077E1, 0x18B74777, 
+    0x88085AE6, 0xFF0F6A70, 0x66063BCA, 0x11010B5C, 
+    0x8F659EFF, 0xF862AE69, 0x616BFFD3, 0x166CCF45, 
+    0xA00AE278, 0xD70DD2EE, 0x4E048354, 0x3903B3C2, 
+    0xA7672661, 0xD06016F7, 0x4969474D, 0x3E6E77DB, 
+    0xAED16A4A, 0xD9D65ADC, 0x40DF0B66, 0x37D83BF0, 
+    0xA9BCAE53, 0xDEBB9EC5, 0x47B2CF7F, 0x30B5FFE9, 
+    0xBDBDF21C, 0xCABAC28A, 0x53B39330, 0x24B4A3A6, 
+    0xBAD03605, 0xCDD70693, 0x54DE5729, 0x23D967BF, 
+    0xB3667A2E, 0xC4614AB8, 0x5D681B02, 0x2A6F2B94, 
+    0xB40BBE37, 0xC30C8EA1, 0x5A05DF1B, 0x2D02EF8D,
+    /* T8_1 */
+    0x00000000, 0x191B3141, 0x32366282, 0x2B2D53C3, 
+    0x646CC504, 0x7D77F445, 0x565AA786, 0x4F4196C7, 
+    0xC8D98A08, 0xD1C2BB49, 0xFAEFE88A, 0xE3F4D9CB, 
+    0xACB54F0C, 0xB5AE7E4D, 0x9E832D8E, 0x87981CCF, 
+    0x4AC21251, 0x53D92310, 0x78F470D3, 0x61EF4192, 
+    0x2EAED755, 0x37B5E614, 0x1C98B5D7, 0x05838496, 
+    0x821B9859, 0x9B00A918, 0xB02DFADB, 0xA936CB9A, 
+    0xE6775D5D, 0xFF6C6C1C, 0xD4413FDF, 0xCD5A0E9E, 
+    0x958424A2, 0x8C9F15E3, 0xA7B24620, 0xBEA97761, 
+    0xF1E8E1A6, 0xE8F3D0E7, 0xC3DE8324, 0xDAC5B265, 
+    0x5D5DAEAA, 0x44469FEB, 0x6F6BCC28, 0x7670FD69, 
+    0x39316BAE, 0x202A5AEF, 0x0B07092C, 0x121C386D, 
+    0xDF4636F3, 0xC65D07B2, 0xED705471, 0xF46B6530, 
+    0xBB2AF3F7, 0xA231C2B6, 0x891C9175, 0x9007A034, 
+    0x179FBCFB, 0x0E848DBA, 0x25A9DE79, 0x3CB2EF38, 
+    0x73F379FF, 0x6AE848BE, 0x41C51B7D, 0x58DE2A3C, 
+    0xF0794F05, 0xE9627E44, 0xC24F2D87, 0xDB541CC6, 
+    0x94158A01, 0x8D0EBB40, 0xA623E883, 0xBF38D9C2, 
+    0x38A0C50D, 0x21BBF44C, 0x0A96A78F, 0x138D96CE, 
+    0x5CCC0009, 0x45D73148, 0x6EFA628B, 0x77E153CA, 
+    0xBABB5D54, 0xA3A06C15, 0x888D3FD6, 0x91960E97, 
+    0xDED79850, 0xC7CCA911, 0xECE1FAD2, 0xF5FACB93, 
+    0x7262D75C, 0x6B79E61D, 0x4054B5DE, 0x594F849F, 
+    0x160E1258, 0x0F152319, 0x243870DA, 0x3D23419B, 
+    0x65FD6BA7, 0x7CE65AE6, 0x57CB0925, 0x4ED03864, 
+    0x0191AEA3, 0x188A9FE2, 0x33A7CC21, 0x2ABCFD60, 
+    0xAD24E1AF, 0xB43FD0EE, 0x9F12832D, 0x8609B26C, 
+    0xC94824AB, 0xD05315EA, 0xFB7E4629, 0xE2657768, 
+    0x2F3F79F6, 0x362448B7, 0x1D091B74, 0x04122A35, 
+    0x4B53BCF2, 0x52488DB3, 0x7965DE70, 0x607EEF31, 
+    0xE7E6F3FE, 0xFEFDC2BF, 0xD5D0917C, 0xCCCBA03D, 
+    0x838A36FA, 0x9A9107BB, 0xB1BC5478, 0xA8A76539, 
+    0x3B83984B, 0x2298A90A, 0x09B5FAC9, 0x10AECB88, 
+    0x5FEF5D4F, 0x46F46C0E, 0x6DD93FCD, 0x74C20E8C, 
+    0xF35A1243, 0xEA412302, 0xC16C70C1, 0xD8774180, 
+    0x9736D747, 0x8E2DE606, 0xA500B5C5, 0xBC1B8484, 
+    0x71418A1A, 0x685ABB5B, 0x4377E898, 0x5A6CD9D9, 
+    0x152D4F1E, 0x0C367E5F, 0x271B2D9C, 0x3E001CDD, 
+    0xB9980012, 0xA0833153, 0x8BAE6290, 0x92B553D1, 
+    0xDDF4C516, 0xC4EFF457, 0xEFC2A794, 0xF6D996D5, 
+    0xAE07BCE9, 0xB71C8DA8, 0x9C31DE6B, 0x852AEF2A, 
+    0xCA6B79ED, 0xD37048AC, 0xF85D1B6F, 0xE1462A2E, 
+    0x66DE36E1, 0x7FC507A0, 0x54E85463, 0x4DF36522, 
+    0x02B2F3E5, 0x1BA9C2A4, 0x30849167, 0x299FA026, 
+    0xE4C5AEB8, 0xFDDE9FF9, 0xD6F3CC3A, 0xCFE8FD7B, 
+    0x80A96BBC, 0x99B25AFD, 0xB29F093E, 0xAB84387F, 
+    0x2C1C24B0, 0x350715F1, 0x1E2A4632, 0x07317773, 
+    0x4870E1B4, 0x516BD0F5, 0x7A468336, 0x635DB277, 
+    0xCBFAD74E, 0xD2E1E60F, 0xF9CCB5CC, 0xE0D7848D, 
+    0xAF96124A, 0xB68D230B, 0x9DA070C8, 0x84BB4189, 
+    0x03235D46, 0x1A386C07, 0x31153FC4, 0x280E0E85, 
+    0x674F9842, 0x7E54A903, 0x5579FAC0, 0x4C62CB81, 
+    0x8138C51F, 0x9823F45E, 0xB30EA79D, 0xAA1596DC, 
+    0xE554001B, 0xFC4F315A, 0xD7626299, 0xCE7953D8, 
+    0x49E14F17, 0x50FA7E56, 0x7BD72D95, 0x62CC1CD4, 
+    0x2D8D8A13, 0x3496BB52, 0x1FBBE891, 0x06A0D9D0, 
+    0x5E7EF3EC, 0x4765C2AD, 0x6C48916E, 0x7553A02F, 
+    0x3A1236E8, 0x230907A9, 0x0824546A, 0x113F652B, 
+    0x96A779E4, 0x8FBC48A5, 0xA4911B66, 0xBD8A2A27, 
+    0xF2CBBCE0, 0xEBD08DA1, 0xC0FDDE62, 0xD9E6EF23, 
+    0x14BCE1BD, 0x0DA7D0FC, 0x268A833F, 0x3F91B27E, 
+    0x70D024B9, 0x69CB15F8, 0x42E6463B, 0x5BFD777A, 
+    0xDC656BB5, 0xC57E5AF4, 0xEE530937, 0xF7483876, 
+    0xB809AEB1, 0xA1129FF0, 0x8A3FCC33, 0x9324FD72,
+    /* T8_2 */
+    0x00000000, 0x01C26A37, 0x0384D46E, 0x0246BE59, 
+    0x0709A8DC, 0x06CBC2EB, 0x048D7CB2, 0x054F1685, 
+    0x0E1351B8, 0x0FD13B8F, 0x0D9785D6, 0x0C55EFE1, 
+    0x091AF964, 0x08D89353, 0x0A9E2D0A, 0x0B5C473D, 
+    0x1C26A370, 0x1DE4C947, 0x1FA2771E, 0x1E601D29, 
+    0x1B2F0BAC, 0x1AED619B, 0x18ABDFC2, 0x1969B5F5, 
+    0x1235F2C8, 0x13F798FF, 0x11B126A6, 0x10734C91, 
+    0x153C5A14, 0x14FE3023, 0x16B88E7A, 0x177AE44D, 
+    0x384D46E0, 0x398F2CD7, 0x3BC9928E, 0x3A0BF8B9, 
+    0x3F44EE3C, 0x3E86840B, 0x3CC03A52, 0x3D025065, 
+    0x365E1758, 0x379C7D6F, 0x35DAC336, 0x3418A901, 
+    0x3157BF84, 0x3095D5B3, 0x32D36BEA, 0x331101DD, 
+    0x246BE590, 0x25A98FA7, 0x27EF31FE, 0x262D5BC9, 
+    0x23624D4C, 0x22A0277B, 0x20E69922, 0x2124F315, 
+    0x2A78B428, 0x2BBADE1F, 0x29FC6046, 0x283E0A71, 
+    0x2D711CF4, 0x2CB376C3, 0x2EF5C89A, 0x2F37A2AD, 
+    0x709A8DC0, 0x7158E7F7, 0x731E59AE, 0x72DC3399, 
+    0x7793251C, 0x76514F2B, 0x7417F172, 0x75D59B45, 
+    0x7E89DC78, 0x7F4BB64F, 0x7D0D0816, 0x7CCF6221, 
+    0x798074A4, 0x78421E93, 0x7A04A0CA, 0x7BC6CAFD, 
+    0x6CBC2EB0, 0x6D7E4487, 0x6F38FADE, 0x6EFA90E9, 
+    0x6BB5866C, 0x6A77EC5B, 0x68315202, 0x69F33835, 
+    0x62AF7F08, 0x636D153F, 0x612BAB66, 0x60E9C151, 
+    0x65A6D7D4, 0x6464BDE3, 0x662203BA, 0x67E0698D, 
+    0x48D7CB20, 0x4915A117, 0x4B531F4E, 0x4A917579, 
+    0x4FDE63FC, 0x4E1C09CB, 0x4C5AB792, 0x4D98DDA5, 
+    0x46C49A98, 0x4706F0AF, 0x45404EF6, 0x448224C1, 
+    0x41CD3244, 0x400F5873, 0x4249E62A, 0x438B8C1D, 
+    0x54F16850, 0x55330267, 0x5775BC3E, 0x56B7D609, 
+    0x53F8C08C, 0x523AAABB, 0x507C14E2, 0x51BE7ED5, 
+    0x5AE239E8, 0x5B2053DF, 0x5966ED86, 0x58A487B1, 
+    0x5DEB9134, 0x5C29FB03, 0x5E6F455A, 0x5FAD2F6D, 
+    0xE1351B80, 0xE0F771B7, 0xE2B1CFEE, 0xE373A5D9, 
+    0xE63CB35C, 0xE7FED96B, 0xE5B86732, 0xE47A0D05, 
+    0xEF264A38, 0xEEE4200F, 0xECA29E56, 0xED60F461, 
+    0xE82FE2E4, 0xE9ED88D3, 0xEBAB368A, 0xEA695CBD, 
+    0xFD13B8F0, 0xFCD1D2C7, 0xFE976C9E, 0xFF5506A9, 
+    0xFA1A102C, 0xFBD87A1B, 0xF99EC442, 0xF85CAE75, 
+    0xF300E948, 0xF2C2837F, 0xF0843D26, 0xF1465711, 
+    0xF4094194, 0xF5CB2BA3, 0xF78D95FA, 0xF64FFFCD, 
+    0xD9785D60, 0xD8BA3757, 0xDAFC890E, 0xDB3EE339, 
+    0xDE71F5BC, 0xDFB39F8B, 0xDDF521D2, 0xDC374BE5, 
+    0xD76B0CD8, 0xD6A966EF, 0xD4EFD8B6, 0xD52DB281, 
+    0xD062A404, 0xD1A0CE33, 0xD3E6706A, 0xD2241A5D, 
+    0xC55EFE10, 0xC49C9427, 0xC6DA2A7E, 0xC7184049, 
+    0xC25756CC, 0xC3953CFB, 0xC1D382A2, 0xC011E895, 
+    0xCB4DAFA8, 0xCA8FC59F, 0xC8C97BC6, 0xC90B11F1, 
+    0xCC440774, 0xCD866D43, 0xCFC0D31A, 0xCE02B92D, 
+    0x91AF9640, 0x906DFC77, 0x922B422E, 0x93E92819, 
+    0x96A63E9C, 0x976454AB, 0x9522EAF2, 0x94E080C5, 
+    0x9FBCC7F8, 0x9E7EADCF, 0x9C381396, 0x9DFA79A1, 
+    0x98B56F24, 0x99770513, 0x9B31BB4A, 0x9AF3D17D, 
+    0x8D893530, 0x8C4B5F07, 0x8E0DE15E, 0x8FCF8B69, 
+    0x8A809DEC, 0x8B42F7DB, 0x89044982, 0x88C623B5, 
+    0x839A6488, 0x82580EBF, 0x801EB0E6, 0x81DCDAD1, 
+    0x8493CC54, 0x8551A663, 0x8717183A, 0x86D5720D, 
+    0xA9E2D0A0, 0xA820BA97, 0xAA6604CE, 0xABA46EF9, 
+    0xAEEB787C, 0xAF29124B, 0xAD6FAC12, 0xACADC625, 
+    0xA7F18118, 0xA633EB2F, 0xA4755576, 0xA5B73F41, 
+    0xA0F829C4, 0xA13A43F3, 0xA37CFDAA, 0xA2BE979D, 
+    0xB5C473D0, 0xB40619E7, 0xB640A7BE, 0xB782CD89, 
+    0xB2CDDB0C, 0xB30FB13B, 0xB1490F62, 0xB08B6555, 
+    0xBBD72268, 0xBA15485F, 0xB853F606, 0xB9919C31, 
+    0xBCDE8AB4, 0xBD1CE083, 0xBF5A5EDA, 0xBE9834ED,
+    /* T8_3 */
+    0x00000000, 0xB8BC6765, 0xAA09C88B, 0x12B5AFEE, 
+    0x8F629757, 0x37DEF032, 0x256B5FDC, 0x9DD738B9, 
+    0xC5B428EF, 0x7D084F8A, 0x6FBDE064, 0xD7018701, 
+    0x4AD6BFB8, 0xF26AD8DD, 0xE0DF7733, 0x58631056, 
+    0x5019579F, 0xE8A530FA, 0xFA109F14, 0x42ACF871, 
+    0xDF7BC0C8, 0x67C7A7AD, 0x75720843, 0xCDCE6F26, 
+    0x95AD7F70, 0x2D111815, 0x3FA4B7FB, 0x8718D09E, 
+    0x1ACFE827, 0xA2738F42, 0xB0C620AC, 0x087A47C9, 
+    0xA032AF3E, 0x188EC85B, 0x0A3B67B5, 0xB28700D0, 
+    0x2F503869, 0x97EC5F0C, 0x8559F0E2, 0x3DE59787, 
+    0x658687D1, 0xDD3AE0B4, 0xCF8F4F5A, 0x7733283F, 
+    0xEAE41086, 0x525877E3, 0x40EDD80D, 0xF851BF68, 
+    0xF02BF8A1, 0x48979FC4, 0x5A22302A, 0xE29E574F, 
+    0x7F496FF6, 0xC7F50893, 0xD540A77D, 0x6DFCC018, 
+    0x359FD04E, 0x8D23B72B, 0x9F9618C5, 0x272A7FA0, 
+    0xBAFD4719, 0x0241207C, 0x10F48F92, 0xA848E8F7, 
+    0x9B14583D, 0x23A83F58, 0x311D90B6, 0x89A1F7D3, 
+    0x1476CF6A, 0xACCAA80F, 0xBE7F07E1, 0x06C36084, 
+    0x5EA070D2, 0xE61C17B7, 0xF4A9B859, 0x4C15DF3C, 
+    0xD1C2E785, 0x697E80E0, 0x7BCB2F0E, 0xC377486B, 
+    0xCB0D0FA2, 0x73B168C7, 0x6104C729, 0xD9B8A04C, 
+    0x446F98F5, 0xFCD3FF90, 0xEE66507E, 0x56DA371B, 
+    0x0EB9274D, 0xB6054028, 0xA4B0EFC6, 0x1C0C88A3, 
+    0x81DBB01A, 0x3967D77F, 0x2BD27891, 0x936E1FF4, 
+    0x3B26F703, 0x839A9066, 0x912F3F88, 0x299358ED, 
+    0xB4446054, 0x0CF80731, 0x1E4DA8DF, 0xA6F1CFBA, 
+    0xFE92DFEC, 0x462EB889, 0x549B1767, 0xEC277002, 
+    0x71F048BB, 0xC94C2FDE, 0xDBF98030, 0x6345E755, 
+    0x6B3FA09C, 0xD383C7F9, 0xC1366817, 0x798A0F72, 
+    0xE45D37CB, 0x5CE150AE, 0x4E54FF40, 0xF6E89825, 
+    0xAE8B8873, 0x1637EF16, 0x048240F8, 0xBC3E279D, 
+    0x21E91F24, 0x99557841, 0x8BE0D7AF, 0x335CB0CA, 
+    0xED59B63B, 0x55E5D15E, 0x47507EB0, 0xFFEC19D5, 
+    0x623B216C, 0xDA874609, 0xC832E9E7, 0x708E8E82, 
+    0x28ED9ED4, 0x9051F9B1, 0x82E4565F, 0x3A58313A, 
+    0xA78F0983, 0x1F336EE6, 0x0D86C108, 0xB53AA66D, 
+    0xBD40E1A4, 0x05FC86C1, 0x1749292F, 0xAFF54E4A, 
+    0x322276F3, 0x8A9E1196, 0x982BBE78, 0x2097D91D, 
+    0x78F4C94B, 0xC048AE2E, 0xD2FD01C0, 0x6A4166A5, 
+    0xF7965E1C, 0x4F2A3979, 0x5D9F9697, 0xE523F1F2, 
+    0x4D6B1905, 0xF5D77E60, 0xE762D18E, 0x5FDEB6EB, 
+    0xC2098E52, 0x7AB5E937, 0x680046D9, 0xD0BC21BC, 
+    0x88DF31EA, 0x3063568F, 0x22D6F961, 0x9A6A9E04, 
+    0x07BDA6BD, 0xBF01C1D8, 0xADB46E36, 0x15080953, 
+    0x1D724E9A, 0xA5CE29FF, 0xB77B8611, 0x0FC7E174, 
+    0x9210D9CD, 0x2AACBEA8, 0x38191146, 0x80A57623, 
+    0xD8C66675, 0x607A0110, 0x72CFAEFE, 0xCA73C99B, 
+    0x57A4F122, 0xEF189647, 0xFDAD39A9, 0x45115ECC, 
+    0x764DEE06, 0xCEF18963, 0xDC44268D, 0x64F841E8, 
+    0xF92F7951, 0x41931E34, 0x5326B1DA, 0xEB9AD6BF, 
+    0xB3F9C6E9, 0x0B45A18C, 0x19F00E62, 0xA14C6907, 
+    0x3C9B51BE, 0x842736DB, 0x96929935, 0x2E2EFE50, 
+    0x2654B999, 0x9EE8DEFC, 0x8C5D7112, 0x34E11677, 
+    0xA9362ECE, 0x118A49AB, 0x033FE645, 0xBB838120, 
+    0xE3E09176, 0x5B5CF613, 0x49E959FD, 0xF1553E98, 
+    0x6C820621, 0xD43E6144, 0xC68BCEAA, 0x7E37A9CF, 
+    0xD67F4138, 0x6EC3265D, 0x7C7689B3, 0xC4CAEED6, 
+    0x591DD66F, 0xE1A1B10A, 0xF3141EE4, 0x4BA87981, 
+    0x13CB69D7, 0xAB770EB2, 0xB9C2A15C, 0x017EC639, 
+    0x9CA9FE80, 0x241599E5, 0x36A0360B, 0x8E1C516E, 
+    0x866616A7, 0x3EDA71C2, 0x2C6FDE2C, 0x94D3B949, 
+    0x090481F0, 0xB1B8E695, 0xA30D497B, 0x1BB12E1E, 
+    0x43D23E48, 0xFB6E592D, 0xE9DBF6C3, 0x516791A6, 
+    0xCCB0A91F, 0x740CCE7A, 0x66B96194, 0xDE0506F1,
+    /* T8_4 */
+    0x00000000, 0x3D6029B0, 0x7AC05360, 0x47A07AD0, 
+    0xF580A6C0, 0xC8E08F70, 0x8F40F5A0, 0xB220DC10, 
+    0x30704BC1, 0x0D106271, 0x4AB018A1, 0x77D03111, 
+    0xC5F0ED01, 0xF890C4B1, 0xBF30BE61, 0x825097D1, 
+    0x60E09782, 0x5D80BE32, 0x1A20C4E2, 0x2740ED52, 
+    0x95603142, 0xA80018F2, 0xEFA06222, 0xD2C04B92, 
+    0x5090DC43, 0x6DF0F5F3, 0x2A508F23, 0x1730A693, 
+    0xA5107A83, 0x98705333, 0xDFD029E3, 0xE2B00053, 
+    0xC1C12F04, 0xFCA106B4, 0xBB017C64, 0x866155D4, 
+    0x344189C4, 0x0921A074, 0x4E81DAA4, 0x73E1F314, 
+    0xF1B164C5, 0xCCD14D75, 0x8B7137A5, 0xB6111E15, 
+    0x0431C205, 0x3951EBB5, 0x7EF19165, 0x4391B8D5, 
+    0xA121B886, 0x9C419136, 0xDBE1EBE6, 0xE681C256, 
+    0x54A11E46, 0x69C137F6, 0x2E614D26, 0x13016496, 
+    0x9151F347, 0xAC31DAF7, 0xEB91A027, 0xD6F18997, 
+    0x64D15587, 0x59B17C37, 0x1E1106E7, 0x23712F57, 
+    0x58F35849, 0x659371F9, 0x22330B29, 0x1F532299, 
+    0xAD73FE89, 0x9013D739, 0xD7B3ADE9, 0xEAD38459, 
+    0x68831388, 0x55E33A38, 0x124340E8, 0x2F236958, 
+    0x9D03B548, 0xA0639CF8, 0xE7C3E628, 0xDAA3CF98, 
+    0x3813CFCB, 0x0573E67B, 0x42D39CAB, 0x7FB3B51B, 
+    0xCD93690B, 0xF0F340BB, 0xB7533A6B, 0x8A3313DB, 
+    0x0863840A, 0x3503ADBA, 0x72A3D76A, 0x4FC3FEDA, 
+    0xFDE322CA, 0xC0830B7A, 0x872371AA, 0xBA43581A, 
+    0x9932774D, 0xA4525EFD, 0xE3F2242D, 0xDE920D9D, 
+    0x6CB2D18D, 0x51D2F83D, 0x167282ED, 0x2B12AB5D, 
+    0xA9423C8C, 0x9422153C, 0xD3826FEC, 0xEEE2465C, 
+    0x5CC29A4C, 0x61A2B3FC, 0x2602C92C, 0x1B62E09C, 
+    0xF9D2E0CF, 0xC4B2C97F, 0x8312B3AF, 0xBE729A1F, 
+    0x0C52460F, 0x31326FBF, 0x7692156F, 0x4BF23CDF, 
+    0xC9A2AB0E, 0xF4C282BE, 0xB362F86E, 0x8E02D1DE, 
+    0x3C220DCE, 0x0142247E, 0x46E25EAE, 0x7B82771E, 
+    0xB1E6B092, 0x8C869922, 0xCB26E3F2, 0xF646CA42, 
+    0x44661652, 0x79063FE2, 0x3EA64532, 0x03C66C82, 
+    0x8196FB53, 0xBCF6D2E3, 0xFB56A833, 0xC6368183, 
+    0x74165D93, 0x49767423, 0x0ED60EF3, 0x33B62743, 
+    0xD1062710, 0xEC660EA0, 0xABC67470, 0x96A65DC0, 
+    0x248681D0, 0x19E6A860, 0x5E46D2B0, 0x6326FB00, 
+    0xE1766CD1, 0xDC164561, 0x9BB63FB1, 0xA6D61601, 
+    0x14F6CA11, 0x2996E3A1, 0x6E369971, 0x5356B0C1, 
+    0x70279F96, 0x4D47B626, 0x0AE7CCF6, 0x3787E546, 
+    0x85A73956, 0xB8C710E6, 0xFF676A36, 0xC2074386, 
+    0x4057D457, 0x7D37FDE7, 0x3A978737, 0x07F7AE87, 
+    0xB5D77297, 0x88B75B27, 0xCF1721F7, 0xF2770847, 
+    0x10C70814, 0x2DA721A4, 0x6A075B74, 0x576772C4, 
+    0xE547AED4, 0xD8278764, 0x9F87FDB4, 0xA2E7D404, 
+    0x20B743D5, 0x1DD76A65, 0x5A7710B5, 0x67173905, 
+    0xD537E515, 0xE857CCA5, 0xAFF7B675, 0x92979FC5, 
+    0xE915E8DB, 0xD475C16B, 0x93D5BBBB, 0xAEB5920B, 
+    0x1C954E1B, 0x21F567AB, 0x66551D7B, 0x5B3534CB, 
+    0xD965A31A, 0xE4058AAA, 0xA3A5F07A, 0x9EC5D9CA, 
+    0x2CE505DA, 0x11852C6A, 0x562556BA, 0x6B457F0A, 
+    0x89F57F59, 0xB49556E9, 0xF3352C39, 0xCE550589, 
+    0x7C75D999, 0x4115F029, 0x06B58AF9, 0x3BD5A349, 
+    0xB9853498, 0x84E51D28, 0xC34567F8, 0xFE254E48, 
+    0x4C059258, 0x7165BBE8, 0x36C5C138, 0x0BA5E888, 
+    0x28D4C7DF, 0x15B4EE6F, 0x521494BF, 0x6F74BD0F, 
+    0xDD54611F, 0xE03448AF, 0xA794327F, 0x9AF41BCF, 
+    0x18A48C1E, 0x25C4A5AE, 0x6264DF7E, 0x5F04F6CE, 
+    0xED242ADE, 0xD044036E, 0x97E479BE, 0xAA84500E, 
+    0x4834505D, 0x755479ED, 0x32F4033D, 0x0F942A8D, 
+    0xBDB4F69D, 0x80D4DF2D, 0xC774A5FD, 0xFA148C4D, 
+    0x78441B9C, 0x4524322C, 0x028448FC, 0x3FE4614C, 
+    0x8DC4BD5C, 0xB0A494EC, 0xF704EE3C, 0xCA64C78C,
+    /* T8_5 */
+    0x00000000, 0xCB5CD3A5, 0x4DC8A10B, 0x869472AE, 
+    0x9B914216, 0x50CD91B3, 0xD659E31D, 0x1D0530B8, 
+    0xEC53826D, 0x270F51C8, 0xA19B2366, 0x6AC7F0C3, 
+    0x77C2C07B, 0xBC9E13DE, 0x3A0A6170, 0xF156B2D5, 
+    0x03D6029B, 0xC88AD13E, 0x4E1EA390, 0x85427035, 
+    0x9847408D, 0x531B9328, 0xD58FE186, 0x1ED33223, 
+    0xEF8580F6, 0x24D95353, 0xA24D21FD, 0x6911F258, 
+    0x7414C2E0, 0xBF481145, 0x39DC63EB, 0xF280B04E, 
+    0x07AC0536, 0xCCF0D693, 0x4A64A43D, 0x81387798, 
+    0x9C3D4720, 0x57619485, 0xD1F5E62B, 0x1AA9358E, 
+    0xEBFF875B, 0x20A354FE, 0xA6372650, 0x6D6BF5F5, 
+    0x706EC54D, 0xBB3216E8, 0x3DA66446, 0xF6FAB7E3, 
+    0x047A07AD, 0xCF26D408, 0x49B2A6A6, 0x82EE7503, 
+    0x9FEB45BB, 0x54B7961E, 0xD223E4B0, 0x197F3715, 
+    0xE82985C0, 0x23755665, 0xA5E124CB, 0x6EBDF76E, 
+    0x73B8C7D6, 0xB8E41473, 0x3E7066DD, 0xF52CB578, 
+    0x0F580A6C, 0xC404D9C9, 0x4290AB67, 0x89CC78C2, 
+    0x94C9487A, 0x5F959BDF, 0xD901E971, 0x125D3AD4, 
+    0xE30B8801, 0x28575BA4, 0xAEC3290A, 0x659FFAAF, 
+    0x789ACA17, 0xB3C619B2, 0x35526B1C, 0xFE0EB8B9, 
+    0x0C8E08F7, 0xC7D2DB52, 0x4146A9FC, 0x8A1A7A59, 
+    0x971F4AE1, 0x5C439944, 0xDAD7EBEA, 0x118B384F, 
+    0xE0DD8A9A, 0x2B81593F, 0xAD152B91, 0x6649F834, 
+    0x7B4CC88C, 0xB0101B29, 0x36846987, 0xFDD8BA22, 
+    0x08F40F5A, 0xC3A8DCFF, 0x453CAE51, 0x8E607DF4, 
+    0x93654D4C, 0x58399EE9, 0xDEADEC47, 0x15F13FE2, 
+    0xE4A78D37, 0x2FFB5E92, 0xA96F2C3C, 0x6233FF99, 
+    0x7F36CF21, 0xB46A1C84, 0x32FE6E2A, 0xF9A2BD8F, 
+    0x0B220DC1, 0xC07EDE64, 0x46EAACCA, 0x8DB67F6F, 
+    0x90B34FD7, 0x5BEF9C72, 0xDD7BEEDC, 0x16273D79, 
+    0xE7718FAC, 0x2C2D5C09, 0xAAB92EA7, 0x61E5FD02, 
+    0x7CE0CDBA, 0xB7BC1E1F, 0x31286CB1, 0xFA74BF14, 
+    0x1EB014D8, 0xD5ECC77D, 0x5378B5D3, 0x98246676, 
+    0x852156CE, 0x4E7D856B, 0xC8E9F7C5, 0x03B52460, 
+    0xF2E396B5, 0x39BF4510, 0xBF2B37BE, 0x7477E41B, 
+    0x6972D4A3, 0xA22E0706, 0x24BA75A8, 0xEFE6A60D, 
+    0x1D661643, 0xD63AC5E6, 0x50AEB748, 0x9BF264ED, 
+    0x86F75455, 0x4DAB87F0, 0xCB3FF55E, 0x006326FB, 
+    0xF135942E, 0x3A69478B, 0xBCFD3525, 0x77A1E680, 
+    0x6AA4D638, 0xA1F8059D, 0x276C7733, 0xEC30A496, 
+    0x191C11EE, 0xD240C24B, 0x54D4B0E5, 0x9F886340, 
+    0x828D53F8, 0x49D1805D, 0xCF45F2F3, 0x04192156, 
+    0xF54F9383, 0x3E134026, 0xB8873288, 0x73DBE12D, 
+    0x6EDED195, 0xA5820230, 0x2316709E, 0xE84AA33B, 
+    0x1ACA1375, 0xD196C0D0, 0x5702B27E, 0x9C5E61DB, 
+    0x815B5163, 0x4A0782C6, 0xCC93F068, 0x07CF23CD, 
+    0xF6999118, 0x3DC542BD, 0xBB513013, 0x700DE3B6, 
+    0x6D08D30E, 0xA65400AB, 0x20C07205, 0xEB9CA1A0, 
+    0x11E81EB4, 0xDAB4CD11, 0x5C20BFBF, 0x977C6C1A, 
+    0x8A795CA2, 0x41258F07, 0xC7B1FDA9, 0x0CED2E0C, 
+    0xFDBB9CD9, 0x36E74F7C, 0xB0733DD2, 0x7B2FEE77, 
+    0x662ADECF, 0xAD760D6A, 0x2BE27FC4, 0xE0BEAC61, 
+    0x123E1C2F, 0xD962CF8A, 0x5FF6BD24, 0x94AA6E81, 
+    0x89AF5E39, 0x42F38D9C, 0xC467FF32, 0x0F3B2C97, 
+    0xFE6D9E42, 0x35314DE7, 0xB3A53F49, 0x78F9ECEC, 
+    0x65FCDC54, 0xAEA00FF1, 0x28347D5F, 0xE368AEFA, 
+    0x16441B82, 0xDD18C827, 0x5B8CBA89, 0x90D0692C, 
+    0x8DD55994, 0x46898A31, 0xC01DF89F, 0x0B412B3A, 
+    0xFA1799EF, 0x314B4A4A, 0xB7DF38E4, 0x7C83EB41, 
+    0x6186DBF9, 0xAADA085C, 0x2C4E7AF2, 0xE712A957, 
+    0x15921919, 0xDECECABC, 0x585AB812, 0x93066BB7, 
+    0x8E035B0F, 0x455F88AA, 0xC3CBFA04, 0x089729A1, 
+    0xF9C19B74, 0x329D48D1, 0xB4093A7F, 0x7F55E9DA, 
+    0x6250D962, 0xA90C0AC7, 0x2F987869, 0xE4C4ABCC,
+    /* T8_6 */
+    0x00000000, 0xA6770BB4, 0x979F1129, 0x31E81A9D, 
+    0xF44F2413, 0x52382FA7, 0x63D0353A, 0xC5A73E8E, 
+    0x33EF4E67, 0x959845D3, 0xA4705F4E, 0x020754FA, 
+    0xC7A06A74, 0x61D761C0, 0x503F7B5D, 0xF64870E9, 
+    0x67DE9CCE, 0xC1A9977A, 0xF0418DE7, 0x56368653, 
+    0x9391B8DD, 0x35E6B369, 0x040EA9F4, 0xA279A240, 
+    0x5431D2A9, 0xF246D91D, 0xC3AEC380, 0x65D9C834, 
+    0xA07EF6BA, 0x0609FD0E, 0x37E1E793, 0x9196EC27, 
+    0xCFBD399C, 0x69CA3228, 0x582228B5, 0xFE552301, 
+    0x3BF21D8F, 0x9D85163B, 0xAC6D0CA6, 0x0A1A0712, 
+    0xFC5277FB, 0x5A257C4F, 0x6BCD66D2, 0xCDBA6D66, 
+    0x081D53E8, 0xAE6A585C, 0x9F8242C1, 0x39F54975, 
+    0xA863A552, 0x0E14AEE6, 0x3FFCB47B, 0x998BBFCF, 
+    0x5C2C8141, 0xFA5B8AF5, 0xCBB39068, 0x6DC49BDC, 
+    0x9B8CEB35, 0x3DFBE081, 0x0C13FA1C, 0xAA64F1A8, 
+    0x6FC3CF26, 0xC9B4C492, 0xF85CDE0F, 0x5E2BD5BB, 
+    0x440B7579, 0xE27C7ECD, 0xD3946450, 0x75E36FE4, 
+    0xB044516A, 0x16335ADE, 0x27DB4043, 0x81AC4BF7, 
+    0x77E43B1E, 0xD19330AA, 0xE07B2A37, 0x460C2183, 
+    0x83AB1F0D, 0x25DC14B9, 0x14340E24, 0xB2430590, 
+    0x23D5E9B7, 0x85A2E203, 0xB44AF89E, 0x123DF32A, 
+    0xD79ACDA4, 0x71EDC610, 0x4005DC8D, 0xE672D739, 
+    0x103AA7D0, 0xB64DAC64, 0x87A5B6F9, 0x21D2BD4D, 
+    0xE47583C3, 0x42028877, 0x73EA92EA, 0xD59D995E, 
+    0x8BB64CE5, 0x2DC14751, 0x1C295DCC, 0xBA5E5678, 
+    0x7FF968F6, 0xD98E6342, 0xE86679DF, 0x4E11726B, 
+    0xB8590282, 0x1E2E0936, 0x2FC613AB, 0x89B1181F, 
+    0x4C162691, 0xEA612D25, 0xDB8937B8, 0x7DFE3C0C, 
+    0xEC68D02B, 0x4A1FDB9F, 0x7BF7C102, 0xDD80CAB6, 
+    0x1827F438, 0xBE50FF8C, 0x8FB8E511, 0x29CFEEA5, 
+    0xDF879E4C, 0x79F095F8, 0x48188F65, 0xEE6F84D1, 
+    0x2BC8BA5F, 0x8DBFB1EB, 0xBC57AB76, 0x1A20A0C2, 
+    0x8816EAF2, 0x2E61E146, 0x1F89FBDB, 0xB9FEF06F, 
+    0x7C59CEE1, 0xDA2EC555, 0xEBC6DFC8, 0x4DB1D47C, 
+    0xBBF9A495, 0x1D8EAF21, 0x2C66B5BC, 0x8A11BE08, 
+    0x4FB68086, 0xE9C18B32, 0xD82991AF, 0x7E5E9A1B, 
+    0xEFC8763C, 0x49BF7D88, 0x78576715, 0xDE206CA1, 
+    0x1B87522F, 0xBDF0599B, 0x8C184306, 0x2A6F48B2, 
+    0xDC27385B, 0x7A5033EF, 0x4BB82972, 0xEDCF22C6, 
+    0x28681C48, 0x8E1F17FC, 0xBFF70D61, 0x198006D5, 
+    0x47ABD36E, 0xE1DCD8DA, 0xD034C247, 0x7643C9F3, 
+    0xB3E4F77D, 0x1593FCC9, 0x247BE654, 0x820CEDE0, 
+    0x74449D09, 0xD23396BD, 0xE3DB8C20, 0x45AC8794, 
+    0x800BB91A, 0x267CB2AE, 0x1794A833, 0xB1E3A387, 
+    0x20754FA0, 0x86024414, 0xB7EA5E89, 0x119D553D, 
+    0xD43A6BB3, 0x724D6007, 0x43A57A9A, 0xE5D2712E, 
+    0x139A01C7, 0xB5ED0A73, 0x840510EE, 0x22721B5A, 
+    0xE7D525D4, 0x41A22E60, 0x704A34FD, 0xD63D3F49, 
+    0xCC1D9F8B, 0x6A6A943F, 0x5B828EA2, 0xFDF58516, 
+    0x3852BB98, 0x9E25B02C, 0xAFCDAAB1, 0x09BAA105, 
+    0xFFF2D1EC, 0x5985DA58, 0x686DC0C5, 0xCE1ACB71, 
+    0x0BBDF5FF, 0xADCAFE4B, 0x9C22E4D6, 0x3A55EF62, 
+    0xABC30345, 0x0DB408F1, 0x3C5C126C, 0x9A2B19D8, 
+    0x5F8C2756, 0xF9FB2CE2, 0xC813367F, 0x6E643DCB, 
+    0x982C4D22, 0x3E5B4696, 0x0FB35C0B, 0xA9C457BF, 
+    0x6C636931, 0xCA146285, 0xFBFC7818, 0x5D8B73AC, 
+    0x03A0A617, 0xA5D7ADA3, 0x943FB73E, 0x3248BC8A, 
+    0xF7EF8204, 0x519889B0, 0x6070932D, 0xC6079899, 
+    0x304FE870, 0x9638E3C4, 0xA7D0F959, 0x01A7F2ED, 
+    0xC400CC63, 0x6277C7D7, 0x539FDD4A, 0xF5E8D6FE, 
+    0x647E3AD9, 0xC209316D, 0xF3E12BF0, 0x55962044, 
+    0x90311ECA, 0x3646157E, 0x07AE0FE3, 0xA1D90457, 
+    0x579174BE, 0xF1E67F0A, 0xC00E6597, 0x66796E23, 
+    0xA3DE50AD, 0x05A95B19, 0x34414184, 0x92364A30,
+    /* T8_7 */
+    0x00000000, 0xCCAA009E, 0x4225077D, 0x8E8F07E3, 
+    0x844A0EFA, 0x48E00E64, 0xC66F0987, 0x0AC50919, 
+    0xD3E51BB5, 0x1F4F1B2B, 0x91C01CC8, 0x5D6A1C56, 
+    0x57AF154F, 0x9B0515D1, 0x158A1232, 0xD92012AC, 
+    0x7CBB312B, 0xB01131B5, 0x3E9E3656, 0xF23436C8, 
+    0xF8F13FD1, 0x345B3F4F, 0xBAD438AC, 0x767E3832, 
+    0xAF5E2A9E, 0x63F42A00, 0xED7B2DE3, 0x21D12D7D, 
+    0x2B142464, 0xE7BE24FA, 0x69312319, 0xA59B2387, 
+    0xF9766256, 0x35DC62C8, 0xBB53652B, 0x77F965B5, 
+    0x7D3C6CAC, 0xB1966C32, 0x3F196BD1, 0xF3B36B4F, 
+    0x2A9379E3, 0xE639797D, 0x68B67E9E, 0xA41C7E00, 
+    0xAED97719, 0x62737787, 0xECFC7064, 0x205670FA, 
+    0x85CD537D, 0x496753E3, 0xC7E85400, 0x0B42549E, 
+    0x01875D87, 0xCD2D5D19, 0x43A25AFA, 0x8F085A64, 
+    0x562848C8, 0x9A824856, 0x140D4FB5, 0xD8A74F2B, 
+    0xD2624632, 0x1EC846AC, 0x9047414F, 0x5CED41D1, 
+    0x299DC2ED, 0xE537C273, 0x6BB8C590, 0xA712C50E, 
+    0xADD7CC17, 0x617DCC89, 0xEFF2CB6A, 0x2358CBF4, 
+    0xFA78D958, 0x36D2D9C6, 0xB85DDE25, 0x74F7DEBB, 
+    0x7E32D7A2, 0xB298D73C, 0x3C17D0DF, 0xF0BDD041, 
+    0x5526F3C6, 0x998CF358, 0x1703F4BB, 0xDBA9F425, 
+    0xD16CFD3C, 0x1DC6FDA2, 0x9349FA41, 0x5FE3FADF, 
+    0x86C3E873, 0x4A69E8ED, 0xC4E6EF0E, 0x084CEF90, 
+    0x0289E689, 0xCE23E617, 0x40ACE1F4, 0x8C06E16A, 
+    0xD0EBA0BB, 0x1C41A025, 0x92CEA7C6, 0x5E64A758, 
+    0x54A1AE41, 0x980BAEDF, 0x1684A93C, 0xDA2EA9A2, 
+    0x030EBB0E, 0xCFA4BB90, 0x412BBC73, 0x8D81BCED, 
+    0x8744B5F4, 0x4BEEB56A, 0xC561B289, 0x09CBB217, 
+    0xAC509190, 0x60FA910E, 0xEE7596ED, 0x22DF9673, 
+    0x281A9F6A, 0xE4B09FF4, 0x6A3F9817, 0xA6959889, 
+    0x7FB58A25, 0xB31F8ABB, 0x3D908D58, 0xF13A8DC6, 
+    0xFBFF84DF, 0x37558441, 0xB9DA83A2, 0x7570833C, 
+    0x533B85DA, 0x9F918544, 0x111E82A7, 0xDDB48239, 
+    0xD7718B20, 0x1BDB8BBE, 0x95548C5D, 0x59FE8CC3, 
+    0x80DE9E6F, 0x4C749EF1, 0xC2FB9912, 0x0E51998C, 
+    0x04949095, 0xC83E900B, 0x46B197E8, 0x8A1B9776, 
+    0x2F80B4F1, 0xE32AB46F, 0x6DA5B38C, 0xA10FB312, 
+    0xABCABA0B, 0x6760BA95, 0xE9EFBD76, 0x2545BDE8, 
+    0xFC65AF44, 0x30CFAFDA, 0xBE40A839, 0x72EAA8A7, 
+    0x782FA1BE, 0xB485A120, 0x3A0AA6C3, 0xF6A0A65D, 
+    0xAA4DE78C, 0x66E7E712, 0xE868E0F1, 0x24C2E06F, 
+    0x2E07E976, 0xE2ADE9E8, 0x6C22EE0B, 0xA088EE95, 
+    0x79A8FC39, 0xB502FCA7, 0x3B8DFB44, 0xF727FBDA, 
+    0xFDE2F2C3, 0x3148F25D, 0xBFC7F5BE, 0x736DF520, 
+    0xD6F6D6A7, 0x1A5CD639, 0x94D3D1DA, 0x5879D144, 
+    0x52BCD85D, 0x9E16D8C3, 0x1099DF20, 0xDC33DFBE, 
+    0x0513CD12, 0xC9B9CD8C, 0x4736CA6F, 0x8B9CCAF1, 
+    0x8159C3E8, 0x4DF3C376, 0xC37CC495, 0x0FD6C40B, 
+    0x7AA64737, 0xB60C47A9, 0x3883404A, 0xF42940D4, 
+    0xFEEC49CD, 0x32464953, 0xBCC94EB0, 0x70634E2E, 
+    0xA9435C82, 0x65E95C1C, 0xEB665BFF, 0x27CC5B61, 
+    0x2D095278, 0xE1A352E6, 0x6F2C5505, 0xA386559B, 
+    0x061D761C, 0xCAB77682, 0x44387161, 0x889271FF, 
+    0x825778E6, 0x4EFD7878, 0xC0727F9B, 0x0CD87F05, 
+    0xD5F86DA9, 0x19526D37, 0x97DD6AD4, 0x5B776A4A, 
+    0x51B26353, 0x9D1863CD, 0x1397642E, 0xDF3D64B0, 
+    0x83D02561, 0x4F7A25FF, 0xC1F5221C, 0x0D5F2282, 
+    0x079A2B9B, 0xCB302B05, 0x45BF2CE6, 0x89152C78, 
+    0x50353ED4, 0x9C9F3E4A, 0x121039A9, 0xDEBA3937, 
+    0xD47F302E, 0x18D530B0, 0x965A3753, 0x5AF037CD, 
+    0xFF6B144A, 0x33C114D4, 0xBD4E1337, 0x71E413A9, 
+    0x7B211AB0, 0xB78B1A2E, 0x39041DCD, 0xF5AE1D53, 
+    0x2C8E0FFF, 0xE0240F61, 0x6EAB0882, 0xA201081C, 
+    0xA8C40105, 0x646E019B, 0xEAE10678, 0x264B06E6
+  };
+}


[54/54] [abbrv] incubator-ratis git commit: Initial code.

Posted by ji...@apache.org.
Initial code.


Project: http://git-wip-us.apache.org/repos/asf/incubator-ratis/repo
Commit: http://git-wip-us.apache.org/repos/asf/incubator-ratis/commit/813db4b7
Tree: http://git-wip-us.apache.org/repos/asf/incubator-ratis/tree/813db4b7
Diff: http://git-wip-us.apache.org/repos/asf/incubator-ratis/diff/813db4b7

Branch: refs/heads/master
Commit: 813db4b7847891bb76164fd663a732927d7d25a4
Parents: d960d1e d16c5c6
Author: Jitendra Pandey <ji...@apache.org>
Authored: Tue Jan 31 13:15:45 2017 -0800
Committer: Jitendra Pandey <ji...@apache.org>
Committed: Tue Jan 31 13:15:45 2017 -0800

----------------------------------------------------------------------
 .gitignore                                      |   17 +
 BUILD.md                                        |   11 +
 README.md                                       |   10 +
 pom.xml                                         |  252 +++
 ratis-client/pom.xml                            |   54 +
 .../org/apache/ratis/client/RaftClient.java     |  123 ++
 .../ratis/client/RaftClientConfigKeys.java      |   23 +
 .../ratis/client/RaftClientRequestSender.java   |   34 +
 .../ratis/client/impl/ClientImplUtils.java      |   33 +
 .../ratis/client/impl/ClientProtoUtils.java     |  131 ++
 .../ratis/client/impl/RaftClientImpl.java       |  163 ++
 ratis-common/RatisCommon.cmake                  |  208 +++
 ratis-common/RatisJNI.cmake                     |   97 +
 ratis-common/pom.xml                            |  143 ++
 ratis-common/src/CMakeLists.txt                 |  108 ++
 ratis-common/src/config.h.cmake                 |   24 +
 ratis-common/src/main/conf/log4j.properties     |   74 +
 .../org/apache/ratis/conf/RaftProperties.java   | 1658 ++++++++++++++++++
 .../main/java/org/apache/ratis/io/MD5Hash.java  |  205 +++
 .../org/apache/ratis/io/nativeio/Errno.java     |   64 +
 .../org/apache/ratis/io/nativeio/NativeIO.java  |  805 +++++++++
 .../ratis/io/nativeio/NativeIOException.java    |   70 +
 .../ratis/protocol/AlreadyExistsException.java  |   36 +
 .../ratis/protocol/ChecksumException.java       |   35 +
 .../java/org/apache/ratis/protocol/Message.java |   30 +
 .../ratis/protocol/NotLeaderException.java      |   40 +
 .../RaftClientAsynchronousProtocol.java         |   30 +
 .../ratis/protocol/RaftClientProtocol.java      |   26 +
 .../apache/ratis/protocol/RaftClientReply.java  |   91 +
 .../ratis/protocol/RaftClientRequest.java       |   73 +
 .../apache/ratis/protocol/RaftException.java    |   36 +
 .../org/apache/ratis/protocol/RaftPeer.java     |   79 +
 .../apache/ratis/protocol/RaftRpcMessage.java   |   33 +
 .../ReconfigurationInProgressException.java     |   24 +
 .../ReconfigurationTimeoutException.java        |   24 +
 .../ratis/protocol/SetConfigurationRequest.java |   39 +
 .../ratis/protocol/StateMachineException.java   |   28 +
 .../ratis/util/AtomicFileOutputStream.java      |  112 ++
 .../apache/ratis/util/AutoCloseableLock.java    |   57 +
 .../org/apache/ratis/util/CheckedFunction.java  |   30 +
 .../org/apache/ratis/util/CheckedRunnable.java  |   28 +
 .../ratis/util/CodeInjectionForTesting.java     |   67 +
 .../main/java/org/apache/ratis/util/Daemon.java |   35 +
 .../java/org/apache/ratis/util/ExitUtils.java   |   98 ++
 .../java/org/apache/ratis/util/FileUtils.java   |  207 +++
 .../java/org/apache/ratis/util/LifeCycle.java   |  210 +++
 .../java/org/apache/ratis/util/MD5FileUtil.java |  177 ++
 .../org/apache/ratis/util/NativeCodeLoader.java |   70 +
 .../java/org/apache/ratis/util/NativeCrc32.java |  143 ++
 .../apache/ratis/util/NativeLibraryChecker.java |   64 +
 .../java/org/apache/ratis/util/NetUtils.java    |  151 ++
 .../org/apache/ratis/util/PeerProxyMap.java     |  128 ++
 .../java/org/apache/ratis/util/ProtoUtils.java  |  147 ++
 .../org/apache/ratis/util/PureJavaCrc32.java    |  619 +++++++
 .../org/apache/ratis/util/PureJavaCrc32C.java   |  632 +++++++
 .../java/org/apache/ratis/util/RaftUtils.java   |  240 +++
 .../java/org/apache/ratis/util/StringUtils.java |  108 ++
 .../java/org/apache/ratis/util/Timestamp.java   |   84 +
 ratis-common/src/main/native/src/exception.c    |  124 ++
 ratis-common/src/main/native/src/exception.h    |  104 ++
 .../src/org/apache/ratis/io/nativeio/NativeIO.c | 1061 +++++++++++
 .../org/apache/ratis/io/nativeio/errno_enum.c   |  123 ++
 .../org/apache/ratis/io/nativeio/errno_enum.h   |   27 +
 .../apache/ratis/io/nativeio/file_descriptor.c  |  115 ++
 .../apache/ratis/io/nativeio/file_descriptor.h  |   36 +
 .../org/apache/ratis/util/NativeCodeLoader.c    |   56 +
 .../src/org/apache/ratis/util/NativeCrc32.c     |  276 +++
 .../src/org/apache/ratis/util/bulk_crc32.c      |  244 +++
 .../src/org/apache/ratis/util/bulk_crc32.h      |   73 +
 .../org/apache/ratis/util/bulk_crc32_aarch64.c  |  362 ++++
 .../src/org/apache/ratis/util/bulk_crc32_x86.c  |  345 ++++
 .../ratis/util/crc32_zlib_polynomial_tables.h   |  552 ++++++
 .../src/org/apache/ratis/util/crc32c_tables.h   |  550 ++++++
 .../org/apache/ratis/util/gcc_optimizations.h   |   30 +
 .../src/main/native/src/org_apache_ratis.h      |  189 ++
 .../org/apache/ratis/util/test_bulk_crc32.c     |  113 ++
 .../org/apache/ratis/util/TestLifeCycle.java    |   54 +
 ratis-examples/pom.xml                          |  130 ++
 .../arithmetic/ArithmeticStateMachine.java      |  181 ++
 .../examples/arithmetic/AssignmentMessage.java  |   83 +
 .../ratis/examples/arithmetic/Evaluable.java    |   24 +
 .../arithmetic/expression/BinaryExpression.java |  103 ++
 .../arithmetic/expression/DoubleValue.java      |   61 +
 .../arithmetic/expression/Expression.java       |  112 ++
 .../arithmetic/expression/NullValue.java        |   55 +
 .../arithmetic/expression/UnaryExpression.java  |   95 +
 .../arithmetic/expression/Variable.java         |  126 ++
 .../java/org/apache/ratis/TestBatchAppend.java  |  172 ++
 .../org/apache/ratis/TestRestartRaftPeer.java   |  116 ++
 .../ratis/examples/RaftExamplesTestUtil.java    |   90 +
 .../examples/arithmetic/TestArithmetic.java     |  106 ++
 .../arithmetic/expression/TestExpression.java   |  107 ++
 .../TestRaftStateMachineException.java          |   88 +
 .../src/test/resources/log4j.properties         |   18 +
 ratis-grpc/pom.xml                              |   93 +
 .../org/apache/ratis/grpc/RaftGRpcService.java  |  185 ++
 .../apache/ratis/grpc/RaftGrpcConfigKeys.java   |   47 +
 .../org/apache/ratis/grpc/RaftGrpcUtil.java     |   87 +
 .../ratis/grpc/client/AppendStreamer.java       |  396 +++++
 .../grpc/client/RaftClientProtocolClient.java   |   73 +
 .../grpc/client/RaftClientProtocolProxy.java    |  104 ++
 .../grpc/client/RaftClientProtocolService.java  |  195 ++
 .../grpc/client/RaftClientSenderWithGrpc.java   |  119 ++
 .../ratis/grpc/client/RaftOutputStream.java     |  112 ++
 .../ratis/grpc/server/GRpcLogAppender.java      |  416 +++++
 .../server/PipelinedLogAppenderFactory.java     |   32 +
 .../grpc/server/RaftServerProtocolClient.java   |   63 +
 .../grpc/server/RaftServerProtocolService.java  |  116 ++
 .../ratis/grpc/MiniRaftClusterWithGRpc.java     |  144 ++
 .../grpc/TestNotLeaderExceptionWithGrpc.java    |   39 +
 .../grpc/TestRaftReconfigurationWithGRpc.java   |   47 +
 .../ratis/grpc/TestRaftSnapshotWithGrpc.java    |   32 +
 .../org/apache/ratis/grpc/TestRaftStream.java   |  319 ++++
 .../org/apache/ratis/grpc/TestRaftWithGrpc.java |   73 +
 ratis-grpc/src/test/resources/log4j.properties  |   18 +
 ratis-hadoop/pom.xml                            |   99 ++
 .../hadoop/ipc/ProtobufRpcEngineShaded.java     |  623 +++++++
 .../apache/ratis/hadooprpc/HadoopConstants.java |   29 +
 .../java/org/apache/ratis/hadooprpc/Proxy.java  |   55 +
 .../client/HadoopClientRequestSender.java       |   68 +
 ...aftClientProtocolClientSideTranslatorPB.java |   70 +
 .../hadooprpc/client/RaftClientProtocolPB.java  |   37 +
 ...aftClientProtocolServerSideTranslatorPB.java |   69 +
 .../hadooprpc/server/HadoopRpcService.java      |  209 +++
 .../hadooprpc/server/RaftServerProtocolPB.java  |   37 +
 ...aftServerProtocolServerSideTranslatorPB.java |   73 +
 .../hadooprpc/MiniRaftClusterWithHadoopRpc.java |  123 ++
 .../TestNotLeaderExceptionWithHadoopRpc.java    |   37 +
 .../TestRaftReconfigurationWithHadoopRpc.java   |   40 +
 .../TestRaftSnapshotWithHadoopRpc.java          |   32 +
 .../ratis/hadooprpc/TestRaftWithHadoopRpc.java  |   69 +
 .../src/test/resources/log4j.properties         |   18 +
 ratis-netty/pom.xml                             |   99 ++
 .../org/apache/ratis/netty/NettyClient.java     |   73 +
 .../org/apache/ratis/netty/NettyRpcProxy.java   |  186 ++
 .../netty/client/NettyClientRequestSender.java  |   71 +
 .../ratis/netty/server/NettyRpcService.java     |  275 +++
 .../ratis/netty/MiniRaftClusterWithNetty.java   |  117 ++
 .../netty/TestNotLeaderExceptionWithNetty.java  |   33 +
 .../netty/TestRaftReconfigurationWithNetty.java |   31 +
 .../ratis/netty/TestRaftSnapshotWithNetty.java  |   32 +
 .../apache/ratis/netty/TestRaftWithNetty.java   |   62 +
 ratis-netty/src/test/resources/log4j.properties |   18 +
 ratis-project-dist/pom.xml                      |  169 ++
 ratis-project/pom.xml                           |  409 +++++
 ratis-proto-shaded/.gitignore                   |    2 +
 ratis-proto-shaded/README.md                    |   23 +
 ratis-proto-shaded/pom.xml                      |  426 +++++
 ratis-proto-shaded/src/main/proto/GRpc.proto    |   45 +
 ratis-proto-shaded/src/main/proto/Hadoop.proto  |   44 +
 ratis-proto-shaded/src/main/proto/Netty.proto   |   49 +
 ratis-proto-shaded/src/main/proto/Raft.proto    |  165 ++
 ....ratis.shaded.io.grpc.ManagedChannelProvider |   16 +
 ...he.ratis.shaded.io.grpc.NameResolverProvider |   16 +
 ...g.apache.ratis.shaded.io.grpc.ServerProvider |   16 +
 ratis-server/pom.xml                            |   80 +
 .../org/apache/ratis/server/RaftServer.java     |   98 ++
 .../ratis/server/RaftServerConfigKeys.java      |  150 ++
 .../org/apache/ratis/server/RaftServerRpc.java  |   74 +
 .../ratis/server/impl/ConfigurationManager.java |   91 +
 .../apache/ratis/server/impl/FollowerInfo.java  |  103 ++
 .../apache/ratis/server/impl/FollowerState.java |   91 +
 .../ratis/server/impl/LeaderElection.java       |  247 +++
 .../apache/ratis/server/impl/LeaderState.java   |  601 +++++++
 .../apache/ratis/server/impl/LogAppender.java   |  494 ++++++
 .../ratis/server/impl/LogAppenderFactory.java   |   31 +
 .../ratis/server/impl/PeerConfiguration.java    |   91 +
 .../ratis/server/impl/PendingRequest.java       |   88 +
 .../ratis/server/impl/PendingRequests.java      |  130 ++
 .../ratis/server/impl/RaftConfiguration.java    |  262 +++
 .../ratis/server/impl/RaftServerConstants.java  |   46 +
 .../ratis/server/impl/RaftServerImpl.java       |  830 +++++++++
 .../ratis/server/impl/ServerImplUtils.java      |  101 ++
 .../ratis/server/impl/ServerProtoUtils.java     |  191 ++
 .../apache/ratis/server/impl/ServerState.java   |  350 ++++
 .../ratis/server/impl/StateMachineUpdater.java  |  214 +++
 .../server/protocol/RaftServerProtocol.java     |   36 +
 .../apache/ratis/server/protocol/TermIndex.java |   36 +
 .../server/storage/BufferedChannelBase.java     |   52 +
 .../server/storage/BufferedWriteChannel.java    |  159 ++
 .../apache/ratis/server/storage/FileInfo.java   |   59 +
 .../ratis/server/storage/LogInputStream.java    |  259 +++
 .../ratis/server/storage/LogOutputStream.java   |  181 ++
 .../apache/ratis/server/storage/LogReader.java  |  302 ++++
 .../apache/ratis/server/storage/LogSegment.java |  233 +++
 .../ratis/server/storage/MemoryRaftLog.java     |  183 ++
 .../apache/ratis/server/storage/MetaFile.java   |  131 ++
 .../apache/ratis/server/storage/RaftLog.java    |  293 ++++
 .../ratis/server/storage/RaftLogCache.java      |  328 ++++
 .../ratis/server/storage/RaftLogWorker.java     |  371 ++++
 .../ratis/server/storage/RaftStorage.java       |  145 ++
 .../server/storage/RaftStorageDirectory.java    |  361 ++++
 .../ratis/server/storage/SegmentedRaftLog.java  |  328 ++++
 .../ratis/server/storage/SnapshotManager.java   |  134 ++
 .../ratis/statemachine/BaseStateMachine.java    |  151 ++
 .../statemachine/FileListSnapshotInfo.java      |   64 +
 .../statemachine/SimpleStateMachineStorage.java |  135 ++
 .../statemachine/SingleFileSnapshotInfo.java    |   38 +
 .../apache/ratis/statemachine/SnapshotInfo.java |   58 +
 .../apache/ratis/statemachine/StateMachine.java |  168 ++
 .../ratis/statemachine/StateMachineStorage.java |   40 +
 .../ratis/statemachine/TransactionContext.java  |  210 +++
 .../java/org/apache/ratis/MiniRaftCluster.java  |  453 +++++
 .../java/org/apache/ratis/RaftBasicTests.java   |  199 +++
 .../ratis/RaftNotLeaderExceptionBaseTest.java   |  161 ++
 .../java/org/apache/ratis/RaftTestUtil.java     |  306 ++++
 .../impl/BlockRequestHandlingInjection.java     |   85 +
 .../impl/DelayLocalExecutionInjection.java      |   67 +
 .../impl/RaftReconfigurationBaseTest.java       |  576 ++++++
 .../ratis/server/impl/RaftServerTestUtil.java   |   76 +
 .../MiniRaftClusterWithSimulatedRpc.java        |  136 ++
 .../server/simulation/RaftServerReply.java      |  100 ++
 .../server/simulation/RaftServerRequest.java    |   98 ++
 .../ratis/server/simulation/RequestHandler.java |  134 ++
 .../simulation/SimulatedClientRequestReply.java |   45 +
 .../simulation/SimulatedRequestReply.java       |  202 +++
 .../server/simulation/SimulatedServerRpc.java   |  175 ++
 .../TestNotLeaderExceptionWithSimulation.java   |   32 +
 ...TestRaftReconfigurationWithSimulatedRpc.java |   31 +
 .../TestRaftSnapshotWithSimulatedRpc.java       |   32 +
 .../simulation/TestRaftWithSimulatedRpc.java    |   51 +
 .../ratis/server/storage/TestRaftLogCache.java  |  255 +++
 .../server/storage/TestRaftLogReadWrite.java    |  269 +++
 .../server/storage/TestRaftLogSegment.java      |  305 ++++
 .../ratis/server/storage/TestRaftStorage.java   |  215 +++
 .../server/storage/TestSegmentedRaftLog.java    |  330 ++++
 .../statemachine/RaftSnapshotBaseTest.java      |  214 +++
 .../SimpleStateMachine4Testing.java             |  246 +++
 .../ratis/statemachine/TermIndexTracker.java    |   67 +
 .../ratis/statemachine/TestStateMachine.java    |  192 ++
 .../src/test/resources/log4j.properties         |   18 +
 231 files changed, 35102 insertions(+)
----------------------------------------------------------------------



[28/54] [abbrv] incubator-ratis git commit: Renamed the packages from raft to ratis in preperation for Apache Incubation - Moved all java packages from org.apache.raft to org.apache.ratis. - Moved native package to org_apache_ratis, and native lib to l

Posted by ji...@apache.org.
http://git-wip-us.apache.org/repos/asf/incubator-ratis/blob/7e71a2e0/ratis-common/src/main/java/org/apache/ratis/util/PureJavaCrc32C.java
----------------------------------------------------------------------
diff --git a/ratis-common/src/main/java/org/apache/ratis/util/PureJavaCrc32C.java b/ratis-common/src/main/java/org/apache/ratis/util/PureJavaCrc32C.java
new file mode 100644
index 0000000..ce646db
--- /dev/null
+++ b/ratis-common/src/main/java/org/apache/ratis/util/PureJavaCrc32C.java
@@ -0,0 +1,632 @@
+/**
+ * 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.
+ * 
+ * Some portions of this file Copyright (c) 2004-2006 Intel Corportation
+ * and licensed under the BSD license.
+ */
+package org.apache.ratis.util;
+
+import java.util.zip.Checksum;
+
+/**
+ * A pure-java implementation of the CRC32 checksum that uses
+ * the CRC32-C polynomial, the same polynomial used by iSCSI
+ * and implemented on many Intel chipsets supporting SSE4.2.
+ */
+public class PureJavaCrc32C implements Checksum {
+
+  /** the current CRC value, bit-flipped */
+  private int crc;
+
+  /** Create a new PureJavaCrc32 object. */
+  public PureJavaCrc32C() {
+    reset();
+  }
+
+  @Override
+  public long getValue() {
+    long ret = crc;
+    return (~ret) & 0xffffffffL;
+  }
+
+  @Override
+  public void reset() {
+    crc = 0xffffffff;
+  }
+
+  @Override
+  public void update(byte[] b, int off, int len) {
+    int localCrc = crc;
+
+    while(len > 7) {
+      final int c0 =(b[off+0] ^ localCrc) & 0xff;
+      final int c1 =(b[off+1] ^ (localCrc >>>= 8)) & 0xff;
+      final int c2 =(b[off+2] ^ (localCrc >>>= 8)) & 0xff;
+      final int c3 =(b[off+3] ^ (localCrc >>>= 8)) & 0xff;
+      localCrc = (T[T8_7_start + c0] ^ T[T8_6_start + c1])
+          ^ (T[T8_5_start + c2] ^ T[T8_4_start + c3]);
+
+      final int c4 = b[off+4] & 0xff;
+      final int c5 = b[off+5] & 0xff;
+      final int c6 = b[off+6] & 0xff;
+      final int c7 = b[off+7] & 0xff;
+
+      localCrc ^= (T[T8_3_start + c4] ^ T[T8_2_start + c5])
+           ^ (T[T8_1_start + c6] ^ T[T8_0_start + c7]);
+
+      off += 8;
+      len -= 8;
+    }
+
+    /* loop unroll - duff's device style */
+    switch(len) {
+      case 7: localCrc = (localCrc >>> 8) ^ T[T8_0_start + ((localCrc ^ b[off++]) & 0xff)];
+      case 6: localCrc = (localCrc >>> 8) ^ T[T8_0_start + ((localCrc ^ b[off++]) & 0xff)];
+      case 5: localCrc = (localCrc >>> 8) ^ T[T8_0_start + ((localCrc ^ b[off++]) & 0xff)];
+      case 4: localCrc = (localCrc >>> 8) ^ T[T8_0_start + ((localCrc ^ b[off++]) & 0xff)];
+      case 3: localCrc = (localCrc >>> 8) ^ T[T8_0_start + ((localCrc ^ b[off++]) & 0xff)];
+      case 2: localCrc = (localCrc >>> 8) ^ T[T8_0_start + ((localCrc ^ b[off++]) & 0xff)];
+      case 1: localCrc = (localCrc >>> 8) ^ T[T8_0_start + ((localCrc ^ b[off++]) & 0xff)];
+      default:
+        /* nothing */
+    }
+    
+    // Publish crc out to object
+    crc = localCrc;
+  }
+
+  @Override
+  final public void update(int b) {
+    crc = (crc >>> 8) ^ T[T8_0_start + ((crc ^ b) & 0xff)];
+  }
+    
+  // CRC polynomial tables generated by:
+  // java -cp build/test/classes/:build/classes/ \
+  //   org.apache.hadoop.util.TestPureJavaCrc32\$Table 82F63B78
+
+  private static final int T8_0_start = 0*256;
+  private static final int T8_1_start = 1*256;
+  private static final int T8_2_start = 2*256;
+  private static final int T8_3_start = 3*256;
+  private static final int T8_4_start = 4*256;
+  private static final int T8_5_start = 5*256;
+  private static final int T8_6_start = 6*256;
+  private static final int T8_7_start = 7*256;
+
+  private static final int[] T = new int[] {
+    /* T8_0 */
+    0x00000000, 0xF26B8303, 0xE13B70F7, 0x1350F3F4, 
+    0xC79A971F, 0x35F1141C, 0x26A1E7E8, 0xD4CA64EB, 
+    0x8AD958CF, 0x78B2DBCC, 0x6BE22838, 0x9989AB3B, 
+    0x4D43CFD0, 0xBF284CD3, 0xAC78BF27, 0x5E133C24, 
+    0x105EC76F, 0xE235446C, 0xF165B798, 0x030E349B, 
+    0xD7C45070, 0x25AFD373, 0x36FF2087, 0xC494A384, 
+    0x9A879FA0, 0x68EC1CA3, 0x7BBCEF57, 0x89D76C54, 
+    0x5D1D08BF, 0xAF768BBC, 0xBC267848, 0x4E4DFB4B, 
+    0x20BD8EDE, 0xD2D60DDD, 0xC186FE29, 0x33ED7D2A, 
+    0xE72719C1, 0x154C9AC2, 0x061C6936, 0xF477EA35, 
+    0xAA64D611, 0x580F5512, 0x4B5FA6E6, 0xB93425E5, 
+    0x6DFE410E, 0x9F95C20D, 0x8CC531F9, 0x7EAEB2FA, 
+    0x30E349B1, 0xC288CAB2, 0xD1D83946, 0x23B3BA45, 
+    0xF779DEAE, 0x05125DAD, 0x1642AE59, 0xE4292D5A, 
+    0xBA3A117E, 0x4851927D, 0x5B016189, 0xA96AE28A, 
+    0x7DA08661, 0x8FCB0562, 0x9C9BF696, 0x6EF07595, 
+    0x417B1DBC, 0xB3109EBF, 0xA0406D4B, 0x522BEE48, 
+    0x86E18AA3, 0x748A09A0, 0x67DAFA54, 0x95B17957, 
+    0xCBA24573, 0x39C9C670, 0x2A993584, 0xD8F2B687, 
+    0x0C38D26C, 0xFE53516F, 0xED03A29B, 0x1F682198, 
+    0x5125DAD3, 0xA34E59D0, 0xB01EAA24, 0x42752927, 
+    0x96BF4DCC, 0x64D4CECF, 0x77843D3B, 0x85EFBE38, 
+    0xDBFC821C, 0x2997011F, 0x3AC7F2EB, 0xC8AC71E8, 
+    0x1C661503, 0xEE0D9600, 0xFD5D65F4, 0x0F36E6F7, 
+    0x61C69362, 0x93AD1061, 0x80FDE395, 0x72966096, 
+    0xA65C047D, 0x5437877E, 0x4767748A, 0xB50CF789, 
+    0xEB1FCBAD, 0x197448AE, 0x0A24BB5A, 0xF84F3859, 
+    0x2C855CB2, 0xDEEEDFB1, 0xCDBE2C45, 0x3FD5AF46, 
+    0x7198540D, 0x83F3D70E, 0x90A324FA, 0x62C8A7F9, 
+    0xB602C312, 0x44694011, 0x5739B3E5, 0xA55230E6, 
+    0xFB410CC2, 0x092A8FC1, 0x1A7A7C35, 0xE811FF36, 
+    0x3CDB9BDD, 0xCEB018DE, 0xDDE0EB2A, 0x2F8B6829, 
+    0x82F63B78, 0x709DB87B, 0x63CD4B8F, 0x91A6C88C, 
+    0x456CAC67, 0xB7072F64, 0xA457DC90, 0x563C5F93, 
+    0x082F63B7, 0xFA44E0B4, 0xE9141340, 0x1B7F9043, 
+    0xCFB5F4A8, 0x3DDE77AB, 0x2E8E845F, 0xDCE5075C, 
+    0x92A8FC17, 0x60C37F14, 0x73938CE0, 0x81F80FE3, 
+    0x55326B08, 0xA759E80B, 0xB4091BFF, 0x466298FC, 
+    0x1871A4D8, 0xEA1A27DB, 0xF94AD42F, 0x0B21572C, 
+    0xDFEB33C7, 0x2D80B0C4, 0x3ED04330, 0xCCBBC033, 
+    0xA24BB5A6, 0x502036A5, 0x4370C551, 0xB11B4652, 
+    0x65D122B9, 0x97BAA1BA, 0x84EA524E, 0x7681D14D, 
+    0x2892ED69, 0xDAF96E6A, 0xC9A99D9E, 0x3BC21E9D, 
+    0xEF087A76, 0x1D63F975, 0x0E330A81, 0xFC588982, 
+    0xB21572C9, 0x407EF1CA, 0x532E023E, 0xA145813D, 
+    0x758FE5D6, 0x87E466D5, 0x94B49521, 0x66DF1622, 
+    0x38CC2A06, 0xCAA7A905, 0xD9F75AF1, 0x2B9CD9F2, 
+    0xFF56BD19, 0x0D3D3E1A, 0x1E6DCDEE, 0xEC064EED, 
+    0xC38D26C4, 0x31E6A5C7, 0x22B65633, 0xD0DDD530, 
+    0x0417B1DB, 0xF67C32D8, 0xE52CC12C, 0x1747422F, 
+    0x49547E0B, 0xBB3FFD08, 0xA86F0EFC, 0x5A048DFF, 
+    0x8ECEE914, 0x7CA56A17, 0x6FF599E3, 0x9D9E1AE0, 
+    0xD3D3E1AB, 0x21B862A8, 0x32E8915C, 0xC083125F, 
+    0x144976B4, 0xE622F5B7, 0xF5720643, 0x07198540, 
+    0x590AB964, 0xAB613A67, 0xB831C993, 0x4A5A4A90, 
+    0x9E902E7B, 0x6CFBAD78, 0x7FAB5E8C, 0x8DC0DD8F, 
+    0xE330A81A, 0x115B2B19, 0x020BD8ED, 0xF0605BEE, 
+    0x24AA3F05, 0xD6C1BC06, 0xC5914FF2, 0x37FACCF1, 
+    0x69E9F0D5, 0x9B8273D6, 0x88D28022, 0x7AB90321, 
+    0xAE7367CA, 0x5C18E4C9, 0x4F48173D, 0xBD23943E, 
+    0xF36E6F75, 0x0105EC76, 0x12551F82, 0xE03E9C81, 
+    0x34F4F86A, 0xC69F7B69, 0xD5CF889D, 0x27A40B9E, 
+    0x79B737BA, 0x8BDCB4B9, 0x988C474D, 0x6AE7C44E, 
+    0xBE2DA0A5, 0x4C4623A6, 0x5F16D052, 0xAD7D5351, 
+    /* T8_1 */
+    0x00000000, 0x13A29877, 0x274530EE, 0x34E7A899, 
+    0x4E8A61DC, 0x5D28F9AB, 0x69CF5132, 0x7A6DC945, 
+    0x9D14C3B8, 0x8EB65BCF, 0xBA51F356, 0xA9F36B21, 
+    0xD39EA264, 0xC03C3A13, 0xF4DB928A, 0xE7790AFD, 
+    0x3FC5F181, 0x2C6769F6, 0x1880C16F, 0x0B225918, 
+    0x714F905D, 0x62ED082A, 0x560AA0B3, 0x45A838C4, 
+    0xA2D13239, 0xB173AA4E, 0x859402D7, 0x96369AA0, 
+    0xEC5B53E5, 0xFFF9CB92, 0xCB1E630B, 0xD8BCFB7C, 
+    0x7F8BE302, 0x6C297B75, 0x58CED3EC, 0x4B6C4B9B, 
+    0x310182DE, 0x22A31AA9, 0x1644B230, 0x05E62A47, 
+    0xE29F20BA, 0xF13DB8CD, 0xC5DA1054, 0xD6788823, 
+    0xAC154166, 0xBFB7D911, 0x8B507188, 0x98F2E9FF, 
+    0x404E1283, 0x53EC8AF4, 0x670B226D, 0x74A9BA1A, 
+    0x0EC4735F, 0x1D66EB28, 0x298143B1, 0x3A23DBC6, 
+    0xDD5AD13B, 0xCEF8494C, 0xFA1FE1D5, 0xE9BD79A2, 
+    0x93D0B0E7, 0x80722890, 0xB4958009, 0xA737187E, 
+    0xFF17C604, 0xECB55E73, 0xD852F6EA, 0xCBF06E9D, 
+    0xB19DA7D8, 0xA23F3FAF, 0x96D89736, 0x857A0F41, 
+    0x620305BC, 0x71A19DCB, 0x45463552, 0x56E4AD25, 
+    0x2C896460, 0x3F2BFC17, 0x0BCC548E, 0x186ECCF9, 
+    0xC0D23785, 0xD370AFF2, 0xE797076B, 0xF4359F1C, 
+    0x8E585659, 0x9DFACE2E, 0xA91D66B7, 0xBABFFEC0, 
+    0x5DC6F43D, 0x4E646C4A, 0x7A83C4D3, 0x69215CA4, 
+    0x134C95E1, 0x00EE0D96, 0x3409A50F, 0x27AB3D78, 
+    0x809C2506, 0x933EBD71, 0xA7D915E8, 0xB47B8D9F, 
+    0xCE1644DA, 0xDDB4DCAD, 0xE9537434, 0xFAF1EC43, 
+    0x1D88E6BE, 0x0E2A7EC9, 0x3ACDD650, 0x296F4E27, 
+    0x53028762, 0x40A01F15, 0x7447B78C, 0x67E52FFB, 
+    0xBF59D487, 0xACFB4CF0, 0x981CE469, 0x8BBE7C1E, 
+    0xF1D3B55B, 0xE2712D2C, 0xD69685B5, 0xC5341DC2, 
+    0x224D173F, 0x31EF8F48, 0x050827D1, 0x16AABFA6, 
+    0x6CC776E3, 0x7F65EE94, 0x4B82460D, 0x5820DE7A, 
+    0xFBC3FAF9, 0xE861628E, 0xDC86CA17, 0xCF245260, 
+    0xB5499B25, 0xA6EB0352, 0x920CABCB, 0x81AE33BC, 
+    0x66D73941, 0x7575A136, 0x419209AF, 0x523091D8, 
+    0x285D589D, 0x3BFFC0EA, 0x0F186873, 0x1CBAF004, 
+    0xC4060B78, 0xD7A4930F, 0xE3433B96, 0xF0E1A3E1, 
+    0x8A8C6AA4, 0x992EF2D3, 0xADC95A4A, 0xBE6BC23D, 
+    0x5912C8C0, 0x4AB050B7, 0x7E57F82E, 0x6DF56059, 
+    0x1798A91C, 0x043A316B, 0x30DD99F2, 0x237F0185, 
+    0x844819FB, 0x97EA818C, 0xA30D2915, 0xB0AFB162, 
+    0xCAC27827, 0xD960E050, 0xED8748C9, 0xFE25D0BE, 
+    0x195CDA43, 0x0AFE4234, 0x3E19EAAD, 0x2DBB72DA, 
+    0x57D6BB9F, 0x447423E8, 0x70938B71, 0x63311306, 
+    0xBB8DE87A, 0xA82F700D, 0x9CC8D894, 0x8F6A40E3, 
+    0xF50789A6, 0xE6A511D1, 0xD242B948, 0xC1E0213F, 
+    0x26992BC2, 0x353BB3B5, 0x01DC1B2C, 0x127E835B, 
+    0x68134A1E, 0x7BB1D269, 0x4F567AF0, 0x5CF4E287, 
+    0x04D43CFD, 0x1776A48A, 0x23910C13, 0x30339464, 
+    0x4A5E5D21, 0x59FCC556, 0x6D1B6DCF, 0x7EB9F5B8, 
+    0x99C0FF45, 0x8A626732, 0xBE85CFAB, 0xAD2757DC, 
+    0xD74A9E99, 0xC4E806EE, 0xF00FAE77, 0xE3AD3600, 
+    0x3B11CD7C, 0x28B3550B, 0x1C54FD92, 0x0FF665E5, 
+    0x759BACA0, 0x663934D7, 0x52DE9C4E, 0x417C0439, 
+    0xA6050EC4, 0xB5A796B3, 0x81403E2A, 0x92E2A65D, 
+    0xE88F6F18, 0xFB2DF76F, 0xCFCA5FF6, 0xDC68C781, 
+    0x7B5FDFFF, 0x68FD4788, 0x5C1AEF11, 0x4FB87766, 
+    0x35D5BE23, 0x26772654, 0x12908ECD, 0x013216BA, 
+    0xE64B1C47, 0xF5E98430, 0xC10E2CA9, 0xD2ACB4DE, 
+    0xA8C17D9B, 0xBB63E5EC, 0x8F844D75, 0x9C26D502, 
+    0x449A2E7E, 0x5738B609, 0x63DF1E90, 0x707D86E7, 
+    0x0A104FA2, 0x19B2D7D5, 0x2D557F4C, 0x3EF7E73B, 
+    0xD98EEDC6, 0xCA2C75B1, 0xFECBDD28, 0xED69455F, 
+    0x97048C1A, 0x84A6146D, 0xB041BCF4, 0xA3E32483, 
+    /* T8_2 */
+    0x00000000, 0xA541927E, 0x4F6F520D, 0xEA2EC073, 
+    0x9EDEA41A, 0x3B9F3664, 0xD1B1F617, 0x74F06469, 
+    0x38513EC5, 0x9D10ACBB, 0x773E6CC8, 0xD27FFEB6, 
+    0xA68F9ADF, 0x03CE08A1, 0xE9E0C8D2, 0x4CA15AAC, 
+    0x70A27D8A, 0xD5E3EFF4, 0x3FCD2F87, 0x9A8CBDF9, 
+    0xEE7CD990, 0x4B3D4BEE, 0xA1138B9D, 0x045219E3, 
+    0x48F3434F, 0xEDB2D131, 0x079C1142, 0xA2DD833C, 
+    0xD62DE755, 0x736C752B, 0x9942B558, 0x3C032726, 
+    0xE144FB14, 0x4405696A, 0xAE2BA919, 0x0B6A3B67, 
+    0x7F9A5F0E, 0xDADBCD70, 0x30F50D03, 0x95B49F7D, 
+    0xD915C5D1, 0x7C5457AF, 0x967A97DC, 0x333B05A2, 
+    0x47CB61CB, 0xE28AF3B5, 0x08A433C6, 0xADE5A1B8, 
+    0x91E6869E, 0x34A714E0, 0xDE89D493, 0x7BC846ED, 
+    0x0F382284, 0xAA79B0FA, 0x40577089, 0xE516E2F7, 
+    0xA9B7B85B, 0x0CF62A25, 0xE6D8EA56, 0x43997828, 
+    0x37691C41, 0x92288E3F, 0x78064E4C, 0xDD47DC32, 
+    0xC76580D9, 0x622412A7, 0x880AD2D4, 0x2D4B40AA, 
+    0x59BB24C3, 0xFCFAB6BD, 0x16D476CE, 0xB395E4B0, 
+    0xFF34BE1C, 0x5A752C62, 0xB05BEC11, 0x151A7E6F, 
+    0x61EA1A06, 0xC4AB8878, 0x2E85480B, 0x8BC4DA75, 
+    0xB7C7FD53, 0x12866F2D, 0xF8A8AF5E, 0x5DE93D20, 
+    0x29195949, 0x8C58CB37, 0x66760B44, 0xC337993A, 
+    0x8F96C396, 0x2AD751E8, 0xC0F9919B, 0x65B803E5, 
+    0x1148678C, 0xB409F5F2, 0x5E273581, 0xFB66A7FF, 
+    0x26217BCD, 0x8360E9B3, 0x694E29C0, 0xCC0FBBBE, 
+    0xB8FFDFD7, 0x1DBE4DA9, 0xF7908DDA, 0x52D11FA4, 
+    0x1E704508, 0xBB31D776, 0x511F1705, 0xF45E857B, 
+    0x80AEE112, 0x25EF736C, 0xCFC1B31F, 0x6A802161, 
+    0x56830647, 0xF3C29439, 0x19EC544A, 0xBCADC634, 
+    0xC85DA25D, 0x6D1C3023, 0x8732F050, 0x2273622E, 
+    0x6ED23882, 0xCB93AAFC, 0x21BD6A8F, 0x84FCF8F1, 
+    0xF00C9C98, 0x554D0EE6, 0xBF63CE95, 0x1A225CEB, 
+    0x8B277743, 0x2E66E53D, 0xC448254E, 0x6109B730, 
+    0x15F9D359, 0xB0B84127, 0x5A968154, 0xFFD7132A, 
+    0xB3764986, 0x1637DBF8, 0xFC191B8B, 0x595889F5, 
+    0x2DA8ED9C, 0x88E97FE2, 0x62C7BF91, 0xC7862DEF, 
+    0xFB850AC9, 0x5EC498B7, 0xB4EA58C4, 0x11ABCABA, 
+    0x655BAED3, 0xC01A3CAD, 0x2A34FCDE, 0x8F756EA0, 
+    0xC3D4340C, 0x6695A672, 0x8CBB6601, 0x29FAF47F, 
+    0x5D0A9016, 0xF84B0268, 0x1265C21B, 0xB7245065, 
+    0x6A638C57, 0xCF221E29, 0x250CDE5A, 0x804D4C24, 
+    0xF4BD284D, 0x51FCBA33, 0xBBD27A40, 0x1E93E83E, 
+    0x5232B292, 0xF77320EC, 0x1D5DE09F, 0xB81C72E1, 
+    0xCCEC1688, 0x69AD84F6, 0x83834485, 0x26C2D6FB, 
+    0x1AC1F1DD, 0xBF8063A3, 0x55AEA3D0, 0xF0EF31AE, 
+    0x841F55C7, 0x215EC7B9, 0xCB7007CA, 0x6E3195B4, 
+    0x2290CF18, 0x87D15D66, 0x6DFF9D15, 0xC8BE0F6B, 
+    0xBC4E6B02, 0x190FF97C, 0xF321390F, 0x5660AB71, 
+    0x4C42F79A, 0xE90365E4, 0x032DA597, 0xA66C37E9, 
+    0xD29C5380, 0x77DDC1FE, 0x9DF3018D, 0x38B293F3, 
+    0x7413C95F, 0xD1525B21, 0x3B7C9B52, 0x9E3D092C, 
+    0xEACD6D45, 0x4F8CFF3B, 0xA5A23F48, 0x00E3AD36, 
+    0x3CE08A10, 0x99A1186E, 0x738FD81D, 0xD6CE4A63, 
+    0xA23E2E0A, 0x077FBC74, 0xED517C07, 0x4810EE79, 
+    0x04B1B4D5, 0xA1F026AB, 0x4BDEE6D8, 0xEE9F74A6, 
+    0x9A6F10CF, 0x3F2E82B1, 0xD50042C2, 0x7041D0BC, 
+    0xAD060C8E, 0x08479EF0, 0xE2695E83, 0x4728CCFD, 
+    0x33D8A894, 0x96993AEA, 0x7CB7FA99, 0xD9F668E7, 
+    0x9557324B, 0x3016A035, 0xDA386046, 0x7F79F238, 
+    0x0B899651, 0xAEC8042F, 0x44E6C45C, 0xE1A75622, 
+    0xDDA47104, 0x78E5E37A, 0x92CB2309, 0x378AB177, 
+    0x437AD51E, 0xE63B4760, 0x0C158713, 0xA954156D, 
+    0xE5F54FC1, 0x40B4DDBF, 0xAA9A1DCC, 0x0FDB8FB2, 
+    0x7B2BEBDB, 0xDE6A79A5, 0x3444B9D6, 0x91052BA8, 
+    /* T8_3 */
+    0x00000000, 0xDD45AAB8, 0xBF672381, 0x62228939, 
+    0x7B2231F3, 0xA6679B4B, 0xC4451272, 0x1900B8CA, 
+    0xF64463E6, 0x2B01C95E, 0x49234067, 0x9466EADF, 
+    0x8D665215, 0x5023F8AD, 0x32017194, 0xEF44DB2C, 
+    0xE964B13D, 0x34211B85, 0x560392BC, 0x8B463804, 
+    0x924680CE, 0x4F032A76, 0x2D21A34F, 0xF06409F7, 
+    0x1F20D2DB, 0xC2657863, 0xA047F15A, 0x7D025BE2, 
+    0x6402E328, 0xB9474990, 0xDB65C0A9, 0x06206A11, 
+    0xD725148B, 0x0A60BE33, 0x6842370A, 0xB5079DB2, 
+    0xAC072578, 0x71428FC0, 0x136006F9, 0xCE25AC41, 
+    0x2161776D, 0xFC24DDD5, 0x9E0654EC, 0x4343FE54, 
+    0x5A43469E, 0x8706EC26, 0xE524651F, 0x3861CFA7, 
+    0x3E41A5B6, 0xE3040F0E, 0x81268637, 0x5C632C8F, 
+    0x45639445, 0x98263EFD, 0xFA04B7C4, 0x27411D7C, 
+    0xC805C650, 0x15406CE8, 0x7762E5D1, 0xAA274F69, 
+    0xB327F7A3, 0x6E625D1B, 0x0C40D422, 0xD1057E9A, 
+    0xABA65FE7, 0x76E3F55F, 0x14C17C66, 0xC984D6DE, 
+    0xD0846E14, 0x0DC1C4AC, 0x6FE34D95, 0xB2A6E72D, 
+    0x5DE23C01, 0x80A796B9, 0xE2851F80, 0x3FC0B538, 
+    0x26C00DF2, 0xFB85A74A, 0x99A72E73, 0x44E284CB, 
+    0x42C2EEDA, 0x9F874462, 0xFDA5CD5B, 0x20E067E3, 
+    0x39E0DF29, 0xE4A57591, 0x8687FCA8, 0x5BC25610, 
+    0xB4868D3C, 0x69C32784, 0x0BE1AEBD, 0xD6A40405, 
+    0xCFA4BCCF, 0x12E11677, 0x70C39F4E, 0xAD8635F6, 
+    0x7C834B6C, 0xA1C6E1D4, 0xC3E468ED, 0x1EA1C255, 
+    0x07A17A9F, 0xDAE4D027, 0xB8C6591E, 0x6583F3A6, 
+    0x8AC7288A, 0x57828232, 0x35A00B0B, 0xE8E5A1B3, 
+    0xF1E51979, 0x2CA0B3C1, 0x4E823AF8, 0x93C79040, 
+    0x95E7FA51, 0x48A250E9, 0x2A80D9D0, 0xF7C57368, 
+    0xEEC5CBA2, 0x3380611A, 0x51A2E823, 0x8CE7429B, 
+    0x63A399B7, 0xBEE6330F, 0xDCC4BA36, 0x0181108E, 
+    0x1881A844, 0xC5C402FC, 0xA7E68BC5, 0x7AA3217D, 
+    0x52A0C93F, 0x8FE56387, 0xEDC7EABE, 0x30824006, 
+    0x2982F8CC, 0xF4C75274, 0x96E5DB4D, 0x4BA071F5, 
+    0xA4E4AAD9, 0x79A10061, 0x1B838958, 0xC6C623E0, 
+    0xDFC69B2A, 0x02833192, 0x60A1B8AB, 0xBDE41213, 
+    0xBBC47802, 0x6681D2BA, 0x04A35B83, 0xD9E6F13B, 
+    0xC0E649F1, 0x1DA3E349, 0x7F816A70, 0xA2C4C0C8, 
+    0x4D801BE4, 0x90C5B15C, 0xF2E73865, 0x2FA292DD, 
+    0x36A22A17, 0xEBE780AF, 0x89C50996, 0x5480A32E, 
+    0x8585DDB4, 0x58C0770C, 0x3AE2FE35, 0xE7A7548D, 
+    0xFEA7EC47, 0x23E246FF, 0x41C0CFC6, 0x9C85657E, 
+    0x73C1BE52, 0xAE8414EA, 0xCCA69DD3, 0x11E3376B, 
+    0x08E38FA1, 0xD5A62519, 0xB784AC20, 0x6AC10698, 
+    0x6CE16C89, 0xB1A4C631, 0xD3864F08, 0x0EC3E5B0, 
+    0x17C35D7A, 0xCA86F7C2, 0xA8A47EFB, 0x75E1D443, 
+    0x9AA50F6F, 0x47E0A5D7, 0x25C22CEE, 0xF8878656, 
+    0xE1873E9C, 0x3CC29424, 0x5EE01D1D, 0x83A5B7A5, 
+    0xF90696D8, 0x24433C60, 0x4661B559, 0x9B241FE1, 
+    0x8224A72B, 0x5F610D93, 0x3D4384AA, 0xE0062E12, 
+    0x0F42F53E, 0xD2075F86, 0xB025D6BF, 0x6D607C07, 
+    0x7460C4CD, 0xA9256E75, 0xCB07E74C, 0x16424DF4, 
+    0x106227E5, 0xCD278D5D, 0xAF050464, 0x7240AEDC, 
+    0x6B401616, 0xB605BCAE, 0xD4273597, 0x09629F2F, 
+    0xE6264403, 0x3B63EEBB, 0x59416782, 0x8404CD3A, 
+    0x9D0475F0, 0x4041DF48, 0x22635671, 0xFF26FCC9, 
+    0x2E238253, 0xF36628EB, 0x9144A1D2, 0x4C010B6A, 
+    0x5501B3A0, 0x88441918, 0xEA669021, 0x37233A99, 
+    0xD867E1B5, 0x05224B0D, 0x6700C234, 0xBA45688C, 
+    0xA345D046, 0x7E007AFE, 0x1C22F3C7, 0xC167597F, 
+    0xC747336E, 0x1A0299D6, 0x782010EF, 0xA565BA57, 
+    0xBC65029D, 0x6120A825, 0x0302211C, 0xDE478BA4, 
+    0x31035088, 0xEC46FA30, 0x8E647309, 0x5321D9B1, 
+    0x4A21617B, 0x9764CBC3, 0xF54642FA, 0x2803E842, 
+    /* T8_4 */
+    0x00000000, 0x38116FAC, 0x7022DF58, 0x4833B0F4, 
+    0xE045BEB0, 0xD854D11C, 0x906761E8, 0xA8760E44, 
+    0xC5670B91, 0xFD76643D, 0xB545D4C9, 0x8D54BB65, 
+    0x2522B521, 0x1D33DA8D, 0x55006A79, 0x6D1105D5, 
+    0x8F2261D3, 0xB7330E7F, 0xFF00BE8B, 0xC711D127, 
+    0x6F67DF63, 0x5776B0CF, 0x1F45003B, 0x27546F97, 
+    0x4A456A42, 0x725405EE, 0x3A67B51A, 0x0276DAB6, 
+    0xAA00D4F2, 0x9211BB5E, 0xDA220BAA, 0xE2336406, 
+    0x1BA8B557, 0x23B9DAFB, 0x6B8A6A0F, 0x539B05A3, 
+    0xFBED0BE7, 0xC3FC644B, 0x8BCFD4BF, 0xB3DEBB13, 
+    0xDECFBEC6, 0xE6DED16A, 0xAEED619E, 0x96FC0E32, 
+    0x3E8A0076, 0x069B6FDA, 0x4EA8DF2E, 0x76B9B082, 
+    0x948AD484, 0xAC9BBB28, 0xE4A80BDC, 0xDCB96470, 
+    0x74CF6A34, 0x4CDE0598, 0x04EDB56C, 0x3CFCDAC0, 
+    0x51EDDF15, 0x69FCB0B9, 0x21CF004D, 0x19DE6FE1, 
+    0xB1A861A5, 0x89B90E09, 0xC18ABEFD, 0xF99BD151, 
+    0x37516AAE, 0x0F400502, 0x4773B5F6, 0x7F62DA5A, 
+    0xD714D41E, 0xEF05BBB2, 0xA7360B46, 0x9F2764EA, 
+    0xF236613F, 0xCA270E93, 0x8214BE67, 0xBA05D1CB, 
+    0x1273DF8F, 0x2A62B023, 0x625100D7, 0x5A406F7B, 
+    0xB8730B7D, 0x806264D1, 0xC851D425, 0xF040BB89, 
+    0x5836B5CD, 0x6027DA61, 0x28146A95, 0x10050539, 
+    0x7D1400EC, 0x45056F40, 0x0D36DFB4, 0x3527B018, 
+    0x9D51BE5C, 0xA540D1F0, 0xED736104, 0xD5620EA8, 
+    0x2CF9DFF9, 0x14E8B055, 0x5CDB00A1, 0x64CA6F0D, 
+    0xCCBC6149, 0xF4AD0EE5, 0xBC9EBE11, 0x848FD1BD, 
+    0xE99ED468, 0xD18FBBC4, 0x99BC0B30, 0xA1AD649C, 
+    0x09DB6AD8, 0x31CA0574, 0x79F9B580, 0x41E8DA2C, 
+    0xA3DBBE2A, 0x9BCAD186, 0xD3F96172, 0xEBE80EDE, 
+    0x439E009A, 0x7B8F6F36, 0x33BCDFC2, 0x0BADB06E, 
+    0x66BCB5BB, 0x5EADDA17, 0x169E6AE3, 0x2E8F054F, 
+    0x86F90B0B, 0xBEE864A7, 0xF6DBD453, 0xCECABBFF, 
+    0x6EA2D55C, 0x56B3BAF0, 0x1E800A04, 0x269165A8, 
+    0x8EE76BEC, 0xB6F60440, 0xFEC5B4B4, 0xC6D4DB18, 
+    0xABC5DECD, 0x93D4B161, 0xDBE70195, 0xE3F66E39, 
+    0x4B80607D, 0x73910FD1, 0x3BA2BF25, 0x03B3D089, 
+    0xE180B48F, 0xD991DB23, 0x91A26BD7, 0xA9B3047B, 
+    0x01C50A3F, 0x39D46593, 0x71E7D567, 0x49F6BACB, 
+    0x24E7BF1E, 0x1CF6D0B2, 0x54C56046, 0x6CD40FEA, 
+    0xC4A201AE, 0xFCB36E02, 0xB480DEF6, 0x8C91B15A, 
+    0x750A600B, 0x4D1B0FA7, 0x0528BF53, 0x3D39D0FF, 
+    0x954FDEBB, 0xAD5EB117, 0xE56D01E3, 0xDD7C6E4F, 
+    0xB06D6B9A, 0x887C0436, 0xC04FB4C2, 0xF85EDB6E, 
+    0x5028D52A, 0x6839BA86, 0x200A0A72, 0x181B65DE, 
+    0xFA2801D8, 0xC2396E74, 0x8A0ADE80, 0xB21BB12C, 
+    0x1A6DBF68, 0x227CD0C4, 0x6A4F6030, 0x525E0F9C, 
+    0x3F4F0A49, 0x075E65E5, 0x4F6DD511, 0x777CBABD, 
+    0xDF0AB4F9, 0xE71BDB55, 0xAF286BA1, 0x9739040D, 
+    0x59F3BFF2, 0x61E2D05E, 0x29D160AA, 0x11C00F06, 
+    0xB9B60142, 0x81A76EEE, 0xC994DE1A, 0xF185B1B6, 
+    0x9C94B463, 0xA485DBCF, 0xECB66B3B, 0xD4A70497, 
+    0x7CD10AD3, 0x44C0657F, 0x0CF3D58B, 0x34E2BA27, 
+    0xD6D1DE21, 0xEEC0B18D, 0xA6F30179, 0x9EE26ED5, 
+    0x36946091, 0x0E850F3D, 0x46B6BFC9, 0x7EA7D065, 
+    0x13B6D5B0, 0x2BA7BA1C, 0x63940AE8, 0x5B856544, 
+    0xF3F36B00, 0xCBE204AC, 0x83D1B458, 0xBBC0DBF4, 
+    0x425B0AA5, 0x7A4A6509, 0x3279D5FD, 0x0A68BA51, 
+    0xA21EB415, 0x9A0FDBB9, 0xD23C6B4D, 0xEA2D04E1, 
+    0x873C0134, 0xBF2D6E98, 0xF71EDE6C, 0xCF0FB1C0, 
+    0x6779BF84, 0x5F68D028, 0x175B60DC, 0x2F4A0F70, 
+    0xCD796B76, 0xF56804DA, 0xBD5BB42E, 0x854ADB82, 
+    0x2D3CD5C6, 0x152DBA6A, 0x5D1E0A9E, 0x650F6532, 
+    0x081E60E7, 0x300F0F4B, 0x783CBFBF, 0x402DD013, 
+    0xE85BDE57, 0xD04AB1FB, 0x9879010F, 0xA0686EA3, 
+    /* T8_5 */
+    0x00000000, 0xEF306B19, 0xDB8CA0C3, 0x34BCCBDA, 
+    0xB2F53777, 0x5DC55C6E, 0x697997B4, 0x8649FCAD, 
+    0x6006181F, 0x8F367306, 0xBB8AB8DC, 0x54BAD3C5, 
+    0xD2F32F68, 0x3DC34471, 0x097F8FAB, 0xE64FE4B2, 
+    0xC00C303E, 0x2F3C5B27, 0x1B8090FD, 0xF4B0FBE4, 
+    0x72F90749, 0x9DC96C50, 0xA975A78A, 0x4645CC93, 
+    0xA00A2821, 0x4F3A4338, 0x7B8688E2, 0x94B6E3FB, 
+    0x12FF1F56, 0xFDCF744F, 0xC973BF95, 0x2643D48C, 
+    0x85F4168D, 0x6AC47D94, 0x5E78B64E, 0xB148DD57, 
+    0x370121FA, 0xD8314AE3, 0xEC8D8139, 0x03BDEA20, 
+    0xE5F20E92, 0x0AC2658B, 0x3E7EAE51, 0xD14EC548, 
+    0x570739E5, 0xB83752FC, 0x8C8B9926, 0x63BBF23F, 
+    0x45F826B3, 0xAAC84DAA, 0x9E748670, 0x7144ED69, 
+    0xF70D11C4, 0x183D7ADD, 0x2C81B107, 0xC3B1DA1E, 
+    0x25FE3EAC, 0xCACE55B5, 0xFE729E6F, 0x1142F576, 
+    0x970B09DB, 0x783B62C2, 0x4C87A918, 0xA3B7C201, 
+    0x0E045BEB, 0xE13430F2, 0xD588FB28, 0x3AB89031, 
+    0xBCF16C9C, 0x53C10785, 0x677DCC5F, 0x884DA746, 
+    0x6E0243F4, 0x813228ED, 0xB58EE337, 0x5ABE882E, 
+    0xDCF77483, 0x33C71F9A, 0x077BD440, 0xE84BBF59, 
+    0xCE086BD5, 0x213800CC, 0x1584CB16, 0xFAB4A00F, 
+    0x7CFD5CA2, 0x93CD37BB, 0xA771FC61, 0x48419778, 
+    0xAE0E73CA, 0x413E18D3, 0x7582D309, 0x9AB2B810, 
+    0x1CFB44BD, 0xF3CB2FA4, 0xC777E47E, 0x28478F67, 
+    0x8BF04D66, 0x64C0267F, 0x507CEDA5, 0xBF4C86BC, 
+    0x39057A11, 0xD6351108, 0xE289DAD2, 0x0DB9B1CB, 
+    0xEBF65579, 0x04C63E60, 0x307AF5BA, 0xDF4A9EA3, 
+    0x5903620E, 0xB6330917, 0x828FC2CD, 0x6DBFA9D4, 
+    0x4BFC7D58, 0xA4CC1641, 0x9070DD9B, 0x7F40B682, 
+    0xF9094A2F, 0x16392136, 0x2285EAEC, 0xCDB581F5, 
+    0x2BFA6547, 0xC4CA0E5E, 0xF076C584, 0x1F46AE9D, 
+    0x990F5230, 0x763F3929, 0x4283F2F3, 0xADB399EA, 
+    0x1C08B7D6, 0xF338DCCF, 0xC7841715, 0x28B47C0C, 
+    0xAEFD80A1, 0x41CDEBB8, 0x75712062, 0x9A414B7B, 
+    0x7C0EAFC9, 0x933EC4D0, 0xA7820F0A, 0x48B26413, 
+    0xCEFB98BE, 0x21CBF3A7, 0x1577387D, 0xFA475364, 
+    0xDC0487E8, 0x3334ECF1, 0x0788272B, 0xE8B84C32, 
+    0x6EF1B09F, 0x81C1DB86, 0xB57D105C, 0x5A4D7B45, 
+    0xBC029FF7, 0x5332F4EE, 0x678E3F34, 0x88BE542D, 
+    0x0EF7A880, 0xE1C7C399, 0xD57B0843, 0x3A4B635A, 
+    0x99FCA15B, 0x76CCCA42, 0x42700198, 0xAD406A81, 
+    0x2B09962C, 0xC439FD35, 0xF08536EF, 0x1FB55DF6, 
+    0xF9FAB944, 0x16CAD25D, 0x22761987, 0xCD46729E, 
+    0x4B0F8E33, 0xA43FE52A, 0x90832EF0, 0x7FB345E9, 
+    0x59F09165, 0xB6C0FA7C, 0x827C31A6, 0x6D4C5ABF, 
+    0xEB05A612, 0x0435CD0B, 0x308906D1, 0xDFB96DC8, 
+    0x39F6897A, 0xD6C6E263, 0xE27A29B9, 0x0D4A42A0, 
+    0x8B03BE0D, 0x6433D514, 0x508F1ECE, 0xBFBF75D7, 
+    0x120CEC3D, 0xFD3C8724, 0xC9804CFE, 0x26B027E7, 
+    0xA0F9DB4A, 0x4FC9B053, 0x7B757B89, 0x94451090, 
+    0x720AF422, 0x9D3A9F3B, 0xA98654E1, 0x46B63FF8, 
+    0xC0FFC355, 0x2FCFA84C, 0x1B736396, 0xF443088F, 
+    0xD200DC03, 0x3D30B71A, 0x098C7CC0, 0xE6BC17D9, 
+    0x60F5EB74, 0x8FC5806D, 0xBB794BB7, 0x544920AE, 
+    0xB206C41C, 0x5D36AF05, 0x698A64DF, 0x86BA0FC6, 
+    0x00F3F36B, 0xEFC39872, 0xDB7F53A8, 0x344F38B1, 
+    0x97F8FAB0, 0x78C891A9, 0x4C745A73, 0xA344316A, 
+    0x250DCDC7, 0xCA3DA6DE, 0xFE816D04, 0x11B1061D, 
+    0xF7FEE2AF, 0x18CE89B6, 0x2C72426C, 0xC3422975, 
+    0x450BD5D8, 0xAA3BBEC1, 0x9E87751B, 0x71B71E02, 
+    0x57F4CA8E, 0xB8C4A197, 0x8C786A4D, 0x63480154, 
+    0xE501FDF9, 0x0A3196E0, 0x3E8D5D3A, 0xD1BD3623, 
+    0x37F2D291, 0xD8C2B988, 0xEC7E7252, 0x034E194B, 
+    0x8507E5E6, 0x6A378EFF, 0x5E8B4525, 0xB1BB2E3C, 
+    /* T8_6 */
+    0x00000000, 0x68032CC8, 0xD0065990, 0xB8057558, 
+    0xA5E0C5D1, 0xCDE3E919, 0x75E69C41, 0x1DE5B089, 
+    0x4E2DFD53, 0x262ED19B, 0x9E2BA4C3, 0xF628880B, 
+    0xEBCD3882, 0x83CE144A, 0x3BCB6112, 0x53C84DDA, 
+    0x9C5BFAA6, 0xF458D66E, 0x4C5DA336, 0x245E8FFE, 
+    0x39BB3F77, 0x51B813BF, 0xE9BD66E7, 0x81BE4A2F, 
+    0xD27607F5, 0xBA752B3D, 0x02705E65, 0x6A7372AD, 
+    0x7796C224, 0x1F95EEEC, 0xA7909BB4, 0xCF93B77C, 
+    0x3D5B83BD, 0x5558AF75, 0xED5DDA2D, 0x855EF6E5, 
+    0x98BB466C, 0xF0B86AA4, 0x48BD1FFC, 0x20BE3334, 
+    0x73767EEE, 0x1B755226, 0xA370277E, 0xCB730BB6, 
+    0xD696BB3F, 0xBE9597F7, 0x0690E2AF, 0x6E93CE67, 
+    0xA100791B, 0xC90355D3, 0x7106208B, 0x19050C43, 
+    0x04E0BCCA, 0x6CE39002, 0xD4E6E55A, 0xBCE5C992, 
+    0xEF2D8448, 0x872EA880, 0x3F2BDDD8, 0x5728F110, 
+    0x4ACD4199, 0x22CE6D51, 0x9ACB1809, 0xF2C834C1, 
+    0x7AB7077A, 0x12B42BB2, 0xAAB15EEA, 0xC2B27222, 
+    0xDF57C2AB, 0xB754EE63, 0x0F519B3B, 0x6752B7F3, 
+    0x349AFA29, 0x5C99D6E1, 0xE49CA3B9, 0x8C9F8F71, 
+    0x917A3FF8, 0xF9791330, 0x417C6668, 0x297F4AA0, 
+    0xE6ECFDDC, 0x8EEFD114, 0x36EAA44C, 0x5EE98884, 
+    0x430C380D, 0x2B0F14C5, 0x930A619D, 0xFB094D55, 
+    0xA8C1008F, 0xC0C22C47, 0x78C7591F, 0x10C475D7, 
+    0x0D21C55E, 0x6522E996, 0xDD279CCE, 0xB524B006, 
+    0x47EC84C7, 0x2FEFA80F, 0x97EADD57, 0xFFE9F19F, 
+    0xE20C4116, 0x8A0F6DDE, 0x320A1886, 0x5A09344E, 
+    0x09C17994, 0x61C2555C, 0xD9C72004, 0xB1C40CCC, 
+    0xAC21BC45, 0xC422908D, 0x7C27E5D5, 0x1424C91D, 
+    0xDBB77E61, 0xB3B452A9, 0x0BB127F1, 0x63B20B39, 
+    0x7E57BBB0, 0x16549778, 0xAE51E220, 0xC652CEE8, 
+    0x959A8332, 0xFD99AFFA, 0x459CDAA2, 0x2D9FF66A, 
+    0x307A46E3, 0x58796A2B, 0xE07C1F73, 0x887F33BB, 
+    0xF56E0EF4, 0x9D6D223C, 0x25685764, 0x4D6B7BAC, 
+    0x508ECB25, 0x388DE7ED, 0x808892B5, 0xE88BBE7D, 
+    0xBB43F3A7, 0xD340DF6F, 0x6B45AA37, 0x034686FF, 
+    0x1EA33676, 0x76A01ABE, 0xCEA56FE6, 0xA6A6432E, 
+    0x6935F452, 0x0136D89A, 0xB933ADC2, 0xD130810A, 
+    0xCCD53183, 0xA4D61D4B, 0x1CD36813, 0x74D044DB, 
+    0x27180901, 0x4F1B25C9, 0xF71E5091, 0x9F1D7C59, 
+    0x82F8CCD0, 0xEAFBE018, 0x52FE9540, 0x3AFDB988, 
+    0xC8358D49, 0xA036A181, 0x1833D4D9, 0x7030F811, 
+    0x6DD54898, 0x05D66450, 0xBDD31108, 0xD5D03DC0, 
+    0x8618701A, 0xEE1B5CD2, 0x561E298A, 0x3E1D0542, 
+    0x23F8B5CB, 0x4BFB9903, 0xF3FEEC5B, 0x9BFDC093, 
+    0x546E77EF, 0x3C6D5B27, 0x84682E7F, 0xEC6B02B7, 
+    0xF18EB23E, 0x998D9EF6, 0x2188EBAE, 0x498BC766, 
+    0x1A438ABC, 0x7240A674, 0xCA45D32C, 0xA246FFE4, 
+    0xBFA34F6D, 0xD7A063A5, 0x6FA516FD, 0x07A63A35, 
+    0x8FD9098E, 0xE7DA2546, 0x5FDF501E, 0x37DC7CD6, 
+    0x2A39CC5F, 0x423AE097, 0xFA3F95CF, 0x923CB907, 
+    0xC1F4F4DD, 0xA9F7D815, 0x11F2AD4D, 0x79F18185, 
+    0x6414310C, 0x0C171DC4, 0xB412689C, 0xDC114454, 
+    0x1382F328, 0x7B81DFE0, 0xC384AAB8, 0xAB878670, 
+    0xB66236F9, 0xDE611A31, 0x66646F69, 0x0E6743A1, 
+    0x5DAF0E7B, 0x35AC22B3, 0x8DA957EB, 0xE5AA7B23, 
+    0xF84FCBAA, 0x904CE762, 0x2849923A, 0x404ABEF2, 
+    0xB2828A33, 0xDA81A6FB, 0x6284D3A3, 0x0A87FF6B, 
+    0x17624FE2, 0x7F61632A, 0xC7641672, 0xAF673ABA, 
+    0xFCAF7760, 0x94AC5BA8, 0x2CA92EF0, 0x44AA0238, 
+    0x594FB2B1, 0x314C9E79, 0x8949EB21, 0xE14AC7E9, 
+    0x2ED97095, 0x46DA5C5D, 0xFEDF2905, 0x96DC05CD, 
+    0x8B39B544, 0xE33A998C, 0x5B3FECD4, 0x333CC01C, 
+    0x60F48DC6, 0x08F7A10E, 0xB0F2D456, 0xD8F1F89E, 
+    0xC5144817, 0xAD1764DF, 0x15121187, 0x7D113D4F, 
+    /* T8_7 */
+    0x00000000, 0x493C7D27, 0x9278FA4E, 0xDB448769, 
+    0x211D826D, 0x6821FF4A, 0xB3657823, 0xFA590504, 
+    0x423B04DA, 0x0B0779FD, 0xD043FE94, 0x997F83B3, 
+    0x632686B7, 0x2A1AFB90, 0xF15E7CF9, 0xB86201DE, 
+    0x847609B4, 0xCD4A7493, 0x160EF3FA, 0x5F328EDD, 
+    0xA56B8BD9, 0xEC57F6FE, 0x37137197, 0x7E2F0CB0, 
+    0xC64D0D6E, 0x8F717049, 0x5435F720, 0x1D098A07, 
+    0xE7508F03, 0xAE6CF224, 0x7528754D, 0x3C14086A, 
+    0x0D006599, 0x443C18BE, 0x9F789FD7, 0xD644E2F0, 
+    0x2C1DE7F4, 0x65219AD3, 0xBE651DBA, 0xF759609D, 
+    0x4F3B6143, 0x06071C64, 0xDD439B0D, 0x947FE62A, 
+    0x6E26E32E, 0x271A9E09, 0xFC5E1960, 0xB5626447, 
+    0x89766C2D, 0xC04A110A, 0x1B0E9663, 0x5232EB44, 
+    0xA86BEE40, 0xE1579367, 0x3A13140E, 0x732F6929, 
+    0xCB4D68F7, 0x827115D0, 0x593592B9, 0x1009EF9E, 
+    0xEA50EA9A, 0xA36C97BD, 0x782810D4, 0x31146DF3, 
+    0x1A00CB32, 0x533CB615, 0x8878317C, 0xC1444C5B, 
+    0x3B1D495F, 0x72213478, 0xA965B311, 0xE059CE36, 
+    0x583BCFE8, 0x1107B2CF, 0xCA4335A6, 0x837F4881, 
+    0x79264D85, 0x301A30A2, 0xEB5EB7CB, 0xA262CAEC, 
+    0x9E76C286, 0xD74ABFA1, 0x0C0E38C8, 0x453245EF, 
+    0xBF6B40EB, 0xF6573DCC, 0x2D13BAA5, 0x642FC782, 
+    0xDC4DC65C, 0x9571BB7B, 0x4E353C12, 0x07094135, 
+    0xFD504431, 0xB46C3916, 0x6F28BE7F, 0x2614C358, 
+    0x1700AEAB, 0x5E3CD38C, 0x857854E5, 0xCC4429C2, 
+    0x361D2CC6, 0x7F2151E1, 0xA465D688, 0xED59ABAF, 
+    0x553BAA71, 0x1C07D756, 0xC743503F, 0x8E7F2D18, 
+    0x7426281C, 0x3D1A553B, 0xE65ED252, 0xAF62AF75, 
+    0x9376A71F, 0xDA4ADA38, 0x010E5D51, 0x48322076, 
+    0xB26B2572, 0xFB575855, 0x2013DF3C, 0x692FA21B, 
+    0xD14DA3C5, 0x9871DEE2, 0x4335598B, 0x0A0924AC, 
+    0xF05021A8, 0xB96C5C8F, 0x6228DBE6, 0x2B14A6C1, 
+    0x34019664, 0x7D3DEB43, 0xA6796C2A, 0xEF45110D, 
+    0x151C1409, 0x5C20692E, 0x8764EE47, 0xCE589360, 
+    0x763A92BE, 0x3F06EF99, 0xE44268F0, 0xAD7E15D7, 
+    0x572710D3, 0x1E1B6DF4, 0xC55FEA9D, 0x8C6397BA, 
+    0xB0779FD0, 0xF94BE2F7, 0x220F659E, 0x6B3318B9, 
+    0x916A1DBD, 0xD856609A, 0x0312E7F3, 0x4A2E9AD4, 
+    0xF24C9B0A, 0xBB70E62D, 0x60346144, 0x29081C63, 
+    0xD3511967, 0x9A6D6440, 0x4129E329, 0x08159E0E, 
+    0x3901F3FD, 0x703D8EDA, 0xAB7909B3, 0xE2457494, 
+    0x181C7190, 0x51200CB7, 0x8A648BDE, 0xC358F6F9, 
+    0x7B3AF727, 0x32068A00, 0xE9420D69, 0xA07E704E, 
+    0x5A27754A, 0x131B086D, 0xC85F8F04, 0x8163F223, 
+    0xBD77FA49, 0xF44B876E, 0x2F0F0007, 0x66337D20, 
+    0x9C6A7824, 0xD5560503, 0x0E12826A, 0x472EFF4D, 
+    0xFF4CFE93, 0xB67083B4, 0x6D3404DD, 0x240879FA, 
+    0xDE517CFE, 0x976D01D9, 0x4C2986B0, 0x0515FB97, 
+    0x2E015D56, 0x673D2071, 0xBC79A718, 0xF545DA3F, 
+    0x0F1CDF3B, 0x4620A21C, 0x9D642575, 0xD4585852, 
+    0x6C3A598C, 0x250624AB, 0xFE42A3C2, 0xB77EDEE5, 
+    0x4D27DBE1, 0x041BA6C6, 0xDF5F21AF, 0x96635C88, 
+    0xAA7754E2, 0xE34B29C5, 0x380FAEAC, 0x7133D38B, 
+    0x8B6AD68F, 0xC256ABA8, 0x19122CC1, 0x502E51E6, 
+    0xE84C5038, 0xA1702D1F, 0x7A34AA76, 0x3308D751, 
+    0xC951D255, 0x806DAF72, 0x5B29281B, 0x1215553C, 
+    0x230138CF, 0x6A3D45E8, 0xB179C281, 0xF845BFA6, 
+    0x021CBAA2, 0x4B20C785, 0x906440EC, 0xD9583DCB, 
+    0x613A3C15, 0x28064132, 0xF342C65B, 0xBA7EBB7C, 
+    0x4027BE78, 0x091BC35F, 0xD25F4436, 0x9B633911, 
+    0xA777317B, 0xEE4B4C5C, 0x350FCB35, 0x7C33B612, 
+    0x866AB316, 0xCF56CE31, 0x14124958, 0x5D2E347F, 
+    0xE54C35A1, 0xAC704886, 0x7734CFEF, 0x3E08B2C8, 
+    0xC451B7CC, 0x8D6DCAEB, 0x56294D82, 0x1F1530A5 
+  };
+}

http://git-wip-us.apache.org/repos/asf/incubator-ratis/blob/7e71a2e0/ratis-common/src/main/java/org/apache/ratis/util/RaftUtils.java
----------------------------------------------------------------------
diff --git a/ratis-common/src/main/java/org/apache/ratis/util/RaftUtils.java b/ratis-common/src/main/java/org/apache/ratis/util/RaftUtils.java
new file mode 100644
index 0000000..8621cef
--- /dev/null
+++ b/ratis-common/src/main/java/org/apache/ratis/util/RaftUtils.java
@@ -0,0 +1,240 @@
+/**
+ * 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.ratis.util;
+
+import com.google.common.base.Preconditions;
+import org.apache.log4j.Level;
+import org.apache.log4j.LogManager;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.io.*;
+import java.lang.reflect.Constructor;
+import java.nio.ByteBuffer;
+import java.nio.channels.FileChannel;
+import java.util.Map;
+import java.util.concurrent.ConcurrentHashMap;
+import java.util.concurrent.ExecutionException;
+import java.util.concurrent.ThreadLocalRandom;
+
+public abstract class RaftUtils {
+  public static final Logger LOG = LoggerFactory.getLogger(RaftUtils.class);
+  private static final Class<?>[] EMPTY_CLASS_ARRAY = {};
+
+  // OSType detection
+  public enum OSType {
+    OS_TYPE_LINUX,
+    OS_TYPE_WIN,
+    OS_TYPE_SOLARIS,
+    OS_TYPE_MAC,
+    OS_TYPE_FREEBSD,
+    OS_TYPE_OTHER
+  }
+
+  /**
+   * Get the type of the operating system, as determined from parsing
+   * the <code>os.name</code> property.
+   */
+  private static final OSType osType = getOSType();
+
+  private static OSType getOSType() {
+    String osName = System.getProperty("os.name");
+    if (osName.startsWith("Windows")) {
+      return OSType.OS_TYPE_WIN;
+    } else if (osName.contains("SunOS") || osName.contains("Solaris")) {
+      return OSType.OS_TYPE_SOLARIS;
+    } else if (osName.contains("Mac")) {
+      return OSType.OS_TYPE_MAC;
+    } else if (osName.contains("FreeBSD")) {
+      return OSType.OS_TYPE_FREEBSD;
+    } else if (osName.startsWith("Linux")) {
+      return OSType.OS_TYPE_LINUX;
+    } else {
+      // Some other form of Unix
+      return OSType.OS_TYPE_OTHER;
+    }
+  }
+
+  // Helper static vars for each platform
+  public static final boolean WINDOWS = (osType == OSType.OS_TYPE_WIN);
+  public static final boolean SOLARIS = (osType == OSType.OS_TYPE_SOLARIS);
+  public static final boolean MAC     = (osType == OSType.OS_TYPE_MAC);
+  public static final boolean FREEBSD = (osType == OSType.OS_TYPE_FREEBSD);
+  public static final boolean LINUX   = (osType == OSType.OS_TYPE_LINUX);
+  public static final boolean OTHER   = (osType == OSType.OS_TYPE_OTHER);
+
+  public static final boolean PPC_64
+      = System.getProperties().getProperty("os.arch").contains("ppc64");
+
+  /**
+   * Cache of constructors for each class. Pins the classes so they
+   * can't be garbage collected until ReflectionUtils can be collected.
+   */
+  private static final Map<Class<?>, Constructor<?>> CONSTRUCTOR_CACHE =
+      new ConcurrentHashMap<>();
+
+  public static InterruptedIOException toInterruptedIOException(
+      String message, InterruptedException e) {
+    final InterruptedIOException iioe = new InterruptedIOException(message);
+    iioe.initCause(e);
+    return iioe;
+  }
+
+  public static IOException asIOException(Throwable t) {
+    return t instanceof IOException? (IOException)t : new IOException(t);
+  }
+
+  public static IOException toIOException(ExecutionException e) {
+    final Throwable cause = e.getCause();
+    return cause != null? asIOException(cause): new IOException(e);
+  }
+
+  /** Is the given object an instance of one of the given classes? */
+  public static boolean isInstance(Object obj, Class<?>... classes) {
+    for(Class<?> c : classes) {
+      if (c.isInstance(obj)) {
+        return true;
+      }
+    }
+    return false;
+  }
+
+  /**
+   * Create an object for the given class and initialize it from conf
+   *
+   * @param theClass class of which an object is created
+   * @return a new object
+   */
+  @SuppressWarnings("unchecked")
+  public static <T> T newInstance(Class<T> theClass, Object... initArgs) {
+    T result;
+    try {
+      Constructor<T> meth = (Constructor<T>) CONSTRUCTOR_CACHE.get(theClass);
+      if (meth == null) {
+        meth = theClass.getDeclaredConstructor(EMPTY_CLASS_ARRAY);
+        meth.setAccessible(true);
+        CONSTRUCTOR_CACHE.put(theClass, meth);
+      }
+      result = meth.newInstance(initArgs);
+    } catch (Exception e) {
+      throw new RuntimeException(e);
+    }
+    return result;
+  }
+
+  public static int getRandomBetween(int min, int max) {
+    Preconditions.checkArgument(max > min);
+    return ThreadLocalRandom.current().nextInt(max -min) + min;
+  }
+
+  public static void setLogLevel(Logger logger, Level level) {
+    LogManager.getLogger(logger.getName()).setLevel(level);
+  }
+
+
+  public static void readFully(InputStream in, int buffSize) throws IOException {
+    final byte buf[] = new byte[buffSize];
+    for(int bytesRead = in.read(buf); bytesRead >= 0; ) {
+      bytesRead = in.read(buf);
+    }
+  }
+
+  /**
+   * Reads len bytes in a loop.
+   *
+   * @param in InputStream to read from
+   * @param buf The buffer to fill
+   * @param off offset from the buffer
+   * @param len the length of bytes to read
+   * @throws IOException if it could not read requested number of bytes
+   * for any reason (including EOF)
+   */
+  public static void readFully(InputStream in, byte[] buf, int off, int len)
+      throws IOException {
+    for(int toRead = len; toRead > 0; ) {
+      final int ret = in.read(buf, off, toRead);
+      if (ret < 0) {
+        throw new IOException( "Premature EOF from inputStream");
+      }
+      toRead -= ret;
+      off += ret;
+    }
+  }
+
+  /**
+   * Write a ByteBuffer to a FileChannel at a given offset,
+   * handling short writes.
+   *
+   * @param fc               The FileChannel to write to
+   * @param buf              The input buffer
+   * @param offset           The offset in the file to start writing at
+   * @throws IOException     On I/O error
+   */
+  public static void writeFully(FileChannel fc, ByteBuffer buf, long offset)
+      throws IOException {
+    do {
+      offset += fc.write(buf, offset);
+    } while (buf.remaining() > 0);
+  }
+
+  /**
+   * Similar to readFully(). Skips bytes in a loop.
+   * @param in The InputStream to skip bytes from
+   * @param len number of bytes to skip.
+   * @throws IOException if it could not skip requested number of bytes
+   * for any reason (including EOF)
+   */
+  public static void skipFully(InputStream in, long len) throws IOException {
+    long amt = len;
+    while (amt > 0) {
+      long ret = in.skip(amt);
+      if (ret == 0) {
+        // skip may return 0 even if we're not at EOF.  Luckily, we can
+        // use the read() method to figure out if we're at the end.
+        int b = in.read();
+        if (b == -1) {
+          throw new EOFException( "Premature EOF from inputStream after " +
+              "skipping " + (len - amt) + " byte(s).");
+        }
+        ret = 1;
+      }
+      amt -= ret;
+    }
+  }
+
+  /**
+   * Close the Closeable objects and <b>ignore</b> any {@link Throwable} or
+   * null pointers. Must only be used for cleanup in exception handlers.
+   *
+   * @param log the log to record problems to at debug level. Can be null.
+   * @param closeables the objects to close
+   */
+  public static void cleanup(Logger log, Closeable... closeables) {
+    for (Closeable c : closeables) {
+      if (c != null) {
+        try {
+          c.close();
+        } catch(Throwable e) {
+          if (log != null && log.isDebugEnabled()) {
+            log.debug("Exception in closing " + c, e);
+          }
+        }
+      }
+    }
+  }
+}

http://git-wip-us.apache.org/repos/asf/incubator-ratis/blob/7e71a2e0/ratis-common/src/main/java/org/apache/ratis/util/StringUtils.java
----------------------------------------------------------------------
diff --git a/ratis-common/src/main/java/org/apache/ratis/util/StringUtils.java b/ratis-common/src/main/java/org/apache/ratis/util/StringUtils.java
new file mode 100644
index 0000000..511ebbe
--- /dev/null
+++ b/ratis-common/src/main/java/org/apache/ratis/util/StringUtils.java
@@ -0,0 +1,108 @@
+/**
+ * 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.ratis.util;
+
+import com.google.common.base.Preconditions;
+import com.google.common.collect.Interner;
+import com.google.common.collect.Interners;
+
+import java.util.Iterator;
+import java.util.Locale;
+
+public class StringUtils {
+  public static final String[] EMPTY_STRING_ARRAY = {};
+
+  /** Retains a weak reference to each string instance it has interned. */
+  private static final Interner<String> WEAK_INTERNER = Interners.newWeakInterner();
+
+  /**
+   * Interns and returns a reference to the representative instance
+   * for any of a collection of string instances that are equal to each other.
+   * Retains weak reference to the instance,
+   * and so does not prevent it from being garbage-collected.
+   *
+   * @param sample string instance to be interned
+   * @return weak reference to interned string instance
+   */
+  public static String weakIntern(String sample) {
+    return sample == null? null: WEAK_INTERNER.intern(sample);
+  }
+
+  /**
+   * Splits the given comma separated {@link String}.
+   * Each split value is trimmed.
+   *
+   * @param s a comma separated {@link String}, or possibly null.
+   * @return the split strings, or an empty array if the given string is null.
+   */
+  public static String[] getTrimmedStrings(String s) {
+    return s == null? EMPTY_STRING_ARRAY
+        : (s = s.trim()).isEmpty()? EMPTY_STRING_ARRAY
+        : s.split("\\s*,\\s*");
+  }
+
+  /** The same as String.format(Locale.ENGLISH, format, objects). */
+  public static String format(final String format, final Object... objects) {
+    return String.format(Locale.ENGLISH, format, objects);
+  }
+
+  public static String bytes2HexString(byte[] bytes) {
+    Preconditions.checkNotNull(bytes);
+
+    final StringBuilder s = new StringBuilder(2 * bytes.length);
+    for(byte b : bytes) {
+      s.append(format("%02x", b));
+    }
+    return s.toString();
+  }
+
+  public static boolean string2boolean(String s, boolean defaultValue) {
+    if (s == null || s.isEmpty()) {
+      return defaultValue;
+    }
+
+    if ("true".equalsIgnoreCase(s)) {
+      return true;
+    } else if ("false".equalsIgnoreCase(s)) {
+      return false;
+    } else {
+      return defaultValue;
+    }
+  }
+
+  /**
+   *  @return the next string in the iteration right after the given string;
+   *          if the given string is not in the iteration, return the first string.
+   */
+  public static String next(final String given, final Iterable<String> iteration) {
+    Preconditions.checkNotNull(given);
+    Preconditions.checkNotNull(iteration);
+    final Iterator<String> i = iteration.iterator();
+    Preconditions.checkArgument(i.hasNext());
+
+    final String first = i.next();
+    for(String current = first; i.hasNext(); ) {
+      final String next = i.next();
+      if (given.equals(current)) {
+        return next;
+      }
+      current = next;
+    }
+    return first;
+  }
+}

http://git-wip-us.apache.org/repos/asf/incubator-ratis/blob/7e71a2e0/ratis-common/src/main/java/org/apache/ratis/util/Timestamp.java
----------------------------------------------------------------------
diff --git a/ratis-common/src/main/java/org/apache/ratis/util/Timestamp.java b/ratis-common/src/main/java/org/apache/ratis/util/Timestamp.java
new file mode 100644
index 0000000..96e2a57
--- /dev/null
+++ b/ratis-common/src/main/java/org/apache/ratis/util/Timestamp.java
@@ -0,0 +1,84 @@
+/**
+ * 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.ratis.util;
+
+/**
+ * Use {@link System#nanoTime()} as timestamps.
+ *
+ * This class takes care the possibility of numerical overflow.
+ *
+ * The objects of this class are immutable.
+ */
+public class Timestamp implements Comparable<Timestamp> {
+  private static final long NANOSECONDS_PER_MILLISECOND = 1000000;
+
+  private static final long START_TIME = System.nanoTime();
+
+  /** @return the latest timestamp. */
+  public static Timestamp latest(Timestamp a, Timestamp b) {
+    return a.compareTo(b) > 0? a: b;
+  }
+
+  private final long nanos;
+
+  private Timestamp(long nanos) {
+    this.nanos = nanos;
+  }
+
+  /** Construct a timestamp with the current time. */
+  public Timestamp() {
+    this(System.nanoTime());
+  }
+
+  /**
+   * @param milliseconds the time period to be added.
+   * @return a new {@link Timestamp} whose value is calculated
+   *         by adding the given milliseconds to this timestamp.
+   */
+  public Timestamp addTimeMs(long milliseconds) {
+    return new Timestamp(nanos + milliseconds * NANOSECONDS_PER_MILLISECOND);
+  }
+
+  /**
+   * @return the elapsed time in milliseconds.
+   *         If the timestamp is a future time,
+   *         this method returns a negative value.
+   */
+  public long elapsedTimeMs() {
+    final long d = System.nanoTime() - nanos;
+    return d / NANOSECONDS_PER_MILLISECOND;
+  }
+
+  /**
+   * Compare two timestamps, t0 (this) and t1 (that).
+   * This method uses {@code t0 - t1 < 0}, not {@code t0 < t1},
+   * in order to take care the possibility of numerical overflow.
+   *
+   * @see System#nanoTime()
+   */
+  @Override
+  public int compareTo(Timestamp that) {
+    final long d = this.nanos - that.nanos;
+    return d > 0? 1: d == 0? 0: -1;
+  }
+
+  @Override
+  public String toString() {
+    return (nanos - START_TIME)/NANOSECONDS_PER_MILLISECOND + "ms";
+  }
+}

http://git-wip-us.apache.org/repos/asf/incubator-ratis/blob/7e71a2e0/ratis-common/src/main/native/src/exception.c
----------------------------------------------------------------------
diff --git a/ratis-common/src/main/native/src/exception.c b/ratis-common/src/main/native/src/exception.c
new file mode 100644
index 0000000..fc072e8
--- /dev/null
+++ b/ratis-common/src/main/native/src/exception.c
@@ -0,0 +1,124 @@
+/**
+ * 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.
+ */
+
+#include "exception.h"
+
+#include <jni.h>
+#include <stdio.h>
+#include <stdlib.h>
+#include <string.h>
+
+jthrowable newExceptionV(JNIEnv* env, const char *name,
+                         const char *fmt, va_list ap)
+{
+  int need;
+  char buf[1], *msg = NULL;
+  va_list ap2;
+  jstring jstr = NULL;
+  jthrowable jthr;
+  jclass clazz;
+  jmethodID excCtor;
+
+  va_copy(ap2, ap);
+  clazz = (*env)->FindClass(env, name);
+  if (!clazz) {
+    jthr = (*env)->ExceptionOccurred(env);
+    (*env)->ExceptionClear(env);
+    goto done;
+  }
+  excCtor = (*env)->GetMethodID(env,
+        clazz, "<init>", "(Ljava/lang/String;)V");
+  if (!excCtor) {
+    jthr = (*env)->ExceptionOccurred(env);
+    (*env)->ExceptionClear(env);
+    goto done;
+  }
+  need = vsnprintf(buf, sizeof(buf), fmt, ap);
+  if (need < 0) {
+    fmt = "vsnprintf error";
+    need = strlen(fmt);
+  }
+  msg = malloc(need + 1);
+  vsnprintf(msg, need + 1, fmt, ap2);
+  jstr = (*env)->NewStringUTF(env, msg);
+  if (!jstr) {
+    jthr = (*env)->ExceptionOccurred(env);
+    (*env)->ExceptionClear(env);
+    goto done;
+  }
+  jthr = (*env)->NewObject(env, clazz, excCtor, jstr);
+  if (!jthr) {
+    jthr = (*env)->ExceptionOccurred(env);
+    (*env)->ExceptionClear(env);
+    goto done;
+  }
+
+done:
+  free(msg);
+  va_end(ap2);
+  (*env)->DeleteLocalRef(env, jstr);
+  return jthr;
+}
+
+jthrowable newException(JNIEnv* env, const char *name, const char *fmt, ...)
+{
+  va_list ap;
+  jthrowable jthr;
+
+  va_start(ap, fmt);
+  jthr = newExceptionV(env, name, fmt, ap);
+  va_end(ap);
+  return jthr;
+}
+
+jthrowable newRuntimeException(JNIEnv* env, const char *fmt, ...)
+{
+  va_list ap;
+  jthrowable jthr;
+
+  va_start(ap, fmt);
+  jthr = newExceptionV(env, "java/lang/RuntimeException", fmt, ap);
+  va_end(ap);
+  return jthr;
+}
+
+jthrowable newIOException(JNIEnv* env, const char *fmt, ...)
+{
+  va_list ap;
+  jthrowable jthr;
+
+  va_start(ap, fmt);
+  jthr = newExceptionV(env, "java/io/IOException", fmt, ap);
+  va_end(ap);
+  return jthr;
+}
+
+const char* terror(int errnum)
+{
+
+#if defined(__sun)
+// MT-Safe under Solaris which doesn't support sys_errlist/sys_nerr
+  return strerror(errnum); 
+#else
+  if ((errnum < 0) || (errnum >= sys_nerr)) {
+    return "unknown error.";
+  }
+  return sys_errlist[errnum];
+#endif
+}
+

http://git-wip-us.apache.org/repos/asf/incubator-ratis/blob/7e71a2e0/ratis-common/src/main/native/src/exception.h
----------------------------------------------------------------------
diff --git a/ratis-common/src/main/native/src/exception.h b/ratis-common/src/main/native/src/exception.h
new file mode 100644
index 0000000..1aea296
--- /dev/null
+++ b/ratis-common/src/main/native/src/exception.h
@@ -0,0 +1,104 @@
+/*
+ *  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.
+ */
+#ifndef RATIS_MAIN_NATIVE_SRC_EXCEPTION_H
+#define RATIS_MAIN_NATIVE_SRC_EXCEPTION_H
+
+#include <jni.h> /* for jthrowable */
+#include <stdarg.h> /* for va_list */
+#include "org_apache_ratis.h"
+
+#ifdef WINDOWS
+/*
+ * gcc-style type-checked format arguments are not supported on Windows, so just
+ * stub this macro.
+ */
+#define TYPE_CHECKED_PRINTF_FORMAT(formatArg, varArgs)
+# else
+/* Use gcc type-checked format arguments. */
+#define TYPE_CHECKED_PRINTF_FORMAT(formatArg, varArgs) \
+  __attribute__((format(printf, formatArg, varArgs)))
+#endif
+
+/**
+ * Create a new Exception.
+ *
+ * No exceptions will be pending on return.
+ *
+ * @param env           The JNI environment
+ * @param name          full name of the Java exception class
+ * @param fmt           printf-style format string
+ * @param ap            printf-style arguments
+ *
+ * @return              The RuntimeException
+ */
+jthrowable newExceptionV(JNIEnv* env, const char *name,
+                         const char *fmt, va_list ap);
+
+/**
+ * Create a new Exception.
+ *
+ * No exceptions will be pending on return.
+ *
+ * @param env           The JNI environment
+ * @param name          full name of the Java exception class
+ * @param fmt           printf-style format string
+ * @param ...           printf-style arguments
+ *
+ * @return              The RuntimeException
+ */
+jthrowable newException(JNIEnv* env, const char *name, const char *fmt, ...)
+    TYPE_CHECKED_PRINTF_FORMAT(3, 4);
+
+/**
+ * Create a new RuntimeException.
+ *
+ * No exceptions will be pending on return.
+ *
+ * @param env           The JNI environment
+ * @param fmt           printf-style format string
+ * @param ...           printf-style arguments
+ *
+ * @return              The RuntimeException
+ */
+jthrowable newRuntimeException(JNIEnv* env, const char *fmt, ...)
+    TYPE_CHECKED_PRINTF_FORMAT(2, 3);
+
+/**
+ * Create a new IOException.
+ *
+ * No exceptions will be pending on return.
+ *
+ * @param env           The JNI environment
+ * @param fmt           printf-style format string
+ * @param ...           printf-style arguments
+ *
+ * @return              The IOException, or another exception if we failed
+ *                      to create the NativeIOException.
+ */
+jthrowable newIOException(JNIEnv* env, const char *fmt, ...)
+    TYPE_CHECKED_PRINTF_FORMAT(2, 3);
+
+/**
+ * Thread-safe strerror alternative.
+ *
+ * @param errnum        Error number.
+ * @return              Statically allocated error string.
+ */
+const char* terror(int errnum);
+
+#undef TYPE_CHECKED_PRINTF_FORMAT
+#endif

http://git-wip-us.apache.org/repos/asf/incubator-ratis/blob/7e71a2e0/ratis-common/src/main/native/src/org/apache/ratis/io/nativeio/NativeIO.c
----------------------------------------------------------------------
diff --git a/ratis-common/src/main/native/src/org/apache/ratis/io/nativeio/NativeIO.c b/ratis-common/src/main/native/src/org/apache/ratis/io/nativeio/NativeIO.c
new file mode 100644
index 0000000..e315c41
--- /dev/null
+++ b/ratis-common/src/main/native/src/org/apache/ratis/io/nativeio/NativeIO.c
@@ -0,0 +1,1061 @@
+/*
+ * 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.
+ */
+
+#include "org_apache_ratis.h"
+#include "org_apache_ratis_io_nativeio_NativeIO.h"
+#include "org_apache_ratis_io_nativeio_NativeIO_POSIX.h"
+#include "exception.h"
+
+#ifdef UNIX
+#include <assert.h>
+#include <errno.h>
+#include <fcntl.h>
+#include <grp.h>
+#include <jni.h>
+#include <pwd.h>
+#include <stdint.h>
+#include <stdio.h>
+#include <stdlib.h>
+#include <string.h>
+#include <sys/mman.h>
+#include <sys/resource.h>
+#include <sys/stat.h>
+#include <sys/syscall.h>
+#if !(defined(__FreeBSD__) || defined(__MACH__))
+#include <sys/sendfile.h>
+#endif
+#include <sys/time.h>
+#include <sys/types.h>
+#include <unistd.h>
+#include "config.h"
+#endif
+
+#ifdef WINDOWS
+#include <assert.h>
+#include <Windows.h>
+#include "winutils.h"
+#endif
+
+#include "file_descriptor.h"
+#include "errno_enum.h"
+
+#define MMAP_PROT_READ org_apache_ratis_io_nativeio_NativeIO_POSIX_MMAP_PROT_READ
+#define MMAP_PROT_WRITE org_apache_ratis_io_nativeio_NativeIO_POSIX_MMAP_PROT_WRITE
+#define MMAP_PROT_EXEC org_apache_ratis_io_nativeio_NativeIO_POSIX_MMAP_PROT_EXEC
+
+#define NATIVE_IO_POSIX_CLASS "org/apache/ratis/io/nativeio/NativeIO$POSIX"
+#define NATIVE_IO_STAT_CLASS "org/apache/ratis/io/nativeio/NativeIO$POSIX$Stat"
+
+#define SET_INT_OR_RETURN(E, C, F) \
+  { \
+    setStaticInt(E, C, #F, F); \
+    if ((*E)->ExceptionCheck(E)) return; \
+  }
+
+// the NativeIO$POSIX$Stat inner class and its constructor
+static jclass stat_clazz;
+static jmethodID stat_ctor;
+static jmethodID stat_ctor2;
+
+// the NativeIOException class and its constructor
+static jclass nioe_clazz;
+static jmethodID nioe_ctor;
+
+// the monitor used for working around non-threadsafe implementations
+// of getpwuid_r, observed on platforms including RHEL 6.0.
+// Please see HADOOP-7156 for details.
+jobject pw_lock_object;
+
+/*
+ * Throw a java.IO.IOException, generating the message from errno.
+ * NB. this is also used form windows_secure_container_executor.c
+ */
+extern void throw_ioe(JNIEnv* env, int errnum);
+
+// Internal functions
+#ifdef UNIX
+static ssize_t get_pw_buflen();
+#endif
+
+/**
+ * Returns non-zero if the user has specified that the system
+ * has non-threadsafe implementations of getpwuid_r or getgrgid_r.
+ **/
+static int workaround_non_threadsafe_calls(JNIEnv *env, jclass clazz) {
+  jboolean result;
+  jfieldID needs_workaround_field = (*env)->GetStaticFieldID(
+    env, clazz,
+    "workaroundNonThreadSafePasswdCalls",
+    "Z");
+  PASS_EXCEPTIONS_RET(env, 0);
+  assert(needs_workaround_field);
+
+  result = (*env)->GetStaticBooleanField(
+    env, clazz, needs_workaround_field);
+  return result;
+}
+
+/**
+ * Sets a static boolean field to the specified value.
+ */
+static void setStaticBoolean(JNIEnv *env, jclass clazz, char *field,
+  jboolean val) {
+    jfieldID fid = (*env)->GetStaticFieldID(env, clazz, field, "Z");
+    if (fid != NULL) {
+      (*env)->SetStaticBooleanField(env, clazz, fid, val);
+    }
+}
+
+/**
+ * Sets a static int field to the specified value.
+ */
+static void setStaticInt(JNIEnv *env, jclass clazz, char *field,
+  jint val) {
+    jfieldID fid = (*env)->GetStaticFieldID(env, clazz, field, "I");
+    if (fid != NULL) {
+      (*env)->SetStaticIntField(env, clazz, fid, val);
+    }
+}
+
+#ifdef UNIX
+/**
+ * Initialises a list of java constants that are platform specific.
+ * These are only initialized in UNIX.
+ * Any exceptions that occur will be dealt at the level above.
+**/
+static void consts_init(JNIEnv *env) {
+  jclass clazz = (*env)->FindClass(env, NATIVE_IO_POSIX_CLASS);
+  if (clazz == NULL) {
+    return; // exception has been raised
+  }
+  SET_INT_OR_RETURN(env, clazz, O_RDONLY);
+  SET_INT_OR_RETURN(env, clazz, O_WRONLY);
+  SET_INT_OR_RETURN(env, clazz, O_RDWR);
+  SET_INT_OR_RETURN(env, clazz, O_CREAT);
+  SET_INT_OR_RETURN(env, clazz, O_EXCL);
+  SET_INT_OR_RETURN(env, clazz, O_NOCTTY);
+  SET_INT_OR_RETURN(env, clazz, O_TRUNC);
+  SET_INT_OR_RETURN(env, clazz, O_APPEND);
+  SET_INT_OR_RETURN(env, clazz, O_NONBLOCK);
+  SET_INT_OR_RETURN(env, clazz, O_SYNC);
+#ifdef HAVE_POSIX_FADVISE
+  setStaticBoolean(env, clazz, "fadvisePossible", JNI_TRUE);
+  SET_INT_OR_RETURN(env, clazz, POSIX_FADV_NORMAL);
+  SET_INT_OR_RETURN(env, clazz, POSIX_FADV_RANDOM);
+  SET_INT_OR_RETURN(env, clazz, POSIX_FADV_SEQUENTIAL);
+  SET_INT_OR_RETURN(env, clazz, POSIX_FADV_WILLNEED);
+  SET_INT_OR_RETURN(env, clazz, POSIX_FADV_DONTNEED);
+  SET_INT_OR_RETURN(env, clazz, POSIX_FADV_NOREUSE);
+#else
+  setStaticBoolean(env, clazz, "fadvisePossible", JNI_FALSE);
+#endif
+#ifdef HAVE_SYNC_FILE_RANGE
+  SET_INT_OR_RETURN(env, clazz, SYNC_FILE_RANGE_WAIT_BEFORE);
+  SET_INT_OR_RETURN(env, clazz, SYNC_FILE_RANGE_WRITE);
+  SET_INT_OR_RETURN(env, clazz, SYNC_FILE_RANGE_WAIT_AFTER);
+#endif
+  clazz = (*env)->FindClass(env, NATIVE_IO_STAT_CLASS);
+  if (clazz == NULL) {
+    return; // exception has been raised
+  }
+  SET_INT_OR_RETURN(env, clazz, S_IFMT);
+  SET_INT_OR_RETURN(env, clazz, S_IFIFO);
+  SET_INT_OR_RETURN(env, clazz, S_IFCHR);
+  SET_INT_OR_RETURN(env, clazz, S_IFDIR);
+  SET_INT_OR_RETURN(env, clazz, S_IFBLK);
+  SET_INT_OR_RETURN(env, clazz, S_IFREG);
+  SET_INT_OR_RETURN(env, clazz, S_IFLNK);
+  SET_INT_OR_RETURN(env, clazz, S_IFSOCK);
+  SET_INT_OR_RETURN(env, clazz, S_ISUID);
+  SET_INT_OR_RETURN(env, clazz, S_ISGID);
+  SET_INT_OR_RETURN(env, clazz, S_ISVTX);
+  SET_INT_OR_RETURN(env, clazz, S_IRUSR);
+  SET_INT_OR_RETURN(env, clazz, S_IWUSR);
+  SET_INT_OR_RETURN(env, clazz, S_IXUSR);
+}
+#endif
+
+static void stat_init(JNIEnv *env, jclass nativeio_class) {
+  jclass clazz = NULL;
+  jclass obj_class = NULL;
+  jmethodID  obj_ctor = NULL;
+  // Init Stat
+  clazz = (*env)->FindClass(env, NATIVE_IO_STAT_CLASS);
+  if (!clazz) {
+    return; // exception has been raised
+  }
+  stat_clazz = (*env)->NewGlobalRef(env, clazz);
+  if (!stat_clazz) {
+    return; // exception has been raised
+  }
+  stat_ctor = (*env)->GetMethodID(env, stat_clazz, "<init>",
+    "(III)V");
+  if (!stat_ctor) {
+    return; // exception has been raised
+  }
+  stat_ctor2 = (*env)->GetMethodID(env, stat_clazz, "<init>",
+    "(Ljava/lang/String;Ljava/lang/String;I)V");
+  if (!stat_ctor2) {
+    return; // exception has been raised
+  }
+  obj_class = (*env)->FindClass(env, "java/lang/Object");
+  if (!obj_class) {
+    return; // exception has been raised
+  }
+  obj_ctor = (*env)->GetMethodID(env, obj_class,
+    "<init>", "()V");
+  if (!obj_ctor) {
+    return; // exception has been raised
+  }
+
+  if (workaround_non_threadsafe_calls(env, nativeio_class)) {
+    pw_lock_object = (*env)->NewObject(env, obj_class, obj_ctor);
+    PASS_EXCEPTIONS(env);
+    pw_lock_object = (*env)->NewGlobalRef(env, pw_lock_object);
+
+    PASS_EXCEPTIONS(env);
+  }
+}
+
+static void stat_deinit(JNIEnv *env) {
+  if (stat_clazz != NULL) {  
+    (*env)->DeleteGlobalRef(env, stat_clazz);
+    stat_clazz = NULL;
+  }
+  if (pw_lock_object != NULL) {
+    (*env)->DeleteGlobalRef(env, pw_lock_object);
+    pw_lock_object = NULL;
+  }
+}
+
+static void nioe_init(JNIEnv *env) {
+  // Init NativeIOException
+  nioe_clazz = (*env)->FindClass(
+    env, "org/apache/ratis/io/nativeio/NativeIOException");
+  PASS_EXCEPTIONS(env);
+
+  nioe_clazz = (*env)->NewGlobalRef(env, nioe_clazz);
+#ifdef UNIX
+  nioe_ctor = (*env)->GetMethodID(env, nioe_clazz, "<init>",
+    "(Ljava/lang/String;Lorg/apache/ratis/io/nativeio/Errno;)V");
+#endif
+
+#ifdef WINDOWS
+  nioe_ctor = (*env)->GetMethodID(env, nioe_clazz, "<init>",
+    "(Ljava/lang/String;I)V");
+#endif
+}
+
+static void nioe_deinit(JNIEnv *env) {
+  if (nioe_clazz != NULL) {
+    (*env)->DeleteGlobalRef(env, nioe_clazz);
+    nioe_clazz = NULL;
+  }
+  nioe_ctor = NULL;
+}
+
+/*
+ * private static native void initNative();
+ *
+ * We rely on this function rather than lazy initialization because
+ * the lazy approach may have a race if multiple callers try to
+ * init at the same time.
+ */
+JNIEXPORT void JNICALL
+Java_org_apache_ratis_io_nativeio_NativeIO_initNative(
+  JNIEnv *env, jclass clazz) {
+#ifdef UNIX
+  consts_init(env);
+  PASS_EXCEPTIONS_GOTO(env, error);
+#endif
+  stat_init(env, clazz);
+  PASS_EXCEPTIONS_GOTO(env, error);
+  nioe_init(env);
+  PASS_EXCEPTIONS_GOTO(env, error);
+  fd_init(env);
+  PASS_EXCEPTIONS_GOTO(env, error);
+#ifdef UNIX
+  errno_enum_init(env);
+  PASS_EXCEPTIONS_GOTO(env, error);
+#endif
+  return;
+error:
+  // these are all idempodent and safe to call even if the
+  // class wasn't initted yet
+#ifdef UNIX
+  stat_deinit(env);
+#endif
+  nioe_deinit(env);
+  fd_deinit(env);
+#ifdef UNIX
+  errno_enum_deinit(env);
+#endif
+}
+
+/*
+ * Class:     org_apache_ratis_io_nativeio_NativeIO_POSIX
+ * Method:    fstat
+ * Signature: (Ljava/io/FileDescriptor;)Lorg/apache/ratis/io/nativeio/NativeIO$POSIX$Stat;
+ * public static native Stat fstat(FileDescriptor fd);
+ *
+ * The "00024" in the function name is an artifact of how JNI encodes
+ * special characters. U+0024 is '$'.
+ */
+JNIEXPORT jobject JNICALL
+Java_org_apache_ratis_io_nativeio_NativeIO_00024POSIX_fstat(
+  JNIEnv *env, jclass clazz, jobject fd_object)
+{
+#ifdef UNIX
+  jobject ret = NULL;
+
+  int fd = fd_get(env, fd_object);
+  PASS_EXCEPTIONS_GOTO(env, cleanup);
+
+  struct stat s;
+  int rc = fstat(fd, &s);
+  if (rc != 0) {
+    throw_ioe(env, errno);
+    goto cleanup;
+  }
+
+  // Construct result
+  ret = (*env)->NewObject(env, stat_clazz, stat_ctor,
+    (jint)s.st_uid, (jint)s.st_gid, (jint)s.st_mode);
+
+cleanup:
+  return ret;
+#endif
+
+#ifdef WINDOWS
+  LPWSTR owner = NULL;
+  LPWSTR group = NULL;
+  int mode = 0;
+  jstring jstr_owner = NULL;
+  jstring jstr_group = NULL;
+  int rc;
+  jobject ret = NULL;
+  HANDLE hFile = (HANDLE) fd_get(env, fd_object);
+  PASS_EXCEPTIONS_GOTO(env, cleanup);
+
+  rc = FindFileOwnerAndPermissionByHandle(hFile, &owner, &group, &mode);
+  if (rc != ERROR_SUCCESS) {
+    throw_ioe(env, rc);
+    goto cleanup;
+  }
+
+  jstr_owner = (*env)->NewString(env, owner, (jsize) wcslen(owner));
+  if (jstr_owner == NULL) goto cleanup;
+
+  jstr_group = (*env)->NewString(env, group, (jsize) wcslen(group));;
+  if (jstr_group == NULL) goto cleanup;
+
+  ret = (*env)->NewObject(env, stat_clazz, stat_ctor2,
+    jstr_owner, jstr_group, (jint)mode);
+
+cleanup:
+  if (ret == NULL) {
+    if (jstr_owner != NULL)
+      (*env)->ReleaseStringChars(env, jstr_owner, owner);
+
+    if (jstr_group != NULL)
+      (*env)->ReleaseStringChars(env, jstr_group, group);
+  }
+
+  LocalFree(owner);
+  LocalFree(group);
+
+  return ret;
+#endif
+}
+
+
+
+/**
+ * public static native void posix_fadvise(
+ *   FileDescriptor fd, long offset, long len, int flags);
+ *
+ * The "00024" in the function name is an artifact of how JNI encodes
+ * special characters. U+0024 is '$'.
+ */
+JNIEXPORT void JNICALL
+Java_org_apache_ratis_io_nativeio_NativeIO_00024POSIX_posix_1fadvise(
+  JNIEnv *env, jclass clazz,
+  jobject fd_object, jlong offset, jlong len, jint flags)
+{
+#ifndef HAVE_POSIX_FADVISE
+  THROW(env, "java/lang/UnsupportedOperationException",
+        "fadvise support not available");
+#else
+  int fd = fd_get(env, fd_object);
+  PASS_EXCEPTIONS(env);
+
+  int err = 0;
+  if ((err = posix_fadvise(fd, (off_t)offset, (off_t)len, flags))) {
+#ifdef __FreeBSD__
+    throw_ioe(env, errno);
+#else
+    throw_ioe(env, err);
+#endif
+  }
+#endif
+}
+
+#if defined(HAVE_SYNC_FILE_RANGE)
+#  define my_sync_file_range sync_file_range
+#elif defined(SYS_sync_file_range)
+// RHEL 5 kernels have sync_file_range support, but the glibc
+// included does not have the library function. We can
+// still call it directly, and if it's not supported by the
+// kernel, we'd get ENOSYS. See RedHat Bugzilla #518581
+static int manual_sync_file_range (int fd, __off64_t from, __off64_t to, unsigned int flags)
+{
+#ifdef __x86_64__
+  return syscall( SYS_sync_file_range, fd, from, to, flags);
+#else
+  return syscall (SYS_sync_file_range, fd,
+    __LONG_LONG_PAIR ((long) (from >> 32), (long) from),
+    __LONG_LONG_PAIR ((long) (to >> 32), (long) to),
+    flags);
+#endif
+}
+#define my_sync_file_range manual_sync_file_range
+#endif
+
+/**
+ * public static native void sync_file_range(
+ *   FileDescriptor fd, long offset, long len, int flags);
+ *
+ * The "00024" in the function name is an artifact of how JNI encodes
+ * special characters. U+0024 is '$'.
+ */
+JNIEXPORT void JNICALL
+Java_org_apache_ratis_io_nativeio_NativeIO_00024POSIX_sync_1file_1range(
+  JNIEnv *env, jclass clazz,
+  jobject fd_object, jlong offset, jlong len, jint flags)
+{
+#ifndef my_sync_file_range
+  THROW(env, "java/lang/UnsupportedOperationException",
+        "sync_file_range support not available");
+#else
+  int fd = fd_get(env, fd_object);
+  PASS_EXCEPTIONS(env);
+
+  if (my_sync_file_range(fd, (off_t)offset, (off_t)len, flags)) {
+    if (errno == ENOSYS) {
+      // we know the syscall number, but it's not compiled
+      // into the running kernel
+      THROW(env, "java/lang/UnsupportedOperationException",
+            "sync_file_range kernel support not available");
+      return;
+    } else {
+      throw_ioe(env, errno);
+    }
+  }
+#endif
+}
+
+#define CHECK_DIRECT_BUFFER_ADDRESS(buf) \
+  { \
+    if (!buf) { \
+      THROW(env, "java/lang/UnsupportedOperationException", \
+        "JNI access to direct buffers not available"); \
+      return; \
+    } \
+  }
+
+/**
+ * public static native void mlock_native(
+ *   ByteBuffer buffer, long offset);
+ *
+ * The "00024" in the function name is an artifact of how JNI encodes
+ * special characters. U+0024 is '$'.
+ */
+JNIEXPORT void JNICALL
+Java_org_apache_ratis_io_nativeio_NativeIO_00024POSIX_mlock_1native(
+  JNIEnv *env, jclass clazz,
+  jobject buffer, jlong len)
+{
+  void* buf = (void*)(*env)->GetDirectBufferAddress(env, buffer);
+  PASS_EXCEPTIONS(env);
+
+#ifdef UNIX
+  if (mlock(buf, len)) {
+    CHECK_DIRECT_BUFFER_ADDRESS(buf);
+    throw_ioe(env, errno);
+  }
+#endif
+
+#ifdef WINDOWS
+  if (!VirtualLock(buf, len)) {
+    CHECK_DIRECT_BUFFER_ADDRESS(buf);
+    throw_ioe(env, GetLastError());
+  }
+#endif
+}
+
+/*
+ * Class:     org_apache_ratis_io_nativeio_NativeIO_POSIX
+ * Method:    open
+ * Signature: (Ljava/lang/String;II)Ljava/io/FileDescriptor;
+ * public static native FileDescriptor open(String path, int flags, int mode);
+ *
+ * The "00024" in the function name is an artifact of how JNI encodes
+ * special characters. U+0024 is '$'.
+ */
+JNIEXPORT jobject JNICALL
+Java_org_apache_ratis_io_nativeio_NativeIO_00024POSIX_open(
+  JNIEnv *env, jclass clazz, jstring j_path,
+  jint flags, jint mode)
+{
+#ifdef UNIX
+  jobject ret = NULL;
+
+  const char *path = (*env)->GetStringUTFChars(env, j_path, NULL);
+  if (path == NULL) goto cleanup; // JVM throws Exception for us
+
+  int fd;  
+  if (flags & O_CREAT) {
+    fd = open(path, flags, mode);
+  } else {
+    fd = open(path, flags);
+  }
+
+  if (fd == -1) {
+    throw_ioe(env, errno);
+    goto cleanup;
+  }
+
+  ret = fd_create(env, fd);
+
+cleanup:
+  if (path != NULL) {
+    (*env)->ReleaseStringUTFChars(env, j_path, path);
+  }
+  return ret;
+#endif
+
+#ifdef WINDOWS
+  THROW(env, "java/io/IOException",
+    "The function POSIX.open() is not supported on Windows");
+  return NULL;
+#endif
+}
+
+/*
+ * Class:     org_apache_ratis_io_nativeio_NativeIO_Windows
+ * Method:    createDirectoryWithMode0
+ * Signature: (Ljava/lang/String;I)V
+ *
+ * The "00024" in the function name is an artifact of how JNI encodes
+ * special characters. U+0024 is '$'.
+ */
+JNIEXPORT void JNICALL
+  Java_org_apache_ratis_io_nativeio_NativeIO_00024Windows_createDirectoryWithMode0
+  (JNIEnv *env, jclass clazz, jstring j_path, jint mode)
+{
+#ifdef WINDOWS
+  DWORD dwRtnCode = ERROR_SUCCESS;
+
+  LPCWSTR path = (LPCWSTR) (*env)->GetStringChars(env, j_path, NULL);
+  if (!path) {
+    goto done;
+  }
+
+  dwRtnCode = CreateDirectoryWithMode(path, mode);
+
+done:
+  if (path) {
+    (*env)->ReleaseStringChars(env, j_path, (const jchar*) path);
+  }
+  if (dwRtnCode != ERROR_SUCCESS) {
+    throw_ioe(env, dwRtnCode);
+  }
+#else
+  THROW(env, "java/io/IOException",
+    "The function Windows.createDirectoryWithMode0() is not supported on this platform");
+#endif
+}
+
+/*
+ * Class:     org_apache_ratis_io_nativeio_NativeIO_Windows
+ * Method:    createFileWithMode0
+ * Signature: (Ljava/lang/String;JJJI)Ljava/io/FileDescriptor;
+ *
+ * The "00024" in the function name is an artifact of how JNI encodes
+ * special characters. U+0024 is '$'.
+ */
+JNIEXPORT jobject JNICALL
+  Java_org_apache_ratis_io_nativeio_NativeIO_00024Windows_createFileWithMode0
+  (JNIEnv *env, jclass clazz, jstring j_path,
+  jlong desiredAccess, jlong shareMode, jlong creationDisposition, jint mode)
+{
+#ifdef WINDOWS
+  DWORD dwRtnCode = ERROR_SUCCESS;
+  HANDLE hFile = INVALID_HANDLE_VALUE;
+  jobject fd = NULL;
+
+  LPCWSTR path = (LPCWSTR) (*env)->GetStringChars(env, j_path, NULL);
+  if (!path) {
+    goto done;
+  }
+
+  dwRtnCode = CreateFileWithMode(path, desiredAccess, shareMode,
+      creationDisposition, mode, &hFile);
+  if (dwRtnCode != ERROR_SUCCESS) {
+    goto done;
+  }
+
+  fd = fd_create(env, (long) hFile);
+
+done:
+  if (path) {
+    (*env)->ReleaseStringChars(env, j_path, (const jchar*) path);
+  }
+  if (dwRtnCode != ERROR_SUCCESS) {
+    throw_ioe(env, dwRtnCode);
+  }
+  return fd;
+#else
+  THROW(env, "java/io/IOException",
+    "The function Windows.createFileWithMode0() is not supported on this platform");
+  return NULL;
+#endif
+}
+
+/*
+ * Class:     org_apache_ratis_io_nativeio_NativeIO_Windows
+ * Method:    createFile
+ * Signature: (Ljava/lang/String;JJJ)Ljava/io/FileDescriptor;
+ *
+ * The "00024" in the function name is an artifact of how JNI encodes
+ * special characters. U+0024 is '$'.
+ */
+JNIEXPORT jobject JNICALL Java_org_apache_ratis_io_nativeio_NativeIO_00024Windows_createFile
+  (JNIEnv *env, jclass clazz, jstring j_path,
+  jlong desiredAccess, jlong shareMode, jlong creationDisposition)
+{
+#ifdef UNIX
+  THROW(env, "java/io/IOException",
+    "The function Windows.createFile() is not supported on Unix");
+  return NULL;
+#endif
+
+#ifdef WINDOWS
+  DWORD dwRtnCode = ERROR_SUCCESS;
+  BOOL isSymlink = FALSE;
+  BOOL isJunction = FALSE;
+  DWORD dwFlagsAndAttributes = FILE_ATTRIBUTE_NORMAL | FILE_FLAG_BACKUP_SEMANTICS;
+  jobject ret = (jobject) NULL;
+  HANDLE hFile = INVALID_HANDLE_VALUE;
+  WCHAR *path = (WCHAR *) (*env)->GetStringChars(env, j_path, (jboolean*)NULL);
+  if (path == NULL) goto cleanup;
+
+  // Set the flag for a symbolic link or a junctions point only when it exists.
+  // According to MSDN if the call to CreateFile() function creates a file,
+  // there is no change in behavior. So we do not throw if no file is found.
+  //
+  dwRtnCode = SymbolicLinkCheck(path, &isSymlink);
+  if (dwRtnCode != ERROR_SUCCESS && dwRtnCode != ERROR_FILE_NOT_FOUND) {
+    throw_ioe(env, dwRtnCode);
+    goto cleanup;
+  }
+  dwRtnCode = JunctionPointCheck(path, &isJunction);
+  if (dwRtnCode != ERROR_SUCCESS && dwRtnCode != ERROR_FILE_NOT_FOUND) {
+    throw_ioe(env, dwRtnCode);
+    goto cleanup;
+  }
+  if (isSymlink || isJunction)
+    dwFlagsAndAttributes |= FILE_FLAG_OPEN_REPARSE_POINT;
+
+  hFile = CreateFile(path,
+    (DWORD) desiredAccess,
+    (DWORD) shareMode,
+    (LPSECURITY_ATTRIBUTES ) NULL,
+    (DWORD) creationDisposition,
+    dwFlagsAndAttributes,
+    NULL);
+  if (hFile == INVALID_HANDLE_VALUE) {
+    throw_ioe(env, GetLastError());
+    goto cleanup;
+  }
+
+  ret = fd_create(env, (long) hFile);
+cleanup:
+  if (path != NULL) {
+    (*env)->ReleaseStringChars(env, j_path, (const jchar*)path);
+  }
+  return (jobject) ret;
+#endif
+}
+
+/*
+ * Class:     org_apache_ratis_io_nativeio_NativeIO_POSIX
+ * Method:    chmod
+ * Signature: (Ljava/lang/String;I)V
+ */
+JNIEXPORT void JNICALL Java_org_apache_ratis_io_nativeio_NativeIO_00024POSIX_chmodImpl
+  (JNIEnv *env, jclass clazz, jstring j_path, jint mode)
+{
+#ifdef UNIX
+  const char *path = (*env)->GetStringUTFChars(env, j_path, NULL);
+  if (path == NULL) return; // JVM throws Exception for us
+
+  if (chmod(path, mode) != 0) {
+    throw_ioe(env, errno);
+  }
+
+  (*env)->ReleaseStringUTFChars(env, j_path, path);
+#endif
+
+#ifdef WINDOWS
+  DWORD dwRtnCode = ERROR_SUCCESS;
+  LPCWSTR path = (LPCWSTR) (*env)->GetStringChars(env, j_path, NULL);
+  if (path == NULL) return; // JVM throws Exception for us
+
+  if ((dwRtnCode = ChangeFileModeByMask((LPCWSTR) path, mode)) != ERROR_SUCCESS)
+  {
+    throw_ioe(env, dwRtnCode);
+  }
+
+  (*env)->ReleaseStringChars(env, j_path, (const jchar*) path);
+#endif
+}
+
+JNIEXPORT jlong JNICALL 
+Java_org_apache_ratis_io_nativeio_NativeIO_00024POSIX_mmap(
+  JNIEnv *env, jclass clazz, jobject jfd, jint jprot,
+  jboolean jshared, jlong length)
+{
+#ifdef UNIX
+  void *addr = 0;
+  int prot, flags, fd;
+  
+  prot = ((jprot & MMAP_PROT_READ) ? PROT_READ : 0) |
+         ((jprot & MMAP_PROT_WRITE) ? PROT_WRITE : 0) |
+         ((jprot & MMAP_PROT_EXEC) ? PROT_EXEC : 0);
+  flags = (jshared == JNI_TRUE) ? MAP_SHARED : MAP_PRIVATE;
+  fd = fd_get(env, jfd);
+  addr = mmap(NULL, length, prot, flags, fd, 0);
+  if (addr == MAP_FAILED) {
+    throw_ioe(env, errno);
+  }
+  return (jlong)(intptr_t)addr;
+#endif  //   UNIX
+
+#ifdef WINDOWS
+  THROW(env, "java/io/IOException",
+    "The function POSIX.mmap() is not supported on Windows");
+  return (jlong)(intptr_t)NULL;
+#endif
+}
+
+JNIEXPORT void JNICALL 
+Java_org_apache_ratis_io_nativeio_NativeIO_00024POSIX_munmap(
+  JNIEnv *env, jclass clazz, jlong jaddr, jlong length)
+{
+#ifdef UNIX
+  void *addr;
+
+  addr = (void*)(intptr_t)jaddr;
+  if (munmap(addr, length) < 0) {
+    throw_ioe(env, errno);
+  }
+#endif  //   UNIX
+
+#ifdef WINDOWS
+  THROW(env, "java/io/IOException",
+    "The function POSIX.munmap() is not supported on Windows");
+#endif
+}
+
+/*
+ * Throw a java.IO.IOException, generating the message from errno.
+ */
+void throw_ioe(JNIEnv* env, int errnum)
+{
+#ifdef UNIX
+  char message[80];
+  jstring jstr_message;
+
+  snprintf(message,sizeof(message),"%s",terror(errnum));
+
+  jobject errno_obj = errno_to_enum(env, errnum);
+
+  if ((jstr_message = (*env)->NewStringUTF(env, message)) == NULL)
+    goto err;
+
+  jthrowable obj = (jthrowable)(*env)->NewObject(env, nioe_clazz, nioe_ctor,
+    jstr_message, errno_obj);
+  if (obj == NULL) goto err;
+
+  (*env)->Throw(env, obj);
+  return;
+
+err:
+  if (jstr_message != NULL)
+    (*env)->ReleaseStringUTFChars(env, jstr_message, message);
+#endif
+
+#ifdef WINDOWS
+  DWORD len = 0;
+  LPWSTR buffer = NULL;
+  const jchar* message = NULL;
+  jstring jstr_message = NULL;
+  jthrowable obj = NULL;
+
+  len = FormatMessageW(
+    FORMAT_MESSAGE_ALLOCATE_BUFFER | FORMAT_MESSAGE_FROM_SYSTEM,
+    NULL, *(DWORD*) (&errnum), // reinterpret cast
+    MAKELANGID(LANG_NEUTRAL, SUBLANG_DEFAULT),
+    (LPWSTR) &buffer, 0, NULL);
+
+  if (len > 0)
+  {
+    message = (const jchar*) buffer;
+  }
+  else
+  {
+    message = (const jchar*) L"Unknown error.";
+  }
+
+  if ((jstr_message = (*env)->NewString(env, message, len)) == NULL)
+    goto err;
+  LocalFree(buffer);
+  buffer = NULL; // Set buffer to NULL to avoid double free
+
+  obj = (jthrowable)(*env)->NewObject(env, nioe_clazz, nioe_ctor,
+    jstr_message, errnum);
+  if (obj == NULL) goto err;
+
+  (*env)->Throw(env, obj);
+  return;
+
+err:
+  if (jstr_message != NULL)
+    (*env)->ReleaseStringChars(env, jstr_message, message);
+  LocalFree(buffer);
+  return;
+#endif
+}
+
+#ifdef UNIX
+/*
+ * Determine how big a buffer we need for reentrant getpwuid_r and getgrnam_r
+ */
+ssize_t get_pw_buflen() {
+  long ret = 0;
+  #ifdef _SC_GETPW_R_SIZE_MAX
+  ret = sysconf(_SC_GETPW_R_SIZE_MAX);
+  #endif
+  return (ret > 512) ? ret : 512;
+}
+#endif
+
+/*
+ * Class:     org_apache_ratis_io_nativeio_NativeIO_Windows
+ * Method:    setFilePointer
+ * Signature: (Ljava/io/FileDescriptor;JJ)J
+ *
+ * The "00024" in the function name is an artifact of how JNI encodes
+ * special characters. U+0024 is '$'.
+ */
+JNIEXPORT jlong JNICALL
+Java_org_apache_ratis_io_nativeio_NativeIO_00024Windows_setFilePointer
+  (JNIEnv *env, jclass clazz, jobject fd_object, jlong distanceToMove, jlong moveMethod)
+{
+#ifdef UNIX
+  THROW(env, "java/io/IOException",
+    "The function setFilePointer(FileDescriptor) is not supported on Unix");
+  return (jlong)(intptr_t)NULL;
+#endif
+
+#ifdef WINDOWS
+  DWORD distanceToMoveLow = (DWORD) distanceToMove;
+  LONG distanceToMoveHigh = (LONG) (distanceToMove >> 32);
+  DWORD distanceMovedLow = 0;
+  HANDLE hFile = (HANDLE) fd_get(env, fd_object);
+  PASS_EXCEPTIONS_GOTO(env, cleanup);
+
+  distanceMovedLow = SetFilePointer(hFile,
+    distanceToMoveLow, &distanceToMoveHigh, (DWORD) moveMethod);
+
+  if (distanceMovedLow == INVALID_SET_FILE_POINTER) {
+     throw_ioe(env, GetLastError());
+     return -1;
+  }
+
+cleanup:
+
+  return ((jlong) distanceToMoveHigh << 32) | (jlong) distanceMovedLow;
+#endif
+}
+
+/*
+ * Class:     org_apache_ratis_io_nativeio_NativeIO_Windows
+ * Method:    access0
+ * Signature: (Ljava/lang/String;I)Z
+ */
+JNIEXPORT jboolean JNICALL Java_org_apache_ratis_io_nativeio_NativeIO_00024Windows_access0
+  (JNIEnv *env, jclass clazz, jstring jpath, jint jaccess)
+{
+#ifdef UNIX
+  THROW(env, "java/io/IOException",
+    "The function access0(path, access) is not supported on Unix");
+  return (jlong)(intptr_t)NULL;
+#endif
+
+#ifdef WINDOWS
+  LPCWSTR path = NULL;
+  DWORD dwRtnCode = ERROR_SUCCESS;
+  ACCESS_MASK access = (ACCESS_MASK)jaccess;
+  BOOL allowed = FALSE;
+
+  path = (LPCWSTR) (*env)->GetStringChars(env, jpath, NULL);
+  if (!path) goto cleanup; // exception was thrown
+
+  dwRtnCode = CheckAccessForCurrentUser(path, access, &allowed);
+  if (dwRtnCode != ERROR_SUCCESS) {
+    throw_ioe(env, dwRtnCode);
+    goto cleanup;
+  }
+
+cleanup:
+  if (path) (*env)->ReleaseStringChars(env, jpath, path);
+
+  return (jboolean)allowed;
+#endif
+}
+
+/*
+ * Class:     org_apache_ratis_io_nativeio_NativeIO_Windows
+ * Method:    extendWorkingSetSize
+ * Signature: (J)V
+ *
+ * The "00024" in the function name is an artifact of how JNI encodes
+ * special characters. U+0024 is '$'.
+ */
+JNIEXPORT void JNICALL
+Java_org_apache_ratis_io_nativeio_NativeIO_00024Windows_extendWorkingSetSize(
+  JNIEnv *env, jclass clazz, jlong delta)
+{
+#ifdef UNIX
+  THROW(env, "java/io/IOException",
+    "The function extendWorkingSetSize(delta) is not supported on Unix");
+#endif
+
+#ifdef WINDOWS
+  SIZE_T min, max;
+  HANDLE hProcess = GetCurrentProcess();
+  if (!GetProcessWorkingSetSize(hProcess, &min, &max)) {
+    throw_ioe(env, GetLastError());
+    return;
+  }
+  if (!SetProcessWorkingSetSizeEx(hProcess, min + delta, max + delta,
+      QUOTA_LIMITS_HARDWS_MIN_DISABLE | QUOTA_LIMITS_HARDWS_MAX_DISABLE)) {
+    throw_ioe(env, GetLastError());
+    return;
+  }
+  // There is no need to call CloseHandle on the pseudo-handle returned from
+  // GetCurrentProcess.
+#endif
+}
+
+JNIEXPORT void JNICALL 
+Java_org_apache_ratis_io_nativeio_NativeIO_renameTo0(JNIEnv *env, 
+jclass clazz, jstring jsrc, jstring jdst)
+{
+#ifdef UNIX
+  const char *src = NULL, *dst = NULL;
+  
+  src = (*env)->GetStringUTFChars(env, jsrc, NULL);
+  if (!src) goto done; // exception was thrown
+  dst = (*env)->GetStringUTFChars(env, jdst, NULL);
+  if (!dst) goto done; // exception was thrown
+  if (rename(src, dst)) {
+    throw_ioe(env, errno);
+  }
+
+done:
+  if (src) (*env)->ReleaseStringUTFChars(env, jsrc, src);
+  if (dst) (*env)->ReleaseStringUTFChars(env, jdst, dst);
+#endif
+
+#ifdef WINDOWS
+  LPCWSTR src = NULL, dst = NULL;
+
+  src = (LPCWSTR) (*env)->GetStringChars(env, jsrc, NULL);
+  if (!src) goto done; // exception was thrown
+  dst = (LPCWSTR) (*env)->GetStringChars(env, jdst, NULL);
+  if (!dst) goto done; // exception was thrown
+  if (!MoveFile(src, dst)) {
+    throw_ioe(env, GetLastError());
+  }
+
+done:
+  if (src) (*env)->ReleaseStringChars(env, jsrc, src);
+  if (dst) (*env)->ReleaseStringChars(env, jdst, dst);
+#endif
+}
+
+JNIEXPORT jlong JNICALL
+Java_org_apache_ratis_io_nativeio_NativeIO_getMemlockLimit0(
+JNIEnv *env, jclass clazz)
+{
+#ifdef RLIMIT_MEMLOCK
+  struct rlimit rlim;
+  int rc = getrlimit(RLIMIT_MEMLOCK, &rlim);
+  if (rc != 0) {
+    throw_ioe(env, errno);
+    return 0;
+  }
+  return (rlim.rlim_cur == RLIM_INFINITY) ?
+    INT64_MAX : rlim.rlim_cur;
+#else
+  return 0;
+#endif
+}
+
+JNIEXPORT void JNICALL
+Java_org_apache_ratis_io_nativeio_NativeIO_copyFileUnbuffered0(
+JNIEnv *env, jclass clazz, jstring jsrc, jstring jdst)
+{
+#ifdef UNIX
+  THROW(env, "java/lang/UnsupportedOperationException",
+    "The function copyFileUnbuffered0 should not be used on Unix. Use FileChannel#transferTo instead.");
+#endif
+
+#ifdef WINDOWS
+  LPCWSTR src = NULL, dst = NULL;
+
+  src = (LPCWSTR) (*env)->GetStringChars(env, jsrc, NULL);
+  if (!src) goto cleanup; // exception was thrown
+  dst = (LPCWSTR) (*env)->GetStringChars(env, jdst, NULL);
+  if (!dst) goto cleanup; // exception was thrown
+  if (!CopyFileEx(src, dst, NULL, NULL, NULL, COPY_FILE_NO_BUFFERING)) {
+    throw_ioe(env, GetLastError());
+  }
+
+cleanup:
+  if (src) (*env)->ReleaseStringChars(env, jsrc, src);
+  if (dst) (*env)->ReleaseStringChars(env, jdst, dst);
+#endif
+}
+
+/**
+ * vim: sw=2: ts=2: et:
+ */


[17/54] [abbrv] incubator-ratis git commit: Renamed the packages from raft to ratis in preperation for Apache Incubation - Moved all java packages from org.apache.raft to org.apache.ratis. - Moved native package to org_apache_ratis, and native lib to l

Posted by ji...@apache.org.
http://git-wip-us.apache.org/repos/asf/incubator-ratis/blob/7e71a2e0/ratis-server/src/test/java/org/apache/ratis/RaftTestUtil.java
----------------------------------------------------------------------
diff --git a/ratis-server/src/test/java/org/apache/ratis/RaftTestUtil.java b/ratis-server/src/test/java/org/apache/ratis/RaftTestUtil.java
new file mode 100644
index 0000000..910ec6e
--- /dev/null
+++ b/ratis-server/src/test/java/org/apache/ratis/RaftTestUtil.java
@@ -0,0 +1,306 @@
+/**
+ * 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.ratis;
+
+import static org.apache.ratis.util.ProtoUtils.toByteString;
+
+import java.io.File;
+import java.io.IOException;
+import java.io.UnsupportedEncodingException;
+import java.nio.charset.Charset;
+import java.util.Arrays;
+import java.util.Collection;
+import java.util.function.BooleanSupplier;
+import java.util.function.IntSupplier;
+
+import org.apache.commons.lang.RandomStringUtils;
+import org.apache.ratis.protocol.Message;
+import org.apache.ratis.server.RaftServerConfigKeys;
+import org.apache.ratis.server.impl.BlockRequestHandlingInjection;
+import org.apache.ratis.server.impl.DelayLocalExecutionInjection;
+import org.apache.ratis.server.impl.RaftServerImpl;
+import org.apache.ratis.shaded.com.google.protobuf.ByteString;
+import org.apache.ratis.shaded.proto.RaftProtos.LogEntryProto;
+import org.apache.ratis.shaded.proto.RaftProtos.SMLogEntryProto;
+import org.apache.ratis.util.CheckedRunnable;
+import org.junit.Assert;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import com.google.common.base.Preconditions;
+
+public class RaftTestUtil {
+  static final Logger LOG = LoggerFactory.getLogger(RaftTestUtil.class);
+
+  public static RaftServerImpl waitForLeader(MiniRaftCluster cluster)
+      throws InterruptedException {
+    final long sleepTime = (cluster.getMaxTimeout() * 3) >> 1;
+    LOG.info(cluster.printServers());
+    RaftServerImpl leader = null;
+    for(int i = 0; leader == null && i < 10; i++) {
+      Thread.sleep(sleepTime);
+      leader = cluster.getLeader();
+    }
+    LOG.info(cluster.printServers());
+    return leader;
+  }
+
+  public static RaftServerImpl waitForLeader(MiniRaftCluster cluster,
+                                             final String leaderId) throws InterruptedException {
+    LOG.info(cluster.printServers());
+    for(int i = 0; !cluster.tryEnforceLeader(leaderId) && i < 10; i++) {
+      RaftServerImpl currLeader = cluster.getLeader();
+      if (LOG.isDebugEnabled()) {
+        LOG.debug("try enforcing leader to " + leaderId + " but "
+            + (currLeader == null? "no leader for this round"
+                : "new leader is " + currLeader.getId()));
+      }
+    }
+    LOG.info(cluster.printServers());
+
+    final RaftServerImpl leader = cluster.getLeader();
+    Assert.assertEquals(leaderId, leader.getId());
+    return leader;
+  }
+
+  public static String waitAndKillLeader(MiniRaftCluster cluster,
+      boolean expectLeader) throws InterruptedException {
+    final RaftServerImpl leader = waitForLeader(cluster);
+    if (!expectLeader) {
+      Assert.assertNull(leader);
+    } else {
+      Assert.assertNotNull(leader);
+      LOG.info("killing leader = " + leader);
+      cluster.killServer(leader.getId());
+    }
+    return leader != null ? leader.getId() : null;
+  }
+
+  public static boolean logEntriesContains(LogEntryProto[] entries,
+      SimpleMessage... expectedMessages) {
+    int idxEntries = 0;
+    int idxExpected = 0;
+    while (idxEntries < entries.length
+        && idxExpected < expectedMessages.length) {
+      if (Arrays.equals(expectedMessages[idxExpected].getContent().toByteArray(),
+          entries[idxEntries].getSmLogEntry().getData().toByteArray())) {
+        ++idxExpected;
+      }
+      ++idxEntries;
+    }
+    return idxExpected == expectedMessages.length;
+  }
+
+  public static void assertLogEntries(Collection<RaftServerImpl> servers,
+                                      SimpleMessage... expectedMessages) {
+    final int size = servers.size();
+    final long count = servers.stream()
+        .filter(RaftServerImpl::isAlive)
+        .map(s -> s.getState().getLog().getEntries(0, Long.MAX_VALUE))
+        .filter(e -> logEntriesContains(e, expectedMessages))
+        .count();
+    if (2*count <= size) {
+      throw new AssertionError("Not in majority: size=" + size
+          + " but count=" + count);
+    }
+  }
+
+  public static void assertLogEntries(LogEntryProto[] entries, long startIndex,
+      long expertedTerm, SimpleMessage... expectedMessages) {
+    Assert.assertEquals(expectedMessages.length, entries.length);
+    for(int i = 0; i < entries.length; i++) {
+      final LogEntryProto e = entries[i];
+      Assert.assertEquals(expertedTerm, e.getTerm());
+      Assert.assertEquals(startIndex + i, e.getIndex());
+      Assert.assertArrayEquals(expectedMessages[i].getContent().toByteArray(),
+          e.getSmLogEntry().getData().toByteArray());
+    }
+  }
+
+  public static class SimpleMessage implements Message {
+    public static SimpleMessage[] create(int numMessages) {
+      return create(numMessages, "m");
+    }
+
+    public static SimpleMessage[] create(int numMessages, String prefix) {
+      final SimpleMessage[] messages = new SimpleMessage[numMessages];
+      for (int i = 0; i < messages.length; i++) {
+        messages[i] = new SimpleMessage(prefix + i);
+      }
+      return messages;
+    }
+
+    final String messageId;
+
+    public SimpleMessage(final String messageId) {
+      this.messageId = messageId;
+    }
+
+    @Override
+    public String toString() {
+      return messageId;
+    }
+
+    @Override
+    public boolean equals(Object obj) {
+      if (obj == this) {
+        return true;
+      } else if (obj == null || !(obj instanceof SimpleMessage)) {
+        return false;
+      } else {
+        final SimpleMessage that = (SimpleMessage)obj;
+        return this.messageId.equals(that.messageId);
+      }
+    }
+
+    @Override
+    public int hashCode() {
+      return messageId.hashCode();
+    }
+
+    @Override
+    public ByteString getContent() {
+      return toByteString(messageId.getBytes(Charset.forName("UTF-8")));
+    }
+  }
+
+  public static class SimpleOperation {
+    private final String op;
+
+    public SimpleOperation(String op) {
+      Preconditions.checkArgument(op != null);
+      this.op = op;
+    }
+
+    @Override
+    public String toString() {
+      return op;
+    }
+
+    @Override
+    public boolean equals(Object obj) {
+      return obj == this ||
+          (obj instanceof SimpleOperation &&
+              ((SimpleOperation) obj).op.equals(op));
+    }
+
+    @Override
+    public int hashCode() {
+      return op.hashCode();
+    }
+
+    public SMLogEntryProto getLogEntryContent() {
+      try {
+        return SMLogEntryProto.newBuilder()
+            .setData(toByteString(op.getBytes("UTF-8"))).build();
+      } catch (UnsupportedEncodingException e) {
+        throw new RuntimeException(e);
+      }
+    }
+  }
+
+  public static File getTestDir(Class<?> caller) throws IOException {
+    File dir = new File(System.getProperty("test.build.data", "target/test/data")
+            + "/" + RandomStringUtils.randomAlphanumeric(10),
+            caller.getSimpleName());
+    if (dir.exists() && !dir.isDirectory()) {
+      throw new IOException(dir + " already exists and is not a directory");
+    } else if (!dir.exists() && !dir.mkdirs()) {
+      throw new IOException("Cannot create directory " + dir);
+    }
+    return dir;
+  }
+
+  public static void block(BooleanSupplier isBlocked) throws InterruptedException {
+    for(; isBlocked.getAsBoolean(); ) {
+      Thread.sleep(RaftServerConfigKeys.RAFT_SERVER_RPC_TIMEOUT_MAX_MS_DEFAULT);
+    }
+  }
+
+  public static void delay(IntSupplier getDelayMs) throws InterruptedException {
+    final int t = getDelayMs.getAsInt();
+    if (t > 0) {
+      Thread.sleep(t);
+    }
+  }
+
+  public static <T extends Throwable> void attempt(
+      int n, long sleepMs, CheckedRunnable<T> runnable)
+      throws T, InterruptedException {
+    for(int i = 1; i <= n; i++) {
+      LOG.info("Attempt #" + i + "/" + n +  ": sleep " + sleepMs + "ms");
+      if (sleepMs > 0) {
+        Thread.sleep(sleepMs);
+      }
+      try {
+        runnable.run();
+        return;
+      } catch (Throwable t) {
+        if (i == n) {
+          throw t;
+        }
+        LOG.warn("Attempt #" + i + "/" + n + ": Ignoring " + t + " and retry.");
+      }
+    }
+  }
+
+  public static String changeLeader(MiniRaftCluster cluster, String oldLeader)
+      throws InterruptedException {
+    cluster.setBlockRequestsFrom(oldLeader, true);
+    String newLeader = oldLeader;
+    for(int i = 0; i < 10 && newLeader.equals(oldLeader); i++) {
+      newLeader = RaftTestUtil.waitForLeader(cluster).getId();
+    }
+    cluster.setBlockRequestsFrom(oldLeader, false);
+    return newLeader;
+  }
+
+  public static void blockQueueAndSetDelay(Collection<RaftServerImpl> servers,
+      DelayLocalExecutionInjection injection, String leaderId, int delayMs,
+      long maxTimeout) throws InterruptedException {
+    // block reqeusts sent to leader if delayMs > 0
+    final boolean block = delayMs > 0;
+    LOG.debug("{} requests sent to leader {} and set {}ms delay for the others",
+        block? "Block": "Unblock", leaderId, delayMs);
+    if (block) {
+      BlockRequestHandlingInjection.getInstance().blockReplier(leaderId);
+    } else {
+      BlockRequestHandlingInjection.getInstance().unblockReplier(leaderId);
+    }
+
+    // delay RaftServerRequest for other servers
+    servers.stream().filter(s -> !s.getId().equals(leaderId))
+        .forEach(s -> {
+          if (block) {
+            injection.setDelayMs(s.getId(), delayMs);
+          } else {
+            injection.removeDelay(s.getId());
+          }
+        });
+
+    Thread.sleep(3 * maxTimeout);
+  }
+
+  public static void setBlockRequestsFrom(String src, boolean block) {
+    if (block) {
+      BlockRequestHandlingInjection.getInstance().blockRequestor(src);
+    } else {
+      BlockRequestHandlingInjection.getInstance().unblockRequestor(src);
+    }
+  }
+}

http://git-wip-us.apache.org/repos/asf/incubator-ratis/blob/7e71a2e0/ratis-server/src/test/java/org/apache/ratis/server/impl/BlockRequestHandlingInjection.java
----------------------------------------------------------------------
diff --git a/ratis-server/src/test/java/org/apache/ratis/server/impl/BlockRequestHandlingInjection.java b/ratis-server/src/test/java/org/apache/ratis/server/impl/BlockRequestHandlingInjection.java
new file mode 100644
index 0000000..a7f1b6d
--- /dev/null
+++ b/ratis-server/src/test/java/org/apache/ratis/server/impl/BlockRequestHandlingInjection.java
@@ -0,0 +1,85 @@
+/**
+ * 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.ratis.server.impl;
+
+import org.apache.ratis.RaftTestUtil;
+import org.apache.ratis.server.impl.RaftServerImpl;
+import org.apache.ratis.util.CodeInjectionForTesting;
+
+import java.util.Map;
+import java.util.concurrent.ConcurrentHashMap;
+
+/** Inject code to block a server from handling incoming requests. */
+public class BlockRequestHandlingInjection implements CodeInjectionForTesting.Code {
+  private static final BlockRequestHandlingInjection INSTANCE =
+      new BlockRequestHandlingInjection();
+
+  static {
+    CodeInjectionForTesting.put(RaftServerImpl.REQUEST_VOTE, INSTANCE);
+    CodeInjectionForTesting.put(RaftServerImpl.APPEND_ENTRIES, INSTANCE);
+    CodeInjectionForTesting.put(RaftServerImpl.INSTALL_SNAPSHOT, INSTANCE);
+  }
+
+  public static BlockRequestHandlingInjection getInstance() {
+    return INSTANCE;
+  }
+
+  private final Map<String, Boolean> requestors = new ConcurrentHashMap<>();
+  private final Map<String, Boolean> repliers = new ConcurrentHashMap<>();
+
+  private BlockRequestHandlingInjection() {}
+
+  public void blockRequestor(String requestor) {
+    requestors.put(requestor, true);
+  }
+
+  public void unblockRequestor(String requestor) {
+    requestors.remove(requestor);
+  }
+
+  public void blockReplier(String replier) {
+    repliers.put(replier, true);
+  }
+
+  public void unblockReplier(String replier) {
+    repliers.remove(replier);
+  }
+
+  public void unblockAll() {
+    requestors.clear();
+    repliers.clear();
+  }
+
+  @Override
+  public boolean execute(String localId, String remoteId, Object... args) {
+    if (shouldBlock(localId, remoteId)) {
+      try {
+        RaftTestUtil.block(() -> shouldBlock(localId, remoteId));
+        return true;
+      } catch (InterruptedException e) {
+        LOG.debug("Interrupted while blocking request handling from " + remoteId
+            + " to " + localId);
+      }
+    }
+    return false;
+  }
+
+  private boolean shouldBlock(String localId, String remoteId) {
+    return repliers.containsKey(localId) || requestors.containsKey(remoteId);
+  }
+}

http://git-wip-us.apache.org/repos/asf/incubator-ratis/blob/7e71a2e0/ratis-server/src/test/java/org/apache/ratis/server/impl/DelayLocalExecutionInjection.java
----------------------------------------------------------------------
diff --git a/ratis-server/src/test/java/org/apache/ratis/server/impl/DelayLocalExecutionInjection.java b/ratis-server/src/test/java/org/apache/ratis/server/impl/DelayLocalExecutionInjection.java
new file mode 100644
index 0000000..8de2474
--- /dev/null
+++ b/ratis-server/src/test/java/org/apache/ratis/server/impl/DelayLocalExecutionInjection.java
@@ -0,0 +1,67 @@
+/**
+ * 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.ratis.server.impl;
+
+import org.apache.ratis.RaftTestUtil;
+import org.apache.ratis.util.CodeInjectionForTesting;
+
+import java.util.Arrays;
+import java.util.Map;
+import java.util.concurrent.ConcurrentHashMap;
+import java.util.concurrent.atomic.AtomicInteger;
+
+/** Inject code to delay particular servers. */
+public class DelayLocalExecutionInjection implements CodeInjectionForTesting.Code {
+  private final Map<String, AtomicInteger> delays = new ConcurrentHashMap<>();
+
+  public DelayLocalExecutionInjection(String method) {
+    CodeInjectionForTesting.put(method, this);
+  }
+
+  public void clear() {
+    delays.clear();
+  }
+
+  public void setDelayMs(String id, int delayMs) {
+    AtomicInteger d = delays.get(id);
+    if (d == null) {
+      delays.put(id, d = new AtomicInteger());
+    }
+    d.set(delayMs);
+  }
+
+  public void removeDelay(String id) {
+    delays.remove(id);
+  }
+
+  @Override
+  public boolean execute(String localId, String remoteId, Object... args) {
+    final AtomicInteger d = delays.get(localId);
+    if (d == null) {
+      return false;
+    }
+    LOG.info("{} delay {} ms, args={}", localId, d.get(),
+        Arrays.toString(args));
+    try {
+      RaftTestUtil.delay(d::get);
+    } catch (InterruptedException e) {
+      LOG.debug("Interrupted while delaying " + localId);
+    }
+    return true;
+  }
+}

http://git-wip-us.apache.org/repos/asf/incubator-ratis/blob/7e71a2e0/ratis-server/src/test/java/org/apache/ratis/server/impl/RaftReconfigurationBaseTest.java
----------------------------------------------------------------------
diff --git a/ratis-server/src/test/java/org/apache/ratis/server/impl/RaftReconfigurationBaseTest.java b/ratis-server/src/test/java/org/apache/ratis/server/impl/RaftReconfigurationBaseTest.java
new file mode 100644
index 0000000..30b334f
--- /dev/null
+++ b/ratis-server/src/test/java/org/apache/ratis/server/impl/RaftReconfigurationBaseTest.java
@@ -0,0 +1,581 @@
+/**
+ * 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.ratis.server.impl;
+
+import static java.util.Arrays.asList;
+import static org.apache.ratis.MiniRaftCluster.logSyncDelay;
+import static org.apache.ratis.server.impl.RaftServerConstants.DEFAULT_SEQNUM;
+import static org.apache.ratis.server.impl.RaftServerTestUtil.waitAndCheckNewConf;
+import static org.apache.ratis.shaded.proto.RaftProtos.LogEntryProto.LogEntryBodyCase.CONFIGURATIONENTRY;
+
+import java.io.IOException;
+import java.util.ArrayList;
+import java.util.Arrays;
+import java.util.Collections;
+import java.util.concurrent.CountDownLatch;
+import java.util.concurrent.atomic.AtomicBoolean;
+import java.util.concurrent.atomic.AtomicReference;
+
+import org.apache.log4j.Level;
+import org.apache.ratis.MiniRaftCluster;
+import org.apache.ratis.MiniRaftCluster.PeerChanges;
+import org.apache.ratis.RaftTestUtil;
+import org.apache.ratis.RaftTestUtil.SimpleMessage;
+import org.apache.ratis.client.RaftClient;
+import org.apache.ratis.client.RaftClientRequestSender;
+import org.apache.ratis.client.impl.RaftClientImpl;
+import org.apache.ratis.conf.RaftProperties;
+import org.apache.ratis.protocol.RaftClientReply;
+import org.apache.ratis.protocol.RaftPeer;
+import org.apache.ratis.protocol.ReconfigurationInProgressException;
+import org.apache.ratis.protocol.ReconfigurationTimeoutException;
+import org.apache.ratis.protocol.SetConfigurationRequest;
+import org.apache.ratis.server.RaftServerConfigKeys;
+import org.apache.ratis.server.simulation.RequestHandler;
+import org.apache.ratis.server.storage.RaftLog;
+import org.apache.ratis.util.RaftUtils;
+import org.junit.Assert;
+import org.junit.BeforeClass;
+import org.junit.Test;
+import org.mockito.internal.util.reflection.Whitebox;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+public abstract class RaftReconfigurationBaseTest {
+  static {
+    RaftUtils.setLogLevel(RaftServerImpl.LOG, Level.DEBUG);
+    RaftUtils.setLogLevel(RequestHandler.LOG, Level.DEBUG);
+    RaftUtils.setLogLevel(RaftClient.LOG, Level.DEBUG);
+  }
+  static final Logger LOG = LoggerFactory.getLogger(RaftReconfigurationBaseTest.class);
+
+  protected static final RaftProperties prop = new RaftProperties();
+
+  @BeforeClass
+  public static void setup() {
+    // set a small gap for tests
+    prop.setInt(RaftServerConfigKeys.RAFT_SERVER_STAGING_CATCHUP_GAP_KEY, 10);
+  }
+
+  public abstract MiniRaftCluster getCluster(int peerNum) throws IOException;
+
+  private static int getStagingGap() {
+    return prop.getInt(RaftServerConfigKeys.RAFT_SERVER_STAGING_CATCHUP_GAP_KEY,
+        RaftServerConfigKeys.RAFT_SERVER_STAGING_CATCHUP_GAP_DEFAULT);
+  }
+
+  /**
+   * add 2 new peers (3 peers -> 5 peers), no leader change
+   */
+  @Test
+  public void testAddPeers() throws Exception {
+    LOG.info("Start testAddPeers");
+    MiniRaftCluster cluster = getCluster(3);
+    cluster.start();
+    try {
+      RaftTestUtil.waitForLeader(cluster);
+
+      // add new peers
+      RaftPeer[] allPeers = cluster.addNewPeers(2, true).allPeersInNewConf;
+
+      // trigger setConfiguration
+      SetConfigurationRequest request = new SetConfigurationRequest("client",
+          cluster.getLeader().getId(), DEFAULT_SEQNUM, allPeers);
+      LOG.info("Start changing the configuration: {}", request);
+      cluster.getLeader().setConfiguration(request);
+
+      // wait for the new configuration to take effect
+      waitAndCheckNewConf(cluster, allPeers, 0, null);
+    } finally {
+      cluster.shutdown();
+    }
+  }
+
+  /**
+   * remove 2 peers (5 peers -> 3 peers), no leader change
+   */
+  @Test
+  public void testRemovePeers() throws Exception {
+    LOG.info("Start testRemovePeers");
+    MiniRaftCluster cluster = getCluster(5);
+    cluster.start();
+    try {
+      RaftTestUtil.waitForLeader(cluster);
+
+      // remove peers, leader still included in the new conf
+      RaftPeer[] allPeers = cluster
+          .removePeers(2, false, Collections.emptyList()).allPeersInNewConf;
+
+      // trigger setConfiguration
+      SetConfigurationRequest request = new SetConfigurationRequest("client",
+          cluster.getLeader().getId(), DEFAULT_SEQNUM, allPeers);
+      LOG.info("Start changing the configuration: {}", request);
+      cluster.getLeader().setConfiguration(request);
+
+      // wait for the new configuration to take effect
+      waitAndCheckNewConf(cluster, allPeers, 2, null);
+    } finally {
+      cluster.shutdown();
+    }
+  }
+
+  /**
+   * 5 peers -> 5 peers, remove 2 old, add 2 new, no leader change
+   */
+  @Test
+  public void testAddRemovePeers() throws Exception {
+    LOG.info("Start testAddRemovePeers");
+    testAddRemovePeers(false);
+  }
+
+  @Test
+  public void testLeaderStepDown() throws Exception {
+    LOG.info("Start testLeaderStepDown");
+    testAddRemovePeers(true);
+  }
+
+  private void testAddRemovePeers(boolean leaderStepdown) throws Exception {
+    MiniRaftCluster cluster = getCluster(5);
+    cluster.start();
+    try {
+      RaftTestUtil.waitForLeader(cluster);
+
+      PeerChanges change = cluster.addNewPeers(2, true);
+      RaftPeer[] allPeers = cluster.removePeers(2, leaderStepdown,
+          asList(change.newPeers)).allPeersInNewConf;
+
+      // trigger setConfiguration
+      SetConfigurationRequest request = new SetConfigurationRequest("client",
+          cluster.getLeader().getId(), DEFAULT_SEQNUM, allPeers);
+      LOG.info("Start changing the configuration: {}", request);
+      cluster.getLeader().setConfiguration(request);
+
+      // wait for the new configuration to take effect
+      waitAndCheckNewConf(cluster, allPeers, 2, null);
+    } finally {
+      cluster.shutdown();
+    }
+  }
+
+  @Test(timeout = 30000)
+  public void testReconfTwice() throws Exception {
+    LOG.info("Start testReconfTwice");
+    final MiniRaftCluster cluster = getCluster(3);
+    cluster.start();
+    try {
+      RaftTestUtil.waitForLeader(cluster);
+      final String leaderId = cluster.getLeader().getId();
+      final RaftClient client = cluster.createClient("client", leaderId);
+
+      // submit some msgs before reconf
+      for (int i = 0; i < getStagingGap() * 2; i++) {
+        RaftClientReply reply = client.send(new SimpleMessage("m" + i));
+        Assert.assertTrue(reply.isSuccess());
+      }
+
+      final AtomicBoolean reconf1 = new AtomicBoolean(false);
+      final AtomicBoolean reconf2 = new AtomicBoolean(false);
+      final AtomicReference<RaftPeer[]> finalPeers = new AtomicReference<>(null);
+      final AtomicReference<RaftPeer[]> deadPeers = new AtomicReference<>(null);
+      CountDownLatch latch = new CountDownLatch(1);
+      Thread clientThread = new Thread(() -> {
+        try {
+          PeerChanges c1 = cluster.addNewPeers(2, true);
+          LOG.info("Start changing the configuration: {}",
+              asList(c1.allPeersInNewConf));
+
+          RaftClientReply reply = client.setConfiguration(c1.allPeersInNewConf);
+          reconf1.set(reply.isSuccess());
+
+          PeerChanges c2 = cluster.removePeers(2, true, asList(c1.newPeers));
+          finalPeers.set(c2.allPeersInNewConf);
+          deadPeers.set(c2.removedPeers);
+
+          LOG.info("Start changing the configuration again: {}",
+              asList(c2.allPeersInNewConf));
+          reply = client.setConfiguration(c2.allPeersInNewConf);
+          reconf2.set(reply.isSuccess());
+
+          latch.countDown();
+          client.close();
+        } catch (IOException ignored) {
+        }
+      });
+      clientThread.start();
+
+      latch.await();
+      Assert.assertTrue(reconf1.get());
+      Assert.assertTrue(reconf2.get());
+      waitAndCheckNewConf(cluster, finalPeers.get(), 2, null);
+
+      // check configuration manager's internal state
+      // each reconf will generate two configurations: (old, new) and (new)
+      cluster.getServers().stream().filter(RaftServerImpl::isAlive)
+          .forEach(server -> {
+        ConfigurationManager confManager =
+            (ConfigurationManager) Whitebox.getInternalState(server.getState(),
+                "configurationManager");
+        // each reconf will generate two configurations: (old, new) and (new)
+        Assert.assertEquals(5, confManager.numOfConf());
+      });
+    } finally {
+      cluster.shutdown();
+    }
+  }
+
+  @Test
+  public void testReconfTimeout() throws Exception {
+    LOG.info("Start testReconfTimeout");
+    // originally 3 peers
+    final MiniRaftCluster cluster = getCluster(3);
+    cluster.start();
+    try {
+      RaftTestUtil.waitForLeader(cluster);
+      final String leaderId = cluster.getLeader().getId();
+      final RaftClient client = cluster.createClient("client", leaderId);
+
+      PeerChanges c1 = cluster.addNewPeers(2, false);
+
+      LOG.info("Start changing the configuration: {}",
+          asList(c1.allPeersInNewConf));
+      Assert.assertFalse(cluster.getLeader().getRaftConf().isTransitional());
+
+      final RaftClientRequestSender sender = ((RaftClientImpl)client).getRequestSender();
+      final SetConfigurationRequest request = new SetConfigurationRequest(
+          "client", leaderId, DEFAULT_SEQNUM, c1.allPeersInNewConf);
+      try {
+        sender.sendRequest(request);
+        Assert.fail("did not get expected exception");
+      } catch (IOException e) {
+        Assert.assertTrue("Got exception " + e,
+            e instanceof ReconfigurationTimeoutException);
+      }
+
+      // the two new peers have not started yet, the bootstrapping must timeout
+      LOG.info(cluster.printServers());
+
+      // resend the same request, make sure the server has correctly reset its
+      // state so that we still get timeout instead of in-progress exception
+      try {
+        sender.sendRequest(request);
+        Assert.fail("did not get expected exception");
+      } catch (IOException e) {
+        Assert.assertTrue("Got exception " + e,
+            e instanceof ReconfigurationTimeoutException);
+      }
+
+      // start the two new peers
+      LOG.info("Start new peers");
+      for (RaftPeer np : c1.newPeers) {
+        cluster.startServer(np.getId());
+      }
+      Assert.assertTrue(client.setConfiguration(c1.allPeersInNewConf).isSuccess());
+      client.close();
+    } finally {
+      cluster.shutdown();
+    }
+  }
+
+  @Test
+  public void testBootstrapReconf() throws Exception {
+    LOG.info("Start testBootstrapReconf");
+    // originally 3 peers
+    final MiniRaftCluster cluster = getCluster(3);
+    cluster.start();
+    try {
+      RaftTestUtil.waitForLeader(cluster);
+      final String leaderId = cluster.getLeader().getId();
+      final RaftClient client = cluster.createClient("client", leaderId);
+
+      // submit some msgs before reconf
+      for (int i = 0; i < getStagingGap() * 2; i++) {
+        RaftClientReply reply = client.send(new SimpleMessage("m" + i));
+        Assert.assertTrue(reply.isSuccess());
+      }
+
+      PeerChanges c1 = cluster.addNewPeers(2, true);
+      LOG.info("Start changing the configuration: {}",
+          asList(c1.allPeersInNewConf));
+      final AtomicReference<Boolean> success = new AtomicReference<>();
+
+      Thread clientThread = new Thread(() -> {
+        try {
+          RaftClientReply reply = client.setConfiguration(c1.allPeersInNewConf);
+          success.set(reply.isSuccess());
+          client.close();
+        } catch (IOException ioe) {
+          LOG.error("FAILED", ioe);
+        }
+      });
+      clientThread.start();
+
+      Thread.sleep(5000);
+      LOG.info(cluster.printServers());
+      assertSuccess(success);
+
+      final RaftLog leaderLog = cluster.getLeader().getState().getLog();
+      for (RaftPeer newPeer : c1.newPeers) {
+        Assert.assertArrayEquals(leaderLog.getEntries(0, Long.MAX_VALUE),
+            cluster.getServer(newPeer.getId()).getState().getLog()
+                .getEntries(0, Long.MAX_VALUE));
+      }
+    } finally {
+      cluster.shutdown();
+    }
+  }
+
+  /**
+   * kill the leader before reconfiguration finishes. Make sure the client keeps
+   * retrying.
+   */
+  @Test
+  public void testKillLeaderDuringReconf() throws Exception {
+    LOG.info("Start testKillLeaderDuringReconf");
+    // originally 3 peers
+    final MiniRaftCluster cluster = getCluster(3);
+    cluster.start();
+    try {
+      RaftTestUtil.waitForLeader(cluster);
+      final String leaderId = cluster.getLeader().getId();
+      final RaftClient client = cluster.createClient("client", leaderId);
+
+      PeerChanges c1 = cluster.addNewPeers(2, false);
+      PeerChanges c2 = cluster.removePeers(2, false, asList(c1.newPeers));
+
+      LOG.info("Start changing the configuration: {}",
+          asList(c2.allPeersInNewConf));
+      final AtomicReference<Boolean> success = new AtomicReference<>();
+      final AtomicBoolean clientRunning = new AtomicBoolean(true);
+      Thread clientThread = new Thread(() -> {
+        try {
+          boolean r = false;
+          while (clientRunning.get() && !r) {
+            r = client.setConfiguration(c2.allPeersInNewConf).isSuccess();
+          }
+          success.set(r);
+          client.close();
+        } catch (IOException ignored) {
+        }
+      });
+      clientThread.start();
+
+      // the leader cannot generate the (old, new) conf, and it will keep
+      // bootstrapping the 2 new peers since they have not started yet
+      LOG.info(cluster.printServers());
+      Assert.assertFalse(cluster.getLeader().getRaftConf().isTransitional());
+
+      // only the first empty entry got committed
+      final long committedIndex = cluster.getLeader().getState().getLog()
+          .getLastCommittedIndex();
+      Assert.assertTrue("committedIndex is " + committedIndex,
+          committedIndex <= 1);
+
+      LOG.info("kill the current leader");
+      final String oldLeaderId = RaftTestUtil.waitAndKillLeader(cluster, true);
+      LOG.info("start the two new peers: {}", Arrays.asList(c1.newPeers));
+      for (RaftPeer np : c1.newPeers) {
+        cluster.startServer(np.getId());
+      }
+
+      Thread.sleep(3000);
+      // the client should get the NotLeaderException from the first leader, and
+      // will retry the same setConfiguration request
+      waitAndCheckNewConf(cluster, c2.allPeersInNewConf, 2,
+          Collections.singletonList(oldLeaderId));
+      clientRunning.set(false);
+      //Assert.assertTrue(success.get());
+    } finally {
+      cluster.shutdown();
+    }
+  }
+
+  static void assertSuccess(final AtomicReference<Boolean> success) {
+    final String s = "success=" + success;
+    Assert.assertNotNull(s, success.get());
+    Assert.assertTrue(s, success.get());
+  }
+
+  /**
+   * When a request's new configuration is the same with the current one, make
+   * sure we return success immediately and no log entry is recorded.
+   */
+  @Test
+  public void testNoChangeRequest() throws Exception {
+    LOG.info("Start testNoChangeRequest");
+    // originally 3 peers
+    final MiniRaftCluster cluster = getCluster(3);
+    try {
+      cluster.start();
+      RaftTestUtil.waitForLeader(cluster);
+
+      final String leaderId = cluster.getLeader().getId();
+      final RaftClient client = cluster.createClient("client", leaderId);
+      client.send(new SimpleMessage("m"));
+
+      final long committedIndex = cluster.getLeader().getState().getLog()
+          .getLastCommittedIndex();
+      final RaftConfiguration confBefore = cluster.getLeader().getRaftConf();
+
+      // no real configuration change in the request
+      RaftClientReply reply = client.setConfiguration(cluster.getPeers()
+          .toArray(new RaftPeer[0]));
+      Assert.assertTrue(reply.isSuccess());
+      Assert.assertEquals(committedIndex, cluster.getLeader().getState()
+          .getLog().getLastCommittedIndex());
+      Assert.assertSame(confBefore, cluster.getLeader().getRaftConf());
+      client.close();
+    } finally {
+      cluster.shutdown();
+    }
+  }
+
+  /**
+   * Make sure a setConfiguration request is rejected if a configuration change
+   * is still in progress (i.e., has not been committed yet).
+   */
+  @Test
+  public void testOverlappedSetConfRequests() throws Exception {
+    LOG.info("Start testOverlappedSetConfRequests");
+    // originally 3 peers
+    final MiniRaftCluster cluster = getCluster(3);
+    try {
+      cluster.start();
+      RaftTestUtil.waitForLeader(cluster);
+
+      final String leaderId = cluster.getLeader().getId();
+
+      RaftPeer[] newPeers = cluster.addNewPeers(2, true).allPeersInNewConf;
+
+      // delay every peer's logSync so that the setConf request is delayed
+      cluster.getPeers()
+          .forEach(peer -> logSyncDelay.setDelayMs(peer.getId(), 1000));
+
+      final CountDownLatch latch = new CountDownLatch(1);
+      final RaftPeer[] peersInRequest2 = cluster.getPeers().toArray(new RaftPeer[0]);
+      AtomicBoolean caughtException = new AtomicBoolean(false);
+      new Thread(() -> {
+        try(final RaftClient client2 = cluster.createClient("client2", leaderId)) {
+          latch.await();
+          LOG.info("client2 starts to change conf");
+          final RaftClientRequestSender sender2 = ((RaftClientImpl)client2).getRequestSender();
+          sender2.sendRequest(new SetConfigurationRequest(
+              "client2", leaderId, DEFAULT_SEQNUM, peersInRequest2));
+        } catch (ReconfigurationInProgressException e) {
+          caughtException.set(true);
+        } catch (Exception e) {
+          LOG.warn("Got unexpected exception when client2 changes conf", e);
+        }
+      }).start();
+
+      AtomicBoolean confChanged = new AtomicBoolean(false);
+      new Thread(() -> {
+        try(final RaftClient client1 = cluster.createClient("client1", leaderId)) {
+          LOG.info("client1 starts to change conf");
+          confChanged.set(client1.setConfiguration(newPeers).isSuccess());
+        } catch (IOException e) {
+          LOG.warn("Got unexpected exception when client1 changes conf", e);
+        }
+      }).start();
+      Thread.sleep(100);
+      latch.countDown();
+
+      for (int i = 0; i < 10 && !confChanged.get(); i++) {
+        Thread.sleep(1000);
+      }
+      Assert.assertTrue(confChanged.get());
+      Assert.assertTrue(caughtException.get());
+    } finally {
+      logSyncDelay.clear();
+      cluster.shutdown();
+    }
+  }
+
+  /**
+   * Test a scenario where the follower truncates its log entries which causes
+   * configuration change.
+   */
+  @Test
+  public void testRevertConfigurationChange() throws Exception {
+    LOG.info("Start testRevertConfigurationChange");
+    // originally 3 peers
+    final MiniRaftCluster cluster = getCluster(5);
+    try {
+      cluster.start();
+      RaftTestUtil.waitForLeader(cluster);
+
+      final String leaderId = cluster.getLeader().getId();
+
+      final RaftLog log = cluster.getServer(leaderId).getState().getLog();
+      Thread.sleep(1000);
+      Assert.assertEquals(0, log.getLatestFlushedIndex());
+
+      // we block the incoming msg for the leader and block its requests to
+      // followers, so that we force the leader change and the old leader will
+      // not know
+      LOG.info("start blocking the leader");
+      BlockRequestHandlingInjection.getInstance().blockReplier(leaderId);
+      cluster.setBlockRequestsFrom(leaderId, true);
+
+      PeerChanges change = cluster.removePeers(1, false, new ArrayList<>());
+
+      AtomicBoolean gotNotLeader = new AtomicBoolean(false);
+      new Thread(() -> {
+        try(final RaftClient client = cluster.createClient("client1", leaderId)) {
+          LOG.info("client starts to change conf");
+          final RaftClientRequestSender sender = ((RaftClientImpl)client).getRequestSender();
+          RaftClientReply reply = sender.sendRequest(new SetConfigurationRequest(
+              "client", leaderId, DEFAULT_SEQNUM, change.allPeersInNewConf));
+          if (reply.isNotLeader()) {
+            gotNotLeader.set(true);
+          }
+        } catch (IOException e) {
+          LOG.warn("Got unexpected exception when client1 changes conf", e);
+        }
+      }).start();
+
+      // wait till the old leader persist the new conf
+      for (int i = 0; i < 10 && log.getLatestFlushedIndex() < 1; i++) {
+        Thread.sleep(500);
+      }
+      Assert.assertEquals(1, log.getLatestFlushedIndex());
+      Assert.assertEquals(CONFIGURATIONENTRY,
+          log.getLastEntry().getLogEntryBodyCase());
+
+      // unblock the old leader
+      BlockRequestHandlingInjection.getInstance().unblockReplier(leaderId);
+      cluster.setBlockRequestsFrom(leaderId, false);
+
+      // the client should get NotLeaderException
+      for (int i = 0; i < 10 && !gotNotLeader.get(); i++) {
+        Thread.sleep(500);
+      }
+      Assert.assertTrue(gotNotLeader.get());
+
+      // the old leader should have truncated the setConf from the log
+      boolean newState = false;
+      for (int i = 0; i < 10 && !newState; i++) {
+        Thread.sleep(500);
+        newState = log.getLastCommittedIndex() == 1 &&
+            log.getLastEntry().getLogEntryBodyCase() != CONFIGURATIONENTRY;
+      }
+      Assert.assertTrue(newState);
+    } finally {
+      cluster.shutdown();
+    }
+  }
+}

http://git-wip-us.apache.org/repos/asf/incubator-ratis/blob/7e71a2e0/ratis-server/src/test/java/org/apache/ratis/server/impl/RaftServerTestUtil.java
----------------------------------------------------------------------
diff --git a/ratis-server/src/test/java/org/apache/ratis/server/impl/RaftServerTestUtil.java b/ratis-server/src/test/java/org/apache/ratis/server/impl/RaftServerTestUtil.java
new file mode 100644
index 0000000..dc10bd3
--- /dev/null
+++ b/ratis-server/src/test/java/org/apache/ratis/server/impl/RaftServerTestUtil.java
@@ -0,0 +1,75 @@
+/**
+ * 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.ratis.server.impl;
+
+import org.apache.ratis.MiniRaftCluster;
+import org.apache.ratis.RaftTestUtil;
+import org.apache.ratis.protocol.RaftPeer;
+import org.apache.ratis.server.RaftServer;
+import org.apache.ratis.server.impl.RaftConfiguration;
+import org.apache.ratis.server.impl.RaftServerImpl;
+import org.apache.ratis.statemachine.StateMachine;
+import org.junit.Assert;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.util.Collection;
+
+public class RaftServerTestUtil {
+  static final Logger LOG = LoggerFactory.getLogger(RaftServerTestUtil.class);
+
+  public static void waitAndCheckNewConf(MiniRaftCluster cluster,
+      RaftPeer[] peers, int numOfRemovedPeers, Collection<String> deadPeers)
+      throws Exception {
+    final long sleepMs = cluster.getMaxTimeout() * (numOfRemovedPeers + 2);
+    RaftTestUtil.attempt(3, sleepMs,
+        () -> waitAndCheckNewConf(cluster, peers, deadPeers));
+  }
+  private static void waitAndCheckNewConf(MiniRaftCluster cluster,
+      RaftPeer[] peers, Collection<String> deadPeers)
+      throws Exception {
+    LOG.info(cluster.printServers());
+    Assert.assertNotNull(cluster.getLeader());
+
+    int numIncluded = 0;
+    int deadIncluded = 0;
+    final RaftConfiguration current = RaftConfiguration.newBuilder()
+        .setConf(peers).setLogEntryIndex(0).build();
+    for (RaftServerImpl server : cluster.getServers()) {
+      if (deadPeers != null && deadPeers.contains(server.getId())) {
+        if (current.containsInConf(server.getId())) {
+          deadIncluded++;
+        }
+        continue;
+      }
+      if (current.containsInConf(server.getId())) {
+        numIncluded++;
+        Assert.assertTrue(server.getRaftConf().isStable());
+        Assert.assertTrue(server.getRaftConf().hasNoChange(peers));
+      } else {
+        Assert.assertFalse(server.getId() + " is still running: " + server,
+            server.isAlive());
+      }
+    }
+    Assert.assertEquals(peers.length, numIncluded + deadIncluded);
+  }
+
+  public static StateMachine getStateMachine(RaftServerImpl s) {
+    return s.getStateMachine();
+  }
+}

http://git-wip-us.apache.org/repos/asf/incubator-ratis/blob/7e71a2e0/ratis-server/src/test/java/org/apache/ratis/server/simulation/MiniRaftClusterWithSimulatedRpc.java
----------------------------------------------------------------------
diff --git a/ratis-server/src/test/java/org/apache/ratis/server/simulation/MiniRaftClusterWithSimulatedRpc.java b/ratis-server/src/test/java/org/apache/ratis/server/simulation/MiniRaftClusterWithSimulatedRpc.java
new file mode 100644
index 0000000..fa177af
--- /dev/null
+++ b/ratis-server/src/test/java/org/apache/ratis/server/simulation/MiniRaftClusterWithSimulatedRpc.java
@@ -0,0 +1,136 @@
+/**
+ * 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.ratis.server.simulation;
+
+import org.apache.ratis.MiniRaftCluster;
+import org.apache.ratis.client.RaftClientRequestSender;
+import org.apache.ratis.conf.RaftProperties;
+import org.apache.ratis.protocol.RaftPeer;
+import org.apache.ratis.server.impl.RaftServerImpl;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.io.IOException;
+import java.util.Collection;
+import java.util.Collections;
+
+public class MiniRaftClusterWithSimulatedRpc extends MiniRaftCluster {
+  static final Logger LOG = LoggerFactory.getLogger(MiniRaftClusterWithSimulatedRpc.class);
+
+  public static final Factory<MiniRaftClusterWithSimulatedRpc> FACTORY
+      = new Factory<MiniRaftClusterWithSimulatedRpc>() {
+    @Override
+    public MiniRaftClusterWithSimulatedRpc newCluster(
+        String[] ids, RaftProperties prop, boolean formatted) {
+      prop.setInt(SimulatedRequestReply.SIMULATE_LATENCY_KEY, 0);
+      return new MiniRaftClusterWithSimulatedRpc(ids, prop, formatted);
+    }
+  };
+
+  private SimulatedRequestReply<RaftServerRequest, RaftServerReply> serverRequestReply;
+  private SimulatedClientRequestReply client2serverRequestReply;
+
+  public MiniRaftClusterWithSimulatedRpc(int numServers,
+      RaftProperties properties) {
+    this(generateIds(numServers, 0), properties, true);
+  }
+
+  public MiniRaftClusterWithSimulatedRpc(String[] ids,
+      RaftProperties properties, boolean formatted) {
+    super(ids, properties, formatted);
+    initRpc();
+  }
+
+  private void initRpc() {
+    final Collection<RaftPeer> peers = getConf().getPeers();
+    final int simulateLatencyMs = properties.getInt(
+        SimulatedRequestReply.SIMULATE_LATENCY_KEY,
+        SimulatedRequestReply.SIMULATE_LATENCY_DEFAULT);
+    LOG.info(SimulatedRequestReply.SIMULATE_LATENCY_KEY + " = "
+        + simulateLatencyMs);
+    serverRequestReply = new SimulatedRequestReply<>(peers, simulateLatencyMs);
+    client2serverRequestReply = new SimulatedClientRequestReply(peers,
+        simulateLatencyMs);
+
+    setRpcServers(getServers());
+  }
+
+  private void setRpcServers(Collection<RaftServerImpl> newServers) {
+    newServers.forEach(s -> s.setServerRpc(
+        new SimulatedServerRpc(s, serverRequestReply, client2serverRequestReply)));
+  }
+
+  @Override
+  protected void setPeerRpc() {
+    initRpc();
+  }
+
+  private void addPeersToRpc(Collection<RaftPeer> peers) {
+    serverRequestReply.addPeers(peers);
+    client2serverRequestReply.addPeers(peers);
+  }
+
+  @Override
+  public void restartServer(String id, boolean format) throws IOException {
+    super.restartServer(id, format);
+    RaftServerImpl s = getServer(id);
+    addPeersToRpc(Collections.singletonList(conf.getPeer(id)));
+    s.setServerRpc(new SimulatedServerRpc(s, serverRequestReply,
+        client2serverRequestReply));
+    s.start();
+  }
+
+  @Override
+  public Collection<RaftPeer> addNewPeers(Collection<RaftPeer> newPeers,
+                                          Collection<RaftServerImpl> newServers, boolean startService) {
+    addPeersToRpc(newPeers);
+    setRpcServers(newServers);
+    if (startService) {
+      newServers.forEach(RaftServerImpl::start);
+    }
+    return newPeers;
+  }
+
+  @Override
+  public RaftClientRequestSender getRaftClientRequestSender() {
+    return client2serverRequestReply;
+  }
+
+  @Override
+  public void blockQueueAndSetDelay(String leaderId, int delayMs)
+      throws InterruptedException {
+    // block leader sendRequest if delayMs > 0
+    final boolean block = delayMs > 0;
+    LOG.debug("{} leader queue {} and set {}ms delay for the other queues",
+        block? "Block": "Unblock", leaderId, delayMs);
+    serverRequestReply.getQueue(leaderId).blockSendRequestTo.set(block);
+
+    // set delay takeRequest for the other queues
+    getServers().stream().filter(s -> !s.getId().equals(leaderId))
+        .map(s -> serverRequestReply.getQueue(s.getId()))
+        .forEach(q -> q.delayTakeRequestTo.set(delayMs));
+
+    final long sleepMs = 3 * getMaxTimeout() / 2;
+    Thread.sleep(sleepMs);
+  }
+
+  @Override
+  public void setBlockRequestsFrom(String src, boolean block) {
+    serverRequestReply.getQueue(src).blockTakeRequestFrom.set(block);
+  }
+}

http://git-wip-us.apache.org/repos/asf/incubator-ratis/blob/7e71a2e0/ratis-server/src/test/java/org/apache/ratis/server/simulation/RaftServerReply.java
----------------------------------------------------------------------
diff --git a/ratis-server/src/test/java/org/apache/ratis/server/simulation/RaftServerReply.java b/ratis-server/src/test/java/org/apache/ratis/server/simulation/RaftServerReply.java
new file mode 100644
index 0000000..a157524
--- /dev/null
+++ b/ratis-server/src/test/java/org/apache/ratis/server/simulation/RaftServerReply.java
@@ -0,0 +1,100 @@
+/**
+ * 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.ratis.server.simulation;
+
+import org.apache.ratis.protocol.RaftRpcMessage;
+import org.apache.ratis.shaded.proto.RaftProtos.AppendEntriesReplyProto;
+import org.apache.ratis.shaded.proto.RaftProtos.InstallSnapshotReplyProto;
+import org.apache.ratis.shaded.proto.RaftProtos.RequestVoteReplyProto;
+
+import com.google.common.base.Preconditions;
+
+public class RaftServerReply extends RaftRpcMessage {
+  private final AppendEntriesReplyProto appendEntries;
+  private final RequestVoteReplyProto requestVote;
+  private final InstallSnapshotReplyProto installSnapshot;
+
+  RaftServerReply(AppendEntriesReplyProto a) {
+    appendEntries = Preconditions.checkNotNull(a);
+    requestVote = null;
+    installSnapshot = null;
+  }
+
+  RaftServerReply(RequestVoteReplyProto r) {
+    appendEntries = null;
+    requestVote = Preconditions.checkNotNull(r);
+    installSnapshot = null;
+  }
+
+  RaftServerReply(InstallSnapshotReplyProto i) {
+    appendEntries = null;
+    requestVote = null;
+    installSnapshot = Preconditions.checkNotNull(i);
+  }
+
+  boolean isAppendEntries() {
+    return appendEntries != null;
+  }
+
+  boolean isRequestVote() {
+    return requestVote != null;
+  }
+
+  boolean isInstallSnapshot() {
+    return installSnapshot != null;
+  }
+
+  AppendEntriesReplyProto getAppendEntries() {
+    return appendEntries;
+  }
+
+  RequestVoteReplyProto getRequestVote() {
+    return requestVote;
+  }
+
+  InstallSnapshotReplyProto getInstallSnapshot() {
+    return installSnapshot;
+  }
+
+  @Override
+  public boolean isRequest() {
+    return false;
+  }
+
+  @Override
+  public String getRequestorId() {
+    if (isAppendEntries()) {
+      return appendEntries.getServerReply().getRequestorId();
+    } else if (isRequestVote()) {
+      return requestVote.getServerReply().getRequestorId();
+    } else {
+      return installSnapshot.getServerReply().getRequestorId();
+    }
+  }
+
+  @Override
+  public String getReplierId() {
+    if (isAppendEntries()) {
+      return appendEntries.getServerReply().getReplyId();
+    } else if (isRequestVote()) {
+      return requestVote.getServerReply().getReplyId();
+    } else {
+      return installSnapshot.getServerReply().getReplyId();
+    }
+  }
+}

http://git-wip-us.apache.org/repos/asf/incubator-ratis/blob/7e71a2e0/ratis-server/src/test/java/org/apache/ratis/server/simulation/RaftServerRequest.java
----------------------------------------------------------------------
diff --git a/ratis-server/src/test/java/org/apache/ratis/server/simulation/RaftServerRequest.java b/ratis-server/src/test/java/org/apache/ratis/server/simulation/RaftServerRequest.java
new file mode 100644
index 0000000..fd73dff
--- /dev/null
+++ b/ratis-server/src/test/java/org/apache/ratis/server/simulation/RaftServerRequest.java
@@ -0,0 +1,98 @@
+/**
+ * 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.ratis.server.simulation;
+
+import org.apache.ratis.protocol.RaftRpcMessage;
+import org.apache.ratis.shaded.proto.RaftProtos.AppendEntriesRequestProto;
+import org.apache.ratis.shaded.proto.RaftProtos.InstallSnapshotRequestProto;
+import org.apache.ratis.shaded.proto.RaftProtos.RequestVoteRequestProto;
+
+class RaftServerRequest extends RaftRpcMessage {
+  private final AppendEntriesRequestProto appendEntries;
+  private final RequestVoteRequestProto requestVote;
+  private final InstallSnapshotRequestProto installSnapshot;
+
+  RaftServerRequest(AppendEntriesRequestProto a) {
+    appendEntries = a;
+    requestVote = null;
+    installSnapshot = null;
+  }
+
+  RaftServerRequest(RequestVoteRequestProto r) {
+    appendEntries = null;
+    requestVote = r;
+    installSnapshot = null;
+  }
+
+  RaftServerRequest(InstallSnapshotRequestProto i) {
+    appendEntries = null;
+    requestVote = null;
+    installSnapshot = i;
+  }
+
+  boolean isAppendEntries() {
+    return appendEntries != null;
+  }
+
+  boolean isRequestVote() {
+    return requestVote != null;
+  }
+
+  boolean isInstallSnapshot() {
+    return installSnapshot != null;
+  }
+
+  AppendEntriesRequestProto getAppendEntries() {
+    return appendEntries;
+  }
+
+  RequestVoteRequestProto getRequestVote() {
+    return requestVote;
+  }
+
+  InstallSnapshotRequestProto getInstallSnapshot() {
+    return installSnapshot;
+  }
+
+  @Override
+  public boolean isRequest() {
+    return true;
+  }
+
+  @Override
+  public String getRequestorId() {
+    if (isAppendEntries()) {
+      return appendEntries.getServerRequest().getRequestorId();
+    } else if (isRequestVote()) {
+      return requestVote.getServerRequest().getRequestorId();
+    } else {
+      return installSnapshot.getServerRequest().getRequestorId();
+    }
+  }
+
+  @Override
+  public String getReplierId() {
+    if (isAppendEntries()) {
+      return appendEntries.getServerRequest().getReplyId();
+    } else if (isRequestVote()) {
+      return requestVote.getServerRequest().getReplyId();
+    } else {
+      return installSnapshot.getServerRequest().getReplyId();
+    }
+  }
+}

http://git-wip-us.apache.org/repos/asf/incubator-ratis/blob/7e71a2e0/ratis-server/src/test/java/org/apache/ratis/server/simulation/RequestHandler.java
----------------------------------------------------------------------
diff --git a/ratis-server/src/test/java/org/apache/ratis/server/simulation/RequestHandler.java b/ratis-server/src/test/java/org/apache/ratis/server/simulation/RequestHandler.java
new file mode 100644
index 0000000..bd60a3b
--- /dev/null
+++ b/ratis-server/src/test/java/org/apache/ratis/server/simulation/RequestHandler.java
@@ -0,0 +1,134 @@
+/**
+ * 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.ratis.server.simulation;
+
+import org.apache.ratis.protocol.RaftRpcMessage;
+import org.apache.ratis.util.Daemon;
+import org.apache.ratis.util.ExitUtils;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.io.IOException;
+import java.io.InterruptedIOException;
+import java.util.ArrayList;
+import java.util.List;
+
+public class RequestHandler<REQUEST extends RaftRpcMessage,
+    REPLY extends RaftRpcMessage> {
+  public static final Logger LOG = LoggerFactory.getLogger(RequestHandler.class);
+
+  interface HandlerInterface<REQUEST extends RaftRpcMessage,
+      REPLY extends RaftRpcMessage> {
+
+    boolean isAlive();
+
+    REPLY handleRequest(REQUEST r) throws IOException;
+  }
+
+  private final String serverId;
+  private final String name;
+  private final SimulatedRequestReply<REQUEST, REPLY> rpc;
+  private final HandlerInterface<REQUEST, REPLY> handlerImpl;
+  private final List<HandlerDaemon> daemons;
+
+  RequestHandler(String serverId, String name,
+                 SimulatedRequestReply<REQUEST, REPLY> rpc,
+                 HandlerInterface<REQUEST, REPLY> handlerImpl,
+                 int numHandlers) {
+    this.serverId = serverId;
+    this.name = name;
+    this.rpc = rpc;
+    this.handlerImpl = handlerImpl;
+
+    this.daemons = new ArrayList<>(numHandlers);
+    for(int i = 0; i < numHandlers; i++) {
+      daemons.add(new HandlerDaemon(i));
+    }
+  }
+
+  void startDaemon() {
+    daemons.forEach(Thread::start);
+  }
+
+  void shutdown() {
+    rpc.shutdown(serverId);
+  }
+
+  void interruptAndJoinDaemon() throws InterruptedException {
+    daemons.forEach(Thread::interrupt);
+    for (Daemon d : daemons) {
+      d.join();
+    }
+  }
+
+  SimulatedRequestReply<REQUEST, REPLY> getRpc() {
+    return rpc;
+  }
+
+  void handleRequest(REQUEST request) throws IOException {
+    final REPLY reply;
+    try {
+      reply = handlerImpl.handleRequest(request);
+    } catch (IOException ioe) {
+      LOG.debug("IOException for " + request, ioe);
+      rpc.sendReply(request, null, ioe);
+      return;
+    }
+    if (reply != null) {
+      rpc.sendReply(request, reply, null);
+    }
+  }
+
+  /**
+   * A thread keep polling requests from the request queue. Used for simulation.
+   */
+  class HandlerDaemon extends Daemon {
+    private final int id;
+
+    HandlerDaemon(int id) {
+      this.id = id;
+    }
+
+    @Override
+    public String toString() {
+      return serverId + "." + name + id;
+    }
+
+    @Override
+    public void run() {
+      while (handlerImpl.isAlive()) {
+        try {
+          handleRequest(rpc.takeRequest(serverId));
+        } catch (InterruptedIOException e) {
+          LOG.info(this + " is interrupted by " + e);
+          LOG.trace("TRACE", e);
+          break;
+        } catch (IOException e) {
+          LOG.error(this + " has " + e);
+          LOG.trace("TRACE", e);
+        } catch(Throwable t) {
+          if (!handlerImpl.isAlive()) {
+            LOG.info(this + " is stopped.");
+            break;
+          }
+          ExitUtils.terminate(1, this + " is terminating.", t, LOG);
+        }
+      }
+    }
+  }
+}

http://git-wip-us.apache.org/repos/asf/incubator-ratis/blob/7e71a2e0/ratis-server/src/test/java/org/apache/ratis/server/simulation/SimulatedClientRequestReply.java
----------------------------------------------------------------------
diff --git a/ratis-server/src/test/java/org/apache/ratis/server/simulation/SimulatedClientRequestReply.java b/ratis-server/src/test/java/org/apache/ratis/server/simulation/SimulatedClientRequestReply.java
new file mode 100644
index 0000000..9302051
--- /dev/null
+++ b/ratis-server/src/test/java/org/apache/ratis/server/simulation/SimulatedClientRequestReply.java
@@ -0,0 +1,45 @@
+/**
+ * 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.ratis.server.simulation;
+
+import org.apache.ratis.client.RaftClientRequestSender;
+import org.apache.ratis.protocol.RaftClientReply;
+import org.apache.ratis.protocol.RaftClientRequest;
+import org.apache.ratis.protocol.RaftPeer;
+
+import java.io.IOException;
+import java.util.Collection;
+
+public class SimulatedClientRequestReply
+    extends SimulatedRequestReply<RaftClientRequest, RaftClientReply>
+    implements RaftClientRequestSender {
+  SimulatedClientRequestReply(Collection<RaftPeer> allPeers,
+                              int simulateLatencyMs) {
+    super(allPeers, simulateLatencyMs);
+  }
+
+  @Override
+  public void addServers(Iterable<RaftPeer> servers) {
+    // do nothing
+  }
+
+  @Override
+  public void close() {
+    // do nothing
+  }
+}

http://git-wip-us.apache.org/repos/asf/incubator-ratis/blob/7e71a2e0/ratis-server/src/test/java/org/apache/ratis/server/simulation/SimulatedRequestReply.java
----------------------------------------------------------------------
diff --git a/ratis-server/src/test/java/org/apache/ratis/server/simulation/SimulatedRequestReply.java b/ratis-server/src/test/java/org/apache/ratis/server/simulation/SimulatedRequestReply.java
new file mode 100644
index 0000000..559c1e6
--- /dev/null
+++ b/ratis-server/src/test/java/org/apache/ratis/server/simulation/SimulatedRequestReply.java
@@ -0,0 +1,202 @@
+/**
+ * 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.ratis.server.simulation;
+
+import com.google.common.base.Preconditions;
+
+import org.apache.ratis.RaftTestUtil;
+import org.apache.ratis.protocol.RaftPeer;
+import org.apache.ratis.protocol.RaftRpcMessage;
+import org.apache.ratis.server.RaftServerConfigKeys;
+import org.apache.ratis.util.RaftUtils;
+import org.apache.ratis.util.Timestamp;
+
+import java.io.IOException;
+import java.util.Collection;
+import java.util.Map;
+import java.util.concurrent.BlockingQueue;
+import java.util.concurrent.ConcurrentHashMap;
+import java.util.concurrent.LinkedBlockingQueue;
+import java.util.concurrent.ThreadLocalRandom;
+import java.util.concurrent.atomic.AtomicBoolean;
+import java.util.concurrent.atomic.AtomicInteger;
+
+public class SimulatedRequestReply<REQUEST extends RaftRpcMessage,
+    REPLY extends RaftRpcMessage> {
+  public static final String SIMULATE_LATENCY_KEY
+      = SimulatedRequestReply.class.getName() + ".simulateLatencyMs";
+  public static final int SIMULATE_LATENCY_DEFAULT
+      = RaftServerConfigKeys.RAFT_SERVER_RPC_TIMEOUT_MIN_MS_DEFAULT;
+  public static final long TIMEOUT = 3000L;
+
+  private static class ReplyOrException<REPLY> {
+    private final REPLY reply;
+    private final IOException ioe;
+
+    ReplyOrException(REPLY reply, IOException ioe) {
+      Preconditions.checkArgument(reply == null ^ ioe == null);
+      this.reply = reply;
+      this.ioe = ioe;
+    }
+  }
+
+  static class EventQueue<REQUEST, REPLY> {
+    private final BlockingQueue<REQUEST> requestQueue
+        = new LinkedBlockingQueue<>();
+    private final Map<REQUEST, ReplyOrException<REPLY>> replyMap
+        = new ConcurrentHashMap<>();
+
+    /** Block takeRequest for the requests sent from this server. */
+    final AtomicBoolean blockTakeRequestFrom = new AtomicBoolean();
+    /** Block sendRequest for the requests sent to this server. */
+    final AtomicBoolean blockSendRequestTo = new AtomicBoolean();
+    /** Delay takeRequest for the requests sent to this server. */
+    final AtomicInteger delayTakeRequestTo = new AtomicInteger();
+    /** Delay takeRequest for the requests sent from this server. */
+    final AtomicInteger delayTakeRequestFrom = new AtomicInteger();
+
+    REPLY request(REQUEST request) throws InterruptedException, IOException {
+      requestQueue.put(request);
+      synchronized (this) {
+        final Timestamp startTime = new Timestamp();
+        while (startTime.elapsedTimeMs() < TIMEOUT &&
+            !replyMap.containsKey(request)) {
+          this.wait(TIMEOUT); // no need to be precise here
+        }
+      }
+
+      if (!replyMap.containsKey(request)) {
+        throw new IOException("Timeout while waiting for reply of request "
+            + request);
+      }
+      final ReplyOrException<REPLY> re = replyMap.remove(request);
+      if (re.ioe != null) {
+        throw re.ioe;
+      }
+      return re.reply;
+    }
+
+    REQUEST takeRequest() throws InterruptedException {
+      return requestQueue.take();
+    }
+
+    void reply(REQUEST request, REPLY reply, IOException ioe)
+        throws IOException {
+      replyMap.put(request, new ReplyOrException<>(reply, ioe));
+      synchronized (this) {
+        this.notifyAll();
+      }
+    }
+  }
+
+  private final Map<String, EventQueue<REQUEST, REPLY>> queues;
+  private final int simulateLatencyMs;
+
+  SimulatedRequestReply(Collection<RaftPeer> allPeers, int simulateLatencyMs) {
+    queues = new ConcurrentHashMap<>();
+    for (RaftPeer peer : allPeers) {
+      queues.put(peer.getId(), new EventQueue<>());
+    }
+
+    this.simulateLatencyMs = simulateLatencyMs;
+  }
+
+  EventQueue<REQUEST, REPLY> getQueue(String qid) {
+    return queues.get(qid);
+  }
+
+  public REPLY sendRequest(REQUEST request) throws IOException {
+    final String qid = request.getReplierId();
+    final EventQueue<REQUEST, REPLY> q = queues.get(qid);
+    if (q == null) {
+      throw new IOException("The peer " + qid + " is not alive.");
+    }
+    try {
+      RaftTestUtil.block(q.blockSendRequestTo::get);
+      return q.request(request);
+    } catch (InterruptedException e) {
+      throw RaftUtils.toInterruptedIOException("", e);
+    }
+  }
+
+  public REQUEST takeRequest(String qid) throws IOException {
+    final EventQueue<REQUEST, REPLY> q = queues.get(qid);
+    if (q == null) {
+      throw new IOException("The RPC of " + qid + " has already shutdown.");
+    }
+
+    final REQUEST request;
+    try {
+      // delay request for testing
+      RaftTestUtil.delay(q.delayTakeRequestTo::get);
+
+      request = q.takeRequest();
+      Preconditions.checkState(qid.equals(request.getReplierId()));
+
+      // block request for testing
+      final EventQueue<REQUEST, REPLY> reqQ = queues.get(request.getRequestorId());
+      if (reqQ != null) {
+        RaftTestUtil.delay(reqQ.delayTakeRequestFrom::get);
+        RaftTestUtil.block(reqQ.blockTakeRequestFrom::get);
+      }
+    } catch (InterruptedException e) {
+      throw RaftUtils.toInterruptedIOException("", e);
+    }
+    return request;
+  }
+
+  public void sendReply(REQUEST request, REPLY reply, IOException ioe)
+      throws IOException {
+    if (reply != null) {
+      Preconditions.checkArgument(
+          request.getRequestorId().equals(reply.getRequestorId()));
+      Preconditions.checkArgument(
+          request.getReplierId().equals(reply.getReplierId()));
+    }
+    simulateLatency();
+    final String qid = request.getReplierId();
+    EventQueue<REQUEST, REPLY> q = queues.get(qid);
+    if (q != null) {
+      q.reply(request, reply, ioe);
+    }
+  }
+
+  public void shutdown(String id) {
+    queues.remove(id);
+  }
+
+  public void addPeers(Collection<RaftPeer> newPeers) {
+    for (RaftPeer peer : newPeers) {
+      queues.put(peer.getId(), new EventQueue<>());
+    }
+  }
+
+  private void simulateLatency() throws IOException {
+    if (simulateLatencyMs > 0) {
+      int waitExpetation = simulateLatencyMs / 10;
+      int waitHalfRange = waitExpetation / 3;
+      int randomSleepMs = ThreadLocalRandom.current().nextInt(2 * waitHalfRange)
+          + waitExpetation - waitHalfRange;
+      try {
+        Thread.sleep(randomSleepMs);
+      } catch (InterruptedException ie) {
+        throw RaftUtils.toInterruptedIOException("", ie);
+      }
+    }
+  }
+}

http://git-wip-us.apache.org/repos/asf/incubator-ratis/blob/7e71a2e0/ratis-server/src/test/java/org/apache/ratis/server/simulation/SimulatedServerRpc.java
----------------------------------------------------------------------
diff --git a/ratis-server/src/test/java/org/apache/ratis/server/simulation/SimulatedServerRpc.java b/ratis-server/src/test/java/org/apache/ratis/server/simulation/SimulatedServerRpc.java
new file mode 100644
index 0000000..d40cf44
--- /dev/null
+++ b/ratis-server/src/test/java/org/apache/ratis/server/simulation/SimulatedServerRpc.java
@@ -0,0 +1,175 @@
+/**
+ * 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.ratis.server.simulation;
+
+import java.io.IOException;
+import java.net.InetSocketAddress;
+import java.util.concurrent.CompletableFuture;
+import java.util.concurrent.ExecutorService;
+import java.util.concurrent.Executors;
+import java.util.concurrent.TimeUnit;
+
+import org.apache.ratis.protocol.RaftClientReply;
+import org.apache.ratis.protocol.RaftClientRequest;
+import org.apache.ratis.protocol.RaftPeer;
+import org.apache.ratis.protocol.SetConfigurationRequest;
+import org.apache.ratis.server.RaftServerRpc;
+import org.apache.ratis.server.impl.RaftServerImpl;
+import org.apache.ratis.shaded.proto.RaftProtos.AppendEntriesReplyProto;
+import org.apache.ratis.shaded.proto.RaftProtos.AppendEntriesRequestProto;
+import org.apache.ratis.shaded.proto.RaftProtos.InstallSnapshotReplyProto;
+import org.apache.ratis.shaded.proto.RaftProtos.InstallSnapshotRequestProto;
+import org.apache.ratis.shaded.proto.RaftProtos.RequestVoteReplyProto;
+import org.apache.ratis.shaded.proto.RaftProtos.RequestVoteRequestProto;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import com.google.common.util.concurrent.ThreadFactoryBuilder;
+
+class SimulatedServerRpc implements RaftServerRpc {
+  static final Logger LOG = LoggerFactory.getLogger(SimulatedServerRpc.class);
+
+  private final RaftServerImpl server;
+  private final RequestHandler<RaftServerRequest, RaftServerReply> serverHandler;
+  private final RequestHandler<RaftClientRequest, RaftClientReply> clientHandler;
+  private final ExecutorService executor = Executors.newFixedThreadPool(3,
+      new ThreadFactoryBuilder().setDaemon(true).build());
+
+  SimulatedServerRpc(RaftServerImpl server,
+      SimulatedRequestReply<RaftServerRequest, RaftServerReply> serverRequestReply,
+      SimulatedRequestReply<RaftClientRequest, RaftClientReply> clientRequestReply) {
+    this.server = server;
+    this.serverHandler = new RequestHandler<>(server.getId(),
+        "serverHandler", serverRequestReply, serverHandlerImpl, 3);
+    this.clientHandler = new RequestHandler<>(server.getId(),
+        "clientHandler", clientRequestReply, clientHandlerImpl, 3);
+  }
+
+  @Override
+  public void start() {
+    serverHandler.startDaemon();
+    clientHandler.startDaemon();
+  }
+
+  private void interruptAndJoin() throws InterruptedException {
+    clientHandler.interruptAndJoinDaemon();
+    serverHandler.interruptAndJoinDaemon();
+  }
+
+  @Override
+  public void close() {
+    try {
+      interruptAndJoin();
+      executor.shutdown();
+      executor.awaitTermination(1000, TimeUnit.MILLISECONDS);
+    } catch (InterruptedException ignored) {
+    }
+    clientHandler.shutdown();
+    serverHandler.shutdown();
+  }
+
+  @Override
+  public InetSocketAddress getInetSocketAddress() {
+    return null;
+  }
+
+  @Override
+  public AppendEntriesReplyProto appendEntries(AppendEntriesRequestProto request)
+      throws IOException {
+    RaftServerReply reply = serverHandler.getRpc()
+        .sendRequest(new RaftServerRequest(request));
+    return reply.getAppendEntries();
+  }
+
+  @Override
+  public InstallSnapshotReplyProto installSnapshot(InstallSnapshotRequestProto request)
+      throws IOException {
+    RaftServerReply reply = serverHandler.getRpc()
+        .sendRequest(new RaftServerRequest(request));
+    return reply.getInstallSnapshot();
+  }
+
+  @Override
+  public RequestVoteReplyProto requestVote(RequestVoteRequestProto request)
+      throws IOException {
+    RaftServerReply reply = serverHandler.getRpc()
+        .sendRequest(new RaftServerRequest(request));
+    return reply.getRequestVote();
+  }
+
+  @Override
+  public void addPeers(Iterable<RaftPeer> peers) {
+    // do nothing
+  }
+
+  final RequestHandler.HandlerInterface<RaftServerRequest, RaftServerReply> serverHandlerImpl
+      = new RequestHandler.HandlerInterface<RaftServerRequest, RaftServerReply>() {
+    @Override
+    public boolean isAlive() {
+      return server.isAlive();
+    }
+
+    @Override
+    public RaftServerReply handleRequest(RaftServerRequest r)
+        throws IOException {
+      if (r.isAppendEntries()) {
+        return new RaftServerReply(server.appendEntries(r.getAppendEntries()));
+      } else if (r.isRequestVote()) {
+        return new RaftServerReply(server.requestVote(r.getRequestVote()));
+      } else if (r.isInstallSnapshot()) {
+        return new RaftServerReply(server.installSnapshot(r.getInstallSnapshot()));
+      } else {
+        throw new IllegalStateException("unexpected state");
+      }
+    }
+  };
+
+  final RequestHandler.HandlerInterface<RaftClientRequest, RaftClientReply> clientHandlerImpl
+      = new RequestHandler.HandlerInterface<RaftClientRequest, RaftClientReply>() {
+    @Override
+    public boolean isAlive() {
+      return server.isAlive();
+    }
+
+    @Override
+    public RaftClientReply handleRequest(RaftClientRequest request)
+        throws IOException {
+      final CompletableFuture<RaftClientReply> future;
+      if (request instanceof SetConfigurationRequest) {
+        future = server.setConfigurationAsync((SetConfigurationRequest) request);
+      } else {
+        future = server.submitClientRequestAsync(request);
+      }
+
+      future.whenCompleteAsync((reply, exception) -> {
+        try {
+          IOException e = null;
+          if (exception != null) {
+            e = exception instanceof IOException ?
+                (IOException) exception : new IOException(exception);
+          }
+          clientHandler.getRpc().sendReply(request, reply, e);
+        } catch (IOException e) {
+          LOG.warn("Failed to send reply {} for request {} due to exception {}",
+              reply, request, e);
+        }
+      }, executor);
+      return null;
+    }
+  };
+}

http://git-wip-us.apache.org/repos/asf/incubator-ratis/blob/7e71a2e0/ratis-server/src/test/java/org/apache/ratis/server/simulation/TestNotLeaderExceptionWithSimulation.java
----------------------------------------------------------------------
diff --git a/ratis-server/src/test/java/org/apache/ratis/server/simulation/TestNotLeaderExceptionWithSimulation.java b/ratis-server/src/test/java/org/apache/ratis/server/simulation/TestNotLeaderExceptionWithSimulation.java
new file mode 100644
index 0000000..412fb65
--- /dev/null
+++ b/ratis-server/src/test/java/org/apache/ratis/server/simulation/TestNotLeaderExceptionWithSimulation.java
@@ -0,0 +1,32 @@
+/*
+ * 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.ratis.server.simulation;
+
+import org.apache.ratis.MiniRaftCluster;
+import org.apache.ratis.RaftNotLeaderExceptionBaseTest;
+import org.apache.ratis.conf.RaftProperties;
+
+import java.io.IOException;
+
+public class TestNotLeaderExceptionWithSimulation extends RaftNotLeaderExceptionBaseTest {
+  @Override
+  public MiniRaftCluster initCluster() throws IOException {
+    String[] s = MiniRaftCluster.generateIds(NUM_PEERS, 0);
+    return new MiniRaftClusterWithSimulatedRpc(s, new RaftProperties(), true);
+  }
+}

http://git-wip-us.apache.org/repos/asf/incubator-ratis/blob/7e71a2e0/ratis-server/src/test/java/org/apache/ratis/server/simulation/TestRaftReconfigurationWithSimulatedRpc.java
----------------------------------------------------------------------
diff --git a/ratis-server/src/test/java/org/apache/ratis/server/simulation/TestRaftReconfigurationWithSimulatedRpc.java b/ratis-server/src/test/java/org/apache/ratis/server/simulation/TestRaftReconfigurationWithSimulatedRpc.java
new file mode 100644
index 0000000..b8bd679
--- /dev/null
+++ b/ratis-server/src/test/java/org/apache/ratis/server/simulation/TestRaftReconfigurationWithSimulatedRpc.java
@@ -0,0 +1,31 @@
+/**
+ * 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.ratis.server.simulation;
+
+import java.io.IOException;
+
+import org.apache.ratis.MiniRaftCluster;
+import org.apache.ratis.server.impl.RaftReconfigurationBaseTest;
+
+public class TestRaftReconfigurationWithSimulatedRpc
+    extends RaftReconfigurationBaseTest {
+  @Override
+  public MiniRaftCluster getCluster(int peerNum) throws IOException {
+    return new MiniRaftClusterWithSimulatedRpc(peerNum, prop);
+  }
+}

http://git-wip-us.apache.org/repos/asf/incubator-ratis/blob/7e71a2e0/ratis-server/src/test/java/org/apache/ratis/server/simulation/TestRaftSnapshotWithSimulatedRpc.java
----------------------------------------------------------------------
diff --git a/ratis-server/src/test/java/org/apache/ratis/server/simulation/TestRaftSnapshotWithSimulatedRpc.java b/ratis-server/src/test/java/org/apache/ratis/server/simulation/TestRaftSnapshotWithSimulatedRpc.java
new file mode 100644
index 0000000..f2d5cfb
--- /dev/null
+++ b/ratis-server/src/test/java/org/apache/ratis/server/simulation/TestRaftSnapshotWithSimulatedRpc.java
@@ -0,0 +1,32 @@
+/**
+ * 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.ratis.server.simulation;
+
+import org.apache.ratis.MiniRaftCluster;
+import org.apache.ratis.conf.RaftProperties;
+import org.apache.ratis.statemachine.RaftSnapshotBaseTest;
+
+import java.io.IOException;
+
+public class TestRaftSnapshotWithSimulatedRpc extends RaftSnapshotBaseTest {
+  @Override
+  public MiniRaftCluster initCluster(int numServer, RaftProperties prop)
+      throws IOException {
+    return MiniRaftClusterWithSimulatedRpc.FACTORY.newCluster(numServer, prop, true);
+  }
+}

http://git-wip-us.apache.org/repos/asf/incubator-ratis/blob/7e71a2e0/ratis-server/src/test/java/org/apache/ratis/server/simulation/TestRaftWithSimulatedRpc.java
----------------------------------------------------------------------
diff --git a/ratis-server/src/test/java/org/apache/ratis/server/simulation/TestRaftWithSimulatedRpc.java b/ratis-server/src/test/java/org/apache/ratis/server/simulation/TestRaftWithSimulatedRpc.java
new file mode 100644
index 0000000..29ef6ed
--- /dev/null
+++ b/ratis-server/src/test/java/org/apache/ratis/server/simulation/TestRaftWithSimulatedRpc.java
@@ -0,0 +1,51 @@
+/**
+ * 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.ratis.server.simulation;
+
+import org.apache.log4j.Level;
+import org.apache.ratis.RaftBasicTests;
+import org.apache.ratis.client.RaftClient;
+import org.apache.ratis.conf.RaftProperties;
+import org.apache.ratis.server.impl.RaftServerImpl;
+import org.apache.ratis.util.RaftUtils;
+
+import java.io.IOException;
+import java.util.concurrent.ThreadLocalRandom;
+
+public class TestRaftWithSimulatedRpc extends RaftBasicTests {
+  static {
+    RaftUtils.setLogLevel(RaftServerImpl.LOG, Level.DEBUG);
+    RaftUtils.setLogLevel(RaftClient.LOG, Level.DEBUG);
+  }
+
+  private final MiniRaftClusterWithSimulatedRpc cluster;
+
+  public TestRaftWithSimulatedRpc() throws IOException {
+    final RaftProperties properties = getProperties();
+    if (ThreadLocalRandom.current().nextBoolean()) {
+      // turn off simulate latency half of the times.
+      properties.setInt(SimulatedRequestReply.SIMULATE_LATENCY_KEY, 0);
+    }
+    cluster = new MiniRaftClusterWithSimulatedRpc(NUM_SERVERS, properties);
+  }
+
+  @Override
+  public MiniRaftClusterWithSimulatedRpc getCluster() {
+    return cluster;
+  }
+}


[31/54] [abbrv] incubator-ratis git commit: Renamed the packages from raft to ratis in preperation for Apache Incubation - Moved all java packages from org.apache.raft to org.apache.ratis. - Moved native package to org_apache_ratis, and native lib to l

Posted by ji...@apache.org.
http://git-wip-us.apache.org/repos/asf/incubator-ratis/blob/7e71a2e0/ratis-common/src/main/java/org/apache/ratis/conf/RaftProperties.java
----------------------------------------------------------------------
diff --git a/ratis-common/src/main/java/org/apache/ratis/conf/RaftProperties.java b/ratis-common/src/main/java/org/apache/ratis/conf/RaftProperties.java
new file mode 100644
index 0000000..187f6ce
--- /dev/null
+++ b/ratis-common/src/main/java/org/apache/ratis/conf/RaftProperties.java
@@ -0,0 +1,1658 @@
+/**
+ * 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.ratis.conf;
+
+import com.google.common.base.Preconditions;
+
+import org.apache.ratis.util.StringUtils;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+import org.w3c.dom.*;
+import org.xml.sax.SAXException;
+
+import javax.xml.parsers.DocumentBuilder;
+import javax.xml.parsers.DocumentBuilderFactory;
+import javax.xml.parsers.ParserConfigurationException;
+import javax.xml.transform.Transformer;
+import javax.xml.transform.TransformerException;
+import javax.xml.transform.TransformerFactory;
+import javax.xml.transform.dom.DOMSource;
+import javax.xml.transform.stream.StreamResult;
+import java.io.*;
+import java.lang.ref.WeakReference;
+import java.net.JarURLConnection;
+import java.net.URL;
+import java.net.URLConnection;
+import java.util.*;
+import java.util.Map.Entry;
+import java.util.concurrent.ConcurrentHashMap;
+import java.util.concurrent.CopyOnWriteArrayList;
+import java.util.concurrent.TimeUnit;
+import java.util.regex.Matcher;
+import java.util.regex.Pattern;
+import java.util.regex.PatternSyntaxException;
+
+/** 
+ * Provides access to configuration parameters. The current implementation is a
+ * simplified version of hadoop's Configuration.
+ */
+public class RaftProperties {
+  private static final Logger LOG = LoggerFactory.getLogger(RaftProperties.class);
+
+  private static class Resource {
+    private final Object resource;
+    private final String name;
+
+    public Resource(Object resource) {
+      this(resource, resource.toString());
+    }
+
+    public Resource(Object resource, String name) {
+      this.resource = resource;
+      this.name = name;
+    }
+
+    public String getName(){
+      return name;
+    }
+
+    public Object getResource() {
+      return resource;
+    }
+
+    @Override
+    public String toString() {
+      return name;
+    }
+  }
+
+  /**
+   * List of configuration resources.
+   */
+  private ArrayList<Resource> resources = new ArrayList<>();
+
+  /**
+   * The value reported as the setting resource when a key is set
+   * by code rather than a file resource by dumpConfiguration.
+   */
+  static final String UNKNOWN_RESOURCE = "Unknown";
+
+  /**
+   * List of configuration parameters marked <b>final</b>.
+   */
+  private Set<String> finalParameters = Collections.newSetFromMap(
+      new ConcurrentHashMap<String, Boolean>());
+
+  private boolean loadDefaults = true;
+
+  /**
+   * Configuration objects
+   */
+  private static final WeakHashMap<RaftProperties, Object> REGISTRY = new WeakHashMap<>();
+
+  /**
+   * List of default Resources. Resources are loaded in the order of the list
+   * entries
+   */
+  private static final CopyOnWriteArrayList<String> defaultResources =
+    new CopyOnWriteArrayList<>();
+
+  /**
+   * Stores the mapping of key to the resource which modifies or loads
+   * the key most recently
+   */
+  private Map<String, String[]> updatingResource;
+
+  private Properties properties;
+  private Properties overlay;
+  private ClassLoader classLoader;
+  {
+    classLoader = Thread.currentThread().getContextClassLoader();
+    if (classLoader == null) {
+      classLoader = RaftProperties.class.getClassLoader();
+    }
+  }
+
+  /** A new configuration. */
+  public RaftProperties() {
+    this(true);
+  }
+
+  /** A new configuration where the behavior of reading from the default
+   * resources can be turned off.
+   *
+   * If the parameter {@code loadDefaults} is false, the new instance
+   * will not load resources from the default files.
+   * @param loadDefaults specifies whether to load from the default files
+   */
+  public RaftProperties(boolean loadDefaults) {
+    this.loadDefaults = loadDefaults;
+    updatingResource = new ConcurrentHashMap<>();
+    synchronized(RaftProperties.class) {
+      REGISTRY.put(this, null);
+    }
+  }
+
+  /**
+   * A new RaftProperties with the same settings cloned from another.
+   *
+   * @param other the RaftProperties from which to clone settings.
+   */
+  @SuppressWarnings("unchecked")
+  public RaftProperties(RaftProperties other) {
+    this.resources = (ArrayList<Resource>) other.resources.clone();
+    synchronized(other) {
+      if (other.properties != null) {
+        this.properties = (Properties)other.properties.clone();
+      }
+
+      if (other.overlay!=null) {
+        this.overlay = (Properties)other.overlay.clone();
+      }
+
+      this.updatingResource = new ConcurrentHashMap<>(other.updatingResource);
+      this.finalParameters = Collections.newSetFromMap(
+          new ConcurrentHashMap<String, Boolean>());
+      this.finalParameters.addAll(other.finalParameters);
+    }
+
+    synchronized(RaftProperties.class) {
+      REGISTRY.put(this, null);
+    }
+    this.classLoader = other.classLoader;
+    this.loadDefaults = other.loadDefaults;
+  }
+
+  /**
+   * Add a default resource. Resources are loaded in the order of the resources
+   * added.
+   * @param name file name. File should be present in the classpath.
+   */
+  public static synchronized void addDefaultResource(String name) {
+    if(!defaultResources.contains(name)) {
+      defaultResources.add(name);
+      REGISTRY.keySet().stream().filter(conf -> conf.loadDefaults)
+          .forEach(RaftProperties::reloadConfiguration);
+    }
+  }
+
+  /**
+   * Add a configuration resource.
+   *
+   * The properties of this resource will override properties of previously
+   * added resources, unless they were marked <a href="#Final">final</a>.
+   *
+   * @param name resource to be added, the classpath is examined for a file
+   *             with that name.
+   */
+  public void addResource(String name) {
+    addResourceObject(new Resource(name));
+  }
+
+  /**
+   * Add a configuration resource.
+   *
+   * The properties of this resource will override properties of previously
+   * added resources, unless they were marked <a href="#Final">final</a>.
+   *
+   * WARNING: The contents of the InputStream will be cached, by this method.
+   * So use this sparingly because it does increase the memory consumption.
+   *
+   * @param in InputStream to deserialize the object from. In will be read from
+   * when a get or set is called next.  After it is read the stream will be
+   * closed.
+   */
+  public void addResource(InputStream in) {
+    addResourceObject(new Resource(in));
+  }
+
+  /**
+   * Add a configuration resource.
+   *
+   * The properties of this resource will override properties of previously
+   * added resources, unless they were marked <a href="#Final">final</a>.
+   *
+   * @param in InputStream to deserialize the object from.
+   * @param name the name of the resource because InputStream.toString is not
+   * very descriptive some times.
+   */
+  public void addResource(InputStream in, String name) {
+    addResourceObject(new Resource(in, name));
+  }
+
+  /**
+   * Add a configuration resource.
+   *
+   * The properties of this resource will override properties of previously
+   * added resources, unless they were marked <a href="#Final">final</a>.
+   *
+   * @param conf Configuration object from which to load properties
+   */
+  public void addResource(RaftProperties conf) {
+    addResourceObject(new Resource(conf.getProps()));
+  }
+
+
+
+  /**
+   * Reload configuration from previously added resources.
+   *
+   * This method will clear all the configuration read from the added
+   * resources, and final parameters. This will make the resources to
+   * be read again before accessing the values. Values that are added
+   * via set methods will overlay values read from the resources.
+   */
+  public synchronized void reloadConfiguration() {
+    properties = null;                            // trigger reload
+    finalParameters.clear();                      // clear site-limits
+  }
+
+  private synchronized void addResourceObject(Resource resource) {
+    resources.add(resource);                      // add to resources
+    reloadConfiguration();
+  }
+
+  private static final int MAX_SUBST = 20;
+
+  private static final int SUB_START_IDX = 0;
+  private static final int SUB_END_IDX = SUB_START_IDX + 1;
+
+  /**
+   * This is a manual implementation of the following regex
+   * "\\$\\{[^\\}\\$\u0020]+\\}".
+   *
+   * @param eval a string that may contain variables requiring expansion.
+   * @return a 2-element int array res such that
+   * eval.substring(res[0], res[1]) is "var" for the left-most occurrence of
+   * ${var} in eval. If no variable is found -1, -1 is returned.
+   */
+  private static int[] findSubVariable(String eval) {
+    int[] result = {-1, -1};
+
+    int matchStart;
+    int leftBrace;
+
+    // scanning for a brace first because it's less frequent than $
+    // that can occur in nested class names
+    //
+    match_loop:
+    for (matchStart = 1, leftBrace = eval.indexOf('{', matchStart);
+         // minimum left brace position (follows '$')
+         leftBrace > 0
+         // right brace of a smallest valid expression "${c}"
+         && leftBrace + "{c".length() < eval.length();
+         leftBrace = eval.indexOf('{', matchStart)) {
+      int matchedLen = 0;
+      if (eval.charAt(leftBrace - 1) == '$') {
+        int subStart = leftBrace + 1; // after '{'
+        for (int i = subStart; i < eval.length(); i++) {
+          switch (eval.charAt(i)) {
+            case '}':
+              if (matchedLen > 0) { // match
+                result[SUB_START_IDX] = subStart;
+                result[SUB_END_IDX] = subStart + matchedLen;
+                break match_loop;
+              }
+              // fall through to skip 1 char
+            case ' ':
+            case '$':
+              matchStart = i + 1;
+              continue match_loop;
+            default:
+              matchedLen++;
+          }
+        }
+        // scanned from "${"  to the end of eval, and no reset via ' ', '$':
+        //    no match!
+        break;
+      } else {
+        // not a start of a variable
+        //
+        matchStart = leftBrace + 1;
+      }
+    }
+    return result;
+  }
+
+  /**
+   * Attempts to repeatedly expand the value {@code expr} by replacing the
+   * left-most substring of the form "${var}" in the following precedence order
+   * <ol>
+   *   <li>by the value of the environment variable "var" if defined</li>
+   *   <li>by the value of the Java system property "var" if defined</li>
+   *   <li>by the value of the configuration key "var" if defined</li>
+   * </ol>
+   *
+   * If var is unbounded the current state of expansion "prefix${var}suffix" is
+   * returned.
+   *
+   * If a cycle is detected: replacing var1 requires replacing var2 ... requires
+   * replacing var1, i.e., the cycle is shorter than
+   * {@link RaftProperties#MAX_SUBST} then the original expr is returned.
+   *
+   * @param expr the literal value of a config key
+   * @return null if expr is null, otherwise the value resulting from expanding
+   * expr using the algorithm above.
+   * @throws IllegalArgumentException when more than
+   * {@link RaftProperties#MAX_SUBST} replacements are required
+   */
+  private String substituteVars(String expr) {
+    if (expr == null) {
+      return null;
+    }
+    String eval = expr;
+    Set<String> evalSet = null;
+    for(int s = 0; s < MAX_SUBST; s++) {
+      final int[] varBounds = findSubVariable(eval);
+      if (varBounds[SUB_START_IDX] == -1) {
+        return eval;
+      }
+      final String var = eval.substring(varBounds[SUB_START_IDX],
+          varBounds[SUB_END_IDX]);
+      String val = null;
+      try {
+        if (var.startsWith("env.") && 4 < var.length()) {
+          String v = var.substring(4);
+          int i = 0;
+          for (; i < v.length(); i++) {
+            char c = v.charAt(i);
+            if (c == ':' && i < v.length() - 1 && v.charAt(i + 1) == '-') {
+              val = getenv(v.substring(0, i));
+              if (val == null || val.length() == 0) {
+                val = v.substring(i + 2);
+              }
+              break;
+            } else if (c == '-') {
+              val = getenv(v.substring(0, i));
+              if (val == null) {
+                val = v.substring(i + 1);
+              }
+              break;
+            }
+          }
+          if (i == v.length()) {
+            val = getenv(v);
+          }
+        } else {
+          val = getProperty(var);
+        }
+      } catch(SecurityException se) {
+        LOG.warn("Unexpected SecurityException in Configuration", se);
+      }
+      if (val == null) {
+        val = getRaw(var);
+      }
+      if (val == null) {
+        return eval; // return literal ${var}: var is unbound
+      }
+
+      // prevent recursive resolution
+      //
+      final int dollar = varBounds[SUB_START_IDX] - "${".length();
+      final int afterRightBrace = varBounds[SUB_END_IDX] + "}".length();
+      final String refVar = eval.substring(dollar, afterRightBrace);
+      if (evalSet == null) {
+        evalSet = new HashSet<>();
+      }
+      if (!evalSet.add(refVar)) {
+        return expr; // return original expression if there is a loop
+      }
+
+      // substitute
+      eval = eval.substring(0, dollar)
+             + val
+             + eval.substring(afterRightBrace);
+    }
+    throw new IllegalStateException("Variable substitution depth too large: "
+                                    + MAX_SUBST + " " + expr);
+  }
+
+  String getenv(String name) {
+    return System.getenv(name);
+  }
+
+  String getProperty(String key) {
+    return System.getProperty(key);
+  }
+
+  /**
+   * Get the value of the <code>name</code> property, <code>null</code> if
+   * no such property exists. If the key is deprecated, it returns the value of
+   * the first key which replaces the deprecated key and is not null.
+   *
+   * Values are processed for <a href="#VariableExpansion">variable expansion</a>
+   * before being returned.
+   *
+   * @param name the property name, will be trimmed before get value.
+   * @return the value of the <code>name</code> or its replacing property,
+   *         or null if no such property exists.
+   */
+  public String get(String name) {
+    return substituteVars(getRaw(name));
+  }
+
+  /**
+   * Get the value of the <code>name</code> property as a trimmed <code>String</code>,
+   * <code>null</code> if no such property exists.
+   * If the key is deprecated, it returns the value of
+   * the first key which replaces the deprecated key and is not null
+   *
+   * Values are processed for <a href="#VariableExpansion">variable expansion</a>
+   * before being returned.
+   *
+   * @param name the property name.
+   * @return the value of the <code>name</code> or its replacing property,
+   *         or null if no such property exists.
+   */
+  public String getTrimmed(String name) {
+    String value = get(name);
+
+    if (null == value) {
+      return null;
+    } else {
+      return value.trim();
+    }
+  }
+
+  /**
+   * Get the value of the <code>name</code> property as a trimmed <code>String</code>,
+   * <code>defaultValue</code> if no such property exists.
+   * See @{Configuration#getTrimmed} for more details.
+   *
+   * @param name          the property name.
+   * @param defaultValue  the property default value.
+   * @return              the value of the <code>name</code> or defaultValue
+   *                      if it is not set.
+   */
+  public String getTrimmed(String name, String defaultValue) {
+    String ret = getTrimmed(name);
+    return ret == null ? defaultValue : ret;
+  }
+
+  /**
+   * Get the value of the <code>name</code> property, without doing
+   * <a href="#VariableExpansion">variable expansion</a>.If the key is
+   * deprecated, it returns the value of the first key which replaces
+   * the deprecated key and is not null.
+   *
+   * @param name the property name.
+   * @return the value of the <code>name</code> property or
+   *         its replacing property and null if no such property exists.
+   */
+  public String getRaw(String name) {
+    return getProps().getProperty(name.trim());
+  }
+
+  /**
+   * Set the <code>value</code> of the <code>name</code> property. If
+   * <code>name</code> is deprecated, it also sets the <code>value</code> to
+   * the keys that replace the deprecated key. Name will be trimmed before put
+   * into configuration.
+   *
+   * @param name property name.
+   * @param value property value.
+   * @throws IllegalArgumentException when the value or name is null.
+   */
+  public void set(String name, String value) {
+    Preconditions.checkArgument(name != null, "Property name must not be null");
+    Preconditions.checkArgument(value != null,
+        "The value of property " + name + " must not be null");
+    name = name.trim();
+    getProps();
+
+    getOverlay().setProperty(name, value);
+    getProps().setProperty(name, value);
+  }
+
+  /**
+   * Unset a previously set property.
+   */
+  public synchronized void unset(String name) {
+    getOverlay().remove(name);
+    getProps().remove(name);
+  }
+
+  /**
+   * Sets a property if it is currently unset.
+   * @param name the property name
+   * @param value the new value
+   */
+  public synchronized void setIfUnset(String name, String value) {
+    if (get(name) == null) {
+      set(name, value);
+    }
+  }
+
+  private synchronized Properties getOverlay() {
+    if (overlay == null){
+      overlay = new Properties();
+    }
+    return overlay;
+  }
+
+  /**
+   * Get the value of the <code>name</code>. If the key is deprecated,
+   * it returns the value of the first key which replaces the deprecated key
+   * and is not null.
+   * If no such property exists,
+   * then <code>defaultValue</code> is returned.
+   *
+   * @param name property name, will be trimmed before get value.
+   * @param defaultValue default value.
+   * @return property value, or <code>defaultValue</code> if the property
+   *         doesn't exist.
+   */
+  public String get(String name, String defaultValue) {
+    return substituteVars(getProps().getProperty(name, defaultValue));
+  }
+
+  /**
+   * Get the value of the <code>name</code> property as an <code>int</code>.
+   *
+   * If no such property exists, the provided default value is returned,
+   * or if the specified value is not a valid <code>int</code>,
+   * then an error is thrown.
+   *
+   * @param name property name.
+   * @param defaultValue default value.
+   * @throws NumberFormatException when the value is invalid
+   * @return property value as an <code>int</code>,
+   *         or <code>defaultValue</code>.
+   */
+  public int getInt(String name, int defaultValue) {
+    String valueString = getTrimmed(name);
+    if (valueString == null)
+      return defaultValue;
+    String hexString = getHexDigits(valueString);
+    if (hexString != null) {
+      return Integer.parseInt(hexString, 16);
+    }
+    return Integer.parseInt(valueString);
+  }
+
+  /**
+   * Get the value of the <code>name</code> property as a set of comma-delimited
+   * <code>int</code> values.
+   *
+   * If no such property exists, an empty array is returned.
+   *
+   * @param name property name
+   * @return property value interpreted as an array of comma-delimited
+   *         <code>int</code> values
+   */
+  public int[] getInts(String name) {
+    String[] strings = getTrimmedStrings(name);
+    int[] ints = new int[strings.length];
+    for (int i = 0; i < strings.length; i++) {
+      ints[i] = Integer.parseInt(strings[i]);
+    }
+    return ints;
+  }
+
+  /**
+   * Set the value of the <code>name</code> property to an <code>int</code>.
+   *
+   * @param name property name.
+   * @param value <code>int</code> value of the property.
+   */
+  public void setInt(String name, int value) {
+    set(name, Integer.toString(value));
+  }
+
+
+  /**
+   * Get the value of the <code>name</code> property as a <code>long</code>.
+   * If no such property exists, the provided default value is returned,
+   * or if the specified value is not a valid <code>long</code>,
+   * then an error is thrown.
+   *
+   * @param name property name.
+   * @param defaultValue default value.
+   * @throws NumberFormatException when the value is invalid
+   * @return property value as a <code>long</code>,
+   *         or <code>defaultValue</code>.
+   */
+  public long getLong(String name, long defaultValue) {
+    String valueString = getTrimmed(name);
+    if (valueString == null)
+      return defaultValue;
+    String hexString = getHexDigits(valueString);
+    if (hexString != null) {
+      return Long.parseLong(hexString, 16);
+    }
+    return Long.parseLong(valueString);
+  }
+
+  private String getHexDigits(String value) {
+    boolean negative = false;
+    String str = value;
+    String hexString;
+    if (value.startsWith("-")) {
+      negative = true;
+      str = value.substring(1);
+    }
+    if (str.startsWith("0x") || str.startsWith("0X")) {
+      hexString = str.substring(2);
+      if (negative) {
+        hexString = "-" + hexString;
+      }
+      return hexString;
+    }
+    return null;
+  }
+
+  /**
+   * Set the value of the <code>name</code> property to a <code>long</code>.
+   *
+   * @param name property name.
+   * @param value <code>long</code> value of the property.
+   */
+  public void setLong(String name, long value) {
+    set(name, Long.toString(value));
+  }
+
+  /**
+   * Get the value of the <code>name</code> property as a <code>float</code>.
+   * If no such property exists, the provided default value is returned,
+   * or if the specified value is not a valid <code>float</code>,
+   * then an error is thrown.
+   *
+   * @param name property name.
+   * @param defaultValue default value.
+   * @throws NumberFormatException when the value is invalid
+   * @return property value as a <code>float</code>,
+   *         or <code>defaultValue</code>.
+   */
+  public float getFloat(String name, float defaultValue) {
+    String valueString = getTrimmed(name);
+    if (valueString == null)
+      return defaultValue;
+    return Float.parseFloat(valueString);
+  }
+
+  /**
+   * Set the value of the <code>name</code> property to a <code>float</code>.
+   *
+   * @param name property name.
+   * @param value property value.
+   */
+  public void setFloat(String name, float value) {
+    set(name,Float.toString(value));
+  }
+
+  /**
+   * Get the value of the <code>name</code> property as a <code>double</code>.
+   * If no such property exists, the provided default value is returned,
+   * or if the specified value is not a valid <code>double</code>,
+   * then an error is thrown.
+   *
+   * @param name property name.
+   * @param defaultValue default value.
+   * @throws NumberFormatException when the value is invalid
+   * @return property value as a <code>double</code>,
+   *         or <code>defaultValue</code>.
+   */
+  public double getDouble(String name, double defaultValue) {
+    String valueString = getTrimmed(name);
+    if (valueString == null)
+      return defaultValue;
+    return Double.parseDouble(valueString);
+  }
+
+  /**
+   * Set the value of the <code>name</code> property to a <code>double</code>.
+   *
+   * @param name property name.
+   * @param value property value.
+   */
+  public void setDouble(String name, double value) {
+    set(name,Double.toString(value));
+  }
+
+  /**
+   * Get the value of the <code>name</code> property as a <code>boolean</code>.
+   * If no such property is specified, or if the specified value is not a valid
+   * <code>boolean</code>, then <code>defaultValue</code> is returned.
+   *
+   * @param name property name.
+   * @param defaultValue default value.
+   * @return property value as a <code>boolean</code>,
+   *         or <code>defaultValue</code>.
+   */
+  public boolean getBoolean(String name, boolean defaultValue) {
+    String valueString = getTrimmed(name);
+    return StringUtils.string2boolean(valueString, defaultValue);
+  }
+
+  /**
+   * Set the value of the <code>name</code> property to a <code>boolean</code>.
+   *
+   * @param name property name.
+   * @param value <code>boolean</code> value of the property.
+   */
+  public void setBoolean(String name, boolean value) {
+    set(name, Boolean.toString(value));
+  }
+
+  /**
+   * Set the given property, if it is currently unset.
+   * @param name property name
+   * @param value new value
+   */
+  public void setBooleanIfUnset(String name, boolean value) {
+    setIfUnset(name, Boolean.toString(value));
+  }
+
+  /**
+   * Set the value of the <code>name</code> property to the given type. This
+   * is equivalent to <code>set(&lt;name&gt;, value.toString())</code>.
+   * @param name property name
+   * @param value new value
+   */
+  public <T extends Enum<T>> void setEnum(String name, T value) {
+    set(name, value.toString());
+  }
+
+  /**
+   * Return value matching this enumerated type.
+   * Note that the returned value is trimmed by this method.
+   * @param name Property name
+   * @param defaultValue Value returned if no mapping exists
+   * @throws IllegalArgumentException If mapping is illegal for the type
+   * provided
+   */
+  public <T extends Enum<T>> T getEnum(String name, T defaultValue) {
+    final String val = getTrimmed(name);
+    return null == val
+      ? defaultValue
+      : Enum.valueOf(defaultValue.getDeclaringClass(), val);
+  }
+
+  enum ParsedTimeDuration {
+    NS {
+      TimeUnit unit() { return TimeUnit.NANOSECONDS; }
+      String suffix() { return "ns"; }
+    },
+    US {
+      TimeUnit unit() { return TimeUnit.MICROSECONDS; }
+      String suffix() { return "us"; }
+    },
+    MS {
+      TimeUnit unit() { return TimeUnit.MILLISECONDS; }
+      String suffix() { return "ms"; }
+    },
+    S {
+      TimeUnit unit() { return TimeUnit.SECONDS; }
+      String suffix() { return "s"; }
+    },
+    M {
+      TimeUnit unit() { return TimeUnit.MINUTES; }
+      String suffix() { return "m"; }
+    },
+    H {
+      TimeUnit unit() { return TimeUnit.HOURS; }
+      String suffix() { return "h"; }
+    },
+    D {
+      TimeUnit unit() { return TimeUnit.DAYS; }
+      String suffix() { return "d"; }
+    };
+    abstract TimeUnit unit();
+    abstract String suffix();
+    static ParsedTimeDuration unitFor(String s) {
+      for (ParsedTimeDuration ptd : values()) {
+        // iteration order is in decl order, so SECONDS matched last
+        if (s.endsWith(ptd.suffix())) {
+          return ptd;
+        }
+      }
+      return null;
+    }
+    static ParsedTimeDuration unitFor(TimeUnit unit) {
+      for (ParsedTimeDuration ptd : values()) {
+        if (ptd.unit() == unit) {
+          return ptd;
+        }
+      }
+      return null;
+    }
+  }
+
+  /**
+   * Set the value of <code>name</code> to the given time duration. This
+   * is equivalent to <code>set(&lt;name&gt;, value + &lt;time suffix&gt;)</code>.
+   * @param name Property name
+   * @param value Time duration
+   * @param unit Unit of time
+   */
+  public void setTimeDuration(String name, long value, TimeUnit unit) {
+    set(name, value + ParsedTimeDuration.unitFor(unit).suffix());
+  }
+
+  /**
+   * Return time duration in the given time unit. Valid units are encoded in
+   * properties as suffixes: nanoseconds (ns), microseconds (us), milliseconds
+   * (ms), seconds (s), minutes (m), hours (h), and days (d).
+   * @param name Property name
+   * @param defaultValue Value returned if no mapping exists.
+   * @param unit Unit to convert the stored property, if it exists.
+   * @throws NumberFormatException If the property stripped of its unit is not
+   *         a number
+   */
+  public long getTimeDuration(String name, long defaultValue, TimeUnit unit) {
+    String vStr = get(name);
+    if (null == vStr) {
+      return defaultValue;
+    }
+    vStr = vStr.trim();
+    return getTimeDurationHelper(name, vStr, unit);
+  }
+
+  private long getTimeDurationHelper(String name, String vStr, TimeUnit unit) {
+    ParsedTimeDuration vUnit = ParsedTimeDuration.unitFor(vStr);
+    if (null == vUnit) {
+      LOG.warn("No unit for " + name + "(" + vStr + ") assuming " + unit);
+      vUnit = ParsedTimeDuration.unitFor(unit);
+    } else {
+      vStr = vStr.substring(0, vStr.lastIndexOf(vUnit.suffix()));
+    }
+    return unit.convert(Long.parseLong(vStr), vUnit.unit());
+  }
+
+  public long[] getTimeDurations(String name, TimeUnit unit) {
+    String[] strings = getTrimmedStrings(name);
+    long[] durations = new long[strings.length];
+    for (int i = 0; i < strings.length; i++) {
+      durations[i] = getTimeDurationHelper(name, strings[i], unit);
+    }
+    return durations;
+  }
+
+  /**
+   * Get the value of the <code>name</code> property as a <code>Pattern</code>.
+   * If no such property is specified, or if the specified value is not a valid
+   * <code>Pattern</code>, then <code>DefaultValue</code> is returned.
+   * Note that the returned value is NOT trimmed by this method.
+   *
+   * @param name property name
+   * @param defaultValue default value
+   * @return property value as a compiled Pattern, or defaultValue
+   */
+  public Pattern getPattern(String name, Pattern defaultValue) {
+    String valString = get(name);
+    if (null == valString || valString.isEmpty()) {
+      return defaultValue;
+    }
+    try {
+      return Pattern.compile(valString);
+    } catch (PatternSyntaxException pse) {
+      LOG.warn("Regular expression '" + valString + "' for property '" +
+               name + "' not valid. Using default", pse);
+      return defaultValue;
+    }
+  }
+
+  /**
+   * Set the given property to <code>Pattern</code>.
+   * If the pattern is passed as null, sets the empty pattern which results in
+   * further calls to getPattern(...) returning the default value.
+   *
+   * @param name property name
+   * @param pattern new value
+   */
+  public void setPattern(String name, Pattern pattern) {
+    assert pattern != null : "Pattern cannot be null";
+    set(name, pattern.pattern());
+  }
+
+  /**
+   * A class that represents a set of positive integer ranges. It parses
+   * strings of the form: "2-3,5,7-" where ranges are separated by comma and
+   * the lower/upper bounds are separated by dash. Either the lower or upper
+   * bound may be omitted meaning all values up to or over. So the string
+   * above means 2, 3, 5, and 7, 8, 9, ...
+   */
+  public static class IntegerRanges implements Iterable<Integer>{
+    private static class Range {
+      int start;
+      int end;
+    }
+
+    private static class RangeNumberIterator implements Iterator<Integer> {
+      Iterator<Range> internal;
+      int at;
+      int end;
+
+      public RangeNumberIterator(List<Range> ranges) {
+        if (ranges != null) {
+          internal = ranges.iterator();
+        }
+        at = -1;
+        end = -2;
+      }
+
+      @Override
+      public boolean hasNext() {
+        if (at <= end) {
+          return true;
+        } else if (internal != null){
+          return internal.hasNext();
+        }
+        return false;
+      }
+
+      @Override
+      public Integer next() {
+        if (at <= end) {
+          at++;
+          return at - 1;
+        } else if (internal != null){
+          Range found = internal.next();
+          if (found != null) {
+            at = found.start;
+            end = found.end;
+            at++;
+            return at - 1;
+          }
+        }
+        return null;
+      }
+
+      @Override
+      public void remove() {
+        throw new UnsupportedOperationException();
+      }
+    }
+
+    List<Range> ranges = new ArrayList<>();
+
+    public IntegerRanges() {
+    }
+
+    public IntegerRanges(String newValue) {
+      StringTokenizer itr = new StringTokenizer(newValue, ",");
+      while (itr.hasMoreTokens()) {
+        String rng = itr.nextToken().trim();
+        String[] parts = rng.split("-", 3);
+        if (parts.length < 1 || parts.length > 2) {
+          throw new IllegalArgumentException("integer range badly formed: " +
+                                             rng);
+        }
+        Range r = new Range();
+        r.start = convertToInt(parts[0], 0);
+        if (parts.length == 2) {
+          r.end = convertToInt(parts[1], Integer.MAX_VALUE);
+        } else {
+          r.end = r.start;
+        }
+        if (r.start > r.end) {
+          throw new IllegalArgumentException("IntegerRange from " + r.start +
+                                             " to " + r.end + " is invalid");
+        }
+        ranges.add(r);
+      }
+    }
+
+    /**
+     * Convert a string to an int treating empty strings as the default value.
+     * @param value the string value
+     * @param defaultValue the value for if the string is empty
+     * @return the desired integer
+     */
+    private static int convertToInt(String value, int defaultValue) {
+      String trim = value.trim();
+      if (trim.length() == 0) {
+        return defaultValue;
+      }
+      return Integer.parseInt(trim);
+    }
+
+    /**
+     * Is the given value in the set of ranges
+     * @param value the value to check
+     * @return is the value in the ranges?
+     */
+    public boolean isIncluded(int value) {
+      for(Range r: ranges) {
+        if (r.start <= value && value <= r.end) {
+          return true;
+        }
+      }
+      return false;
+    }
+
+    /**
+     * @return true if there are no values in this range, else false.
+     */
+    public boolean isEmpty() {
+      return ranges == null || ranges.isEmpty();
+    }
+
+    @Override
+    public String toString() {
+      StringBuilder result = new StringBuilder();
+      boolean first = true;
+      for(Range r: ranges) {
+        if (first) {
+          first = false;
+        } else {
+          result.append(',');
+        }
+        result.append(r.start);
+        result.append('-');
+        result.append(r.end);
+      }
+      return result.toString();
+    }
+
+    @Override
+    public Iterator<Integer> iterator() {
+      return new RangeNumberIterator(ranges);
+    }
+
+  }
+
+  /**
+   * Parse the given attribute as a set of integer ranges
+   * @param name the attribute name
+   * @param defaultValue the default value if it is not set
+   * @return a new set of ranges from the configured value
+   */
+  public IntegerRanges getRange(String name, String defaultValue) {
+    return new IntegerRanges(get(name, defaultValue));
+  }
+
+  /**
+   * Get the comma delimited values of the <code>name</code> property as
+   * an array of <code>String</code>s, trimmed of the leading and trailing whitespace.
+   * If no such property is specified then an empty array is returned.
+   *
+   * @param name property name.
+   * @return property value as an array of trimmed <code>String</code>s,
+   *         or empty array.
+   */
+  public String[] getTrimmedStrings(String name) {
+    String valueString = get(name);
+    return StringUtils.getTrimmedStrings(valueString);
+  }
+
+  /**
+   * Load a class by name.
+   *
+   * @param name the class name.
+   * @return the class object.
+   * @throws ClassNotFoundException if the class is not found.
+   */
+  public Class<?> getClassByName(String name) throws ClassNotFoundException {
+    Class<?> ret = getClassByNameOrNull(name);
+    if (ret == null) {
+      throw new ClassNotFoundException("Class " + name + " not found");
+    }
+    return ret;
+  }
+
+  private static final Map<ClassLoader, Map<String, WeakReference<Class<?>>>>
+      CACHE_CLASSES = new WeakHashMap<>();
+
+  /**
+   * Sentinel value to store negative cache results in {@link #CACHE_CLASSES}.
+   */
+  private static final Class<?> NEGATIVE_CACHE_SENTINEL =
+      NegativeCacheSentinel.class;
+
+  /**
+   * Load a class by name, returning null rather than throwing an exception
+   * if it couldn't be loaded. This is to avoid the overhead of creating
+   * an exception.
+   *
+   * @param name the class name
+   * @return the class object, or null if it could not be found.
+   */
+  public Class<?> getClassByNameOrNull(String name) {
+    Map<String, WeakReference<Class<?>>> map;
+
+    synchronized (CACHE_CLASSES) {
+      map = CACHE_CLASSES.get(classLoader);
+      if (map == null) {
+        map = Collections.synchronizedMap(
+          new WeakHashMap<String, WeakReference<Class<?>>>());
+        CACHE_CLASSES.put(classLoader, map);
+      }
+    }
+
+    Class<?> clazz = null;
+    WeakReference<Class<?>> ref = map.get(name);
+    if (ref != null) {
+       clazz = ref.get();
+    }
+
+    if (clazz == null) {
+      try {
+        clazz = Class.forName(name, true, classLoader);
+      } catch (ClassNotFoundException e) {
+        // Leave a marker that the class isn't found
+        map.put(name, new WeakReference<>(NEGATIVE_CACHE_SENTINEL));
+        return null;
+      }
+      // two putters can race here, but they'll put the same class
+      map.put(name, new WeakReference<>(clazz));
+      return clazz;
+    } else if (clazz == NEGATIVE_CACHE_SENTINEL) {
+      return null; // not found
+    } else {
+      // cache hit
+      return clazz;
+    }
+  }
+
+  /**
+   * Get the value of the <code>name</code> property
+   * as an array of <code>Class</code>.
+   * The value of the property specifies a list of comma separated class names.
+   * If no such property is specified, then <code>defaultValue</code> is
+   * returned.
+   *
+   * @param name the property name.
+   * @param defaultValue default value.
+   * @return property value as a <code>Class[]</code>,
+   *         or <code>defaultValue</code>.
+   */
+  public Class<?>[] getClasses(String name, Class<?> ... defaultValue) {
+    String valueString = getRaw(name);
+    if (null == valueString) {
+      return defaultValue;
+    }
+    String[] classnames = getTrimmedStrings(name);
+    try {
+      Class<?>[] classes = new Class<?>[classnames.length];
+      for(int i = 0; i < classnames.length; i++) {
+        classes[i] = getClassByName(classnames[i]);
+      }
+      return classes;
+    } catch (ClassNotFoundException e) {
+      throw new RuntimeException(e);
+    }
+  }
+
+  /**
+   * Get the value of the <code>name</code> property as a <code>Class</code>.
+   * If no such property is specified, then <code>defaultValue</code> is
+   * returned.
+   *
+   * @param name the class name.
+   * @param defaultValue default value.
+   * @return property value as a <code>Class</code>,
+   *         or <code>defaultValue</code>.
+   */
+  public Class<?> getClass(String name, Class<?> defaultValue) {
+    String valueString = getTrimmed(name);
+    if (valueString == null)
+      return defaultValue;
+    try {
+      return getClassByName(valueString);
+    } catch (ClassNotFoundException e) {
+      throw new RuntimeException(e);
+    }
+  }
+
+  /**
+   * Get the value of the <code>name</code> property as a <code>Class</code>
+   * implementing the interface specified by <code>xface</code>.
+   *
+   * If no such property is specified, then <code>defaultValue</code> is
+   * returned.
+   *
+   * An exception is thrown if the returned class does not implement the named
+   * interface.
+   *
+   * @param name the class name.
+   * @param defaultValue default value.
+   * @param xface the interface implemented by the named class.
+   * @return property value as a <code>Class</code>,
+   *         or <code>defaultValue</code>.
+   */
+  public <U> Class<? extends U> getClass(String name,
+                                         Class<? extends U> defaultValue,
+                                         Class<U> xface) {
+    try {
+      Class<?> theClass = getClass(name, defaultValue);
+      if (theClass != null && !xface.isAssignableFrom(theClass))
+        throw new RuntimeException(theClass+" not "+xface.getName());
+      else if (theClass != null)
+        return theClass.asSubclass(xface);
+      else
+        return null;
+    } catch (Exception e) {
+      throw new RuntimeException(e);
+    }
+  }
+
+  /**
+   * Set the value of the <code>name</code> property to the name of a
+   * <code>theClass</code> implementing the given interface <code>xface</code>.
+   *
+   * An exception is thrown if <code>theClass</code> does not implement the
+   * interface <code>xface</code>.
+   *
+   * @param name property name.
+   * @param theClass property value.
+   * @param xface the interface implemented by the named class.
+   */
+  public void setClass(String name, Class<?> theClass, Class<?> xface) {
+    if (!xface.isAssignableFrom(theClass))
+      throw new RuntimeException(theClass+" not "+xface.getName());
+    set(name, theClass.getName());
+  }
+
+  /**
+   * Get the {@link URL} for the named resource.
+   *
+   * @param name resource name.
+   * @return the url for the named resource.
+   */
+  public URL getResource(String name) {
+    return classLoader.getResource(name);
+  }
+
+  protected synchronized Properties getProps() {
+    if (properties == null) {
+      properties = new Properties();
+      Map<String, String[]> backup =
+          new ConcurrentHashMap<>(updatingResource);
+      loadResources(properties, resources);
+
+      if (overlay != null) {
+        properties.putAll(overlay);
+        for (Entry<Object,Object> item: overlay.entrySet()) {
+          String key = (String) item.getKey();
+          String[] source = backup.get(key);
+          if(source != null) {
+            updatingResource.put(key, source);
+          }
+        }
+      }
+    }
+    return properties;
+  }
+
+  /**
+   * Return the number of keys in the configuration.
+   *
+   * @return number of keys in the configuration.
+   */
+  public int size() {
+    return getProps().size();
+  }
+
+  /**
+   * Clears all keys from the configuration.
+   */
+  public void clear() {
+    getProps().clear();
+    getOverlay().clear();
+  }
+
+  private Document parse(DocumentBuilder builder, URL url)
+      throws IOException, SAXException {
+    LOG.debug("parsing URL " + url);
+    if (url == null) {
+      return null;
+    }
+
+    URLConnection connection = url.openConnection();
+    if (connection instanceof JarURLConnection) {
+      // Disable caching for JarURLConnection to avoid sharing JarFile
+      // with other users.
+      connection.setUseCaches(false);
+    }
+    return parse(builder, connection.getInputStream(), url.toString());
+  }
+
+  private Document parse(DocumentBuilder builder, InputStream is,
+      String systemId) throws IOException, SAXException {
+    LOG.debug("parsing input stream " + is);
+    if (is == null) {
+      return null;
+    }
+    try {
+      return (systemId == null) ? builder.parse(is) : builder.parse(is,
+          systemId);
+    } finally {
+      is.close();
+    }
+  }
+
+  private void loadResources(Properties properties,
+                             ArrayList<Resource> resources) {
+    if(loadDefaults) {
+      for (String resource : defaultResources) {
+        loadResource(properties, new Resource(resource));
+      }
+    }
+
+    for (int i = 0; i < resources.size(); i++) {
+      Resource ret = loadResource(properties, resources.get(i));
+      if (ret != null) {
+        resources.set(i, ret);
+      }
+    }
+  }
+
+  private Resource loadResource(Properties properties, Resource wrapper) {
+    String name = UNKNOWN_RESOURCE;
+    try {
+      Object resource = wrapper.getResource();
+      name = wrapper.getName();
+
+      DocumentBuilderFactory docBuilderFactory
+        = DocumentBuilderFactory.newInstance();
+      //ignore all comments inside the xml file
+      docBuilderFactory.setIgnoringComments(true);
+
+      //allow includes in the xml file
+      docBuilderFactory.setNamespaceAware(true);
+      try {
+          docBuilderFactory.setXIncludeAware(true);
+      } catch (UnsupportedOperationException e) {
+        LOG.error("Failed to set setXIncludeAware(true) for parser "
+                + docBuilderFactory
+                + ":" + e,
+                e);
+      }
+      DocumentBuilder builder = docBuilderFactory.newDocumentBuilder();
+      Document doc = null;
+      Element root = null;
+      boolean returnCachedProperties = false;
+
+      if (resource instanceof URL) { // an URL resource
+        doc = parse(builder, (URL) resource);
+      } else if (resource instanceof String) { // a CLASSPATH resource
+        URL url = getResource((String) resource);
+        doc = parse(builder, url);
+      } else if (resource instanceof InputStream) {
+        doc = parse(builder, (InputStream) resource, null);
+        returnCachedProperties = true;
+      } else if (resource instanceof Properties) {
+        overlay(properties, (Properties) resource);
+      } else if (resource instanceof Element) {
+        root = (Element) resource;
+      }
+
+      if (root == null) {
+        if (doc == null) {
+          return null;
+        }
+        root = doc.getDocumentElement();
+      }
+      Properties toAddTo = properties;
+      if(returnCachedProperties) {
+        toAddTo = new Properties();
+      }
+      if (!"configuration".equals(root.getTagName())) {
+        LOG.error("bad conf file: top-level element not <configuration>");
+      }
+      NodeList props = root.getChildNodes();
+      for (int i = 0; i < props.getLength(); i++) {
+        Node propNode = props.item(i);
+        if (!(propNode instanceof Element))
+          continue;
+        Element prop = (Element)propNode;
+        if ("configuration".equals(prop.getTagName())) {
+          loadResource(toAddTo, new Resource(prop, name));
+          continue;
+        }
+        if (!"property".equals(prop.getTagName()))
+          LOG.warn("bad conf file: element not <property>");
+
+        String attr = null;
+        String value = null;
+        boolean finalParameter = false;
+        LinkedList<String> source = new LinkedList<>();
+
+        Attr propAttr = prop.getAttributeNode("name");
+        if (propAttr != null)
+          attr = StringUtils.weakIntern(propAttr.getValue());
+        propAttr = prop.getAttributeNode("value");
+        if (propAttr != null)
+          value = StringUtils.weakIntern(propAttr.getValue());
+        propAttr = prop.getAttributeNode("final");
+        if (propAttr != null)
+          finalParameter = "true".equals(propAttr.getValue());
+        propAttr = prop.getAttributeNode("source");
+        if (propAttr != null)
+          source.add(StringUtils.weakIntern(propAttr.getValue()));
+
+        NodeList fields = prop.getChildNodes();
+        for (int j = 0; j < fields.getLength(); j++) {
+          Node fieldNode = fields.item(j);
+          if (!(fieldNode instanceof Element))
+            continue;
+          Element field = (Element)fieldNode;
+          if ("name".equals(field.getTagName()) && field.hasChildNodes())
+            attr = StringUtils.weakIntern(
+                ((Text)field.getFirstChild()).getData().trim());
+          if ("value".equals(field.getTagName()) && field.hasChildNodes())
+            value = StringUtils.weakIntern(
+                ((Text)field.getFirstChild()).getData());
+          if ("final".equals(field.getTagName()) && field.hasChildNodes())
+            finalParameter = "true".equals(((Text)field.getFirstChild()).getData());
+          if ("source".equals(field.getTagName()) && field.hasChildNodes())
+            source.add(StringUtils.weakIntern(
+                ((Text)field.getFirstChild()).getData()));
+        }
+        source.add(name);
+
+        // Ignore this parameter if it has already been marked as 'final'
+        if (attr != null) {
+          loadProperty(toAddTo, name, attr, value, finalParameter,
+              source.toArray(new String[source.size()]));
+        }
+      }
+
+      if (returnCachedProperties) {
+        overlay(properties, toAddTo);
+        return new Resource(toAddTo, name);
+      }
+      return null;
+    } catch (IOException | DOMException | SAXException |
+        ParserConfigurationException e) {
+      LOG.error("error parsing conf " + name, e);
+      throw new RuntimeException(e);
+    }
+  }
+
+  private void overlay(Properties to, Properties from) {
+    for (Entry<Object, Object> entry: from.entrySet()) {
+      to.put(entry.getKey(), entry.getValue());
+    }
+  }
+
+  private void loadProperty(Properties properties, String name, String attr,
+      String value, boolean finalParameter, String[] source) {
+    if (value != null) {
+      if (!finalParameters.contains(attr)) {
+        properties.setProperty(attr, value);
+        if(source != null) {
+          updatingResource.put(attr, source);
+        }
+      } else if (!value.equals(properties.getProperty(attr))) {
+        LOG.warn(name+":an attempt to override final parameter: "+attr
+            +";  Ignoring.");
+      }
+    }
+    if (finalParameter && attr != null) {
+      finalParameters.add(attr);
+    }
+  }
+
+  /**
+   * Write out the non-default properties in this configuration to the given
+   * {@link OutputStream} using UTF-8 encoding.
+   *
+   * @param out the output stream to write to.
+   */
+  public void writeXml(OutputStream out) throws IOException {
+    writeXml(new OutputStreamWriter(out, "UTF-8"));
+  }
+
+  /**
+   * Write out the non-default properties in this configuration to the given
+   * {@link Writer}.
+   *
+   * @param out the writer to write to.
+   */
+  public void writeXml(Writer out) throws IOException {
+    Document doc = asXmlDocument();
+
+    try {
+      DOMSource source = new DOMSource(doc);
+      StreamResult result = new StreamResult(out);
+      TransformerFactory transFactory = TransformerFactory.newInstance();
+      Transformer transformer = transFactory.newTransformer();
+
+      // Important to not hold Configuration log while writing result, since
+      // 'out' may be an HDFS stream which needs to lock this configuration
+      // from another thread.
+      transformer.transform(source, result);
+    } catch (TransformerException te) {
+      throw new IOException(te);
+    }
+  }
+
+  /**
+   * Return the XML DOM corresponding to this Configuration.
+   */
+  private synchronized Document asXmlDocument() throws IOException {
+    Document doc;
+    try {
+      doc =
+        DocumentBuilderFactory.newInstance().newDocumentBuilder().newDocument();
+    } catch (ParserConfigurationException pe) {
+      throw new IOException(pe);
+    }
+    Element conf = doc.createElement("configuration");
+    doc.appendChild(conf);
+    conf.appendChild(doc.createTextNode("\n"));
+    for (Enumeration<Object> e = properties.keys(); e.hasMoreElements();) {
+      String name = (String)e.nextElement();
+      Object object = properties.get(name);
+      String value;
+      if (object instanceof String) {
+        value = (String) object;
+      }else {
+        continue;
+      }
+      Element propNode = doc.createElement("property");
+      conf.appendChild(propNode);
+
+      Element nameNode = doc.createElement("name");
+      nameNode.appendChild(doc.createTextNode(name));
+      propNode.appendChild(nameNode);
+
+      Element valueNode = doc.createElement("value");
+      valueNode.appendChild(doc.createTextNode(value));
+      propNode.appendChild(valueNode);
+
+      if (updatingResource != null) {
+        String[] sources = updatingResource.get(name);
+        if(sources != null) {
+          for(String s : sources) {
+            Element sourceNode = doc.createElement("source");
+            sourceNode.appendChild(doc.createTextNode(s));
+            propNode.appendChild(sourceNode);
+          }
+        }
+      }
+
+      conf.appendChild(doc.createTextNode("\n"));
+    }
+    return doc;
+  }
+
+  /**
+   * Get the {@link ClassLoader} for this job.
+   *
+   * @return the correct class loader.
+   */
+  public ClassLoader getClassLoader() {
+    return classLoader;
+  }
+
+  /**
+   * Set the class loader that will be used to load the various objects.
+   *
+   * @param classLoader the new class loader.
+   */
+  public void setClassLoader(ClassLoader classLoader) {
+    this.classLoader = classLoader;
+  }
+
+  @Override
+  public String toString() {
+    StringBuilder sb = new StringBuilder();
+    sb.append("Configuration: ");
+    if(loadDefaults) {
+      toString(defaultResources, sb);
+      if(resources.size()>0) {
+        sb.append(", ");
+      }
+    }
+    toString(resources, sb);
+    return sb.toString();
+  }
+
+  private <T> void toString(List<T> resources, StringBuilder sb) {
+    ListIterator<T> i = resources.listIterator();
+    while (i.hasNext()) {
+      if (i.nextIndex() != 0) {
+        sb.append(", ");
+      }
+      sb.append(i.next());
+    }
+  }
+
+  /**
+   * get keys matching the the regex
+   * @return a map with matching keys
+   */
+  public Map<String,String> getValByRegex(String regex) {
+    Pattern p = Pattern.compile(regex);
+
+    Map<String,String> result = new HashMap<>();
+    Matcher m;
+
+    for(Entry<Object,Object> item: getProps().entrySet()) {
+      if (item.getKey() instanceof String &&
+          item.getValue() instanceof String) {
+        m = p.matcher((String)item.getKey());
+        if(m.find()) { // match
+          result.put((String) item.getKey(),
+              substituteVars(getProps().getProperty((String) item.getKey())));
+        }
+      }
+    }
+    return result;
+  }
+
+  /**
+   * A unique class which is used as a sentinel value in the caching
+   * for getClassByName. {@link RaftProperties#getClassByNameOrNull(String)}
+   */
+  private static abstract class NegativeCacheSentinel {}
+}

http://git-wip-us.apache.org/repos/asf/incubator-ratis/blob/7e71a2e0/ratis-common/src/main/java/org/apache/ratis/io/MD5Hash.java
----------------------------------------------------------------------
diff --git a/ratis-common/src/main/java/org/apache/ratis/io/MD5Hash.java b/ratis-common/src/main/java/org/apache/ratis/io/MD5Hash.java
new file mode 100644
index 0000000..44447a1
--- /dev/null
+++ b/ratis-common/src/main/java/org/apache/ratis/io/MD5Hash.java
@@ -0,0 +1,205 @@
+/**
+ * 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.ratis.io;
+
+import java.io.IOException;
+import java.io.DataInput;
+import java.io.DataOutput;
+import java.io.InputStream;
+import java.security.MessageDigest;
+import java.security.NoSuchAlgorithmException;
+import java.util.Arrays;
+
+public class MD5Hash {
+  public static final int MD5_LEN = 16;
+
+  private static final ThreadLocal<MessageDigest> DIGESTER_FACTORY =
+      ThreadLocal.withInitial(() -> {
+    try {
+      return MessageDigest.getInstance("MD5");
+    } catch (NoSuchAlgorithmException e) {
+      throw new RuntimeException(e);
+    }
+  });
+
+  private byte[] digest;
+
+  /** Constructs an MD5Hash. */
+  public MD5Hash() {
+    this.digest = new byte[MD5_LEN];
+  }
+
+  /** Constructs an MD5Hash from a hex string. */
+  public MD5Hash(String hex) {
+    setDigest(hex);
+  }
+
+  /** Constructs an MD5Hash with a specified value. */
+  public MD5Hash(byte[] digest) {
+    if (digest.length != MD5_LEN)
+      throw new IllegalArgumentException("Wrong length: " + digest.length);
+    this.digest = digest;
+  }
+
+  public void readFields(DataInput in) throws IOException {
+    in.readFully(digest);
+  }
+
+  /** Constructs, reads and returns an instance. */
+  public static MD5Hash read(DataInput in) throws IOException {
+    MD5Hash result = new MD5Hash();
+    result.readFields(in);
+    return result;
+  }
+
+  public void write(DataOutput out) throws IOException {
+    out.write(digest);
+  }
+
+  /** Copy the contents of another instance into this instance. */
+  public void set(MD5Hash that) {
+    System.arraycopy(that.digest, 0, this.digest, 0, MD5_LEN);
+  }
+
+  /** Returns the digest bytes. */
+  public byte[] getDigest() { return digest; }
+
+  /** Construct a hash value for a byte array. */
+  public static MD5Hash digest(byte[] data) {
+    return digest(data, 0, data.length);
+  }
+
+  /**
+   * Create a thread local MD5 digester
+   */
+  public static MessageDigest getDigester() {
+    MessageDigest digester = DIGESTER_FACTORY.get();
+    digester.reset();
+    return digester;
+  }
+
+  /** Construct a hash value for the content from the InputStream. */
+  public static MD5Hash digest(InputStream in) throws IOException {
+    final byte[] buffer = new byte[4*1024];
+
+    final MessageDigest digester = getDigester();
+    for(int n; (n = in.read(buffer)) != -1; ) {
+      digester.update(buffer, 0, n);
+    }
+
+    return new MD5Hash(digester.digest());
+  }
+
+  /** Construct a hash value for a byte array. */
+  public static MD5Hash digest(byte[] data, int start, int len) {
+    byte[] digest;
+    MessageDigest digester = getDigester();
+    digester.update(data, start, len);
+    digest = digester.digest();
+    return new MD5Hash(digest);
+  }
+
+  /** Construct a hash value for an array of byte array. */
+  public static MD5Hash digest(byte[][] dataArr, int start, int len) {
+    byte[] digest;
+    MessageDigest digester = getDigester();
+    for (byte[] data : dataArr) {
+      digester.update(data, start, len);
+    }
+    digest = digester.digest();
+    return new MD5Hash(digest);
+  }
+
+  /** Construct a half-sized version of this MD5.  Fits in a long **/
+  public long halfDigest() {
+    long value = 0;
+    for (int i = 0; i < 8; i++)
+      value |= ((digest[i] & 0xffL) << (8*(7-i)));
+    return value;
+  }
+
+  /**
+   * Return a 32-bit digest of the MD5.
+   * @return the first 4 bytes of the md5
+   */
+  public int quarterDigest() {
+    int value = 0;
+    for (int i = 0; i < 4; i++)
+      value |= ((digest[i] & 0xff) << (8*(3-i)));
+    return value;
+  }
+
+  /** Returns true iff <code>o</code> is an MD5Hash whose digest contains the
+   * same values.  */
+  @Override
+  public boolean equals(Object o) {
+    if (!(o instanceof MD5Hash))
+      return false;
+    MD5Hash other = (MD5Hash)o;
+    return Arrays.equals(this.digest, other.digest);
+  }
+
+  /** Returns a hash code value for this object.
+   * Only uses the first 4 bytes, since md5s are evenly distributed.
+   */
+  @Override
+  public int hashCode() {
+    return quarterDigest();
+  }
+
+  private static final char[] HEX_DIGITS =
+      {'0','1','2','3','4','5','6','7','8','9','a','b','c','d','e','f'};
+
+  /** Returns a string representation of this object. */
+  @Override
+  public String toString() {
+    StringBuilder buf = new StringBuilder(MD5_LEN*2);
+    for (int i = 0; i < MD5_LEN; i++) {
+      int b = digest[i];
+      buf.append(HEX_DIGITS[(b >> 4) & 0xf]);
+      buf.append(HEX_DIGITS[b & 0xf]);
+    }
+    return buf.toString();
+  }
+
+  /** Sets the digest value from a hex string. */
+  public void setDigest(String hex) {
+    if (hex.length() != MD5_LEN*2)
+      throw new IllegalArgumentException("Wrong length: " + hex.length());
+    byte[] digest = new byte[MD5_LEN];
+    for (int i = 0; i < MD5_LEN; i++) {
+      int j = i << 1;
+      digest[i] = (byte)(charToNibble(hex.charAt(j)) << 4 |
+          charToNibble(hex.charAt(j+1)));
+    }
+    this.digest = digest;
+  }
+
+  private static int charToNibble(char c) {
+    if (c >= '0' && c <= '9') {
+      return c - '0';
+    } else if (c >= 'a' && c <= 'f') {
+      return 0xa + (c - 'a');
+    } else if (c >= 'A' && c <= 'F') {
+      return 0xA + (c - 'A');
+    } else {
+      throw new RuntimeException("Not a hex character: " + c);
+    }
+  }
+}

http://git-wip-us.apache.org/repos/asf/incubator-ratis/blob/7e71a2e0/ratis-common/src/main/java/org/apache/ratis/io/nativeio/Errno.java
----------------------------------------------------------------------
diff --git a/ratis-common/src/main/java/org/apache/ratis/io/nativeio/Errno.java b/ratis-common/src/main/java/org/apache/ratis/io/nativeio/Errno.java
new file mode 100644
index 0000000..2dbeadc
--- /dev/null
+++ b/ratis-common/src/main/java/org/apache/ratis/io/nativeio/Errno.java
@@ -0,0 +1,64 @@
+/**
+ * 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.ratis.io.nativeio;
+
+/**
+ * Enum representing POSIX errno values.
+ */
+public enum Errno {
+  EPERM,
+  ENOENT,
+  ESRCH,
+  EINTR,
+  EIO,
+  ENXIO,
+  E2BIG,
+  ENOEXEC,
+  EBADF,
+  ECHILD,
+  EAGAIN,
+  ENOMEM,
+  EACCES,
+  EFAULT,
+  ENOTBLK,
+  EBUSY,
+  EEXIST,
+  EXDEV,
+  ENODEV,
+  ENOTDIR,
+  EISDIR,
+  EINVAL,
+  ENFILE,
+  EMFILE,
+  ENOTTY,
+  ETXTBSY,
+  EFBIG,
+  ENOSPC,
+  ESPIPE,
+  EROFS,
+  EMLINK,
+  EPIPE,
+  EDOM,
+  ERANGE,
+  ELOOP,
+  ENAMETOOLONG,
+  ENOTEMPTY,
+  EOVERFLOW,
+
+  UNKNOWN
+}


[34/54] [abbrv] incubator-ratis git commit: Renamed the packages from raft to ratis in preperation for Apache Incubation - Moved all java packages from org.apache.raft to org.apache.ratis. - Moved native package to org_apache_ratis, and native lib to l

Posted by ji...@apache.org.
http://git-wip-us.apache.org/repos/asf/incubator-ratis/blob/7e71a2e0/raft-server/src/test/java/org/apache/raft/RaftTestUtil.java
----------------------------------------------------------------------
diff --git a/raft-server/src/test/java/org/apache/raft/RaftTestUtil.java b/raft-server/src/test/java/org/apache/raft/RaftTestUtil.java
deleted file mode 100644
index 461dd15..0000000
--- a/raft-server/src/test/java/org/apache/raft/RaftTestUtil.java
+++ /dev/null
@@ -1,305 +0,0 @@
-/**
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- *     http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-package org.apache.raft;
-
-import com.google.common.base.Preconditions;
-import org.apache.commons.lang.RandomStringUtils;
-import org.apache.raft.protocol.Message;
-import org.apache.raft.server.RaftServerConfigKeys;
-import org.apache.raft.server.impl.BlockRequestHandlingInjection;
-import org.apache.raft.server.impl.DelayLocalExecutionInjection;
-import org.apache.raft.server.impl.RaftServerImpl;
-import org.apache.raft.shaded.com.google.protobuf.ByteString;
-import org.apache.raft.shaded.proto.RaftProtos.LogEntryProto;
-import org.apache.raft.shaded.proto.RaftProtos.SMLogEntryProto;
-import org.apache.raft.util.CheckedRunnable;
-import org.junit.Assert;
-import org.slf4j.Logger;
-import org.slf4j.LoggerFactory;
-
-import java.io.File;
-import java.io.IOException;
-import java.io.UnsupportedEncodingException;
-import java.nio.charset.Charset;
-import java.util.Arrays;
-import java.util.Collection;
-import java.util.function.BooleanSupplier;
-import java.util.function.IntSupplier;
-
-import static org.apache.raft.util.ProtoUtils.toByteString;
-
-public class RaftTestUtil {
-  static final Logger LOG = LoggerFactory.getLogger(RaftTestUtil.class);
-
-  public static RaftServerImpl waitForLeader(MiniRaftCluster cluster)
-      throws InterruptedException {
-    final long sleepTime = (cluster.getMaxTimeout() * 3) >> 1;
-    LOG.info(cluster.printServers());
-    RaftServerImpl leader = null;
-    for(int i = 0; leader == null && i < 10; i++) {
-      Thread.sleep(sleepTime);
-      leader = cluster.getLeader();
-    }
-    LOG.info(cluster.printServers());
-    return leader;
-  }
-
-  public static RaftServerImpl waitForLeader(MiniRaftCluster cluster,
-                                             final String leaderId) throws InterruptedException {
-    LOG.info(cluster.printServers());
-    for(int i = 0; !cluster.tryEnforceLeader(leaderId) && i < 10; i++) {
-      RaftServerImpl currLeader = cluster.getLeader();
-      if (LOG.isDebugEnabled()) {
-        LOG.debug("try enforcing leader to " + leaderId + " but "
-            + (currLeader == null? "no leader for this round"
-                : "new leader is " + currLeader.getId()));
-      }
-    }
-    LOG.info(cluster.printServers());
-
-    final RaftServerImpl leader = cluster.getLeader();
-    Assert.assertEquals(leaderId, leader.getId());
-    return leader;
-  }
-
-  public static String waitAndKillLeader(MiniRaftCluster cluster,
-      boolean expectLeader) throws InterruptedException {
-    final RaftServerImpl leader = waitForLeader(cluster);
-    if (!expectLeader) {
-      Assert.assertNull(leader);
-    } else {
-      Assert.assertNotNull(leader);
-      LOG.info("killing leader = " + leader);
-      cluster.killServer(leader.getId());
-    }
-    return leader != null ? leader.getId() : null;
-  }
-
-  public static boolean logEntriesContains(LogEntryProto[] entries,
-      SimpleMessage... expectedMessages) {
-    int idxEntries = 0;
-    int idxExpected = 0;
-    while (idxEntries < entries.length
-        && idxExpected < expectedMessages.length) {
-      if (Arrays.equals(expectedMessages[idxExpected].getContent().toByteArray(),
-          entries[idxEntries].getSmLogEntry().getData().toByteArray())) {
-        ++idxExpected;
-      }
-      ++idxEntries;
-    }
-    return idxExpected == expectedMessages.length;
-  }
-
-  public static void assertLogEntries(Collection<RaftServerImpl> servers,
-                                      SimpleMessage... expectedMessages) {
-    final int size = servers.size();
-    final long count = servers.stream()
-        .filter(RaftServerImpl::isAlive)
-        .map(s -> s.getState().getLog().getEntries(0, Long.MAX_VALUE))
-        .filter(e -> logEntriesContains(e, expectedMessages))
-        .count();
-    if (2*count <= size) {
-      throw new AssertionError("Not in majority: size=" + size
-          + " but count=" + count);
-    }
-  }
-
-  public static void assertLogEntries(LogEntryProto[] entries, long startIndex,
-      long expertedTerm, SimpleMessage... expectedMessages) {
-    Assert.assertEquals(expectedMessages.length, entries.length);
-    for(int i = 0; i < entries.length; i++) {
-      final LogEntryProto e = entries[i];
-      Assert.assertEquals(expertedTerm, e.getTerm());
-      Assert.assertEquals(startIndex + i, e.getIndex());
-      Assert.assertArrayEquals(expectedMessages[i].getContent().toByteArray(),
-          e.getSmLogEntry().getData().toByteArray());
-    }
-  }
-
-  public static class SimpleMessage implements Message {
-    public static SimpleMessage[] create(int numMessages) {
-      return create(numMessages, "m");
-    }
-
-    public static SimpleMessage[] create(int numMessages, String prefix) {
-      final SimpleMessage[] messages = new SimpleMessage[numMessages];
-      for (int i = 0; i < messages.length; i++) {
-        messages[i] = new SimpleMessage(prefix + i);
-      }
-      return messages;
-    }
-
-    final String messageId;
-
-    public SimpleMessage(final String messageId) {
-      this.messageId = messageId;
-    }
-
-    @Override
-    public String toString() {
-      return messageId;
-    }
-
-    @Override
-    public boolean equals(Object obj) {
-      if (obj == this) {
-        return true;
-      } else if (obj == null || !(obj instanceof SimpleMessage)) {
-        return false;
-      } else {
-        final SimpleMessage that = (SimpleMessage)obj;
-        return this.messageId.equals(that.messageId);
-      }
-    }
-
-    @Override
-    public int hashCode() {
-      return messageId.hashCode();
-    }
-
-    @Override
-    public ByteString getContent() {
-      return toByteString(messageId.getBytes(Charset.forName("UTF-8")));
-    }
-  }
-
-  public static class SimpleOperation {
-    private final String op;
-
-    public SimpleOperation(String op) {
-      Preconditions.checkArgument(op != null);
-      this.op = op;
-    }
-
-    @Override
-    public String toString() {
-      return op;
-    }
-
-    @Override
-    public boolean equals(Object obj) {
-      return obj == this ||
-          (obj instanceof SimpleOperation &&
-              ((SimpleOperation) obj).op.equals(op));
-    }
-
-    @Override
-    public int hashCode() {
-      return op.hashCode();
-    }
-
-    public SMLogEntryProto getLogEntryContent() {
-      try {
-        return SMLogEntryProto.newBuilder()
-            .setData(toByteString(op.getBytes("UTF-8"))).build();
-      } catch (UnsupportedEncodingException e) {
-        throw new RuntimeException(e);
-      }
-    }
-  }
-
-  public static File getTestDir(Class<?> caller) throws IOException {
-    File dir = new File(System.getProperty("test.build.data", "target/test/data")
-            + "/" + RandomStringUtils.randomAlphanumeric(10),
-            caller.getSimpleName());
-    if (dir.exists() && !dir.isDirectory()) {
-      throw new IOException(dir + " already exists and is not a directory");
-    } else if (!dir.exists() && !dir.mkdirs()) {
-      throw new IOException("Cannot create directory " + dir);
-    }
-    return dir;
-  }
-
-  public static void block(BooleanSupplier isBlocked) throws InterruptedException {
-    for(; isBlocked.getAsBoolean(); ) {
-      Thread.sleep(RaftServerConfigKeys.RAFT_SERVER_RPC_TIMEOUT_MAX_MS_DEFAULT);
-    }
-  }
-
-  public static void delay(IntSupplier getDelayMs) throws InterruptedException {
-    final int t = getDelayMs.getAsInt();
-    if (t > 0) {
-      Thread.sleep(t);
-    }
-  }
-
-  public static <T extends Throwable> void attempt(
-      int n, long sleepMs, CheckedRunnable<T> runnable)
-      throws T, InterruptedException {
-    for(int i = 1; i <= n; i++) {
-      LOG.info("Attempt #" + i + "/" + n +  ": sleep " + sleepMs + "ms");
-      if (sleepMs > 0) {
-        Thread.sleep(sleepMs);
-      }
-      try {
-        runnable.run();
-        return;
-      } catch (Throwable t) {
-        if (i == n) {
-          throw t;
-        }
-        LOG.warn("Attempt #" + i + "/" + n + ": Ignoring " + t + " and retry.");
-      }
-    }
-  }
-
-  public static String changeLeader(MiniRaftCluster cluster, String oldLeader)
-      throws InterruptedException {
-    cluster.setBlockRequestsFrom(oldLeader, true);
-    String newLeader = oldLeader;
-    for(int i = 0; i < 10 && newLeader.equals(oldLeader); i++) {
-      newLeader = RaftTestUtil.waitForLeader(cluster).getId();
-    }
-    cluster.setBlockRequestsFrom(oldLeader, false);
-    return newLeader;
-  }
-
-  public static void blockQueueAndSetDelay(Collection<RaftServerImpl> servers,
-      DelayLocalExecutionInjection injection, String leaderId, int delayMs,
-      long maxTimeout) throws InterruptedException {
-    // block reqeusts sent to leader if delayMs > 0
-    final boolean block = delayMs > 0;
-    LOG.debug("{} requests sent to leader {} and set {}ms delay for the others",
-        block? "Block": "Unblock", leaderId, delayMs);
-    if (block) {
-      BlockRequestHandlingInjection.getInstance().blockReplier(leaderId);
-    } else {
-      BlockRequestHandlingInjection.getInstance().unblockReplier(leaderId);
-    }
-
-    // delay RaftServerRequest for other servers
-    servers.stream().filter(s -> !s.getId().equals(leaderId))
-        .forEach(s -> {
-          if (block) {
-            injection.setDelayMs(s.getId(), delayMs);
-          } else {
-            injection.removeDelay(s.getId());
-          }
-        });
-
-    Thread.sleep(3 * maxTimeout);
-  }
-
-  public static void setBlockRequestsFrom(String src, boolean block) {
-    if (block) {
-      BlockRequestHandlingInjection.getInstance().blockRequestor(src);
-    } else {
-      BlockRequestHandlingInjection.getInstance().unblockRequestor(src);
-    }
-  }
-}

http://git-wip-us.apache.org/repos/asf/incubator-ratis/blob/7e71a2e0/raft-server/src/test/java/org/apache/raft/server/impl/BlockRequestHandlingInjection.java
----------------------------------------------------------------------
diff --git a/raft-server/src/test/java/org/apache/raft/server/impl/BlockRequestHandlingInjection.java b/raft-server/src/test/java/org/apache/raft/server/impl/BlockRequestHandlingInjection.java
deleted file mode 100644
index 7f7de9a..0000000
--- a/raft-server/src/test/java/org/apache/raft/server/impl/BlockRequestHandlingInjection.java
+++ /dev/null
@@ -1,84 +0,0 @@
-/**
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- *     http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-package org.apache.raft.server.impl;
-
-import org.apache.raft.RaftTestUtil;
-import org.apache.raft.util.CodeInjectionForTesting;
-
-import java.util.Map;
-import java.util.concurrent.ConcurrentHashMap;
-
-/** Inject code to block a server from handling incoming requests. */
-public class BlockRequestHandlingInjection implements CodeInjectionForTesting.Code {
-  private static final BlockRequestHandlingInjection INSTANCE =
-      new BlockRequestHandlingInjection();
-
-  static {
-    CodeInjectionForTesting.put(RaftServerImpl.REQUEST_VOTE, INSTANCE);
-    CodeInjectionForTesting.put(RaftServerImpl.APPEND_ENTRIES, INSTANCE);
-    CodeInjectionForTesting.put(RaftServerImpl.INSTALL_SNAPSHOT, INSTANCE);
-  }
-
-  public static BlockRequestHandlingInjection getInstance() {
-    return INSTANCE;
-  }
-
-  private final Map<String, Boolean> requestors = new ConcurrentHashMap<>();
-  private final Map<String, Boolean> repliers = new ConcurrentHashMap<>();
-
-  private BlockRequestHandlingInjection() {}
-
-  public void blockRequestor(String requestor) {
-    requestors.put(requestor, true);
-  }
-
-  public void unblockRequestor(String requestor) {
-    requestors.remove(requestor);
-  }
-
-  public void blockReplier(String replier) {
-    repliers.put(replier, true);
-  }
-
-  public void unblockReplier(String replier) {
-    repliers.remove(replier);
-  }
-
-  public void unblockAll() {
-    requestors.clear();
-    repliers.clear();
-  }
-
-  @Override
-  public boolean execute(String localId, String remoteId, Object... args) {
-    if (shouldBlock(localId, remoteId)) {
-      try {
-        RaftTestUtil.block(() -> shouldBlock(localId, remoteId));
-        return true;
-      } catch (InterruptedException e) {
-        LOG.debug("Interrupted while blocking request handling from " + remoteId
-            + " to " + localId);
-      }
-    }
-    return false;
-  }
-
-  private boolean shouldBlock(String localId, String remoteId) {
-    return repliers.containsKey(localId) || requestors.containsKey(remoteId);
-  }
-}

http://git-wip-us.apache.org/repos/asf/incubator-ratis/blob/7e71a2e0/raft-server/src/test/java/org/apache/raft/server/impl/DelayLocalExecutionInjection.java
----------------------------------------------------------------------
diff --git a/raft-server/src/test/java/org/apache/raft/server/impl/DelayLocalExecutionInjection.java b/raft-server/src/test/java/org/apache/raft/server/impl/DelayLocalExecutionInjection.java
deleted file mode 100644
index 26b89d8..0000000
--- a/raft-server/src/test/java/org/apache/raft/server/impl/DelayLocalExecutionInjection.java
+++ /dev/null
@@ -1,67 +0,0 @@
-/**
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- *     http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-package org.apache.raft.server.impl;
-
-import org.apache.raft.RaftTestUtil;
-import org.apache.raft.util.CodeInjectionForTesting;
-
-import java.util.Arrays;
-import java.util.Map;
-import java.util.concurrent.ConcurrentHashMap;
-import java.util.concurrent.atomic.AtomicInteger;
-
-/** Inject code to delay particular servers. */
-public class DelayLocalExecutionInjection implements CodeInjectionForTesting.Code {
-  private final Map<String, AtomicInteger> delays = new ConcurrentHashMap<>();
-
-  public DelayLocalExecutionInjection(String method) {
-    CodeInjectionForTesting.put(method, this);
-  }
-
-  public void clear() {
-    delays.clear();
-  }
-
-  public void setDelayMs(String id, int delayMs) {
-    AtomicInteger d = delays.get(id);
-    if (d == null) {
-      delays.put(id, d = new AtomicInteger());
-    }
-    d.set(delayMs);
-  }
-
-  public void removeDelay(String id) {
-    delays.remove(id);
-  }
-
-  @Override
-  public boolean execute(String localId, String remoteId, Object... args) {
-    final AtomicInteger d = delays.get(localId);
-    if (d == null) {
-      return false;
-    }
-    LOG.info("{} delay {} ms, args={}", localId, d.get(),
-        Arrays.toString(args));
-    try {
-      RaftTestUtil.delay(d::get);
-    } catch (InterruptedException e) {
-      LOG.debug("Interrupted while delaying " + localId);
-    }
-    return true;
-  }
-}

http://git-wip-us.apache.org/repos/asf/incubator-ratis/blob/7e71a2e0/raft-server/src/test/java/org/apache/raft/server/impl/RaftReconfigurationBaseTest.java
----------------------------------------------------------------------
diff --git a/raft-server/src/test/java/org/apache/raft/server/impl/RaftReconfigurationBaseTest.java b/raft-server/src/test/java/org/apache/raft/server/impl/RaftReconfigurationBaseTest.java
deleted file mode 100644
index 8a5af69..0000000
--- a/raft-server/src/test/java/org/apache/raft/server/impl/RaftReconfigurationBaseTest.java
+++ /dev/null
@@ -1,577 +0,0 @@
-/**
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- *     http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-package org.apache.raft.server.impl;
-
-import org.apache.log4j.Level;
-import org.apache.raft.MiniRaftCluster;
-import org.apache.raft.MiniRaftCluster.PeerChanges;
-import org.apache.raft.RaftTestUtil;
-import org.apache.raft.RaftTestUtil.SimpleMessage;
-import org.apache.raft.client.RaftClient;
-import org.apache.raft.client.RaftClientRequestSender;
-import org.apache.raft.client.impl.RaftClientImpl;
-import org.apache.raft.conf.RaftProperties;
-import org.apache.raft.protocol.*;
-import org.apache.raft.server.RaftServerConfigKeys;
-import org.apache.raft.server.simulation.RequestHandler;
-import org.apache.raft.server.storage.RaftLog;
-import org.apache.raft.util.RaftUtils;
-import org.junit.Assert;
-import org.junit.BeforeClass;
-import org.junit.Test;
-import org.mockito.internal.util.reflection.Whitebox;
-import org.slf4j.Logger;
-import org.slf4j.LoggerFactory;
-
-import java.io.IOException;
-import java.util.ArrayList;
-import java.util.Arrays;
-import java.util.Collections;
-import java.util.concurrent.CountDownLatch;
-import java.util.concurrent.atomic.AtomicBoolean;
-import java.util.concurrent.atomic.AtomicReference;
-
-import static java.util.Arrays.asList;
-import static org.apache.raft.MiniRaftCluster.logSyncDelay;
-import static org.apache.raft.server.impl.RaftServerConstants.DEFAULT_SEQNUM;
-import static org.apache.raft.server.impl.RaftServerTestUtil.waitAndCheckNewConf;
-import static org.apache.raft.shaded.proto.RaftProtos.LogEntryProto.LogEntryBodyCase.CONFIGURATIONENTRY;
-
-public abstract class RaftReconfigurationBaseTest {
-  static {
-    RaftUtils.setLogLevel(RaftServerImpl.LOG, Level.DEBUG);
-    RaftUtils.setLogLevel(RequestHandler.LOG, Level.DEBUG);
-    RaftUtils.setLogLevel(RaftClient.LOG, Level.DEBUG);
-  }
-  static final Logger LOG = LoggerFactory.getLogger(RaftReconfigurationBaseTest.class);
-
-  protected static final RaftProperties prop = new RaftProperties();
-
-  @BeforeClass
-  public static void setup() {
-    // set a small gap for tests
-    prop.setInt(RaftServerConfigKeys.RAFT_SERVER_STAGING_CATCHUP_GAP_KEY, 10);
-  }
-
-  public abstract MiniRaftCluster getCluster(int peerNum) throws IOException;
-
-  private static int getStagingGap() {
-    return prop.getInt(RaftServerConfigKeys.RAFT_SERVER_STAGING_CATCHUP_GAP_KEY,
-        RaftServerConfigKeys.RAFT_SERVER_STAGING_CATCHUP_GAP_DEFAULT);
-  }
-
-  /**
-   * add 2 new peers (3 peers -> 5 peers), no leader change
-   */
-  @Test
-  public void testAddPeers() throws Exception {
-    LOG.info("Start testAddPeers");
-    MiniRaftCluster cluster = getCluster(3);
-    cluster.start();
-    try {
-      RaftTestUtil.waitForLeader(cluster);
-
-      // add new peers
-      RaftPeer[] allPeers = cluster.addNewPeers(2, true).allPeersInNewConf;
-
-      // trigger setConfiguration
-      SetConfigurationRequest request = new SetConfigurationRequest("client",
-          cluster.getLeader().getId(), DEFAULT_SEQNUM, allPeers);
-      LOG.info("Start changing the configuration: {}", request);
-      cluster.getLeader().setConfiguration(request);
-
-      // wait for the new configuration to take effect
-      waitAndCheckNewConf(cluster, allPeers, 0, null);
-    } finally {
-      cluster.shutdown();
-    }
-  }
-
-  /**
-   * remove 2 peers (5 peers -> 3 peers), no leader change
-   */
-  @Test
-  public void testRemovePeers() throws Exception {
-    LOG.info("Start testRemovePeers");
-    MiniRaftCluster cluster = getCluster(5);
-    cluster.start();
-    try {
-      RaftTestUtil.waitForLeader(cluster);
-
-      // remove peers, leader still included in the new conf
-      RaftPeer[] allPeers = cluster
-          .removePeers(2, false, Collections.emptyList()).allPeersInNewConf;
-
-      // trigger setConfiguration
-      SetConfigurationRequest request = new SetConfigurationRequest("client",
-          cluster.getLeader().getId(), DEFAULT_SEQNUM, allPeers);
-      LOG.info("Start changing the configuration: {}", request);
-      cluster.getLeader().setConfiguration(request);
-
-      // wait for the new configuration to take effect
-      waitAndCheckNewConf(cluster, allPeers, 2, null);
-    } finally {
-      cluster.shutdown();
-    }
-  }
-
-  /**
-   * 5 peers -> 5 peers, remove 2 old, add 2 new, no leader change
-   */
-  @Test
-  public void testAddRemovePeers() throws Exception {
-    LOG.info("Start testAddRemovePeers");
-    testAddRemovePeers(false);
-  }
-
-  @Test
-  public void testLeaderStepDown() throws Exception {
-    LOG.info("Start testLeaderStepDown");
-    testAddRemovePeers(true);
-  }
-
-  private void testAddRemovePeers(boolean leaderStepdown) throws Exception {
-    MiniRaftCluster cluster = getCluster(5);
-    cluster.start();
-    try {
-      RaftTestUtil.waitForLeader(cluster);
-
-      PeerChanges change = cluster.addNewPeers(2, true);
-      RaftPeer[] allPeers = cluster.removePeers(2, leaderStepdown,
-          asList(change.newPeers)).allPeersInNewConf;
-
-      // trigger setConfiguration
-      SetConfigurationRequest request = new SetConfigurationRequest("client",
-          cluster.getLeader().getId(), DEFAULT_SEQNUM, allPeers);
-      LOG.info("Start changing the configuration: {}", request);
-      cluster.getLeader().setConfiguration(request);
-
-      // wait for the new configuration to take effect
-      waitAndCheckNewConf(cluster, allPeers, 2, null);
-    } finally {
-      cluster.shutdown();
-    }
-  }
-
-  @Test(timeout = 30000)
-  public void testReconfTwice() throws Exception {
-    LOG.info("Start testReconfTwice");
-    final MiniRaftCluster cluster = getCluster(3);
-    cluster.start();
-    try {
-      RaftTestUtil.waitForLeader(cluster);
-      final String leaderId = cluster.getLeader().getId();
-      final RaftClient client = cluster.createClient("client", leaderId);
-
-      // submit some msgs before reconf
-      for (int i = 0; i < getStagingGap() * 2; i++) {
-        RaftClientReply reply = client.send(new SimpleMessage("m" + i));
-        Assert.assertTrue(reply.isSuccess());
-      }
-
-      final AtomicBoolean reconf1 = new AtomicBoolean(false);
-      final AtomicBoolean reconf2 = new AtomicBoolean(false);
-      final AtomicReference<RaftPeer[]> finalPeers = new AtomicReference<>(null);
-      final AtomicReference<RaftPeer[]> deadPeers = new AtomicReference<>(null);
-      CountDownLatch latch = new CountDownLatch(1);
-      Thread clientThread = new Thread(() -> {
-        try {
-          PeerChanges c1 = cluster.addNewPeers(2, true);
-          LOG.info("Start changing the configuration: {}",
-              asList(c1.allPeersInNewConf));
-
-          RaftClientReply reply = client.setConfiguration(c1.allPeersInNewConf);
-          reconf1.set(reply.isSuccess());
-
-          PeerChanges c2 = cluster.removePeers(2, true, asList(c1.newPeers));
-          finalPeers.set(c2.allPeersInNewConf);
-          deadPeers.set(c2.removedPeers);
-
-          LOG.info("Start changing the configuration again: {}",
-              asList(c2.allPeersInNewConf));
-          reply = client.setConfiguration(c2.allPeersInNewConf);
-          reconf2.set(reply.isSuccess());
-
-          latch.countDown();
-          client.close();
-        } catch (IOException ignored) {
-        }
-      });
-      clientThread.start();
-
-      latch.await();
-      Assert.assertTrue(reconf1.get());
-      Assert.assertTrue(reconf2.get());
-      waitAndCheckNewConf(cluster, finalPeers.get(), 2, null);
-
-      // check configuration manager's internal state
-      // each reconf will generate two configurations: (old, new) and (new)
-      cluster.getServers().stream().filter(RaftServerImpl::isAlive)
-          .forEach(server -> {
-        ConfigurationManager confManager =
-            (ConfigurationManager) Whitebox.getInternalState(server.getState(),
-                "configurationManager");
-        // each reconf will generate two configurations: (old, new) and (new)
-        Assert.assertEquals(5, confManager.numOfConf());
-      });
-    } finally {
-      cluster.shutdown();
-    }
-  }
-
-  @Test
-  public void testReconfTimeout() throws Exception {
-    LOG.info("Start testReconfTimeout");
-    // originally 3 peers
-    final MiniRaftCluster cluster = getCluster(3);
-    cluster.start();
-    try {
-      RaftTestUtil.waitForLeader(cluster);
-      final String leaderId = cluster.getLeader().getId();
-      final RaftClient client = cluster.createClient("client", leaderId);
-
-      PeerChanges c1 = cluster.addNewPeers(2, false);
-
-      LOG.info("Start changing the configuration: {}",
-          asList(c1.allPeersInNewConf));
-      Assert.assertFalse(cluster.getLeader().getRaftConf().isTransitional());
-
-      final RaftClientRequestSender sender = ((RaftClientImpl)client).getRequestSender();
-      final SetConfigurationRequest request = new SetConfigurationRequest(
-          "client", leaderId, DEFAULT_SEQNUM, c1.allPeersInNewConf);
-      try {
-        sender.sendRequest(request);
-        Assert.fail("did not get expected exception");
-      } catch (IOException e) {
-        Assert.assertTrue("Got exception " + e,
-            e instanceof ReconfigurationTimeoutException);
-      }
-
-      // the two new peers have not started yet, the bootstrapping must timeout
-      LOG.info(cluster.printServers());
-
-      // resend the same request, make sure the server has correctly reset its
-      // state so that we still get timeout instead of in-progress exception
-      try {
-        sender.sendRequest(request);
-        Assert.fail("did not get expected exception");
-      } catch (IOException e) {
-        Assert.assertTrue("Got exception " + e,
-            e instanceof ReconfigurationTimeoutException);
-      }
-
-      // start the two new peers
-      LOG.info("Start new peers");
-      for (RaftPeer np : c1.newPeers) {
-        cluster.startServer(np.getId());
-      }
-      Assert.assertTrue(client.setConfiguration(c1.allPeersInNewConf).isSuccess());
-      client.close();
-    } finally {
-      cluster.shutdown();
-    }
-  }
-
-  @Test
-  public void testBootstrapReconf() throws Exception {
-    LOG.info("Start testBootstrapReconf");
-    // originally 3 peers
-    final MiniRaftCluster cluster = getCluster(3);
-    cluster.start();
-    try {
-      RaftTestUtil.waitForLeader(cluster);
-      final String leaderId = cluster.getLeader().getId();
-      final RaftClient client = cluster.createClient("client", leaderId);
-
-      // submit some msgs before reconf
-      for (int i = 0; i < getStagingGap() * 2; i++) {
-        RaftClientReply reply = client.send(new SimpleMessage("m" + i));
-        Assert.assertTrue(reply.isSuccess());
-      }
-
-      PeerChanges c1 = cluster.addNewPeers(2, true);
-      LOG.info("Start changing the configuration: {}",
-          asList(c1.allPeersInNewConf));
-      final AtomicReference<Boolean> success = new AtomicReference<>();
-
-      Thread clientThread = new Thread(() -> {
-        try {
-          RaftClientReply reply = client.setConfiguration(c1.allPeersInNewConf);
-          success.set(reply.isSuccess());
-          client.close();
-        } catch (IOException ioe) {
-          LOG.error("FAILED", ioe);
-        }
-      });
-      clientThread.start();
-
-      Thread.sleep(5000);
-      LOG.info(cluster.printServers());
-      assertSuccess(success);
-
-      final RaftLog leaderLog = cluster.getLeader().getState().getLog();
-      for (RaftPeer newPeer : c1.newPeers) {
-        Assert.assertArrayEquals(leaderLog.getEntries(0, Long.MAX_VALUE),
-            cluster.getServer(newPeer.getId()).getState().getLog()
-                .getEntries(0, Long.MAX_VALUE));
-      }
-    } finally {
-      cluster.shutdown();
-    }
-  }
-
-  /**
-   * kill the leader before reconfiguration finishes. Make sure the client keeps
-   * retrying.
-   */
-  @Test
-  public void testKillLeaderDuringReconf() throws Exception {
-    LOG.info("Start testKillLeaderDuringReconf");
-    // originally 3 peers
-    final MiniRaftCluster cluster = getCluster(3);
-    cluster.start();
-    try {
-      RaftTestUtil.waitForLeader(cluster);
-      final String leaderId = cluster.getLeader().getId();
-      final RaftClient client = cluster.createClient("client", leaderId);
-
-      PeerChanges c1 = cluster.addNewPeers(2, false);
-      PeerChanges c2 = cluster.removePeers(2, false, asList(c1.newPeers));
-
-      LOG.info("Start changing the configuration: {}",
-          asList(c2.allPeersInNewConf));
-      final AtomicReference<Boolean> success = new AtomicReference<>();
-      final AtomicBoolean clientRunning = new AtomicBoolean(true);
-      Thread clientThread = new Thread(() -> {
-        try {
-          boolean r = false;
-          while (clientRunning.get() && !r) {
-            r = client.setConfiguration(c2.allPeersInNewConf).isSuccess();
-          }
-          success.set(r);
-          client.close();
-        } catch (IOException ignored) {
-        }
-      });
-      clientThread.start();
-
-      // the leader cannot generate the (old, new) conf, and it will keep
-      // bootstrapping the 2 new peers since they have not started yet
-      LOG.info(cluster.printServers());
-      Assert.assertFalse(cluster.getLeader().getRaftConf().isTransitional());
-
-      // only the first empty entry got committed
-      final long committedIndex = cluster.getLeader().getState().getLog()
-          .getLastCommittedIndex();
-      Assert.assertTrue("committedIndex is " + committedIndex,
-          committedIndex <= 1);
-
-      LOG.info("kill the current leader");
-      final String oldLeaderId = RaftTestUtil.waitAndKillLeader(cluster, true);
-      LOG.info("start the two new peers: {}", Arrays.asList(c1.newPeers));
-      for (RaftPeer np : c1.newPeers) {
-        cluster.startServer(np.getId());
-      }
-
-      Thread.sleep(3000);
-      // the client should get the NotLeaderException from the first leader, and
-      // will retry the same setConfiguration request
-      waitAndCheckNewConf(cluster, c2.allPeersInNewConf, 2,
-          Collections.singletonList(oldLeaderId));
-      clientRunning.set(false);
-      //Assert.assertTrue(success.get());
-    } finally {
-      cluster.shutdown();
-    }
-  }
-
-  static void assertSuccess(final AtomicReference<Boolean> success) {
-    final String s = "success=" + success;
-    Assert.assertNotNull(s, success.get());
-    Assert.assertTrue(s, success.get());
-  }
-
-  /**
-   * When a request's new configuration is the same with the current one, make
-   * sure we return success immediately and no log entry is recorded.
-   */
-  @Test
-  public void testNoChangeRequest() throws Exception {
-    LOG.info("Start testNoChangeRequest");
-    // originally 3 peers
-    final MiniRaftCluster cluster = getCluster(3);
-    try {
-      cluster.start();
-      RaftTestUtil.waitForLeader(cluster);
-
-      final String leaderId = cluster.getLeader().getId();
-      final RaftClient client = cluster.createClient("client", leaderId);
-      client.send(new SimpleMessage("m"));
-
-      final long committedIndex = cluster.getLeader().getState().getLog()
-          .getLastCommittedIndex();
-      final RaftConfiguration confBefore = cluster.getLeader().getRaftConf();
-
-      // no real configuration change in the request
-      RaftClientReply reply = client.setConfiguration(cluster.getPeers()
-          .toArray(new RaftPeer[0]));
-      Assert.assertTrue(reply.isSuccess());
-      Assert.assertEquals(committedIndex, cluster.getLeader().getState()
-          .getLog().getLastCommittedIndex());
-      Assert.assertSame(confBefore, cluster.getLeader().getRaftConf());
-      client.close();
-    } finally {
-      cluster.shutdown();
-    }
-  }
-
-  /**
-   * Make sure a setConfiguration request is rejected if a configuration change
-   * is still in progress (i.e., has not been committed yet).
-   */
-  @Test
-  public void testOverlappedSetConfRequests() throws Exception {
-    LOG.info("Start testOverlappedSetConfRequests");
-    // originally 3 peers
-    final MiniRaftCluster cluster = getCluster(3);
-    try {
-      cluster.start();
-      RaftTestUtil.waitForLeader(cluster);
-
-      final String leaderId = cluster.getLeader().getId();
-
-      RaftPeer[] newPeers = cluster.addNewPeers(2, true).allPeersInNewConf;
-
-      // delay every peer's logSync so that the setConf request is delayed
-      cluster.getPeers()
-          .forEach(peer -> logSyncDelay.setDelayMs(peer.getId(), 1000));
-
-      final CountDownLatch latch = new CountDownLatch(1);
-      final RaftPeer[] peersInRequest2 = cluster.getPeers().toArray(new RaftPeer[0]);
-      AtomicBoolean caughtException = new AtomicBoolean(false);
-      new Thread(() -> {
-        try(final RaftClient client2 = cluster.createClient("client2", leaderId)) {
-          latch.await();
-          LOG.info("client2 starts to change conf");
-          final RaftClientRequestSender sender2 = ((RaftClientImpl)client2).getRequestSender();
-          sender2.sendRequest(new SetConfigurationRequest(
-              "client2", leaderId, DEFAULT_SEQNUM, peersInRequest2));
-        } catch (ReconfigurationInProgressException e) {
-          caughtException.set(true);
-        } catch (Exception e) {
-          LOG.warn("Got unexpected exception when client2 changes conf", e);
-        }
-      }).start();
-
-      AtomicBoolean confChanged = new AtomicBoolean(false);
-      new Thread(() -> {
-        try(final RaftClient client1 = cluster.createClient("client1", leaderId)) {
-          LOG.info("client1 starts to change conf");
-          confChanged.set(client1.setConfiguration(newPeers).isSuccess());
-        } catch (IOException e) {
-          LOG.warn("Got unexpected exception when client1 changes conf", e);
-        }
-      }).start();
-      Thread.sleep(100);
-      latch.countDown();
-
-      for (int i = 0; i < 10 && !confChanged.get(); i++) {
-        Thread.sleep(1000);
-      }
-      Assert.assertTrue(confChanged.get());
-      Assert.assertTrue(caughtException.get());
-    } finally {
-      logSyncDelay.clear();
-      cluster.shutdown();
-    }
-  }
-
-  /**
-   * Test a scenario where the follower truncates its log entries which causes
-   * configuration change.
-   */
-  @Test
-  public void testRevertConfigurationChange() throws Exception {
-    LOG.info("Start testRevertConfigurationChange");
-    // originally 3 peers
-    final MiniRaftCluster cluster = getCluster(5);
-    try {
-      cluster.start();
-      RaftTestUtil.waitForLeader(cluster);
-
-      final String leaderId = cluster.getLeader().getId();
-
-      final RaftLog log = cluster.getServer(leaderId).getState().getLog();
-      Thread.sleep(1000);
-      Assert.assertEquals(0, log.getLatestFlushedIndex());
-
-      // we block the incoming msg for the leader and block its requests to
-      // followers, so that we force the leader change and the old leader will
-      // not know
-      LOG.info("start blocking the leader");
-      BlockRequestHandlingInjection.getInstance().blockReplier(leaderId);
-      cluster.setBlockRequestsFrom(leaderId, true);
-
-      PeerChanges change = cluster.removePeers(1, false, new ArrayList<>());
-
-      AtomicBoolean gotNotLeader = new AtomicBoolean(false);
-      new Thread(() -> {
-        try(final RaftClient client = cluster.createClient("client1", leaderId)) {
-          LOG.info("client starts to change conf");
-          final RaftClientRequestSender sender = ((RaftClientImpl)client).getRequestSender();
-          RaftClientReply reply = sender.sendRequest(new SetConfigurationRequest(
-              "client", leaderId, DEFAULT_SEQNUM, change.allPeersInNewConf));
-          if (reply.isNotLeader()) {
-            gotNotLeader.set(true);
-          }
-        } catch (IOException e) {
-          LOG.warn("Got unexpected exception when client1 changes conf", e);
-        }
-      }).start();
-
-      // wait till the old leader persist the new conf
-      for (int i = 0; i < 10 && log.getLatestFlushedIndex() < 1; i++) {
-        Thread.sleep(500);
-      }
-      Assert.assertEquals(1, log.getLatestFlushedIndex());
-      Assert.assertEquals(CONFIGURATIONENTRY,
-          log.getLastEntry().getLogEntryBodyCase());
-
-      // unblock the old leader
-      BlockRequestHandlingInjection.getInstance().unblockReplier(leaderId);
-      cluster.setBlockRequestsFrom(leaderId, false);
-
-      // the client should get NotLeaderException
-      for (int i = 0; i < 10 && !gotNotLeader.get(); i++) {
-        Thread.sleep(500);
-      }
-      Assert.assertTrue(gotNotLeader.get());
-
-      // the old leader should have truncated the setConf from the log
-      boolean newState = false;
-      for (int i = 0; i < 10 && !newState; i++) {
-        Thread.sleep(500);
-        newState = log.getLastCommittedIndex() == 1 &&
-            log.getLastEntry().getLogEntryBodyCase() != CONFIGURATIONENTRY;
-      }
-      Assert.assertTrue(newState);
-    } finally {
-      cluster.shutdown();
-    }
-  }
-}

http://git-wip-us.apache.org/repos/asf/incubator-ratis/blob/7e71a2e0/raft-server/src/test/java/org/apache/raft/server/impl/RaftServerTestUtil.java
----------------------------------------------------------------------
diff --git a/raft-server/src/test/java/org/apache/raft/server/impl/RaftServerTestUtil.java b/raft-server/src/test/java/org/apache/raft/server/impl/RaftServerTestUtil.java
deleted file mode 100644
index bd1934f..0000000
--- a/raft-server/src/test/java/org/apache/raft/server/impl/RaftServerTestUtil.java
+++ /dev/null
@@ -1,73 +0,0 @@
-/**
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- *     http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-package org.apache.raft.server.impl;
-
-import org.apache.raft.MiniRaftCluster;
-import org.apache.raft.RaftTestUtil;
-import org.apache.raft.protocol.RaftPeer;
-import org.apache.raft.server.RaftServer;
-import org.apache.raft.statemachine.StateMachine;
-import org.junit.Assert;
-import org.slf4j.Logger;
-import org.slf4j.LoggerFactory;
-
-import java.util.Collection;
-
-public class RaftServerTestUtil {
-  static final Logger LOG = LoggerFactory.getLogger(RaftServerTestUtil.class);
-
-  public static void waitAndCheckNewConf(MiniRaftCluster cluster,
-      RaftPeer[] peers, int numOfRemovedPeers, Collection<String> deadPeers)
-      throws Exception {
-    final long sleepMs = cluster.getMaxTimeout() * (numOfRemovedPeers + 2);
-    RaftTestUtil.attempt(3, sleepMs,
-        () -> waitAndCheckNewConf(cluster, peers, deadPeers));
-  }
-  private static void waitAndCheckNewConf(MiniRaftCluster cluster,
-      RaftPeer[] peers, Collection<String> deadPeers)
-      throws Exception {
-    LOG.info(cluster.printServers());
-    Assert.assertNotNull(cluster.getLeader());
-
-    int numIncluded = 0;
-    int deadIncluded = 0;
-    final RaftConfiguration current = RaftConfiguration.newBuilder()
-        .setConf(peers).setLogEntryIndex(0).build();
-    for (RaftServerImpl server : cluster.getServers()) {
-      if (deadPeers != null && deadPeers.contains(server.getId())) {
-        if (current.containsInConf(server.getId())) {
-          deadIncluded++;
-        }
-        continue;
-      }
-      if (current.containsInConf(server.getId())) {
-        numIncluded++;
-        Assert.assertTrue(server.getRaftConf().isStable());
-        Assert.assertTrue(server.getRaftConf().hasNoChange(peers));
-      } else {
-        Assert.assertFalse(server.getId() + " is still running: " + server,
-            server.isAlive());
-      }
-    }
-    Assert.assertEquals(peers.length, numIncluded + deadIncluded);
-  }
-
-  public static StateMachine getStateMachine(RaftServerImpl s) {
-    return s.getStateMachine();
-  }
-}

http://git-wip-us.apache.org/repos/asf/incubator-ratis/blob/7e71a2e0/raft-server/src/test/java/org/apache/raft/server/simulation/MiniRaftClusterWithSimulatedRpc.java
----------------------------------------------------------------------
diff --git a/raft-server/src/test/java/org/apache/raft/server/simulation/MiniRaftClusterWithSimulatedRpc.java b/raft-server/src/test/java/org/apache/raft/server/simulation/MiniRaftClusterWithSimulatedRpc.java
deleted file mode 100644
index 7414872..0000000
--- a/raft-server/src/test/java/org/apache/raft/server/simulation/MiniRaftClusterWithSimulatedRpc.java
+++ /dev/null
@@ -1,136 +0,0 @@
-/**
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- *     http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-package org.apache.raft.server.simulation;
-
-import org.apache.raft.MiniRaftCluster;
-import org.apache.raft.client.RaftClientRequestSender;
-import org.apache.raft.conf.RaftProperties;
-import org.apache.raft.protocol.RaftPeer;
-import org.apache.raft.server.impl.RaftServerImpl;
-import org.slf4j.Logger;
-import org.slf4j.LoggerFactory;
-
-import java.io.IOException;
-import java.util.Collection;
-import java.util.Collections;
-
-public class MiniRaftClusterWithSimulatedRpc extends MiniRaftCluster {
-  static final Logger LOG = LoggerFactory.getLogger(MiniRaftClusterWithSimulatedRpc.class);
-
-  public static final Factory<MiniRaftClusterWithSimulatedRpc> FACTORY
-      = new Factory<MiniRaftClusterWithSimulatedRpc>() {
-    @Override
-    public MiniRaftClusterWithSimulatedRpc newCluster(
-        String[] ids, RaftProperties prop, boolean formatted) {
-      prop.setInt(SimulatedRequestReply.SIMULATE_LATENCY_KEY, 0);
-      return new MiniRaftClusterWithSimulatedRpc(ids, prop, formatted);
-    }
-  };
-
-  private SimulatedRequestReply<RaftServerRequest, RaftServerReply> serverRequestReply;
-  private SimulatedClientRequestReply client2serverRequestReply;
-
-  public MiniRaftClusterWithSimulatedRpc(int numServers,
-      RaftProperties properties) {
-    this(generateIds(numServers, 0), properties, true);
-  }
-
-  public MiniRaftClusterWithSimulatedRpc(String[] ids,
-      RaftProperties properties, boolean formatted) {
-    super(ids, properties, formatted);
-    initRpc();
-  }
-
-  private void initRpc() {
-    final Collection<RaftPeer> peers = getConf().getPeers();
-    final int simulateLatencyMs = properties.getInt(
-        SimulatedRequestReply.SIMULATE_LATENCY_KEY,
-        SimulatedRequestReply.SIMULATE_LATENCY_DEFAULT);
-    LOG.info(SimulatedRequestReply.SIMULATE_LATENCY_KEY + " = "
-        + simulateLatencyMs);
-    serverRequestReply = new SimulatedRequestReply<>(peers, simulateLatencyMs);
-    client2serverRequestReply = new SimulatedClientRequestReply(peers,
-        simulateLatencyMs);
-
-    setRpcServers(getServers());
-  }
-
-  private void setRpcServers(Collection<RaftServerImpl> newServers) {
-    newServers.forEach(s -> s.setServerRpc(
-        new SimulatedServerRpc(s, serverRequestReply, client2serverRequestReply)));
-  }
-
-  @Override
-  protected void setPeerRpc() {
-    initRpc();
-  }
-
-  private void addPeersToRpc(Collection<RaftPeer> peers) {
-    serverRequestReply.addPeers(peers);
-    client2serverRequestReply.addPeers(peers);
-  }
-
-  @Override
-  public void restartServer(String id, boolean format) throws IOException {
-    super.restartServer(id, format);
-    RaftServerImpl s = getServer(id);
-    addPeersToRpc(Collections.singletonList(conf.getPeer(id)));
-    s.setServerRpc(new SimulatedServerRpc(s, serverRequestReply,
-        client2serverRequestReply));
-    s.start();
-  }
-
-  @Override
-  public Collection<RaftPeer> addNewPeers(Collection<RaftPeer> newPeers,
-                                          Collection<RaftServerImpl> newServers, boolean startService) {
-    addPeersToRpc(newPeers);
-    setRpcServers(newServers);
-    if (startService) {
-      newServers.forEach(RaftServerImpl::start);
-    }
-    return newPeers;
-  }
-
-  @Override
-  public RaftClientRequestSender getRaftClientRequestSender() {
-    return client2serverRequestReply;
-  }
-
-  @Override
-  public void blockQueueAndSetDelay(String leaderId, int delayMs)
-      throws InterruptedException {
-    // block leader sendRequest if delayMs > 0
-    final boolean block = delayMs > 0;
-    LOG.debug("{} leader queue {} and set {}ms delay for the other queues",
-        block? "Block": "Unblock", leaderId, delayMs);
-    serverRequestReply.getQueue(leaderId).blockSendRequestTo.set(block);
-
-    // set delay takeRequest for the other queues
-    getServers().stream().filter(s -> !s.getId().equals(leaderId))
-        .map(s -> serverRequestReply.getQueue(s.getId()))
-        .forEach(q -> q.delayTakeRequestTo.set(delayMs));
-
-    final long sleepMs = 3 * getMaxTimeout() / 2;
-    Thread.sleep(sleepMs);
-  }
-
-  @Override
-  public void setBlockRequestsFrom(String src, boolean block) {
-    serverRequestReply.getQueue(src).blockTakeRequestFrom.set(block);
-  }
-}

http://git-wip-us.apache.org/repos/asf/incubator-ratis/blob/7e71a2e0/raft-server/src/test/java/org/apache/raft/server/simulation/RaftServerReply.java
----------------------------------------------------------------------
diff --git a/raft-server/src/test/java/org/apache/raft/server/simulation/RaftServerReply.java b/raft-server/src/test/java/org/apache/raft/server/simulation/RaftServerReply.java
deleted file mode 100644
index 7a03d75..0000000
--- a/raft-server/src/test/java/org/apache/raft/server/simulation/RaftServerReply.java
+++ /dev/null
@@ -1,99 +0,0 @@
-/**
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- *     http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-package org.apache.raft.server.simulation;
-
-import com.google.common.base.Preconditions;
-import org.apache.raft.protocol.RaftRpcMessage;
-import org.apache.raft.shaded.proto.RaftProtos.AppendEntriesReplyProto;
-import org.apache.raft.shaded.proto.RaftProtos.InstallSnapshotReplyProto;
-import org.apache.raft.shaded.proto.RaftProtos.RequestVoteReplyProto;
-
-public class RaftServerReply extends RaftRpcMessage {
-  private final AppendEntriesReplyProto appendEntries;
-  private final RequestVoteReplyProto requestVote;
-  private final InstallSnapshotReplyProto installSnapshot;
-
-  RaftServerReply(AppendEntriesReplyProto a) {
-    appendEntries = Preconditions.checkNotNull(a);
-    requestVote = null;
-    installSnapshot = null;
-  }
-
-  RaftServerReply(RequestVoteReplyProto r) {
-    appendEntries = null;
-    requestVote = Preconditions.checkNotNull(r);
-    installSnapshot = null;
-  }
-
-  RaftServerReply(InstallSnapshotReplyProto i) {
-    appendEntries = null;
-    requestVote = null;
-    installSnapshot = Preconditions.checkNotNull(i);
-  }
-
-  boolean isAppendEntries() {
-    return appendEntries != null;
-  }
-
-  boolean isRequestVote() {
-    return requestVote != null;
-  }
-
-  boolean isInstallSnapshot() {
-    return installSnapshot != null;
-  }
-
-  AppendEntriesReplyProto getAppendEntries() {
-    return appendEntries;
-  }
-
-  RequestVoteReplyProto getRequestVote() {
-    return requestVote;
-  }
-
-  InstallSnapshotReplyProto getInstallSnapshot() {
-    return installSnapshot;
-  }
-
-  @Override
-  public boolean isRequest() {
-    return false;
-  }
-
-  @Override
-  public String getRequestorId() {
-    if (isAppendEntries()) {
-      return appendEntries.getServerReply().getRequestorId();
-    } else if (isRequestVote()) {
-      return requestVote.getServerReply().getRequestorId();
-    } else {
-      return installSnapshot.getServerReply().getRequestorId();
-    }
-  }
-
-  @Override
-  public String getReplierId() {
-    if (isAppendEntries()) {
-      return appendEntries.getServerReply().getReplyId();
-    } else if (isRequestVote()) {
-      return requestVote.getServerReply().getReplyId();
-    } else {
-      return installSnapshot.getServerReply().getReplyId();
-    }
-  }
-}

http://git-wip-us.apache.org/repos/asf/incubator-ratis/blob/7e71a2e0/raft-server/src/test/java/org/apache/raft/server/simulation/RaftServerRequest.java
----------------------------------------------------------------------
diff --git a/raft-server/src/test/java/org/apache/raft/server/simulation/RaftServerRequest.java b/raft-server/src/test/java/org/apache/raft/server/simulation/RaftServerRequest.java
deleted file mode 100644
index 499b12d..0000000
--- a/raft-server/src/test/java/org/apache/raft/server/simulation/RaftServerRequest.java
+++ /dev/null
@@ -1,98 +0,0 @@
-/**
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- *     http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-package org.apache.raft.server.simulation;
-
-import org.apache.raft.protocol.RaftRpcMessage;
-import org.apache.raft.shaded.proto.RaftProtos.AppendEntriesRequestProto;
-import org.apache.raft.shaded.proto.RaftProtos.InstallSnapshotRequestProto;
-import org.apache.raft.shaded.proto.RaftProtos.RequestVoteRequestProto;
-
-class RaftServerRequest extends RaftRpcMessage {
-  private final AppendEntriesRequestProto appendEntries;
-  private final RequestVoteRequestProto requestVote;
-  private final InstallSnapshotRequestProto installSnapshot;
-
-  RaftServerRequest(AppendEntriesRequestProto a) {
-    appendEntries = a;
-    requestVote = null;
-    installSnapshot = null;
-  }
-
-  RaftServerRequest(RequestVoteRequestProto r) {
-    appendEntries = null;
-    requestVote = r;
-    installSnapshot = null;
-  }
-
-  RaftServerRequest(InstallSnapshotRequestProto i) {
-    appendEntries = null;
-    requestVote = null;
-    installSnapshot = i;
-  }
-
-  boolean isAppendEntries() {
-    return appendEntries != null;
-  }
-
-  boolean isRequestVote() {
-    return requestVote != null;
-  }
-
-  boolean isInstallSnapshot() {
-    return installSnapshot != null;
-  }
-
-  AppendEntriesRequestProto getAppendEntries() {
-    return appendEntries;
-  }
-
-  RequestVoteRequestProto getRequestVote() {
-    return requestVote;
-  }
-
-  InstallSnapshotRequestProto getInstallSnapshot() {
-    return installSnapshot;
-  }
-
-  @Override
-  public boolean isRequest() {
-    return true;
-  }
-
-  @Override
-  public String getRequestorId() {
-    if (isAppendEntries()) {
-      return appendEntries.getServerRequest().getRequestorId();
-    } else if (isRequestVote()) {
-      return requestVote.getServerRequest().getRequestorId();
-    } else {
-      return installSnapshot.getServerRequest().getRequestorId();
-    }
-  }
-
-  @Override
-  public String getReplierId() {
-    if (isAppendEntries()) {
-      return appendEntries.getServerRequest().getReplyId();
-    } else if (isRequestVote()) {
-      return requestVote.getServerRequest().getReplyId();
-    } else {
-      return installSnapshot.getServerRequest().getReplyId();
-    }
-  }
-}

http://git-wip-us.apache.org/repos/asf/incubator-ratis/blob/7e71a2e0/raft-server/src/test/java/org/apache/raft/server/simulation/RequestHandler.java
----------------------------------------------------------------------
diff --git a/raft-server/src/test/java/org/apache/raft/server/simulation/RequestHandler.java b/raft-server/src/test/java/org/apache/raft/server/simulation/RequestHandler.java
deleted file mode 100644
index dee13a6..0000000
--- a/raft-server/src/test/java/org/apache/raft/server/simulation/RequestHandler.java
+++ /dev/null
@@ -1,134 +0,0 @@
-/**
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- *     http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-package org.apache.raft.server.simulation;
-
-import org.apache.raft.protocol.RaftRpcMessage;
-import org.apache.raft.util.Daemon;
-import org.apache.raft.util.ExitUtils;
-import org.slf4j.Logger;
-import org.slf4j.LoggerFactory;
-
-import java.io.IOException;
-import java.io.InterruptedIOException;
-import java.util.ArrayList;
-import java.util.List;
-
-public class RequestHandler<REQUEST extends RaftRpcMessage,
-    REPLY extends RaftRpcMessage> {
-  public static final Logger LOG = LoggerFactory.getLogger(RequestHandler.class);
-
-  interface HandlerInterface<REQUEST extends RaftRpcMessage,
-      REPLY extends RaftRpcMessage> {
-
-    boolean isAlive();
-
-    REPLY handleRequest(REQUEST r) throws IOException;
-  }
-
-  private final String serverId;
-  private final String name;
-  private final SimulatedRequestReply<REQUEST, REPLY> rpc;
-  private final HandlerInterface<REQUEST, REPLY> handlerImpl;
-  private final List<HandlerDaemon> daemons;
-
-  RequestHandler(String serverId, String name,
-                 SimulatedRequestReply<REQUEST, REPLY> rpc,
-                 HandlerInterface<REQUEST, REPLY> handlerImpl,
-                 int numHandlers) {
-    this.serverId = serverId;
-    this.name = name;
-    this.rpc = rpc;
-    this.handlerImpl = handlerImpl;
-
-    this.daemons = new ArrayList<>(numHandlers);
-    for(int i = 0; i < numHandlers; i++) {
-      daemons.add(new HandlerDaemon(i));
-    }
-  }
-
-  void startDaemon() {
-    daemons.forEach(Thread::start);
-  }
-
-  void shutdown() {
-    rpc.shutdown(serverId);
-  }
-
-  void interruptAndJoinDaemon() throws InterruptedException {
-    daemons.forEach(Thread::interrupt);
-    for (Daemon d : daemons) {
-      d.join();
-    }
-  }
-
-  SimulatedRequestReply<REQUEST, REPLY> getRpc() {
-    return rpc;
-  }
-
-  void handleRequest(REQUEST request) throws IOException {
-    final REPLY reply;
-    try {
-      reply = handlerImpl.handleRequest(request);
-    } catch (IOException ioe) {
-      LOG.debug("IOException for " + request, ioe);
-      rpc.sendReply(request, null, ioe);
-      return;
-    }
-    if (reply != null) {
-      rpc.sendReply(request, reply, null);
-    }
-  }
-
-  /**
-   * A thread keep polling requests from the request queue. Used for simulation.
-   */
-  class HandlerDaemon extends Daemon {
-    private final int id;
-
-    HandlerDaemon(int id) {
-      this.id = id;
-    }
-
-    @Override
-    public String toString() {
-      return serverId + "." + name + id;
-    }
-
-    @Override
-    public void run() {
-      while (handlerImpl.isAlive()) {
-        try {
-          handleRequest(rpc.takeRequest(serverId));
-        } catch (InterruptedIOException e) {
-          LOG.info(this + " is interrupted by " + e);
-          LOG.trace("TRACE", e);
-          break;
-        } catch (IOException e) {
-          LOG.error(this + " has " + e);
-          LOG.trace("TRACE", e);
-        } catch(Throwable t) {
-          if (!handlerImpl.isAlive()) {
-            LOG.info(this + " is stopped.");
-            break;
-          }
-          ExitUtils.terminate(1, this + " is terminating.", t, LOG);
-        }
-      }
-    }
-  }
-}

http://git-wip-us.apache.org/repos/asf/incubator-ratis/blob/7e71a2e0/raft-server/src/test/java/org/apache/raft/server/simulation/SimulatedClientRequestReply.java
----------------------------------------------------------------------
diff --git a/raft-server/src/test/java/org/apache/raft/server/simulation/SimulatedClientRequestReply.java b/raft-server/src/test/java/org/apache/raft/server/simulation/SimulatedClientRequestReply.java
deleted file mode 100644
index 828e2a2..0000000
--- a/raft-server/src/test/java/org/apache/raft/server/simulation/SimulatedClientRequestReply.java
+++ /dev/null
@@ -1,45 +0,0 @@
-/**
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- *     http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-package org.apache.raft.server.simulation;
-
-import org.apache.raft.client.RaftClientRequestSender;
-import org.apache.raft.protocol.RaftClientReply;
-import org.apache.raft.protocol.RaftClientRequest;
-import org.apache.raft.protocol.RaftPeer;
-
-import java.io.IOException;
-import java.util.Collection;
-
-public class SimulatedClientRequestReply
-    extends SimulatedRequestReply<RaftClientRequest, RaftClientReply>
-    implements RaftClientRequestSender {
-  SimulatedClientRequestReply(Collection<RaftPeer> allPeers,
-                              int simulateLatencyMs) {
-    super(allPeers, simulateLatencyMs);
-  }
-
-  @Override
-  public void addServers(Iterable<RaftPeer> servers) {
-    // do nothing
-  }
-
-  @Override
-  public void close() {
-    // do nothing
-  }
-}

http://git-wip-us.apache.org/repos/asf/incubator-ratis/blob/7e71a2e0/raft-server/src/test/java/org/apache/raft/server/simulation/SimulatedRequestReply.java
----------------------------------------------------------------------
diff --git a/raft-server/src/test/java/org/apache/raft/server/simulation/SimulatedRequestReply.java b/raft-server/src/test/java/org/apache/raft/server/simulation/SimulatedRequestReply.java
deleted file mode 100644
index 100fd60..0000000
--- a/raft-server/src/test/java/org/apache/raft/server/simulation/SimulatedRequestReply.java
+++ /dev/null
@@ -1,201 +0,0 @@
-/**
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- *     http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-package org.apache.raft.server.simulation;
-
-import com.google.common.base.Preconditions;
-import org.apache.raft.RaftTestUtil;
-import org.apache.raft.protocol.RaftPeer;
-import org.apache.raft.protocol.RaftRpcMessage;
-import org.apache.raft.server.RaftServerConfigKeys;
-import org.apache.raft.util.RaftUtils;
-import org.apache.raft.util.Timestamp;
-
-import java.io.IOException;
-import java.util.Collection;
-import java.util.Map;
-import java.util.concurrent.BlockingQueue;
-import java.util.concurrent.ConcurrentHashMap;
-import java.util.concurrent.LinkedBlockingQueue;
-import java.util.concurrent.ThreadLocalRandom;
-import java.util.concurrent.atomic.AtomicBoolean;
-import java.util.concurrent.atomic.AtomicInteger;
-
-public class SimulatedRequestReply<REQUEST extends RaftRpcMessage,
-    REPLY extends RaftRpcMessage> {
-  public static final String SIMULATE_LATENCY_KEY
-      = SimulatedRequestReply.class.getName() + ".simulateLatencyMs";
-  public static final int SIMULATE_LATENCY_DEFAULT
-      = RaftServerConfigKeys.RAFT_SERVER_RPC_TIMEOUT_MIN_MS_DEFAULT;
-  public static final long TIMEOUT = 3000L;
-
-  private static class ReplyOrException<REPLY> {
-    private final REPLY reply;
-    private final IOException ioe;
-
-    ReplyOrException(REPLY reply, IOException ioe) {
-      Preconditions.checkArgument(reply == null ^ ioe == null);
-      this.reply = reply;
-      this.ioe = ioe;
-    }
-  }
-
-  static class EventQueue<REQUEST, REPLY> {
-    private final BlockingQueue<REQUEST> requestQueue
-        = new LinkedBlockingQueue<>();
-    private final Map<REQUEST, ReplyOrException<REPLY>> replyMap
-        = new ConcurrentHashMap<>();
-
-    /** Block takeRequest for the requests sent from this server. */
-    final AtomicBoolean blockTakeRequestFrom = new AtomicBoolean();
-    /** Block sendRequest for the requests sent to this server. */
-    final AtomicBoolean blockSendRequestTo = new AtomicBoolean();
-    /** Delay takeRequest for the requests sent to this server. */
-    final AtomicInteger delayTakeRequestTo = new AtomicInteger();
-    /** Delay takeRequest for the requests sent from this server. */
-    final AtomicInteger delayTakeRequestFrom = new AtomicInteger();
-
-    REPLY request(REQUEST request) throws InterruptedException, IOException {
-      requestQueue.put(request);
-      synchronized (this) {
-        final Timestamp startTime = new Timestamp();
-        while (startTime.elapsedTimeMs() < TIMEOUT &&
-            !replyMap.containsKey(request)) {
-          this.wait(TIMEOUT); // no need to be precise here
-        }
-      }
-
-      if (!replyMap.containsKey(request)) {
-        throw new IOException("Timeout while waiting for reply of request "
-            + request);
-      }
-      final ReplyOrException<REPLY> re = replyMap.remove(request);
-      if (re.ioe != null) {
-        throw re.ioe;
-      }
-      return re.reply;
-    }
-
-    REQUEST takeRequest() throws InterruptedException {
-      return requestQueue.take();
-    }
-
-    void reply(REQUEST request, REPLY reply, IOException ioe)
-        throws IOException {
-      replyMap.put(request, new ReplyOrException<>(reply, ioe));
-      synchronized (this) {
-        this.notifyAll();
-      }
-    }
-  }
-
-  private final Map<String, EventQueue<REQUEST, REPLY>> queues;
-  private final int simulateLatencyMs;
-
-  SimulatedRequestReply(Collection<RaftPeer> allPeers, int simulateLatencyMs) {
-    queues = new ConcurrentHashMap<>();
-    for (RaftPeer peer : allPeers) {
-      queues.put(peer.getId(), new EventQueue<>());
-    }
-
-    this.simulateLatencyMs = simulateLatencyMs;
-  }
-
-  EventQueue<REQUEST, REPLY> getQueue(String qid) {
-    return queues.get(qid);
-  }
-
-  public REPLY sendRequest(REQUEST request) throws IOException {
-    final String qid = request.getReplierId();
-    final EventQueue<REQUEST, REPLY> q = queues.get(qid);
-    if (q == null) {
-      throw new IOException("The peer " + qid + " is not alive.");
-    }
-    try {
-      RaftTestUtil.block(q.blockSendRequestTo::get);
-      return q.request(request);
-    } catch (InterruptedException e) {
-      throw RaftUtils.toInterruptedIOException("", e);
-    }
-  }
-
-  public REQUEST takeRequest(String qid) throws IOException {
-    final EventQueue<REQUEST, REPLY> q = queues.get(qid);
-    if (q == null) {
-      throw new IOException("The RPC of " + qid + " has already shutdown.");
-    }
-
-    final REQUEST request;
-    try {
-      // delay request for testing
-      RaftTestUtil.delay(q.delayTakeRequestTo::get);
-
-      request = q.takeRequest();
-      Preconditions.checkState(qid.equals(request.getReplierId()));
-
-      // block request for testing
-      final EventQueue<REQUEST, REPLY> reqQ = queues.get(request.getRequestorId());
-      if (reqQ != null) {
-        RaftTestUtil.delay(reqQ.delayTakeRequestFrom::get);
-        RaftTestUtil.block(reqQ.blockTakeRequestFrom::get);
-      }
-    } catch (InterruptedException e) {
-      throw RaftUtils.toInterruptedIOException("", e);
-    }
-    return request;
-  }
-
-  public void sendReply(REQUEST request, REPLY reply, IOException ioe)
-      throws IOException {
-    if (reply != null) {
-      Preconditions.checkArgument(
-          request.getRequestorId().equals(reply.getRequestorId()));
-      Preconditions.checkArgument(
-          request.getReplierId().equals(reply.getReplierId()));
-    }
-    simulateLatency();
-    final String qid = request.getReplierId();
-    EventQueue<REQUEST, REPLY> q = queues.get(qid);
-    if (q != null) {
-      q.reply(request, reply, ioe);
-    }
-  }
-
-  public void shutdown(String id) {
-    queues.remove(id);
-  }
-
-  public void addPeers(Collection<RaftPeer> newPeers) {
-    for (RaftPeer peer : newPeers) {
-      queues.put(peer.getId(), new EventQueue<>());
-    }
-  }
-
-  private void simulateLatency() throws IOException {
-    if (simulateLatencyMs > 0) {
-      int waitExpetation = simulateLatencyMs / 10;
-      int waitHalfRange = waitExpetation / 3;
-      int randomSleepMs = ThreadLocalRandom.current().nextInt(2 * waitHalfRange)
-          + waitExpetation - waitHalfRange;
-      try {
-        Thread.sleep(randomSleepMs);
-      } catch (InterruptedException ie) {
-        throw RaftUtils.toInterruptedIOException("", ie);
-      }
-    }
-  }
-}

http://git-wip-us.apache.org/repos/asf/incubator-ratis/blob/7e71a2e0/raft-server/src/test/java/org/apache/raft/server/simulation/SimulatedServerRpc.java
----------------------------------------------------------------------
diff --git a/raft-server/src/test/java/org/apache/raft/server/simulation/SimulatedServerRpc.java b/raft-server/src/test/java/org/apache/raft/server/simulation/SimulatedServerRpc.java
deleted file mode 100644
index 799ee65..0000000
--- a/raft-server/src/test/java/org/apache/raft/server/simulation/SimulatedServerRpc.java
+++ /dev/null
@@ -1,167 +0,0 @@
-/**
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- *     http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-package org.apache.raft.server.simulation;
-
-import com.google.common.util.concurrent.ThreadFactoryBuilder;
-import org.apache.raft.protocol.*;
-import org.apache.raft.server.RaftServerRpc;
-import org.apache.raft.server.impl.RaftServerImpl;
-import org.apache.raft.server.protocol.RaftServerProtocol;
-import org.apache.raft.shaded.proto.RaftProtos.*;
-import org.slf4j.Logger;
-import org.slf4j.LoggerFactory;
-
-import java.io.IOException;
-import java.net.InetSocketAddress;
-import java.util.concurrent.CompletableFuture;
-import java.util.concurrent.ExecutorService;
-import java.util.concurrent.Executors;
-import java.util.concurrent.TimeUnit;
-
-class SimulatedServerRpc implements RaftServerRpc {
-  static final Logger LOG = LoggerFactory.getLogger(SimulatedServerRpc.class);
-
-  private final RaftServerImpl server;
-  private final RequestHandler<RaftServerRequest, RaftServerReply> serverHandler;
-  private final RequestHandler<RaftClientRequest, RaftClientReply> clientHandler;
-  private final ExecutorService executor = Executors.newFixedThreadPool(3,
-      new ThreadFactoryBuilder().setDaemon(true).build());
-
-  SimulatedServerRpc(RaftServerImpl server,
-      SimulatedRequestReply<RaftServerRequest, RaftServerReply> serverRequestReply,
-      SimulatedRequestReply<RaftClientRequest, RaftClientReply> clientRequestReply) {
-    this.server = server;
-    this.serverHandler = new RequestHandler<>(server.getId(),
-        "serverHandler", serverRequestReply, serverHandlerImpl, 3);
-    this.clientHandler = new RequestHandler<>(server.getId(),
-        "clientHandler", clientRequestReply, clientHandlerImpl, 3);
-  }
-
-  @Override
-  public void start() {
-    serverHandler.startDaemon();
-    clientHandler.startDaemon();
-  }
-
-  private void interruptAndJoin() throws InterruptedException {
-    clientHandler.interruptAndJoinDaemon();
-    serverHandler.interruptAndJoinDaemon();
-  }
-
-  @Override
-  public void close() {
-    try {
-      interruptAndJoin();
-      executor.shutdown();
-      executor.awaitTermination(1000, TimeUnit.MILLISECONDS);
-    } catch (InterruptedException ignored) {
-    }
-    clientHandler.shutdown();
-    serverHandler.shutdown();
-  }
-
-  @Override
-  public InetSocketAddress getInetSocketAddress() {
-    return null;
-  }
-
-  @Override
-  public AppendEntriesReplyProto appendEntries(AppendEntriesRequestProto request)
-      throws IOException {
-    RaftServerReply reply = serverHandler.getRpc()
-        .sendRequest(new RaftServerRequest(request));
-    return reply.getAppendEntries();
-  }
-
-  @Override
-  public InstallSnapshotReplyProto installSnapshot(InstallSnapshotRequestProto request)
-      throws IOException {
-    RaftServerReply reply = serverHandler.getRpc()
-        .sendRequest(new RaftServerRequest(request));
-    return reply.getInstallSnapshot();
-  }
-
-  @Override
-  public RequestVoteReplyProto requestVote(RequestVoteRequestProto request)
-      throws IOException {
-    RaftServerReply reply = serverHandler.getRpc()
-        .sendRequest(new RaftServerRequest(request));
-    return reply.getRequestVote();
-  }
-
-  @Override
-  public void addPeers(Iterable<RaftPeer> peers) {
-    // do nothing
-  }
-
-  final RequestHandler.HandlerInterface<RaftServerRequest, RaftServerReply> serverHandlerImpl
-      = new RequestHandler.HandlerInterface<RaftServerRequest, RaftServerReply>() {
-    @Override
-    public boolean isAlive() {
-      return server.isAlive();
-    }
-
-    @Override
-    public RaftServerReply handleRequest(RaftServerRequest r)
-        throws IOException {
-      if (r.isAppendEntries()) {
-        return new RaftServerReply(server.appendEntries(r.getAppendEntries()));
-      } else if (r.isRequestVote()) {
-        return new RaftServerReply(server.requestVote(r.getRequestVote()));
-      } else if (r.isInstallSnapshot()) {
-        return new RaftServerReply(server.installSnapshot(r.getInstallSnapshot()));
-      } else {
-        throw new IllegalStateException("unexpected state");
-      }
-    }
-  };
-
-  final RequestHandler.HandlerInterface<RaftClientRequest, RaftClientReply> clientHandlerImpl
-      = new RequestHandler.HandlerInterface<RaftClientRequest, RaftClientReply>() {
-    @Override
-    public boolean isAlive() {
-      return server.isAlive();
-    }
-
-    @Override
-    public RaftClientReply handleRequest(RaftClientRequest request)
-        throws IOException {
-      final CompletableFuture<RaftClientReply> future;
-      if (request instanceof SetConfigurationRequest) {
-        future = server.setConfigurationAsync((SetConfigurationRequest) request);
-      } else {
-        future = server.submitClientRequestAsync(request);
-      }
-
-      future.whenCompleteAsync((reply, exception) -> {
-        try {
-          IOException e = null;
-          if (exception != null) {
-            e = exception instanceof IOException ?
-                (IOException) exception : new IOException(exception);
-          }
-          clientHandler.getRpc().sendReply(request, reply, e);
-        } catch (IOException e) {
-          LOG.warn("Failed to send reply {} for request {} due to exception {}",
-              reply, request, e);
-        }
-      }, executor);
-      return null;
-    }
-  };
-}

http://git-wip-us.apache.org/repos/asf/incubator-ratis/blob/7e71a2e0/raft-server/src/test/java/org/apache/raft/server/simulation/TestNotLeaderExceptionWithSimulation.java
----------------------------------------------------------------------
diff --git a/raft-server/src/test/java/org/apache/raft/server/simulation/TestNotLeaderExceptionWithSimulation.java b/raft-server/src/test/java/org/apache/raft/server/simulation/TestNotLeaderExceptionWithSimulation.java
deleted file mode 100644
index d6f6acb..0000000
--- a/raft-server/src/test/java/org/apache/raft/server/simulation/TestNotLeaderExceptionWithSimulation.java
+++ /dev/null
@@ -1,32 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- *  with the License.  You may obtain a copy of the License at
- *
- *      http://www.apache.org/licenses/LICENSE-2.0
- *
- *  Unless required by applicable law or agreed to in writing, software
- *  distributed under the License is distributed on an "AS IS" BASIS,
- *  WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- *  See the License for the specific language governing permissions and
- *  limitations under the License.
- */
-package org.apache.raft.server.simulation;
-
-import org.apache.raft.MiniRaftCluster;
-import org.apache.raft.RaftNotLeaderExceptionBaseTest;
-import org.apache.raft.conf.RaftProperties;
-
-import java.io.IOException;
-
-public class TestNotLeaderExceptionWithSimulation extends RaftNotLeaderExceptionBaseTest {
-  @Override
-  public MiniRaftCluster initCluster() throws IOException {
-    String[] s = MiniRaftCluster.generateIds(NUM_PEERS, 0);
-    return new MiniRaftClusterWithSimulatedRpc(s, new RaftProperties(), true);
-  }
-}

http://git-wip-us.apache.org/repos/asf/incubator-ratis/blob/7e71a2e0/raft-server/src/test/java/org/apache/raft/server/simulation/TestRaftReconfigurationWithSimulatedRpc.java
----------------------------------------------------------------------
diff --git a/raft-server/src/test/java/org/apache/raft/server/simulation/TestRaftReconfigurationWithSimulatedRpc.java b/raft-server/src/test/java/org/apache/raft/server/simulation/TestRaftReconfigurationWithSimulatedRpc.java
deleted file mode 100644
index b0eb456..0000000
--- a/raft-server/src/test/java/org/apache/raft/server/simulation/TestRaftReconfigurationWithSimulatedRpc.java
+++ /dev/null
@@ -1,31 +0,0 @@
-/**
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- *     http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-package org.apache.raft.server.simulation;
-
-import org.apache.raft.MiniRaftCluster;
-import org.apache.raft.server.impl.RaftReconfigurationBaseTest;
-
-import java.io.IOException;
-
-public class TestRaftReconfigurationWithSimulatedRpc
-    extends RaftReconfigurationBaseTest {
-  @Override
-  public MiniRaftCluster getCluster(int peerNum) throws IOException {
-    return new MiniRaftClusterWithSimulatedRpc(peerNum, prop);
-  }
-}

http://git-wip-us.apache.org/repos/asf/incubator-ratis/blob/7e71a2e0/raft-server/src/test/java/org/apache/raft/server/simulation/TestRaftSnapshotWithSimulatedRpc.java
----------------------------------------------------------------------
diff --git a/raft-server/src/test/java/org/apache/raft/server/simulation/TestRaftSnapshotWithSimulatedRpc.java b/raft-server/src/test/java/org/apache/raft/server/simulation/TestRaftSnapshotWithSimulatedRpc.java
deleted file mode 100644
index 7c8f873..0000000
--- a/raft-server/src/test/java/org/apache/raft/server/simulation/TestRaftSnapshotWithSimulatedRpc.java
+++ /dev/null
@@ -1,32 +0,0 @@
-/**
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- *     http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-package org.apache.raft.server.simulation;
-
-import org.apache.raft.MiniRaftCluster;
-import org.apache.raft.conf.RaftProperties;
-import org.apache.raft.statemachine.RaftSnapshotBaseTest;
-
-import java.io.IOException;
-
-public class TestRaftSnapshotWithSimulatedRpc extends RaftSnapshotBaseTest {
-  @Override
-  public MiniRaftCluster initCluster(int numServer, RaftProperties prop)
-      throws IOException {
-    return MiniRaftClusterWithSimulatedRpc.FACTORY.newCluster(numServer, prop, true);
-  }
-}

http://git-wip-us.apache.org/repos/asf/incubator-ratis/blob/7e71a2e0/raft-server/src/test/java/org/apache/raft/server/simulation/TestRaftWithSimulatedRpc.java
----------------------------------------------------------------------
diff --git a/raft-server/src/test/java/org/apache/raft/server/simulation/TestRaftWithSimulatedRpc.java b/raft-server/src/test/java/org/apache/raft/server/simulation/TestRaftWithSimulatedRpc.java
deleted file mode 100644
index faa9dd8..0000000
--- a/raft-server/src/test/java/org/apache/raft/server/simulation/TestRaftWithSimulatedRpc.java
+++ /dev/null
@@ -1,51 +0,0 @@
-/**
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- *     http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-package org.apache.raft.server.simulation;
-
-import org.apache.log4j.Level;
-import org.apache.raft.RaftBasicTests;
-import org.apache.raft.client.RaftClient;
-import org.apache.raft.conf.RaftProperties;
-import org.apache.raft.server.impl.RaftServerImpl;
-import org.apache.raft.util.RaftUtils;
-
-import java.io.IOException;
-import java.util.concurrent.ThreadLocalRandom;
-
-public class TestRaftWithSimulatedRpc extends RaftBasicTests {
-  static {
-    RaftUtils.setLogLevel(RaftServerImpl.LOG, Level.DEBUG);
-    RaftUtils.setLogLevel(RaftClient.LOG, Level.DEBUG);
-  }
-
-  private final MiniRaftClusterWithSimulatedRpc cluster;
-
-  public TestRaftWithSimulatedRpc() throws IOException {
-    final RaftProperties properties = getProperties();
-    if (ThreadLocalRandom.current().nextBoolean()) {
-      // turn off simulate latency half of the times.
-      properties.setInt(SimulatedRequestReply.SIMULATE_LATENCY_KEY, 0);
-    }
-    cluster = new MiniRaftClusterWithSimulatedRpc(NUM_SERVERS, properties);
-  }
-
-  @Override
-  public MiniRaftClusterWithSimulatedRpc getCluster() {
-    return cluster;
-  }
-}


[40/54] [abbrv] incubator-ratis git commit: Renamed the packages from raft to ratis in preperation for Apache Incubation - Moved all java packages from org.apache.raft to org.apache.ratis. - Moved native package to org_apache_ratis, and native lib to l

Posted by ji...@apache.org.
http://git-wip-us.apache.org/repos/asf/incubator-ratis/blob/7e71a2e0/raft-hadoop/src/main/java/org/apache/raft/hadooprpc/client/RaftClientProtocolClientSideTranslatorPB.java
----------------------------------------------------------------------
diff --git a/raft-hadoop/src/main/java/org/apache/raft/hadooprpc/client/RaftClientProtocolClientSideTranslatorPB.java b/raft-hadoop/src/main/java/org/apache/raft/hadooprpc/client/RaftClientProtocolClientSideTranslatorPB.java
deleted file mode 100644
index f93bd0a..0000000
--- a/raft-hadoop/src/main/java/org/apache/raft/hadooprpc/client/RaftClientProtocolClientSideTranslatorPB.java
+++ /dev/null
@@ -1,70 +0,0 @@
-/**
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- *     http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-package org.apache.raft.hadooprpc.client;
-
-import org.apache.hadoop.classification.InterfaceAudience;
-import org.apache.hadoop.conf.Configuration;
-import org.apache.raft.client.impl.ClientProtoUtils;
-import org.apache.raft.hadooprpc.Proxy;
-import org.apache.raft.protocol.RaftClientProtocol;
-import org.apache.raft.protocol.RaftClientReply;
-import org.apache.raft.protocol.RaftClientRequest;
-import org.apache.raft.protocol.SetConfigurationRequest;
-import org.apache.raft.shaded.com.google.protobuf.ServiceException;
-import org.apache.raft.shaded.proto.RaftProtos.RaftClientReplyProto;
-import org.apache.raft.shaded.proto.RaftProtos.RaftClientRequestProto;
-import org.apache.raft.shaded.proto.RaftProtos.SetConfigurationRequestProto;
-import org.apache.raft.util.ProtoUtils;
-
-import java.io.IOException;
-
-@InterfaceAudience.Private
-public class RaftClientProtocolClientSideTranslatorPB
-    extends Proxy<RaftClientProtocolPB>
-    implements RaftClientProtocol {
-
-  public RaftClientProtocolClientSideTranslatorPB(
-      String addressStr, Configuration conf) throws IOException {
-    super(RaftClientProtocolPB.class, addressStr, conf);
-  }
-
-  @Override
-  public RaftClientReply submitClientRequest(RaftClientRequest request)
-      throws IOException {
-    final RaftClientRequestProto p = ClientProtoUtils.toRaftClientRequestProto(request);
-    try {
-      final RaftClientReplyProto reply = getProtocol().submitClientRequest(null, p);
-      return ClientProtoUtils.toRaftClientReply(reply);
-    } catch (ServiceException se) {
-      throw ProtoUtils.toIOException(se);
-    }
-  }
-
-  @Override
-  public RaftClientReply setConfiguration(SetConfigurationRequest request)
-      throws IOException {
-    final SetConfigurationRequestProto p
-        = ClientProtoUtils.toSetConfigurationRequestProto(request);
-    try {
-      final RaftClientReplyProto reply = getProtocol().setConfiguration(null, p);
-      return ClientProtoUtils.toRaftClientReply(reply);
-    } catch (ServiceException se) {
-      throw ProtoUtils.toIOException(se);
-    }
-  }
-}

http://git-wip-us.apache.org/repos/asf/incubator-ratis/blob/7e71a2e0/raft-hadoop/src/main/java/org/apache/raft/hadooprpc/client/RaftClientProtocolPB.java
----------------------------------------------------------------------
diff --git a/raft-hadoop/src/main/java/org/apache/raft/hadooprpc/client/RaftClientProtocolPB.java b/raft-hadoop/src/main/java/org/apache/raft/hadooprpc/client/RaftClientProtocolPB.java
deleted file mode 100644
index ca2443a..0000000
--- a/raft-hadoop/src/main/java/org/apache/raft/hadooprpc/client/RaftClientProtocolPB.java
+++ /dev/null
@@ -1,37 +0,0 @@
-/**
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- *     http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-package org.apache.raft.hadooprpc.client;
-
-import org.apache.hadoop.classification.InterfaceAudience;
-import org.apache.hadoop.classification.InterfaceStability;
-import org.apache.hadoop.ipc.ProtocolInfo;
-import org.apache.hadoop.security.KerberosInfo;
-import org.apache.raft.hadooprpc.HadoopConstants;
-import org.apache.raft.shaded.proto.hadoop.HadoopProtos.RaftClientProtocolService;
-
-@InterfaceAudience.Private
-@InterfaceStability.Unstable
-@KerberosInfo(
-    serverPrincipal = HadoopConstants.RAFT_SERVER_KERBEROS_PRINCIPAL_KEY,
-    clientPrincipal = HadoopConstants.RAFT_CLIENT_KERBEROS_PRINCIPAL_KEY)
-@ProtocolInfo(
-    protocolName = HadoopConstants.RAFT_CLIENT_PROTOCOL_NAME,
-    protocolVersion = 1)
-public interface RaftClientProtocolPB extends
-    RaftClientProtocolService.BlockingInterface {
-}

http://git-wip-us.apache.org/repos/asf/incubator-ratis/blob/7e71a2e0/raft-hadoop/src/main/java/org/apache/raft/hadooprpc/client/RaftClientProtocolServerSideTranslatorPB.java
----------------------------------------------------------------------
diff --git a/raft-hadoop/src/main/java/org/apache/raft/hadooprpc/client/RaftClientProtocolServerSideTranslatorPB.java b/raft-hadoop/src/main/java/org/apache/raft/hadooprpc/client/RaftClientProtocolServerSideTranslatorPB.java
deleted file mode 100644
index 4802fd6..0000000
--- a/raft-hadoop/src/main/java/org/apache/raft/hadooprpc/client/RaftClientProtocolServerSideTranslatorPB.java
+++ /dev/null
@@ -1,69 +0,0 @@
-/**
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- *     http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-package org.apache.raft.hadooprpc.client;
-
-import org.apache.hadoop.classification.InterfaceAudience;
-import org.apache.raft.client.impl.ClientProtoUtils;
-import org.apache.raft.protocol.RaftClientProtocol;
-import org.apache.raft.protocol.RaftClientReply;
-import org.apache.raft.protocol.RaftClientRequest;
-import org.apache.raft.protocol.SetConfigurationRequest;
-import org.apache.raft.shaded.com.google.protobuf.RpcController;
-import org.apache.raft.shaded.com.google.protobuf.ServiceException;
-import org.apache.raft.shaded.proto.RaftProtos.RaftClientReplyProto;
-import org.apache.raft.shaded.proto.RaftProtos.RaftClientRequestProto;
-import org.apache.raft.shaded.proto.RaftProtos.SetConfigurationRequestProto;
-
-import java.io.IOException;
-
-@InterfaceAudience.Private
-public class RaftClientProtocolServerSideTranslatorPB
-    implements RaftClientProtocolPB {
-  private final RaftClientProtocol impl;
-
-  public RaftClientProtocolServerSideTranslatorPB(RaftClientProtocol impl) {
-    this.impl = impl;
-  }
-
-  @Override
-  public RaftClientReplyProto submitClientRequest(
-      RpcController unused, RaftClientRequestProto proto)
-      throws ServiceException {
-    final RaftClientRequest request = ClientProtoUtils.toRaftClientRequest(proto);
-    try {
-      final RaftClientReply reply = impl.submitClientRequest(request);
-      return ClientProtoUtils.toRaftClientReplyProto(reply);
-    } catch(IOException ioe) {
-      throw new ServiceException(ioe);
-    }
-  }
-
-  @Override
-  public RaftClientReplyProto setConfiguration(
-      RpcController unused, SetConfigurationRequestProto proto)
-      throws ServiceException {
-    final SetConfigurationRequest request;
-    try {
-      request = ClientProtoUtils.toSetConfigurationRequest(proto);
-      final RaftClientReply reply = impl.setConfiguration(request);
-      return ClientProtoUtils.toRaftClientReplyProto(reply);
-    } catch(IOException ioe) {
-      throw new ServiceException(ioe);
-    }
-  }
-}

http://git-wip-us.apache.org/repos/asf/incubator-ratis/blob/7e71a2e0/raft-hadoop/src/main/java/org/apache/raft/hadooprpc/server/HadoopRpcService.java
----------------------------------------------------------------------
diff --git a/raft-hadoop/src/main/java/org/apache/raft/hadooprpc/server/HadoopRpcService.java b/raft-hadoop/src/main/java/org/apache/raft/hadooprpc/server/HadoopRpcService.java
deleted file mode 100644
index 24e1d2c..0000000
--- a/raft-hadoop/src/main/java/org/apache/raft/hadooprpc/server/HadoopRpcService.java
+++ /dev/null
@@ -1,178 +0,0 @@
-/**
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- *     http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-package org.apache.raft.hadooprpc.server;
-
-import com.google.common.base.Preconditions;
-import org.apache.hadoop.conf.Configuration;
-import org.apache.hadoop.ipc.ProtobufRpcEngineShaded;
-import org.apache.hadoop.ipc.RPC;
-import org.apache.raft.hadooprpc.Proxy;
-import org.apache.raft.hadooprpc.client.RaftClientProtocolPB;
-import org.apache.raft.hadooprpc.client.RaftClientProtocolServerSideTranslatorPB;
-import org.apache.raft.protocol.RaftClientProtocol;
-import org.apache.raft.protocol.RaftPeer;
-import org.apache.raft.server.RaftServer;
-import org.apache.raft.server.RaftServerConfigKeys;
-import org.apache.raft.server.RaftServerRpc;
-import org.apache.raft.server.protocol.RaftServerProtocol;
-import org.apache.raft.shaded.com.google.protobuf.BlockingService;
-import org.apache.raft.shaded.com.google.protobuf.ServiceException;
-import org.apache.raft.shaded.proto.RaftProtos.*;
-import org.apache.raft.shaded.proto.hadoop.HadoopProtos.RaftClientProtocolService;
-import org.apache.raft.shaded.proto.hadoop.HadoopProtos.RaftServerProtocolService;
-import org.apache.raft.util.CodeInjectionForTesting;
-import org.apache.raft.util.PeerProxyMap;
-import org.apache.raft.util.ProtoUtils;
-import org.slf4j.Logger;
-import org.slf4j.LoggerFactory;
-
-import java.io.IOException;
-import java.net.InetSocketAddress;
-
-/** Server side Hadoop RPC service. */
-public class HadoopRpcService implements RaftServerRpc {
-  public static final Logger LOG = LoggerFactory.getLogger(HadoopRpcService.class);
-  static final String CLASS_NAME = HadoopRpcService.class.getSimpleName();
-  public static final String SEND_SERVER_REQUEST = CLASS_NAME + ".sendServerRequest";
-
-  private final String id;
-  private final RPC.Server ipcServer;
-  private final InetSocketAddress ipcServerAddress;
-
-  private final PeerProxyMap<Proxy<RaftServerProtocolPB>> proxies;
-
-  public HadoopRpcService(RaftServer server, final Configuration conf)
-      throws IOException {
-    this.proxies = new PeerProxyMap<>(
-        p -> new Proxy(RaftServerProtocolPB.class, p.getAddress(), conf));
-    this.id = server.getId();
-    this.ipcServer = newRpcServer(server, conf);
-    this.ipcServerAddress = ipcServer.getListenerAddress();
-
-    addRaftClientProtocol(server, conf);
-
-    LOG.info(getClass().getSimpleName() + " created RPC.Server at "
-        + ipcServerAddress);
-  }
-
-  @Override
-  public InetSocketAddress getInetSocketAddress() {
-    return ipcServerAddress;
-  }
-
-  private RPC.Server newRpcServer(RaftServerProtocol serverProtocol, final Configuration conf)
-      throws IOException {
-    final RaftServerConfigKeys.Get get = new RaftServerConfigKeys.Get() {
-      @Override
-      protected int getInt(String key, int defaultValue) {
-        return conf.getInt(key, defaultValue);
-      }
-
-      @Override
-      protected String getTrimmed(String key, String defaultValue) {
-        return conf.getTrimmed(key, defaultValue);
-      }
-    };
-
-    final int handlerCount = get.ipc().handlers();
-    final InetSocketAddress address = get.ipc().address();
-
-    final BlockingService service
-        = RaftServerProtocolService.newReflectiveBlockingService(
-            new RaftServerProtocolServerSideTranslatorPB(serverProtocol));
-    RPC.setProtocolEngine(conf, RaftServerProtocolPB.class, ProtobufRpcEngineShaded.class);
-    return new RPC.Builder(conf)
-        .setProtocol(RaftServerProtocolPB.class)
-        .setInstance(service)
-        .setBindAddress(address.getHostName())
-        .setPort(address.getPort())
-        .setNumHandlers(handlerCount)
-        .setVerbose(false)
-        .build();
-  }
-
-  private void addRaftClientProtocol(RaftClientProtocol clientProtocol, Configuration conf) {
-    final Class<?> protocol = RaftClientProtocolPB.class;
-    RPC.setProtocolEngine(conf,protocol, ProtobufRpcEngineShaded.class);
-
-    final BlockingService service
-        = RaftClientProtocolService.newReflectiveBlockingService(
-        new RaftClientProtocolServerSideTranslatorPB(clientProtocol));
-    ipcServer.addProtocol(RPC.RpcKind.RPC_PROTOCOL_BUFFER, protocol, service);
-  }
-
-  @Override
-  public void start() {
-    ipcServer.start();
-  }
-
-  @Override
-  public void close() {
-    ipcServer.stop();
-  }
-
-  @Override
-  public AppendEntriesReplyProto appendEntries(
-      AppendEntriesRequestProto request) throws IOException {
-    Preconditions.checkArgument(id.equals(request.getServerRequest().getRequestorId()));
-    CodeInjectionForTesting.execute(SEND_SERVER_REQUEST, id, null, request);
-
-    final RaftServerProtocolPB proxy = proxies.getProxy(
-        request.getServerRequest().getReplyId()).getProtocol();
-    try {
-      return proxy.appendEntries(null, request);
-    } catch (ServiceException se) {
-      throw ProtoUtils.toIOException(se);
-    }
-  }
-
-  @Override
-  public InstallSnapshotReplyProto installSnapshot(
-      InstallSnapshotRequestProto request) throws IOException {
-    Preconditions.checkArgument(id.equals(request.getServerRequest().getRequestorId()));
-    CodeInjectionForTesting.execute(SEND_SERVER_REQUEST, id, null, request);
-
-    final RaftServerProtocolPB proxy = proxies.getProxy(
-        request.getServerRequest().getReplyId()).getProtocol();
-    try {
-      return proxy.installSnapshot(null, request);
-    } catch (ServiceException se) {
-      throw ProtoUtils.toIOException(se);
-    }
-  }
-
-  @Override
-  public RequestVoteReplyProto requestVote(
-      RequestVoteRequestProto request) throws IOException {
-    Preconditions.checkArgument(id.equals(request.getServerRequest().getRequestorId()));
-    CodeInjectionForTesting.execute(SEND_SERVER_REQUEST, id, null, request);
-
-    final RaftServerProtocolPB proxy = proxies.getProxy(
-        request.getServerRequest().getReplyId()).getProtocol();
-    try {
-      return proxy.requestVote(null, request);
-    } catch (ServiceException se) {
-      throw ProtoUtils.toIOException(se);
-    }
-  }
-
-  @Override
-  public void addPeers(Iterable<RaftPeer> peers) {
-    proxies.addPeers(peers);
-  }
-}

http://git-wip-us.apache.org/repos/asf/incubator-ratis/blob/7e71a2e0/raft-hadoop/src/main/java/org/apache/raft/hadooprpc/server/RaftServerProtocolPB.java
----------------------------------------------------------------------
diff --git a/raft-hadoop/src/main/java/org/apache/raft/hadooprpc/server/RaftServerProtocolPB.java b/raft-hadoop/src/main/java/org/apache/raft/hadooprpc/server/RaftServerProtocolPB.java
deleted file mode 100644
index 27d3bba..0000000
--- a/raft-hadoop/src/main/java/org/apache/raft/hadooprpc/server/RaftServerProtocolPB.java
+++ /dev/null
@@ -1,37 +0,0 @@
-/**
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- *     http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-package org.apache.raft.hadooprpc.server;
-
-import org.apache.hadoop.classification.InterfaceAudience;
-import org.apache.hadoop.classification.InterfaceStability;
-import org.apache.hadoop.ipc.ProtocolInfo;
-import org.apache.hadoop.security.KerberosInfo;
-import org.apache.raft.hadooprpc.HadoopConstants;
-import org.apache.raft.shaded.proto.hadoop.HadoopProtos.RaftServerProtocolService;
-
-@InterfaceAudience.Private
-@InterfaceStability.Unstable
-@KerberosInfo(
-    serverPrincipal = HadoopConstants.RAFT_SERVER_KERBEROS_PRINCIPAL_KEY,
-    clientPrincipal = HadoopConstants.RAFT_SERVER_KERBEROS_PRINCIPAL_KEY)
-@ProtocolInfo(
-    protocolName = HadoopConstants.RAFT_SERVER_PROTOCOL_NAME,
-    protocolVersion = 1)
-public interface RaftServerProtocolPB extends
-    RaftServerProtocolService.BlockingInterface {
-}

http://git-wip-us.apache.org/repos/asf/incubator-ratis/blob/7e71a2e0/raft-hadoop/src/main/java/org/apache/raft/hadooprpc/server/RaftServerProtocolServerSideTranslatorPB.java
----------------------------------------------------------------------
diff --git a/raft-hadoop/src/main/java/org/apache/raft/hadooprpc/server/RaftServerProtocolServerSideTranslatorPB.java b/raft-hadoop/src/main/java/org/apache/raft/hadooprpc/server/RaftServerProtocolServerSideTranslatorPB.java
deleted file mode 100644
index c598eb3..0000000
--- a/raft-hadoop/src/main/java/org/apache/raft/hadooprpc/server/RaftServerProtocolServerSideTranslatorPB.java
+++ /dev/null
@@ -1,68 +0,0 @@
-/**
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- *     http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-package org.apache.raft.hadooprpc.server;
-
-import org.apache.hadoop.classification.InterfaceAudience;
-import org.apache.raft.server.protocol.RaftServerProtocol;
-import org.apache.raft.shaded.com.google.protobuf.RpcController;
-import org.apache.raft.shaded.com.google.protobuf.ServiceException;
-import org.apache.raft.shaded.proto.RaftProtos.*;
-
-import java.io.IOException;
-
-@InterfaceAudience.Private
-public class RaftServerProtocolServerSideTranslatorPB
-    implements RaftServerProtocolPB {
-  private final RaftServerProtocol impl;
-
-  public RaftServerProtocolServerSideTranslatorPB(RaftServerProtocol impl) {
-    this.impl = impl;
-  }
-
-  @Override
-  public RequestVoteReplyProto requestVote(
-      RpcController unused, RequestVoteRequestProto request)
-      throws ServiceException {
-    try {
-      return impl.requestVote(request);
-    } catch(IOException ioe) {
-      throw new ServiceException(ioe);
-    }
-  }
-
-  @Override
-  public AppendEntriesReplyProto appendEntries(
-      RpcController unused, AppendEntriesRequestProto request)
-      throws ServiceException {
-    try {
-      return impl.appendEntries(request);
-    } catch(IOException ioe) {
-      throw new ServiceException(ioe);
-    }
-  }
-
-  @Override
-  public InstallSnapshotReplyProto installSnapshot(RpcController controller,
-      InstallSnapshotRequestProto request) throws ServiceException {
-    try {
-      return impl.installSnapshot(request);
-    } catch(IOException ioe) {
-      throw new ServiceException(ioe);
-    }
-  }
-}

http://git-wip-us.apache.org/repos/asf/incubator-ratis/blob/7e71a2e0/raft-hadoop/src/test/java/org/apache/raft/hadooprpc/MiniRaftClusterWithHadoopRpc.java
----------------------------------------------------------------------
diff --git a/raft-hadoop/src/test/java/org/apache/raft/hadooprpc/MiniRaftClusterWithHadoopRpc.java b/raft-hadoop/src/test/java/org/apache/raft/hadooprpc/MiniRaftClusterWithHadoopRpc.java
deleted file mode 100644
index 87b4d1c..0000000
--- a/raft-hadoop/src/test/java/org/apache/raft/hadooprpc/MiniRaftClusterWithHadoopRpc.java
+++ /dev/null
@@ -1,117 +0,0 @@
-/**
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- *     http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-package org.apache.raft.hadooprpc;
-
-import com.google.common.base.Preconditions;
-import org.apache.hadoop.conf.Configuration;
-import org.apache.raft.MiniRaftCluster;
-import org.apache.raft.RaftTestUtil;
-import org.apache.raft.client.RaftClientRequestSender;
-import org.apache.raft.conf.RaftProperties;
-import org.apache.raft.hadooprpc.client.HadoopClientRequestSender;
-import org.apache.raft.hadooprpc.server.HadoopRpcService;
-import org.apache.raft.protocol.RaftPeer;
-import org.apache.raft.server.impl.DelayLocalExecutionInjection;
-import org.apache.raft.server.impl.RaftServerImpl;
-import org.apache.raft.server.RaftServerConfigKeys;
-import org.slf4j.Logger;
-import org.slf4j.LoggerFactory;
-
-import java.io.IOException;
-import java.util.Collection;
-import java.util.HashMap;
-import java.util.Map;
-
-public class MiniRaftClusterWithHadoopRpc extends MiniRaftCluster.RpcBase {
-  static final Logger LOG = LoggerFactory.getLogger(MiniRaftClusterWithHadoopRpc.class);
-
-  public static final Factory<MiniRaftClusterWithHadoopRpc> FACTORY
-      = new Factory<MiniRaftClusterWithHadoopRpc>() {
-    @Override
-    public MiniRaftClusterWithHadoopRpc newCluster(
-        String[] ids, RaftProperties prop, boolean formatted) throws IOException {
-      final Configuration conf = new Configuration();
-      conf.set(RaftServerConfigKeys.Ipc.ADDRESS_KEY, "0.0.0.0:0");
-      return new MiniRaftClusterWithHadoopRpc(ids, prop, conf, formatted);
-    }
-  };
-
-  public static final DelayLocalExecutionInjection sendServerRequest =
-      new DelayLocalExecutionInjection(HadoopRpcService.SEND_SERVER_REQUEST);
-
-  private final Configuration hadoopConf;
-
-  public MiniRaftClusterWithHadoopRpc(int numServers, RaftProperties properties,
-      Configuration conf) throws IOException {
-    this(generateIds(numServers, 0), properties, conf, true);
-  }
-
-  public MiniRaftClusterWithHadoopRpc(String[] ids, RaftProperties properties,
-      Configuration hadoopConf, boolean formatted) throws IOException {
-    super(ids, properties, formatted);
-    this.hadoopConf = hadoopConf;
-
-    init(initRpcServices(getServers(), hadoopConf));
-  }
-
-  private static Map<RaftPeer, HadoopRpcService> initRpcServices(
-      Collection<RaftServerImpl> servers, Configuration hadoopConf) throws IOException {
-    final Map<RaftPeer, HadoopRpcService> peerRpcs = new HashMap<>();
-
-    for(RaftServerImpl s : servers) {
-      final HadoopRpcService rpc = new HadoopRpcService(s, hadoopConf);
-      peerRpcs.put(new RaftPeer(s.getId(), rpc.getInetSocketAddress()), rpc);
-    }
-    return peerRpcs;
-  }
-
-  @Override
-  protected RaftServerImpl setPeerRpc(RaftPeer peer) throws IOException {
-    Configuration hconf = new Configuration(hadoopConf);
-    hconf.set(RaftServerConfigKeys.Ipc.ADDRESS_KEY, peer.getAddress());
-
-    RaftServerImpl server = servers.get(peer.getId());
-    final HadoopRpcService rpc = new HadoopRpcService(server, hconf);
-    Preconditions.checkState(
-        rpc.getInetSocketAddress().toString().contains(peer.getAddress()),
-        "address in the raft conf: %s, address in rpc server: %s",
-        peer.getAddress(), rpc.getInetSocketAddress().toString());
-    server.setServerRpc(rpc);
-    return server;
-  }
-
-  @Override
-  public Collection<RaftPeer> addNewPeers(Collection<RaftPeer> newPeers,
-                                          Collection<RaftServerImpl> newServers, boolean startService)
-      throws IOException {
-    return addNewPeers(initRpcServices(newServers, hadoopConf),
-        newServers, startService);
-  }
-
-  @Override
-  public RaftClientRequestSender getRaftClientRequestSender() {
-    return new HadoopClientRequestSender(getPeers(), hadoopConf);
-  }
-
-  @Override
-  public void blockQueueAndSetDelay(String leaderId, int delayMs)
-      throws InterruptedException {
-    RaftTestUtil.blockQueueAndSetDelay(getServers(), sendServerRequest,
-        leaderId, delayMs, getMaxTimeout());
-  }
-}

http://git-wip-us.apache.org/repos/asf/incubator-ratis/blob/7e71a2e0/raft-hadoop/src/test/java/org/apache/raft/hadooprpc/TestNotLeaderExceptionWithHadoopRpc.java
----------------------------------------------------------------------
diff --git a/raft-hadoop/src/test/java/org/apache/raft/hadooprpc/TestNotLeaderExceptionWithHadoopRpc.java b/raft-hadoop/src/test/java/org/apache/raft/hadooprpc/TestNotLeaderExceptionWithHadoopRpc.java
deleted file mode 100644
index 6ff13e4..0000000
--- a/raft-hadoop/src/test/java/org/apache/raft/hadooprpc/TestNotLeaderExceptionWithHadoopRpc.java
+++ /dev/null
@@ -1,37 +0,0 @@
-/**
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- *     http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-package org.apache.raft.hadooprpc;
-
-import org.apache.hadoop.conf.Configuration;
-import org.apache.raft.MiniRaftCluster;
-import org.apache.raft.RaftNotLeaderExceptionBaseTest;
-import org.apache.raft.conf.RaftProperties;
-import org.apache.raft.server.RaftServerConfigKeys;
-
-import java.io.IOException;
-
-public class TestNotLeaderExceptionWithHadoopRpc extends RaftNotLeaderExceptionBaseTest {
-  @Override
-  public MiniRaftCluster initCluster() throws IOException {
-    String[] s = MiniRaftCluster.generateIds(NUM_PEERS, 0);
-    final Configuration conf = new Configuration();
-    conf.set(RaftServerConfigKeys.Ipc.ADDRESS_KEY, "0.0.0.0:0");
-    RaftProperties prop = new RaftProperties();
-    return new MiniRaftClusterWithHadoopRpc(s, prop, conf, true);
-  }
-}

http://git-wip-us.apache.org/repos/asf/incubator-ratis/blob/7e71a2e0/raft-hadoop/src/test/java/org/apache/raft/hadooprpc/TestRaftReconfigurationWithHadoopRpc.java
----------------------------------------------------------------------
diff --git a/raft-hadoop/src/test/java/org/apache/raft/hadooprpc/TestRaftReconfigurationWithHadoopRpc.java b/raft-hadoop/src/test/java/org/apache/raft/hadooprpc/TestRaftReconfigurationWithHadoopRpc.java
deleted file mode 100644
index 0116280..0000000
--- a/raft-hadoop/src/test/java/org/apache/raft/hadooprpc/TestRaftReconfigurationWithHadoopRpc.java
+++ /dev/null
@@ -1,40 +0,0 @@
-/**
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- *     http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-package org.apache.raft.hadooprpc;
-
-import org.apache.hadoop.conf.Configuration;
-import org.apache.raft.MiniRaftCluster;
-import org.apache.raft.server.RaftServerConfigKeys;
-import org.apache.raft.server.impl.RaftReconfigurationBaseTest;
-
-import java.io.IOException;
-
-import static org.apache.hadoop.fs.CommonConfigurationKeysPublic.IPC_CLIENT_CONNECT_MAX_RETRIES_KEY;
-import static org.apache.hadoop.fs.CommonConfigurationKeysPublic.IPC_CLIENT_CONNECT_TIMEOUT_KEY;
-
-public class TestRaftReconfigurationWithHadoopRpc
-    extends RaftReconfigurationBaseTest {
-  @Override
-  public MiniRaftCluster getCluster(int peerNum) throws IOException {
-    final Configuration hadoopConf = new Configuration();
-    hadoopConf.setInt(IPC_CLIENT_CONNECT_TIMEOUT_KEY, 1000);
-    hadoopConf.setInt(IPC_CLIENT_CONNECT_MAX_RETRIES_KEY, 0);
-    hadoopConf.set(RaftServerConfigKeys.Ipc.ADDRESS_KEY, "0.0.0.0:0");
-    return new MiniRaftClusterWithHadoopRpc(peerNum, prop, hadoopConf);
-  }
-}

http://git-wip-us.apache.org/repos/asf/incubator-ratis/blob/7e71a2e0/raft-hadoop/src/test/java/org/apache/raft/hadooprpc/TestRaftSnapshotWithHadoopRpc.java
----------------------------------------------------------------------
diff --git a/raft-hadoop/src/test/java/org/apache/raft/hadooprpc/TestRaftSnapshotWithHadoopRpc.java b/raft-hadoop/src/test/java/org/apache/raft/hadooprpc/TestRaftSnapshotWithHadoopRpc.java
deleted file mode 100644
index 43dc8e4..0000000
--- a/raft-hadoop/src/test/java/org/apache/raft/hadooprpc/TestRaftSnapshotWithHadoopRpc.java
+++ /dev/null
@@ -1,32 +0,0 @@
-/**
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- *     http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-package org.apache.raft.hadooprpc;
-
-import org.apache.raft.MiniRaftCluster;
-import org.apache.raft.conf.RaftProperties;
-import org.apache.raft.statemachine.RaftSnapshotBaseTest;
-
-import java.io.IOException;
-
-public class TestRaftSnapshotWithHadoopRpc extends RaftSnapshotBaseTest {
-  @Override
-  public MiniRaftCluster initCluster(int numServer, RaftProperties prop)
-      throws IOException {
-    return MiniRaftClusterWithHadoopRpc.FACTORY.newCluster(numServer, prop, true);
-  }
-}

http://git-wip-us.apache.org/repos/asf/incubator-ratis/blob/7e71a2e0/raft-hadoop/src/test/java/org/apache/raft/hadooprpc/TestRaftWithHadoopRpc.java
----------------------------------------------------------------------
diff --git a/raft-hadoop/src/test/java/org/apache/raft/hadooprpc/TestRaftWithHadoopRpc.java b/raft-hadoop/src/test/java/org/apache/raft/hadooprpc/TestRaftWithHadoopRpc.java
deleted file mode 100644
index 4848196..0000000
--- a/raft-hadoop/src/test/java/org/apache/raft/hadooprpc/TestRaftWithHadoopRpc.java
+++ /dev/null
@@ -1,69 +0,0 @@
-/**
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- *     http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-package org.apache.raft.hadooprpc;
-
-import org.apache.hadoop.conf.Configuration;
-import org.apache.log4j.Level;
-import org.apache.raft.RaftBasicTests;
-import org.apache.raft.client.RaftClient;
-import org.apache.raft.server.impl.BlockRequestHandlingInjection;
-import org.apache.raft.server.impl.RaftServerImpl;
-import org.apache.raft.server.RaftServerConfigKeys;
-import org.apache.raft.util.RaftUtils;
-import org.junit.Test;
-
-import java.io.IOException;
-
-import static org.apache.raft.hadooprpc.MiniRaftClusterWithHadoopRpc.sendServerRequest;
-
-public class TestRaftWithHadoopRpc extends RaftBasicTests {
-  static {
-    RaftUtils.setLogLevel(RaftServerImpl.LOG, Level.DEBUG);
-    RaftUtils.setLogLevel(RaftClient.LOG, Level.DEBUG);
-    RaftUtils.setLogLevel(MiniRaftClusterWithHadoopRpc.LOG, Level.DEBUG);
-  }
-
-  private final MiniRaftClusterWithHadoopRpc cluster;
-
-  public TestRaftWithHadoopRpc() throws IOException {
-    Configuration conf = new Configuration();
-    conf.set(RaftServerConfigKeys.Ipc.ADDRESS_KEY, "0.0.0.0:0");
-    cluster = new MiniRaftClusterWithHadoopRpc(NUM_SERVERS, getProperties(), conf);
-  }
-
-  @Override
-  public MiniRaftClusterWithHadoopRpc getCluster() {
-    return cluster;
-  }
-
-  @Override
-  @Test
-  public void testEnforceLeader() throws Exception {
-    super.testEnforceLeader();
-
-    sendServerRequest.clear();
-    BlockRequestHandlingInjection.getInstance().unblockAll();
-  }
-
-  @Override
-  @Test
-  public void testWithLoad() throws Exception {
-    super.testWithLoad();
-    BlockRequestHandlingInjection.getInstance().unblockAll();
-  }
-}

http://git-wip-us.apache.org/repos/asf/incubator-ratis/blob/7e71a2e0/raft-hadoop/src/test/resources/log4j.properties
----------------------------------------------------------------------
diff --git a/raft-hadoop/src/test/resources/log4j.properties b/raft-hadoop/src/test/resources/log4j.properties
deleted file mode 100644
index ced0687..0000000
--- a/raft-hadoop/src/test/resources/log4j.properties
+++ /dev/null
@@ -1,18 +0,0 @@
-#   Licensed 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.
-# log4j configuration used during build and unit tests
-
-log4j.rootLogger=info,stdout
-log4j.threshold=ALL
-log4j.appender.stdout=org.apache.log4j.ConsoleAppender
-log4j.appender.stdout.layout=org.apache.log4j.PatternLayout
-log4j.appender.stdout.layout.ConversionPattern=%d{ISO8601} %-5p %c{2} (%F:%M(%L)) - %m%n

http://git-wip-us.apache.org/repos/asf/incubator-ratis/blob/7e71a2e0/raft-netty/pom.xml
----------------------------------------------------------------------
diff --git a/raft-netty/pom.xml b/raft-netty/pom.xml
deleted file mode 100644
index 41494a0..0000000
--- a/raft-netty/pom.xml
+++ /dev/null
@@ -1,99 +0,0 @@
-<?xml version="1.0" encoding="UTF-8"?>
-<!--
-  Licensed 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. See accompanying LICENSE file.
--->
-<project xmlns="http://maven.apache.org/POM/4.0.0"
-         xmlns:xsi="http://www.w3.org/2001/XMLSchema-instance"
-         xsi:schemaLocation="http://maven.apache.org/POM/4.0.0 http://maven.apache.org/xsd/maven-4.0.0.xsd">
-  <modelVersion>4.0.0</modelVersion>
-  <parent>
-    <artifactId>raft-project-dist</artifactId>
-    <groupId>com.hortonworks.raft</groupId>
-    <version>1.0-SNAPSHOT</version>
-    <relativePath>../raft-project-dist</relativePath>
-  </parent>
-
-  <artifactId>raft-netty</artifactId>
-  <name>Raft Netty Support</name>
-
-  <dependencies>
-    <dependency>
-      <artifactId>raft-proto-shaded</artifactId>
-      <groupId>com.hortonworks.raft</groupId>
-      <scope>provided</scope>
-    </dependency>
-
-    <dependency>
-      <artifactId>raft-common</artifactId>
-      <groupId>com.hortonworks.raft</groupId>
-      <scope>provided</scope>
-    </dependency>
-    <dependency>
-      <artifactId>raft-common</artifactId>
-      <groupId>com.hortonworks.raft</groupId>
-      <scope>test</scope>
-      <type>test-jar</type>
-    </dependency>
-    
-    <dependency>
-      <artifactId>raft-client</artifactId>
-      <groupId>com.hortonworks.raft</groupId>
-      <scope>provided</scope>
-    </dependency>
-    <dependency>
-      <artifactId>raft-client</artifactId>
-      <groupId>com.hortonworks.raft</groupId>
-      <scope>test</scope>
-      <type>test-jar</type>
-    </dependency>
-    
-    <dependency>
-      <artifactId>raft-server</artifactId>
-      <groupId>com.hortonworks.raft</groupId>
-      <scope>provided</scope>
-    </dependency>
-    <dependency>
-      <artifactId>raft-server</artifactId>
-      <groupId>com.hortonworks.raft</groupId>
-      <scope>test</scope>
-      <type>test-jar</type>
-    </dependency>
-    
-    <dependency>
-      <groupId>org.slf4j</groupId>
-      <artifactId>slf4j-api</artifactId>
-    </dependency>
-
-    <dependency>
-      <groupId>com.google.guava</groupId>
-      <artifactId>guava</artifactId>
-    </dependency>
-
-    <dependency>
-      <groupId>junit</groupId>
-      <artifactId>junit</artifactId>
-      <scope>test</scope>
-    </dependency>
-    <dependency>
-      <groupId>org.mockito</groupId>
-      <artifactId>mockito-all</artifactId>
-      <scope>test</scope>
-    </dependency>
-
-    <dependency>
-      <groupId>io.netty</groupId>
-      <artifactId>netty-all</artifactId>
-      <scope>compile</scope>
-    </dependency>
-  </dependencies>
-</project>

http://git-wip-us.apache.org/repos/asf/incubator-ratis/blob/7e71a2e0/raft-netty/src/main/java/org/apache/raft/netty/NettyClient.java
----------------------------------------------------------------------
diff --git a/raft-netty/src/main/java/org/apache/raft/netty/NettyClient.java b/raft-netty/src/main/java/org/apache/raft/netty/NettyClient.java
deleted file mode 100644
index acb44ef..0000000
--- a/raft-netty/src/main/java/org/apache/raft/netty/NettyClient.java
+++ /dev/null
@@ -1,72 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- *  with the License.  You may obtain a copy of the License at
- *
- *      http://www.apache.org/licenses/LICENSE-2.0
- *
- *  Unless required by applicable law or agreed to in writing, software
- *  distributed under the License is distributed on an "AS IS" BASIS,
- *  WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- *  See the License for the specific language governing permissions and
- *  limitations under the License.
- */
-package org.apache.raft.netty;
-
-import com.google.common.base.Preconditions;
-import io.netty.bootstrap.Bootstrap;
-import io.netty.channel.Channel;
-import io.netty.channel.ChannelFuture;
-import io.netty.channel.ChannelInitializer;
-import io.netty.channel.EventLoopGroup;
-import io.netty.channel.socket.SocketChannel;
-import io.netty.channel.socket.nio.NioSocketChannel;
-import io.netty.handler.logging.LogLevel;
-import io.netty.handler.logging.LoggingHandler;
-import org.apache.raft.util.LifeCycle;
-import org.apache.raft.util.NetUtils;
-
-import java.io.Closeable;
-import java.net.InetSocketAddress;
-
-public class NettyClient implements Closeable {
-  private final LifeCycle lifeCycle = new LifeCycle(getClass().getSimpleName());
-
-  private Channel channel;
-
-  /** Connects to the given server address. */
-  public void connect(String serverAddress, EventLoopGroup group,
-                      ChannelInitializer<SocketChannel> initializer)
-      throws InterruptedException {
-    final InetSocketAddress address = NetUtils.newInetSocketAddress(serverAddress);
-    Preconditions.checkNotNull(address,
-        "Failed to create InetSocketAddress from %s.", serverAddress);
-
-    lifeCycle.startAndTransition(
-        () -> channel = new Bootstrap()
-            .group(group)
-            .channel(NioSocketChannel.class)
-            .handler(new LoggingHandler(LogLevel.INFO))
-            .handler(initializer)
-            .connect(address)
-            .sync()
-            .channel(),
-        InterruptedException.class);
-  }
-
-  @Override
-  public void close() {
-    lifeCycle.checkStateAndClose(() -> {
-      channel.close().syncUninterruptibly();
-    });
-  }
-
-  public ChannelFuture writeAndFlush(Object msg) {
-    lifeCycle.assertCurrentState(LifeCycle.State.RUNNING);
-    return channel.writeAndFlush(msg);
-  }
-}

http://git-wip-us.apache.org/repos/asf/incubator-ratis/blob/7e71a2e0/raft-netty/src/main/java/org/apache/raft/netty/NettyRpcProxy.java
----------------------------------------------------------------------
diff --git a/raft-netty/src/main/java/org/apache/raft/netty/NettyRpcProxy.java b/raft-netty/src/main/java/org/apache/raft/netty/NettyRpcProxy.java
deleted file mode 100644
index 2d5756a..0000000
--- a/raft-netty/src/main/java/org/apache/raft/netty/NettyRpcProxy.java
+++ /dev/null
@@ -1,180 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- *  with the License.  You may obtain a copy of the License at
- *
- *      http://www.apache.org/licenses/LICENSE-2.0
- *
- *  Unless required by applicable law or agreed to in writing, software
- *  distributed under the License is distributed on an "AS IS" BASIS,
- *  WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- *  See the License for the specific language governing permissions and
- *  limitations under the License.
- */
-package org.apache.raft.netty;
-
-import io.netty.channel.*;
-import io.netty.channel.nio.NioEventLoopGroup;
-import io.netty.channel.socket.SocketChannel;
-import org.apache.raft.protocol.RaftPeer;
-import org.apache.raft.shaded.io.netty.handler.codec.protobuf.ProtobufDecoder;
-import org.apache.raft.shaded.io.netty.handler.codec.protobuf.ProtobufEncoder;
-import org.apache.raft.shaded.io.netty.handler.codec.protobuf.ProtobufVarint32FrameDecoder;
-import org.apache.raft.shaded.io.netty.handler.codec.protobuf.ProtobufVarint32LengthFieldPrepender;
-import org.apache.raft.shaded.proto.RaftProtos.RaftRpcRequestProto;
-import org.apache.raft.shaded.proto.netty.NettyProtos.RaftNettyServerReplyProto;
-import org.apache.raft.shaded.proto.netty.NettyProtos.RaftNettyServerRequestProto;
-import org.apache.raft.util.PeerProxyMap;
-import org.apache.raft.util.ProtoUtils;
-import org.apache.raft.util.RaftUtils;
-
-import java.io.Closeable;
-import java.io.IOException;
-import java.util.LinkedList;
-import java.util.Queue;
-import java.util.concurrent.CompletableFuture;
-import java.util.concurrent.ExecutionException;
-
-import static org.apache.raft.shaded.proto.netty.NettyProtos.RaftNettyServerReplyProto.RaftNettyServerReplyCase.EXCEPTIONREPLY;
-
-
-public class NettyRpcProxy implements Closeable {
-  public static class PeerMap extends PeerProxyMap<NettyRpcProxy> {
-    private final EventLoopGroup group = new NioEventLoopGroup();
-
-    @Override
-    public NettyRpcProxy createProxyImpl(RaftPeer peer)
-        throws IOException {
-      try {
-        return new NettyRpcProxy(peer, group);
-      } catch (InterruptedException e) {
-        throw RaftUtils.toInterruptedIOException("Failed connecting to " + peer, e);
-      }
-    }
-
-    @Override
-    public void close() {
-      super.close();
-      group.shutdownGracefully();
-    }
-  }
-
-  public static long getSeqNum(RaftNettyServerReplyProto proto) {
-    switch (proto.getRaftNettyServerReplyCase()) {
-      case REQUESTVOTEREPLY:
-        return proto.getRequestVoteReply().getServerReply().getSeqNum();
-      case APPENDENTRIESREPLY:
-        return proto.getAppendEntriesReply().getServerReply().getSeqNum();
-      case INSTALLSNAPSHOTREPLY:
-        return proto.getInstallSnapshotReply().getServerReply().getSeqNum();
-      case RAFTCLIENTREPLY:
-        return proto.getRaftClientReply().getRpcReply().getSeqNum();
-      case EXCEPTIONREPLY:
-        return proto.getExceptionReply().getRpcReply().getSeqNum();
-      case RAFTNETTYSERVERREPLY_NOT_SET:
-        throw new IllegalArgumentException("Reply case not set in proto: "
-            + proto.getRaftNettyServerReplyCase());
-      default:
-        throw new UnsupportedOperationException("Reply case not supported: "
-            + proto.getRaftNettyServerReplyCase());
-    }
-  }
-
-
-  class Connection implements Closeable {
-    private final NettyClient client = new NettyClient();
-    private final Queue<CompletableFuture<RaftNettyServerReplyProto>> replies
-        = new LinkedList<>();
-
-    Connection(EventLoopGroup group) throws InterruptedException {
-      final ChannelInboundHandler inboundHandler
-          = new SimpleChannelInboundHandler<RaftNettyServerReplyProto>() {
-        @Override
-        protected void channelRead0(ChannelHandlerContext ctx,
-                                    RaftNettyServerReplyProto proto) {
-          final CompletableFuture<RaftNettyServerReplyProto> future = pollReply();
-          if (future == null) {
-            throw new IllegalStateException("Request #" + getSeqNum(proto)
-                + " not found");
-          }
-          if (proto.getRaftNettyServerReplyCase() == EXCEPTIONREPLY) {
-            final Object ioe = ProtoUtils.toObject(proto.getExceptionReply().getException());
-            future.completeExceptionally((IOException)ioe);
-          } else {
-            future.complete(proto);
-          }
-        }
-      };
-      final ChannelInitializer<SocketChannel> initializer
-          = new ChannelInitializer<SocketChannel>() {
-        @Override
-        protected void initChannel(SocketChannel ch) throws Exception {
-          final ChannelPipeline p = ch.pipeline();
-
-          p.addLast(new ProtobufVarint32FrameDecoder());
-          p.addLast(new ProtobufDecoder(RaftNettyServerReplyProto.getDefaultInstance()));
-          p.addLast(new ProtobufVarint32LengthFieldPrepender());
-          p.addLast(new ProtobufEncoder());
-
-          p.addLast(inboundHandler);
-        }
-      };
-
-      client.connect(peer.getAddress(), group, initializer);
-    }
-
-    synchronized ChannelFuture offer(RaftNettyServerRequestProto request,
-        CompletableFuture<RaftNettyServerReplyProto> reply) {
-      replies.offer(reply);
-      return client.writeAndFlush(request);
-    }
-
-    synchronized CompletableFuture<RaftNettyServerReplyProto> pollReply() {
-      return replies.poll();
-    }
-
-    @Override
-    public synchronized void close() {
-      client.close();
-      if (!replies.isEmpty()) {
-        final IOException e = new IOException("Connection to " + peer + " is closed.");
-        replies.stream().forEach(f -> f.completeExceptionally(e));
-        replies.clear();
-      }
-    }
-  }
-
-  private final RaftPeer peer;
-  private final Connection connection;
-
-  public NettyRpcProxy(RaftPeer peer, EventLoopGroup group) throws InterruptedException {
-    this.peer = peer;
-    this.connection = new Connection(group);
-  }
-
-  @Override
-  public void close() {
-    connection.close();
-  }
-
-  public RaftNettyServerReplyProto send(
-      RaftRpcRequestProto request, RaftNettyServerRequestProto proto)
-      throws IOException {
-    final CompletableFuture<RaftNettyServerReplyProto> reply = new CompletableFuture<>();
-    final ChannelFuture channelFuture = connection.offer(proto, reply);
-
-    try {
-      channelFuture.sync();
-      return reply.get();
-    } catch (InterruptedException e) {
-      throw RaftUtils.toInterruptedIOException(ProtoUtils.toString(request)
-          + " sending from " + peer + " is interrupted.", e);
-    } catch (ExecutionException e) {
-      throw RaftUtils.toIOException(e);
-    }
-  }
-}

http://git-wip-us.apache.org/repos/asf/incubator-ratis/blob/7e71a2e0/raft-netty/src/main/java/org/apache/raft/netty/client/NettyClientRequestSender.java
----------------------------------------------------------------------
diff --git a/raft-netty/src/main/java/org/apache/raft/netty/client/NettyClientRequestSender.java b/raft-netty/src/main/java/org/apache/raft/netty/client/NettyClientRequestSender.java
deleted file mode 100644
index ba58b75..0000000
--- a/raft-netty/src/main/java/org/apache/raft/netty/client/NettyClientRequestSender.java
+++ /dev/null
@@ -1,71 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- *  with the License.  You may obtain a copy of the License at
- *
- *      http://www.apache.org/licenses/LICENSE-2.0
- *
- *  Unless required by applicable law or agreed to in writing, software
- *  distributed under the License is distributed on an "AS IS" BASIS,
- *  WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- *  See the License for the specific language governing permissions and
- *  limitations under the License.
- */
-package org.apache.raft.netty.client;
-
-import org.apache.raft.client.impl.ClientProtoUtils;
-import org.apache.raft.client.RaftClientRequestSender;
-import org.apache.raft.netty.NettyRpcProxy;
-import org.apache.raft.protocol.RaftClientReply;
-import org.apache.raft.protocol.RaftClientRequest;
-import org.apache.raft.protocol.RaftPeer;
-import org.apache.raft.protocol.SetConfigurationRequest;
-import org.apache.raft.shaded.proto.RaftProtos.RaftClientRequestProto;
-import org.apache.raft.shaded.proto.RaftProtos.RaftRpcRequestProto;
-import org.apache.raft.shaded.proto.RaftProtos.SetConfigurationRequestProto;
-import org.apache.raft.shaded.proto.netty.NettyProtos.RaftNettyServerRequestProto;
-
-import java.io.IOException;
-
-public class NettyClientRequestSender implements RaftClientRequestSender {
-  private final NettyRpcProxy.PeerMap proxies = new NettyRpcProxy.PeerMap();
-
-  public NettyClientRequestSender(Iterable<RaftPeer> servers) {
-    addServers(servers);
-  }
-
-  @Override
-  public RaftClientReply sendRequest(RaftClientRequest request) throws IOException {
-    final String serverId = request.getReplierId();
-    final NettyRpcProxy proxy = proxies.getProxy(serverId);
-
-    final RaftNettyServerRequestProto.Builder b = RaftNettyServerRequestProto.newBuilder();
-    final RaftRpcRequestProto rpcRequest;
-    if (request instanceof SetConfigurationRequest) {
-      final SetConfigurationRequestProto proto = ClientProtoUtils.toSetConfigurationRequestProto(
-          (SetConfigurationRequest)request);
-      b.setSetConfigurationRequest(proto);
-      rpcRequest = proto.getRpcRequest();
-    } else {
-      final RaftClientRequestProto proto = ClientProtoUtils.toRaftClientRequestProto(request);
-      b.setRaftClientRequest(proto);
-      rpcRequest = proto.getRpcRequest();
-    }
-    return ClientProtoUtils.toRaftClientReply(
-        proxy.send(rpcRequest, b.build()).getRaftClientReply());
-  }
-
-  @Override
-  public void addServers(Iterable<RaftPeer> servers) {
-    proxies.addPeers(servers);
-  }
-
-  @Override
-  public void close() {
-    proxies.close();
-  }
-}

http://git-wip-us.apache.org/repos/asf/incubator-ratis/blob/7e71a2e0/raft-netty/src/main/java/org/apache/raft/netty/server/NettyRpcService.java
----------------------------------------------------------------------
diff --git a/raft-netty/src/main/java/org/apache/raft/netty/server/NettyRpcService.java b/raft-netty/src/main/java/org/apache/raft/netty/server/NettyRpcService.java
deleted file mode 100644
index 50833fb..0000000
--- a/raft-netty/src/main/java/org/apache/raft/netty/server/NettyRpcService.java
+++ /dev/null
@@ -1,253 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- *  with the License.  You may obtain a copy of the License at
- *
- *      http://www.apache.org/licenses/LICENSE-2.0
- *
- *  Unless required by applicable law or agreed to in writing, software
- *  distributed under the License is distributed on an "AS IS" BASIS,
- *  WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- *  See the License for the specific language governing permissions and
- *  limitations under the License.
- */
-package org.apache.raft.netty.server;
-
-import com.google.common.base.Preconditions;
-import io.netty.bootstrap.ServerBootstrap;
-import io.netty.channel.*;
-import io.netty.channel.nio.NioEventLoopGroup;
-import io.netty.channel.socket.SocketChannel;
-import io.netty.channel.socket.nio.NioServerSocketChannel;
-import io.netty.handler.logging.LogLevel;
-import io.netty.handler.logging.LoggingHandler;
-import org.apache.raft.client.impl.ClientProtoUtils;
-import org.apache.raft.netty.NettyRpcProxy;
-import org.apache.raft.protocol.RaftClientReply;
-import org.apache.raft.protocol.RaftPeer;
-import org.apache.raft.server.RaftServer;
-import org.apache.raft.server.RaftServerRpc;
-import org.apache.raft.shaded.io.netty.handler.codec.protobuf.ProtobufDecoder;
-import org.apache.raft.shaded.io.netty.handler.codec.protobuf.ProtobufEncoder;
-import org.apache.raft.shaded.io.netty.handler.codec.protobuf.ProtobufVarint32FrameDecoder;
-import org.apache.raft.shaded.io.netty.handler.codec.protobuf.ProtobufVarint32LengthFieldPrepender;
-import org.apache.raft.shaded.proto.RaftProtos.*;
-import org.apache.raft.shaded.proto.netty.NettyProtos.RaftNettyExceptionReplyProto;
-import org.apache.raft.shaded.proto.netty.NettyProtos.RaftNettyServerReplyProto;
-import org.apache.raft.shaded.proto.netty.NettyProtos.RaftNettyServerRequestProto;
-import org.apache.raft.util.CodeInjectionForTesting;
-import org.apache.raft.util.LifeCycle;
-import org.apache.raft.util.ProtoUtils;
-
-import java.io.IOException;
-import java.net.InetSocketAddress;
-import java.nio.channels.ClosedChannelException;
-
-/**
- * A netty server endpoint that acts as the communication layer.
- */
-public final class NettyRpcService implements RaftServerRpc {
-  static final String CLASS_NAME = NettyRpcService.class.getSimpleName();
-  public static final String SEND_SERVER_REQUEST = CLASS_NAME + ".sendServerRequest";
-
-  private final LifeCycle lifeCycle = new LifeCycle(getClass().getSimpleName());
-  private final RaftServer server;
-  private final String id;
-
-  private final EventLoopGroup bossGroup = new NioEventLoopGroup();
-  private final EventLoopGroup workerGroup = new NioEventLoopGroup();
-  private final ChannelFuture channelFuture;
-
-  private final NettyRpcProxy.PeerMap proxies = new NettyRpcProxy.PeerMap();
-
-  @ChannelHandler.Sharable
-  class InboundHandler extends SimpleChannelInboundHandler<RaftNettyServerRequestProto> {
-    @Override
-    protected void channelRead0(ChannelHandlerContext ctx, RaftNettyServerRequestProto proto) {
-      final RaftNettyServerReplyProto reply = handle(proto);
-      ctx.writeAndFlush(reply);
-    }
-  }
-
-  /** Constructs a netty server with the given port. */
-  public NettyRpcService(int port, RaftServer server) {
-    this.server = server;
-    this.id = server.getId();
-
-    final ChannelInitializer<SocketChannel> initializer
-        = new ChannelInitializer<SocketChannel>() {
-      @Override
-      protected void initChannel(SocketChannel ch) throws Exception {
-        final ChannelPipeline p = ch.pipeline();
-
-        p.addLast(new ProtobufVarint32FrameDecoder());
-        p.addLast(new ProtobufDecoder(RaftNettyServerRequestProto.getDefaultInstance()));
-        p.addLast(new ProtobufVarint32LengthFieldPrepender());
-        p.addLast(new ProtobufEncoder());
-
-        p.addLast(new InboundHandler());
-      }
-    };
-
-    channelFuture = new ServerBootstrap()
-        .group(bossGroup, workerGroup)
-        .channel(NioServerSocketChannel.class)
-        .handler(new LoggingHandler(LogLevel.INFO))
-        .childHandler(initializer)
-        .bind(port);
-  }
-
-  private Channel getChannel() {
-    return channelFuture.awaitUninterruptibly().channel();
-  }
-
-  @Override
-  public void start() {
-    lifeCycle.startAndTransition(() -> channelFuture.syncUninterruptibly());
-  }
-
-  @Override
-  public void close() {
-    lifeCycle.checkStateAndClose(() -> {
-      bossGroup.shutdownGracefully();
-      workerGroup.shutdownGracefully();
-      final ChannelFuture f = getChannel().close();
-      proxies.close();
-      f.syncUninterruptibly();
-    });
-  }
-
-  @Override
-  public InetSocketAddress getInetSocketAddress() {
-    return (InetSocketAddress)getChannel().localAddress();
-  }
-
-  RaftNettyServerReplyProto handle(RaftNettyServerRequestProto proto) {
-    RaftRpcRequestProto rpcRequest = null;
-    try {
-      switch (proto.getRaftNettyServerRequestCase()) {
-        case REQUESTVOTEREQUEST: {
-          final RequestVoteRequestProto request = proto.getRequestVoteRequest();
-          rpcRequest = request.getServerRequest();
-          final RequestVoteReplyProto reply = server.requestVote(request);
-          return RaftNettyServerReplyProto.newBuilder()
-              .setRequestVoteReply(reply)
-              .build();
-        }
-        case APPENDENTRIESREQUEST: {
-          final AppendEntriesRequestProto request = proto.getAppendEntriesRequest();
-          rpcRequest = request.getServerRequest();
-          final AppendEntriesReplyProto reply = server.appendEntries(request);
-          return RaftNettyServerReplyProto.newBuilder()
-              .setAppendEntriesReply(reply)
-              .build();
-        }
-        case INSTALLSNAPSHOTREQUEST: {
-          final InstallSnapshotRequestProto request = proto.getInstallSnapshotRequest();
-          rpcRequest = request.getServerRequest();
-          final InstallSnapshotReplyProto reply = server.installSnapshot(request);
-          return RaftNettyServerReplyProto.newBuilder()
-              .setInstallSnapshotReply(reply)
-              .build();
-        }
-        case RAFTCLIENTREQUEST: {
-          final RaftClientRequestProto request = proto.getRaftClientRequest();
-          rpcRequest = request.getRpcRequest();
-          final RaftClientReply reply = server.submitClientRequest(
-              ClientProtoUtils.toRaftClientRequest(request));
-          return RaftNettyServerReplyProto.newBuilder()
-              .setRaftClientReply(ClientProtoUtils.toRaftClientReplyProto(reply))
-              .build();
-        }
-        case SETCONFIGURATIONREQUEST: {
-          final SetConfigurationRequestProto request = proto.getSetConfigurationRequest();
-          rpcRequest = request.getRpcRequest();
-          final RaftClientReply reply = server.setConfiguration(
-              ClientProtoUtils.toSetConfigurationRequest(request));
-          return RaftNettyServerReplyProto.newBuilder()
-              .setRaftClientReply(ClientProtoUtils.toRaftClientReplyProto(reply))
-              .build();
-        }
-        case RAFTNETTYSERVERREQUEST_NOT_SET:
-          throw new IllegalArgumentException("Request case not set in proto: "
-              + proto.getRaftNettyServerRequestCase());
-        default:
-          throw new UnsupportedOperationException("Request case not supported: "
-              + proto.getRaftNettyServerRequestCase());
-      }
-    } catch (IOException ioe) {
-      Preconditions.checkNotNull(rpcRequest);
-      return toRaftNettyServerReplyProto(rpcRequest, ioe);
-    }
-  }
-
-  private static RaftNettyServerReplyProto toRaftNettyServerReplyProto(
-      RaftRpcRequestProto request, IOException e) {
-    final RaftRpcReplyProto.Builder rpcReply = ClientProtoUtils.toRaftRpcReplyProtoBuilder(
-        request.getRequestorId(),
-        request.getReplyId(),
-        request.getSeqNum(), false);
-    final RaftNettyExceptionReplyProto.Builder ioe = RaftNettyExceptionReplyProto.newBuilder()
-        .setRpcReply(rpcReply)
-        .setException(ProtoUtils.toByteString(e));
-    return RaftNettyServerReplyProto.newBuilder().setExceptionReply(ioe).build();
-  }
-
-  @Override
-  public RequestVoteReplyProto requestVote(RequestVoteRequestProto request) throws IOException {
-    Preconditions.checkArgument(id.equals(request.getServerRequest().getRequestorId()));
-    CodeInjectionForTesting.execute(SEND_SERVER_REQUEST, id, null, request);
-
-    final RaftNettyServerRequestProto proto = RaftNettyServerRequestProto.newBuilder()
-        .setRequestVoteRequest(request)
-        .build();
-    final RaftRpcRequestProto serverRequest = request.getServerRequest();
-    return sendRaftNettyServerRequestProto(serverRequest, proto).getRequestVoteReply();
-  }
-
-  @Override
-  public AppendEntriesReplyProto appendEntries(AppendEntriesRequestProto request) throws IOException {
-    Preconditions.checkArgument(id.equals(request.getServerRequest().getRequestorId()));
-    CodeInjectionForTesting.execute(SEND_SERVER_REQUEST, id, null, request);
-
-    final RaftNettyServerRequestProto proto = RaftNettyServerRequestProto.newBuilder()
-        .setAppendEntriesRequest(request)
-        .build();
-    final RaftRpcRequestProto serverRequest = request.getServerRequest();
-    return sendRaftNettyServerRequestProto(serverRequest, proto).getAppendEntriesReply();
-  }
-
-  @Override
-  public InstallSnapshotReplyProto installSnapshot(InstallSnapshotRequestProto request) throws IOException {
-    Preconditions.checkArgument(id.equals(request.getServerRequest().getRequestorId()));
-    CodeInjectionForTesting.execute(SEND_SERVER_REQUEST, id, null, request);
-
-    final RaftNettyServerRequestProto proto = RaftNettyServerRequestProto.newBuilder()
-        .setInstallSnapshotRequest(request)
-        .build();
-    final RaftRpcRequestProto serverRequest = request.getServerRequest();
-    return sendRaftNettyServerRequestProto(serverRequest, proto).getInstallSnapshotReply();
-  }
-
-  private RaftNettyServerReplyProto sendRaftNettyServerRequestProto(
-      RaftRpcRequestProto request, RaftNettyServerRequestProto proto)
-      throws IOException {
-    final String id = request.getReplyId();
-    final NettyRpcProxy p = proxies.getProxy(id);
-    try {
-      return p.send(request, proto);
-    } catch (ClosedChannelException cce) {
-      proxies.resetProxy(id);
-      throw cce;
-    }
-  }
-
-  @Override
-  public void addPeers(Iterable<RaftPeer> peers) {
-    proxies.addPeers(peers);
-  }
-}

http://git-wip-us.apache.org/repos/asf/incubator-ratis/blob/7e71a2e0/raft-netty/src/test/java/org/apache/raft/netty/MiniRaftClusterWithNetty.java
----------------------------------------------------------------------
diff --git a/raft-netty/src/test/java/org/apache/raft/netty/MiniRaftClusterWithNetty.java b/raft-netty/src/test/java/org/apache/raft/netty/MiniRaftClusterWithNetty.java
deleted file mode 100644
index 12cdf13..0000000
--- a/raft-netty/src/test/java/org/apache/raft/netty/MiniRaftClusterWithNetty.java
+++ /dev/null
@@ -1,117 +0,0 @@
-/**
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- *     http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-package org.apache.raft.netty;
-
-import org.apache.raft.MiniRaftCluster;
-import org.apache.raft.RaftTestUtil;
-import org.apache.raft.client.RaftClientRequestSender;
-import org.apache.raft.conf.RaftProperties;
-import org.apache.raft.netty.client.NettyClientRequestSender;
-import org.apache.raft.netty.server.NettyRpcService;
-import org.apache.raft.protocol.RaftPeer;
-import org.apache.raft.server.impl.DelayLocalExecutionInjection;
-import org.apache.raft.server.impl.RaftConfiguration;
-import org.apache.raft.server.impl.RaftServerImpl;
-import org.apache.raft.util.NetUtils;
-
-import java.io.IOException;
-import java.util.Collection;
-import java.util.HashMap;
-import java.util.Map;
-
-public class MiniRaftClusterWithNetty extends MiniRaftCluster.RpcBase {
-  public static final Factory<MiniRaftClusterWithNetty> FACTORY
-      = new Factory<MiniRaftClusterWithNetty>() {
-    @Override
-    public MiniRaftClusterWithNetty newCluster(
-        String[] ids, RaftProperties prop, boolean formatted) {
-      return new MiniRaftClusterWithNetty(ids, prop, formatted);
-    }
-  };
-
-  public static final DelayLocalExecutionInjection sendServerRequest
-      = new DelayLocalExecutionInjection(NettyRpcService.SEND_SERVER_REQUEST);
-
-  public MiniRaftClusterWithNetty(int numServers, RaftProperties properties) {
-    this(generateIds(numServers, 0), properties, true);
-  }
-
-  public MiniRaftClusterWithNetty(
-      String[] ids, RaftProperties properties, boolean formatted) {
-    super(ids, properties, formatted);
-    init(initRpcServices(getServers(), getConf()));
-  }
-
-  private static String getAddress(String id, RaftConfiguration conf) {
-    final RaftPeer peer = conf.getPeer(id);
-    if (peer != null) {
-      final String address = peer.getAddress();
-      if (address != null) {
-        return address;
-      }
-    }
-    return "0.0.0.0:0";
-  }
-
-  private static NettyRpcService newNettyRpcService(
-      RaftServerImpl s, RaftConfiguration conf) {
-    final String address = getAddress(s.getId(), conf);
-    final int port = NetUtils.newInetSocketAddress(address).getPort();
-    return new NettyRpcService(port, s);
-  }
-
-  private static Map<RaftPeer, NettyRpcService> initRpcServices(
-      Collection<RaftServerImpl> servers, RaftConfiguration conf) {
-    final Map<RaftPeer, NettyRpcService> peerRpcs = new HashMap<>();
-
-    for (RaftServerImpl s : servers) {
-      final NettyRpcService rpc = newNettyRpcService(s, conf);
-      peerRpcs.put(new RaftPeer(s.getId(), rpc.getInetSocketAddress()), rpc);
-    }
-
-    return peerRpcs;
-  }
-
-  @Override
-  protected RaftServerImpl setPeerRpc(RaftPeer peer) throws IOException {
-    final RaftServerImpl s = servers.get(peer.getId());
-    final NettyRpcService rpc = newNettyRpcService(s, conf);
-    s.setServerRpc(rpc);
-    return s;
-  }
-
-  @Override
-  protected Collection<RaftPeer> addNewPeers(
-      Collection<RaftPeer> newPeers, Collection<RaftServerImpl> newServers,
-      boolean startService) throws IOException {
-    return addNewPeers(initRpcServices(newServers, conf),
-        newServers, startService);
-  }
-
-  @Override
-  public RaftClientRequestSender getRaftClientRequestSender() {
-    return new NettyClientRequestSender(getPeers());
-  }
-
-  @Override
-  protected void blockQueueAndSetDelay(String leaderId, int delayMs)
-      throws InterruptedException {
-    RaftTestUtil.blockQueueAndSetDelay(getServers(), sendServerRequest,
-        leaderId, delayMs, getMaxTimeout());
-  }
-}

http://git-wip-us.apache.org/repos/asf/incubator-ratis/blob/7e71a2e0/raft-netty/src/test/java/org/apache/raft/netty/TestNotLeaderExceptionWithNetty.java
----------------------------------------------------------------------
diff --git a/raft-netty/src/test/java/org/apache/raft/netty/TestNotLeaderExceptionWithNetty.java b/raft-netty/src/test/java/org/apache/raft/netty/TestNotLeaderExceptionWithNetty.java
deleted file mode 100644
index cfc7c01..0000000
--- a/raft-netty/src/test/java/org/apache/raft/netty/TestNotLeaderExceptionWithNetty.java
+++ /dev/null
@@ -1,33 +0,0 @@
-/**
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- *     http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-package org.apache.raft.netty;
-
-import org.apache.raft.MiniRaftCluster;
-import org.apache.raft.RaftNotLeaderExceptionBaseTest;
-import org.apache.raft.conf.RaftProperties;
-
-import java.io.IOException;
-
-public class TestNotLeaderExceptionWithNetty extends RaftNotLeaderExceptionBaseTest {
-  @Override
-  public MiniRaftCluster initCluster() throws IOException {
-    String[] s = MiniRaftCluster.generateIds(NUM_PEERS, 0);
-    RaftProperties prop = new RaftProperties();
-    return new MiniRaftClusterWithNetty(s, prop, true);
-  }
-}

http://git-wip-us.apache.org/repos/asf/incubator-ratis/blob/7e71a2e0/raft-netty/src/test/java/org/apache/raft/netty/TestRaftReconfigurationWithNetty.java
----------------------------------------------------------------------
diff --git a/raft-netty/src/test/java/org/apache/raft/netty/TestRaftReconfigurationWithNetty.java b/raft-netty/src/test/java/org/apache/raft/netty/TestRaftReconfigurationWithNetty.java
deleted file mode 100644
index c4dd914..0000000
--- a/raft-netty/src/test/java/org/apache/raft/netty/TestRaftReconfigurationWithNetty.java
+++ /dev/null
@@ -1,31 +0,0 @@
-/**
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- *     http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-package org.apache.raft.netty;
-
-import org.apache.raft.MiniRaftCluster;
-import org.apache.raft.server.impl.RaftReconfigurationBaseTest;
-
-import java.io.IOException;
-
-public class TestRaftReconfigurationWithNetty
-    extends RaftReconfigurationBaseTest {
-  @Override
-  public MiniRaftCluster getCluster(int peerNum) throws IOException {
-    return MiniRaftClusterWithNetty.FACTORY.newCluster(peerNum, prop, true);
-  }
-}

http://git-wip-us.apache.org/repos/asf/incubator-ratis/blob/7e71a2e0/raft-netty/src/test/java/org/apache/raft/netty/TestRaftSnapshotWithNetty.java
----------------------------------------------------------------------
diff --git a/raft-netty/src/test/java/org/apache/raft/netty/TestRaftSnapshotWithNetty.java b/raft-netty/src/test/java/org/apache/raft/netty/TestRaftSnapshotWithNetty.java
deleted file mode 100644
index 269ebd7..0000000
--- a/raft-netty/src/test/java/org/apache/raft/netty/TestRaftSnapshotWithNetty.java
+++ /dev/null
@@ -1,32 +0,0 @@
-/**
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- *     http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-package org.apache.raft.netty;
-
-import org.apache.raft.MiniRaftCluster;
-import org.apache.raft.conf.RaftProperties;
-import org.apache.raft.statemachine.RaftSnapshotBaseTest;
-
-import java.io.IOException;
-
-public class TestRaftSnapshotWithNetty extends RaftSnapshotBaseTest {
-  @Override
-  public MiniRaftCluster initCluster(int numServer, RaftProperties prop)
-      throws IOException {
-    return MiniRaftClusterWithNetty.FACTORY.newCluster(numServer, prop, true);
-  }
-}

http://git-wip-us.apache.org/repos/asf/incubator-ratis/blob/7e71a2e0/raft-netty/src/test/java/org/apache/raft/netty/TestRaftWithNetty.java
----------------------------------------------------------------------
diff --git a/raft-netty/src/test/java/org/apache/raft/netty/TestRaftWithNetty.java b/raft-netty/src/test/java/org/apache/raft/netty/TestRaftWithNetty.java
deleted file mode 100644
index cba991d..0000000
--- a/raft-netty/src/test/java/org/apache/raft/netty/TestRaftWithNetty.java
+++ /dev/null
@@ -1,62 +0,0 @@
-/**
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- *     http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-package org.apache.raft.netty;
-
-import org.apache.log4j.Level;
-import org.apache.raft.RaftBasicTests;
-import org.apache.raft.client.RaftClient;
-import org.apache.raft.server.impl.BlockRequestHandlingInjection;
-import org.apache.raft.server.impl.RaftServerImpl;
-import org.apache.raft.util.RaftUtils;
-import org.junit.Test;
-
-import java.io.IOException;
-
-public class TestRaftWithNetty extends RaftBasicTests {
-  static {
-    RaftUtils.setLogLevel(RaftServerImpl.LOG, Level.DEBUG);
-    RaftUtils.setLogLevel(RaftClient.LOG, Level.DEBUG);
-  }
-
-  private final MiniRaftClusterWithNetty cluster;
-
-  public TestRaftWithNetty() throws IOException {
-    cluster = new MiniRaftClusterWithNetty(NUM_SERVERS, getProperties());
-  }
-
-  @Override
-  public MiniRaftClusterWithNetty getCluster() {
-    return cluster;
-  }
-
-  @Override
-  @Test
-  public void testEnforceLeader() throws Exception {
-    super.testEnforceLeader();
-
-    MiniRaftClusterWithNetty.sendServerRequest.clear();
-    BlockRequestHandlingInjection.getInstance().unblockAll();
-  }
-
-  @Override
-  @Test
-  public void testWithLoad() throws Exception {
-    super.testWithLoad();
-    BlockRequestHandlingInjection.getInstance().unblockAll();
-  }
-}

http://git-wip-us.apache.org/repos/asf/incubator-ratis/blob/7e71a2e0/raft-netty/src/test/resources/log4j.properties
----------------------------------------------------------------------
diff --git a/raft-netty/src/test/resources/log4j.properties b/raft-netty/src/test/resources/log4j.properties
deleted file mode 100644
index ced0687..0000000
--- a/raft-netty/src/test/resources/log4j.properties
+++ /dev/null
@@ -1,18 +0,0 @@
-#   Licensed 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.
-# log4j configuration used during build and unit tests
-
-log4j.rootLogger=info,stdout
-log4j.threshold=ALL
-log4j.appender.stdout=org.apache.log4j.ConsoleAppender
-log4j.appender.stdout.layout=org.apache.log4j.PatternLayout
-log4j.appender.stdout.layout.ConversionPattern=%d{ISO8601} %-5p %c{2} (%F:%M(%L)) - %m%n

http://git-wip-us.apache.org/repos/asf/incubator-ratis/blob/7e71a2e0/raft-project-dist/pom.xml
----------------------------------------------------------------------
diff --git a/raft-project-dist/pom.xml b/raft-project-dist/pom.xml
deleted file mode 100644
index d979285..0000000
--- a/raft-project-dist/pom.xml
+++ /dev/null
@@ -1,169 +0,0 @@
-<?xml version="1.0" encoding="UTF-8"?>
-<!--
-  Licensed 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. See accompanying LICENSE file.
--->
-<project xmlns="http://maven.apache.org/POM/4.0.0"
-         xmlns:xsi="http://www.w3.org/2001/XMLSchema-instance"
-         xsi:schemaLocation="http://maven.apache.org/POM/4.0.0 http://maven.apache.org/xsd/maven-4.0.0.xsd">
-  <modelVersion>4.0.0</modelVersion>
-  <parent>
-    <artifactId>raft-project</artifactId>
-    <groupId>com.hortonworks.raft</groupId>
-    <version>1.0-SNAPSHOT</version>
-    <relativePath>../raft-project</relativePath>
-  </parent>
-
-  <artifactId>raft-project-dist</artifactId>
-  <groupId>com.hortonworks.raft</groupId>
-  <name>Raft Project Dist POM</name>
-  <version>1.0-SNAPSHOT</version>
-
-  <packaging>pom</packaging>
-
-  <properties>
-    <raft.tmp.dir>${project.build.directory}/test</raft.tmp.dir>
-    <test.build.data>${project.build.directory}/test/data</test.build.data>
-    <raft.log.dir>${project.build.directory}/log</raft.log.dir>
-    <test.build.webapps>${project.build.directory}/test-classes/webapps</test.build.webapps>
-    <test.cache.data>${project.build.directory}/test-classes</test.cache.data>
-    <test.build.classes>${project.build.directory}/test-classes</test.build.classes>
-  </properties>
-
-  <build>
-    <plugins>
-      <plugin>
-        <groupId>org.apache.maven.plugins</groupId>
-        <artifactId>maven-jar-plugin</artifactId>
-        <executions>
-          <execution>
-            <id>default-jar</id>
-            <phase>never</phase>
-          </execution>
-          <execution>
-            <id>prepare-jar</id>
-            <phase>prepare-package</phase>
-            <goals>
-              <goal>jar</goal>
-            </goals>
-          </execution>
-          <execution>
-            <id>prepare-test-jar</id>
-            <phase>prepare-package</phase>
-            <goals>
-              <goal>test-jar</goal>
-            </goals>
-            <configuration>
-              <includes>
-                <include>**/*.class</include>
-                <include>webapps/**</include>
-              </includes>
-            </configuration>
-          </execution>
-        </executions>
-      </plugin>
-
-      <plugin>
-        <groupId>org.apache.maven.plugins</groupId>
-        <artifactId>maven-source-plugin</artifactId>
-        <version>${maven-source-plugin.version}</version>
-        <executions>
-          <execution>
-            <phase>prepare-package</phase>
-            <goals>
-              <goal>jar</goal>
-              <goal>test-jar</goal>
-            </goals>
-          </execution>
-        </executions>
-        <configuration>
-          <attach>true</attach>
-        </configuration>
-      </plugin>
-      <plugin>
-        <groupId>org.codehaus.mojo</groupId>
-        <artifactId>findbugs-maven-plugin</artifactId>
-        <configuration>
-          <excludeFilterFile>${basedir}/dev-support/findbugsExcludeFile.xml</excludeFilterFile>
-          <fork>true</fork>
-          <maxHeap>2048</maxHeap>
-        </configuration>
-      </plugin>
-    </plugins>
-  </build>
-
-  <profiles>
-    <profile>
-      <id>dist</id>
-      <activation>
-        <activeByDefault>false</activeByDefault>
-      </activation>
-      <build>
-        <plugins>
-          <plugin>
-            <groupId>org.apache.maven.plugins</groupId>
-            <artifactId>maven-antrun-plugin</artifactId>
-            <executions>
-              <execution>
-                <id>tar</id>
-                <phase>package</phase>
-                <goals>
-                  <goal>run</goal>
-                </goals>
-                <configuration>
-                  <target if="tar">
-                    <!-- Using Unix script to preserve symlinks -->
-                    <echo file="${project.build.directory}/dist-maketar.sh">
-                      cd "${project.build.directory}"
-                      tar cf - ${project.artifactId}-${project.version} | gzip > ${project.artifactId}-${project.version}.tar.gz
-                    </echo>
-                    <exec executable="${shell-executable}" dir="${project.build.directory}" failonerror="true">
-                      <arg line="./dist-maketar.sh"/>
-                    </exec>
-                  </target>
-                </configuration>
-              </execution>
-            </executions>
-          </plugin>
-          <plugin>
-            <groupId>org.apache.maven.plugins</groupId>
-            <artifactId>maven-assembly-plugin</artifactId>
-            <dependencies>
-              <dependency>
-                <groupId>org.apache.hadoop</groupId>
-                <artifactId>hadoop-assemblies</artifactId>
-                <version>${project.version}</version>
-              </dependency>
-            </dependencies>
-            <executions>
-              <execution>
-                <id>dist</id>
-                <phase>prepare-package</phase>
-                <goals>
-                  <goal>single</goal>
-                </goals>
-                <configuration>
-                  <appendAssemblyId>false</appendAssemblyId>
-                  <attach>false</attach>
-                  <finalName>${project.artifactId}-${project.version}</finalName>
-                  <descriptorRefs>
-                    <descriptorRef>hadoop-dist</descriptorRef>
-                  </descriptorRefs>
-                </configuration>
-              </execution>
-            </executions>
-          </plugin>
-        </plugins>
-      </build>
-    </profile>
-  </profiles>
-</project>



[14/54] [abbrv] incubator-ratis git commit: Refactor SnapshotInfo and its implementations.

Posted by ji...@apache.org.
Refactor SnapshotInfo and its implementations.


Project: http://git-wip-us.apache.org/repos/asf/incubator-ratis/repo
Commit: http://git-wip-us.apache.org/repos/asf/incubator-ratis/commit/f58b5ef8
Tree: http://git-wip-us.apache.org/repos/asf/incubator-ratis/tree/f58b5ef8
Diff: http://git-wip-us.apache.org/repos/asf/incubator-ratis/diff/f58b5ef8

Branch: refs/heads/master
Commit: f58b5ef8f8f96ad640ff1b6b2d707f21712ec87a
Parents: 5946058
Author: Tsz-Wo Nicholas Sze <sz...@hortonworks.com>
Authored: Fri Jan 6 16:03:30 2017 +0800
Committer: Tsz-Wo Nicholas Sze <sz...@hortonworks.com>
Committed: Fri Jan 6 16:03:30 2017 +0800

----------------------------------------------------------------------
 .../arithmetic/ArithmeticStateMachine.java      |  1 -
 .../apache/raft/server/storage/FileInfo.java    | 11 +++-
 .../raft/statemachine/FileListSnapshotInfo.java | 64 ++++++++++++++++++
 .../statemachine/SimpleStateMachineStorage.java | 18 +-----
 .../statemachine/SingleFileSnapshotInfo.java    | 38 +++++++++++
 .../apache/raft/statemachine/SnapshotInfo.java  |  7 --
 .../raft/statemachine/SnapshotInfoImpl.java     | 68 --------------------
 .../SimpleStateMachine4Testing.java             |  1 -
 8 files changed, 113 insertions(+), 95 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-ratis/blob/f58b5ef8/raft-examples/src/main/java/org/apache/raft/examples/arithmetic/ArithmeticStateMachine.java
----------------------------------------------------------------------
diff --git a/raft-examples/src/main/java/org/apache/raft/examples/arithmetic/ArithmeticStateMachine.java b/raft-examples/src/main/java/org/apache/raft/examples/arithmetic/ArithmeticStateMachine.java
index 53b6122..5112e14 100644
--- a/raft-examples/src/main/java/org/apache/raft/examples/arithmetic/ArithmeticStateMachine.java
+++ b/raft-examples/src/main/java/org/apache/raft/examples/arithmetic/ArithmeticStateMachine.java
@@ -28,7 +28,6 @@ import org.apache.raft.server.protocol.TermIndex;
 import org.apache.raft.server.storage.RaftStorage;
 import org.apache.raft.shaded.proto.RaftProtos.LogEntryProto;
 import org.apache.raft.statemachine.*;
-import org.apache.raft.statemachine.SimpleStateMachineStorage.SingleFileSnapshotInfo;
 import org.apache.raft.util.AutoCloseableLock;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;

http://git-wip-us.apache.org/repos/asf/incubator-ratis/blob/f58b5ef8/raft-server/src/main/java/org/apache/raft/server/storage/FileInfo.java
----------------------------------------------------------------------
diff --git a/raft-server/src/main/java/org/apache/raft/server/storage/FileInfo.java b/raft-server/src/main/java/org/apache/raft/server/storage/FileInfo.java
index 78e0301..cdeb622 100644
--- a/raft-server/src/main/java/org/apache/raft/server/storage/FileInfo.java
+++ b/raft-server/src/main/java/org/apache/raft/server/storage/FileInfo.java
@@ -23,11 +23,13 @@ import java.nio.file.Path;
 
 /**
  * Metadata about a file.
+ *
+ * The objects of this class are immutable.
  */
 public class FileInfo {
-  private Path path;
-  private MD5Hash fileDigest;
-  private long fileSize;
+  private final Path path;
+  private final MD5Hash fileDigest;
+  private final long fileSize;
 
   public FileInfo(Path path, MD5Hash fileDigest) {
     this.path = path;
@@ -40,14 +42,17 @@ public class FileInfo {
     return path.toString();
   }
 
+  /** @return the path of the file. */
   public Path getPath() {
     return path;
   }
 
+  /** @return the MD5 file digest of the file. */
   public MD5Hash getFileDigest() {
     return fileDigest;
   }
 
+  /** @return the size of the file. */
   public long getFileSize() {
     return fileSize;
   }

http://git-wip-us.apache.org/repos/asf/incubator-ratis/blob/f58b5ef8/raft-server/src/main/java/org/apache/raft/statemachine/FileListSnapshotInfo.java
----------------------------------------------------------------------
diff --git a/raft-server/src/main/java/org/apache/raft/statemachine/FileListSnapshotInfo.java b/raft-server/src/main/java/org/apache/raft/statemachine/FileListSnapshotInfo.java
new file mode 100644
index 0000000..b65fc13
--- /dev/null
+++ b/raft-server/src/main/java/org/apache/raft/statemachine/FileListSnapshotInfo.java
@@ -0,0 +1,64 @@
+/**
+ * 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.raft.statemachine;
+
+import org.apache.raft.server.protocol.TermIndex;
+import org.apache.raft.server.storage.FileInfo;
+
+import java.util.Collections;
+import java.util.List;
+
+/**
+ * Each snapshot has a list of files.
+ *
+ * The objects of this class are immutable.
+ */
+public class FileListSnapshotInfo implements SnapshotInfo {
+  private final TermIndex termIndex;
+  private final List<FileInfo> files;
+
+  public FileListSnapshotInfo(List<FileInfo> files, long term, long index) {
+    this.termIndex = TermIndex.newTermIndex(term, index);
+    this.files = Collections.unmodifiableList(files);
+  }
+
+  @Override
+  public TermIndex getTermIndex() {
+    return termIndex;
+  }
+
+  @Override
+  public long getTerm() {
+    return termIndex.getTerm();
+  }
+
+  @Override
+  public long getIndex() {
+    return termIndex.getIndex();
+  }
+
+  @Override
+  public List<FileInfo> getFiles() {
+    return files;
+  }
+
+  @Override
+  public String toString() {
+    return termIndex + ":" + files;
+  }
+}

http://git-wip-us.apache.org/repos/asf/incubator-ratis/blob/f58b5ef8/raft-server/src/main/java/org/apache/raft/statemachine/SimpleStateMachineStorage.java
----------------------------------------------------------------------
diff --git a/raft-server/src/main/java/org/apache/raft/statemachine/SimpleStateMachineStorage.java b/raft-server/src/main/java/org/apache/raft/statemachine/SimpleStateMachineStorage.java
index c317eb8..a779f98 100644
--- a/raft-server/src/main/java/org/apache/raft/statemachine/SimpleStateMachineStorage.java
+++ b/raft-server/src/main/java/org/apache/raft/statemachine/SimpleStateMachineStorage.java
@@ -19,9 +19,7 @@ package org.apache.raft.statemachine;
 
 
 import com.google.common.annotations.VisibleForTesting;
-import com.google.common.collect.Lists;
 import org.apache.raft.io.MD5Hash;
-import org.apache.raft.server.impl.RaftConfiguration;
 import org.apache.raft.server.protocol.TermIndex;
 import org.apache.raft.server.storage.FileInfo;
 import org.apache.raft.server.storage.RaftStorage;
@@ -56,17 +54,6 @@ public class SimpleStateMachineStorage implements StateMachineStorage {
 
   private volatile SingleFileSnapshotInfo currentSnapshot = null;
 
-  public static class SingleFileSnapshotInfo extends SnapshotInfoImpl {
-    protected  SingleFileSnapshotInfo(RaftConfiguration raftConfiguration,
-                                      Path path, MD5Hash fileDigest, long term, long endIndex) {
-      super(raftConfiguration, Lists.newArrayList(new FileInfo(path, fileDigest)), term, endIndex);
-    }
-
-    public FileInfo getFile() {
-      return getFiles().get(0);
-    }
-  }
-
   public void init(RaftStorage raftStorage) throws IOException {
     this.raftStorage = raftStorage;
     this.smDir = raftStorage.getStorageDir().getStateMachineDir();
@@ -120,10 +107,11 @@ public class SimpleStateMachineStorage implements StateMachineStorage {
         Matcher matcher = SNAPSHOT_REGEX.matcher(path.getFileName().toString());
         if (matcher.matches()) {
           final long endIndex = Long.parseLong(matcher.group(2));
-          if (latest == null || endIndex > latest.termIndex.getIndex()) {
+          if (latest == null || endIndex > latest.getIndex()) {
             final long term = Long.parseLong(matcher.group(1));
             MD5Hash fileDigest = MD5FileUtil.readStoredMd5ForFile(path.toFile());
-            latest = new SingleFileSnapshotInfo(null, path, fileDigest, term, endIndex);
+            final FileInfo fileInfo = new FileInfo(path, fileDigest);
+            latest = new SingleFileSnapshotInfo(fileInfo, term, endIndex);
           }
         }
       }

http://git-wip-us.apache.org/repos/asf/incubator-ratis/blob/f58b5ef8/raft-server/src/main/java/org/apache/raft/statemachine/SingleFileSnapshotInfo.java
----------------------------------------------------------------------
diff --git a/raft-server/src/main/java/org/apache/raft/statemachine/SingleFileSnapshotInfo.java b/raft-server/src/main/java/org/apache/raft/statemachine/SingleFileSnapshotInfo.java
new file mode 100644
index 0000000..6b01e17
--- /dev/null
+++ b/raft-server/src/main/java/org/apache/raft/statemachine/SingleFileSnapshotInfo.java
@@ -0,0 +1,38 @@
+/**
+ * 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.raft.statemachine;
+
+import org.apache.raft.server.storage.FileInfo;
+
+import java.util.Arrays;
+
+/**
+ * Each snapshot only has a single file.
+ *
+ * The objects of this class are immutable.
+ */
+public class SingleFileSnapshotInfo extends FileListSnapshotInfo {
+  public SingleFileSnapshotInfo(FileInfo fileInfo, long term, long endIndex) {
+    super(Arrays.asList(fileInfo), term, endIndex);
+  }
+
+  /** @return the file associated with the snapshot. */
+  public FileInfo getFile() {
+    return getFiles().get(0);
+  }
+}

http://git-wip-us.apache.org/repos/asf/incubator-ratis/blob/f58b5ef8/raft-server/src/main/java/org/apache/raft/statemachine/SnapshotInfo.java
----------------------------------------------------------------------
diff --git a/raft-server/src/main/java/org/apache/raft/statemachine/SnapshotInfo.java b/raft-server/src/main/java/org/apache/raft/statemachine/SnapshotInfo.java
index b7deb9a..0fdcbc3 100644
--- a/raft-server/src/main/java/org/apache/raft/statemachine/SnapshotInfo.java
+++ b/raft-server/src/main/java/org/apache/raft/statemachine/SnapshotInfo.java
@@ -17,7 +17,6 @@
  */
 package org.apache.raft.statemachine;
 
-import org.apache.raft.server.impl.RaftConfiguration;
 import org.apache.raft.server.protocol.TermIndex;
 import org.apache.raft.server.storage.FileInfo;
 
@@ -56,10 +55,4 @@ public interface SnapshotInfo {
    * @return a list of Files corresponding to the this snapshot.
    */
   List<FileInfo> getFiles();
-
-  /**
-   * Returns the RaftConfiguration corresponding to this snapshot.
-   * @return the RaftConfiguration corresponding to this snapshot.
-   */
-  RaftConfiguration getRaftConfiguration();
 }

http://git-wip-us.apache.org/repos/asf/incubator-ratis/blob/f58b5ef8/raft-server/src/main/java/org/apache/raft/statemachine/SnapshotInfoImpl.java
----------------------------------------------------------------------
diff --git a/raft-server/src/main/java/org/apache/raft/statemachine/SnapshotInfoImpl.java b/raft-server/src/main/java/org/apache/raft/statemachine/SnapshotInfoImpl.java
deleted file mode 100644
index 1929614..0000000
--- a/raft-server/src/main/java/org/apache/raft/statemachine/SnapshotInfoImpl.java
+++ /dev/null
@@ -1,68 +0,0 @@
-/**
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- *     http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-package org.apache.raft.statemachine;
-
-import org.apache.raft.server.impl.RaftConfiguration;
-import org.apache.raft.server.protocol.TermIndex;
-import org.apache.raft.server.storage.FileInfo;
-
-import java.util.List;
-
-public class SnapshotInfoImpl implements SnapshotInfo {
-
-  protected final RaftConfiguration raftConfiguration;
-  protected final List<FileInfo> files;
-  protected final TermIndex termIndex;
-
-  public SnapshotInfoImpl(RaftConfiguration raftConfiguration,
-                          List<FileInfo> files, long term, long index) {
-    this.raftConfiguration = raftConfiguration;
-    this.files = files;
-    this.termIndex = TermIndex.newTermIndex(term, index);
-  }
-
-  @Override
-  public TermIndex getTermIndex() {
-    return termIndex;
-  }
-
-  @Override
-  public long getTerm() {
-    return termIndex.getTerm();
-  }
-
-  @Override
-  public long getIndex() {
-    return termIndex.getIndex();
-  }
-
-  @Override
-  public List<FileInfo> getFiles() {
-    return files;
-  }
-
-  @Override
-  public RaftConfiguration getRaftConfiguration() {
-    return raftConfiguration;
-  }
-
-  @Override
-  public String toString() {
-    return raftConfiguration + "." + files + "." + termIndex;
-  }
-}

http://git-wip-us.apache.org/repos/asf/incubator-ratis/blob/f58b5ef8/raft-server/src/test/java/org/apache/raft/statemachine/SimpleStateMachine4Testing.java
----------------------------------------------------------------------
diff --git a/raft-server/src/test/java/org/apache/raft/statemachine/SimpleStateMachine4Testing.java b/raft-server/src/test/java/org/apache/raft/statemachine/SimpleStateMachine4Testing.java
index 0709d22..b8dd3f3 100644
--- a/raft-server/src/test/java/org/apache/raft/statemachine/SimpleStateMachine4Testing.java
+++ b/raft-server/src/test/java/org/apache/raft/statemachine/SimpleStateMachine4Testing.java
@@ -34,7 +34,6 @@ import org.apache.raft.server.storage.LogOutputStream;
 import org.apache.raft.server.storage.RaftStorage;
 import org.apache.raft.shaded.proto.RaftProtos.LogEntryProto;
 import org.apache.raft.shaded.proto.RaftProtos.SMLogEntryProto;
-import org.apache.raft.statemachine.SimpleStateMachineStorage.SingleFileSnapshotInfo;
 import org.apache.raft.util.Daemon;
 import org.apache.raft.util.LifeCycle;
 import org.apache.raft.util.MD5FileUtil;


[22/54] [abbrv] incubator-ratis git commit: Renamed the packages from raft to ratis in preperation for Apache Incubation - Moved all java packages from org.apache.raft to org.apache.ratis. - Moved native package to org_apache_ratis, and native lib to l

Posted by ji...@apache.org.
http://git-wip-us.apache.org/repos/asf/incubator-ratis/blob/7e71a2e0/ratis-project-dist/pom.xml
----------------------------------------------------------------------
diff --git a/ratis-project-dist/pom.xml b/ratis-project-dist/pom.xml
new file mode 100644
index 0000000..5af489d
--- /dev/null
+++ b/ratis-project-dist/pom.xml
@@ -0,0 +1,169 @@
+<?xml version="1.0" encoding="UTF-8"?>
+<!--
+  Licensed 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. See accompanying LICENSE file.
+-->
+<project xmlns="http://maven.apache.org/POM/4.0.0"
+         xmlns:xsi="http://www.w3.org/2001/XMLSchema-instance"
+         xsi:schemaLocation="http://maven.apache.org/POM/4.0.0 http://maven.apache.org/xsd/maven-4.0.0.xsd">
+  <modelVersion>4.0.0</modelVersion>
+  <parent>
+    <artifactId>ratis-project</artifactId>
+    <groupId>org.apache.ratis</groupId>
+    <version>1.0-SNAPSHOT</version>
+    <relativePath>../ratis-project</relativePath>
+  </parent>
+
+  <artifactId>ratis-project-dist</artifactId>
+  <groupId>org.apache.ratis</groupId>
+  <name>Ratis Project Dist POM</name>
+  <version>1.0-SNAPSHOT</version>
+
+  <packaging>pom</packaging>
+
+  <properties>
+    <ratis.tmp.dir>${project.build.directory}/test</ratis.tmp.dir>
+    <test.build.data>${project.build.directory}/test/data</test.build.data>
+    <ratis.log.dir>${project.build.directory}/log</ratis.log.dir>
+    <test.build.webapps>${project.build.directory}/test-classes/webapps</test.build.webapps>
+    <test.cache.data>${project.build.directory}/test-classes</test.cache.data>
+    <test.build.classes>${project.build.directory}/test-classes</test.build.classes>
+  </properties>
+
+  <build>
+    <plugins>
+      <plugin>
+        <groupId>org.apache.maven.plugins</groupId>
+        <artifactId>maven-jar-plugin</artifactId>
+        <executions>
+          <execution>
+            <id>default-jar</id>
+            <phase>never</phase>
+          </execution>
+          <execution>
+            <id>prepare-jar</id>
+            <phase>prepare-package</phase>
+            <goals>
+              <goal>jar</goal>
+            </goals>
+          </execution>
+          <execution>
+            <id>prepare-test-jar</id>
+            <phase>prepare-package</phase>
+            <goals>
+              <goal>test-jar</goal>
+            </goals>
+            <configuration>
+              <includes>
+                <include>**/*.class</include>
+                <include>webapps/**</include>
+              </includes>
+            </configuration>
+          </execution>
+        </executions>
+      </plugin>
+
+      <plugin>
+        <groupId>org.apache.maven.plugins</groupId>
+        <artifactId>maven-source-plugin</artifactId>
+        <version>${maven-source-plugin.version}</version>
+        <executions>
+          <execution>
+            <phase>prepare-package</phase>
+            <goals>
+              <goal>jar</goal>
+              <goal>test-jar</goal>
+            </goals>
+          </execution>
+        </executions>
+        <configuration>
+          <attach>true</attach>
+        </configuration>
+      </plugin>
+      <plugin>
+        <groupId>org.codehaus.mojo</groupId>
+        <artifactId>findbugs-maven-plugin</artifactId>
+        <configuration>
+          <excludeFilterFile>${basedir}/dev-support/findbugsExcludeFile.xml</excludeFilterFile>
+          <fork>true</fork>
+          <maxHeap>2048</maxHeap>
+        </configuration>
+      </plugin>
+    </plugins>
+  </build>
+
+  <profiles>
+    <profile>
+      <id>dist</id>
+      <activation>
+        <activeByDefault>false</activeByDefault>
+      </activation>
+      <build>
+        <plugins>
+          <plugin>
+            <groupId>org.apache.maven.plugins</groupId>
+            <artifactId>maven-antrun-plugin</artifactId>
+            <executions>
+              <execution>
+                <id>tar</id>
+                <phase>package</phase>
+                <goals>
+                  <goal>run</goal>
+                </goals>
+                <configuration>
+                  <target if="tar">
+                    <!-- Using Unix script to preserve symlinks -->
+                    <echo file="${project.build.directory}/dist-maketar.sh">
+                      cd "${project.build.directory}"
+                      tar cf - ${project.artifactId}-${project.version} | gzip > ${project.artifactId}-${project.version}.tar.gz
+                    </echo>
+                    <exec executable="${shell-executable}" dir="${project.build.directory}" failonerror="true">
+                      <arg line="./dist-maketar.sh"/>
+                    </exec>
+                  </target>
+                </configuration>
+              </execution>
+            </executions>
+          </plugin>
+          <plugin>
+            <groupId>org.apache.maven.plugins</groupId>
+            <artifactId>maven-assembly-plugin</artifactId>
+            <dependencies>
+              <dependency>
+                <groupId>org.apache.hadoop</groupId>
+                <artifactId>hadoop-assemblies</artifactId>
+                <version>${project.version}</version>
+              </dependency>
+            </dependencies>
+            <executions>
+              <execution>
+                <id>dist</id>
+                <phase>prepare-package</phase>
+                <goals>
+                  <goal>single</goal>
+                </goals>
+                <configuration>
+                  <appendAssemblyId>false</appendAssemblyId>
+                  <attach>false</attach>
+                  <finalName>${project.artifactId}-${project.version}</finalName>
+                  <descriptorRefs>
+                    <descriptorRef>hadoop-dist</descriptorRef>
+                  </descriptorRefs>
+                </configuration>
+              </execution>
+            </executions>
+          </plugin>
+        </plugins>
+      </build>
+    </profile>
+  </profiles>
+</project>

http://git-wip-us.apache.org/repos/asf/incubator-ratis/blob/7e71a2e0/ratis-project/pom.xml
----------------------------------------------------------------------
diff --git a/ratis-project/pom.xml b/ratis-project/pom.xml
new file mode 100644
index 0000000..8abf5af
--- /dev/null
+++ b/ratis-project/pom.xml
@@ -0,0 +1,409 @@
+<?xml version="1.0" encoding="UTF-8"?>
+<!--
+  Licensed 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. See accompanying LICENSE file.
+-->
+<project xmlns="http://maven.apache.org/POM/4.0.0"
+         xmlns:xsi="http://www.w3.org/2001/XMLSchema-instance"
+         xsi:schemaLocation="http://maven.apache.org/POM/4.0.0 http://maven.apache.org/xsd/maven-4.0.0.xsd">
+  <modelVersion>4.0.0</modelVersion>
+  <parent>
+    <artifactId>ratis-main</artifactId>
+    <groupId>org.apache.ratis</groupId>
+    <version>1.0-SNAPSHOT</version>
+  </parent>
+
+  <artifactId>ratis-project</artifactId>
+  <groupId>org.apache.ratis</groupId>
+  <name>Ratis Project</name>
+  <packaging>pom</packaging>
+
+  <modules>
+    <module>../ratis-project-dist</module>
+    <module>../ratis-common</module>
+    <module>../ratis-client</module>
+    <module>../ratis-server</module>
+    <module>../ratis-hadoop</module>
+    <module>../ratis-grpc</module>
+    <module>../ratis-netty</module>
+    <module>../ratis-examples</module>
+  </modules>
+
+  <properties>
+    <!-- Set the Release year during release -->
+    <release-year>2016</release-year>
+
+    <maven.test.redirectTestOutputToFile>true
+    </maven.test.redirectTestOutputToFile>
+    <test.exclude>_</test.exclude>
+    <test.exclude.pattern>_</test.exclude.pattern>
+
+    <!-- number of threads/forks to use when running tests in parallel, see parallel-tests profile -->
+    <testsThreadCount>4</testsThreadCount>
+
+    <test.build.dir>${project.build.directory}/test-dir</test.build.dir>
+    <test.build.data>${test.build.dir}</test.build.data>
+
+    <findbugs.version>3.0.0</findbugs.version>
+
+    <maven-pdf-plugin.version>1.2</maven-pdf-plugin.version>
+    <build-helper-maven-plugin.version>1.9</build-helper-maven-plugin.version>
+    <exec-maven-plugin.version>1.3.1</exec-maven-plugin.version>
+    <make-maven-plugin.version>1.0-beta-1</make-maven-plugin.version>
+    <native-maven-plugin.version>1.0-alpha-8</native-maven-plugin.version>
+
+    <hadoop.version>3.0.0-alpha1</hadoop.version>
+  </properties>
+
+  <dependencyManagement>
+    <dependencies>
+      <dependency>
+        <artifactId>ratis-proto-shaded</artifactId>
+        <groupId>org.apache.ratis</groupId>
+        <version>${project.version}</version>
+      </dependency>
+
+      <dependency>
+        <artifactId>ratis-common</artifactId>
+        <groupId>org.apache.ratis</groupId>
+        <version>${project.version}</version>
+      </dependency>
+      <dependency>
+        <artifactId>ratis-common</artifactId>
+        <groupId>org.apache.ratis</groupId>
+        <version>${project.version}</version>
+        <type>test-jar</type>
+      </dependency>
+
+      <dependency>
+        <artifactId>ratis-client</artifactId>
+        <groupId>org.apache.ratis</groupId>
+        <version>${project.version}</version>
+      </dependency>
+      <dependency>
+        <artifactId>ratis-client</artifactId>
+        <groupId>org.apache.ratis</groupId>
+        <version>${project.version}</version>
+        <type>test-jar</type>
+      </dependency>
+
+      <dependency>
+        <artifactId>ratis-hadoop</artifactId>
+        <groupId>org.apache.ratis</groupId>
+        <version>${project.version}</version>
+      </dependency>
+      <dependency>
+        <artifactId>ratis-hadoop</artifactId>
+        <groupId>org.apache.ratis</groupId>
+        <version>${project.version}</version>
+        <type>test-jar</type>
+      </dependency>
+
+      <dependency>
+        <artifactId>ratis-grpc</artifactId>
+        <groupId>org.apache.ratis</groupId>
+        <version>${project.version}</version>
+      </dependency>
+      <dependency>
+        <artifactId>ratis-grpc</artifactId>
+        <groupId>org.apache.ratis</groupId>
+        <version>${project.version}</version>
+        <type>test-jar</type>
+      </dependency>
+
+      <dependency>
+        <artifactId>ratis-netty</artifactId>
+        <groupId>org.apache.ratis</groupId>
+        <version>${project.version}</version>
+      </dependency>
+      <dependency>
+        <artifactId>ratis-netty</artifactId>
+        <groupId>org.apache.ratis</groupId>
+        <version>${project.version}</version>
+        <type>test-jar</type>
+      </dependency>
+
+      <dependency>
+        <artifactId>ratis-server</artifactId>
+        <groupId>org.apache.ratis</groupId>
+        <version>${project.version}</version>
+      </dependency>
+      <dependency>
+        <artifactId>ratis-server</artifactId>
+        <groupId>org.apache.ratis</groupId>
+        <version>${project.version}</version>
+        <type>test-jar</type>
+      </dependency>
+
+      <dependency>
+        <groupId>org.slf4j</groupId>
+        <artifactId>slf4j-api</artifactId>
+        <version>1.7.10</version>
+      </dependency>
+      <dependency>
+        <groupId>org.slf4j</groupId>
+        <artifactId>slf4j-log4j12</artifactId>
+        <version>1.7.10</version>
+      </dependency>
+
+      <dependency>
+        <groupId>com.google.guava</groupId>
+        <artifactId>guava</artifactId>
+        <version>20.0</version>
+      </dependency>
+
+      <dependency>
+        <groupId>io.netty</groupId>
+        <artifactId>netty-all</artifactId>
+        <version>4.1.6.Final</version>
+      </dependency>
+
+      <dependency>
+        <groupId>junit</groupId>
+        <artifactId>junit</artifactId>
+        <version>4.11</version>
+      </dependency>
+      <dependency>
+        <groupId>org.mockito</groupId>
+        <artifactId>mockito-all</artifactId>
+        <version>1.8.5</version>
+      </dependency>
+    </dependencies>
+  </dependencyManagement>
+
+  <build>
+    <pluginManagement>
+      <plugins>
+        <plugin>
+          <artifactId>maven-clean-plugin</artifactId>
+          <version>${maven-clean-plugin.version}</version>
+        </plugin>
+        <plugin>
+          <groupId>org.codehaus.mojo</groupId>
+          <artifactId>build-helper-maven-plugin</artifactId>
+          <version>${build-helper-maven-plugin.version}</version>
+        </plugin>
+        <plugin>
+          <groupId>org.apache.maven.plugins</groupId>
+          <artifactId>maven-install-plugin</artifactId>
+          <version>${maven-install-plugin.version}</version>
+        </plugin>
+        <plugin>
+          <groupId>org.apache.maven.plugins</groupId>
+          <artifactId>maven-jar-plugin</artifactId>
+          <version>${maven-jar-plugin.version}</version>
+        </plugin>
+        <plugin>
+          <groupId>org.apache.maven.plugins</groupId>
+          <artifactId>maven-javadoc-plugin</artifactId>
+          <version>${maven-javadoc-plugin.version}</version>
+          <configuration>
+            <additionalparam>-Xmaxwarns 10000</additionalparam>
+          </configuration>
+        </plugin>
+        <plugin>
+          <groupId>org.codehaus.mojo</groupId>
+          <artifactId>findbugs-maven-plugin</artifactId>
+          <version>${findbugs.version}</version>
+        </plugin>
+        <plugin>
+          <groupId>org.codehaus.mojo</groupId>
+          <artifactId>native-maven-plugin</artifactId>
+          <version>${native-maven-plugin.version}</version>
+        </plugin>
+        <plugin>
+          <groupId>org.codehaus.mojo</groupId>
+          <artifactId>make-maven-plugin</artifactId>
+          <version>${make-maven-plugin.version}</version>
+        </plugin>
+        <plugin>
+          <groupId>org.apache.maven.plugins</groupId>
+          <artifactId>maven-resources-plugin</artifactId>
+          <version>${maven-resources-plugin.version}</version>
+        </plugin>
+        <plugin>
+          <groupId>org.codehaus.mojo</groupId>
+          <artifactId>exec-maven-plugin</artifactId>
+          <version>${exec-maven-plugin.version}</version>
+        </plugin>
+        <plugin>
+          <groupId>org.apache.maven.plugins</groupId>
+          <artifactId>maven-pdf-plugin</artifactId>
+          <version>${maven-pdf-plugin.version}</version>
+        </plugin>
+
+        <plugin>
+          <groupId>org.apache.hadoop</groupId>
+          <artifactId>hadoop-maven-plugins</artifactId>
+          <version>${hadoop.version}</version>
+        </plugin>
+      </plugins>
+    </pluginManagement>
+
+    <plugins>
+      <plugin>
+        <groupId>org.codehaus.mojo</groupId>
+        <artifactId>findbugs-maven-plugin</artifactId>
+      </plugin>
+
+      <plugin>
+        <groupId>org.apache.maven.plugins</groupId>
+        <artifactId>maven-antrun-plugin</artifactId>
+        <executions>
+          <execution>
+            <id>create-testdirs</id>
+            <phase>validate</phase>
+            <goals>
+              <goal>run</goal>
+            </goals>
+            <configuration>
+              <target>
+                <mkdir dir="${test.build.dir}"/>
+                <mkdir dir="${test.build.data}"/>
+              </target>
+            </configuration>
+          </execution>
+        </executions>
+      </plugin>
+
+      <plugin>
+        <groupId>org.apache.maven.plugins</groupId>
+        <artifactId>maven-pdf-plugin</artifactId>
+        <configuration>
+          <outputDirectory>${project.reporting.outputDirectory}
+          </outputDirectory>
+          <includeReports>false</includeReports>
+        </configuration>
+      </plugin>
+    </plugins>
+  </build>
+
+
+  <profiles>
+    <profile>
+      <id>os.linux</id>
+      <activation>
+        <os>
+          <family>!Mac</family>
+        </os>
+      </activation>
+      <properties>
+        <build.platform>${os.name}-${os.arch}-${sun.arch.data.model}
+        </build.platform>
+      </properties>
+    </profile>
+    <profile>
+      <id>os.mac</id>
+      <activation>
+        <os>
+          <family>Mac</family>
+        </os>
+      </activation>
+      <properties>
+        <build.platform>Mac_OS_X-${sun.arch.data.model}</build.platform>
+      </properties>
+    </profile>
+    <profile>
+      <id>native-win</id>
+      <activation>
+        <os>
+          <family>Windows</family>
+        </os>
+      </activation>
+      <properties>
+        <!-- We must use this exact string for egd on Windows, because the -->
+        <!-- JVM will check for an exact string match on this.  If found, it -->
+        <!-- will use a native entropy provider.  This will not really -->
+        <!-- attempt to open a file at this path. -->
+        <java.security.egd>file:/dev/urandom</java.security.egd>
+        <bundle.snappy.in.bin>true</bundle.snappy.in.bin>
+        <bundle.openssl.in.bin>true</bundle.openssl.in.bin>
+      </properties>
+    </profile>
+    <profile>
+      <id>test-patch</id>
+      <activation>
+        <activeByDefault>false</activeByDefault>
+      </activation>
+      <build>
+        <plugins>
+          <plugin>
+            <groupId>org.apache.maven.plugins</groupId>
+            <artifactId>maven-compiler-plugin</artifactId>
+            <configuration>
+              <fork>true</fork>
+              <source>${javac.version}</source>
+              <target>${javac.version}</target>
+              <compilerArguments>
+                <Xlint/>
+                <Xmaxwarns>9999</Xmaxwarns>
+              </compilerArguments>
+            </configuration>
+          </plugin>
+        </plugins>
+      </build>
+    </profile>
+    <profile>
+      <id>dist</id>
+      <build>
+        <plugins>
+          <plugin>
+            <groupId>org.apache.maven.plugins</groupId>
+            <artifactId>maven-javadoc-plugin</artifactId>
+            <executions>
+              <execution>
+                <!-- build javadoc jars per jar for publishing to maven -->
+                <id>module-javadocs</id>
+                <phase>package</phase>
+                <goals>
+                  <goal>jar</goal>
+                </goals>
+                <configuration>
+                  <destDir>${project.build.directory}</destDir>
+                </configuration>
+              </execution>
+            </executions>
+          </plugin>
+          <plugin>
+            <groupId>org.apache.maven.plugins</groupId>
+            <artifactId>maven-source-plugin</artifactId>
+            <executions>
+              <execution>
+                <!-- builds source jars and attaches them to the project for publishing -->
+                <id>ratis-java-sources</id>
+                <phase>package</phase>
+                <goals>
+                  <goal>jar-no-fork</goal>
+                  <goal>test-jar-no-fork</goal>
+                </goals>
+              </execution>
+            </executions>
+          </plugin>
+          <plugin>
+            <groupId>org.apache.maven.plugins</groupId>
+            <artifactId>maven-enforcer-plugin</artifactId>
+            <executions>
+              <execution>
+                <id>dist-enforce</id>
+                <phase>package</phase>
+                <goals>
+                  <goal>enforce</goal>
+                </goals>
+              </execution>
+            </executions>
+          </plugin>
+        </plugins>
+      </build>
+    </profile>
+  </profiles>
+
+</project>

http://git-wip-us.apache.org/repos/asf/incubator-ratis/blob/7e71a2e0/ratis-proto-shaded/.gitignore
----------------------------------------------------------------------
diff --git a/ratis-proto-shaded/.gitignore b/ratis-proto-shaded/.gitignore
new file mode 100644
index 0000000..ce50505
--- /dev/null
+++ b/ratis-proto-shaded/.gitignore
@@ -0,0 +1,2 @@
+src/main/java
+dependency-reduced-pom.xml

http://git-wip-us.apache.org/repos/asf/incubator-ratis/blob/7e71a2e0/ratis-proto-shaded/README.md
----------------------------------------------------------------------
diff --git a/ratis-proto-shaded/README.md b/ratis-proto-shaded/README.md
new file mode 100644
index 0000000..f7a25d4
--- /dev/null
+++ b/ratis-proto-shaded/README.md
@@ -0,0 +1,23 @@
+# Raft Proto Shaded
+
+This module is to shade protos, protobuf and other libraries such as Netty, gRPC and Hadoop
+so that applications using Raft may use protobuf and other libraries with versions different 
+from the versions used here.
+
+Other modules require the shaded sources for compilation. To generate them,
+run the following command under `ratis-proto-shaded/`
+
+- `mvn package -Dcompile-protobuf -DskipTests`
+
+The generated sources are stored in `ratis-proto-shaded/src/main/java/`.
+
+## What are shaded?
+
+| Original packages                 | Shaded packages                                          |
+| ----------------------------------|----------------------------------------------------------|
+| `com.google.protobuf`             | `org.apache.ratis.shaded.com.google.protobuf`             |
+| `io.grpc`                         | `org.apache.ratis.shaded.io.grpc`                         |
+| `io.netty.handler.codec.protobuf` | `org.apache.ratis.shaded.io.netty.handler.codec.protobuf` |
+| `org.apache.hadoop.ipc.protobuf`  | `org.apache.ratis.shaded.org.apache.hadoop.ipc.protobuf`  |
+
+The protos defined in this project are stored in the `org.apache.ratis.shaded.proto` package.

http://git-wip-us.apache.org/repos/asf/incubator-ratis/blob/7e71a2e0/ratis-proto-shaded/pom.xml
----------------------------------------------------------------------
diff --git a/ratis-proto-shaded/pom.xml b/ratis-proto-shaded/pom.xml
new file mode 100644
index 0000000..a196811
--- /dev/null
+++ b/ratis-proto-shaded/pom.xml
@@ -0,0 +1,426 @@
+<?xml version="1.0" encoding="UTF-8"?>
+<!--
+  Licensed 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. See accompanying LICENSE file.
+-->
+<project xmlns="http://maven.apache.org/POM/4.0.0"
+         xmlns:xsi="http://www.w3.org/2001/XMLSchema-instance"
+         xsi:schemaLocation="http://maven.apache.org/POM/4.0.0 http://maven.apache.org/xsd/maven-4.0.0.xsd">
+  <modelVersion>4.0.0</modelVersion>
+  <parent>
+    <artifactId>ratis-main</artifactId>
+    <groupId>org.apache.ratis</groupId>
+    <version>1.0-SNAPSHOT</version>
+    <relativePath>..</relativePath>
+  </parent>
+
+  <artifactId>ratis-proto-shaded</artifactId>
+  <name>Ratis Proto Shaded</name>
+  <properties>
+    <maven.javadoc.skip>true</maven.javadoc.skip>
+    <!--The Default target dir-->
+    <classes.dir>${project.build.directory}/classes</classes.dir>
+    <!--The Default location for sources-->
+    <sources.dir>src/main/java</sources.dir>
+
+    <!--Version of protobuf to be shaded -->
+    <shaded.protobuf.version>3.1.0</shaded.protobuf.version>
+    <!--Version of grpc to be shaded -->
+    <shaded.grpc.version>1.0.1</shaded.grpc.version>
+    <!--Version of Hadoop to be shaded -->
+    <shaded.hadoop.version>3.0.0-alpha1</shaded.hadoop.version>
+  </properties>
+
+  <build>
+    <!--I want to override these in profile so define them
+         with variables up here-->
+    <sourceDirectory>${sources.dir}</sourceDirectory>
+    <outputDirectory>${classes.dir}</outputDirectory>
+    <plugins>
+      <!-- Make a jar and put the sources in the jar -->
+      <plugin>
+        <groupId>org.apache.maven.plugins</groupId>
+        <artifactId>maven-source-plugin</artifactId>
+        <version>${maven-source-plugin.version}</version>
+      </plugin>
+      <plugin>
+        <!--Make it so assembly:single does nothing in here-->
+        <artifactId>maven-assembly-plugin</artifactId>
+        <version>${maven-assembly-plugin.version}</version>
+        <configuration>
+          <skipAssembly>true</skipAssembly>
+        </configuration>
+      </plugin>
+      <plugin>
+        <artifactId>maven-surefire-plugin</artifactId>
+        <!-- Always skip the second part executions
+             since we only run simple unit tests in this module -->
+        <executions>
+          <execution>
+            <id>secondPartTestsExecution</id>
+            <phase>test</phase>
+            <goals>
+              <goal>test</goal>
+            </goals>
+            <configuration>
+              <skip>true</skip>
+            </configuration>
+          </execution>
+        </executions>
+      </plugin>
+    </plugins>
+  </build>
+
+  <dependencies>
+    <dependency>
+      <groupId>com.google.protobuf</groupId>
+      <artifactId>protobuf-java</artifactId>
+      <version>${shaded.protobuf.version}</version>
+    </dependency>
+    <dependency>
+      <groupId>com.google.protobuf.nano</groupId>
+      <artifactId>protobuf-javanano</artifactId>
+      <version>${shaded.protobuf.version}</version>
+    </dependency>
+
+    <dependency>
+      <groupId>io.grpc</groupId>
+      <artifactId>grpc-netty</artifactId>
+      <version>${shaded.grpc.version}</version>
+    </dependency>
+    <dependency>
+      <groupId>io.grpc</groupId>
+      <artifactId>grpc-protobuf</artifactId>
+      <version>${shaded.grpc.version}</version>
+    </dependency>
+    <dependency>
+      <groupId>io.grpc</groupId>
+      <artifactId>grpc-stub</artifactId>
+      <version>${shaded.grpc.version}</version>
+    </dependency>
+
+    <dependency>
+      <groupId>org.apache.hadoop</groupId>
+      <artifactId>hadoop-common</artifactId>
+      <version>${shaded.hadoop.version}</version>
+    </dependency>
+  </dependencies>
+
+  <profiles>
+    <!-- Skip the tests in this module -->
+    <profile>
+      <id>skip-proto-shaded-tests</id>
+      <activation>
+        <property>
+          <name>skip-proto-shaded-tests</name>
+        </property>
+      </activation>
+      <properties>
+        <surefire.skipFirstPart>true</surefire.skipFirstPart>
+      </properties>
+    </profile>
+
+    <profile>
+      <id>compile-protobuf</id>
+      <!--
+         Generate and shade proto files. Drops generated java files
+         under src/main/java. Check in the generated files so available
+         at build time. Run this profile/step everytime you change proto
+         files or update the protobuf version.
+
+         The below does a bunch of ugly stuff. It purges current content
+         of the generated and shaded com.google.protobuf java files first.
+         It does this because later we apply patches later and patches
+         fail they've already been applied. We remove too because we
+         overlay the shaded protobuf and if files have been removed or
+         added, it'll be more plain if we have first done this delete.
+
+         Next up we generate proto, build a jar, shade it (which
+         includes the referenced protobuf), undo it over the src/main/java
+         directory, and then apply patches.
+
+         The result needs to be checked in.
+      -->
+      <activation>
+        <property>
+          <name>compile-protobuf</name>
+        </property>
+      </activation>
+      <properties>
+        <profile.id>compile-protobuf</profile.id>
+        <sources.dir>${project.build.directory}/protoc-generated-sources</sources.dir>
+        <classes.dir>${project.build.directory}/protoc-generated-classes</classes.dir>
+        <!--When the compile for this profile runs, make sure it makes jars that
+             can be related back to this shading profile. Give them a shading prefix.
+         -->
+        <jar.finalName>${profile.id}.${project.artifactId}-${project.version}</jar.finalName>
+      </properties>
+      <build>
+        <finalName>${jar.finalName}</finalName>
+        <plugins>
+          <plugin>
+            <artifactId>maven-clean-plugin</artifactId>
+            <version>${maven-clean-plugin.version}</version>
+            <executions>
+              <execution>
+                <id>pre-compile-protoc</id>
+                <phase>generate-sources</phase>
+                <goals>
+                  <goal>clean</goal>
+                </goals>
+                <configuration>
+                  <filesets>
+                    <fileset>
+                      <directory>
+                        ${basedir}/src/main/java/
+                      </directory>
+                      <followSymlinks>false</followSymlinks>
+                    </fileset>
+                  </filesets>
+                </configuration>
+              </execution>
+            </executions>
+          </plugin>
+
+          <plugin>
+            <groupId>org.xolstice.maven.plugins</groupId>
+            <artifactId>protobuf-maven-plugin</artifactId>
+            <version>${maven-xolstice-plugin.version}</version>
+            <configuration>
+              <protocArtifact>
+                com.google.protobuf:protoc:${shaded.protobuf.version}:exe:${os.detected.classifier}
+              </protocArtifact>
+            </configuration>
+            <executions>
+              <execution>
+                <id>1</id>
+                <goals>
+                  <goal>compile</goal>
+                  <goal>test-compile</goal>
+                </goals>
+              </execution>
+              <execution>
+                <id>2</id>
+                <phase>generate-sources</phase>
+                <goals>
+                  <goal>compile-custom</goal>
+                  <goal>test-compile-custom</goal>
+                </goals>
+                <configuration>
+                  <pluginId>grpc-java</pluginId>
+                  <pluginArtifact>
+                    io.grpc:protoc-gen-grpc-java:${shaded.grpc.version}:exe:${os.detected.classifier}
+                  </pluginArtifact>
+                </configuration>
+              </execution>
+            </executions>
+          </plugin>
+
+          <plugin>
+            <groupId>org.apache.maven.plugins</groupId>
+            <artifactId>maven-shade-plugin</artifactId>
+            <version>2.4.3</version>
+            <executions>
+              <execution>
+                <phase>package</phase>
+                <goals>
+                  <goal>shade</goal>
+                </goals>
+                <configuration>
+                  <shadeSourcesContent>true</shadeSourcesContent>
+                  <createSourcesJar>true</createSourcesJar>
+                  <relocations>
+                    <relocation>
+                      <pattern>com.google.protobuf</pattern>
+                      <shadedPattern>org.apache.ratis.shaded.com.google.protobuf</shadedPattern>
+                    </relocation>
+                    <relocation>
+                      <pattern>io.grpc</pattern>
+                      <shadedPattern>org.apache.ratis.shaded.io.grpc</shadedPattern>
+                    </relocation>
+                    <relocation>
+                      <pattern>io.netty.handler.codec.protobuf</pattern>
+                      <shadedPattern>org.apache.ratis.shaded.io.netty.handler.codec.protobuf</shadedPattern>
+                    </relocation>
+
+                    <relocation>
+                      <pattern>org.apache.hadoop.ipc.protobuf</pattern>
+                      <shadedPattern>org.apache.ratis.shaded.org.apache.hadoop.ipc.protobuf</shadedPattern>
+                    </relocation>
+                  </relocations>
+
+                  <filters>
+                    <filter>
+                      <artifact>io.netty:netty-codec</artifact>
+                      <includes>
+                        <include>io/netty/handler/codec/protobuf/**</include>
+                      </includes>
+                    </filter>
+                    <filter>
+                      <artifact>org.apache.hadoop:hadoop-common</artifact>
+                      <includes>
+                        <include>org/apache/hadoop/ipc/protobuf/**</include>
+                      </includes>
+                    </filter>
+                  </filters>
+
+                  <artifactSet>
+                    <excludes>
+                      <exclude>asm:asm</exclude>
+
+                      <exclude>com.google.code.findbugs</exclude>
+                      <exclude>com.google.code.gson:gson</exclude>
+                      <exclude>com.google.guava:guava</exclude>
+		      com.google.re2j:re2j
+
+                      <exclude>com.jamesmurty.utils:java-xmlbuilder</exclude>
+                      <exclude>com.jcraft:jsch</exclude>
+
+                      <exclude>com.sun.jersey:jersey-core</exclude>
+                      <exclude>com.sun.jersey:jersey-json</exclude>
+                      <exclude>com.sun.jersey:jersey-server</exclude>
+                      <exclude>com.sun.jersey:jersey-servlet</exclude>
+                      <exclude>com.sun.xml.bind:jaxb-impl</exclude>
+ 
+                      <exclude>com.thoughtworks.paranamer:paranamer</exclude>
+
+                      <exclude>commons-beanutils:commons-beanutils-core</exclude>
+                      <exclude>commons-beanutils:commons-beanutils</exclude>
+                      <exclude>commons-cli:commons-cli</exclude>
+                      <exclude>commons-codec:commons-codec</exclude>
+                      <exclude>commons-collections:commons-collections</exclude>
+                      <exclude>commons-configuration:commons-configuration</exclude>
+                      <exclude>commons-digester:commons-digester</exclude>
+                      <exclude>commons-httpclient:commons-httpclient</exclude>
+                      <exclude>commons-io:commons-io</exclude>
+                      <exclude>commons-lang:commons-lang</exclude>
+                      <exclude>commons-logging:commons-logging</exclude>
+                      <exclude>commons-net:commons-net</exclude>
+
+                      <exclude>io.netty:netty-buffer</exclude>
+                      <exclude>io.netty:netty-codec-http2</exclude>
+                      <exclude>io.netty:netty-codec-http</exclude>
+                      <exclude>io.netty:netty-common</exclude>
+                      <exclude>io.netty:netty-handler</exclude>
+                      <exclude>io.netty:netty-resolver</exclude>
+                      <exclude>io.netty:netty-transport</exclude>
+                      <exclude>io.netty:netty</exclude>
+
+                      <exclude>javax.activation:activation</exclude>
+                      <exclude>javax.servlet.jsp:jsp-api</exclude>
+                      <exclude>javax.servlet:servlet-api</exclude>
+                      <exclude>javax.xml.bind:jaxb-api</exclude>
+                      <exclude>javax.xml.stream:stax-api</exclude>
+		      <exclude>javax.servlet:javax.servlet-api</exclude>
+                      <exclude>javax.ws.rs:jsr311-api</exclude>
+
+                      <exclude>log4j:log4j</exclude>
+                      <exclude>net.java.dev.jets3t:jets3t</exclude>
+
+                      <exclude>org.apache.avro:avro</exclude>
+                      <exclude>org.apache.commons:commons-compress</exclude>
+                      <exclude>org.apache.commons:commons-math3</exclude>
+                      <exclude>org.apache.curator:curator-client</exclude>
+                      <exclude>org.apache.curator:curator-framework</exclude>
+                      <exclude>org.apache.curator:curator-recipes</exclude>
+                      <exclude>org.apache.directory.api:api-asn1-api</exclude>
+                      <exclude>org.apache.directory.api:api-util</exclude>
+                      <exclude>org.apache.directory.server:apacheds-i18n</exclude>
+                      <exclude>org.apache.directory.server:apacheds-kerberos-codec</exclude>
+                      <exclude>org.apache.hadoop:hadoop-annotations</exclude>
+                      <exclude>org.apache.hadoop:hadoop-auth</exclude>
+                      <exclude>org.apache.htrace:htrace-core</exclude>
+                      <exclude>org.apache.httpcomponents:httpclient</exclude>
+                      <exclude>org.apache.httpcomponents:httpcore</exclude>
+                      <exclude>org.apache.zookeeper:zookeeper</exclude>
+
+                      <exclude>org.codehaus.jackson:jackson-core-asl</exclude>
+                      <exclude>org.codehaus.jackson:jackson-jaxrs</exclude>
+                      <exclude>org.codehaus.jackson:jackson-mapper-asl</exclude>
+                      <exclude>org.codehaus.jackson:jackson-xc</exclude>
+                      <exclude>org.codehaus.jettison:jettison</exclude>
+
+                      <exclude>org.mortbay.jetty:jetty-util</exclude>
+                      <exclude>org.mortbay.jetty:jetty</exclude>
+                      <exclude>org.eclipse.jetty:jetty-server</exclude>
+                      <exclude>org.eclipse.jetty:jetty-util</exclude>
+                      <exclude>org.eclipse.jetty:jetty-servlet</exclude>
+                      <exclude>org.eclipse.jetty:jetty-webapp</exclude>
+                      <exclude>org.eclipse.jetty:jetty-util-ajax</exclude>
+		      <exclude>org.mortbay.jetty:jetty-sslengine</exclude>
+
+                      <exclude>org.slf4j:slf4j-api</exclude>
+                      <exclude>org.slf4j:slf4j-log4j12</exclude>
+                      <exclude>org.tukaani:xz</exclude>
+                      <exclude>org.xerial.snappy:snappy-java</exclude>
+                      <exclude>xmlenc:xmlenc</exclude>
+
+                      <exclude>com.nimbusds:nimbus-jose-jwt</exclude>
+                      <exclude>net.jcip:jcip-annotations</exclude>
+                      <exclude>net.minidev:json-smart</exclude>
+
+                      <exclude>org.apache.htrace:htrace-core4</exclude>
+                      <exclude>org.apache.kerby:kerb-simplekdc</exclude>
+                      <exclude>org.apache.kerby:kerby-config</exclude>
+                      <exclude>org.apache.kerby:kerb-core</exclude>
+                      <exclude>org.apache.kerby:kerby-asn1</exclude>
+                      <exclude>org.apache.kerby:kerby-pkix</exclude>
+                      <exclude>org.apache.kerby:kerby-util</exclude>
+                      <exclude>org.apache.kerby:kerb-client</exclude>
+                      <exclude>org.apache.kerby:kerb-common</exclude>
+                      <exclude>org.apache.kerby:kerb-util</exclude>
+                      <exclude>org.apache.kerby:kerb-crypto</exclude>
+                      <exclude>org.apache.kerby:kerb-server</exclude>
+                      <exclude>org.apache.kerby:kerb-identity</exclude>
+                      <exclude>org.apache.kerby:kerb-admin</exclude>
+                    </excludes>
+                  </artifactSet>
+                </configuration>
+              </execution>
+            </executions>
+          </plugin>
+          <!--Now unpack the shaded jar made above so the shaded classes
+             are available to subsequent modules-->
+          <plugin>
+            <groupId>org.apache.maven.plugins</groupId>
+            <artifactId>maven-dependency-plugin</artifactId>
+            <version>${maven-dependency-plugin.version}</version>
+            <executions>
+              <execution>
+                <id>unpack</id>
+                <phase>package</phase>
+                <goals>
+                  <goal>unpack</goal>
+                </goals>
+                <configuration>
+                  <artifactItems>
+                    <artifactItem>
+                      <groupId>${project.groupId}</groupId>
+                      <artifactId>${project.artifactId}</artifactId>
+                      <version>${project.version}</version>
+                      <classifier>sources</classifier>
+                      <type>jar</type>
+                      <overWrite>true</overWrite>
+                      <outputDirectory>${basedir}/src/main/java
+                      </outputDirectory>
+                      <includes>**/*.java</includes>
+                    </artifactItem>
+                  </artifactItems>
+                </configuration>
+              </execution>
+            </executions>
+          </plugin>
+        </plugins>
+      </build>
+    </profile>
+  </profiles>
+</project>

http://git-wip-us.apache.org/repos/asf/incubator-ratis/blob/7e71a2e0/ratis-proto-shaded/src/main/proto/GRpc.proto
----------------------------------------------------------------------
diff --git a/ratis-proto-shaded/src/main/proto/GRpc.proto b/ratis-proto-shaded/src/main/proto/GRpc.proto
new file mode 100644
index 0000000..267f579
--- /dev/null
+++ b/ratis-proto-shaded/src/main/proto/GRpc.proto
@@ -0,0 +1,45 @@
+/**
+ * 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.
+ */
+syntax = "proto3";
+option java_package = "org.apache.ratis.shaded.proto.grpc";
+option java_outer_classname = "GRpcProtos";
+option java_generate_equals_and_hash = true;
+package ratis.grpc;
+
+import "Raft.proto";
+
+service RaftClientProtocolService {
+  // A client-to-server RPC to set new raft configuration
+  rpc setConfiguration(ratis.common.SetConfigurationRequestProto)
+      returns(ratis.common.RaftClientReplyProto) {}
+
+  // A client-to-server stream RPC to append data
+  rpc append(stream ratis.common.RaftClientRequestProto)
+      returns (stream ratis.common.RaftClientReplyProto) {}
+}
+
+service RaftServerProtocolService {
+  rpc requestVote(ratis.common.RequestVoteRequestProto)
+      returns(ratis.common.RequestVoteReplyProto) {}
+
+  rpc appendEntries(stream ratis.common.AppendEntriesRequestProto)
+      returns(stream ratis.common.AppendEntriesReplyProto) {}
+
+  rpc installSnapshot(stream ratis.common.InstallSnapshotRequestProto)
+      returns(ratis.common.InstallSnapshotReplyProto) {}
+}

http://git-wip-us.apache.org/repos/asf/incubator-ratis/blob/7e71a2e0/ratis-proto-shaded/src/main/proto/Hadoop.proto
----------------------------------------------------------------------
diff --git a/ratis-proto-shaded/src/main/proto/Hadoop.proto b/ratis-proto-shaded/src/main/proto/Hadoop.proto
new file mode 100644
index 0000000..b85b9a2
--- /dev/null
+++ b/ratis-proto-shaded/src/main/proto/Hadoop.proto
@@ -0,0 +1,44 @@
+/**
+ * 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.
+ */
+syntax = "proto3";
+option java_package = "org.apache.ratis.shaded.proto.hadoop";
+option java_outer_classname = "HadoopProtos";
+option java_generic_services = true;
+option java_generate_equals_and_hash = true;
+package ratis.hadoop;
+
+import "Raft.proto";
+
+service RaftClientProtocolService {
+  rpc submitClientRequest(ratis.common.RaftClientRequestProto)
+      returns(ratis.common.RaftClientReplyProto);
+
+  rpc setConfiguration(ratis.common.SetConfigurationRequestProto)
+      returns(ratis.common.RaftClientReplyProto);
+}
+
+service RaftServerProtocolService {
+  rpc requestVote(ratis.common.RequestVoteRequestProto)
+      returns(ratis.common.RequestVoteReplyProto);
+
+  rpc appendEntries(ratis.common.AppendEntriesRequestProto)
+      returns(ratis.common.AppendEntriesReplyProto);
+
+  rpc installSnapshot(ratis.common.InstallSnapshotRequestProto)
+      returns(ratis.common.InstallSnapshotReplyProto);
+}

http://git-wip-us.apache.org/repos/asf/incubator-ratis/blob/7e71a2e0/ratis-proto-shaded/src/main/proto/Netty.proto
----------------------------------------------------------------------
diff --git a/ratis-proto-shaded/src/main/proto/Netty.proto b/ratis-proto-shaded/src/main/proto/Netty.proto
new file mode 100644
index 0000000..d1634d7
--- /dev/null
+++ b/ratis-proto-shaded/src/main/proto/Netty.proto
@@ -0,0 +1,49 @@
+/**
+ * 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.
+ */
+syntax = "proto3";
+option java_package = "org.apache.ratis.shaded.proto.netty";
+option java_outer_classname = "NettyProtos";
+option java_generate_equals_and_hash = true;
+package ratis.netty;
+
+import "Raft.proto";
+
+message RaftNettyExceptionReplyProto {
+  ratis.common.RaftRpcReplyProto rpcReply = 1;
+  bytes exception = 2;
+}
+
+message RaftNettyServerRequestProto {
+  oneof raftNettyServerRequest {
+    ratis.common.RequestVoteRequestProto requestVoteRequest = 1;
+    ratis.common.AppendEntriesRequestProto appendEntriesRequest = 2;
+    ratis.common.InstallSnapshotRequestProto installSnapshotRequest = 3;
+    ratis.common.RaftClientRequestProto raftClientRequest = 4;
+    ratis.common.SetConfigurationRequestProto setConfigurationRequest = 5;
+  }
+}
+
+message RaftNettyServerReplyProto {
+  oneof raftNettyServerReply {
+    ratis.common.RequestVoteReplyProto requestVoteReply = 1;
+    ratis.common.AppendEntriesReplyProto appendEntriesReply = 2;
+    ratis.common.InstallSnapshotReplyProto installSnapshotReply = 3;
+    ratis.common.RaftClientReplyProto raftClientReply = 4;
+    RaftNettyExceptionReplyProto exceptionReply = 5;
+  }
+}

http://git-wip-us.apache.org/repos/asf/incubator-ratis/blob/7e71a2e0/ratis-proto-shaded/src/main/proto/Raft.proto
----------------------------------------------------------------------
diff --git a/ratis-proto-shaded/src/main/proto/Raft.proto b/ratis-proto-shaded/src/main/proto/Raft.proto
new file mode 100644
index 0000000..80c4b8c
--- /dev/null
+++ b/ratis-proto-shaded/src/main/proto/Raft.proto
@@ -0,0 +1,165 @@
+/**
+ * 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.
+ */
+syntax = "proto3";
+option java_package = "org.apache.ratis.shaded.proto";
+option java_outer_classname = "RaftProtos";
+option java_generate_equals_and_hash = true;
+package ratis.common;
+
+message RaftPeerProto {
+  string id = 1;      // id of the peer
+  string address = 2; // e.g. IP address, hostname etc.
+}
+
+message RaftConfigurationProto {
+  repeated RaftPeerProto peers = 1; // the peers in the current or new conf
+  repeated RaftPeerProto oldPeers = 2; // the peers in the old conf
+}
+
+message SMLogEntryProto {
+  // TODO: This is not super efficient if the SM itself uses PB to serialize its own data for a
+  // log entry. Data will be copied twice. We should directly support having any Message from SM
+  bytes data = 1;
+}
+
+message LeaderNoOp {
+  // empty
+}
+
+message LogEntryProto {
+  uint64 term = 1;
+  uint64 index = 2;
+
+  oneof LogEntryBody {
+    SMLogEntryProto smLogEntry = 3;
+    RaftConfigurationProto configurationEntry = 4;
+    LeaderNoOp noOp = 5;
+  }
+}
+
+message TermIndexProto {
+  uint64 term = 1;
+  uint64 index = 2;
+}
+
+message RaftRpcRequestProto {
+  string requestorId = 1;
+  string replyId = 2;
+  uint64 seqNum = 3;
+}
+
+message RaftRpcReplyProto {
+  string requestorId = 1;
+  string replyId = 2;
+  uint64 seqNum = 3;
+  bool success = 4;
+}
+
+message FileChunkProto {
+  string filename = 1; // relative to root
+  uint64 totalSize = 2;
+  bytes fileDigest = 3;
+  uint32 chunkIndex = 4;
+  uint64 offset = 5;
+  bytes data = 6;
+  bool done = 7;
+}
+
+enum InstallSnapshotResult {
+  SUCCESS = 0;
+  NOT_LEADER = 1;
+}
+
+message RequestVoteRequestProto {
+  RaftRpcRequestProto serverRequest = 1;
+  uint64 candidateTerm = 2;
+  TermIndexProto candidateLastEntry = 3;
+}
+
+message RequestVoteReplyProto {
+  RaftRpcReplyProto serverReply = 1;
+  uint64 term = 2;
+  bool shouldShutdown = 3;
+}
+
+message AppendEntriesRequestProto {
+  RaftRpcRequestProto serverRequest = 1;
+  uint64 leaderTerm = 2;
+  TermIndexProto previousLog = 3;
+  repeated LogEntryProto entries = 4;
+  uint64 leaderCommit = 5;
+  bool initializing = 6;
+}
+
+message AppendEntriesReplyProto {
+  enum AppendResult {
+    SUCCESS = 0;
+    NOT_LEADER = 1; // the requester's term is not large enough
+    INCONSISTENCY = 2; // gap between the local log and the entries
+  }
+
+  RaftRpcReplyProto serverReply = 1;
+  uint64 term = 2;
+  uint64 nextIndex = 3;
+  AppendResult result = 4;
+}
+
+message InstallSnapshotRequestProto {
+  RaftRpcRequestProto serverRequest = 1;
+  string requestId = 2; // an identifier for chunked-requests.
+  uint32 requestIndex = 3; // the index for this request chunk. Starts from 0.
+  RaftConfigurationProto raftConfiguration = 4;
+  uint64 leaderTerm = 5;
+  TermIndexProto termIndex = 6;
+  repeated FileChunkProto fileChunks = 7;
+  uint64 totalSize = 8;
+  bool done = 9; // whether this is the final chunk for the same req.
+}
+
+message InstallSnapshotReplyProto {
+  RaftRpcReplyProto serverReply = 1;
+  uint32 requestIndex = 2;
+  uint64 term = 3;
+  InstallSnapshotResult result = 4;
+}
+
+message ClientMessageEntryProto {
+  bytes content = 1;
+}
+
+// normal client request
+message RaftClientRequestProto {
+  RaftRpcRequestProto rpcRequest = 1;
+  ClientMessageEntryProto message = 2;
+  bool readOnly = 3;
+}
+
+message RaftClientReplyProto {
+  RaftRpcReplyProto rpcReply = 1;
+  ClientMessageEntryProto message = 2;
+  // the following 3 fields are used to indicate the server is not leader
+  bool isNotLeader = 3;
+  RaftPeerProto suggestedLeader = 4;
+  repeated RaftPeerProto peersInConf = 5;
+}
+
+// setConfiguration request
+message SetConfigurationRequestProto {
+  RaftRpcRequestProto rpcRequest = 1;
+  repeated RaftPeerProto peers = 2;
+}

http://git-wip-us.apache.org/repos/asf/incubator-ratis/blob/7e71a2e0/ratis-proto-shaded/src/main/resources/META-INF/services/org.apache.ratis.shaded.io.grpc.ManagedChannelProvider
----------------------------------------------------------------------
diff --git a/ratis-proto-shaded/src/main/resources/META-INF/services/org.apache.ratis.shaded.io.grpc.ManagedChannelProvider b/ratis-proto-shaded/src/main/resources/META-INF/services/org.apache.ratis.shaded.io.grpc.ManagedChannelProvider
new file mode 100644
index 0000000..dbf2d84
--- /dev/null
+++ b/ratis-proto-shaded/src/main/resources/META-INF/services/org.apache.ratis.shaded.io.grpc.ManagedChannelProvider
@@ -0,0 +1,16 @@
+# 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.
+
+org.apache.ratis.shaded.io.grpc.netty.NettyChannelProvider

http://git-wip-us.apache.org/repos/asf/incubator-ratis/blob/7e71a2e0/ratis-proto-shaded/src/main/resources/META-INF/services/org.apache.ratis.shaded.io.grpc.NameResolverProvider
----------------------------------------------------------------------
diff --git a/ratis-proto-shaded/src/main/resources/META-INF/services/org.apache.ratis.shaded.io.grpc.NameResolverProvider b/ratis-proto-shaded/src/main/resources/META-INF/services/org.apache.ratis.shaded.io.grpc.NameResolverProvider
new file mode 100644
index 0000000..439b1d8
--- /dev/null
+++ b/ratis-proto-shaded/src/main/resources/META-INF/services/org.apache.ratis.shaded.io.grpc.NameResolverProvider
@@ -0,0 +1,16 @@
+# 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.
+
+org.apache.ratis.shaded.io.grpc.internal.DnsNameResolverProvider

http://git-wip-us.apache.org/repos/asf/incubator-ratis/blob/7e71a2e0/ratis-proto-shaded/src/main/resources/META-INF/services/org.apache.ratis.shaded.io.grpc.ServerProvider
----------------------------------------------------------------------
diff --git a/ratis-proto-shaded/src/main/resources/META-INF/services/org.apache.ratis.shaded.io.grpc.ServerProvider b/ratis-proto-shaded/src/main/resources/META-INF/services/org.apache.ratis.shaded.io.grpc.ServerProvider
new file mode 100644
index 0000000..f251467
--- /dev/null
+++ b/ratis-proto-shaded/src/main/resources/META-INF/services/org.apache.ratis.shaded.io.grpc.ServerProvider
@@ -0,0 +1,16 @@
+# 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.
+
+org.apache.ratis.shaded.io.grpc.netty.NettyServerProvider

http://git-wip-us.apache.org/repos/asf/incubator-ratis/blob/7e71a2e0/ratis-server/pom.xml
----------------------------------------------------------------------
diff --git a/ratis-server/pom.xml b/ratis-server/pom.xml
new file mode 100644
index 0000000..af9afd0
--- /dev/null
+++ b/ratis-server/pom.xml
@@ -0,0 +1,80 @@
+<?xml version="1.0" encoding="UTF-8"?>
+<!--
+  Licensed 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. See accompanying LICENSE file.
+-->
+<project xmlns="http://maven.apache.org/POM/4.0.0"
+         xmlns:xsi="http://www.w3.org/2001/XMLSchema-instance"
+         xsi:schemaLocation="http://maven.apache.org/POM/4.0.0 http://maven.apache.org/xsd/maven-4.0.0.xsd">
+  <modelVersion>4.0.0</modelVersion>
+  <parent>
+    <artifactId>ratis-project-dist</artifactId>
+    <groupId>org.apache.ratis</groupId>
+    <version>1.0-SNAPSHOT</version>
+    <relativePath>../ratis-project-dist</relativePath>
+  </parent>
+
+  <artifactId>ratis-server</artifactId>
+  <name>Ratis Server</name>
+
+  <dependencies>
+    <dependency>
+      <artifactId>ratis-proto-shaded</artifactId>
+      <groupId>org.apache.ratis</groupId>
+    </dependency>
+
+    <dependency>
+      <artifactId>ratis-common</artifactId>
+      <groupId>org.apache.ratis</groupId>
+      <scope>provided</scope>
+    </dependency>
+    <dependency>
+      <artifactId>ratis-common</artifactId>
+      <groupId>org.apache.ratis</groupId>
+      <scope>test</scope>
+      <type>test-jar</type>
+    </dependency>
+    
+    <dependency>
+      <artifactId>ratis-client</artifactId>
+      <groupId>org.apache.ratis</groupId>
+      <scope>provided</scope>
+    </dependency>
+    <dependency>
+      <artifactId>ratis-client</artifactId>
+      <groupId>org.apache.ratis</groupId>
+      <scope>test</scope>
+      <type>test-jar</type>
+    </dependency>
+    
+    <dependency>
+      <groupId>org.slf4j</groupId>
+      <artifactId>slf4j-api</artifactId>
+    </dependency>
+
+    <dependency>
+      <groupId>com.google.guava</groupId>
+      <artifactId>guava</artifactId>
+    </dependency>
+
+    <dependency>
+      <groupId>junit</groupId>
+      <artifactId>junit</artifactId>
+      <scope>test</scope>
+    </dependency>
+    <dependency>
+      <groupId>org.mockito</groupId>
+      <artifactId>mockito-all</artifactId>
+      <scope>test</scope>
+    </dependency>
+  </dependencies>
+</project>

http://git-wip-us.apache.org/repos/asf/incubator-ratis/blob/7e71a2e0/ratis-server/src/main/java/org/apache/ratis/server/RaftServer.java
----------------------------------------------------------------------
diff --git a/ratis-server/src/main/java/org/apache/ratis/server/RaftServer.java b/ratis-server/src/main/java/org/apache/ratis/server/RaftServer.java
new file mode 100644
index 0000000..06967ce
--- /dev/null
+++ b/ratis-server/src/main/java/org/apache/ratis/server/RaftServer.java
@@ -0,0 +1,44 @@
+/**
+ * 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.ratis.server;
+
+import org.apache.ratis.protocol.RaftClientAsynchronousProtocol;
+import org.apache.ratis.protocol.RaftClientProtocol;
+import org.apache.ratis.server.protocol.RaftServerProtocol;
+import org.apache.ratis.statemachine.StateMachine;
+
+import java.io.Closeable;
+
+/** Raft server interface */
+public interface RaftServer extends Closeable, RaftServerProtocol,
+    RaftClientProtocol, RaftClientAsynchronousProtocol {
+  /** @return the server ID. */
+  String getId();
+
+  /** Set server RPC service. */
+  void setServerRpc(RaftServerRpc serverRpc);
+
+  /** Start this server. */
+  void start();
+
+  /**
+   * Returns the StateMachine instance.
+   * @return the StateMachine instance.
+   */
+  StateMachine getStateMachine();
+}

http://git-wip-us.apache.org/repos/asf/incubator-ratis/blob/7e71a2e0/ratis-server/src/main/java/org/apache/ratis/server/RaftServerConfigKeys.java
----------------------------------------------------------------------
diff --git a/ratis-server/src/main/java/org/apache/ratis/server/RaftServerConfigKeys.java b/ratis-server/src/main/java/org/apache/ratis/server/RaftServerConfigKeys.java
new file mode 100644
index 0000000..cdf7651
--- /dev/null
+++ b/ratis-server/src/main/java/org/apache/ratis/server/RaftServerConfigKeys.java
@@ -0,0 +1,150 @@
+/**
+ * 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.ratis.server;
+
+import org.apache.ratis.server.impl.LogAppenderFactory;
+import org.apache.ratis.util.NetUtils;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.net.InetSocketAddress;
+
+public interface RaftServerConfigKeys {
+
+  String PREFIX = "raft.server";
+
+  /** IPC server configurations */
+  interface Ipc {
+    String PREFIX  = RaftServerConfigKeys.PREFIX + ".ipc";
+
+    String ADDRESS_KEY = PREFIX + ".address";
+    int    DEFAULT_PORT = 10718;
+    String ADDRESS_DEFAULT = "0.0.0.0:" + DEFAULT_PORT;
+
+    String HANDLERS_KEY = PREFIX + ".handlers";
+    int    HANDLERS_DEFAULT = 10;
+
+    class Getters {
+      private final Get get;
+
+      Getters(Get get) {
+        this.get = get;
+      }
+
+      public int handlers() {
+        return get.getInt(HANDLERS_KEY, HANDLERS_DEFAULT, 1, null);
+      }
+
+      public InetSocketAddress address() {
+        return get.getInetSocketAddress(ADDRESS_KEY, ADDRESS_DEFAULT);
+      }
+    }
+  }
+
+  String RAFT_SERVER_USE_MEMORY_LOG_KEY = "raft.server.use.memory.log";
+  boolean RAFT_SERVER_USE_MEMORY_LOG_DEFAULT = false;
+
+  String RAFT_SERVER_STORAGE_DIR_KEY = "raft.server.storage.dir";
+  String RAFT_SERVER_STORAGE_DIR_DEFAULT = "file:///tmp/raft-server/";
+
+  String RAFT_SERVER_LOG_APPENDER_FACTORY_CLASS_KEY = "raft.server.log.appender.factory.class";
+  Class<? extends LogAppenderFactory> RAFT_SERVER_LOG_APPENDER_FACTORY_CLASS_DEFAULT
+      = LogAppenderFactory.SynchronousLogAppenderFactory.class;
+
+  /** whether trigger snapshot when log size exceeds limit */
+  String RAFT_SERVER_AUTO_SNAPSHOT_ENABLED_KEY = "raft.server.auto.snapshot.enabled";
+  /** by default let the state machine to decide when to do checkpoint */
+  boolean RAFT_SERVER_AUTO_SNAPSHOT_ENABLED_DEFAULT = false;
+
+  /** log size limit (in number of log entries) that triggers the snapshot */
+  String RAFT_SERVER_SNAPSHOT_TRIGGER_THRESHOLD_KEY = "raft.server.snapshot.trigger.threshold";
+  long RAFT_SERVER_SNAPSHOT_TRIGGER_THRESHOLD_DEFAULT = 400000;
+
+  String RAFT_LOG_SEGMENT_MAX_SIZE_KEY = "raft.log.segment.max.size";
+  long RAFT_LOG_SEGMENT_MAX_SIZE_DEFAULT = 1024L * 1024 * 1024 * 2; // 2GB
+
+  String RAFT_LOG_SEGMENT_PREALLOCATED_SIZE_KEY = "raft.log.segment.preallocated.size";
+  int RAFT_LOG_SEGMENT_PREALLOCATED_SIZE_DEFAULT = 1024 * 1024 * 16; // 16MB
+
+  String RAFT_LOG_WRITE_BUFFER_SIZE_KEY = "raft.log.write.buffer.size";
+  int RAFT_LOG_WRITE_BUFFER_SIZE_DEFAULT = 64 * 1024;
+
+  String RAFT_SNAPSHOT_CHUNK_MAX_SIZE_KEY = "raft.snapshot.chunk.max.size";
+  int RAFT_SNAPSHOT_CHUNK_MAX_SIZE_DEFAULT = 1024 * 1024 * 16;
+
+  String RAFT_LOG_FORCE_SYNC_NUM_KEY = "raft.log.force.sync.num";
+  int RAFT_LOG_FORCE_SYNC_NUM_DEFAULT = 128;
+
+  /** server rpc timeout related */
+  String RAFT_SERVER_RPC_TIMEOUT_MIN_MS_KEY = "raft.server.rpc.timeout.min.ms";
+  int RAFT_SERVER_RPC_TIMEOUT_MIN_MS_DEFAULT = 150;
+
+  String RAFT_SERVER_RPC_TIMEOUT_MAX_MS_KEY = "raft.server.rpc.timeout.max.ms";
+  int RAFT_SERVER_RPC_TIMEOUT_MAX_MS_DEFAULT = 300;
+
+  String RAFT_SERVER_RPC_SLEEP_TIME_MS_KEY = "raft.server.rpc.sleep.time.ms";
+  int RAFT_SERVER_RPC_SLEEP_TIME_MS_DEFAULT = 25;
+
+  /**
+   * When bootstrapping a new peer, If the gap between the match index of the
+   * peer and the leader's latest committed index is less than this gap, we
+   * treat the peer as caught-up.
+   */
+  String RAFT_SERVER_STAGING_CATCHUP_GAP_KEY = "raft.server.staging.catchup.gap";
+  int RAFT_SERVER_STAGING_CATCHUP_GAP_DEFAULT = 1000; // increase this number when write throughput is high
+
+  String RAFT_SERVER_LOG_APPENDER_BUFFER_CAPACITY_KEY = "raft.server.log.appender.buffer.capacity";
+  int RAFT_SERVER_LOG_APPENDER_BUFFER_CAPACITY_DEFAULT = 4 * 1024 * 1024; // 4MB
+
+  String RAFT_SERVER_LOG_APPENDER_BATCH_ENABLED_KEY = "raft.server.log.appender.batch.enabled";
+  boolean RAFT_SERVER_LOG_APPENDER_BATCH_ENABLED_DEFAULT = false;
+
+  /** An utility class to get conf values. */
+  abstract class Get {
+    static Logger LOG = LoggerFactory.getLogger(RaftServerConfigKeys.class);
+
+    private final Ipc.Getters ipc = new Ipc.Getters(this);
+
+    protected abstract int getInt(String key, int defaultValue);
+
+    int getInt(String key, int defaultValue, Integer min, Integer max) {
+      final int value = getInt(key, defaultValue);
+      final String s = key + " = " + value;
+      if (min != null && value < min) {
+        throw new IllegalArgumentException(s + " < min = " + min);
+      }
+      if (max != null && value > max) {
+        throw new IllegalArgumentException(s + " > max = " + max);
+      }
+      LOG.info(s);
+      return value;
+    }
+
+    protected abstract String getTrimmed(String key, String defaultValue);
+
+    InetSocketAddress getInetSocketAddress(String key, String defaultValue) {
+      final String address = getTrimmed(key, defaultValue);
+      LOG.info(key + " = " + address);
+      return NetUtils.createSocketAddr(address);
+    }
+
+    public Ipc.Getters ipc() {
+      return ipc;
+    }
+  }
+}

http://git-wip-us.apache.org/repos/asf/incubator-ratis/blob/7e71a2e0/ratis-server/src/main/java/org/apache/ratis/server/RaftServerRpc.java
----------------------------------------------------------------------
diff --git a/ratis-server/src/main/java/org/apache/ratis/server/RaftServerRpc.java b/ratis-server/src/main/java/org/apache/ratis/server/RaftServerRpc.java
new file mode 100644
index 0000000..5fecce3
--- /dev/null
+++ b/ratis-server/src/main/java/org/apache/ratis/server/RaftServerRpc.java
@@ -0,0 +1,39 @@
+/**
+ * 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.ratis.server;
+
+import org.apache.ratis.protocol.RaftPeer;
+import org.apache.ratis.server.protocol.RaftServerProtocol;
+
+import java.io.Closeable;
+import java.net.InetSocketAddress;
+
+/**
+ * An server-side interface for supporting different RPC implementations
+ * such as Netty, gRPC and Hadoop.
+ */
+public interface RaftServerRpc extends RaftServerProtocol, Closeable {
+  /** Start the RPC service. */
+  void start();
+
+  /** @return the address where this RPC server is listening to. */
+  InetSocketAddress getInetSocketAddress();
+
+  /** add information of the given peers */
+  void addPeers(Iterable<RaftPeer> peers);
+}

http://git-wip-us.apache.org/repos/asf/incubator-ratis/blob/7e71a2e0/ratis-server/src/main/java/org/apache/ratis/server/impl/ConfigurationManager.java
----------------------------------------------------------------------
diff --git a/ratis-server/src/main/java/org/apache/ratis/server/impl/ConfigurationManager.java b/ratis-server/src/main/java/org/apache/ratis/server/impl/ConfigurationManager.java
new file mode 100644
index 0000000..f495c28
--- /dev/null
+++ b/ratis-server/src/main/java/org/apache/ratis/server/impl/ConfigurationManager.java
@@ -0,0 +1,91 @@
+/**
+ * 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.ratis.server.impl;
+
+import com.google.common.annotations.VisibleForTesting;
+import com.google.common.base.Preconditions;
+
+import java.util.*;
+
+/**
+ * Maintain the mappings between log index and corresponding raft configuration.
+ * Initialized when starting the raft peer. The mappings are loaded from the
+ * raft log, and updated while appending/truncating configuration related log
+ * entries.
+ */
+public class ConfigurationManager {
+  private RaftConfiguration initialConf;
+  private final NavigableMap<Long, RaftConfiguration> configurations =
+      new TreeMap<>();
+  /**
+   * The current raft configuration. If configurations is not empty, should be
+   * the last entry of the map. Otherwise is initialConf.
+   */
+  private RaftConfiguration currentConf;
+
+  public ConfigurationManager(RaftConfiguration initialConf) {
+    setInitialConf(initialConf);
+  }
+
+  @VisibleForTesting
+  public synchronized void setInitialConf(RaftConfiguration initialConf) {
+    /**
+     * initialConf should actually be defined as "final". But for tests we want
+     * to change the initial configuration to reflect the correct port binding.
+     */
+    this.initialConf = initialConf;
+    this.currentConf = initialConf;
+  }
+
+  public synchronized void addConfiguration(long logIndex,
+      RaftConfiguration conf) {
+    Preconditions.checkArgument(configurations.isEmpty() ||
+        configurations.lastEntry().getKey() < logIndex);
+    configurations.put(logIndex, conf);
+    this.currentConf = conf;
+  }
+
+  synchronized RaftConfiguration getCurrent() {
+    return currentConf;
+  }
+
+  /**
+   * Remove all the configurations whose log index is >= the given index.
+   * @param index The given index. All the configurations whose log index is >=
+   *              this value will be removed.
+   * @return The configuration with largest log index < the given index.
+   */
+  synchronized RaftConfiguration removeConfigurations(long index) {
+    SortedMap<Long, RaftConfiguration> toRemove = configurations.tailMap(index);
+    for (Iterator<Map.Entry<Long, RaftConfiguration>> iter =
+         toRemove.entrySet().iterator(); iter.hasNext();) {
+      iter.next();
+      iter.remove();
+    }
+    currentConf = configurations.isEmpty() ? initialConf :
+        configurations.lastEntry().getValue();
+    return currentConf;
+  }
+
+  @VisibleForTesting
+  synchronized int numOfConf() {
+    return 1 + configurations.size();
+  }
+
+  // TODO: remove Configuration entries after they are committed
+}

http://git-wip-us.apache.org/repos/asf/incubator-ratis/blob/7e71a2e0/ratis-server/src/main/java/org/apache/ratis/server/impl/FollowerInfo.java
----------------------------------------------------------------------
diff --git a/ratis-server/src/main/java/org/apache/ratis/server/impl/FollowerInfo.java b/ratis-server/src/main/java/org/apache/ratis/server/impl/FollowerInfo.java
new file mode 100644
index 0000000..f72e037
--- /dev/null
+++ b/ratis-server/src/main/java/org/apache/ratis/server/impl/FollowerInfo.java
@@ -0,0 +1,103 @@
+/**
+ * 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.ratis.server.impl;
+
+import java.util.concurrent.atomic.AtomicLong;
+import java.util.concurrent.atomic.AtomicReference;
+
+import org.apache.ratis.protocol.RaftPeer;
+import org.apache.ratis.util.Timestamp;
+
+public class FollowerInfo {
+  private final RaftPeer peer;
+  private final AtomicReference<Timestamp> lastRpcResponseTime;
+  private final AtomicReference<Timestamp> lastRpcSendTime;
+  private long nextIndex;
+  private final AtomicLong matchIndex;
+  private volatile boolean attendVote;
+
+  FollowerInfo(RaftPeer peer, Timestamp lastRpcTime, long nextIndex,
+      boolean attendVote) {
+    this.peer = peer;
+    this.lastRpcResponseTime = new AtomicReference<>(lastRpcTime);
+    this.lastRpcSendTime = new AtomicReference<>(lastRpcTime);
+    this.nextIndex = nextIndex;
+    this.matchIndex = new AtomicLong(0);
+    this.attendVote = attendVote;
+  }
+
+  public void updateMatchIndex(final long matchIndex) {
+    this.matchIndex.set(matchIndex);
+  }
+
+  long getMatchIndex() {
+    return matchIndex.get();
+  }
+
+  public synchronized long getNextIndex() {
+    return nextIndex;
+  }
+
+  public synchronized void updateNextIndex(long i) {
+    nextIndex = i;
+  }
+
+  public synchronized void decreaseNextIndex(long targetIndex) {
+    if (nextIndex > 0) {
+      nextIndex = Math.min(nextIndex - 1, targetIndex);
+    }
+  }
+
+  @Override
+  public String toString() {
+    return peer.getId() + "(next=" + nextIndex + ", match=" + matchIndex + "," +
+        " attendVote=" + attendVote +
+        ", lastRpcSendTime=" + lastRpcSendTime +
+        ", lastRpcResponseTime=" + lastRpcResponseTime + ")";
+  }
+
+  void startAttendVote() {
+    attendVote = true;
+  }
+
+  public boolean isAttendingVote() {
+    return attendVote;
+  }
+
+  public RaftPeer getPeer() {
+    return peer;
+  }
+
+  /** Update lastRpcResponseTime to the current time. */
+  public void updateLastRpcResponseTime() {
+    lastRpcResponseTime.set(new Timestamp());
+  }
+
+  public Timestamp getLastRpcResponseTime() {
+    return lastRpcResponseTime.get();
+  }
+
+  /** Update lastRpcSendTime to the current time. */
+  public void updateLastRpcSendTime() {
+    lastRpcSendTime.set(new Timestamp());
+  }
+
+  public Timestamp getLastRpcTime() {
+    return Timestamp.latest(lastRpcResponseTime.get(), lastRpcSendTime.get());
+  }
+}

http://git-wip-us.apache.org/repos/asf/incubator-ratis/blob/7e71a2e0/ratis-server/src/main/java/org/apache/ratis/server/impl/FollowerState.java
----------------------------------------------------------------------
diff --git a/ratis-server/src/main/java/org/apache/ratis/server/impl/FollowerState.java b/ratis-server/src/main/java/org/apache/ratis/server/impl/FollowerState.java
new file mode 100644
index 0000000..1e57fa2
--- /dev/null
+++ b/ratis-server/src/main/java/org/apache/ratis/server/impl/FollowerState.java
@@ -0,0 +1,91 @@
+/**
+ * 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.ratis.server.impl;
+
+import org.apache.ratis.util.Daemon;
+import org.apache.ratis.util.Timestamp;
+import org.slf4j.Logger;
+
+/**
+ * Used when the peer is a follower. Used to track the election timeout.
+ */
+class FollowerState extends Daemon {
+  static final Logger LOG = RaftServerImpl.LOG;
+
+  private final RaftServerImpl server;
+
+  private volatile Timestamp lastRpcTime = new Timestamp();
+  private volatile boolean monitorRunning = true;
+  private volatile boolean inLogSync = false;
+
+  FollowerState(RaftServerImpl server) {
+    this.server = server;
+  }
+
+  void updateLastRpcTime(boolean inLogSync) {
+    lastRpcTime = new Timestamp();
+    LOG.trace("{} update last rpc time to {}", server.getId(), lastRpcTime);
+    this.inLogSync = inLogSync;
+  }
+
+  Timestamp getLastRpcTime() {
+    return lastRpcTime;
+  }
+
+  boolean shouldWithholdVotes() {
+    return lastRpcTime.elapsedTimeMs() < server.getMinTimeoutMs();
+  }
+
+  void stopRunning() {
+    this.monitorRunning = false;
+  }
+
+  @Override
+  public  void run() {
+    while (monitorRunning && server.isFollower()) {
+      final long electionTimeout = server.getRandomTimeoutMs();
+      try {
+        Thread.sleep(electionTimeout);
+        if (!monitorRunning || !server.isFollower()) {
+          LOG.info("{} heartbeat monitor quit", server.getId());
+          break;
+        }
+        synchronized (server) {
+          if (!inLogSync && lastRpcTime.elapsedTimeMs() >= electionTimeout) {
+            LOG.info("{} changes to CANDIDATE, lastRpcTime:{}, electionTimeout:{}ms",
+                server.getId(), lastRpcTime, electionTimeout);
+            // election timeout, should become a candidate
+            server.changeToCandidate();
+            break;
+          }
+        }
+      } catch (InterruptedException e) {
+        LOG.info(this + " was interrupted: " + e);
+        LOG.trace("TRACE", e);
+        return;
+      } catch (Exception e) {
+        LOG.warn(this + " caught an exception", e);
+      }
+    }
+  }
+
+  @Override
+  public String toString() {
+    return server.getId() + ": " + getClass().getSimpleName();
+  }
+}


[18/54] [abbrv] incubator-ratis git commit: Renamed the packages from raft to ratis in preperation for Apache Incubation - Moved all java packages from org.apache.raft to org.apache.ratis. - Moved native package to org_apache_ratis, and native lib to l

Posted by ji...@apache.org.
http://git-wip-us.apache.org/repos/asf/incubator-ratis/blob/7e71a2e0/ratis-server/src/main/java/org/apache/ratis/server/storage/SegmentedRaftLog.java
----------------------------------------------------------------------
diff --git a/ratis-server/src/main/java/org/apache/ratis/server/storage/SegmentedRaftLog.java b/ratis-server/src/main/java/org/apache/ratis/server/storage/SegmentedRaftLog.java
new file mode 100644
index 0000000..9b2932c
--- /dev/null
+++ b/ratis-server/src/main/java/org/apache/ratis/server/storage/SegmentedRaftLog.java
@@ -0,0 +1,328 @@
+/**
+ * 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.ratis.server.storage;
+
+import static org.apache.ratis.server.RaftServerConfigKeys.RAFT_LOG_SEGMENT_MAX_SIZE_DEFAULT;
+import static org.apache.ratis.server.RaftServerConfigKeys.RAFT_LOG_SEGMENT_MAX_SIZE_KEY;
+
+import java.io.File;
+import java.io.IOException;
+import java.util.Iterator;
+import java.util.List;
+
+import org.apache.commons.io.Charsets;
+import org.apache.ratis.conf.RaftProperties;
+import org.apache.ratis.server.impl.ConfigurationManager;
+import org.apache.ratis.server.impl.RaftServerConstants;
+import org.apache.ratis.server.impl.RaftServerImpl;
+import org.apache.ratis.server.storage.RaftStorageDirectory.LogPathAndIndex;
+import org.apache.ratis.shaded.proto.RaftProtos.LogEntryProto;
+import org.apache.ratis.util.AutoCloseableLock;
+import org.apache.ratis.util.CodeInjectionForTesting;
+
+import com.google.common.annotations.VisibleForTesting;
+import com.google.common.base.Preconditions;
+
+/**
+ * The RaftLog implementation that writes log entries into segmented files in
+ * local disk.
+ *
+ * The max log segment size is 8MB. The real log segment size may not be
+ * exactly equal to this limit. If a log entry's size exceeds 8MB, this entry
+ * will be stored in a single segment.
+ *
+ * There are two types of segments: closed segment and open segment. The former
+ * is named as "log_startindex-endindex", the later is named as
+ * "log_inprogress_startindex".
+ *
+ * There can be multiple closed segments but there is at most one open segment.
+ * When the open segment reaches the size limit, or the log term increases, we
+ * close the open segment and start a new open segment. A closed segment cannot
+ * be appended anymore, but it can be truncated in case that a follower's log is
+ * inconsistent with the current leader.
+ *
+ * Every closed segment should be non-empty, i.e., it should contain at least
+ * one entry.
+ *
+ * There should not be any gap between segments. The first segment may not start
+ * from index 0 since there may be snapshots as log compaction. The last index
+ * in segments should be no smaller than the last index of snapshot, otherwise
+ * we may have hole when append further log.
+ */
+public class SegmentedRaftLog extends RaftLog {
+  static final String HEADER_STR = "RAFTLOG1";
+  static final byte[] HEADER_BYTES = HEADER_STR.getBytes(Charsets.UTF_8);
+
+  /**
+   * I/O task definitions.
+   */
+  static abstract class Task {
+    private boolean done = false;
+
+    synchronized void done() {
+      done = true;
+      notifyAll();
+    }
+
+    synchronized void waitForDone() throws InterruptedException {
+      while (!done) {
+        wait();
+      }
+    }
+
+    abstract void execute() throws IOException;
+
+    abstract long getEndIndex();
+
+    @Override
+    public String toString() {
+      return getClass().getSimpleName() + "-" + getEndIndex();
+    }
+  }
+  private static final ThreadLocal<Task> myTask = new ThreadLocal<>();
+
+  private final RaftStorage storage;
+  private final RaftLogCache cache;
+  private final RaftLogWorker fileLogWorker;
+  private final long segmentMaxSize;
+
+  public SegmentedRaftLog(String selfId, RaftServerImpl server, RaftStorage storage,
+                          long lastIndexInSnapshot, RaftProperties properties) throws IOException {
+    super(selfId);
+    this.storage = storage;
+    this.segmentMaxSize = properties.getLong(RAFT_LOG_SEGMENT_MAX_SIZE_KEY,
+        RAFT_LOG_SEGMENT_MAX_SIZE_DEFAULT);
+    cache = new RaftLogCache();
+    fileLogWorker = new RaftLogWorker(server, storage, properties);
+    lastCommitted.set(lastIndexInSnapshot);
+  }
+
+  @Override
+  public void open(ConfigurationManager confManager, long lastIndexInSnapshot)
+      throws IOException {
+    loadLogSegments(confManager, lastIndexInSnapshot);
+    File openSegmentFile = null;
+    if (cache.getOpenSegment() != null) {
+      openSegmentFile = storage.getStorageDir()
+          .getOpenLogFile(cache.getOpenSegment().getStartIndex());
+    }
+    fileLogWorker.start(Math.max(cache.getEndIndex(), lastIndexInSnapshot),
+        openSegmentFile);
+    super.open(confManager, lastIndexInSnapshot);
+  }
+
+  @Override
+  public long getStartIndex() {
+    return cache.getStartIndex();
+  }
+
+  private void loadLogSegments(ConfigurationManager confManager,
+      long lastIndexInSnapshot) throws IOException {
+    try(AutoCloseableLock writeLock = writeLock()) {
+      List<LogPathAndIndex> paths = storage.getStorageDir().getLogSegmentFiles();
+      for (LogPathAndIndex pi : paths) {
+        LogSegment logSegment = parseLogSegment(pi, confManager);
+        cache.addSegment(logSegment);
+      }
+
+      // if the largest index is smaller than the last index in snapshot, we do
+      // not load the log to avoid holes between log segments. This may happen
+      // when the local I/O worker is too slow to persist log (slower than
+      // committing the log and taking snapshot)
+      if (!cache.isEmpty() && cache.getEndIndex() < lastIndexInSnapshot) {
+        LOG.warn("End log index {} is smaller than last index in snapshot {}",
+            cache.getEndIndex(), lastIndexInSnapshot);
+        cache.clear();
+        // TODO purge all segment files
+      }
+    }
+  }
+
+  private LogSegment parseLogSegment(LogPathAndIndex pi,
+      ConfigurationManager confManager) throws IOException {
+    final boolean isOpen = pi.endIndex == RaftServerConstants.INVALID_LOG_INDEX;
+    return LogSegment.loadSegment(pi.path.toFile(), pi.startIndex, pi.endIndex,
+        isOpen, confManager);
+  }
+
+  @Override
+  public LogEntryProto get(long index) {
+    checkLogState();
+    try(AutoCloseableLock readLock = readLock()) {
+      return cache.getEntry(index);
+    }
+  }
+
+  @Override
+  public LogEntryProto[] getEntries(long startIndex, long endIndex) {
+    checkLogState();
+    try(AutoCloseableLock readLock = readLock()) {
+      return cache.getEntries(startIndex, endIndex);
+    }
+  }
+
+  @Override
+  public LogEntryProto getLastEntry() {
+    checkLogState();
+    try(AutoCloseableLock readLock = readLock()) {
+      return cache.getLastEntry();
+    }
+  }
+
+  /**
+   * The method, along with {@link #appendEntry} and
+   * {@link #append(LogEntryProto...)} need protection of RaftServer's lock.
+   */
+  @Override
+  void truncate(long index) {
+    checkLogState();
+    try(AutoCloseableLock writeLock = writeLock()) {
+      RaftLogCache.TruncationSegments ts = cache.truncate(index);
+      if (ts != null) {
+        Task task = fileLogWorker.truncate(ts);
+        myTask.set(task);
+      }
+    }
+  }
+
+  @Override
+  void appendEntry(LogEntryProto entry) {
+    checkLogState();
+    try(AutoCloseableLock writeLock = writeLock()) {
+      final LogSegment currentOpenSegment = cache.getOpenSegment();
+      if (currentOpenSegment == null) {
+        cache.addSegment(LogSegment.newOpenSegment(entry.getIndex()));
+        fileLogWorker.startLogSegment(getNextIndex());
+      } else if (isSegmentFull(currentOpenSegment, entry)) {
+        cache.rollOpenSegment(true);
+        fileLogWorker.rollLogSegment(currentOpenSegment);
+      } else if (currentOpenSegment.numOfEntries() > 0 &&
+          currentOpenSegment.getLastRecord().entry.getTerm() != entry.getTerm()) {
+        // the term changes
+        final long currentTerm = currentOpenSegment.getLastRecord().entry
+            .getTerm();
+        Preconditions.checkState(currentTerm < entry.getTerm(),
+            "open segment's term %s is larger than the new entry's term %s",
+            currentTerm, entry.getTerm());
+        cache.rollOpenSegment(true);
+        fileLogWorker.rollLogSegment(currentOpenSegment);
+      }
+
+      cache.appendEntry(entry);
+      myTask.set(fileLogWorker.writeLogEntry(entry));
+    }
+  }
+
+  private boolean isSegmentFull(LogSegment segment, LogEntryProto entry) {
+    if (segment.getTotalSize() >= segmentMaxSize) {
+      return true;
+    } else {
+      final long entrySize = LogSegment.getEntrySize(entry);
+      // if entry size is greater than the max segment size, write it directly
+      // into the current segment
+      return entrySize <= segmentMaxSize &&
+          segment.getTotalSize() + entrySize > segmentMaxSize;
+    }
+  }
+
+  @Override
+  public void append(LogEntryProto... entries) {
+    checkLogState();
+    if (entries == null || entries.length == 0) {
+      return;
+    }
+    try(AutoCloseableLock writeLock = writeLock()) {
+      Iterator<LogEntryProto> iter = cache.iterator(entries[0].getIndex());
+      int index = 0;
+      long truncateIndex = -1;
+      for (; iter.hasNext() && index < entries.length; index++) {
+        LogEntryProto storedEntry = iter.next();
+        Preconditions.checkState(
+            storedEntry.getIndex() == entries[index].getIndex(),
+            "The stored entry's index %s is not consistent with" +
+                " the received entries[%s]'s index %s", storedEntry.getIndex(),
+            index, entries[index].getIndex());
+
+        if (storedEntry.getTerm() != entries[index].getTerm()) {
+          // we should truncate from the storedEntry's index
+          truncateIndex = storedEntry.getIndex();
+          break;
+        }
+      }
+      if (truncateIndex != -1) {
+        // truncate from truncateIndex
+        truncate(truncateIndex);
+      }
+      // append from entries[index]
+      for (int i = index; i < entries.length; i++) {
+        appendEntry(entries[i]);
+      }
+    }
+  }
+
+  @Override
+  public void logSync() throws InterruptedException {
+    CodeInjectionForTesting.execute(LOG_SYNC, getSelfId(), null);
+    final Task task = myTask.get();
+    if (task != null) {
+      task.waitForDone();
+    }
+  }
+
+  @Override
+  public long getLatestFlushedIndex() {
+    return fileLogWorker.getFlushedIndex();
+  }
+
+  /**
+   * {@inheritDoc}
+   *
+   * This operation is protected by the RaftServer's lock
+   */
+  @Override
+  public void writeMetadata(long term, String votedFor) throws IOException {
+    storage.getMetaFile().set(term, votedFor);
+  }
+
+  @Override
+  public Metadata loadMetadata() throws IOException {
+    return new Metadata(storage.getMetaFile().getVotedFor(),
+        storage.getMetaFile().getTerm());
+  }
+
+  @Override
+  public void syncWithSnapshot(long lastSnapshotIndex) {
+    fileLogWorker.syncWithSnapshot(lastSnapshotIndex);
+    // TODO purge log files and normal/tmp/corrupt snapshot files
+    // if the last index in snapshot is larger than the index of the last
+    // log entry, we should delete all the log entries and their cache to avoid
+    // gaps between log segments.
+  }
+
+  @Override
+  public void close() throws IOException {
+    super.close();
+    fileLogWorker.close();
+    storage.close();
+  }
+
+  @VisibleForTesting
+  RaftLogCache getRaftLogCache() {
+    return cache;
+  }
+}

http://git-wip-us.apache.org/repos/asf/incubator-ratis/blob/7e71a2e0/ratis-server/src/main/java/org/apache/ratis/server/storage/SnapshotManager.java
----------------------------------------------------------------------
diff --git a/ratis-server/src/main/java/org/apache/ratis/server/storage/SnapshotManager.java b/ratis-server/src/main/java/org/apache/ratis/server/storage/SnapshotManager.java
new file mode 100644
index 0000000..73b2af9
--- /dev/null
+++ b/ratis-server/src/main/java/org/apache/ratis/server/storage/SnapshotManager.java
@@ -0,0 +1,134 @@
+/**
+ * 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.ratis.server.storage;
+
+import java.io.File;
+import java.io.FileOutputStream;
+import java.io.IOException;
+import java.nio.channels.FileChannel;
+
+import org.apache.ratis.io.MD5Hash;
+import org.apache.ratis.shaded.proto.RaftProtos.FileChunkProto;
+import org.apache.ratis.shaded.proto.RaftProtos.InstallSnapshotRequestProto;
+import org.apache.ratis.statemachine.SnapshotInfo;
+import org.apache.ratis.statemachine.StateMachine;
+import org.apache.ratis.util.FileUtils;
+import org.apache.ratis.util.MD5FileUtil;
+import org.apache.ratis.util.RaftUtils;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import com.google.common.base.Preconditions;
+
+/**
+ * Manage snapshots of a raft peer.
+ * TODO: snapshot should be treated as compaction log thus can be merged into
+ *       RaftLog. In this way we can have a unified getLastTermIndex interface.
+ */
+public class SnapshotManager {
+  private static final Logger LOG = LoggerFactory.getLogger(SnapshotManager.class);
+
+  private final RaftStorage storage;
+  private final String selfId;
+
+  public SnapshotManager(RaftStorage storage, String selfId)
+      throws IOException {
+    this.storage = storage;
+    this.selfId = selfId;
+  }
+
+  public void installSnapshot(StateMachine stateMachine,
+      InstallSnapshotRequestProto request) throws IOException {
+    final long lastIncludedIndex = request.getTermIndex().getIndex();
+    final RaftStorageDirectory dir = storage.getStorageDir();
+
+    File tmpDir = dir.getNewTempDir();
+    tmpDir.mkdirs();
+    tmpDir.deleteOnExit();
+
+    LOG.info("Installing snapshot:{}, to tmp dir:{}", request, tmpDir);
+
+    // TODO: Make sure that subsequent requests for the same installSnapshot are coming in order,
+    // and are not lost when whole request cycle is done. Check requestId and requestIndex here
+
+    for (FileChunkProto chunk : request.getFileChunksList()) {
+      SnapshotInfo pi = stateMachine.getLatestSnapshot();
+      if (pi != null && pi.getTermIndex().getIndex() >= lastIncludedIndex) {
+        throw new IOException("There exists snapshot file "
+            + pi.getFiles() + " in " + selfId
+            + " with endIndex >= lastIncludedIndex " + lastIncludedIndex);
+      }
+
+      String fileName = chunk.getFilename(); // this is relative to the root dir
+      // TODO: assumes flat layout inside SM dir
+      File tmpSnapshotFile = new File(tmpDir,
+          new File(dir.getRoot(), fileName).getName());
+
+      FileOutputStream out = null;
+      try {
+        // if offset is 0, delete any existing temp snapshot file if it has the
+        // same last index.
+        if (chunk.getOffset() == 0) {
+          if (tmpSnapshotFile.exists()) {
+            FileUtils.fullyDelete(tmpSnapshotFile);
+          }
+          // create the temp snapshot file and put padding inside
+          out = new FileOutputStream(tmpSnapshotFile);
+        } else {
+          Preconditions.checkState(tmpSnapshotFile.exists());
+          out = new FileOutputStream(tmpSnapshotFile, true);
+          FileChannel fc = out.getChannel();
+          fc.position(chunk.getOffset());
+        }
+
+        // write data to the file
+        out.write(chunk.getData().toByteArray());
+      } finally {
+        RaftUtils.cleanup(null, out);
+      }
+
+      // rename the temp snapshot file if this is the last chunk. also verify
+      // the md5 digest and create the md5 meta-file.
+      if (chunk.getDone()) {
+        final MD5Hash expectedDigest =
+            new MD5Hash(chunk.getFileDigest().toByteArray());
+        // calculate the checksum of the snapshot file and compare it with the
+        // file digest in the request
+        MD5Hash digest = MD5FileUtil.computeMd5ForFile(tmpSnapshotFile);
+        if (!digest.equals(expectedDigest)) {
+          LOG.warn("The snapshot md5 digest {} does not match expected {}",
+              digest, expectedDigest);
+          // rename the temp snapshot file to .corrupt
+//          NativeIO.renameTo(tmpSnapshotFile, // TODO:
+//              dir.getCorruptSnapshotFile(lastIncludedTerm, lastIncludedIndex));
+          throw new IOException("MD5 mismatch for snapshot-" + lastIncludedIndex
+              + " installation");
+        } else {
+          MD5FileUtil.saveMD5File(tmpSnapshotFile, digest);
+        }
+      }
+    }
+
+    if (request.getDone()) {
+      LOG.info("Install snapshot is done, renaming tnp dir:{} to:{}",
+          tmpDir, dir.getStateMachineDir());
+      dir.getStateMachineDir().delete();
+      tmpDir.renameTo(dir.getStateMachineDir());
+    }
+  }
+}

http://git-wip-us.apache.org/repos/asf/incubator-ratis/blob/7e71a2e0/ratis-server/src/main/java/org/apache/ratis/statemachine/BaseStateMachine.java
----------------------------------------------------------------------
diff --git a/ratis-server/src/main/java/org/apache/ratis/statemachine/BaseStateMachine.java b/ratis-server/src/main/java/org/apache/ratis/statemachine/BaseStateMachine.java
new file mode 100644
index 0000000..397a12b
--- /dev/null
+++ b/ratis-server/src/main/java/org/apache/ratis/statemachine/BaseStateMachine.java
@@ -0,0 +1,151 @@
+/**
+ * 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.ratis.statemachine;
+
+import java.io.IOException;
+import java.util.Collection;
+import java.util.concurrent.CompletableFuture;
+
+import org.apache.ratis.conf.RaftProperties;
+import org.apache.ratis.protocol.Message;
+import org.apache.ratis.protocol.RaftClientReply;
+import org.apache.ratis.protocol.RaftClientRequest;
+import org.apache.ratis.server.impl.RaftConfiguration;
+import org.apache.ratis.server.impl.RaftServerConstants;
+import org.apache.ratis.server.storage.RaftStorage;
+import org.apache.ratis.shaded.proto.RaftProtos.SMLogEntryProto;
+import org.apache.ratis.util.LifeCycle;
+
+/**
+ * Base implementation for StateMachines.
+ */
+public class BaseStateMachine implements StateMachine {
+
+  protected RaftProperties properties;
+  protected RaftStorage storage;
+  protected RaftConfiguration raftConf;
+  protected final LifeCycle lifeCycle = new LifeCycle(getClass().getSimpleName());
+
+  @Override
+  public LifeCycle.State getLifeCycleState() {
+    return lifeCycle.getCurrentState();
+  }
+
+  @Override
+  public void initialize(String id, RaftProperties properties, RaftStorage storage)
+      throws IOException {
+    lifeCycle.setName(getClass().getSimpleName() + ":" + id);
+    this.properties = properties;
+    this.storage = storage;
+  }
+
+  @Override
+  public void setRaftConfiguration(RaftConfiguration conf) {
+    this.raftConf = conf;
+  }
+
+  @Override
+  public RaftConfiguration getRaftConfiguration() {
+    return this.raftConf;
+  }
+
+  @Override
+  public SnapshotInfo getLatestSnapshot() {
+    return getStateMachineStorage().getLatestSnapshot();
+  }
+
+  @Override
+  public void notifyNotLeader(Collection<TransactionContext> pendingEntries) throws IOException {
+    // do nothing
+  }
+
+  @Override
+  public void pause() {
+  }
+
+  @Override
+  public void reinitialize(String id, RaftProperties properties, RaftStorage storage)
+      throws IOException {
+  }
+
+  @Override
+  public TransactionContext applyTransactionSerial(TransactionContext trx) throws IOException {
+    return trx;
+  }
+
+  @Override
+  public CompletableFuture<Message> applyTransaction(TransactionContext trx) throws IOException {
+    // return the same message contained in the entry
+    Message msg = () -> trx.getLogEntry().get().getSmLogEntry().getData();
+    return CompletableFuture.completedFuture(msg);
+  }
+
+  @Override
+  public long takeSnapshot() throws IOException {
+    return RaftServerConstants.INVALID_LOG_INDEX;
+  }
+
+  @Override
+  public StateMachineStorage getStateMachineStorage() {
+    return new StateMachineStorage() {
+      @Override
+      public void init(RaftStorage raftStorage) throws IOException {
+      }
+
+      @Override
+      public SnapshotInfo getLatestSnapshot() {
+        return null;
+      }
+
+      @Override
+      public void format() throws IOException {
+      }
+    };
+  }
+
+  @Override
+  public CompletableFuture<RaftClientReply> query(
+      RaftClientRequest request) {
+    return null;
+  }
+
+  @Override
+  public TransactionContext startTransaction(RaftClientRequest request)
+      throws IOException {
+    return new TransactionContext(this, request,
+        SMLogEntryProto.newBuilder()
+            .setData(request.getMessage().getContent())
+            .build());
+  }
+
+  @Override
+  public TransactionContext cancelTransaction(TransactionContext trx) throws IOException {
+    return trx;
+  }
+
+  @Override
+  public TransactionContext preAppendTransaction(TransactionContext trx) throws IOException {
+    return trx;
+  }
+
+  @Override
+  public void close() throws IOException {
+    // do nothing
+  }
+}

http://git-wip-us.apache.org/repos/asf/incubator-ratis/blob/7e71a2e0/ratis-server/src/main/java/org/apache/ratis/statemachine/FileListSnapshotInfo.java
----------------------------------------------------------------------
diff --git a/ratis-server/src/main/java/org/apache/ratis/statemachine/FileListSnapshotInfo.java b/ratis-server/src/main/java/org/apache/ratis/statemachine/FileListSnapshotInfo.java
new file mode 100644
index 0000000..1858603
--- /dev/null
+++ b/ratis-server/src/main/java/org/apache/ratis/statemachine/FileListSnapshotInfo.java
@@ -0,0 +1,64 @@
+/**
+ * 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.ratis.statemachine;
+
+import java.util.Collections;
+import java.util.List;
+
+import org.apache.ratis.server.protocol.TermIndex;
+import org.apache.ratis.server.storage.FileInfo;
+
+/**
+ * Each snapshot has a list of files.
+ *
+ * The objects of this class are immutable.
+ */
+public class FileListSnapshotInfo implements SnapshotInfo {
+  private final TermIndex termIndex;
+  private final List<FileInfo> files;
+
+  public FileListSnapshotInfo(List<FileInfo> files, long term, long index) {
+    this.termIndex = TermIndex.newTermIndex(term, index);
+    this.files = Collections.unmodifiableList(files);
+  }
+
+  @Override
+  public TermIndex getTermIndex() {
+    return termIndex;
+  }
+
+  @Override
+  public long getTerm() {
+    return termIndex.getTerm();
+  }
+
+  @Override
+  public long getIndex() {
+    return termIndex.getIndex();
+  }
+
+  @Override
+  public List<FileInfo> getFiles() {
+    return files;
+  }
+
+  @Override
+  public String toString() {
+    return termIndex + ":" + files;
+  }
+}

http://git-wip-us.apache.org/repos/asf/incubator-ratis/blob/7e71a2e0/ratis-server/src/main/java/org/apache/ratis/statemachine/SimpleStateMachineStorage.java
----------------------------------------------------------------------
diff --git a/ratis-server/src/main/java/org/apache/ratis/statemachine/SimpleStateMachineStorage.java b/ratis-server/src/main/java/org/apache/ratis/statemachine/SimpleStateMachineStorage.java
new file mode 100644
index 0000000..d417db7
--- /dev/null
+++ b/ratis-server/src/main/java/org/apache/ratis/statemachine/SimpleStateMachineStorage.java
@@ -0,0 +1,135 @@
+/**
+ * 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.ratis.statemachine;
+
+
+import com.google.common.annotations.VisibleForTesting;
+
+import org.apache.ratis.io.MD5Hash;
+import org.apache.ratis.server.protocol.TermIndex;
+import org.apache.ratis.server.storage.FileInfo;
+import org.apache.ratis.server.storage.RaftStorage;
+import org.apache.ratis.util.AtomicFileOutputStream;
+import org.apache.ratis.util.MD5FileUtil;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.io.File;
+import java.io.IOException;
+import java.nio.file.DirectoryStream;
+import java.nio.file.Files;
+import java.nio.file.Path;
+import java.util.regex.Matcher;
+import java.util.regex.Pattern;
+
+/**
+ * A StateMachineStorage that stores the snapshot in a single file.
+ */
+public class SimpleStateMachineStorage implements StateMachineStorage {
+
+  private static final Logger LOG = LoggerFactory.getLogger(SimpleStateMachineStorage.class);
+
+  static final String SNAPSHOT_FILE_PREFIX = "snapshot";
+  static final String CORRUPT_SNAPSHOT_FILE_SUFFIX = ".corrupt";
+  /** snapshot.term_index */
+  static final Pattern SNAPSHOT_REGEX =
+      Pattern.compile(SNAPSHOT_FILE_PREFIX + "\\.(\\d+)_(\\d+)");
+
+  private RaftStorage raftStorage;
+  private File smDir = null;
+
+  private volatile SingleFileSnapshotInfo currentSnapshot = null;
+
+  public void init(RaftStorage raftStorage) throws IOException {
+    this.raftStorage = raftStorage;
+    this.smDir = raftStorage.getStorageDir().getStateMachineDir();
+    loadLatestSnapshot();
+  }
+
+  @Override
+  public void format() throws IOException {
+    // TODO
+  }
+
+  @VisibleForTesting
+  public static TermIndex getTermIndexFromSnapshotFile(File file) {
+    final String name = file.getName();
+    final Matcher m = SNAPSHOT_REGEX.matcher(name);
+    if (!m.matches()) {
+      throw new IllegalArgumentException("File \"" + file
+          + "\" does not match snapshot file name pattern \""
+          + SNAPSHOT_REGEX + "\"");
+    }
+    final long term = Long.parseLong(m.group(1));
+    final long index = Long.parseLong(m.group(2));
+    return TermIndex.newTermIndex(term, index);
+  }
+
+  protected static String getTmpSnapshotFileName(long term, long endIndex) {
+    return getSnapshotFileName(term, endIndex) + AtomicFileOutputStream.TMP_EXTENSION;
+  }
+
+  protected static String getCorruptSnapshotFileName(long term, long endIndex) {
+    return getSnapshotFileName(term, endIndex) + CORRUPT_SNAPSHOT_FILE_SUFFIX;
+  }
+
+  public File getSnapshotFile(long term, long endIndex) {
+    return new File(smDir, getSnapshotFileName(term, endIndex));
+  }
+
+  protected File getTmpSnapshotFile(long term, long endIndex) {
+    return new File(smDir, getTmpSnapshotFileName(term, endIndex));
+  }
+
+  protected File getCorruptSnapshotFile(long term, long endIndex) {
+    return new File(smDir, getCorruptSnapshotFileName(term, endIndex));
+  }
+
+  public SingleFileSnapshotInfo findLatestSnapshot() throws IOException {
+    SingleFileSnapshotInfo latest = null;
+    try (DirectoryStream<Path> stream =
+             Files.newDirectoryStream(smDir.toPath())) {
+      for (Path path : stream) {
+        Matcher matcher = SNAPSHOT_REGEX.matcher(path.getFileName().toString());
+        if (matcher.matches()) {
+          final long endIndex = Long.parseLong(matcher.group(2));
+          if (latest == null || endIndex > latest.getIndex()) {
+            final long term = Long.parseLong(matcher.group(1));
+            MD5Hash fileDigest = MD5FileUtil.readStoredMd5ForFile(path.toFile());
+            final FileInfo fileInfo = new FileInfo(path, fileDigest);
+            latest = new SingleFileSnapshotInfo(fileInfo, term, endIndex);
+          }
+        }
+      }
+    }
+    return latest;
+  }
+
+  public void loadLatestSnapshot() throws IOException {
+    this.currentSnapshot = findLatestSnapshot();
+  }
+
+  public static String getSnapshotFileName(long term, long endIndex) {
+    return SNAPSHOT_FILE_PREFIX + "." + term + "_" + endIndex;
+  }
+
+  @Override
+  public SingleFileSnapshotInfo getLatestSnapshot() {
+    return currentSnapshot;
+  }
+}

http://git-wip-us.apache.org/repos/asf/incubator-ratis/blob/7e71a2e0/ratis-server/src/main/java/org/apache/ratis/statemachine/SingleFileSnapshotInfo.java
----------------------------------------------------------------------
diff --git a/ratis-server/src/main/java/org/apache/ratis/statemachine/SingleFileSnapshotInfo.java b/ratis-server/src/main/java/org/apache/ratis/statemachine/SingleFileSnapshotInfo.java
new file mode 100644
index 0000000..5bca2c9
--- /dev/null
+++ b/ratis-server/src/main/java/org/apache/ratis/statemachine/SingleFileSnapshotInfo.java
@@ -0,0 +1,38 @@
+/**
+ * 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.ratis.statemachine;
+
+import java.util.Arrays;
+
+import org.apache.ratis.server.storage.FileInfo;
+
+/**
+ * Each snapshot only has a single file.
+ *
+ * The objects of this class are immutable.
+ */
+public class SingleFileSnapshotInfo extends FileListSnapshotInfo {
+  public SingleFileSnapshotInfo(FileInfo fileInfo, long term, long endIndex) {
+    super(Arrays.asList(fileInfo), term, endIndex);
+  }
+
+  /** @return the file associated with the snapshot. */
+  public FileInfo getFile() {
+    return getFiles().get(0);
+  }
+}

http://git-wip-us.apache.org/repos/asf/incubator-ratis/blob/7e71a2e0/ratis-server/src/main/java/org/apache/ratis/statemachine/SnapshotInfo.java
----------------------------------------------------------------------
diff --git a/ratis-server/src/main/java/org/apache/ratis/statemachine/SnapshotInfo.java b/ratis-server/src/main/java/org/apache/ratis/statemachine/SnapshotInfo.java
new file mode 100644
index 0000000..f0aadd9
--- /dev/null
+++ b/ratis-server/src/main/java/org/apache/ratis/statemachine/SnapshotInfo.java
@@ -0,0 +1,58 @@
+/**
+ * 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.ratis.statemachine;
+
+import java.util.List;
+
+import org.apache.ratis.server.protocol.TermIndex;
+import org.apache.ratis.server.storage.FileInfo;
+
+/**
+ * SnapshotInfo represents a durable state by the state machine. The state machine implementation is
+ * responsible for the layout of the snapshot files as well as making the data durable. Latest term,
+ * latest index, and the raft configuration must be saved together with any data files in the
+ * snapshot.
+ */
+public interface SnapshotInfo {
+
+  /**
+   * Returns the term and index corresponding to this snapshot.
+   * @return The term and index corresponding to this snapshot.
+   */
+  TermIndex getTermIndex();
+
+  /**
+   * Returns the term corresponding to this snapshot.
+   * @return The term corresponding to this snapshot.
+   */
+  long getTerm();
+
+  /**
+   * Returns the index corresponding to this snapshot.
+   * @return The index corresponding to this snapshot.
+   */
+  long getIndex();
+
+  /**
+   * Returns a list of files corresponding to this snapshot. This list should include all
+   * the files that the state machine keeps in its data directory. This list of files will be
+   * copied as to other replicas in install snapshot RPCs.
+   * @return a list of Files corresponding to the this snapshot.
+   */
+  List<FileInfo> getFiles();
+}

http://git-wip-us.apache.org/repos/asf/incubator-ratis/blob/7e71a2e0/ratis-server/src/main/java/org/apache/ratis/statemachine/StateMachine.java
----------------------------------------------------------------------
diff --git a/ratis-server/src/main/java/org/apache/ratis/statemachine/StateMachine.java b/ratis-server/src/main/java/org/apache/ratis/statemachine/StateMachine.java
new file mode 100644
index 0000000..e377aa7
--- /dev/null
+++ b/ratis-server/src/main/java/org/apache/ratis/statemachine/StateMachine.java
@@ -0,0 +1,168 @@
+/**
+ * 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.ratis.statemachine;
+
+import org.apache.ratis.conf.RaftProperties;
+import org.apache.ratis.protocol.Message;
+import org.apache.ratis.protocol.RaftClientReply;
+import org.apache.ratis.protocol.RaftClientRequest;
+import org.apache.ratis.server.RaftServerConfigKeys;
+import org.apache.ratis.server.impl.RaftConfiguration;
+import org.apache.ratis.server.storage.RaftStorage;
+import org.apache.ratis.util.LifeCycle;
+
+import java.io.Closeable;
+import java.io.IOException;
+import java.util.Collection;
+import java.util.concurrent.CompletableFuture;
+
+/**
+ * StateMachine is the entry point for the custom implementation of replicated state as defined in
+ * the "State Machine Approach" in the literature
+ * (see https://en.wikipedia.org/wiki/State_machine_replication).
+ */
+public interface StateMachine extends Closeable {
+  /**
+   * Initializes the State Machine with the given properties and storage. The state machine is
+   * responsible reading the latest snapshot from the file system (if any) and initialize itself
+   * with the latest term and index there including all the edits.
+   */
+  void initialize(String id, RaftProperties properties, RaftStorage storage)
+      throws IOException;
+
+  /**
+   * Returns the lifecycle state for this StateMachine.
+   * @return the lifecycle state.
+   */
+  LifeCycle.State getLifeCycleState();
+
+  /**
+   * Pauses the state machine. On return, the state machine should have closed all open files so
+   * that a new snapshot can be installed.
+   */
+  void pause();
+
+  /**
+   * Re-initializes the State Machine in PAUSED state with the given properties and storage. The
+   * state machine is responsible reading the latest snapshot from the file system (if any) and
+   * initialize itself with the latest term and index there including all the edits.
+   */
+  void reinitialize(String id, RaftProperties properties, RaftStorage storage)
+      throws IOException;
+
+  /**
+   * Dump the in-memory state into a snapshot file in the RaftStorage. The
+   * StateMachine implementation can decide 1) its own snapshot format, 2) when
+   * a snapshot is taken, and 3) how the snapshot is taken (e.g., whether the
+   * snapshot blocks the state machine, and whether to purge log entries after
+   * a snapshot is done).
+   *
+   * In the meanwhile, when the size of raft log outside of the latest snapshot
+   * exceeds certain threshold, the RaftServer may choose to trigger a snapshot
+   * if {@link RaftServerConfigKeys#RAFT_SERVER_AUTO_SNAPSHOT_ENABLED_KEY} is
+   * enabled.
+   *
+   * The snapshot should include the latest raft configuration.
+   *
+   * @return the largest index of the log entry that has been applied to the
+   *         state machine and also included in the snapshot. Note the log purge
+   *         should be handled separately.
+   */
+  // TODO: refactor this
+  long takeSnapshot() throws IOException;
+
+  /**
+   * Record the RaftConfiguration in the state machine. The RaftConfiguration
+   * should also be stored in the snapshot.
+   */
+  void setRaftConfiguration(RaftConfiguration conf);
+
+  /**
+   * @return the latest raft configuration recorded in the state machine.
+   */
+  RaftConfiguration getRaftConfiguration();
+
+  /**
+   * @return StateMachineStorage to interact with the durability guarantees provided by the
+   * state machine.
+   */
+  StateMachineStorage getStateMachineStorage();
+
+  /**
+   * Returns the information for the latest durable snapshot.
+   */
+  SnapshotInfo getLatestSnapshot();
+
+  /**
+   * Query the state machine. The request must be read-only.
+   * TODO: extend RaftClientRequest to have a read-only request subclass.
+   */
+  CompletableFuture<RaftClientReply> query(RaftClientRequest request);
+
+  /**
+   * Validate/pre-process the incoming update request in the state machine.
+   * @return the content to be written to the log entry. Null means the request
+   * should be rejected.
+   * @throws IOException thrown by the state machine while validation
+   */
+  TransactionContext startTransaction(RaftClientRequest request)
+      throws IOException;
+
+  /**
+   * This is called before the transaction passed from the StateMachine is appended to the raft log.
+   * This method will be called from log append and having the same strict serial order that the
+   * transactions will have in the RAFT log. Since this is called serially in the critical path of
+   * log append, it is important to do only required operations here.
+   * @return The Transaction context.
+   */
+  TransactionContext preAppendTransaction(TransactionContext trx) throws IOException;
+
+  /**
+   * Called to notify the state machine that the Transaction passed cannot be appended (or synced).
+   * The exception field will indicate whether there was an exception or not.
+   * @param trx the transaction to cancel
+   * @return cancelled transaction
+   */
+  TransactionContext cancelTransaction(TransactionContext trx) throws IOException;
+
+  /**
+   * Called for transactions that have been committed to the RAFT log. This step is called
+   * sequentially in strict serial order that the transactions have been committed in the log.
+   * The SM is expected to do only necessary work, and leave the actual apply operation to the
+   * applyTransaction calls that can happen concurrently.
+   * @param trx the transaction state including the log entry that has been committed to a quorum
+   *            of the raft peers
+   * @return The Transaction context.
+   */
+  TransactionContext applyTransactionSerial(TransactionContext trx) throws IOException;
+
+  /**
+   * Apply a committed log entry to the state machine. This method can be called concurrently with
+   * the other calls, and there is no guarantee that the calls will be ordered according to the
+   * log commit order.
+   * @param trx the transaction state including the log entry that has been committed to a quorum
+   *            of the raft peers
+   */
+  // TODO: We do not need to return CompletableFuture
+  CompletableFuture<Message> applyTransaction(TransactionContext trx) throws IOException;
+
+  /**
+   * Notify the state machine that the raft peer is no longer leader.
+   */
+  void notifyNotLeader(Collection<TransactionContext> pendingEntries) throws IOException;
+}

http://git-wip-us.apache.org/repos/asf/incubator-ratis/blob/7e71a2e0/ratis-server/src/main/java/org/apache/ratis/statemachine/StateMachineStorage.java
----------------------------------------------------------------------
diff --git a/ratis-server/src/main/java/org/apache/ratis/statemachine/StateMachineStorage.java b/ratis-server/src/main/java/org/apache/ratis/statemachine/StateMachineStorage.java
new file mode 100644
index 0000000..4f7951a
--- /dev/null
+++ b/ratis-server/src/main/java/org/apache/ratis/statemachine/StateMachineStorage.java
@@ -0,0 +1,40 @@
+/**
+ * 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.ratis.statemachine;
+
+import java.io.IOException;
+
+import org.apache.ratis.server.storage.RaftStorage;
+
+public interface StateMachineStorage {
+
+  void init(RaftStorage raftStorage) throws IOException;
+
+  /**
+   * Returns the information for the latest durable snapshot.
+   */
+  SnapshotInfo getLatestSnapshot();
+
+  // TODO: StateMachine can decide to compact the files independently of concurrent install snapshot
+  // etc requests. We should have ref counting for the SnapshotInfo with a release mechanism
+  // so that raft server will release the files after the snapshot file copy in case a compaction
+  // is waiting for deleting these files.
+
+  void format() throws IOException;
+
+}

http://git-wip-us.apache.org/repos/asf/incubator-ratis/blob/7e71a2e0/ratis-server/src/main/java/org/apache/ratis/statemachine/TransactionContext.java
----------------------------------------------------------------------
diff --git a/ratis-server/src/main/java/org/apache/ratis/statemachine/TransactionContext.java b/ratis-server/src/main/java/org/apache/ratis/statemachine/TransactionContext.java
new file mode 100644
index 0000000..81bea45
--- /dev/null
+++ b/ratis-server/src/main/java/org/apache/ratis/statemachine/TransactionContext.java
@@ -0,0 +1,210 @@
+/**
+ * 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.ratis.statemachine;
+
+import java.io.IOException;
+import java.util.Collection;
+import java.util.Optional;
+
+import org.apache.ratis.protocol.RaftClientRequest;
+import org.apache.ratis.shaded.proto.RaftProtos.LogEntryProto;
+import org.apache.ratis.shaded.proto.RaftProtos.SMLogEntryProto;
+
+/**
+ * Context for a transaction.
+ * The transaction might have originated from a client request, or it
+ * maybe coming from another replica of the state machine through the RAFT log.
+ * {@link TransactionContext} can be created from
+ * either the {@link StateMachine} or the state machine updater.
+ *
+ * In the first case, the {@link StateMachine} is a leader. When it receives
+ * a {@link StateMachine#startTransaction(RaftClientRequest)} request, it returns
+ * a {@link TransactionContext} with the changes from the {@link StateMachine}.
+ * The same context will be passed back to the {@link StateMachine}
+ * via the {@link StateMachine#applyTransaction(TransactionContext)} call
+ * or the {@link StateMachine#notifyNotLeader(Collection)} call.
+ *
+ * In the second case, the {@link StateMachine} is a follower.
+ * The {@link TransactionContext} will be a committed entry coming from
+ * the RAFT log from the leader.
+ */
+public class TransactionContext {
+
+  /** The {@link StateMachine} that originated the transaction. */
+  private final StateMachine stateMachine;
+
+  /** Original request from the client */
+  private Optional<RaftClientRequest> clientRequest = Optional.empty();
+
+  /** Exception from the {@link StateMachine} or from the log */
+  private Optional<Exception> exception = Optional.empty();
+
+  /** Data from the {@link StateMachine} */
+  private Optional<SMLogEntryProto> smLogEntryProto = Optional.empty();
+
+  /**
+   * Context specific to the state machine.
+   * The {@link StateMachine} can use this object to carry state between
+   * {@link StateMachine#startTransaction(RaftClientRequest)} and
+   * {@link StateMachine#applyTransaction(TransactionContext)}.
+   */
+  private Optional<Object> stateMachineContext = Optional.empty();
+
+  /**
+   * Whether to commit the transaction to the RAFT Log.
+   * In some cases the {@link StateMachine} may want to indicate
+   * that the transaction should not be committed
+   */
+  private boolean shouldCommit = true;
+
+  /** Committed LogEntry. */
+  private Optional<LogEntryProto> logEntry = Optional.empty();
+
+  private TransactionContext(StateMachine stateMachine) {
+    this.stateMachine = stateMachine;
+  }
+
+  /** The same as this(stateMachine, clientRequest, smLogEntryProto, null). */
+  public TransactionContext(
+      StateMachine stateMachine, RaftClientRequest clientRequest,
+      SMLogEntryProto smLogEntryProto) {
+    this(stateMachine, clientRequest, smLogEntryProto, null);
+  }
+
+  /**
+   * Construct a {@link TransactionContext} from a client request.
+   * Used by the state machine to start a transaction
+   * and send the Log entry representing the transaction data
+   * to be applied to the raft log.
+   */
+  public TransactionContext(
+      StateMachine stateMachine, RaftClientRequest clientRequest,
+      SMLogEntryProto smLogEntryProto, Object stateMachineContext) {
+    this(stateMachine);
+    this.clientRequest = Optional.of(clientRequest);
+    this.smLogEntryProto = Optional.ofNullable(smLogEntryProto);
+    this.stateMachineContext = Optional.ofNullable(stateMachineContext);
+  }
+
+  /** The same as this(stateMachine, clientRequest, exception, null). */
+  public TransactionContext(
+      StateMachine stateMachine, RaftClientRequest clientRequest,
+      Exception exception) {
+    this(stateMachine, clientRequest, exception, null);
+  }
+
+  /**
+   * Construct a {@link TransactionContext} from a client request to signal
+   * an exception so that the RAFT server will fail the request on behalf
+   * of the {@link StateMachine}.
+   */
+  public TransactionContext(
+      StateMachine stateMachine, RaftClientRequest clientRequest,
+      Exception exception, Object stateMachineContext) {
+    this(stateMachine);
+    this.clientRequest = Optional.of(clientRequest);
+    this.exception = Optional.of(exception);
+    this.stateMachineContext = Optional.ofNullable(stateMachineContext);
+  }
+
+  /**
+   * Construct a {@link TransactionContext} from a {@link LogEntryProto}.
+   * Used by followers for applying committed entries to the state machine.
+   * @param logEntry the log entry to be applied
+   */
+  public TransactionContext(StateMachine stateMachine, LogEntryProto logEntry) {
+    this(stateMachine);
+    this.smLogEntryProto = Optional.of(logEntry.getSmLogEntry());
+    this.logEntry = Optional.of(logEntry);
+  }
+
+  public Optional<RaftClientRequest> getClientRequest() {
+    return this.clientRequest;
+  }
+
+  public Optional<SMLogEntryProto> getSMLogEntry() {
+    return this.smLogEntryProto;
+  }
+
+  public Optional<Exception> getException() {
+    return this.exception;
+  }
+
+  public TransactionContext setStateMachineContext(Object stateMachineContext) {
+    this.stateMachineContext = Optional.ofNullable(stateMachineContext);
+    return this;
+  }
+
+  public Optional<Object> getStateMachineContext() {
+    return stateMachineContext;
+  }
+
+  public TransactionContext setLogEntry(LogEntryProto logEntry) {
+    this.logEntry = Optional.of(logEntry);
+    return this;
+  }
+
+  public TransactionContext setSmLogEntryProto(SMLogEntryProto smLogEntryProto) {
+    this.smLogEntryProto = Optional.of(smLogEntryProto);
+    return this;
+  }
+
+  public Optional<LogEntryProto> getLogEntry() {
+    return logEntry;
+  }
+
+  private TransactionContext setException(IOException ioe) {
+    assert !this.exception.isPresent();
+    this.exception = Optional.of(ioe);
+    return this;
+  }
+
+  public TransactionContext setShouldCommit(boolean shouldCommit) {
+    this.shouldCommit = shouldCommit;
+    return this;
+  }
+
+  public boolean shouldCommit() {
+    // TODO: Hook this up in the server to bypass the RAFT Log and send back a response to client
+    return this.shouldCommit;
+  }
+
+  // proxy StateMachine methods. We do not want to expose the SM to the RaftLog
+
+  /**
+   * This is called before the transaction passed from the StateMachine is appended to the raft log.
+   * This method will be called from log append and having the same strict serial order that the
+   * Transactions will have in the RAFT log. Since this is called serially in the critical path of
+   * log append, it is important to do only required operations here.
+   * @return The Transaction context.
+   */
+  public TransactionContext preAppendTransaction() throws IOException {
+    return stateMachine.preAppendTransaction(this);
+  }
+
+  /**
+   * Called to notify the state machine that the Transaction passed cannot be appended (or synced).
+   * The exception field will indicate whether there was an exception or not.
+   * @return cancelled transaction
+   */
+  public TransactionContext cancelTransaction() throws IOException {
+    // TODO: This is not called from Raft server / log yet. When an IOException happens, we should
+    // call this to let the SM know that Transaction cannot be synced
+    return stateMachine.cancelTransaction(this);
+  }
+}

http://git-wip-us.apache.org/repos/asf/incubator-ratis/blob/7e71a2e0/ratis-server/src/test/java/org/apache/ratis/MiniRaftCluster.java
----------------------------------------------------------------------
diff --git a/ratis-server/src/test/java/org/apache/ratis/MiniRaftCluster.java b/ratis-server/src/test/java/org/apache/ratis/MiniRaftCluster.java
new file mode 100644
index 0000000..60cbb9c
--- /dev/null
+++ b/ratis-server/src/test/java/org/apache/ratis/MiniRaftCluster.java
@@ -0,0 +1,445 @@
+/**
+ * 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.ratis;
+
+import com.google.common.base.Preconditions;
+
+import org.apache.ratis.client.RaftClient;
+import org.apache.ratis.client.RaftClientRequestSender;
+import org.apache.ratis.client.impl.RaftClientImpl;
+import org.apache.ratis.conf.RaftProperties;
+import org.apache.ratis.protocol.RaftPeer;
+import org.apache.ratis.server.RaftServerConfigKeys;
+import org.apache.ratis.server.RaftServerRpc;
+import org.apache.ratis.server.impl.DelayLocalExecutionInjection;
+import org.apache.ratis.server.impl.RaftConfiguration;
+import org.apache.ratis.server.impl.RaftServerImpl;
+import org.apache.ratis.server.storage.MemoryRaftLog;
+import org.apache.ratis.server.storage.RaftLog;
+import org.apache.ratis.statemachine.BaseStateMachine;
+import org.apache.ratis.statemachine.StateMachine;
+import org.apache.ratis.util.ExitUtils;
+import org.apache.ratis.util.FileUtils;
+import org.apache.ratis.util.RaftUtils;
+import org.junit.Assert;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import static org.apache.ratis.server.RaftServerConfigKeys.RAFT_SERVER_RPC_TIMEOUT_MIN_MS_DEFAULT;
+
+import java.io.File;
+import java.io.IOException;
+import java.util.*;
+import java.util.stream.Collectors;
+
+public abstract class MiniRaftCluster {
+  public static final Logger LOG = LoggerFactory.getLogger(MiniRaftCluster.class);
+  public static final DelayLocalExecutionInjection logSyncDelay =
+      new DelayLocalExecutionInjection(RaftLog.LOG_SYNC);
+
+  public static final String CLASS_NAME = MiniRaftCluster.class.getSimpleName();
+  public static final String STATEMACHINE_CLASS_KEY = CLASS_NAME + ".statemachine.class";
+  public static final Class<? extends StateMachine> STATEMACHINE_CLASS_DEFAULT = BaseStateMachine.class;
+
+  public static abstract class Factory<CLUSTER extends MiniRaftCluster> {
+    public abstract CLUSTER newCluster(
+        String[] ids, RaftProperties prop, boolean formatted)
+        throws IOException;
+
+    public CLUSTER newCluster(
+        int numServer, RaftProperties prop, boolean formatted)
+        throws IOException {
+      return newCluster(generateIds(numServer, 0), prop, formatted);
+    }
+  }
+
+  public static abstract class RpcBase extends MiniRaftCluster {
+    public RpcBase(String[] ids, RaftProperties properties, boolean formatted) {
+      super(ids, properties, formatted);
+    }
+
+    protected abstract RaftServerImpl setPeerRpc(RaftPeer peer) throws IOException;
+
+    @Override
+    protected void setPeerRpc() throws IOException {
+      for (RaftPeer p : conf.getPeers()) {
+        setPeerRpc(p);
+      }
+    }
+
+    @Override
+    public void restartServer(String id, boolean format) throws IOException {
+      super.restartServer(id, format);
+      setPeerRpc(conf.getPeer(id)).start();
+    }
+
+    @Override
+    public void setBlockRequestsFrom(String src, boolean block) {
+      RaftTestUtil.setBlockRequestsFrom(src, block);
+    }
+  }
+
+  public static class PeerChanges {
+    public final RaftPeer[] allPeersInNewConf;
+    public final RaftPeer[] newPeers;
+    public final RaftPeer[] removedPeers;
+
+    public PeerChanges(RaftPeer[] all, RaftPeer[] newPeers, RaftPeer[] removed) {
+      this.allPeersInNewConf = all;
+      this.newPeers = newPeers;
+      this.removedPeers = removed;
+    }
+  }
+
+  public static RaftConfiguration initConfiguration(String[] ids) {
+    return RaftConfiguration.newBuilder()
+        .setConf(Arrays.stream(ids).map(RaftPeer::new).collect(Collectors.toList()))
+        .build();
+  }
+
+  private static String getBaseDirectory() {
+    return System.getProperty("test.build.data", "target/test/data") + "/raft/";
+  }
+
+  private static void formatDir(String dirStr) {
+    final File serverDir = new File(dirStr);
+    Preconditions.checkState(FileUtils.fullyDelete(serverDir),
+        "Failed to format directory %s", dirStr);
+    LOG.info("Formatted directory {}", dirStr);
+  }
+
+  public static String[] generateIds(int numServers, int base) {
+    String[] ids = new String[numServers];
+    for (int i = 0; i < numServers; i++) {
+      ids[i] = "s" + (i + base);
+    }
+    return ids;
+  }
+
+  protected RaftConfiguration conf;
+  protected final RaftProperties properties;
+  private final String testBaseDir;
+  protected final Map<String, RaftServerImpl> servers =
+      Collections.synchronizedMap(new LinkedHashMap<>());
+
+  public MiniRaftCluster(String[] ids, RaftProperties properties,
+      boolean formatted) {
+    this.conf = initConfiguration(ids);
+    this.properties = new RaftProperties(properties);
+    this.testBaseDir = getBaseDirectory();
+
+    conf.getPeers().forEach(
+        p -> servers.put(p.getId(), newRaftServer(p.getId(), conf, formatted)));
+
+    ExitUtils.disableSystemExit();
+  }
+
+  protected <RPC extends  RaftServerRpc> void init(Map<RaftPeer, RPC> peers) {
+    LOG.info("peers = " + peers.keySet());
+    conf = RaftConfiguration.newBuilder().setConf(peers.keySet()).build();
+    for (Map.Entry<RaftPeer, RPC> entry : peers.entrySet()) {
+      final RaftServerImpl server = servers.get(entry.getKey().getId());
+      server.setInitialConf(conf);
+      server.setServerRpc(entry.getValue());
+    }
+  }
+
+  public void start() {
+    LOG.info("Starting " + getClass().getSimpleName());
+    servers.values().forEach(RaftServerImpl::start);
+  }
+
+  /**
+   * start a stopped server again.
+   */
+  public void restartServer(String id, boolean format) throws IOException {
+    killServer(id);
+    servers.remove(id);
+    servers.put(id, newRaftServer(id, conf, format));
+  }
+
+  public final void restart(boolean format) throws IOException {
+    servers.values().stream().filter(RaftServerImpl::isAlive)
+        .forEach(RaftServerImpl::close);
+    List<String> idList = new ArrayList<>(servers.keySet());
+    for (String id : idList) {
+      servers.remove(id);
+      servers.put(id, newRaftServer(id, conf, format));
+    }
+
+    setPeerRpc();
+    start();
+  }
+
+  protected abstract void setPeerRpc() throws IOException;
+
+  public int getMaxTimeout() {
+    return properties.getInt(
+        RaftServerConfigKeys.RAFT_SERVER_RPC_TIMEOUT_MAX_MS_KEY,
+        RaftServerConfigKeys.RAFT_SERVER_RPC_TIMEOUT_MAX_MS_DEFAULT);
+  }
+
+  public RaftConfiguration getConf() {
+    return conf;
+  }
+
+  private RaftServerImpl newRaftServer(String id, RaftConfiguration conf,
+                                       boolean format) {
+    final RaftServerImpl s;
+    try {
+      final String dirStr = testBaseDir + id;
+      if (format) {
+        formatDir(dirStr);
+      }
+      properties.set(RaftServerConfigKeys.RAFT_SERVER_STORAGE_DIR_KEY, dirStr);
+      s = new RaftServerImpl(id, conf, properties, getStateMachine4Test(properties));
+    } catch (IOException e) {
+      throw new RuntimeException(e);
+    }
+    return s;
+  }
+
+  static StateMachine getStateMachine4Test(RaftProperties properties) {
+    final Class<? extends StateMachine> smClass = properties.getClass(
+        STATEMACHINE_CLASS_KEY,
+        STATEMACHINE_CLASS_DEFAULT,
+        StateMachine.class);
+    return RaftUtils.newInstance(smClass);
+  }
+
+  public abstract RaftClientRequestSender getRaftClientRequestSender();
+
+  protected <RPC extends RaftServerRpc> Collection<RaftPeer> addNewPeers(
+      Map<RaftPeer, RPC> newPeers, Collection<RaftServerImpl> newServers,
+      boolean startService) throws IOException {
+    for (Map.Entry<RaftPeer, RPC> entry : newPeers.entrySet()) {
+      RaftServerImpl server = servers.get(entry.getKey().getId());
+      server.setServerRpc(entry.getValue());
+    }
+    if (startService) {
+      newServers.forEach(RaftServerImpl::start);
+    }
+    return new ArrayList<>(newPeers.keySet());
+  }
+
+  protected abstract Collection<RaftPeer> addNewPeers(
+      Collection<RaftPeer> newPeers, Collection<RaftServerImpl> newServers,
+      boolean startService) throws IOException;
+
+  public PeerChanges addNewPeers(int number, boolean startNewPeer)
+      throws IOException {
+    return addNewPeers(generateIds(number, servers.size()), startNewPeer);
+  }
+
+  public PeerChanges addNewPeers(String[] ids,
+      boolean startNewPeer) throws IOException {
+    LOG.info("Add new peers {}", Arrays.asList(ids));
+    Collection<RaftPeer> newPeers = new ArrayList<>(ids.length);
+    for (String id : ids) {
+      newPeers.add(new RaftPeer(id));
+    }
+
+    // create and add new RaftServers
+    final List<RaftServerImpl> newServers = new ArrayList<>(ids.length);
+    for (RaftPeer p : newPeers) {
+      RaftServerImpl newServer = newRaftServer(p.getId(), conf, true);
+      Preconditions.checkArgument(!servers.containsKey(p.getId()));
+      servers.put(p.getId(), newServer);
+      newServers.add(newServer);
+    }
+
+    // for hadoop-rpc-enabled peer, we assign inetsocketaddress here
+    newPeers = addNewPeers(newPeers, newServers, startNewPeer);
+
+    final RaftPeer[] np = newPeers.toArray(new RaftPeer[newPeers.size()]);
+    newPeers.addAll(conf.getPeers());
+    conf = RaftConfiguration.newBuilder().setConf(newPeers).setLogEntryIndex(0).build();
+    RaftPeer[] p = newPeers.toArray(new RaftPeer[newPeers.size()]);
+    return new PeerChanges(p, np, new RaftPeer[0]);
+  }
+
+  public void startServer(String id) {
+    RaftServerImpl server = servers.get(id);
+    assert server != null;
+    server.start();
+  }
+
+  private RaftPeer getPeer(RaftServerImpl s) {
+    return new RaftPeer(s.getId(), s.getServerRpc().getInetSocketAddress());
+  }
+
+  /**
+   * prepare the peer list when removing some peers from the conf
+   */
+  public PeerChanges removePeers(int number, boolean removeLeader,
+      Collection<RaftPeer> excluded) {
+    Collection<RaftPeer> peers = new ArrayList<>(conf.getPeers());
+    List<RaftPeer> removedPeers = new ArrayList<>(number);
+    if (removeLeader) {
+      final RaftPeer leader = getPeer(getLeader());
+      assert !excluded.contains(leader);
+      peers.remove(leader);
+      removedPeers.add(leader);
+    }
+    List<RaftServerImpl> followers = getFollowers();
+    for (int i = 0, removed = 0; i < followers.size() &&
+        removed < (removeLeader ? number - 1 : number); i++) {
+      RaftPeer toRemove = getPeer(followers.get(i));
+      if (!excluded.contains(toRemove)) {
+        peers.remove(toRemove);
+        removedPeers.add(toRemove);
+        removed++;
+      }
+    }
+    conf = RaftConfiguration.newBuilder().setConf(peers).setLogEntryIndex(0).build();
+    RaftPeer[] p = peers.toArray(new RaftPeer[peers.size()]);
+    return new PeerChanges(p, new RaftPeer[0],
+        removedPeers.toArray(new RaftPeer[removedPeers.size()]));
+  }
+
+  public void killServer(String id) {
+    servers.get(id).close();
+  }
+
+  public String printServers() {
+    StringBuilder b = new StringBuilder("\n#servers = " + servers.size() + "\n");
+    for (RaftServerImpl s : servers.values()) {
+      b.append("  ");
+      b.append(s).append("\n");
+    }
+    return b.toString();
+  }
+
+  public String printAllLogs() {
+    StringBuilder b = new StringBuilder("\n#servers = " + servers.size() + "\n");
+    for (RaftServerImpl s : servers.values()) {
+      b.append("  ");
+      b.append(s).append("\n");
+
+      final RaftLog log = s.getState().getLog();
+      if (log instanceof MemoryRaftLog) {
+        b.append("    ");
+        b.append(((MemoryRaftLog) log).getEntryString());
+      }
+    }
+    return b.toString();
+  }
+
+  public RaftServerImpl getLeader() {
+    final List<RaftServerImpl> leaders = new ArrayList<>();
+    servers.values().stream()
+        .filter(s -> s.isAlive() && s.isLeader())
+        .forEach(s -> {
+      if (leaders.isEmpty()) {
+        leaders.add(s);
+      } else {
+        final long leaderTerm = leaders.get(0).getState().getCurrentTerm();
+        final long term = s.getState().getCurrentTerm();
+        if (term >= leaderTerm) {
+          if (term > leaderTerm) {
+            leaders.clear();
+          }
+          leaders.add(s);
+        }
+      }
+    });
+    if (leaders.isEmpty()) {
+      return null;
+    } else if (leaders.size() != 1) {
+      Assert.fail(printServers() + leaders.toString()
+          + "leaders.size() = " + leaders.size() + " != 1");
+    }
+    return leaders.get(0);
+  }
+
+  public boolean isLeader(String leaderId) throws InterruptedException {
+    final RaftServerImpl leader = getLeader();
+    return leader != null && leader.getId().equals(leaderId);
+  }
+
+  public List<RaftServerImpl> getFollowers() {
+    return servers.values().stream()
+        .filter(s -> s.isAlive() && s.isFollower())
+        .collect(Collectors.toList());
+  }
+
+  public Collection<RaftServerImpl> getServers() {
+    return servers.values();
+  }
+
+  public RaftServerImpl getServer(String id) {
+    return servers.get(id);
+  }
+
+  public Collection<RaftPeer> getPeers() {
+    return getServers().stream().map(s ->
+        new RaftPeer(s.getId(), s.getServerRpc().getInetSocketAddress()))
+        .collect(Collectors.toList());
+  }
+
+  public RaftClient createClient(String clientId, String leaderId) {
+    return new RaftClientImpl(clientId, conf.getPeers(),
+        getRaftClientRequestSender(), leaderId, properties);
+  }
+
+  public void shutdown() {
+    LOG.info("Stopping " + getClass().getSimpleName());
+    servers.values().stream().filter(RaftServerImpl::isAlive)
+        .forEach(RaftServerImpl::close);
+
+    if (ExitUtils.isTerminated()) {
+      LOG.error("Test resulted in an unexpected exit",
+          ExitUtils.getFirstExitException());
+      throw new AssertionError("Test resulted in an unexpected exit");
+    }
+  }
+
+  /**
+   * Block all the incoming requests for the peer with leaderId. Also delay
+   * outgoing or incoming msg for all other peers.
+   */
+  protected abstract void blockQueueAndSetDelay(String leaderId, int delayMs)
+      throws InterruptedException;
+
+  /**
+   * Try to enforce the leader of the cluster.
+   * @param leaderId ID of the targeted leader server.
+   * @return true if server has been successfully enforced to the leader, false
+   *         otherwise.
+   */
+  public boolean tryEnforceLeader(String leaderId) throws InterruptedException {
+    // do nothing and see if the given id is already a leader.
+    if (isLeader(leaderId)) {
+      return true;
+    }
+
+    // Blocking all other server's RPC read process to make sure a read takes at
+    // least ELECTION_TIMEOUT_MIN. In this way when the target leader request a
+    // vote, all non-leader servers can grant the vote.
+    // Disable the target leader server RPC so that it can request a vote.
+    blockQueueAndSetDelay(leaderId, RAFT_SERVER_RPC_TIMEOUT_MIN_MS_DEFAULT);
+
+    // Reopen queues so that the vote can make progress.
+    blockQueueAndSetDelay(leaderId, 0);
+
+    return isLeader(leaderId);
+  }
+
+  /** Block/unblock the requests sent from the given source. */
+  public abstract void setBlockRequestsFrom(String src, boolean block);
+}

http://git-wip-us.apache.org/repos/asf/incubator-ratis/blob/7e71a2e0/ratis-server/src/test/java/org/apache/ratis/RaftBasicTests.java
----------------------------------------------------------------------
diff --git a/ratis-server/src/test/java/org/apache/ratis/RaftBasicTests.java b/ratis-server/src/test/java/org/apache/ratis/RaftBasicTests.java
new file mode 100644
index 0000000..4ec78b9
--- /dev/null
+++ b/ratis-server/src/test/java/org/apache/ratis/RaftBasicTests.java
@@ -0,0 +1,199 @@
+/**
+ * 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.ratis;
+
+import org.apache.ratis.RaftTestUtil.SimpleMessage;
+import org.apache.ratis.client.RaftClient;
+import org.apache.ratis.conf.RaftProperties;
+import org.apache.ratis.server.impl.RaftServerImpl;
+import org.junit.*;
+import org.junit.rules.Timeout;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import static org.apache.ratis.RaftTestUtil.waitAndKillLeader;
+import static org.apache.ratis.RaftTestUtil.waitForLeader;
+
+import java.io.IOException;
+import java.util.List;
+import java.util.concurrent.ThreadLocalRandom;
+import java.util.concurrent.atomic.AtomicInteger;
+import java.util.stream.Collectors;
+import java.util.stream.Stream;
+
+public abstract class RaftBasicTests {
+  public static final Logger LOG = LoggerFactory.getLogger(RaftBasicTests.class);
+
+  public static final int NUM_SERVERS = 5;
+
+  protected static final RaftProperties properties = new RaftProperties();
+
+  public abstract MiniRaftCluster getCluster();
+
+  public RaftProperties getProperties() {
+    return properties;
+  }
+
+  @Rule
+  public Timeout globalTimeout = new Timeout(120 * 1000);
+
+  @Before
+  public void setup() throws IOException {
+    Assert.assertNull(getCluster().getLeader());
+    getCluster().start();
+  }
+
+  @After
+  public void tearDown() {
+    final MiniRaftCluster cluster = getCluster();
+    if (cluster != null) {
+      cluster.shutdown();
+    }
+  }
+
+  @Test
+  public void testBasicLeaderElection() throws Exception {
+    LOG.info("Running testBasicLeaderElection");
+    final MiniRaftCluster cluster = getCluster();
+    waitAndKillLeader(cluster, true);
+    waitAndKillLeader(cluster, true);
+    waitAndKillLeader(cluster, true);
+    waitAndKillLeader(cluster, false);
+  }
+
+  @Test
+  public void testBasicAppendEntries() throws Exception {
+    LOG.info("Running testBasicAppendEntries");
+    final MiniRaftCluster cluster = getCluster();
+    RaftServerImpl leader = waitForLeader(cluster);
+    final long term = leader.getState().getCurrentTerm();
+    final String killed = cluster.getFollowers().get(3).getId();
+    cluster.killServer(killed);
+    LOG.info(cluster.printServers());
+
+    final SimpleMessage[] messages = SimpleMessage.create(10);
+    try(final RaftClient client = cluster.createClient("client", null)) {
+      for (SimpleMessage message : messages) {
+        client.send(message);
+      }
+    }
+
+    Thread.sleep(cluster.getMaxTimeout() + 100);
+    LOG.info(cluster.printAllLogs());
+
+    cluster.getServers().stream().filter(RaftServerImpl::isAlive)
+        .map(s -> s.getState().getLog().getEntries(1, Long.MAX_VALUE))
+        .forEach(e -> RaftTestUtil.assertLogEntries(e, 1, term, messages));
+  }
+
+  @Test
+  public void testEnforceLeader() throws Exception {
+    LOG.info("Running testEnforceLeader");
+    final String leader = "s" + ThreadLocalRandom.current().nextInt(NUM_SERVERS);
+    LOG.info("enforce leader to " + leader);
+    final MiniRaftCluster cluster = getCluster();
+    waitForLeader(cluster);
+    waitForLeader(cluster, leader);
+  }
+
+  static class Client4TestWithLoad extends Thread {
+    final RaftClient client;
+    final SimpleMessage[] messages;
+
+    final AtomicInteger step = new AtomicInteger();
+    volatile Exception exceptionInClientThread;
+
+    Client4TestWithLoad(RaftClient client, int numMessages) {
+      this.client = client;
+      this.messages = SimpleMessage.create(numMessages, client.getId());
+    }
+
+    boolean isRunning() {
+      return step.get() < messages.length && exceptionInClientThread == null;
+    }
+
+    @Override
+    public void run() {
+      try {
+        for (; isRunning(); ) {
+          client.send(messages[step.getAndIncrement()]);
+        }
+        client.close();
+      } catch (IOException ioe) {
+        exceptionInClientThread = ioe;
+      }
+    }
+  }
+
+  @Test
+  public void testWithLoad() throws Exception {
+    testWithLoad(10, 500);
+  }
+
+  private void testWithLoad(final int numClients, final int numMessages)
+      throws Exception {
+    LOG.info("Running testWithLoad: numClients=" + numClients
+        + ", numMessages=" + numMessages);
+
+    final MiniRaftCluster cluster = getCluster();
+    LOG.info(cluster.printServers());
+
+    final List<Client4TestWithLoad> clients
+        = Stream.iterate(0, i -> i+1).limit(numClients)
+        .map(i -> cluster.createClient(String.valueOf((char)('a' + i)), null))
+        .map(c -> new Client4TestWithLoad(c, numMessages))
+        .collect(Collectors.toList());
+    clients.forEach(Thread::start);
+
+    int count = 0;
+    for(int lastStep = 0;; ) {
+      if (clients.stream().filter(Client4TestWithLoad::isRunning).count() == 0) {
+        break;
+      }
+
+      final int n = clients.stream().mapToInt(c -> c.step.get()).sum();
+      if (n - lastStep < 50 * numClients) { // Change leader at least 50 steps.
+        Thread.sleep(10);
+        continue;
+      }
+      lastStep = n;
+      count++;
+
+      RaftServerImpl leader = cluster.getLeader();
+      if (leader != null) {
+        final String oldLeader = leader.getId();
+        LOG.info("Block all requests sent by leader " + oldLeader);
+        String newLeader = RaftTestUtil.changeLeader(cluster, oldLeader);
+        LOG.info("Changed leader from " + oldLeader + " to " + newLeader);
+        Assert.assertFalse(newLeader.equals(oldLeader));
+      }
+    }
+
+    for(Client4TestWithLoad c : clients) {
+      c.join();
+    }
+    for(Client4TestWithLoad c : clients) {
+      if (c.exceptionInClientThread != null) {
+        throw new AssertionError(c.exceptionInClientThread);
+      }
+      RaftTestUtil.assertLogEntries(cluster.getServers(), c.messages);
+    }
+
+    LOG.info("Leader change count=" + count + cluster.printAllLogs());
+  }
+}

http://git-wip-us.apache.org/repos/asf/incubator-ratis/blob/7e71a2e0/ratis-server/src/test/java/org/apache/ratis/RaftNotLeaderExceptionBaseTest.java
----------------------------------------------------------------------
diff --git a/ratis-server/src/test/java/org/apache/ratis/RaftNotLeaderExceptionBaseTest.java b/ratis-server/src/test/java/org/apache/ratis/RaftNotLeaderExceptionBaseTest.java
new file mode 100644
index 0000000..6d25835
--- /dev/null
+++ b/ratis-server/src/test/java/org/apache/ratis/RaftNotLeaderExceptionBaseTest.java
@@ -0,0 +1,162 @@
+/**
+ * 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.ratis;
+
+import org.apache.log4j.Level;
+import org.apache.ratis.RaftTestUtil.SimpleMessage;
+import org.apache.ratis.client.RaftClient;
+import org.apache.ratis.client.RaftClientRequestSender;
+import org.apache.ratis.client.impl.RaftClientImpl;
+import org.apache.ratis.protocol.RaftClientReply;
+import org.apache.ratis.protocol.RaftClientRequest;
+import org.apache.ratis.protocol.RaftPeer;
+import org.apache.ratis.server.impl.RaftServerImpl;
+import org.apache.ratis.server.simulation.RequestHandler;
+import org.apache.ratis.server.storage.RaftLog;
+import org.apache.ratis.util.RaftUtils;
+import org.junit.*;
+import org.junit.rules.Timeout;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import static org.apache.ratis.server.impl.RaftServerConstants.DEFAULT_SEQNUM;
+
+import java.io.IOException;
+import java.util.Arrays;
+import java.util.Collection;
+
+public abstract class RaftNotLeaderExceptionBaseTest {
+  static {
+    RaftUtils.setLogLevel(RaftServerImpl.LOG, Level.DEBUG);
+    RaftUtils.setLogLevel(RaftLog.LOG, Level.DEBUG);
+    RaftUtils.setLogLevel(RequestHandler.LOG, Level.DEBUG);
+    RaftUtils.setLogLevel(RaftClient.LOG, Level.DEBUG);
+  }
+
+  public static final Logger LOG =
+      LoggerFactory.getLogger(RaftNotLeaderExceptionBaseTest.class);
+  public static final int NUM_PEERS = 3;
+
+  @Rule
+  public Timeout globalTimeout = new Timeout(60 * 1000);
+
+  private MiniRaftCluster cluster;
+
+  public abstract MiniRaftCluster initCluster() throws IOException;
+
+  @Before
+  public void setup() throws IOException {
+    this.cluster = initCluster();
+    cluster.start();
+  }
+
+  @After
+  public void tearDown() {
+    if (cluster != null) {
+      cluster.shutdown();
+    }
+  }
+
+  @Test
+  public void testHandleNotLeaderException() throws Exception {
+    RaftTestUtil.waitForLeader(cluster);
+    final String leaderId = cluster.getLeader().getId();
+    final RaftClient client = cluster.createClient("client", leaderId);
+
+    RaftClientReply reply = client.send(new SimpleMessage("m1"));
+    Assert.assertTrue(reply.isSuccess());
+
+    // enforce leader change
+    String newLeader = RaftTestUtil.changeLeader(cluster, leaderId);
+    Assert.assertNotEquals(leaderId, newLeader);
+
+    RaftClientRequestSender rpc = ((RaftClientImpl)client).getRequestSender();
+    reply= null;
+    for (int i = 0; reply == null && i < 10; i++) {
+      try {
+        reply = rpc.sendRequest(
+            new RaftClientRequest("client", leaderId, DEFAULT_SEQNUM,
+                new SimpleMessage("m2")));
+      } catch (IOException ignored) {
+        Thread.sleep(1000);
+      }
+    }
+    Assert.assertNotNull(reply);
+    Assert.assertFalse(reply.isSuccess());
+    Assert.assertTrue(reply.isNotLeader());
+    Assert.assertEquals(newLeader,
+        reply.getNotLeaderException().getSuggestedLeader().getId());
+
+    reply = client.send(new SimpleMessage("m3"));
+    Assert.assertTrue(reply.isSuccess());
+    client.close();
+  }
+
+  @Test
+  public void testNotLeaderExceptionWithReconf() throws Exception {
+    Assert.assertNotNull(RaftTestUtil.waitForLeader(cluster));
+
+    final String leaderId = cluster.getLeader().getId();
+    final RaftClient client = cluster.createClient("client", leaderId);
+
+    // enforce leader change
+    String newLeader = RaftTestUtil.changeLeader(cluster, leaderId);
+    Assert.assertNotEquals(leaderId, newLeader);
+
+    // also add two new peers
+    // add two more peers
+    MiniRaftCluster.PeerChanges change = cluster.addNewPeers(
+        new String[]{"ss1", "ss2"}, true);
+    // trigger setConfiguration
+    LOG.info("Start changing the configuration: {}",
+        Arrays.asList(change.allPeersInNewConf));
+    try(final RaftClient c2 = cluster.createClient("client2", newLeader)) {
+      RaftClientReply reply = c2.setConfiguration(change.allPeersInNewConf);
+      Assert.assertTrue(reply.isSuccess());
+    }
+    LOG.info(cluster.printServers());
+
+    RaftClientRequestSender rpc = ((RaftClientImpl)client).getRequestSender();
+    RaftClientReply reply = null;
+    // it is possible that the remote peer's rpc server is not ready. need retry
+    for (int i = 0; reply == null && i < 10; i++) {
+      try {
+        reply = rpc.sendRequest(
+            new RaftClientRequest("client", leaderId, DEFAULT_SEQNUM,
+                new SimpleMessage("m1")));
+      } catch (IOException ignored) {
+        Thread.sleep(1000);
+      }
+    }
+    Assert.assertNotNull(reply);
+    Assert.assertFalse(reply.isSuccess());
+    Assert.assertTrue(reply.isNotLeader());
+    Assert.assertEquals(newLeader,
+        reply.getNotLeaderException().getSuggestedLeader().getId());
+    Collection<RaftPeer> peers = cluster.getPeers();
+    RaftPeer[] peersFromReply = reply.getNotLeaderException().getPeers();
+    Assert.assertEquals(peers.size(), peersFromReply.length);
+    for (RaftPeer p : peersFromReply) {
+      Assert.assertTrue(peers.contains(p));
+    }
+
+    reply = client.send(new SimpleMessage("m2"));
+    Assert.assertTrue(reply.isSuccess());
+    client.close();
+  }
+}


[52/54] [abbrv] incubator-ratis git commit: Update default raft log segment size.

Posted by ji...@apache.org.
Update default raft log segment size.


Project: http://git-wip-us.apache.org/repos/asf/incubator-ratis/repo
Commit: http://git-wip-us.apache.org/repos/asf/incubator-ratis/commit/9c442378
Tree: http://git-wip-us.apache.org/repos/asf/incubator-ratis/tree/9c442378
Diff: http://git-wip-us.apache.org/repos/asf/incubator-ratis/diff/9c442378

Branch: refs/heads/master
Commit: 9c44237880ae2c772d58f036708b0bd3f6b013a8
Parents: 0fed307
Author: Jing Zhao <ji...@apache.org>
Authored: Wed Jan 25 15:48:43 2017 -0800
Committer: Jing Zhao <ji...@apache.org>
Committed: Wed Jan 25 15:48:43 2017 -0800

----------------------------------------------------------------------
 .../main/java/org/apache/ratis/server/RaftServerConfigKeys.java  | 4 ++--
 .../main/java/org/apache/ratis/server/storage/LogSegment.java    | 4 ++--
 .../main/java/org/apache/ratis/server/storage/RaftLogWorker.java | 1 -
 3 files changed, 4 insertions(+), 5 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-ratis/blob/9c442378/ratis-server/src/main/java/org/apache/ratis/server/RaftServerConfigKeys.java
----------------------------------------------------------------------
diff --git a/ratis-server/src/main/java/org/apache/ratis/server/RaftServerConfigKeys.java b/ratis-server/src/main/java/org/apache/ratis/server/RaftServerConfigKeys.java
index cdf7651..d3c5173 100644
--- a/ratis-server/src/main/java/org/apache/ratis/server/RaftServerConfigKeys.java
+++ b/ratis-server/src/main/java/org/apache/ratis/server/RaftServerConfigKeys.java
@@ -76,10 +76,10 @@ public interface RaftServerConfigKeys {
   long RAFT_SERVER_SNAPSHOT_TRIGGER_THRESHOLD_DEFAULT = 400000;
 
   String RAFT_LOG_SEGMENT_MAX_SIZE_KEY = "raft.log.segment.max.size";
-  long RAFT_LOG_SEGMENT_MAX_SIZE_DEFAULT = 1024L * 1024 * 1024 * 2; // 2GB
+  long RAFT_LOG_SEGMENT_MAX_SIZE_DEFAULT = 1024L * 1024 * 8; // 8MB
 
   String RAFT_LOG_SEGMENT_PREALLOCATED_SIZE_KEY = "raft.log.segment.preallocated.size";
-  int RAFT_LOG_SEGMENT_PREALLOCATED_SIZE_DEFAULT = 1024 * 1024 * 16; // 16MB
+  int RAFT_LOG_SEGMENT_PREALLOCATED_SIZE_DEFAULT = 1024 * 1024 * 4; // 4MB
 
   String RAFT_LOG_WRITE_BUFFER_SIZE_KEY = "raft.log.write.buffer.size";
   int RAFT_LOG_WRITE_BUFFER_SIZE_DEFAULT = 64 * 1024;

http://git-wip-us.apache.org/repos/asf/incubator-ratis/blob/9c442378/ratis-server/src/main/java/org/apache/ratis/server/storage/LogSegment.java
----------------------------------------------------------------------
diff --git a/ratis-server/src/main/java/org/apache/ratis/server/storage/LogSegment.java b/ratis-server/src/main/java/org/apache/ratis/server/storage/LogSegment.java
index af9ee66..c8e5306 100644
--- a/ratis-server/src/main/java/org/apache/ratis/server/storage/LogSegment.java
+++ b/ratis-server/src/main/java/org/apache/ratis/server/storage/LogSegment.java
@@ -216,8 +216,8 @@ class LogSegment implements Comparable<Long> {
 
   @Override
   public String toString() {
-    return isOpen() ? "log-" + startIndex + "-inprogress" :
-        "log-" + startIndex + "-" + endIndex;
+    return isOpen() ? "log_" + "inprogress_" + startIndex :
+        "log-" + startIndex + "_" + endIndex;
   }
 
   @Override

http://git-wip-us.apache.org/repos/asf/incubator-ratis/blob/9c442378/ratis-server/src/main/java/org/apache/ratis/server/storage/RaftLogWorker.java
----------------------------------------------------------------------
diff --git a/ratis-server/src/main/java/org/apache/ratis/server/storage/RaftLogWorker.java b/ratis-server/src/main/java/org/apache/ratis/server/storage/RaftLogWorker.java
index e057a51..ae71cc1 100644
--- a/ratis-server/src/main/java/org/apache/ratis/server/storage/RaftLogWorker.java
+++ b/ratis-server/src/main/java/org/apache/ratis/server/storage/RaftLogWorker.java
@@ -221,7 +221,6 @@ class RaftLogWorker implements Runnable {
     return addIOTask(new TruncateLog(ts));
   }
 
-  // TODO we can add another level of buffer for writing here
   private class WriteLog extends Task {
     private final LogEntryProto entry;
 


[32/54] [abbrv] incubator-ratis git commit: Renamed the packages from raft to ratis in preperation for Apache Incubation - Moved all java packages from org.apache.raft to org.apache.ratis. - Moved native package to org_apache_ratis, and native lib to l

Posted by ji...@apache.org.
http://git-wip-us.apache.org/repos/asf/incubator-ratis/blob/7e71a2e0/ratis-client/src/main/java/org/apache/ratis/client/RaftClientRequestSender.java
----------------------------------------------------------------------
diff --git a/ratis-client/src/main/java/org/apache/ratis/client/RaftClientRequestSender.java b/ratis-client/src/main/java/org/apache/ratis/client/RaftClientRequestSender.java
new file mode 100644
index 0000000..b2541e1
--- /dev/null
+++ b/ratis-client/src/main/java/org/apache/ratis/client/RaftClientRequestSender.java
@@ -0,0 +1,34 @@
+/**
+ * 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.ratis.client;
+
+import java.io.Closeable;
+import java.io.IOException;
+
+import org.apache.ratis.protocol.RaftClientReply;
+import org.apache.ratis.protocol.RaftClientRequest;
+import org.apache.ratis.protocol.RaftPeer;
+
+/** Send requests to a raft service. */
+public interface RaftClientRequestSender extends Closeable {
+  /** Send a request. */
+  RaftClientReply sendRequest(RaftClientRequest request) throws IOException;
+
+  /** Add the information of the given raft servers */
+  void addServers(Iterable<RaftPeer> servers);
+}

http://git-wip-us.apache.org/repos/asf/incubator-ratis/blob/7e71a2e0/ratis-client/src/main/java/org/apache/ratis/client/impl/ClientProtoUtils.java
----------------------------------------------------------------------
diff --git a/ratis-client/src/main/java/org/apache/ratis/client/impl/ClientProtoUtils.java b/ratis-client/src/main/java/org/apache/ratis/client/impl/ClientProtoUtils.java
new file mode 100644
index 0000000..24bb4ec
--- /dev/null
+++ b/ratis-client/src/main/java/org/apache/ratis/client/impl/ClientProtoUtils.java
@@ -0,0 +1,131 @@
+/**
+ * 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.ratis.client.impl;
+
+import org.apache.ratis.shaded.com.google.protobuf.ByteString;
+import org.apache.ratis.shaded.proto.RaftProtos.*;
+import org.apache.ratis.protocol.*;
+import org.apache.ratis.util.ProtoUtils;
+
+import java.util.Arrays;
+
+public class ClientProtoUtils {
+  public static RaftRpcReplyProto.Builder toRaftRpcReplyProtoBuilder(
+      String requestorId, String replyId, long seqNum, boolean success) {
+    return RaftRpcReplyProto.newBuilder()
+        .setRequestorId(requestorId).setReplyId(replyId).setSeqNum(seqNum)
+        .setSuccess(success);
+  }
+
+  public static RaftRpcRequestProto.Builder toRaftRpcRequestProtoBuilder(
+      String requesterId, String replyId, long seqNum) {
+    return RaftRpcRequestProto.newBuilder()
+        .setRequestorId(requesterId).setReplyId(replyId).setSeqNum(seqNum);
+  }
+
+  public static RaftClientRequest toRaftClientRequest(RaftClientRequestProto p) {
+    return new RaftClientRequest(p.getRpcRequest().getRequestorId(),
+        p.getRpcRequest().getReplyId(), p.getRpcRequest().getSeqNum(),
+        toMessage(p.getMessage()), p.getReadOnly());
+  }
+
+  public static RaftClientRequestProto toRaftClientRequestProto(
+      RaftClientRequest request) {
+    return RaftClientRequestProto.newBuilder()
+        .setRpcRequest(toRaftRpcRequestProtoBuilder(request.getRequestorId(),
+            request.getReplierId(), request.getSeqNum()))
+        .setMessage(toClientMessageEntryProto(request.getMessage()))
+        .setReadOnly(request.isReadOnly())
+        .build();
+  }
+
+  public static RaftClientRequestProto genRaftClientRequestProto(
+      String requestorId, String replierId, long seqNum, ByteString content,
+      boolean readOnly) {
+    return RaftClientRequestProto.newBuilder()
+        .setRpcRequest(toRaftRpcRequestProtoBuilder(requestorId, replierId, seqNum))
+        .setMessage(ClientMessageEntryProto.newBuilder().setContent(content))
+        .setReadOnly(readOnly)
+        .build();
+  }
+
+  public static RaftClientReplyProto toRaftClientReplyProto(
+      RaftClientReply reply) {
+    final RaftClientReplyProto.Builder b = RaftClientReplyProto.newBuilder();
+    if (reply != null) {
+      b.setRpcReply(toRaftRpcReplyProtoBuilder(reply.getRequestorId(),
+          reply.getReplierId(), reply.getSeqNum(), reply.isSuccess()));
+      if (reply.getMessage() != null) {
+        b.setMessage(toClientMessageEntryProto(reply.getMessage()));
+      }
+      if (reply.isNotLeader()) {
+        b.setIsNotLeader(true);
+        final RaftPeer suggestedLeader = reply.getNotLeaderException()
+            .getSuggestedLeader();
+        if (suggestedLeader != null) {
+          b.setSuggestedLeader(ProtoUtils.toRaftPeerProto(suggestedLeader));
+        }
+        b.addAllPeersInConf(ProtoUtils.toRaftPeerProtos(
+            Arrays.asList(reply.getNotLeaderException().getPeers())));
+      }
+    }
+    return b.build();
+  }
+
+  public static RaftClientReply toRaftClientReply(
+      RaftClientReplyProto replyProto) {
+    final RaftRpcReplyProto rp = replyProto.getRpcReply();
+    NotLeaderException e = null;
+    if (replyProto.getIsNotLeader()) {
+      final RaftPeer suggestedLeader = replyProto.hasSuggestedLeader() ?
+          ProtoUtils.toRaftPeer(replyProto.getSuggestedLeader()) : null;
+      final RaftPeer[] peers = ProtoUtils.toRaftPeerArray(
+          replyProto.getPeersInConfList());
+      e = new NotLeaderException(rp.getReplyId(), suggestedLeader, peers);
+    }
+    return new RaftClientReply(rp.getRequestorId(), rp.getReplyId(),
+        rp.getSeqNum(), rp.getSuccess(), toMessage(replyProto.getMessage()), e);
+  }
+
+  public static Message toMessage(final ClientMessageEntryProto p) {
+    return p::getContent;
+  }
+
+  public static ClientMessageEntryProto toClientMessageEntryProto(Message message) {
+    return ClientMessageEntryProto.newBuilder()
+        .setContent(message.getContent()).build();
+  }
+
+  public static SetConfigurationRequest toSetConfigurationRequest(
+      SetConfigurationRequestProto p) {
+    final RaftRpcRequestProto m = p.getRpcRequest();
+    final RaftPeer[] peers = ProtoUtils.toRaftPeerArray(p.getPeersList());
+    return new SetConfigurationRequest(m.getRequestorId(), m.getReplyId(),
+        p.getRpcRequest().getSeqNum(), peers);
+  }
+
+  public static SetConfigurationRequestProto toSetConfigurationRequestProto(
+      SetConfigurationRequest request) {
+    return SetConfigurationRequestProto.newBuilder()
+        .setRpcRequest(toRaftRpcRequestProtoBuilder(request.getRequestorId(),
+            request.getReplierId(), request.getSeqNum()))
+        .addAllPeers(ProtoUtils.toRaftPeerProtos(
+            Arrays.asList(request.getPeersInNewConf())))
+        .build();
+  }
+}

http://git-wip-us.apache.org/repos/asf/incubator-ratis/blob/7e71a2e0/ratis-client/src/main/java/org/apache/ratis/client/impl/RaftClientImpl.java
----------------------------------------------------------------------
diff --git a/ratis-client/src/main/java/org/apache/ratis/client/impl/RaftClientImpl.java b/ratis-client/src/main/java/org/apache/ratis/client/impl/RaftClientImpl.java
new file mode 100644
index 0000000..e101554
--- /dev/null
+++ b/ratis-client/src/main/java/org/apache/ratis/client/impl/RaftClientImpl.java
@@ -0,0 +1,172 @@
+/**
+ * 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.ratis.client.impl;
+
+import com.google.common.annotations.VisibleForTesting;
+
+import org.apache.ratis.client.RaftClient;
+import org.apache.ratis.client.RaftClientConfigKeys;
+import org.apache.ratis.client.RaftClientRequestSender;
+import org.apache.ratis.conf.RaftProperties;
+import org.apache.ratis.protocol.*;
+import org.apache.ratis.util.RaftUtils;
+import org.apache.ratis.util.StringUtils;
+
+import java.io.IOException;
+import java.io.InterruptedIOException;
+import java.util.Arrays;
+import java.util.Collection;
+import java.util.Map;
+import java.util.function.Function;
+import java.util.function.Supplier;
+import java.util.stream.Collectors;
+
+/** A client who sends requests to a raft service. */
+public final class RaftClientImpl implements RaftClient {
+  public static final long DEFAULT_SEQNUM = 0;
+
+  private final String clientId;
+  private final RaftClientRequestSender requestSender;
+  private final Map<String, RaftPeer> peers;
+  private final int retryInterval;
+
+  private volatile String leaderId;
+
+  public RaftClientImpl(
+      String clientId, Collection<RaftPeer> peers,
+      RaftClientRequestSender requestSender, String leaderId,
+      RaftProperties properties) {
+    this.clientId = clientId;
+    this.requestSender = requestSender;
+    this.peers = peers.stream().collect(
+        Collectors.toMap(RaftPeer::getId, Function.identity()));
+    this.leaderId = leaderId != null? leaderId : peers.iterator().next().getId();
+    this.retryInterval = properties.getInt(
+        RaftClientConfigKeys.RAFT_RPC_TIMEOUT_MS_KEY,
+        RaftClientConfigKeys.RAFT_RPC_TIMEOUT_MS_DEFAULT);
+  }
+
+  @Override
+  public String getId() {
+    return clientId;
+  }
+
+  @Override
+  public RaftClientReply send(Message message) throws IOException {
+    return send(message, false);
+  }
+
+  @Override
+  public RaftClientReply sendReadOnly(Message message) throws IOException {
+    return send(message, true);
+  }
+
+  private RaftClientReply send(Message message, boolean readOnly) throws IOException {
+    return sendRequestWithRetry(() -> new RaftClientRequest(
+        clientId, leaderId, DEFAULT_SEQNUM, message, readOnly));
+  }
+
+  @Override
+  public RaftClientReply setConfiguration(RaftPeer[] peersInNewConf)
+      throws IOException {
+    return sendRequestWithRetry(() -> new SetConfigurationRequest(
+        clientId, leaderId, DEFAULT_SEQNUM, peersInNewConf));
+  }
+
+  private RaftClientReply sendRequestWithRetry(
+      Supplier<RaftClientRequest> supplier)
+      throws InterruptedIOException, StateMachineException {
+    for(;;) {
+      final RaftClientRequest request = supplier.get();
+      LOG.debug("{}: {}", clientId, request);
+      final RaftClientReply reply = sendRequest(request);
+      if (reply != null) {
+        LOG.debug("{}: {}", clientId, reply);
+        return reply;
+      }
+
+      // sleep and then retry
+      try {
+        Thread.sleep(retryInterval);
+      } catch (InterruptedException ie) {
+        Thread.currentThread().interrupt();
+        throw RaftUtils.toInterruptedIOException(
+            "Interrupted when sending " + request, ie);
+      }
+    }
+  }
+
+  private RaftClientReply sendRequest(RaftClientRequest request)
+      throws StateMachineException {
+    try {
+      RaftClientReply reply = requestSender.sendRequest(request);
+      if (reply.isNotLeader()) {
+        handleNotLeaderException(request, reply.getNotLeaderException());
+        return null;
+      } else {
+        return reply;
+      }
+    } catch (StateMachineException e) {
+      throw e;
+    } catch (IOException ioe) {
+      // TODO different retry policies for different exceptions
+      handleIOException(request, ioe, null);
+    }
+    return null;
+  }
+
+  private void handleNotLeaderException(RaftClientRequest request, NotLeaderException nle) {
+    refreshPeers(nle.getPeers());
+    final String newLeader = nle.getSuggestedLeader() == null? null
+        : nle.getSuggestedLeader().getId();
+    handleIOException(request, nle, newLeader);
+  }
+
+  private void refreshPeers(RaftPeer[] newPeers) {
+    if (newPeers != null && newPeers.length > 0) {
+      peers.clear();
+      for (RaftPeer p : newPeers) {
+        peers.put(p.getId(), p);
+      }
+      // also refresh the rpc proxies for these peers
+      requestSender.addServers(Arrays.asList(newPeers));
+    }
+  }
+
+  private void handleIOException(RaftClientRequest request, IOException ioe, String newLeader) {
+    LOG.debug("{}: Failed with {}", clientId, ioe);
+    final String oldLeader = request.getReplierId();
+    if (newLeader == null && oldLeader.equals(leaderId)) {
+      newLeader = StringUtils.next(oldLeader, peers.keySet());
+    }
+    if (newLeader != null && oldLeader.equals(leaderId)) {
+      LOG.debug("{}: change Leader from {} to {}", clientId, oldLeader, newLeader);
+      this.leaderId = newLeader;
+    }
+  }
+
+  @VisibleForTesting
+  public RaftClientRequestSender getRequestSender() {
+    return requestSender;
+  }
+
+  @Override
+  public void close() throws IOException {
+    requestSender.close();
+  }
+}

http://git-wip-us.apache.org/repos/asf/incubator-ratis/blob/7e71a2e0/ratis-common/RatisCommon.cmake
----------------------------------------------------------------------
diff --git a/ratis-common/RatisCommon.cmake b/ratis-common/RatisCommon.cmake
new file mode 100644
index 0000000..e8f70eb
--- /dev/null
+++ b/ratis-common/RatisCommon.cmake
@@ -0,0 +1,208 @@
+#
+# 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.
+#
+
+#
+# Common CMake utilities and configuration, shared by all Native components.
+#
+
+#
+# Platform-specific prerequisite checks.
+#
+
+if(CMAKE_SYSTEM_NAME STREQUAL "SunOS")
+    # Only 64-bit Java is supported.
+    if(NOT JVM_ARCH_DATA_MODEL EQUAL 64)
+        message(FATAL_ERROR "Unrecognised JVM_ARCH_DATA_MODEL '${JVM_ARCH_DATA_MODEL}'. "
+          "A 64-bit JVM must be used on Solaris, make sure that one is installed and, "
+          "if necessary, the MAVEN_OPTS environment variable includes '-d64'")
+    endif()
+
+    # Only gcc is suported for now.
+    if(NOT(CMAKE_COMPILER_IS_GNUCC AND CMAKE_COMPILER_IS_GNUCXX))
+        message(FATAL_ERROR "Only gcc is supported on Solaris")
+    endif()
+endif()
+
+#
+# Helper functions and macros.
+#
+
+# Add flags to all the CMake compiler variables
+macro(ratis_add_compiler_flags FLAGS)
+    set(CMAKE_C_FLAGS "${CMAKE_C_FLAGS} ${FLAGS}")
+    set(CMAKE_CXX_FLAGS "${CMAKE_CXX_FLAGS} ${FLAGS}")
+endmacro()
+
+# Add flags to all the CMake linker variables.
+macro(ratis_add_linker_flags FLAGS)
+    set(CMAKE_EXE_LINKER_FLAGS "${CMAKE_EXE_LINKER_FLAGS} ${FLAGS}")
+    set(CMAKE_SHARED_LINKER_FLAGS "${CMAKE_SHARED_LINKER_FLAGS} ${FLAGS}")
+endmacro()
+
+# Compile a library with both shared and static variants.
+function(ratis_add_dual_library LIBNAME)
+    add_library(${LIBNAME} SHARED ${ARGN})
+    add_library(${LIBNAME}_static STATIC ${ARGN})
+    set_target_properties(${LIBNAME}_static PROPERTIES OUTPUT_NAME ${LIBNAME})
+endfunction()
+
+# Link both a static and a dynamic target against some libraries.
+function(ratis_target_link_dual_libraries LIBNAME)
+    target_link_libraries(${LIBNAME} ${ARGN})
+    target_link_libraries(${LIBNAME}_static ${ARGN})
+endfunction()
+
+# Set all the output directories to the same place.
+function(ratis_output_directory TGT DIR)
+    set_target_properties(${TGT} PROPERTIES RUNTIME_OUTPUT_DIRECTORY "${CMAKE_BINARY_DIR}/${DIR}")
+    set_target_properties(${TGT} PROPERTIES ARCHIVE_OUTPUT_DIRECTORY "${CMAKE_BINARY_DIR}/${DIR}")
+    set_target_properties(${TGT} PROPERTIES LIBRARY_OUTPUT_DIRECTORY "${CMAKE_BINARY_DIR}/${DIR}")
+endfunction()
+
+# Set the target directories for dynamic and static builds.
+function(ratis_dual_output_directory TGT DIR)
+    ratis_output_directory(${TGT} "${DIR}")
+    ratis_output_directory(${TGT}_static "${DIR}")
+endfunction()
+
+# Alter the behavior of find_package and find_library so that we find only
+# shared libraries with a given version suffix.  You should save
+# CMAKE_FIND_LIBRARY_SUFFIXES before calling this function and restore it
+# afterwards.  On Windows this function is a no-op.  Windows does not encode
+# version number information information into library path names.
+macro(ratis_set_find_shared_library_version LVERS)
+    if(${CMAKE_SYSTEM_NAME} MATCHES "Darwin")
+        # Mac OS uses .dylib
+        set(CMAKE_FIND_LIBRARY_SUFFIXES ".${LVERS}.dylib")
+    elseif(${CMAKE_SYSTEM_NAME} MATCHES "FreeBSD")
+        # FreeBSD has always .so installed.
+        set(CMAKE_FIND_LIBRARY_SUFFIXES ".so")
+    elseif(${CMAKE_SYSTEM_NAME} MATCHES "Windows")
+        # Windows doesn't support finding shared libraries by version.
+    else()
+        # Most UNIX variants use .so
+        set(CMAKE_FIND_LIBRARY_SUFFIXES ".so.${LVERS}")
+    endif()
+endmacro()
+
+# Alter the behavior of find_package and find_library so that we find only
+# shared libraries without any version suffix.  You should save
+# CMAKE_FIND_LIBRARY_SUFFIXES before calling this function and restore it
+# afterwards. On Windows this function is a no-op.  Windows does not encode
+# version number information information into library path names.
+macro(ratis_set_find_shared_library_without_version)
+    if(${CMAKE_SYSTEM_NAME} MATCHES "Darwin")
+        # Mac OS uses .dylib
+        set(CMAKE_FIND_LIBRARY_SUFFIXES ".dylib")
+    elseif(${CMAKE_SYSTEM_NAME} MATCHES "Windows")
+        # No effect
+    else()
+        # Most UNIX variants use .so
+        set(CMAKE_FIND_LIBRARY_SUFFIXES ".so")
+    endif()
+endmacro()
+
+#
+# Configuration.
+#
+
+# Initialise the shared gcc/g++ flags if they aren't already defined.
+if(NOT DEFINED GCC_SHARED_FLAGS)
+    set(GCC_SHARED_FLAGS "-g -O2 -Wall -pthread -D_FILE_OFFSET_BITS=64")
+endif()
+
+# Add in support other compilers here, if necessary,
+# the assumption is that GCC or a GCC-compatible compiler is being used.
+
+# Set the shared GCC-compatible compiler and linker flags.
+ratis_add_compiler_flags("${GCC_SHARED_FLAGS}")
+ratis_add_linker_flags("${LINKER_SHARED_FLAGS}")
+
+#
+# Linux-specific configuration.
+#
+if(CMAKE_SYSTEM_NAME STREQUAL "Linux")
+    # Make GNU extensions available.
+    ratis_add_compiler_flags("-D_GNU_SOURCE")
+
+    # If JVM_ARCH_DATA_MODEL is 32, compile all binaries as 32-bit.
+    if(JVM_ARCH_DATA_MODEL EQUAL 32)
+        # Force 32-bit code generation on amd64/x86_64, ppc64, sparc64
+        if(CMAKE_COMPILER_IS_GNUCC AND CMAKE_SYSTEM_PROCESSOR MATCHES ".*64")
+            ratis_add_compiler_flags("-m32")
+            ratis_add_linker_flags("-m32")
+        endif()
+        # Set CMAKE_SYSTEM_PROCESSOR to ensure that find_package(JNI) will use 32-bit libraries
+        if(CMAKE_SYSTEM_PROCESSOR STREQUAL "x86_64" OR CMAKE_SYSTEM_PROCESSOR STREQUAL "amd64")
+            set(CMAKE_SYSTEM_PROCESSOR "i686")
+        endif()
+    endif()
+
+    # Determine float ABI of JVM on ARM.
+    if(CMAKE_SYSTEM_PROCESSOR MATCHES "^arm")
+        find_program(READELF readelf)
+        if(READELF MATCHES "NOTFOUND")
+            message(WARNING "readelf not found; JVM float ABI detection disabled")
+        else(READELF MATCHES "NOTFOUND")
+            execute_process(
+                COMMAND ${READELF} -A ${JAVA_JVM_LIBRARY}
+                OUTPUT_VARIABLE JVM_ELF_ARCH
+                ERROR_QUIET)
+            if(NOT JVM_ELF_ARCH MATCHES "Tag_ABI_VFP_args: VFP registers")
+                # Test compilation with -mfloat-abi=softfp using an arbitrary libc function
+                # (typically fails with "fatal error: bits/predefs.h: No such file or directory"
+                # if soft-float dev libraries are not installed)
+                message("Soft-float JVM detected")
+                include(CMakePushCheckState)
+                cmake_push_check_state()
+                set(CMAKE_REQUIRED_FLAGS "${CMAKE_REQUIRED_FLAGS} -mfloat-abi=softfp")
+                include(CheckSymbolExists)
+                check_symbol_exists(exit stdlib.h SOFTFP_AVAILABLE)
+                if(NOT SOFTFP_AVAILABLE)
+                    message(FATAL_ERROR "Soft-float dev libraries required (e.g. 'apt-get install libc6-dev-armel' on Debian/Ubuntu)")
+                endif()
+                cmake_pop_check_state()
+                ratis_add_compiler_flags("-mfloat-abi=softfp")
+            endif()
+        endif()
+    endif()
+
+#
+# Solaris-specific configuration.
+#
+elseif(CMAKE_SYSTEM_NAME STREQUAL "SunOS")
+    # Solaris flags. 64-bit compilation is mandatory, and is checked earlier.
+    ratis_add_compiler_flags("-m64 -D_POSIX_C_SOURCE=200112L -D__EXTENSIONS__ -D_POSIX_PTHREAD_SEMANTICS")
+    set(CMAKE_C_FLAGS "-std=gnu99 ${CMAKE_C_FLAGS}")
+    set(CMAKE_CXX_FLAGS "-std=gnu++98 ${CMAKE_CXX_FLAGS}")
+    ratis_add_linker_flags("-m64")
+
+    # CMAKE_SYSTEM_PROCESSOR is set to the output of 'uname -p', which on Solaris is
+    # the 'lowest' ISA supported, i.e. 'i386' or 'sparc'. However in order for the
+    # standard CMake modules to look in the right places it needs to reflect the required
+    # compilation mode, i.e. 64 bit. We therefore force it to either 'amd64' or 'sparcv9'.
+    if(CMAKE_SYSTEM_PROCESSOR STREQUAL "i386")
+        set(CMAKE_SYSTEM_PROCESSOR "amd64")
+        set(CMAKE_LIBRARY_ARCHITECTURE "amd64")
+    elseif(CMAKE_SYSTEM_PROCESSOR STREQUAL "sparc")
+        set(CMAKE_SYSTEM_PROCESSOR "sparcv9")
+        set(CMAKE_LIBRARY_ARCHITECTURE "sparcv9")
+    else()
+        message(FATAL_ERROR "Unrecognised CMAKE_SYSTEM_PROCESSOR ${CMAKE_SYSTEM_PROCESSOR}")
+    endif()
+endif()

http://git-wip-us.apache.org/repos/asf/incubator-ratis/blob/7e71a2e0/ratis-common/RatisJNI.cmake
----------------------------------------------------------------------
diff --git a/ratis-common/RatisJNI.cmake b/ratis-common/RatisJNI.cmake
new file mode 100644
index 0000000..78d7ffd
--- /dev/null
+++ b/ratis-common/RatisJNI.cmake
@@ -0,0 +1,97 @@
+#
+# 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.
+#
+
+#
+# Common JNI detection for CMake, shared by all Native components.
+#
+
+# Check the JVM_ARCH_DATA_MODEL variable as been set to 32 or 64 by maven.
+if(NOT DEFINED JVM_ARCH_DATA_MODEL)
+    message(FATAL_ERROR "JVM_ARCH_DATA_MODEL is not defined")
+elseif(NOT (JVM_ARCH_DATA_MODEL EQUAL 32 OR JVM_ARCH_DATA_MODEL EQUAL 64))
+    message(FATAL_ERROR "JVM_ARCH_DATA_MODEL is not 32 or 64")
+endif()
+
+#
+# Linux-specific JNI configuration.
+#
+if(CMAKE_SYSTEM_NAME STREQUAL "Linux")
+    # Locate JNI_INCLUDE_DIRS and JNI_LIBRARIES.
+    # Since we were invoked from Maven, we know that the JAVA_HOME environment
+    # variable is valid.  So we ignore system paths here and just use JAVA_HOME.
+    file(TO_CMAKE_PATH "$ENV{JAVA_HOME}" _java_home)
+    if(CMAKE_SYSTEM_PROCESSOR MATCHES "^i.86$")
+        set(_java_libarch "i386")
+    elseif(CMAKE_SYSTEM_PROCESSOR STREQUAL "x86_64" OR CMAKE_SYSTEM_PROCESSOR STREQUAL "amd64")
+        set(_java_libarch "amd64")
+    elseif(CMAKE_SYSTEM_PROCESSOR MATCHES "^arm")
+        set(_java_libarch "arm")
+    elseif(CMAKE_SYSTEM_PROCESSOR MATCHES "^(powerpc|ppc)64le")
+        if(EXISTS "${_java_home}/jre/lib/ppc64le")
+            set(_java_libarch "ppc64le")
+        else()
+            set(_java_libarch "ppc64")
+        endif()
+    else()
+        set(_java_libarch ${CMAKE_SYSTEM_PROCESSOR})
+    endif()
+    set(_JDK_DIRS "${_java_home}/jre/lib/${_java_libarch}/*"
+                  "${_java_home}/jre/lib/${_java_libarch}"
+                  "${_java_home}/jre/lib/*"
+                  "${_java_home}/jre/lib"
+                  "${_java_home}/lib/*"
+                  "${_java_home}/lib"
+                  "${_java_home}/include/*"
+                  "${_java_home}/include"
+                  "${_java_home}"
+    )
+    find_path(JAVA_INCLUDE_PATH
+        NAMES jni.h
+        PATHS ${_JDK_DIRS}
+        NO_DEFAULT_PATH)
+    #In IBM java, it's jniport.h instead of jni_md.h
+    find_path(JAVA_INCLUDE_PATH2
+        NAMES jni_md.h jniport.h
+        PATHS ${_JDK_DIRS}
+        NO_DEFAULT_PATH)
+    set(JNI_INCLUDE_DIRS ${JAVA_INCLUDE_PATH} ${JAVA_INCLUDE_PATH2})
+    find_library(JAVA_JVM_LIBRARY
+        NAMES jvm JavaVM
+        PATHS ${_JDK_DIRS}
+        NO_DEFAULT_PATH)
+    set(JNI_LIBRARIES ${JAVA_JVM_LIBRARY})
+    unset(_java_libarch)
+    unset(_java_home)
+
+    message("JAVA_HOME=${JAVA_HOME}, JAVA_JVM_LIBRARY=${JAVA_JVM_LIBRARY}")
+    message("JAVA_INCLUDE_PATH=${JAVA_INCLUDE_PATH}, JAVA_INCLUDE_PATH2=${JAVA_INCLUDE_PATH2}")
+    if(JAVA_JVM_LIBRARY AND JAVA_INCLUDE_PATH AND JAVA_INCLUDE_PATH2)
+        message("Located all JNI components successfully.")
+    else()
+        message(FATAL_ERROR "Failed to find a viable JVM installation under JAVA_HOME.")
+    endif()
+
+    # Use the standard FindJNI module to locate the JNI components.
+    find_package(JNI REQUIRED)
+
+#
+# Otherwise, use the standard FindJNI module to locate the JNI components.
+#
+else()
+    find_package(JNI REQUIRED)
+endif()

http://git-wip-us.apache.org/repos/asf/incubator-ratis/blob/7e71a2e0/ratis-common/pom.xml
----------------------------------------------------------------------
diff --git a/ratis-common/pom.xml b/ratis-common/pom.xml
new file mode 100644
index 0000000..1cbf058
--- /dev/null
+++ b/ratis-common/pom.xml
@@ -0,0 +1,143 @@
+<?xml version="1.0" encoding="UTF-8"?>
+<!--
+  Licensed 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. See accompanying LICENSE file.
+-->
+<project xmlns="http://maven.apache.org/POM/4.0.0"
+         xmlns:xsi="http://www.w3.org/2001/XMLSchema-instance"
+         xsi:schemaLocation="http://maven.apache.org/POM/4.0.0 http://maven.apache.org/xsd/maven-4.0.0.xsd">
+  <modelVersion>4.0.0</modelVersion>
+  <parent>
+    <artifactId>ratis-project-dist</artifactId>
+    <groupId>org.apache.ratis</groupId>
+    <version>1.0-SNAPSHOT</version>
+    <relativePath>../ratis-project-dist</relativePath>
+  </parent>
+
+  <artifactId>ratis-common</artifactId>
+  <name>Ratis Common</name>
+
+  <dependencies>
+    <dependency>
+      <artifactId>ratis-proto-shaded</artifactId>
+      <groupId>org.apache.ratis</groupId>
+      <scope>provided</scope>
+    </dependency>
+
+    <dependency>
+      <groupId>org.slf4j</groupId>
+      <artifactId>slf4j-api</artifactId>
+    </dependency>
+
+    <dependency>
+      <groupId>com.google.guava</groupId>
+      <artifactId>guava</artifactId>
+    </dependency>
+
+    <dependency>
+      <groupId>junit</groupId>
+      <artifactId>junit</artifactId>
+      <scope>test</scope>
+    </dependency>
+    <dependency>
+      <groupId>org.mockito</groupId>
+      <artifactId>mockito-all</artifactId>
+      <scope>test</scope>
+    </dependency>
+  </dependencies>
+
+  <profiles>
+    <profile>
+      <id>native</id>
+      <activation>
+        <activeByDefault>false</activeByDefault>
+      </activation>
+      <properties>
+        <runningWithNative>true</runningWithNative>
+      </properties>
+      <build>
+        <plugins>
+          <plugin>
+            <groupId>org.apache.maven.plugins</groupId>
+            <artifactId>maven-enforcer-plugin</artifactId>
+            <executions>
+              <execution>
+                <id>enforce-os</id>
+                <goals>
+                  <goal>enforce</goal>
+                </goals>
+                <configuration>
+                  <rules>
+                    <requireOS>
+                      <family>mac</family>
+                      <family>unix</family>
+                      <message>native build only supported on Mac or Unix</message>
+                    </requireOS>
+                  </rules>
+                  <fail>true</fail>
+                </configuration>
+              </execution>
+            </executions>
+          </plugin>
+          <plugin>
+            <groupId>org.codehaus.mojo</groupId>
+            <artifactId>native-maven-plugin</artifactId>
+            <executions>
+              <execution>
+                <phase>compile</phase>
+                <goals>
+                  <goal>javah</goal>
+                </goals>
+                <configuration>
+                  <javahPath>${env.JAVA_HOME}/bin/javah</javahPath>
+                  <javahClassNames>
+                    <javahClassName>org.apache.ratis.io.nativeio.NativeIO</javahClassName>
+                    <javahClassName>org.apache.ratis.util.NativeCrc32</javahClassName>
+                  </javahClassNames>
+                  <javahOutputDirectory>${project.build.directory}/native/javah</javahOutputDirectory>
+                </configuration>
+              </execution>
+            </executions>
+          </plugin>
+          <plugin>
+            <groupId>org.apache.hadoop</groupId>
+            <artifactId>hadoop-maven-plugins</artifactId>
+            <executions>
+              <execution>
+                <id>cmake-compile</id>
+                <phase>compile</phase>
+                <goals><goal>cmake-compile</goal></goals>
+                <configuration>
+                  <source>${basedir}/src</source>
+                  <vars>
+                    <GENERATED_JAVAH>${project.build.directory}/native/javah</GENERATED_JAVAH>
+                    <JVM_ARCH_DATA_MODEL>${sun.arch.data.model}</JVM_ARCH_DATA_MODEL>
+                  </vars>
+                </configuration>
+              </execution>
+              <execution>
+                <id>test_bulk_crc32</id>
+                <goals><goal>cmake-test</goal></goals>
+                <phase>test</phase>
+                <configuration>
+                  <binary>${project.build.directory}/native/test_bulk_crc32</binary>
+                  <timeout>1200</timeout>
+                  <results>${project.build.directory}/native-results</results>
+                </configuration>
+              </execution>
+            </executions>
+          </plugin>
+        </plugins>
+      </build>
+    </profile>
+  </profiles>
+</project>

http://git-wip-us.apache.org/repos/asf/incubator-ratis/blob/7e71a2e0/ratis-common/src/CMakeLists.txt
----------------------------------------------------------------------
diff --git a/ratis-common/src/CMakeLists.txt b/ratis-common/src/CMakeLists.txt
new file mode 100644
index 0000000..460b3b1
--- /dev/null
+++ b/ratis-common/src/CMakeLists.txt
@@ -0,0 +1,108 @@
+#
+# 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.
+#
+
+#
+# CMake configuration.
+#
+
+cmake_minimum_required(VERSION 2.6 FATAL_ERROR)
+
+list(APPEND CMAKE_MODULE_PATH ${CMAKE_SOURCE_DIR}/..)
+include(RatisCommon)
+
+# Source and test locations.
+set(SRC main/native/src/org/apache/ratis)
+set(TST main/native/src/test/org/apache/ratis)
+
+#
+# Main configuration.
+#
+
+# The caller must specify where the generated headers have been placed.
+if(NOT GENERATED_JAVAH)
+    message(FATAL_ERROR "You must set the CMake variable GENERATED_JAVAH")
+endif()
+
+# Configure JNI.
+include(RatisJNI)
+
+# Build hardware CRC32 acceleration, if supported on the platform.
+if(CMAKE_SYSTEM_PROCESSOR MATCHES "^i.86$" OR CMAKE_SYSTEM_PROCESSOR STREQUAL "x86_64" OR CMAKE_SYSTEM_PROCESSOR STREQUAL "amd64")
+  set(BULK_CRC_ARCH_SOURCE_FIlE "${SRC}/util/bulk_crc32_x86.c")
+elseif(CMAKE_SYSTEM_PROCESSOR STREQUAL "aarch64")
+  set(BULK_CRC_ARCH_SOURCE_FIlE "${SRC}/util/bulk_crc32_aarch64.c")
+else()
+  message("No HW CRC acceleration for ${CMAKE_SYSTEM_PROCESSOR}, falling back to SW")
+endif()
+
+# Check for platform-specific functions and libraries.
+include(CheckFunctionExists)
+include(CheckLibraryExists)
+check_function_exists(sync_file_range HAVE_SYNC_FILE_RANGE)
+check_function_exists(posix_fadvise HAVE_POSIX_FADVISE)
+check_library_exists(dl dlopen "" NEED_LINK_DL)
+
+# Configure the build.
+include_directories(
+    ${GENERATED_JAVAH}
+    main/native/src
+    ${CMAKE_CURRENT_SOURCE_DIR}
+    ${CMAKE_CURRENT_SOURCE_DIR}/src
+    ${CMAKE_BINARY_DIR}
+    ${JNI_INCLUDE_DIRS}
+    ${SRC}/util
+)
+configure_file(${CMAKE_SOURCE_DIR}/config.h.cmake ${CMAKE_BINARY_DIR}/config.h)
+
+set(CMAKE_BUILD_WITH_INSTALL_RPATH TRUE)
+ratis_add_dual_library(ratis
+    main/native/src/exception.c
+    ${SRC}/io/nativeio/NativeIO.c
+    ${SRC}/io/nativeio/errno_enum.c
+    ${SRC}/io/nativeio/file_descriptor.c
+    ${SRC}/util/NativeCodeLoader.c
+    ${SRC}/util/NativeCrc32.c
+    ${SRC}/util/bulk_crc32.c
+    ${BULK_CRC_ARCH_SOURCE_FIlE}
+)
+if(NEED_LINK_DL)
+   set(LIB_DL dl)
+endif()
+
+ratis_target_link_dual_libraries(ratis ${LIB_DL} ${JAVA_JVM_LIBRARY})
+set(LIBRATIS_VERSION "1.0.0")
+set_target_properties(ratis PROPERTIES SOVERSION ${LIBRATIS_VERSION})
+ratis_dual_output_directory(ratis target/usr/local/lib)
+
+# By embedding '$ORIGIN' into the RPATH of libratis.so, dlopen will look in
+# the directory containing libratis.so. However, $ORIGIN is not supported by
+# all operating systems.
+if(${CMAKE_SYSTEM_NAME} MATCHES "Linux|SunOS")
+    set(RPATH "\$ORIGIN/")
+    if(EXTRA_LIBRATIS_RPATH)
+        set(RPATH "${RPATH}:${EXTRA_LIBRATIS_RPATH}/")
+    endif()
+    set_target_properties(ratis PROPERTIES INSTALL_RPATH "${RPATH}")
+endif()
+
+# Build the CRC32 test executable.
+add_executable(test_bulk_crc32
+    ${SRC}/util/bulk_crc32.c
+    ${BULK_CRC_ARCH_SOURCE_FIlE}
+    ${TST}/util/test_bulk_crc32.c
+)

http://git-wip-us.apache.org/repos/asf/incubator-ratis/blob/7e71a2e0/ratis-common/src/config.h.cmake
----------------------------------------------------------------------
diff --git a/ratis-common/src/config.h.cmake b/ratis-common/src/config.h.cmake
new file mode 100644
index 0000000..709fc75
--- /dev/null
+++ b/ratis-common/src/config.h.cmake
@@ -0,0 +1,24 @@
+/**
+* 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.
+*/
+#ifndef CONFIG_H
+#define CONFIG_H
+
+#cmakedefine HAVE_SYNC_FILE_RANGE
+#cmakedefine HAVE_POSIX_FADVISE
+
+#endif

http://git-wip-us.apache.org/repos/asf/incubator-ratis/blob/7e71a2e0/ratis-common/src/main/conf/log4j.properties
----------------------------------------------------------------------
diff --git a/ratis-common/src/main/conf/log4j.properties b/ratis-common/src/main/conf/log4j.properties
new file mode 100644
index 0000000..64c1922
--- /dev/null
+++ b/ratis-common/src/main/conf/log4j.properties
@@ -0,0 +1,74 @@
+# 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.
+
+# Define some default values that can be overridden by system properties
+ratis.root.logger=INFO,console
+ratis.log.dir=.
+ratis.log.file=ratis.log
+
+# Define the root logger to the system property "ratis.root.logger".
+log4j.rootLogger=${ratis.root.logger}
+
+# Logging Threshold
+log4j.threshold=ALL
+
+# Null Appender
+log4j.appender.NullAppender=org.apache.log4j.varia.NullAppender
+
+#
+# Rolling File Appender - cap space usage at 5gb.
+#
+ratis.log.maxfilesize=256MB
+ratis.log.maxbackupindex=20
+log4j.appender.RFA=org.apache.log4j.RollingFileAppender
+log4j.appender.RFA.File=${ratis.log.dir}/${ratis.log.file}
+
+log4j.appender.RFA.MaxFileSize=${ratis.log.maxfilesize}
+log4j.appender.RFA.MaxBackupIndex=${ratis.log.maxbackupindex}
+
+log4j.appender.RFA.layout=org.apache.log4j.PatternLayout
+
+# Pattern format: Date LogLevel LoggerName LogMessage
+log4j.appender.RFA.layout.ConversionPattern=%d{ISO8601} %p %c: %m%n
+# Debugging Pattern format
+#log4j.appender.RFA.layout.ConversionPattern=%d{ISO8601} %-5p %c{2} (%F:%M(%L)) - %m%n
+
+
+#
+# Daily Rolling File Appender
+#
+log4j.appender.DRFA=org.apache.log4j.DailyRollingFileAppender
+log4j.appender.DRFA.File=${ratis.log.dir}/${ratis.log.file}
+
+# Rollover at midnight
+log4j.appender.DRFA.DatePattern=.yyyy-MM-dd
+
+log4j.appender.DRFA.layout=org.apache.log4j.PatternLayout
+
+# Pattern format: Date LogLevel LoggerName LogMessage
+log4j.appender.DRFA.layout.ConversionPattern=%d{ISO8601} %p %c: %m%n
+# Debugging Pattern format
+#log4j.appender.DRFA.layout.ConversionPattern=%d{ISO8601} %-5p %c{2} (%F:%M(%L)) - %m%n
+
+
+#
+# console
+# Add "console" to rootlogger above if you want to use this
+#
+log4j.appender.console=org.apache.log4j.ConsoleAppender
+log4j.appender.console.target=System.err
+log4j.appender.console.layout=org.apache.log4j.PatternLayout
+log4j.appender.console.layout.ConversionPattern=%d{yy/MM/dd HH:mm:ss} %p %c{2}: %m%n



[45/54] [abbrv] incubator-ratis git commit: Renamed the packages from raft to ratis in preperation for Apache Incubation - Moved all java packages from org.apache.raft to org.apache.ratis. - Moved native package to org_apache_ratis, and native lib to l

Posted by ji...@apache.org.
http://git-wip-us.apache.org/repos/asf/incubator-ratis/blob/7e71a2e0/raft-common/src/main/java/org/apache/raft/util/PureJavaCrc32C.java
----------------------------------------------------------------------
diff --git a/raft-common/src/main/java/org/apache/raft/util/PureJavaCrc32C.java b/raft-common/src/main/java/org/apache/raft/util/PureJavaCrc32C.java
deleted file mode 100644
index 8d7321a..0000000
--- a/raft-common/src/main/java/org/apache/raft/util/PureJavaCrc32C.java
+++ /dev/null
@@ -1,632 +0,0 @@
-/**
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- *     http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- * 
- * Some portions of this file Copyright (c) 2004-2006 Intel Corportation
- * and licensed under the BSD license.
- */
-package org.apache.raft.util;
-
-import java.util.zip.Checksum;
-
-/**
- * A pure-java implementation of the CRC32 checksum that uses
- * the CRC32-C polynomial, the same polynomial used by iSCSI
- * and implemented on many Intel chipsets supporting SSE4.2.
- */
-public class PureJavaCrc32C implements Checksum {
-
-  /** the current CRC value, bit-flipped */
-  private int crc;
-
-  /** Create a new PureJavaCrc32 object. */
-  public PureJavaCrc32C() {
-    reset();
-  }
-
-  @Override
-  public long getValue() {
-    long ret = crc;
-    return (~ret) & 0xffffffffL;
-  }
-
-  @Override
-  public void reset() {
-    crc = 0xffffffff;
-  }
-
-  @Override
-  public void update(byte[] b, int off, int len) {
-    int localCrc = crc;
-
-    while(len > 7) {
-      final int c0 =(b[off+0] ^ localCrc) & 0xff;
-      final int c1 =(b[off+1] ^ (localCrc >>>= 8)) & 0xff;
-      final int c2 =(b[off+2] ^ (localCrc >>>= 8)) & 0xff;
-      final int c3 =(b[off+3] ^ (localCrc >>>= 8)) & 0xff;
-      localCrc = (T[T8_7_start + c0] ^ T[T8_6_start + c1])
-          ^ (T[T8_5_start + c2] ^ T[T8_4_start + c3]);
-
-      final int c4 = b[off+4] & 0xff;
-      final int c5 = b[off+5] & 0xff;
-      final int c6 = b[off+6] & 0xff;
-      final int c7 = b[off+7] & 0xff;
-
-      localCrc ^= (T[T8_3_start + c4] ^ T[T8_2_start + c5])
-           ^ (T[T8_1_start + c6] ^ T[T8_0_start + c7]);
-
-      off += 8;
-      len -= 8;
-    }
-
-    /* loop unroll - duff's device style */
-    switch(len) {
-      case 7: localCrc = (localCrc >>> 8) ^ T[T8_0_start + ((localCrc ^ b[off++]) & 0xff)];
-      case 6: localCrc = (localCrc >>> 8) ^ T[T8_0_start + ((localCrc ^ b[off++]) & 0xff)];
-      case 5: localCrc = (localCrc >>> 8) ^ T[T8_0_start + ((localCrc ^ b[off++]) & 0xff)];
-      case 4: localCrc = (localCrc >>> 8) ^ T[T8_0_start + ((localCrc ^ b[off++]) & 0xff)];
-      case 3: localCrc = (localCrc >>> 8) ^ T[T8_0_start + ((localCrc ^ b[off++]) & 0xff)];
-      case 2: localCrc = (localCrc >>> 8) ^ T[T8_0_start + ((localCrc ^ b[off++]) & 0xff)];
-      case 1: localCrc = (localCrc >>> 8) ^ T[T8_0_start + ((localCrc ^ b[off++]) & 0xff)];
-      default:
-        /* nothing */
-    }
-    
-    // Publish crc out to object
-    crc = localCrc;
-  }
-
-  @Override
-  final public void update(int b) {
-    crc = (crc >>> 8) ^ T[T8_0_start + ((crc ^ b) & 0xff)];
-  }
-    
-  // CRC polynomial tables generated by:
-  // java -cp build/test/classes/:build/classes/ \
-  //   org.apache.hadoop.util.TestPureJavaCrc32\$Table 82F63B78
-
-  private static final int T8_0_start = 0*256;
-  private static final int T8_1_start = 1*256;
-  private static final int T8_2_start = 2*256;
-  private static final int T8_3_start = 3*256;
-  private static final int T8_4_start = 4*256;
-  private static final int T8_5_start = 5*256;
-  private static final int T8_6_start = 6*256;
-  private static final int T8_7_start = 7*256;
-
-  private static final int[] T = new int[] {
-    /* T8_0 */
-    0x00000000, 0xF26B8303, 0xE13B70F7, 0x1350F3F4, 
-    0xC79A971F, 0x35F1141C, 0x26A1E7E8, 0xD4CA64EB, 
-    0x8AD958CF, 0x78B2DBCC, 0x6BE22838, 0x9989AB3B, 
-    0x4D43CFD0, 0xBF284CD3, 0xAC78BF27, 0x5E133C24, 
-    0x105EC76F, 0xE235446C, 0xF165B798, 0x030E349B, 
-    0xD7C45070, 0x25AFD373, 0x36FF2087, 0xC494A384, 
-    0x9A879FA0, 0x68EC1CA3, 0x7BBCEF57, 0x89D76C54, 
-    0x5D1D08BF, 0xAF768BBC, 0xBC267848, 0x4E4DFB4B, 
-    0x20BD8EDE, 0xD2D60DDD, 0xC186FE29, 0x33ED7D2A, 
-    0xE72719C1, 0x154C9AC2, 0x061C6936, 0xF477EA35, 
-    0xAA64D611, 0x580F5512, 0x4B5FA6E6, 0xB93425E5, 
-    0x6DFE410E, 0x9F95C20D, 0x8CC531F9, 0x7EAEB2FA, 
-    0x30E349B1, 0xC288CAB2, 0xD1D83946, 0x23B3BA45, 
-    0xF779DEAE, 0x05125DAD, 0x1642AE59, 0xE4292D5A, 
-    0xBA3A117E, 0x4851927D, 0x5B016189, 0xA96AE28A, 
-    0x7DA08661, 0x8FCB0562, 0x9C9BF696, 0x6EF07595, 
-    0x417B1DBC, 0xB3109EBF, 0xA0406D4B, 0x522BEE48, 
-    0x86E18AA3, 0x748A09A0, 0x67DAFA54, 0x95B17957, 
-    0xCBA24573, 0x39C9C670, 0x2A993584, 0xD8F2B687, 
-    0x0C38D26C, 0xFE53516F, 0xED03A29B, 0x1F682198, 
-    0x5125DAD3, 0xA34E59D0, 0xB01EAA24, 0x42752927, 
-    0x96BF4DCC, 0x64D4CECF, 0x77843D3B, 0x85EFBE38, 
-    0xDBFC821C, 0x2997011F, 0x3AC7F2EB, 0xC8AC71E8, 
-    0x1C661503, 0xEE0D9600, 0xFD5D65F4, 0x0F36E6F7, 
-    0x61C69362, 0x93AD1061, 0x80FDE395, 0x72966096, 
-    0xA65C047D, 0x5437877E, 0x4767748A, 0xB50CF789, 
-    0xEB1FCBAD, 0x197448AE, 0x0A24BB5A, 0xF84F3859, 
-    0x2C855CB2, 0xDEEEDFB1, 0xCDBE2C45, 0x3FD5AF46, 
-    0x7198540D, 0x83F3D70E, 0x90A324FA, 0x62C8A7F9, 
-    0xB602C312, 0x44694011, 0x5739B3E5, 0xA55230E6, 
-    0xFB410CC2, 0x092A8FC1, 0x1A7A7C35, 0xE811FF36, 
-    0x3CDB9BDD, 0xCEB018DE, 0xDDE0EB2A, 0x2F8B6829, 
-    0x82F63B78, 0x709DB87B, 0x63CD4B8F, 0x91A6C88C, 
-    0x456CAC67, 0xB7072F64, 0xA457DC90, 0x563C5F93, 
-    0x082F63B7, 0xFA44E0B4, 0xE9141340, 0x1B7F9043, 
-    0xCFB5F4A8, 0x3DDE77AB, 0x2E8E845F, 0xDCE5075C, 
-    0x92A8FC17, 0x60C37F14, 0x73938CE0, 0x81F80FE3, 
-    0x55326B08, 0xA759E80B, 0xB4091BFF, 0x466298FC, 
-    0x1871A4D8, 0xEA1A27DB, 0xF94AD42F, 0x0B21572C, 
-    0xDFEB33C7, 0x2D80B0C4, 0x3ED04330, 0xCCBBC033, 
-    0xA24BB5A6, 0x502036A5, 0x4370C551, 0xB11B4652, 
-    0x65D122B9, 0x97BAA1BA, 0x84EA524E, 0x7681D14D, 
-    0x2892ED69, 0xDAF96E6A, 0xC9A99D9E, 0x3BC21E9D, 
-    0xEF087A76, 0x1D63F975, 0x0E330A81, 0xFC588982, 
-    0xB21572C9, 0x407EF1CA, 0x532E023E, 0xA145813D, 
-    0x758FE5D6, 0x87E466D5, 0x94B49521, 0x66DF1622, 
-    0x38CC2A06, 0xCAA7A905, 0xD9F75AF1, 0x2B9CD9F2, 
-    0xFF56BD19, 0x0D3D3E1A, 0x1E6DCDEE, 0xEC064EED, 
-    0xC38D26C4, 0x31E6A5C7, 0x22B65633, 0xD0DDD530, 
-    0x0417B1DB, 0xF67C32D8, 0xE52CC12C, 0x1747422F, 
-    0x49547E0B, 0xBB3FFD08, 0xA86F0EFC, 0x5A048DFF, 
-    0x8ECEE914, 0x7CA56A17, 0x6FF599E3, 0x9D9E1AE0, 
-    0xD3D3E1AB, 0x21B862A8, 0x32E8915C, 0xC083125F, 
-    0x144976B4, 0xE622F5B7, 0xF5720643, 0x07198540, 
-    0x590AB964, 0xAB613A67, 0xB831C993, 0x4A5A4A90, 
-    0x9E902E7B, 0x6CFBAD78, 0x7FAB5E8C, 0x8DC0DD8F, 
-    0xE330A81A, 0x115B2B19, 0x020BD8ED, 0xF0605BEE, 
-    0x24AA3F05, 0xD6C1BC06, 0xC5914FF2, 0x37FACCF1, 
-    0x69E9F0D5, 0x9B8273D6, 0x88D28022, 0x7AB90321, 
-    0xAE7367CA, 0x5C18E4C9, 0x4F48173D, 0xBD23943E, 
-    0xF36E6F75, 0x0105EC76, 0x12551F82, 0xE03E9C81, 
-    0x34F4F86A, 0xC69F7B69, 0xD5CF889D, 0x27A40B9E, 
-    0x79B737BA, 0x8BDCB4B9, 0x988C474D, 0x6AE7C44E, 
-    0xBE2DA0A5, 0x4C4623A6, 0x5F16D052, 0xAD7D5351, 
-    /* T8_1 */
-    0x00000000, 0x13A29877, 0x274530EE, 0x34E7A899, 
-    0x4E8A61DC, 0x5D28F9AB, 0x69CF5132, 0x7A6DC945, 
-    0x9D14C3B8, 0x8EB65BCF, 0xBA51F356, 0xA9F36B21, 
-    0xD39EA264, 0xC03C3A13, 0xF4DB928A, 0xE7790AFD, 
-    0x3FC5F181, 0x2C6769F6, 0x1880C16F, 0x0B225918, 
-    0x714F905D, 0x62ED082A, 0x560AA0B3, 0x45A838C4, 
-    0xA2D13239, 0xB173AA4E, 0x859402D7, 0x96369AA0, 
-    0xEC5B53E5, 0xFFF9CB92, 0xCB1E630B, 0xD8BCFB7C, 
-    0x7F8BE302, 0x6C297B75, 0x58CED3EC, 0x4B6C4B9B, 
-    0x310182DE, 0x22A31AA9, 0x1644B230, 0x05E62A47, 
-    0xE29F20BA, 0xF13DB8CD, 0xC5DA1054, 0xD6788823, 
-    0xAC154166, 0xBFB7D911, 0x8B507188, 0x98F2E9FF, 
-    0x404E1283, 0x53EC8AF4, 0x670B226D, 0x74A9BA1A, 
-    0x0EC4735F, 0x1D66EB28, 0x298143B1, 0x3A23DBC6, 
-    0xDD5AD13B, 0xCEF8494C, 0xFA1FE1D5, 0xE9BD79A2, 
-    0x93D0B0E7, 0x80722890, 0xB4958009, 0xA737187E, 
-    0xFF17C604, 0xECB55E73, 0xD852F6EA, 0xCBF06E9D, 
-    0xB19DA7D8, 0xA23F3FAF, 0x96D89736, 0x857A0F41, 
-    0x620305BC, 0x71A19DCB, 0x45463552, 0x56E4AD25, 
-    0x2C896460, 0x3F2BFC17, 0x0BCC548E, 0x186ECCF9, 
-    0xC0D23785, 0xD370AFF2, 0xE797076B, 0xF4359F1C, 
-    0x8E585659, 0x9DFACE2E, 0xA91D66B7, 0xBABFFEC0, 
-    0x5DC6F43D, 0x4E646C4A, 0x7A83C4D3, 0x69215CA4, 
-    0x134C95E1, 0x00EE0D96, 0x3409A50F, 0x27AB3D78, 
-    0x809C2506, 0x933EBD71, 0xA7D915E8, 0xB47B8D9F, 
-    0xCE1644DA, 0xDDB4DCAD, 0xE9537434, 0xFAF1EC43, 
-    0x1D88E6BE, 0x0E2A7EC9, 0x3ACDD650, 0x296F4E27, 
-    0x53028762, 0x40A01F15, 0x7447B78C, 0x67E52FFB, 
-    0xBF59D487, 0xACFB4CF0, 0x981CE469, 0x8BBE7C1E, 
-    0xF1D3B55B, 0xE2712D2C, 0xD69685B5, 0xC5341DC2, 
-    0x224D173F, 0x31EF8F48, 0x050827D1, 0x16AABFA6, 
-    0x6CC776E3, 0x7F65EE94, 0x4B82460D, 0x5820DE7A, 
-    0xFBC3FAF9, 0xE861628E, 0xDC86CA17, 0xCF245260, 
-    0xB5499B25, 0xA6EB0352, 0x920CABCB, 0x81AE33BC, 
-    0x66D73941, 0x7575A136, 0x419209AF, 0x523091D8, 
-    0x285D589D, 0x3BFFC0EA, 0x0F186873, 0x1CBAF004, 
-    0xC4060B78, 0xD7A4930F, 0xE3433B96, 0xF0E1A3E1, 
-    0x8A8C6AA4, 0x992EF2D3, 0xADC95A4A, 0xBE6BC23D, 
-    0x5912C8C0, 0x4AB050B7, 0x7E57F82E, 0x6DF56059, 
-    0x1798A91C, 0x043A316B, 0x30DD99F2, 0x237F0185, 
-    0x844819FB, 0x97EA818C, 0xA30D2915, 0xB0AFB162, 
-    0xCAC27827, 0xD960E050, 0xED8748C9, 0xFE25D0BE, 
-    0x195CDA43, 0x0AFE4234, 0x3E19EAAD, 0x2DBB72DA, 
-    0x57D6BB9F, 0x447423E8, 0x70938B71, 0x63311306, 
-    0xBB8DE87A, 0xA82F700D, 0x9CC8D894, 0x8F6A40E3, 
-    0xF50789A6, 0xE6A511D1, 0xD242B948, 0xC1E0213F, 
-    0x26992BC2, 0x353BB3B5, 0x01DC1B2C, 0x127E835B, 
-    0x68134A1E, 0x7BB1D269, 0x4F567AF0, 0x5CF4E287, 
-    0x04D43CFD, 0x1776A48A, 0x23910C13, 0x30339464, 
-    0x4A5E5D21, 0x59FCC556, 0x6D1B6DCF, 0x7EB9F5B8, 
-    0x99C0FF45, 0x8A626732, 0xBE85CFAB, 0xAD2757DC, 
-    0xD74A9E99, 0xC4E806EE, 0xF00FAE77, 0xE3AD3600, 
-    0x3B11CD7C, 0x28B3550B, 0x1C54FD92, 0x0FF665E5, 
-    0x759BACA0, 0x663934D7, 0x52DE9C4E, 0x417C0439, 
-    0xA6050EC4, 0xB5A796B3, 0x81403E2A, 0x92E2A65D, 
-    0xE88F6F18, 0xFB2DF76F, 0xCFCA5FF6, 0xDC68C781, 
-    0x7B5FDFFF, 0x68FD4788, 0x5C1AEF11, 0x4FB87766, 
-    0x35D5BE23, 0x26772654, 0x12908ECD, 0x013216BA, 
-    0xE64B1C47, 0xF5E98430, 0xC10E2CA9, 0xD2ACB4DE, 
-    0xA8C17D9B, 0xBB63E5EC, 0x8F844D75, 0x9C26D502, 
-    0x449A2E7E, 0x5738B609, 0x63DF1E90, 0x707D86E7, 
-    0x0A104FA2, 0x19B2D7D5, 0x2D557F4C, 0x3EF7E73B, 
-    0xD98EEDC6, 0xCA2C75B1, 0xFECBDD28, 0xED69455F, 
-    0x97048C1A, 0x84A6146D, 0xB041BCF4, 0xA3E32483, 
-    /* T8_2 */
-    0x00000000, 0xA541927E, 0x4F6F520D, 0xEA2EC073, 
-    0x9EDEA41A, 0x3B9F3664, 0xD1B1F617, 0x74F06469, 
-    0x38513EC5, 0x9D10ACBB, 0x773E6CC8, 0xD27FFEB6, 
-    0xA68F9ADF, 0x03CE08A1, 0xE9E0C8D2, 0x4CA15AAC, 
-    0x70A27D8A, 0xD5E3EFF4, 0x3FCD2F87, 0x9A8CBDF9, 
-    0xEE7CD990, 0x4B3D4BEE, 0xA1138B9D, 0x045219E3, 
-    0x48F3434F, 0xEDB2D131, 0x079C1142, 0xA2DD833C, 
-    0xD62DE755, 0x736C752B, 0x9942B558, 0x3C032726, 
-    0xE144FB14, 0x4405696A, 0xAE2BA919, 0x0B6A3B67, 
-    0x7F9A5F0E, 0xDADBCD70, 0x30F50D03, 0x95B49F7D, 
-    0xD915C5D1, 0x7C5457AF, 0x967A97DC, 0x333B05A2, 
-    0x47CB61CB, 0xE28AF3B5, 0x08A433C6, 0xADE5A1B8, 
-    0x91E6869E, 0x34A714E0, 0xDE89D493, 0x7BC846ED, 
-    0x0F382284, 0xAA79B0FA, 0x40577089, 0xE516E2F7, 
-    0xA9B7B85B, 0x0CF62A25, 0xE6D8EA56, 0x43997828, 
-    0x37691C41, 0x92288E3F, 0x78064E4C, 0xDD47DC32, 
-    0xC76580D9, 0x622412A7, 0x880AD2D4, 0x2D4B40AA, 
-    0x59BB24C3, 0xFCFAB6BD, 0x16D476CE, 0xB395E4B0, 
-    0xFF34BE1C, 0x5A752C62, 0xB05BEC11, 0x151A7E6F, 
-    0x61EA1A06, 0xC4AB8878, 0x2E85480B, 0x8BC4DA75, 
-    0xB7C7FD53, 0x12866F2D, 0xF8A8AF5E, 0x5DE93D20, 
-    0x29195949, 0x8C58CB37, 0x66760B44, 0xC337993A, 
-    0x8F96C396, 0x2AD751E8, 0xC0F9919B, 0x65B803E5, 
-    0x1148678C, 0xB409F5F2, 0x5E273581, 0xFB66A7FF, 
-    0x26217BCD, 0x8360E9B3, 0x694E29C0, 0xCC0FBBBE, 
-    0xB8FFDFD7, 0x1DBE4DA9, 0xF7908DDA, 0x52D11FA4, 
-    0x1E704508, 0xBB31D776, 0x511F1705, 0xF45E857B, 
-    0x80AEE112, 0x25EF736C, 0xCFC1B31F, 0x6A802161, 
-    0x56830647, 0xF3C29439, 0x19EC544A, 0xBCADC634, 
-    0xC85DA25D, 0x6D1C3023, 0x8732F050, 0x2273622E, 
-    0x6ED23882, 0xCB93AAFC, 0x21BD6A8F, 0x84FCF8F1, 
-    0xF00C9C98, 0x554D0EE6, 0xBF63CE95, 0x1A225CEB, 
-    0x8B277743, 0x2E66E53D, 0xC448254E, 0x6109B730, 
-    0x15F9D359, 0xB0B84127, 0x5A968154, 0xFFD7132A, 
-    0xB3764986, 0x1637DBF8, 0xFC191B8B, 0x595889F5, 
-    0x2DA8ED9C, 0x88E97FE2, 0x62C7BF91, 0xC7862DEF, 
-    0xFB850AC9, 0x5EC498B7, 0xB4EA58C4, 0x11ABCABA, 
-    0x655BAED3, 0xC01A3CAD, 0x2A34FCDE, 0x8F756EA0, 
-    0xC3D4340C, 0x6695A672, 0x8CBB6601, 0x29FAF47F, 
-    0x5D0A9016, 0xF84B0268, 0x1265C21B, 0xB7245065, 
-    0x6A638C57, 0xCF221E29, 0x250CDE5A, 0x804D4C24, 
-    0xF4BD284D, 0x51FCBA33, 0xBBD27A40, 0x1E93E83E, 
-    0x5232B292, 0xF77320EC, 0x1D5DE09F, 0xB81C72E1, 
-    0xCCEC1688, 0x69AD84F6, 0x83834485, 0x26C2D6FB, 
-    0x1AC1F1DD, 0xBF8063A3, 0x55AEA3D0, 0xF0EF31AE, 
-    0x841F55C7, 0x215EC7B9, 0xCB7007CA, 0x6E3195B4, 
-    0x2290CF18, 0x87D15D66, 0x6DFF9D15, 0xC8BE0F6B, 
-    0xBC4E6B02, 0x190FF97C, 0xF321390F, 0x5660AB71, 
-    0x4C42F79A, 0xE90365E4, 0x032DA597, 0xA66C37E9, 
-    0xD29C5380, 0x77DDC1FE, 0x9DF3018D, 0x38B293F3, 
-    0x7413C95F, 0xD1525B21, 0x3B7C9B52, 0x9E3D092C, 
-    0xEACD6D45, 0x4F8CFF3B, 0xA5A23F48, 0x00E3AD36, 
-    0x3CE08A10, 0x99A1186E, 0x738FD81D, 0xD6CE4A63, 
-    0xA23E2E0A, 0x077FBC74, 0xED517C07, 0x4810EE79, 
-    0x04B1B4D5, 0xA1F026AB, 0x4BDEE6D8, 0xEE9F74A6, 
-    0x9A6F10CF, 0x3F2E82B1, 0xD50042C2, 0x7041D0BC, 
-    0xAD060C8E, 0x08479EF0, 0xE2695E83, 0x4728CCFD, 
-    0x33D8A894, 0x96993AEA, 0x7CB7FA99, 0xD9F668E7, 
-    0x9557324B, 0x3016A035, 0xDA386046, 0x7F79F238, 
-    0x0B899651, 0xAEC8042F, 0x44E6C45C, 0xE1A75622, 
-    0xDDA47104, 0x78E5E37A, 0x92CB2309, 0x378AB177, 
-    0x437AD51E, 0xE63B4760, 0x0C158713, 0xA954156D, 
-    0xE5F54FC1, 0x40B4DDBF, 0xAA9A1DCC, 0x0FDB8FB2, 
-    0x7B2BEBDB, 0xDE6A79A5, 0x3444B9D6, 0x91052BA8, 
-    /* T8_3 */
-    0x00000000, 0xDD45AAB8, 0xBF672381, 0x62228939, 
-    0x7B2231F3, 0xA6679B4B, 0xC4451272, 0x1900B8CA, 
-    0xF64463E6, 0x2B01C95E, 0x49234067, 0x9466EADF, 
-    0x8D665215, 0x5023F8AD, 0x32017194, 0xEF44DB2C, 
-    0xE964B13D, 0x34211B85, 0x560392BC, 0x8B463804, 
-    0x924680CE, 0x4F032A76, 0x2D21A34F, 0xF06409F7, 
-    0x1F20D2DB, 0xC2657863, 0xA047F15A, 0x7D025BE2, 
-    0x6402E328, 0xB9474990, 0xDB65C0A9, 0x06206A11, 
-    0xD725148B, 0x0A60BE33, 0x6842370A, 0xB5079DB2, 
-    0xAC072578, 0x71428FC0, 0x136006F9, 0xCE25AC41, 
-    0x2161776D, 0xFC24DDD5, 0x9E0654EC, 0x4343FE54, 
-    0x5A43469E, 0x8706EC26, 0xE524651F, 0x3861CFA7, 
-    0x3E41A5B6, 0xE3040F0E, 0x81268637, 0x5C632C8F, 
-    0x45639445, 0x98263EFD, 0xFA04B7C4, 0x27411D7C, 
-    0xC805C650, 0x15406CE8, 0x7762E5D1, 0xAA274F69, 
-    0xB327F7A3, 0x6E625D1B, 0x0C40D422, 0xD1057E9A, 
-    0xABA65FE7, 0x76E3F55F, 0x14C17C66, 0xC984D6DE, 
-    0xD0846E14, 0x0DC1C4AC, 0x6FE34D95, 0xB2A6E72D, 
-    0x5DE23C01, 0x80A796B9, 0xE2851F80, 0x3FC0B538, 
-    0x26C00DF2, 0xFB85A74A, 0x99A72E73, 0x44E284CB, 
-    0x42C2EEDA, 0x9F874462, 0xFDA5CD5B, 0x20E067E3, 
-    0x39E0DF29, 0xE4A57591, 0x8687FCA8, 0x5BC25610, 
-    0xB4868D3C, 0x69C32784, 0x0BE1AEBD, 0xD6A40405, 
-    0xCFA4BCCF, 0x12E11677, 0x70C39F4E, 0xAD8635F6, 
-    0x7C834B6C, 0xA1C6E1D4, 0xC3E468ED, 0x1EA1C255, 
-    0x07A17A9F, 0xDAE4D027, 0xB8C6591E, 0x6583F3A6, 
-    0x8AC7288A, 0x57828232, 0x35A00B0B, 0xE8E5A1B3, 
-    0xF1E51979, 0x2CA0B3C1, 0x4E823AF8, 0x93C79040, 
-    0x95E7FA51, 0x48A250E9, 0x2A80D9D0, 0xF7C57368, 
-    0xEEC5CBA2, 0x3380611A, 0x51A2E823, 0x8CE7429B, 
-    0x63A399B7, 0xBEE6330F, 0xDCC4BA36, 0x0181108E, 
-    0x1881A844, 0xC5C402FC, 0xA7E68BC5, 0x7AA3217D, 
-    0x52A0C93F, 0x8FE56387, 0xEDC7EABE, 0x30824006, 
-    0x2982F8CC, 0xF4C75274, 0x96E5DB4D, 0x4BA071F5, 
-    0xA4E4AAD9, 0x79A10061, 0x1B838958, 0xC6C623E0, 
-    0xDFC69B2A, 0x02833192, 0x60A1B8AB, 0xBDE41213, 
-    0xBBC47802, 0x6681D2BA, 0x04A35B83, 0xD9E6F13B, 
-    0xC0E649F1, 0x1DA3E349, 0x7F816A70, 0xA2C4C0C8, 
-    0x4D801BE4, 0x90C5B15C, 0xF2E73865, 0x2FA292DD, 
-    0x36A22A17, 0xEBE780AF, 0x89C50996, 0x5480A32E, 
-    0x8585DDB4, 0x58C0770C, 0x3AE2FE35, 0xE7A7548D, 
-    0xFEA7EC47, 0x23E246FF, 0x41C0CFC6, 0x9C85657E, 
-    0x73C1BE52, 0xAE8414EA, 0xCCA69DD3, 0x11E3376B, 
-    0x08E38FA1, 0xD5A62519, 0xB784AC20, 0x6AC10698, 
-    0x6CE16C89, 0xB1A4C631, 0xD3864F08, 0x0EC3E5B0, 
-    0x17C35D7A, 0xCA86F7C2, 0xA8A47EFB, 0x75E1D443, 
-    0x9AA50F6F, 0x47E0A5D7, 0x25C22CEE, 0xF8878656, 
-    0xE1873E9C, 0x3CC29424, 0x5EE01D1D, 0x83A5B7A5, 
-    0xF90696D8, 0x24433C60, 0x4661B559, 0x9B241FE1, 
-    0x8224A72B, 0x5F610D93, 0x3D4384AA, 0xE0062E12, 
-    0x0F42F53E, 0xD2075F86, 0xB025D6BF, 0x6D607C07, 
-    0x7460C4CD, 0xA9256E75, 0xCB07E74C, 0x16424DF4, 
-    0x106227E5, 0xCD278D5D, 0xAF050464, 0x7240AEDC, 
-    0x6B401616, 0xB605BCAE, 0xD4273597, 0x09629F2F, 
-    0xE6264403, 0x3B63EEBB, 0x59416782, 0x8404CD3A, 
-    0x9D0475F0, 0x4041DF48, 0x22635671, 0xFF26FCC9, 
-    0x2E238253, 0xF36628EB, 0x9144A1D2, 0x4C010B6A, 
-    0x5501B3A0, 0x88441918, 0xEA669021, 0x37233A99, 
-    0xD867E1B5, 0x05224B0D, 0x6700C234, 0xBA45688C, 
-    0xA345D046, 0x7E007AFE, 0x1C22F3C7, 0xC167597F, 
-    0xC747336E, 0x1A0299D6, 0x782010EF, 0xA565BA57, 
-    0xBC65029D, 0x6120A825, 0x0302211C, 0xDE478BA4, 
-    0x31035088, 0xEC46FA30, 0x8E647309, 0x5321D9B1, 
-    0x4A21617B, 0x9764CBC3, 0xF54642FA, 0x2803E842, 
-    /* T8_4 */
-    0x00000000, 0x38116FAC, 0x7022DF58, 0x4833B0F4, 
-    0xE045BEB0, 0xD854D11C, 0x906761E8, 0xA8760E44, 
-    0xC5670B91, 0xFD76643D, 0xB545D4C9, 0x8D54BB65, 
-    0x2522B521, 0x1D33DA8D, 0x55006A79, 0x6D1105D5, 
-    0x8F2261D3, 0xB7330E7F, 0xFF00BE8B, 0xC711D127, 
-    0x6F67DF63, 0x5776B0CF, 0x1F45003B, 0x27546F97, 
-    0x4A456A42, 0x725405EE, 0x3A67B51A, 0x0276DAB6, 
-    0xAA00D4F2, 0x9211BB5E, 0xDA220BAA, 0xE2336406, 
-    0x1BA8B557, 0x23B9DAFB, 0x6B8A6A0F, 0x539B05A3, 
-    0xFBED0BE7, 0xC3FC644B, 0x8BCFD4BF, 0xB3DEBB13, 
-    0xDECFBEC6, 0xE6DED16A, 0xAEED619E, 0x96FC0E32, 
-    0x3E8A0076, 0x069B6FDA, 0x4EA8DF2E, 0x76B9B082, 
-    0x948AD484, 0xAC9BBB28, 0xE4A80BDC, 0xDCB96470, 
-    0x74CF6A34, 0x4CDE0598, 0x04EDB56C, 0x3CFCDAC0, 
-    0x51EDDF15, 0x69FCB0B9, 0x21CF004D, 0x19DE6FE1, 
-    0xB1A861A5, 0x89B90E09, 0xC18ABEFD, 0xF99BD151, 
-    0x37516AAE, 0x0F400502, 0x4773B5F6, 0x7F62DA5A, 
-    0xD714D41E, 0xEF05BBB2, 0xA7360B46, 0x9F2764EA, 
-    0xF236613F, 0xCA270E93, 0x8214BE67, 0xBA05D1CB, 
-    0x1273DF8F, 0x2A62B023, 0x625100D7, 0x5A406F7B, 
-    0xB8730B7D, 0x806264D1, 0xC851D425, 0xF040BB89, 
-    0x5836B5CD, 0x6027DA61, 0x28146A95, 0x10050539, 
-    0x7D1400EC, 0x45056F40, 0x0D36DFB4, 0x3527B018, 
-    0x9D51BE5C, 0xA540D1F0, 0xED736104, 0xD5620EA8, 
-    0x2CF9DFF9, 0x14E8B055, 0x5CDB00A1, 0x64CA6F0D, 
-    0xCCBC6149, 0xF4AD0EE5, 0xBC9EBE11, 0x848FD1BD, 
-    0xE99ED468, 0xD18FBBC4, 0x99BC0B30, 0xA1AD649C, 
-    0x09DB6AD8, 0x31CA0574, 0x79F9B580, 0x41E8DA2C, 
-    0xA3DBBE2A, 0x9BCAD186, 0xD3F96172, 0xEBE80EDE, 
-    0x439E009A, 0x7B8F6F36, 0x33BCDFC2, 0x0BADB06E, 
-    0x66BCB5BB, 0x5EADDA17, 0x169E6AE3, 0x2E8F054F, 
-    0x86F90B0B, 0xBEE864A7, 0xF6DBD453, 0xCECABBFF, 
-    0x6EA2D55C, 0x56B3BAF0, 0x1E800A04, 0x269165A8, 
-    0x8EE76BEC, 0xB6F60440, 0xFEC5B4B4, 0xC6D4DB18, 
-    0xABC5DECD, 0x93D4B161, 0xDBE70195, 0xE3F66E39, 
-    0x4B80607D, 0x73910FD1, 0x3BA2BF25, 0x03B3D089, 
-    0xE180B48F, 0xD991DB23, 0x91A26BD7, 0xA9B3047B, 
-    0x01C50A3F, 0x39D46593, 0x71E7D567, 0x49F6BACB, 
-    0x24E7BF1E, 0x1CF6D0B2, 0x54C56046, 0x6CD40FEA, 
-    0xC4A201AE, 0xFCB36E02, 0xB480DEF6, 0x8C91B15A, 
-    0x750A600B, 0x4D1B0FA7, 0x0528BF53, 0x3D39D0FF, 
-    0x954FDEBB, 0xAD5EB117, 0xE56D01E3, 0xDD7C6E4F, 
-    0xB06D6B9A, 0x887C0436, 0xC04FB4C2, 0xF85EDB6E, 
-    0x5028D52A, 0x6839BA86, 0x200A0A72, 0x181B65DE, 
-    0xFA2801D8, 0xC2396E74, 0x8A0ADE80, 0xB21BB12C, 
-    0x1A6DBF68, 0x227CD0C4, 0x6A4F6030, 0x525E0F9C, 
-    0x3F4F0A49, 0x075E65E5, 0x4F6DD511, 0x777CBABD, 
-    0xDF0AB4F9, 0xE71BDB55, 0xAF286BA1, 0x9739040D, 
-    0x59F3BFF2, 0x61E2D05E, 0x29D160AA, 0x11C00F06, 
-    0xB9B60142, 0x81A76EEE, 0xC994DE1A, 0xF185B1B6, 
-    0x9C94B463, 0xA485DBCF, 0xECB66B3B, 0xD4A70497, 
-    0x7CD10AD3, 0x44C0657F, 0x0CF3D58B, 0x34E2BA27, 
-    0xD6D1DE21, 0xEEC0B18D, 0xA6F30179, 0x9EE26ED5, 
-    0x36946091, 0x0E850F3D, 0x46B6BFC9, 0x7EA7D065, 
-    0x13B6D5B0, 0x2BA7BA1C, 0x63940AE8, 0x5B856544, 
-    0xF3F36B00, 0xCBE204AC, 0x83D1B458, 0xBBC0DBF4, 
-    0x425B0AA5, 0x7A4A6509, 0x3279D5FD, 0x0A68BA51, 
-    0xA21EB415, 0x9A0FDBB9, 0xD23C6B4D, 0xEA2D04E1, 
-    0x873C0134, 0xBF2D6E98, 0xF71EDE6C, 0xCF0FB1C0, 
-    0x6779BF84, 0x5F68D028, 0x175B60DC, 0x2F4A0F70, 
-    0xCD796B76, 0xF56804DA, 0xBD5BB42E, 0x854ADB82, 
-    0x2D3CD5C6, 0x152DBA6A, 0x5D1E0A9E, 0x650F6532, 
-    0x081E60E7, 0x300F0F4B, 0x783CBFBF, 0x402DD013, 
-    0xE85BDE57, 0xD04AB1FB, 0x9879010F, 0xA0686EA3, 
-    /* T8_5 */
-    0x00000000, 0xEF306B19, 0xDB8CA0C3, 0x34BCCBDA, 
-    0xB2F53777, 0x5DC55C6E, 0x697997B4, 0x8649FCAD, 
-    0x6006181F, 0x8F367306, 0xBB8AB8DC, 0x54BAD3C5, 
-    0xD2F32F68, 0x3DC34471, 0x097F8FAB, 0xE64FE4B2, 
-    0xC00C303E, 0x2F3C5B27, 0x1B8090FD, 0xF4B0FBE4, 
-    0x72F90749, 0x9DC96C50, 0xA975A78A, 0x4645CC93, 
-    0xA00A2821, 0x4F3A4338, 0x7B8688E2, 0x94B6E3FB, 
-    0x12FF1F56, 0xFDCF744F, 0xC973BF95, 0x2643D48C, 
-    0x85F4168D, 0x6AC47D94, 0x5E78B64E, 0xB148DD57, 
-    0x370121FA, 0xD8314AE3, 0xEC8D8139, 0x03BDEA20, 
-    0xE5F20E92, 0x0AC2658B, 0x3E7EAE51, 0xD14EC548, 
-    0x570739E5, 0xB83752FC, 0x8C8B9926, 0x63BBF23F, 
-    0x45F826B3, 0xAAC84DAA, 0x9E748670, 0x7144ED69, 
-    0xF70D11C4, 0x183D7ADD, 0x2C81B107, 0xC3B1DA1E, 
-    0x25FE3EAC, 0xCACE55B5, 0xFE729E6F, 0x1142F576, 
-    0x970B09DB, 0x783B62C2, 0x4C87A918, 0xA3B7C201, 
-    0x0E045BEB, 0xE13430F2, 0xD588FB28, 0x3AB89031, 
-    0xBCF16C9C, 0x53C10785, 0x677DCC5F, 0x884DA746, 
-    0x6E0243F4, 0x813228ED, 0xB58EE337, 0x5ABE882E, 
-    0xDCF77483, 0x33C71F9A, 0x077BD440, 0xE84BBF59, 
-    0xCE086BD5, 0x213800CC, 0x1584CB16, 0xFAB4A00F, 
-    0x7CFD5CA2, 0x93CD37BB, 0xA771FC61, 0x48419778, 
-    0xAE0E73CA, 0x413E18D3, 0x7582D309, 0x9AB2B810, 
-    0x1CFB44BD, 0xF3CB2FA4, 0xC777E47E, 0x28478F67, 
-    0x8BF04D66, 0x64C0267F, 0x507CEDA5, 0xBF4C86BC, 
-    0x39057A11, 0xD6351108, 0xE289DAD2, 0x0DB9B1CB, 
-    0xEBF65579, 0x04C63E60, 0x307AF5BA, 0xDF4A9EA3, 
-    0x5903620E, 0xB6330917, 0x828FC2CD, 0x6DBFA9D4, 
-    0x4BFC7D58, 0xA4CC1641, 0x9070DD9B, 0x7F40B682, 
-    0xF9094A2F, 0x16392136, 0x2285EAEC, 0xCDB581F5, 
-    0x2BFA6547, 0xC4CA0E5E, 0xF076C584, 0x1F46AE9D, 
-    0x990F5230, 0x763F3929, 0x4283F2F3, 0xADB399EA, 
-    0x1C08B7D6, 0xF338DCCF, 0xC7841715, 0x28B47C0C, 
-    0xAEFD80A1, 0x41CDEBB8, 0x75712062, 0x9A414B7B, 
-    0x7C0EAFC9, 0x933EC4D0, 0xA7820F0A, 0x48B26413, 
-    0xCEFB98BE, 0x21CBF3A7, 0x1577387D, 0xFA475364, 
-    0xDC0487E8, 0x3334ECF1, 0x0788272B, 0xE8B84C32, 
-    0x6EF1B09F, 0x81C1DB86, 0xB57D105C, 0x5A4D7B45, 
-    0xBC029FF7, 0x5332F4EE, 0x678E3F34, 0x88BE542D, 
-    0x0EF7A880, 0xE1C7C399, 0xD57B0843, 0x3A4B635A, 
-    0x99FCA15B, 0x76CCCA42, 0x42700198, 0xAD406A81, 
-    0x2B09962C, 0xC439FD35, 0xF08536EF, 0x1FB55DF6, 
-    0xF9FAB944, 0x16CAD25D, 0x22761987, 0xCD46729E, 
-    0x4B0F8E33, 0xA43FE52A, 0x90832EF0, 0x7FB345E9, 
-    0x59F09165, 0xB6C0FA7C, 0x827C31A6, 0x6D4C5ABF, 
-    0xEB05A612, 0x0435CD0B, 0x308906D1, 0xDFB96DC8, 
-    0x39F6897A, 0xD6C6E263, 0xE27A29B9, 0x0D4A42A0, 
-    0x8B03BE0D, 0x6433D514, 0x508F1ECE, 0xBFBF75D7, 
-    0x120CEC3D, 0xFD3C8724, 0xC9804CFE, 0x26B027E7, 
-    0xA0F9DB4A, 0x4FC9B053, 0x7B757B89, 0x94451090, 
-    0x720AF422, 0x9D3A9F3B, 0xA98654E1, 0x46B63FF8, 
-    0xC0FFC355, 0x2FCFA84C, 0x1B736396, 0xF443088F, 
-    0xD200DC03, 0x3D30B71A, 0x098C7CC0, 0xE6BC17D9, 
-    0x60F5EB74, 0x8FC5806D, 0xBB794BB7, 0x544920AE, 
-    0xB206C41C, 0x5D36AF05, 0x698A64DF, 0x86BA0FC6, 
-    0x00F3F36B, 0xEFC39872, 0xDB7F53A8, 0x344F38B1, 
-    0x97F8FAB0, 0x78C891A9, 0x4C745A73, 0xA344316A, 
-    0x250DCDC7, 0xCA3DA6DE, 0xFE816D04, 0x11B1061D, 
-    0xF7FEE2AF, 0x18CE89B6, 0x2C72426C, 0xC3422975, 
-    0x450BD5D8, 0xAA3BBEC1, 0x9E87751B, 0x71B71E02, 
-    0x57F4CA8E, 0xB8C4A197, 0x8C786A4D, 0x63480154, 
-    0xE501FDF9, 0x0A3196E0, 0x3E8D5D3A, 0xD1BD3623, 
-    0x37F2D291, 0xD8C2B988, 0xEC7E7252, 0x034E194B, 
-    0x8507E5E6, 0x6A378EFF, 0x5E8B4525, 0xB1BB2E3C, 
-    /* T8_6 */
-    0x00000000, 0x68032CC8, 0xD0065990, 0xB8057558, 
-    0xA5E0C5D1, 0xCDE3E919, 0x75E69C41, 0x1DE5B089, 
-    0x4E2DFD53, 0x262ED19B, 0x9E2BA4C3, 0xF628880B, 
-    0xEBCD3882, 0x83CE144A, 0x3BCB6112, 0x53C84DDA, 
-    0x9C5BFAA6, 0xF458D66E, 0x4C5DA336, 0x245E8FFE, 
-    0x39BB3F77, 0x51B813BF, 0xE9BD66E7, 0x81BE4A2F, 
-    0xD27607F5, 0xBA752B3D, 0x02705E65, 0x6A7372AD, 
-    0x7796C224, 0x1F95EEEC, 0xA7909BB4, 0xCF93B77C, 
-    0x3D5B83BD, 0x5558AF75, 0xED5DDA2D, 0x855EF6E5, 
-    0x98BB466C, 0xF0B86AA4, 0x48BD1FFC, 0x20BE3334, 
-    0x73767EEE, 0x1B755226, 0xA370277E, 0xCB730BB6, 
-    0xD696BB3F, 0xBE9597F7, 0x0690E2AF, 0x6E93CE67, 
-    0xA100791B, 0xC90355D3, 0x7106208B, 0x19050C43, 
-    0x04E0BCCA, 0x6CE39002, 0xD4E6E55A, 0xBCE5C992, 
-    0xEF2D8448, 0x872EA880, 0x3F2BDDD8, 0x5728F110, 
-    0x4ACD4199, 0x22CE6D51, 0x9ACB1809, 0xF2C834C1, 
-    0x7AB7077A, 0x12B42BB2, 0xAAB15EEA, 0xC2B27222, 
-    0xDF57C2AB, 0xB754EE63, 0x0F519B3B, 0x6752B7F3, 
-    0x349AFA29, 0x5C99D6E1, 0xE49CA3B9, 0x8C9F8F71, 
-    0x917A3FF8, 0xF9791330, 0x417C6668, 0x297F4AA0, 
-    0xE6ECFDDC, 0x8EEFD114, 0x36EAA44C, 0x5EE98884, 
-    0x430C380D, 0x2B0F14C5, 0x930A619D, 0xFB094D55, 
-    0xA8C1008F, 0xC0C22C47, 0x78C7591F, 0x10C475D7, 
-    0x0D21C55E, 0x6522E996, 0xDD279CCE, 0xB524B006, 
-    0x47EC84C7, 0x2FEFA80F, 0x97EADD57, 0xFFE9F19F, 
-    0xE20C4116, 0x8A0F6DDE, 0x320A1886, 0x5A09344E, 
-    0x09C17994, 0x61C2555C, 0xD9C72004, 0xB1C40CCC, 
-    0xAC21BC45, 0xC422908D, 0x7C27E5D5, 0x1424C91D, 
-    0xDBB77E61, 0xB3B452A9, 0x0BB127F1, 0x63B20B39, 
-    0x7E57BBB0, 0x16549778, 0xAE51E220, 0xC652CEE8, 
-    0x959A8332, 0xFD99AFFA, 0x459CDAA2, 0x2D9FF66A, 
-    0x307A46E3, 0x58796A2B, 0xE07C1F73, 0x887F33BB, 
-    0xF56E0EF4, 0x9D6D223C, 0x25685764, 0x4D6B7BAC, 
-    0x508ECB25, 0x388DE7ED, 0x808892B5, 0xE88BBE7D, 
-    0xBB43F3A7, 0xD340DF6F, 0x6B45AA37, 0x034686FF, 
-    0x1EA33676, 0x76A01ABE, 0xCEA56FE6, 0xA6A6432E, 
-    0x6935F452, 0x0136D89A, 0xB933ADC2, 0xD130810A, 
-    0xCCD53183, 0xA4D61D4B, 0x1CD36813, 0x74D044DB, 
-    0x27180901, 0x4F1B25C9, 0xF71E5091, 0x9F1D7C59, 
-    0x82F8CCD0, 0xEAFBE018, 0x52FE9540, 0x3AFDB988, 
-    0xC8358D49, 0xA036A181, 0x1833D4D9, 0x7030F811, 
-    0x6DD54898, 0x05D66450, 0xBDD31108, 0xD5D03DC0, 
-    0x8618701A, 0xEE1B5CD2, 0x561E298A, 0x3E1D0542, 
-    0x23F8B5CB, 0x4BFB9903, 0xF3FEEC5B, 0x9BFDC093, 
-    0x546E77EF, 0x3C6D5B27, 0x84682E7F, 0xEC6B02B7, 
-    0xF18EB23E, 0x998D9EF6, 0x2188EBAE, 0x498BC766, 
-    0x1A438ABC, 0x7240A674, 0xCA45D32C, 0xA246FFE4, 
-    0xBFA34F6D, 0xD7A063A5, 0x6FA516FD, 0x07A63A35, 
-    0x8FD9098E, 0xE7DA2546, 0x5FDF501E, 0x37DC7CD6, 
-    0x2A39CC5F, 0x423AE097, 0xFA3F95CF, 0x923CB907, 
-    0xC1F4F4DD, 0xA9F7D815, 0x11F2AD4D, 0x79F18185, 
-    0x6414310C, 0x0C171DC4, 0xB412689C, 0xDC114454, 
-    0x1382F328, 0x7B81DFE0, 0xC384AAB8, 0xAB878670, 
-    0xB66236F9, 0xDE611A31, 0x66646F69, 0x0E6743A1, 
-    0x5DAF0E7B, 0x35AC22B3, 0x8DA957EB, 0xE5AA7B23, 
-    0xF84FCBAA, 0x904CE762, 0x2849923A, 0x404ABEF2, 
-    0xB2828A33, 0xDA81A6FB, 0x6284D3A3, 0x0A87FF6B, 
-    0x17624FE2, 0x7F61632A, 0xC7641672, 0xAF673ABA, 
-    0xFCAF7760, 0x94AC5BA8, 0x2CA92EF0, 0x44AA0238, 
-    0x594FB2B1, 0x314C9E79, 0x8949EB21, 0xE14AC7E9, 
-    0x2ED97095, 0x46DA5C5D, 0xFEDF2905, 0x96DC05CD, 
-    0x8B39B544, 0xE33A998C, 0x5B3FECD4, 0x333CC01C, 
-    0x60F48DC6, 0x08F7A10E, 0xB0F2D456, 0xD8F1F89E, 
-    0xC5144817, 0xAD1764DF, 0x15121187, 0x7D113D4F, 
-    /* T8_7 */
-    0x00000000, 0x493C7D27, 0x9278FA4E, 0xDB448769, 
-    0x211D826D, 0x6821FF4A, 0xB3657823, 0xFA590504, 
-    0x423B04DA, 0x0B0779FD, 0xD043FE94, 0x997F83B3, 
-    0x632686B7, 0x2A1AFB90, 0xF15E7CF9, 0xB86201DE, 
-    0x847609B4, 0xCD4A7493, 0x160EF3FA, 0x5F328EDD, 
-    0xA56B8BD9, 0xEC57F6FE, 0x37137197, 0x7E2F0CB0, 
-    0xC64D0D6E, 0x8F717049, 0x5435F720, 0x1D098A07, 
-    0xE7508F03, 0xAE6CF224, 0x7528754D, 0x3C14086A, 
-    0x0D006599, 0x443C18BE, 0x9F789FD7, 0xD644E2F0, 
-    0x2C1DE7F4, 0x65219AD3, 0xBE651DBA, 0xF759609D, 
-    0x4F3B6143, 0x06071C64, 0xDD439B0D, 0x947FE62A, 
-    0x6E26E32E, 0x271A9E09, 0xFC5E1960, 0xB5626447, 
-    0x89766C2D, 0xC04A110A, 0x1B0E9663, 0x5232EB44, 
-    0xA86BEE40, 0xE1579367, 0x3A13140E, 0x732F6929, 
-    0xCB4D68F7, 0x827115D0, 0x593592B9, 0x1009EF9E, 
-    0xEA50EA9A, 0xA36C97BD, 0x782810D4, 0x31146DF3, 
-    0x1A00CB32, 0x533CB615, 0x8878317C, 0xC1444C5B, 
-    0x3B1D495F, 0x72213478, 0xA965B311, 0xE059CE36, 
-    0x583BCFE8, 0x1107B2CF, 0xCA4335A6, 0x837F4881, 
-    0x79264D85, 0x301A30A2, 0xEB5EB7CB, 0xA262CAEC, 
-    0x9E76C286, 0xD74ABFA1, 0x0C0E38C8, 0x453245EF, 
-    0xBF6B40EB, 0xF6573DCC, 0x2D13BAA5, 0x642FC782, 
-    0xDC4DC65C, 0x9571BB7B, 0x4E353C12, 0x07094135, 
-    0xFD504431, 0xB46C3916, 0x6F28BE7F, 0x2614C358, 
-    0x1700AEAB, 0x5E3CD38C, 0x857854E5, 0xCC4429C2, 
-    0x361D2CC6, 0x7F2151E1, 0xA465D688, 0xED59ABAF, 
-    0x553BAA71, 0x1C07D756, 0xC743503F, 0x8E7F2D18, 
-    0x7426281C, 0x3D1A553B, 0xE65ED252, 0xAF62AF75, 
-    0x9376A71F, 0xDA4ADA38, 0x010E5D51, 0x48322076, 
-    0xB26B2572, 0xFB575855, 0x2013DF3C, 0x692FA21B, 
-    0xD14DA3C5, 0x9871DEE2, 0x4335598B, 0x0A0924AC, 
-    0xF05021A8, 0xB96C5C8F, 0x6228DBE6, 0x2B14A6C1, 
-    0x34019664, 0x7D3DEB43, 0xA6796C2A, 0xEF45110D, 
-    0x151C1409, 0x5C20692E, 0x8764EE47, 0xCE589360, 
-    0x763A92BE, 0x3F06EF99, 0xE44268F0, 0xAD7E15D7, 
-    0x572710D3, 0x1E1B6DF4, 0xC55FEA9D, 0x8C6397BA, 
-    0xB0779FD0, 0xF94BE2F7, 0x220F659E, 0x6B3318B9, 
-    0x916A1DBD, 0xD856609A, 0x0312E7F3, 0x4A2E9AD4, 
-    0xF24C9B0A, 0xBB70E62D, 0x60346144, 0x29081C63, 
-    0xD3511967, 0x9A6D6440, 0x4129E329, 0x08159E0E, 
-    0x3901F3FD, 0x703D8EDA, 0xAB7909B3, 0xE2457494, 
-    0x181C7190, 0x51200CB7, 0x8A648BDE, 0xC358F6F9, 
-    0x7B3AF727, 0x32068A00, 0xE9420D69, 0xA07E704E, 
-    0x5A27754A, 0x131B086D, 0xC85F8F04, 0x8163F223, 
-    0xBD77FA49, 0xF44B876E, 0x2F0F0007, 0x66337D20, 
-    0x9C6A7824, 0xD5560503, 0x0E12826A, 0x472EFF4D, 
-    0xFF4CFE93, 0xB67083B4, 0x6D3404DD, 0x240879FA, 
-    0xDE517CFE, 0x976D01D9, 0x4C2986B0, 0x0515FB97, 
-    0x2E015D56, 0x673D2071, 0xBC79A718, 0xF545DA3F, 
-    0x0F1CDF3B, 0x4620A21C, 0x9D642575, 0xD4585852, 
-    0x6C3A598C, 0x250624AB, 0xFE42A3C2, 0xB77EDEE5, 
-    0x4D27DBE1, 0x041BA6C6, 0xDF5F21AF, 0x96635C88, 
-    0xAA7754E2, 0xE34B29C5, 0x380FAEAC, 0x7133D38B, 
-    0x8B6AD68F, 0xC256ABA8, 0x19122CC1, 0x502E51E6, 
-    0xE84C5038, 0xA1702D1F, 0x7A34AA76, 0x3308D751, 
-    0xC951D255, 0x806DAF72, 0x5B29281B, 0x1215553C, 
-    0x230138CF, 0x6A3D45E8, 0xB179C281, 0xF845BFA6, 
-    0x021CBAA2, 0x4B20C785, 0x906440EC, 0xD9583DCB, 
-    0x613A3C15, 0x28064132, 0xF342C65B, 0xBA7EBB7C, 
-    0x4027BE78, 0x091BC35F, 0xD25F4436, 0x9B633911, 
-    0xA777317B, 0xEE4B4C5C, 0x350FCB35, 0x7C33B612, 
-    0x866AB316, 0xCF56CE31, 0x14124958, 0x5D2E347F, 
-    0xE54C35A1, 0xAC704886, 0x7734CFEF, 0x3E08B2C8, 
-    0xC451B7CC, 0x8D6DCAEB, 0x56294D82, 0x1F1530A5 
-  };
-}

http://git-wip-us.apache.org/repos/asf/incubator-ratis/blob/7e71a2e0/raft-common/src/main/java/org/apache/raft/util/RaftUtils.java
----------------------------------------------------------------------
diff --git a/raft-common/src/main/java/org/apache/raft/util/RaftUtils.java b/raft-common/src/main/java/org/apache/raft/util/RaftUtils.java
deleted file mode 100644
index af76ad9..0000000
--- a/raft-common/src/main/java/org/apache/raft/util/RaftUtils.java
+++ /dev/null
@@ -1,240 +0,0 @@
-/**
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- *     http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-package org.apache.raft.util;
-
-import com.google.common.base.Preconditions;
-import org.apache.log4j.Level;
-import org.apache.log4j.LogManager;
-import org.slf4j.Logger;
-import org.slf4j.LoggerFactory;
-
-import java.io.*;
-import java.lang.reflect.Constructor;
-import java.nio.ByteBuffer;
-import java.nio.channels.FileChannel;
-import java.util.Map;
-import java.util.concurrent.ConcurrentHashMap;
-import java.util.concurrent.ExecutionException;
-import java.util.concurrent.ThreadLocalRandom;
-
-public abstract class RaftUtils {
-  public static final Logger LOG = LoggerFactory.getLogger(RaftUtils.class);
-  private static final Class<?>[] EMPTY_CLASS_ARRAY = {};
-
-  // OSType detection
-  public enum OSType {
-    OS_TYPE_LINUX,
-    OS_TYPE_WIN,
-    OS_TYPE_SOLARIS,
-    OS_TYPE_MAC,
-    OS_TYPE_FREEBSD,
-    OS_TYPE_OTHER
-  }
-
-  /**
-   * Get the type of the operating system, as determined from parsing
-   * the <code>os.name</code> property.
-   */
-  private static final OSType osType = getOSType();
-
-  private static OSType getOSType() {
-    String osName = System.getProperty("os.name");
-    if (osName.startsWith("Windows")) {
-      return OSType.OS_TYPE_WIN;
-    } else if (osName.contains("SunOS") || osName.contains("Solaris")) {
-      return OSType.OS_TYPE_SOLARIS;
-    } else if (osName.contains("Mac")) {
-      return OSType.OS_TYPE_MAC;
-    } else if (osName.contains("FreeBSD")) {
-      return OSType.OS_TYPE_FREEBSD;
-    } else if (osName.startsWith("Linux")) {
-      return OSType.OS_TYPE_LINUX;
-    } else {
-      // Some other form of Unix
-      return OSType.OS_TYPE_OTHER;
-    }
-  }
-
-  // Helper static vars for each platform
-  public static final boolean WINDOWS = (osType == OSType.OS_TYPE_WIN);
-  public static final boolean SOLARIS = (osType == OSType.OS_TYPE_SOLARIS);
-  public static final boolean MAC     = (osType == OSType.OS_TYPE_MAC);
-  public static final boolean FREEBSD = (osType == OSType.OS_TYPE_FREEBSD);
-  public static final boolean LINUX   = (osType == OSType.OS_TYPE_LINUX);
-  public static final boolean OTHER   = (osType == OSType.OS_TYPE_OTHER);
-
-  public static final boolean PPC_64
-      = System.getProperties().getProperty("os.arch").contains("ppc64");
-
-  /**
-   * Cache of constructors for each class. Pins the classes so they
-   * can't be garbage collected until ReflectionUtils can be collected.
-   */
-  private static final Map<Class<?>, Constructor<?>> CONSTRUCTOR_CACHE =
-      new ConcurrentHashMap<>();
-
-  public static InterruptedIOException toInterruptedIOException(
-      String message, InterruptedException e) {
-    final InterruptedIOException iioe = new InterruptedIOException(message);
-    iioe.initCause(e);
-    return iioe;
-  }
-
-  public static IOException asIOException(Throwable t) {
-    return t instanceof IOException? (IOException)t : new IOException(t);
-  }
-
-  public static IOException toIOException(ExecutionException e) {
-    final Throwable cause = e.getCause();
-    return cause != null? asIOException(cause): new IOException(e);
-  }
-
-  /** Is the given object an instance of one of the given classes? */
-  public static boolean isInstance(Object obj, Class<?>... classes) {
-    for(Class<?> c : classes) {
-      if (c.isInstance(obj)) {
-        return true;
-      }
-    }
-    return false;
-  }
-
-  /**
-   * Create an object for the given class and initialize it from conf
-   *
-   * @param theClass class of which an object is created
-   * @return a new object
-   */
-  @SuppressWarnings("unchecked")
-  public static <T> T newInstance(Class<T> theClass, Object... initArgs) {
-    T result;
-    try {
-      Constructor<T> meth = (Constructor<T>) CONSTRUCTOR_CACHE.get(theClass);
-      if (meth == null) {
-        meth = theClass.getDeclaredConstructor(EMPTY_CLASS_ARRAY);
-        meth.setAccessible(true);
-        CONSTRUCTOR_CACHE.put(theClass, meth);
-      }
-      result = meth.newInstance(initArgs);
-    } catch (Exception e) {
-      throw new RuntimeException(e);
-    }
-    return result;
-  }
-
-  public static int getRandomBetween(int min, int max) {
-    Preconditions.checkArgument(max > min);
-    return ThreadLocalRandom.current().nextInt(max -min) + min;
-  }
-
-  public static void setLogLevel(Logger logger, Level level) {
-    LogManager.getLogger(logger.getName()).setLevel(level);
-  }
-
-
-  public static void readFully(InputStream in, int buffSize) throws IOException {
-    final byte buf[] = new byte[buffSize];
-    for(int bytesRead = in.read(buf); bytesRead >= 0; ) {
-      bytesRead = in.read(buf);
-    }
-  }
-
-  /**
-   * Reads len bytes in a loop.
-   *
-   * @param in InputStream to read from
-   * @param buf The buffer to fill
-   * @param off offset from the buffer
-   * @param len the length of bytes to read
-   * @throws IOException if it could not read requested number of bytes
-   * for any reason (including EOF)
-   */
-  public static void readFully(InputStream in, byte[] buf, int off, int len)
-      throws IOException {
-    for(int toRead = len; toRead > 0; ) {
-      final int ret = in.read(buf, off, toRead);
-      if (ret < 0) {
-        throw new IOException( "Premature EOF from inputStream");
-      }
-      toRead -= ret;
-      off += ret;
-    }
-  }
-
-  /**
-   * Write a ByteBuffer to a FileChannel at a given offset,
-   * handling short writes.
-   *
-   * @param fc               The FileChannel to write to
-   * @param buf              The input buffer
-   * @param offset           The offset in the file to start writing at
-   * @throws IOException     On I/O error
-   */
-  public static void writeFully(FileChannel fc, ByteBuffer buf, long offset)
-      throws IOException {
-    do {
-      offset += fc.write(buf, offset);
-    } while (buf.remaining() > 0);
-  }
-
-  /**
-   * Similar to readFully(). Skips bytes in a loop.
-   * @param in The InputStream to skip bytes from
-   * @param len number of bytes to skip.
-   * @throws IOException if it could not skip requested number of bytes
-   * for any reason (including EOF)
-   */
-  public static void skipFully(InputStream in, long len) throws IOException {
-    long amt = len;
-    while (amt > 0) {
-      long ret = in.skip(amt);
-      if (ret == 0) {
-        // skip may return 0 even if we're not at EOF.  Luckily, we can
-        // use the read() method to figure out if we're at the end.
-        int b = in.read();
-        if (b == -1) {
-          throw new EOFException( "Premature EOF from inputStream after " +
-              "skipping " + (len - amt) + " byte(s).");
-        }
-        ret = 1;
-      }
-      amt -= ret;
-    }
-  }
-
-  /**
-   * Close the Closeable objects and <b>ignore</b> any {@link Throwable} or
-   * null pointers. Must only be used for cleanup in exception handlers.
-   *
-   * @param log the log to record problems to at debug level. Can be null.
-   * @param closeables the objects to close
-   */
-  public static void cleanup(Logger log, Closeable... closeables) {
-    for (Closeable c : closeables) {
-      if (c != null) {
-        try {
-          c.close();
-        } catch(Throwable e) {
-          if (log != null && log.isDebugEnabled()) {
-            log.debug("Exception in closing " + c, e);
-          }
-        }
-      }
-    }
-  }
-}

http://git-wip-us.apache.org/repos/asf/incubator-ratis/blob/7e71a2e0/raft-common/src/main/java/org/apache/raft/util/StringUtils.java
----------------------------------------------------------------------
diff --git a/raft-common/src/main/java/org/apache/raft/util/StringUtils.java b/raft-common/src/main/java/org/apache/raft/util/StringUtils.java
deleted file mode 100644
index c644572..0000000
--- a/raft-common/src/main/java/org/apache/raft/util/StringUtils.java
+++ /dev/null
@@ -1,108 +0,0 @@
-/**
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- *     http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-package org.apache.raft.util;
-
-import com.google.common.base.Preconditions;
-import com.google.common.collect.Interner;
-import com.google.common.collect.Interners;
-
-import java.util.Iterator;
-import java.util.Locale;
-
-public class StringUtils {
-  public static final String[] EMPTY_STRING_ARRAY = {};
-
-  /** Retains a weak reference to each string instance it has interned. */
-  private static final Interner<String> WEAK_INTERNER = Interners.newWeakInterner();
-
-  /**
-   * Interns and returns a reference to the representative instance
-   * for any of a collection of string instances that are equal to each other.
-   * Retains weak reference to the instance,
-   * and so does not prevent it from being garbage-collected.
-   *
-   * @param sample string instance to be interned
-   * @return weak reference to interned string instance
-   */
-  public static String weakIntern(String sample) {
-    return sample == null? null: WEAK_INTERNER.intern(sample);
-  }
-
-  /**
-   * Splits the given comma separated {@link String}.
-   * Each split value is trimmed.
-   *
-   * @param s a comma separated {@link String}, or possibly null.
-   * @return the split strings, or an empty array if the given string is null.
-   */
-  public static String[] getTrimmedStrings(String s) {
-    return s == null? EMPTY_STRING_ARRAY
-        : (s = s.trim()).isEmpty()? EMPTY_STRING_ARRAY
-        : s.split("\\s*,\\s*");
-  }
-
-  /** The same as String.format(Locale.ENGLISH, format, objects). */
-  public static String format(final String format, final Object... objects) {
-    return String.format(Locale.ENGLISH, format, objects);
-  }
-
-  public static String bytes2HexString(byte[] bytes) {
-    Preconditions.checkNotNull(bytes);
-
-    final StringBuilder s = new StringBuilder(2 * bytes.length);
-    for(byte b : bytes) {
-      s.append(format("%02x", b));
-    }
-    return s.toString();
-  }
-
-  public static boolean string2boolean(String s, boolean defaultValue) {
-    if (s == null || s.isEmpty()) {
-      return defaultValue;
-    }
-
-    if ("true".equalsIgnoreCase(s)) {
-      return true;
-    } else if ("false".equalsIgnoreCase(s)) {
-      return false;
-    } else {
-      return defaultValue;
-    }
-  }
-
-  /**
-   *  @return the next string in the iteration right after the given string;
-   *          if the given string is not in the iteration, return the first string.
-   */
-  public static String next(final String given, final Iterable<String> iteration) {
-    Preconditions.checkNotNull(given);
-    Preconditions.checkNotNull(iteration);
-    final Iterator<String> i = iteration.iterator();
-    Preconditions.checkArgument(i.hasNext());
-
-    final String first = i.next();
-    for(String current = first; i.hasNext(); ) {
-      final String next = i.next();
-      if (given.equals(current)) {
-        return next;
-      }
-      current = next;
-    }
-    return first;
-  }
-}

http://git-wip-us.apache.org/repos/asf/incubator-ratis/blob/7e71a2e0/raft-common/src/main/java/org/apache/raft/util/Timestamp.java
----------------------------------------------------------------------
diff --git a/raft-common/src/main/java/org/apache/raft/util/Timestamp.java b/raft-common/src/main/java/org/apache/raft/util/Timestamp.java
deleted file mode 100644
index aa1b283..0000000
--- a/raft-common/src/main/java/org/apache/raft/util/Timestamp.java
+++ /dev/null
@@ -1,84 +0,0 @@
-/**
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- *     http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-package org.apache.raft.util;
-
-/**
- * Use {@link System#nanoTime()} as timestamps.
- *
- * This class takes care the possibility of numerical overflow.
- *
- * The objects of this class are immutable.
- */
-public class Timestamp implements Comparable<Timestamp> {
-  private static final long NANOSECONDS_PER_MILLISECOND = 1000000;
-
-  private static final long START_TIME = System.nanoTime();
-
-  /** @return the latest timestamp. */
-  public static Timestamp latest(Timestamp a, Timestamp b) {
-    return a.compareTo(b) > 0? a: b;
-  }
-
-  private final long nanos;
-
-  private Timestamp(long nanos) {
-    this.nanos = nanos;
-  }
-
-  /** Construct a timestamp with the current time. */
-  public Timestamp() {
-    this(System.nanoTime());
-  }
-
-  /**
-   * @param milliseconds the time period to be added.
-   * @return a new {@link Timestamp} whose value is calculated
-   *         by adding the given milliseconds to this timestamp.
-   */
-  public Timestamp addTimeMs(long milliseconds) {
-    return new Timestamp(nanos + milliseconds * NANOSECONDS_PER_MILLISECOND);
-  }
-
-  /**
-   * @return the elapsed time in milliseconds.
-   *         If the timestamp is a future time,
-   *         this method returns a negative value.
-   */
-  public long elapsedTimeMs() {
-    final long d = System.nanoTime() - nanos;
-    return d / NANOSECONDS_PER_MILLISECOND;
-  }
-
-  /**
-   * Compare two timestamps, t0 (this) and t1 (that).
-   * This method uses {@code t0 - t1 < 0}, not {@code t0 < t1},
-   * in order to take care the possibility of numerical overflow.
-   *
-   * @see System#nanoTime()
-   */
-  @Override
-  public int compareTo(Timestamp that) {
-    final long d = this.nanos - that.nanos;
-    return d > 0? 1: d == 0? 0: -1;
-  }
-
-  @Override
-  public String toString() {
-    return (nanos - START_TIME)/NANOSECONDS_PER_MILLISECOND + "ms";
-  }
-}

http://git-wip-us.apache.org/repos/asf/incubator-ratis/blob/7e71a2e0/raft-common/src/main/native/src/exception.c
----------------------------------------------------------------------
diff --git a/raft-common/src/main/native/src/exception.c b/raft-common/src/main/native/src/exception.c
deleted file mode 100644
index fc072e8..0000000
--- a/raft-common/src/main/native/src/exception.c
+++ /dev/null
@@ -1,124 +0,0 @@
-/**
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- *     http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-
-#include "exception.h"
-
-#include <jni.h>
-#include <stdio.h>
-#include <stdlib.h>
-#include <string.h>
-
-jthrowable newExceptionV(JNIEnv* env, const char *name,
-                         const char *fmt, va_list ap)
-{
-  int need;
-  char buf[1], *msg = NULL;
-  va_list ap2;
-  jstring jstr = NULL;
-  jthrowable jthr;
-  jclass clazz;
-  jmethodID excCtor;
-
-  va_copy(ap2, ap);
-  clazz = (*env)->FindClass(env, name);
-  if (!clazz) {
-    jthr = (*env)->ExceptionOccurred(env);
-    (*env)->ExceptionClear(env);
-    goto done;
-  }
-  excCtor = (*env)->GetMethodID(env,
-        clazz, "<init>", "(Ljava/lang/String;)V");
-  if (!excCtor) {
-    jthr = (*env)->ExceptionOccurred(env);
-    (*env)->ExceptionClear(env);
-    goto done;
-  }
-  need = vsnprintf(buf, sizeof(buf), fmt, ap);
-  if (need < 0) {
-    fmt = "vsnprintf error";
-    need = strlen(fmt);
-  }
-  msg = malloc(need + 1);
-  vsnprintf(msg, need + 1, fmt, ap2);
-  jstr = (*env)->NewStringUTF(env, msg);
-  if (!jstr) {
-    jthr = (*env)->ExceptionOccurred(env);
-    (*env)->ExceptionClear(env);
-    goto done;
-  }
-  jthr = (*env)->NewObject(env, clazz, excCtor, jstr);
-  if (!jthr) {
-    jthr = (*env)->ExceptionOccurred(env);
-    (*env)->ExceptionClear(env);
-    goto done;
-  }
-
-done:
-  free(msg);
-  va_end(ap2);
-  (*env)->DeleteLocalRef(env, jstr);
-  return jthr;
-}
-
-jthrowable newException(JNIEnv* env, const char *name, const char *fmt, ...)
-{
-  va_list ap;
-  jthrowable jthr;
-
-  va_start(ap, fmt);
-  jthr = newExceptionV(env, name, fmt, ap);
-  va_end(ap);
-  return jthr;
-}
-
-jthrowable newRuntimeException(JNIEnv* env, const char *fmt, ...)
-{
-  va_list ap;
-  jthrowable jthr;
-
-  va_start(ap, fmt);
-  jthr = newExceptionV(env, "java/lang/RuntimeException", fmt, ap);
-  va_end(ap);
-  return jthr;
-}
-
-jthrowable newIOException(JNIEnv* env, const char *fmt, ...)
-{
-  va_list ap;
-  jthrowable jthr;
-
-  va_start(ap, fmt);
-  jthr = newExceptionV(env, "java/io/IOException", fmt, ap);
-  va_end(ap);
-  return jthr;
-}
-
-const char* terror(int errnum)
-{
-
-#if defined(__sun)
-// MT-Safe under Solaris which doesn't support sys_errlist/sys_nerr
-  return strerror(errnum); 
-#else
-  if ((errnum < 0) || (errnum >= sys_nerr)) {
-    return "unknown error.";
-  }
-  return sys_errlist[errnum];
-#endif
-}
-

http://git-wip-us.apache.org/repos/asf/incubator-ratis/blob/7e71a2e0/raft-common/src/main/native/src/exception.h
----------------------------------------------------------------------
diff --git a/raft-common/src/main/native/src/exception.h b/raft-common/src/main/native/src/exception.h
deleted file mode 100644
index e1dc360..0000000
--- a/raft-common/src/main/native/src/exception.h
+++ /dev/null
@@ -1,104 +0,0 @@
-/*
- *  Licensed to the Apache Software Foundation (ASF) under one or more
- *  contributor license agreements.  See the NOTICE file distributed with
- *  this work for additional information regarding copyright ownership.
- *  The ASF licenses this file to You under the Apache License, Version 2.0
- *  (the "License"); you may not use this file except in compliance with
- *  the License.  You may obtain a copy of the License at
- *
- *     http://www.apache.org/licenses/LICENSE-2.0
- *
- *  Unless required by applicable law or agreed to in writing, software
- *  distributed under the License is distributed on an "AS IS" BASIS,
- *  WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- *  See the License for the specific language governing permissions and
- *  limitations under the License.
- */
-#ifndef RAFT_MAIN_NATIVE_SRC_EXCEPTION_H
-#define RAFT_MAIN_NATIVE_SRC_EXCEPTION_H
-
-#include <jni.h> /* for jthrowable */
-#include <stdarg.h> /* for va_list */
-#include "org_apache_raft.h"
-
-#ifdef WINDOWS
-/*
- * gcc-style type-checked format arguments are not supported on Windows, so just
- * stub this macro.
- */
-#define TYPE_CHECKED_PRINTF_FORMAT(formatArg, varArgs)
-# else
-/* Use gcc type-checked format arguments. */
-#define TYPE_CHECKED_PRINTF_FORMAT(formatArg, varArgs) \
-  __attribute__((format(printf, formatArg, varArgs)))
-#endif
-
-/**
- * Create a new Exception.
- *
- * No exceptions will be pending on return.
- *
- * @param env           The JNI environment
- * @param name          full name of the Java exception class
- * @param fmt           printf-style format string
- * @param ap            printf-style arguments
- *
- * @return              The RuntimeException
- */
-jthrowable newExceptionV(JNIEnv* env, const char *name,
-                         const char *fmt, va_list ap);
-
-/**
- * Create a new Exception.
- *
- * No exceptions will be pending on return.
- *
- * @param env           The JNI environment
- * @param name          full name of the Java exception class
- * @param fmt           printf-style format string
- * @param ...           printf-style arguments
- *
- * @return              The RuntimeException
- */
-jthrowable newException(JNIEnv* env, const char *name, const char *fmt, ...)
-    TYPE_CHECKED_PRINTF_FORMAT(3, 4);
-
-/**
- * Create a new RuntimeException.
- *
- * No exceptions will be pending on return.
- *
- * @param env           The JNI environment
- * @param fmt           printf-style format string
- * @param ...           printf-style arguments
- *
- * @return              The RuntimeException
- */
-jthrowable newRuntimeException(JNIEnv* env, const char *fmt, ...)
-    TYPE_CHECKED_PRINTF_FORMAT(2, 3);
-
-/**
- * Create a new IOException.
- *
- * No exceptions will be pending on return.
- *
- * @param env           The JNI environment
- * @param fmt           printf-style format string
- * @param ...           printf-style arguments
- *
- * @return              The IOException, or another exception if we failed
- *                      to create the NativeIOException.
- */
-jthrowable newIOException(JNIEnv* env, const char *fmt, ...)
-    TYPE_CHECKED_PRINTF_FORMAT(2, 3);
-
-/**
- * Thread-safe strerror alternative.
- *
- * @param errnum        Error number.
- * @return              Statically allocated error string.
- */
-const char* terror(int errnum);
-
-#undef TYPE_CHECKED_PRINTF_FORMAT
-#endif

http://git-wip-us.apache.org/repos/asf/incubator-ratis/blob/7e71a2e0/raft-common/src/main/native/src/org/apache/raft/io/nativeio/NativeIO.c
----------------------------------------------------------------------
diff --git a/raft-common/src/main/native/src/org/apache/raft/io/nativeio/NativeIO.c b/raft-common/src/main/native/src/org/apache/raft/io/nativeio/NativeIO.c
deleted file mode 100644
index 2ebbbdb..0000000
--- a/raft-common/src/main/native/src/org/apache/raft/io/nativeio/NativeIO.c
+++ /dev/null
@@ -1,1061 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- *     http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-
-#include "org_apache_raft.h"
-#include "org_apache_raft_io_nativeio_NativeIO.h"
-#include "org_apache_raft_io_nativeio_NativeIO_POSIX.h"
-#include "exception.h"
-
-#ifdef UNIX
-#include <assert.h>
-#include <errno.h>
-#include <fcntl.h>
-#include <grp.h>
-#include <jni.h>
-#include <pwd.h>
-#include <stdint.h>
-#include <stdio.h>
-#include <stdlib.h>
-#include <string.h>
-#include <sys/mman.h>
-#include <sys/resource.h>
-#include <sys/stat.h>
-#include <sys/syscall.h>
-#if !(defined(__FreeBSD__) || defined(__MACH__))
-#include <sys/sendfile.h>
-#endif
-#include <sys/time.h>
-#include <sys/types.h>
-#include <unistd.h>
-#include "config.h"
-#endif
-
-#ifdef WINDOWS
-#include <assert.h>
-#include <Windows.h>
-#include "winutils.h"
-#endif
-
-#include "file_descriptor.h"
-#include "errno_enum.h"
-
-#define MMAP_PROT_READ org_apache_raft_io_nativeio_NativeIO_POSIX_MMAP_PROT_READ
-#define MMAP_PROT_WRITE org_apache_raft_io_nativeio_NativeIO_POSIX_MMAP_PROT_WRITE
-#define MMAP_PROT_EXEC org_apache_raft_io_nativeio_NativeIO_POSIX_MMAP_PROT_EXEC
-
-#define NATIVE_IO_POSIX_CLASS "org/apache/raft/io/nativeio/NativeIO$POSIX"
-#define NATIVE_IO_STAT_CLASS "org/apache/raft/io/nativeio/NativeIO$POSIX$Stat"
-
-#define SET_INT_OR_RETURN(E, C, F) \
-  { \
-    setStaticInt(E, C, #F, F); \
-    if ((*E)->ExceptionCheck(E)) return; \
-  }
-
-// the NativeIO$POSIX$Stat inner class and its constructor
-static jclass stat_clazz;
-static jmethodID stat_ctor;
-static jmethodID stat_ctor2;
-
-// the NativeIOException class and its constructor
-static jclass nioe_clazz;
-static jmethodID nioe_ctor;
-
-// the monitor used for working around non-threadsafe implementations
-// of getpwuid_r, observed on platforms including RHEL 6.0.
-// Please see raft-7156 for details.
-jobject pw_lock_object;
-
-/*
- * Throw a java.IO.IOException, generating the message from errno.
- * NB. this is also used form windows_secure_container_executor.c
- */
-extern void throw_ioe(JNIEnv* env, int errnum);
-
-// Internal functions
-#ifdef UNIX
-static ssize_t get_pw_buflen();
-#endif
-
-/**
- * Returns non-zero if the user has specified that the system
- * has non-threadsafe implementations of getpwuid_r or getgrgid_r.
- **/
-static int workaround_non_threadsafe_calls(JNIEnv *env, jclass clazz) {
-  jboolean result;
-  jfieldID needs_workaround_field = (*env)->GetStaticFieldID(
-    env, clazz,
-    "workaroundNonThreadSafePasswdCalls",
-    "Z");
-  PASS_EXCEPTIONS_RET(env, 0);
-  assert(needs_workaround_field);
-
-  result = (*env)->GetStaticBooleanField(
-    env, clazz, needs_workaround_field);
-  return result;
-}
-
-/**
- * Sets a static boolean field to the specified value.
- */
-static void setStaticBoolean(JNIEnv *env, jclass clazz, char *field,
-  jboolean val) {
-    jfieldID fid = (*env)->GetStaticFieldID(env, clazz, field, "Z");
-    if (fid != NULL) {
-      (*env)->SetStaticBooleanField(env, clazz, fid, val);
-    }
-}
-
-/**
- * Sets a static int field to the specified value.
- */
-static void setStaticInt(JNIEnv *env, jclass clazz, char *field,
-  jint val) {
-    jfieldID fid = (*env)->GetStaticFieldID(env, clazz, field, "I");
-    if (fid != NULL) {
-      (*env)->SetStaticIntField(env, clazz, fid, val);
-    }
-}
-
-#ifdef UNIX
-/**
- * Initialises a list of java constants that are platform specific.
- * These are only initialized in UNIX.
- * Any exceptions that occur will be dealt at the level above.
-**/
-static void consts_init(JNIEnv *env) {
-  jclass clazz = (*env)->FindClass(env, NATIVE_IO_POSIX_CLASS);
-  if (clazz == NULL) {
-    return; // exception has been raised
-  }
-  SET_INT_OR_RETURN(env, clazz, O_RDONLY);
-  SET_INT_OR_RETURN(env, clazz, O_WRONLY);
-  SET_INT_OR_RETURN(env, clazz, O_RDWR);
-  SET_INT_OR_RETURN(env, clazz, O_CREAT);
-  SET_INT_OR_RETURN(env, clazz, O_EXCL);
-  SET_INT_OR_RETURN(env, clazz, O_NOCTTY);
-  SET_INT_OR_RETURN(env, clazz, O_TRUNC);
-  SET_INT_OR_RETURN(env, clazz, O_APPEND);
-  SET_INT_OR_RETURN(env, clazz, O_NONBLOCK);
-  SET_INT_OR_RETURN(env, clazz, O_SYNC);
-#ifdef HAVE_POSIX_FADVISE
-  setStaticBoolean(env, clazz, "fadvisePossible", JNI_TRUE);
-  SET_INT_OR_RETURN(env, clazz, POSIX_FADV_NORMAL);
-  SET_INT_OR_RETURN(env, clazz, POSIX_FADV_RANDOM);
-  SET_INT_OR_RETURN(env, clazz, POSIX_FADV_SEQUENTIAL);
-  SET_INT_OR_RETURN(env, clazz, POSIX_FADV_WILLNEED);
-  SET_INT_OR_RETURN(env, clazz, POSIX_FADV_DONTNEED);
-  SET_INT_OR_RETURN(env, clazz, POSIX_FADV_NOREUSE);
-#else
-  setStaticBoolean(env, clazz, "fadvisePossible", JNI_FALSE);
-#endif
-#ifdef HAVE_SYNC_FILE_RANGE
-  SET_INT_OR_RETURN(env, clazz, SYNC_FILE_RANGE_WAIT_BEFORE);
-  SET_INT_OR_RETURN(env, clazz, SYNC_FILE_RANGE_WRITE);
-  SET_INT_OR_RETURN(env, clazz, SYNC_FILE_RANGE_WAIT_AFTER);
-#endif
-  clazz = (*env)->FindClass(env, NATIVE_IO_STAT_CLASS);
-  if (clazz == NULL) {
-    return; // exception has been raised
-  }
-  SET_INT_OR_RETURN(env, clazz, S_IFMT);
-  SET_INT_OR_RETURN(env, clazz, S_IFIFO);
-  SET_INT_OR_RETURN(env, clazz, S_IFCHR);
-  SET_INT_OR_RETURN(env, clazz, S_IFDIR);
-  SET_INT_OR_RETURN(env, clazz, S_IFBLK);
-  SET_INT_OR_RETURN(env, clazz, S_IFREG);
-  SET_INT_OR_RETURN(env, clazz, S_IFLNK);
-  SET_INT_OR_RETURN(env, clazz, S_IFSOCK);
-  SET_INT_OR_RETURN(env, clazz, S_ISUID);
-  SET_INT_OR_RETURN(env, clazz, S_ISGID);
-  SET_INT_OR_RETURN(env, clazz, S_ISVTX);
-  SET_INT_OR_RETURN(env, clazz, S_IRUSR);
-  SET_INT_OR_RETURN(env, clazz, S_IWUSR);
-  SET_INT_OR_RETURN(env, clazz, S_IXUSR);
-}
-#endif
-
-static void stat_init(JNIEnv *env, jclass nativeio_class) {
-  jclass clazz = NULL;
-  jclass obj_class = NULL;
-  jmethodID  obj_ctor = NULL;
-  // Init Stat
-  clazz = (*env)->FindClass(env, NATIVE_IO_STAT_CLASS);
-  if (!clazz) {
-    return; // exception has been raised
-  }
-  stat_clazz = (*env)->NewGlobalRef(env, clazz);
-  if (!stat_clazz) {
-    return; // exception has been raised
-  }
-  stat_ctor = (*env)->GetMethodID(env, stat_clazz, "<init>",
-    "(III)V");
-  if (!stat_ctor) {
-    return; // exception has been raised
-  }
-  stat_ctor2 = (*env)->GetMethodID(env, stat_clazz, "<init>",
-    "(Ljava/lang/String;Ljava/lang/String;I)V");
-  if (!stat_ctor2) {
-    return; // exception has been raised
-  }
-  obj_class = (*env)->FindClass(env, "java/lang/Object");
-  if (!obj_class) {
-    return; // exception has been raised
-  }
-  obj_ctor = (*env)->GetMethodID(env, obj_class,
-    "<init>", "()V");
-  if (!obj_ctor) {
-    return; // exception has been raised
-  }
-
-  if (workaround_non_threadsafe_calls(env, nativeio_class)) {
-    pw_lock_object = (*env)->NewObject(env, obj_class, obj_ctor);
-    PASS_EXCEPTIONS(env);
-    pw_lock_object = (*env)->NewGlobalRef(env, pw_lock_object);
-
-    PASS_EXCEPTIONS(env);
-  }
-}
-
-static void stat_deinit(JNIEnv *env) {
-  if (stat_clazz != NULL) {  
-    (*env)->DeleteGlobalRef(env, stat_clazz);
-    stat_clazz = NULL;
-  }
-  if (pw_lock_object != NULL) {
-    (*env)->DeleteGlobalRef(env, pw_lock_object);
-    pw_lock_object = NULL;
-  }
-}
-
-static void nioe_init(JNIEnv *env) {
-  // Init NativeIOException
-  nioe_clazz = (*env)->FindClass(
-    env, "org/apache/raft/io/nativeio/NativeIOException");
-  PASS_EXCEPTIONS(env);
-
-  nioe_clazz = (*env)->NewGlobalRef(env, nioe_clazz);
-#ifdef UNIX
-  nioe_ctor = (*env)->GetMethodID(env, nioe_clazz, "<init>",
-    "(Ljava/lang/String;Lorg/apache/raft/io/nativeio/Errno;)V");
-#endif
-
-#ifdef WINDOWS
-  nioe_ctor = (*env)->GetMethodID(env, nioe_clazz, "<init>",
-    "(Ljava/lang/String;I)V");
-#endif
-}
-
-static void nioe_deinit(JNIEnv *env) {
-  if (nioe_clazz != NULL) {
-    (*env)->DeleteGlobalRef(env, nioe_clazz);
-    nioe_clazz = NULL;
-  }
-  nioe_ctor = NULL;
-}
-
-/*
- * private static native void initNative();
- *
- * We rely on this function rather than lazy initialization because
- * the lazy approach may have a race if multiple callers try to
- * init at the same time.
- */
-JNIEXPORT void JNICALL
-Java_org_apache_raft_io_nativeio_NativeIO_initNative(
-  JNIEnv *env, jclass clazz) {
-#ifdef UNIX
-  consts_init(env);
-  PASS_EXCEPTIONS_GOTO(env, error);
-#endif
-  stat_init(env, clazz);
-  PASS_EXCEPTIONS_GOTO(env, error);
-  nioe_init(env);
-  PASS_EXCEPTIONS_GOTO(env, error);
-  fd_init(env);
-  PASS_EXCEPTIONS_GOTO(env, error);
-#ifdef UNIX
-  errno_enum_init(env);
-  PASS_EXCEPTIONS_GOTO(env, error);
-#endif
-  return;
-error:
-  // these are all idempodent and safe to call even if the
-  // class wasn't initted yet
-#ifdef UNIX
-  stat_deinit(env);
-#endif
-  nioe_deinit(env);
-  fd_deinit(env);
-#ifdef UNIX
-  errno_enum_deinit(env);
-#endif
-}
-
-/*
- * Class:     org_apache_raft_io_nativeio_NativeIO_POSIX
- * Method:    fstat
- * Signature: (Ljava/io/FileDescriptor;)Lorg/apache/raft/io/nativeio/NativeIO$POSIX$Stat;
- * public static native Stat fstat(FileDescriptor fd);
- *
- * The "00024" in the function name is an artifact of how JNI encodes
- * special characters. U+0024 is '$'.
- */
-JNIEXPORT jobject JNICALL
-Java_org_apache_raft_io_nativeio_NativeIO_00024POSIX_fstat(
-  JNIEnv *env, jclass clazz, jobject fd_object)
-{
-#ifdef UNIX
-  jobject ret = NULL;
-
-  int fd = fd_get(env, fd_object);
-  PASS_EXCEPTIONS_GOTO(env, cleanup);
-
-  struct stat s;
-  int rc = fstat(fd, &s);
-  if (rc != 0) {
-    throw_ioe(env, errno);
-    goto cleanup;
-  }
-
-  // Construct result
-  ret = (*env)->NewObject(env, stat_clazz, stat_ctor,
-    (jint)s.st_uid, (jint)s.st_gid, (jint)s.st_mode);
-
-cleanup:
-  return ret;
-#endif
-
-#ifdef WINDOWS
-  LPWSTR owner = NULL;
-  LPWSTR group = NULL;
-  int mode = 0;
-  jstring jstr_owner = NULL;
-  jstring jstr_group = NULL;
-  int rc;
-  jobject ret = NULL;
-  HANDLE hFile = (HANDLE) fd_get(env, fd_object);
-  PASS_EXCEPTIONS_GOTO(env, cleanup);
-
-  rc = FindFileOwnerAndPermissionByHandle(hFile, &owner, &group, &mode);
-  if (rc != ERROR_SUCCESS) {
-    throw_ioe(env, rc);
-    goto cleanup;
-  }
-
-  jstr_owner = (*env)->NewString(env, owner, (jsize) wcslen(owner));
-  if (jstr_owner == NULL) goto cleanup;
-
-  jstr_group = (*env)->NewString(env, group, (jsize) wcslen(group));;
-  if (jstr_group == NULL) goto cleanup;
-
-  ret = (*env)->NewObject(env, stat_clazz, stat_ctor2,
-    jstr_owner, jstr_group, (jint)mode);
-
-cleanup:
-  if (ret == NULL) {
-    if (jstr_owner != NULL)
-      (*env)->ReleaseStringChars(env, jstr_owner, owner);
-
-    if (jstr_group != NULL)
-      (*env)->ReleaseStringChars(env, jstr_group, group);
-  }
-
-  LocalFree(owner);
-  LocalFree(group);
-
-  return ret;
-#endif
-}
-
-
-
-/**
- * public static native void posix_fadvise(
- *   FileDescriptor fd, long offset, long len, int flags);
- *
- * The "00024" in the function name is an artifact of how JNI encodes
- * special characters. U+0024 is '$'.
- */
-JNIEXPORT void JNICALL
-Java_org_apache_raft_io_nativeio_NativeIO_00024POSIX_posix_1fadvise(
-  JNIEnv *env, jclass clazz,
-  jobject fd_object, jlong offset, jlong len, jint flags)
-{
-#ifndef HAVE_POSIX_FADVISE
-  THROW(env, "java/lang/UnsupportedOperationException",
-        "fadvise support not available");
-#else
-  int fd = fd_get(env, fd_object);
-  PASS_EXCEPTIONS(env);
-
-  int err = 0;
-  if ((err = posix_fadvise(fd, (off_t)offset, (off_t)len, flags))) {
-#ifdef __FreeBSD__
-    throw_ioe(env, errno);
-#else
-    throw_ioe(env, err);
-#endif
-  }
-#endif
-}
-
-#if defined(HAVE_SYNC_FILE_RANGE)
-#  define my_sync_file_range sync_file_range
-#elif defined(SYS_sync_file_range)
-// RHEL 5 kernels have sync_file_range support, but the glibc
-// included does not have the library function. We can
-// still call it directly, and if it's not supported by the
-// kernel, we'd get ENOSYS. See RedHat Bugzilla #518581
-static int manual_sync_file_range (int fd, __off64_t from, __off64_t to, unsigned int flags)
-{
-#ifdef __x86_64__
-  return syscall( SYS_sync_file_range, fd, from, to, flags);
-#else
-  return syscall (SYS_sync_file_range, fd,
-    __LONG_LONG_PAIR ((long) (from >> 32), (long) from),
-    __LONG_LONG_PAIR ((long) (to >> 32), (long) to),
-    flags);
-#endif
-}
-#define my_sync_file_range manual_sync_file_range
-#endif
-
-/**
- * public static native void sync_file_range(
- *   FileDescriptor fd, long offset, long len, int flags);
- *
- * The "00024" in the function name is an artifact of how JNI encodes
- * special characters. U+0024 is '$'.
- */
-JNIEXPORT void JNICALL
-Java_org_apache_raft_io_nativeio_NativeIO_00024POSIX_sync_1file_1range(
-  JNIEnv *env, jclass clazz,
-  jobject fd_object, jlong offset, jlong len, jint flags)
-{
-#ifndef my_sync_file_range
-  THROW(env, "java/lang/UnsupportedOperationException",
-        "sync_file_range support not available");
-#else
-  int fd = fd_get(env, fd_object);
-  PASS_EXCEPTIONS(env);
-
-  if (my_sync_file_range(fd, (off_t)offset, (off_t)len, flags)) {
-    if (errno == ENOSYS) {
-      // we know the syscall number, but it's not compiled
-      // into the running kernel
-      THROW(env, "java/lang/UnsupportedOperationException",
-            "sync_file_range kernel support not available");
-      return;
-    } else {
-      throw_ioe(env, errno);
-    }
-  }
-#endif
-}
-
-#define CHECK_DIRECT_BUFFER_ADDRESS(buf) \
-  { \
-    if (!buf) { \
-      THROW(env, "java/lang/UnsupportedOperationException", \
-        "JNI access to direct buffers not available"); \
-      return; \
-    } \
-  }
-
-/**
- * public static native void mlock_native(
- *   ByteBuffer buffer, long offset);
- *
- * The "00024" in the function name is an artifact of how JNI encodes
- * special characters. U+0024 is '$'.
- */
-JNIEXPORT void JNICALL
-Java_org_apache_raft_io_nativeio_NativeIO_00024POSIX_mlock_1native(
-  JNIEnv *env, jclass clazz,
-  jobject buffer, jlong len)
-{
-  void* buf = (void*)(*env)->GetDirectBufferAddress(env, buffer);
-  PASS_EXCEPTIONS(env);
-
-#ifdef UNIX
-  if (mlock(buf, len)) {
-    CHECK_DIRECT_BUFFER_ADDRESS(buf);
-    throw_ioe(env, errno);
-  }
-#endif
-
-#ifdef WINDOWS
-  if (!VirtualLock(buf, len)) {
-    CHECK_DIRECT_BUFFER_ADDRESS(buf);
-    throw_ioe(env, GetLastError());
-  }
-#endif
-}
-
-/*
- * Class:     org_apache_raft_io_nativeio_NativeIO_POSIX
- * Method:    open
- * Signature: (Ljava/lang/String;II)Ljava/io/FileDescriptor;
- * public static native FileDescriptor open(String path, int flags, int mode);
- *
- * The "00024" in the function name is an artifact of how JNI encodes
- * special characters. U+0024 is '$'.
- */
-JNIEXPORT jobject JNICALL
-Java_org_apache_raft_io_nativeio_NativeIO_00024POSIX_open(
-  JNIEnv *env, jclass clazz, jstring j_path,
-  jint flags, jint mode)
-{
-#ifdef UNIX
-  jobject ret = NULL;
-
-  const char *path = (*env)->GetStringUTFChars(env, j_path, NULL);
-  if (path == NULL) goto cleanup; // JVM throws Exception for us
-
-  int fd;  
-  if (flags & O_CREAT) {
-    fd = open(path, flags, mode);
-  } else {
-    fd = open(path, flags);
-  }
-
-  if (fd == -1) {
-    throw_ioe(env, errno);
-    goto cleanup;
-  }
-
-  ret = fd_create(env, fd);
-
-cleanup:
-  if (path != NULL) {
-    (*env)->ReleaseStringUTFChars(env, j_path, path);
-  }
-  return ret;
-#endif
-
-#ifdef WINDOWS
-  THROW(env, "java/io/IOException",
-    "The function POSIX.open() is not supported on Windows");
-  return NULL;
-#endif
-}
-
-/*
- * Class:     org_apache_raft_io_nativeio_NativeIO_Windows
- * Method:    createDirectoryWithMode0
- * Signature: (Ljava/lang/String;I)V
- *
- * The "00024" in the function name is an artifact of how JNI encodes
- * special characters. U+0024 is '$'.
- */
-JNIEXPORT void JNICALL
-  Java_org_apache_raft_io_nativeio_NativeIO_00024Windows_createDirectoryWithMode0
-  (JNIEnv *env, jclass clazz, jstring j_path, jint mode)
-{
-#ifdef WINDOWS
-  DWORD dwRtnCode = ERROR_SUCCESS;
-
-  LPCWSTR path = (LPCWSTR) (*env)->GetStringChars(env, j_path, NULL);
-  if (!path) {
-    goto done;
-  }
-
-  dwRtnCode = CreateDirectoryWithMode(path, mode);
-
-done:
-  if (path) {
-    (*env)->ReleaseStringChars(env, j_path, (const jchar*) path);
-  }
-  if (dwRtnCode != ERROR_SUCCESS) {
-    throw_ioe(env, dwRtnCode);
-  }
-#else
-  THROW(env, "java/io/IOException",
-    "The function Windows.createDirectoryWithMode0() is not supported on this platform");
-#endif
-}
-
-/*
- * Class:     org_apache_raft_io_nativeio_NativeIO_Windows
- * Method:    createFileWithMode0
- * Signature: (Ljava/lang/String;JJJI)Ljava/io/FileDescriptor;
- *
- * The "00024" in the function name is an artifact of how JNI encodes
- * special characters. U+0024 is '$'.
- */
-JNIEXPORT jobject JNICALL
-  Java_org_apache_raft_io_nativeio_NativeIO_00024Windows_createFileWithMode0
-  (JNIEnv *env, jclass clazz, jstring j_path,
-  jlong desiredAccess, jlong shareMode, jlong creationDisposition, jint mode)
-{
-#ifdef WINDOWS
-  DWORD dwRtnCode = ERROR_SUCCESS;
-  HANDLE hFile = INVALID_HANDLE_VALUE;
-  jobject fd = NULL;
-
-  LPCWSTR path = (LPCWSTR) (*env)->GetStringChars(env, j_path, NULL);
-  if (!path) {
-    goto done;
-  }
-
-  dwRtnCode = CreateFileWithMode(path, desiredAccess, shareMode,
-      creationDisposition, mode, &hFile);
-  if (dwRtnCode != ERROR_SUCCESS) {
-    goto done;
-  }
-
-  fd = fd_create(env, (long) hFile);
-
-done:
-  if (path) {
-    (*env)->ReleaseStringChars(env, j_path, (const jchar*) path);
-  }
-  if (dwRtnCode != ERROR_SUCCESS) {
-    throw_ioe(env, dwRtnCode);
-  }
-  return fd;
-#else
-  THROW(env, "java/io/IOException",
-    "The function Windows.createFileWithMode0() is not supported on this platform");
-  return NULL;
-#endif
-}
-
-/*
- * Class:     org_apache_raft_io_nativeio_NativeIO_Windows
- * Method:    createFile
- * Signature: (Ljava/lang/String;JJJ)Ljava/io/FileDescriptor;
- *
- * The "00024" in the function name is an artifact of how JNI encodes
- * special characters. U+0024 is '$'.
- */
-JNIEXPORT jobject JNICALL Java_org_apache_raft_io_nativeio_NativeIO_00024Windows_createFile
-  (JNIEnv *env, jclass clazz, jstring j_path,
-  jlong desiredAccess, jlong shareMode, jlong creationDisposition)
-{
-#ifdef UNIX
-  THROW(env, "java/io/IOException",
-    "The function Windows.createFile() is not supported on Unix");
-  return NULL;
-#endif
-
-#ifdef WINDOWS
-  DWORD dwRtnCode = ERROR_SUCCESS;
-  BOOL isSymlink = FALSE;
-  BOOL isJunction = FALSE;
-  DWORD dwFlagsAndAttributes = FILE_ATTRIBUTE_NORMAL | FILE_FLAG_BACKUP_SEMANTICS;
-  jobject ret = (jobject) NULL;
-  HANDLE hFile = INVALID_HANDLE_VALUE;
-  WCHAR *path = (WCHAR *) (*env)->GetStringChars(env, j_path, (jboolean*)NULL);
-  if (path == NULL) goto cleanup;
-
-  // Set the flag for a symbolic link or a junctions point only when it exists.
-  // According to MSDN if the call to CreateFile() function creates a file,
-  // there is no change in behavior. So we do not throw if no file is found.
-  //
-  dwRtnCode = SymbolicLinkCheck(path, &isSymlink);
-  if (dwRtnCode != ERROR_SUCCESS && dwRtnCode != ERROR_FILE_NOT_FOUND) {
-    throw_ioe(env, dwRtnCode);
-    goto cleanup;
-  }
-  dwRtnCode = JunctionPointCheck(path, &isJunction);
-  if (dwRtnCode != ERROR_SUCCESS && dwRtnCode != ERROR_FILE_NOT_FOUND) {
-    throw_ioe(env, dwRtnCode);
-    goto cleanup;
-  }
-  if (isSymlink || isJunction)
-    dwFlagsAndAttributes |= FILE_FLAG_OPEN_REPARSE_POINT;
-
-  hFile = CreateFile(path,
-    (DWORD) desiredAccess,
-    (DWORD) shareMode,
-    (LPSECURITY_ATTRIBUTES ) NULL,
-    (DWORD) creationDisposition,
-    dwFlagsAndAttributes,
-    NULL);
-  if (hFile == INVALID_HANDLE_VALUE) {
-    throw_ioe(env, GetLastError());
-    goto cleanup;
-  }
-
-  ret = fd_create(env, (long) hFile);
-cleanup:
-  if (path != NULL) {
-    (*env)->ReleaseStringChars(env, j_path, (const jchar*)path);
-  }
-  return (jobject) ret;
-#endif
-}
-
-/*
- * Class:     org_apache_raft_io_nativeio_NativeIO_POSIX
- * Method:    chmod
- * Signature: (Ljava/lang/String;I)V
- */
-JNIEXPORT void JNICALL Java_org_apache_raft_io_nativeio_NativeIO_00024POSIX_chmodImpl
-  (JNIEnv *env, jclass clazz, jstring j_path, jint mode)
-{
-#ifdef UNIX
-  const char *path = (*env)->GetStringUTFChars(env, j_path, NULL);
-  if (path == NULL) return; // JVM throws Exception for us
-
-  if (chmod(path, mode) != 0) {
-    throw_ioe(env, errno);
-  }
-
-  (*env)->ReleaseStringUTFChars(env, j_path, path);
-#endif
-
-#ifdef WINDOWS
-  DWORD dwRtnCode = ERROR_SUCCESS;
-  LPCWSTR path = (LPCWSTR) (*env)->GetStringChars(env, j_path, NULL);
-  if (path == NULL) return; // JVM throws Exception for us
-
-  if ((dwRtnCode = ChangeFileModeByMask((LPCWSTR) path, mode)) != ERROR_SUCCESS)
-  {
-    throw_ioe(env, dwRtnCode);
-  }
-
-  (*env)->ReleaseStringChars(env, j_path, (const jchar*) path);
-#endif
-}
-
-JNIEXPORT jlong JNICALL 
-Java_org_apache_raft_io_nativeio_NativeIO_00024POSIX_mmap(
-  JNIEnv *env, jclass clazz, jobject jfd, jint jprot,
-  jboolean jshared, jlong length)
-{
-#ifdef UNIX
-  void *addr = 0;
-  int prot, flags, fd;
-  
-  prot = ((jprot & MMAP_PROT_READ) ? PROT_READ : 0) |
-         ((jprot & MMAP_PROT_WRITE) ? PROT_WRITE : 0) |
-         ((jprot & MMAP_PROT_EXEC) ? PROT_EXEC : 0);
-  flags = (jshared == JNI_TRUE) ? MAP_SHARED : MAP_PRIVATE;
-  fd = fd_get(env, jfd);
-  addr = mmap(NULL, length, prot, flags, fd, 0);
-  if (addr == MAP_FAILED) {
-    throw_ioe(env, errno);
-  }
-  return (jlong)(intptr_t)addr;
-#endif  //   UNIX
-
-#ifdef WINDOWS
-  THROW(env, "java/io/IOException",
-    "The function POSIX.mmap() is not supported on Windows");
-  return (jlong)(intptr_t)NULL;
-#endif
-}
-
-JNIEXPORT void JNICALL 
-Java_org_apache_raft_io_nativeio_NativeIO_00024POSIX_munmap(
-  JNIEnv *env, jclass clazz, jlong jaddr, jlong length)
-{
-#ifdef UNIX
-  void *addr;
-
-  addr = (void*)(intptr_t)jaddr;
-  if (munmap(addr, length) < 0) {
-    throw_ioe(env, errno);
-  }
-#endif  //   UNIX
-
-#ifdef WINDOWS
-  THROW(env, "java/io/IOException",
-    "The function POSIX.munmap() is not supported on Windows");
-#endif
-}
-
-/*
- * Throw a java.IO.IOException, generating the message from errno.
- */
-void throw_ioe(JNIEnv* env, int errnum)
-{
-#ifdef UNIX
-  char message[80];
-  jstring jstr_message;
-
-  snprintf(message,sizeof(message),"%s",terror(errnum));
-
-  jobject errno_obj = errno_to_enum(env, errnum);
-
-  if ((jstr_message = (*env)->NewStringUTF(env, message)) == NULL)
-    goto err;
-
-  jthrowable obj = (jthrowable)(*env)->NewObject(env, nioe_clazz, nioe_ctor,
-    jstr_message, errno_obj);
-  if (obj == NULL) goto err;
-
-  (*env)->Throw(env, obj);
-  return;
-
-err:
-  if (jstr_message != NULL)
-    (*env)->ReleaseStringUTFChars(env, jstr_message, message);
-#endif
-
-#ifdef WINDOWS
-  DWORD len = 0;
-  LPWSTR buffer = NULL;
-  const jchar* message = NULL;
-  jstring jstr_message = NULL;
-  jthrowable obj = NULL;
-
-  len = FormatMessageW(
-    FORMAT_MESSAGE_ALLOCATE_BUFFER | FORMAT_MESSAGE_FROM_SYSTEM,
-    NULL, *(DWORD*) (&errnum), // reinterpret cast
-    MAKELANGID(LANG_NEUTRAL, SUBLANG_DEFAULT),
-    (LPWSTR) &buffer, 0, NULL);
-
-  if (len > 0)
-  {
-    message = (const jchar*) buffer;
-  }
-  else
-  {
-    message = (const jchar*) L"Unknown error.";
-  }
-
-  if ((jstr_message = (*env)->NewString(env, message, len)) == NULL)
-    goto err;
-  LocalFree(buffer);
-  buffer = NULL; // Set buffer to NULL to avoid double free
-
-  obj = (jthrowable)(*env)->NewObject(env, nioe_clazz, nioe_ctor,
-    jstr_message, errnum);
-  if (obj == NULL) goto err;
-
-  (*env)->Throw(env, obj);
-  return;
-
-err:
-  if (jstr_message != NULL)
-    (*env)->ReleaseStringChars(env, jstr_message, message);
-  LocalFree(buffer);
-  return;
-#endif
-}
-
-#ifdef UNIX
-/*
- * Determine how big a buffer we need for reentrant getpwuid_r and getgrnam_r
- */
-ssize_t get_pw_buflen() {
-  long ret = 0;
-  #ifdef _SC_GETPW_R_SIZE_MAX
-  ret = sysconf(_SC_GETPW_R_SIZE_MAX);
-  #endif
-  return (ret > 512) ? ret : 512;
-}
-#endif
-
-/*
- * Class:     org_apache_raft_io_nativeio_NativeIO_Windows
- * Method:    setFilePointer
- * Signature: (Ljava/io/FileDescriptor;JJ)J
- *
- * The "00024" in the function name is an artifact of how JNI encodes
- * special characters. U+0024 is '$'.
- */
-JNIEXPORT jlong JNICALL
-Java_org_apache_raft_io_nativeio_NativeIO_00024Windows_setFilePointer
-  (JNIEnv *env, jclass clazz, jobject fd_object, jlong distanceToMove, jlong moveMethod)
-{
-#ifdef UNIX
-  THROW(env, "java/io/IOException",
-    "The function setFilePointer(FileDescriptor) is not supported on Unix");
-  return (jlong)(intptr_t)NULL;
-#endif
-
-#ifdef WINDOWS
-  DWORD distanceToMoveLow = (DWORD) distanceToMove;
-  LONG distanceToMoveHigh = (LONG) (distanceToMove >> 32);
-  DWORD distanceMovedLow = 0;
-  HANDLE hFile = (HANDLE) fd_get(env, fd_object);
-  PASS_EXCEPTIONS_GOTO(env, cleanup);
-
-  distanceMovedLow = SetFilePointer(hFile,
-    distanceToMoveLow, &distanceToMoveHigh, (DWORD) moveMethod);
-
-  if (distanceMovedLow == INVALID_SET_FILE_POINTER) {
-     throw_ioe(env, GetLastError());
-     return -1;
-  }
-
-cleanup:
-
-  return ((jlong) distanceToMoveHigh << 32) | (jlong) distanceMovedLow;
-#endif
-}
-
-/*
- * Class:     org_apache_raft_io_nativeio_NativeIO_Windows
- * Method:    access0
- * Signature: (Ljava/lang/String;I)Z
- */
-JNIEXPORT jboolean JNICALL Java_org_apache_raft_io_nativeio_NativeIO_00024Windows_access0
-  (JNIEnv *env, jclass clazz, jstring jpath, jint jaccess)
-{
-#ifdef UNIX
-  THROW(env, "java/io/IOException",
-    "The function access0(path, access) is not supported on Unix");
-  return (jlong)(intptr_t)NULL;
-#endif
-
-#ifdef WINDOWS
-  LPCWSTR path = NULL;
-  DWORD dwRtnCode = ERROR_SUCCESS;
-  ACCESS_MASK access = (ACCESS_MASK)jaccess;
-  BOOL allowed = FALSE;
-
-  path = (LPCWSTR) (*env)->GetStringChars(env, jpath, NULL);
-  if (!path) goto cleanup; // exception was thrown
-
-  dwRtnCode = CheckAccessForCurrentUser(path, access, &allowed);
-  if (dwRtnCode != ERROR_SUCCESS) {
-    throw_ioe(env, dwRtnCode);
-    goto cleanup;
-  }
-
-cleanup:
-  if (path) (*env)->ReleaseStringChars(env, jpath, path);
-
-  return (jboolean)allowed;
-#endif
-}
-
-/*
- * Class:     org_apache_raft_io_nativeio_NativeIO_Windows
- * Method:    extendWorkingSetSize
- * Signature: (J)V
- *
- * The "00024" in the function name is an artifact of how JNI encodes
- * special characters. U+0024 is '$'.
- */
-JNIEXPORT void JNICALL
-Java_org_apache_raft_io_nativeio_NativeIO_00024Windows_extendWorkingSetSize(
-  JNIEnv *env, jclass clazz, jlong delta)
-{
-#ifdef UNIX
-  THROW(env, "java/io/IOException",
-    "The function extendWorkingSetSize(delta) is not supported on Unix");
-#endif
-
-#ifdef WINDOWS
-  SIZE_T min, max;
-  HANDLE hProcess = GetCurrentProcess();
-  if (!GetProcessWorkingSetSize(hProcess, &min, &max)) {
-    throw_ioe(env, GetLastError());
-    return;
-  }
-  if (!SetProcessWorkingSetSizeEx(hProcess, min + delta, max + delta,
-      QUOTA_LIMITS_HARDWS_MIN_DISABLE | QUOTA_LIMITS_HARDWS_MAX_DISABLE)) {
-    throw_ioe(env, GetLastError());
-    return;
-  }
-  // There is no need to call CloseHandle on the pseudo-handle returned from
-  // GetCurrentProcess.
-#endif
-}
-
-JNIEXPORT void JNICALL 
-Java_org_apache_raft_io_nativeio_NativeIO_renameTo0(JNIEnv *env, 
-jclass clazz, jstring jsrc, jstring jdst)
-{
-#ifdef UNIX
-  const char *src = NULL, *dst = NULL;
-  
-  src = (*env)->GetStringUTFChars(env, jsrc, NULL);
-  if (!src) goto done; // exception was thrown
-  dst = (*env)->GetStringUTFChars(env, jdst, NULL);
-  if (!dst) goto done; // exception was thrown
-  if (rename(src, dst)) {
-    throw_ioe(env, errno);
-  }
-
-done:
-  if (src) (*env)->ReleaseStringUTFChars(env, jsrc, src);
-  if (dst) (*env)->ReleaseStringUTFChars(env, jdst, dst);
-#endif
-
-#ifdef WINDOWS
-  LPCWSTR src = NULL, dst = NULL;
-
-  src = (LPCWSTR) (*env)->GetStringChars(env, jsrc, NULL);
-  if (!src) goto done; // exception was thrown
-  dst = (LPCWSTR) (*env)->GetStringChars(env, jdst, NULL);
-  if (!dst) goto done; // exception was thrown
-  if (!MoveFile(src, dst)) {
-    throw_ioe(env, GetLastError());
-  }
-
-done:
-  if (src) (*env)->ReleaseStringChars(env, jsrc, src);
-  if (dst) (*env)->ReleaseStringChars(env, jdst, dst);
-#endif
-}
-
-JNIEXPORT jlong JNICALL
-Java_org_apache_raft_io_nativeio_NativeIO_getMemlockLimit0(
-JNIEnv *env, jclass clazz)
-{
-#ifdef RLIMIT_MEMLOCK
-  struct rlimit rlim;
-  int rc = getrlimit(RLIMIT_MEMLOCK, &rlim);
-  if (rc != 0) {
-    throw_ioe(env, errno);
-    return 0;
-  }
-  return (rlim.rlim_cur == RLIM_INFINITY) ?
-    INT64_MAX : rlim.rlim_cur;
-#else
-  return 0;
-#endif
-}
-
-JNIEXPORT void JNICALL
-Java_org_apache_raft_io_nativeio_NativeIO_copyFileUnbuffered0(
-JNIEnv *env, jclass clazz, jstring jsrc, jstring jdst)
-{
-#ifdef UNIX
-  THROW(env, "java/lang/UnsupportedOperationException",
-    "The function copyFileUnbuffered0 should not be used on Unix. Use FileChannel#transferTo instead.");
-#endif
-
-#ifdef WINDOWS
-  LPCWSTR src = NULL, dst = NULL;
-
-  src = (LPCWSTR) (*env)->GetStringChars(env, jsrc, NULL);
-  if (!src) goto cleanup; // exception was thrown
-  dst = (LPCWSTR) (*env)->GetStringChars(env, jdst, NULL);
-  if (!dst) goto cleanup; // exception was thrown
-  if (!CopyFileEx(src, dst, NULL, NULL, NULL, COPY_FILE_NO_BUFFERING)) {
-    throw_ioe(env, GetLastError());
-  }
-
-cleanup:
-  if (src) (*env)->ReleaseStringChars(env, jsrc, src);
-  if (dst) (*env)->ReleaseStringChars(env, jdst, dst);
-#endif
-}
-
-/**
- * vim: sw=2: ts=2: et:
- */


[07/54] [abbrv] incubator-ratis git commit: Rename RaftServer to RaftServerImpl (will add an interface.)

Posted by ji...@apache.org.
Rename RaftServer to RaftServerImpl (will add an interface.)


Project: http://git-wip-us.apache.org/repos/asf/incubator-ratis/repo
Commit: http://git-wip-us.apache.org/repos/asf/incubator-ratis/commit/22f3ee0b
Tree: http://git-wip-us.apache.org/repos/asf/incubator-ratis/tree/22f3ee0b
Diff: http://git-wip-us.apache.org/repos/asf/incubator-ratis/diff/22f3ee0b

Branch: refs/heads/master
Commit: 22f3ee0b46a5485deafa84939eb5290938b5552b
Parents: c36810e
Author: Tsz-Wo Nicholas Sze <sz...@hortonworks.com>
Authored: Mon Jan 2 20:10:28 2017 +0800
Committer: Tsz-Wo Nicholas Sze <sz...@hortonworks.com>
Committed: Mon Jan 2 20:10:28 2017 +0800

----------------------------------------------------------------------
 .../java/org/apache/raft/TestBatchAppend.java   |   4 +-
 .../org/apache/raft/TestRestartRaftPeer.java    |   4 +-
 .../TestRaftStateMachineException.java          |   4 +-
 .../org/apache/raft/grpc/RaftGRpcService.java   |   4 +-
 .../raft/grpc/server/GRpcLogAppender.java       |   6 +-
 .../server/PipelinedLogAppenderFactory.java     |   6 +-
 .../raft/grpc/MiniRaftClusterWithGRpc.java      |  14 +-
 .../org/apache/raft/grpc/TestRaftStream.java    |  12 +-
 .../org/apache/raft/grpc/TestRaftWithGrpc.java  |   4 +-
 .../raft/hadooprpc/server/HadoopRpcService.java |   4 +-
 .../hadooprpc/MiniRaftClusterWithHadoopRpc.java |  12 +-
 .../raft/hadooprpc/TestRaftWithHadoopRpc.java   |   4 +-
 .../raft/netty/server/NettyRpcService.java      |   4 +-
 .../raft/netty/MiniRaftClusterWithNetty.java    |  14 +-
 .../apache/raft/netty/TestRaftWithNetty.java    |   4 +-
 .../apache/raft/server/impl/FollowerState.java  |   6 +-
 .../apache/raft/server/impl/LeaderElection.java |   4 +-
 .../apache/raft/server/impl/LeaderState.java    |   6 +-
 .../apache/raft/server/impl/LogAppender.java    |   6 +-
 .../raft/server/impl/LogAppenderFactory.java    |   8 +-
 .../raft/server/impl/PendingRequests.java       |   6 +-
 .../org/apache/raft/server/impl/RaftServer.java | 749 -------------------
 .../apache/raft/server/impl/RaftServerImpl.java | 749 +++++++++++++++++++
 .../raft/server/impl/RequestDispatcher.java     |   4 +-
 .../apache/raft/server/impl/ServerState.java    |  10 +-
 .../raft/server/impl/StateMachineUpdater.java   |   4 +-
 .../raft/server/storage/RaftLogWorker.java      |   8 +-
 .../raft/server/storage/SegmentedRaftLog.java   |   6 +-
 .../java/org/apache/raft/MiniRaftCluster.java   |  60 +-
 .../java/org/apache/raft/RaftBasicTests.java    |   8 +-
 .../raft/RaftNotLeaderExceptionBaseTest.java    |   4 +-
 .../test/java/org/apache/raft/RaftTestUtil.java |  22 +-
 .../impl/BlockRequestHandlingInjection.java     |   6 +-
 .../impl/RaftReconfigurationBaseTest.java       |   4 +-
 .../raft/server/impl/RaftServerTestUtil.java    |   2 +-
 .../MiniRaftClusterWithSimulatedRpc.java        |  10 +-
 .../server/simulation/SimulatedServerRpc.java   |   6 +-
 .../simulation/TestRaftWithSimulatedRpc.java    |   4 +-
 .../raft/statemachine/RaftSnapshotBaseTest.java |   4 +-
 .../raft/statemachine/TestStateMachine.java     |   8 +-
 40 files changed, 902 insertions(+), 902 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-ratis/blob/22f3ee0b/raft-examples/src/test/java/org/apache/raft/TestBatchAppend.java
----------------------------------------------------------------------
diff --git a/raft-examples/src/test/java/org/apache/raft/TestBatchAppend.java b/raft-examples/src/test/java/org/apache/raft/TestBatchAppend.java
index c730245..1fd8ba0 100644
--- a/raft-examples/src/test/java/org/apache/raft/TestBatchAppend.java
+++ b/raft-examples/src/test/java/org/apache/raft/TestBatchAppend.java
@@ -22,7 +22,7 @@ import org.apache.raft.RaftTestUtil.SimpleMessage;
 import org.apache.raft.client.RaftClient;
 import org.apache.raft.conf.RaftProperties;
 import org.apache.raft.examples.RaftExamplesTestUtil;
-import org.apache.raft.server.impl.RaftServer;
+import org.apache.raft.server.impl.RaftServerImpl;
 import org.apache.raft.server.simulation.RequestHandler;
 import org.apache.raft.statemachine.SimpleStateMachine4Testing;
 import org.apache.raft.statemachine.StateMachine;
@@ -55,7 +55,7 @@ import static org.apache.raft.server.RaftServerConfigKeys.*;
 public class TestBatchAppend {
   static Logger LOG = LoggerFactory.getLogger(TestBatchAppend.class);
   static {
-    RaftUtils.setLogLevel(RaftServer.LOG, Level.DEBUG);
+    RaftUtils.setLogLevel(RaftServerImpl.LOG, Level.DEBUG);
     RaftUtils.setLogLevel(RequestHandler.LOG, Level.DEBUG);
     RaftUtils.setLogLevel(RaftClient.LOG, Level.DEBUG);
   }

http://git-wip-us.apache.org/repos/asf/incubator-ratis/blob/22f3ee0b/raft-examples/src/test/java/org/apache/raft/TestRestartRaftPeer.java
----------------------------------------------------------------------
diff --git a/raft-examples/src/test/java/org/apache/raft/TestRestartRaftPeer.java b/raft-examples/src/test/java/org/apache/raft/TestRestartRaftPeer.java
index fadd7a8..e04d141 100644
--- a/raft-examples/src/test/java/org/apache/raft/TestRestartRaftPeer.java
+++ b/raft-examples/src/test/java/org/apache/raft/TestRestartRaftPeer.java
@@ -22,7 +22,7 @@ import org.apache.raft.RaftTestUtil.SimpleMessage;
 import org.apache.raft.client.RaftClient;
 import org.apache.raft.conf.RaftProperties;
 import org.apache.raft.examples.RaftExamplesTestUtil;
-import org.apache.raft.server.impl.RaftServer;
+import org.apache.raft.server.impl.RaftServerImpl;
 import org.apache.raft.server.RaftServerConfigKeys;
 import org.apache.raft.server.simulation.RequestHandler;
 import org.apache.raft.server.storage.RaftLog;
@@ -49,7 +49,7 @@ import java.util.Collection;
 public class TestRestartRaftPeer {
   static Logger LOG = LoggerFactory.getLogger(TestRestartRaftPeer.class);
   static {
-    RaftUtils.setLogLevel(RaftServer.LOG, Level.DEBUG);
+    RaftUtils.setLogLevel(RaftServerImpl.LOG, Level.DEBUG);
     RaftUtils.setLogLevel(RaftLog.LOG, Level.DEBUG);
     RaftUtils.setLogLevel(RequestHandler.LOG, Level.DEBUG);
     RaftUtils.setLogLevel(RaftClient.LOG, Level.DEBUG);

http://git-wip-us.apache.org/repos/asf/incubator-ratis/blob/22f3ee0b/raft-examples/src/test/java/org/apache/raft/statemachine/TestRaftStateMachineException.java
----------------------------------------------------------------------
diff --git a/raft-examples/src/test/java/org/apache/raft/statemachine/TestRaftStateMachineException.java b/raft-examples/src/test/java/org/apache/raft/statemachine/TestRaftStateMachineException.java
index 2fa6e53..0832579 100644
--- a/raft-examples/src/test/java/org/apache/raft/statemachine/TestRaftStateMachineException.java
+++ b/raft-examples/src/test/java/org/apache/raft/statemachine/TestRaftStateMachineException.java
@@ -24,7 +24,7 @@ import org.apache.raft.client.RaftClient;
 import org.apache.raft.examples.RaftExamplesTestUtil;
 import org.apache.raft.protocol.Message;
 import org.apache.raft.protocol.StateMachineException;
-import org.apache.raft.server.impl.RaftServer;
+import org.apache.raft.server.impl.RaftServerImpl;
 import org.apache.raft.server.simulation.RequestHandler;
 import org.apache.raft.server.storage.RaftLog;
 import org.apache.raft.util.RaftUtils;
@@ -42,7 +42,7 @@ import static org.junit.Assert.fail;
 @RunWith(Parameterized.class)
 public class TestRaftStateMachineException {
   static {
-    RaftUtils.setLogLevel(RaftServer.LOG, Level.DEBUG);
+    RaftUtils.setLogLevel(RaftServerImpl.LOG, Level.DEBUG);
     RaftUtils.setLogLevel(RaftLog.LOG, Level.DEBUG);
     RaftUtils.setLogLevel(RequestHandler.LOG, Level.DEBUG);
     RaftUtils.setLogLevel(RaftClient.LOG, Level.DEBUG);

http://git-wip-us.apache.org/repos/asf/incubator-ratis/blob/22f3ee0b/raft-grpc/src/main/java/org/apache/raft/grpc/RaftGRpcService.java
----------------------------------------------------------------------
diff --git a/raft-grpc/src/main/java/org/apache/raft/grpc/RaftGRpcService.java b/raft-grpc/src/main/java/org/apache/raft/grpc/RaftGRpcService.java
index c9a0daf..d465724 100644
--- a/raft-grpc/src/main/java/org/apache/raft/grpc/RaftGRpcService.java
+++ b/raft-grpc/src/main/java/org/apache/raft/grpc/RaftGRpcService.java
@@ -23,7 +23,7 @@ import org.apache.raft.grpc.client.RaftClientProtocolService;
 import org.apache.raft.grpc.server.RaftServerProtocolClient;
 import org.apache.raft.grpc.server.RaftServerProtocolService;
 import org.apache.raft.protocol.RaftPeer;
-import org.apache.raft.server.impl.RaftServer;
+import org.apache.raft.server.impl.RaftServerImpl;
 import org.apache.raft.server.impl.RaftServerRpc;
 import org.apache.raft.server.impl.RequestDispatcher;
 import org.apache.raft.shaded.io.grpc.Server;
@@ -54,7 +54,7 @@ public class RaftGRpcService implements RaftServerRpc {
       Collections.synchronizedMap(new HashMap<>());
   private final String selfId;
 
-  public RaftGRpcService(RaftServer raftServer, RaftProperties properties) {
+  public RaftGRpcService(RaftServerImpl raftServer, RaftProperties properties) {
     int port = properties.getInt(RAFT_GRPC_SERVER_PORT_KEY,
         RAFT_GRPC_SERVER_PORT_DEFAULT);
     int maxMessageSize = properties.getInt(

http://git-wip-us.apache.org/repos/asf/incubator-ratis/blob/22f3ee0b/raft-grpc/src/main/java/org/apache/raft/grpc/server/GRpcLogAppender.java
----------------------------------------------------------------------
diff --git a/raft-grpc/src/main/java/org/apache/raft/grpc/server/GRpcLogAppender.java b/raft-grpc/src/main/java/org/apache/raft/grpc/server/GRpcLogAppender.java
index b171bed..a8a39bb 100644
--- a/raft-grpc/src/main/java/org/apache/raft/grpc/server/GRpcLogAppender.java
+++ b/raft-grpc/src/main/java/org/apache/raft/grpc/server/GRpcLogAppender.java
@@ -23,7 +23,7 @@ import org.apache.raft.grpc.RaftGrpcConfigKeys;
 import org.apache.raft.server.impl.FollowerInfo;
 import org.apache.raft.server.impl.LeaderState;
 import org.apache.raft.server.impl.LogAppender;
-import org.apache.raft.server.impl.RaftServer;
+import org.apache.raft.server.impl.RaftServerImpl;
 import org.apache.raft.shaded.io.grpc.Status;
 import org.apache.raft.shaded.io.grpc.stub.StreamObserver;
 import org.apache.raft.shaded.proto.RaftProtos.AppendEntriesReplyProto;
@@ -56,8 +56,8 @@ public class GRpcLogAppender extends LogAppender {
   private volatile StreamObserver<AppendEntriesRequestProto> appendLogRequestObserver;
   private StreamObserver<InstallSnapshotRequestProto> snapshotRequestObserver;
 
-  public GRpcLogAppender(RaftServer server, LeaderState leaderState,
-      FollowerInfo f) {
+  public GRpcLogAppender(RaftServerImpl server, LeaderState leaderState,
+                         FollowerInfo f) {
     super(server, leaderState, f);
 
     RaftGRpcService rpcService = (RaftGRpcService) server.getServerRpc();

http://git-wip-us.apache.org/repos/asf/incubator-ratis/blob/22f3ee0b/raft-grpc/src/main/java/org/apache/raft/grpc/server/PipelinedLogAppenderFactory.java
----------------------------------------------------------------------
diff --git a/raft-grpc/src/main/java/org/apache/raft/grpc/server/PipelinedLogAppenderFactory.java b/raft-grpc/src/main/java/org/apache/raft/grpc/server/PipelinedLogAppenderFactory.java
index 5f01980..cc2e513 100644
--- a/raft-grpc/src/main/java/org/apache/raft/grpc/server/PipelinedLogAppenderFactory.java
+++ b/raft-grpc/src/main/java/org/apache/raft/grpc/server/PipelinedLogAppenderFactory.java
@@ -21,12 +21,12 @@ import org.apache.raft.server.impl.FollowerInfo;
 import org.apache.raft.server.impl.LeaderState;
 import org.apache.raft.server.impl.LogAppender;
 import org.apache.raft.server.impl.LogAppenderFactory;
-import org.apache.raft.server.impl.RaftServer;
+import org.apache.raft.server.impl.RaftServerImpl;
 
 public class PipelinedLogAppenderFactory implements LogAppenderFactory {
   @Override
-  public LogAppender getLogAppender(RaftServer server, LeaderState state,
-      FollowerInfo f) {
+  public LogAppender getLogAppender(RaftServerImpl server, LeaderState state,
+                                    FollowerInfo f) {
     return new GRpcLogAppender(server, state, f);
   }
 }

http://git-wip-us.apache.org/repos/asf/incubator-ratis/blob/22f3ee0b/raft-grpc/src/test/java/org/apache/raft/grpc/MiniRaftClusterWithGRpc.java
----------------------------------------------------------------------
diff --git a/raft-grpc/src/test/java/org/apache/raft/grpc/MiniRaftClusterWithGRpc.java b/raft-grpc/src/test/java/org/apache/raft/grpc/MiniRaftClusterWithGRpc.java
index 420ee88..359dabd 100644
--- a/raft-grpc/src/test/java/org/apache/raft/grpc/MiniRaftClusterWithGRpc.java
+++ b/raft-grpc/src/test/java/org/apache/raft/grpc/MiniRaftClusterWithGRpc.java
@@ -28,7 +28,7 @@ import org.apache.raft.protocol.RaftPeer;
 import org.apache.raft.server.impl.BlockRequestHandlingInjection;
 import org.apache.raft.server.impl.DelayLocalExecutionInjection;
 import org.apache.raft.server.impl.LogAppenderFactory;
-import org.apache.raft.server.impl.RaftServer;
+import org.apache.raft.server.impl.RaftServerImpl;
 import org.apache.raft.util.NetUtils;
 
 import java.io.IOException;
@@ -71,10 +71,10 @@ public class MiniRaftClusterWithGRpc extends MiniRaftCluster.RpcBase {
   }
 
   private static Map<RaftPeer, RaftGRpcService> initRpcServices(
-      Collection<RaftServer> servers, RaftProperties prop) throws IOException {
+      Collection<RaftServerImpl> servers, RaftProperties prop) throws IOException {
     final Map<RaftPeer, RaftGRpcService> peerRpcs = new HashMap<>();
 
-    for (RaftServer s : servers) {
+    for (RaftServerImpl s : servers) {
       final RaftGRpcService rpc = new RaftGRpcService(s, prop);
       peerRpcs.put(new RaftPeer(s.getId(), rpc.getInetSocketAddress()), rpc);
     }
@@ -88,11 +88,11 @@ public class MiniRaftClusterWithGRpc extends MiniRaftCluster.RpcBase {
 
   @Override
   protected Collection<RaftPeer> addNewPeers(Collection<RaftPeer> newPeers,
-      Collection<RaftServer> newServers, boolean startService)
+                                             Collection<RaftServerImpl> newServers, boolean startService)
       throws IOException {
     final Map<RaftPeer, RaftGRpcService> peers = initRpcServices(newServers, properties);
     for (Map.Entry<RaftPeer, RaftGRpcService> entry : peers.entrySet()) {
-      RaftServer server = servers.get(entry.getKey().getId());
+      RaftServerImpl server = servers.get(entry.getKey().getId());
       server.setServerRpc(entry.getValue());
       if (!startService) {
         BlockRequestHandlingInjection.getInstance().blockReplier(server.getId());
@@ -104,8 +104,8 @@ public class MiniRaftClusterWithGRpc extends MiniRaftCluster.RpcBase {
   }
 
   @Override
-  protected RaftServer setPeerRpc(RaftPeer peer) throws IOException {
-    RaftServer server = servers.get(peer.getId());
+  protected RaftServerImpl setPeerRpc(RaftPeer peer) throws IOException {
+    RaftServerImpl server = servers.get(peer.getId());
     int port = NetUtils.newInetSocketAddress(peer.getAddress()).getPort();
     int oldPort = properties.getInt(RaftGrpcConfigKeys.RAFT_GRPC_SERVER_PORT_KEY,
         RaftGrpcConfigKeys.RAFT_GRPC_SERVER_PORT_DEFAULT);

http://git-wip-us.apache.org/repos/asf/incubator-ratis/blob/22f3ee0b/raft-grpc/src/test/java/org/apache/raft/grpc/TestRaftStream.java
----------------------------------------------------------------------
diff --git a/raft-grpc/src/test/java/org/apache/raft/grpc/TestRaftStream.java b/raft-grpc/src/test/java/org/apache/raft/grpc/TestRaftStream.java
index 0c85854..82a4e13 100644
--- a/raft-grpc/src/test/java/org/apache/raft/grpc/TestRaftStream.java
+++ b/raft-grpc/src/test/java/org/apache/raft/grpc/TestRaftStream.java
@@ -24,7 +24,7 @@ import org.apache.raft.grpc.client.AppendStreamer;
 import org.apache.raft.grpc.client.RaftOutputStream;
 import org.apache.raft.grpc.server.PipelinedLogAppenderFactory;
 import org.apache.raft.server.impl.LogAppenderFactory;
-import org.apache.raft.server.impl.RaftServer;
+import org.apache.raft.server.impl.RaftServerImpl;
 import org.apache.raft.server.storage.RaftLog;
 import org.apache.raft.shaded.proto.RaftProtos.LogEntryProto;
 import org.apache.raft.util.RaftUtils;
@@ -95,7 +95,7 @@ public class TestRaftStream {
     cluster = new MiniRaftClusterWithGRpc(NUM_SERVERS, prop);
 
     cluster.start();
-    RaftServer leader = waitForLeader(cluster);
+    RaftServerImpl leader = waitForLeader(cluster);
 
     int count = 1;
     try (RaftOutputStream out = new RaftOutputStream(prop, "writer-1",
@@ -137,7 +137,7 @@ public class TestRaftStream {
     cluster = new MiniRaftClusterWithGRpc(NUM_SERVERS, prop);
     cluster.start();
 
-    RaftServer leader = waitForLeader(cluster);
+    RaftServerImpl leader = waitForLeader(cluster);
     RaftOutputStream out = new RaftOutputStream(prop, "writer",
         cluster.getPeers(), leader.getId());
 
@@ -215,7 +215,7 @@ public class TestRaftStream {
 
     cluster = new MiniRaftClusterWithGRpc(NUM_SERVERS, prop);
     cluster.start();
-    RaftServer leader = waitForLeader(cluster);
+    RaftServerImpl leader = waitForLeader(cluster);
 
     RaftOutputStream out = new RaftOutputStream(prop, "writer",
         cluster.getPeers(), leader.getId());
@@ -273,7 +273,7 @@ public class TestRaftStream {
     prop.setInt(RAFT_OUTPUTSTREAM_BUFFER_SIZE_KEY, 4);
     cluster = new MiniRaftClusterWithGRpc(NUM_SERVERS, prop);
     cluster.start();
-    final RaftServer leader = waitForLeader(cluster);
+    final RaftServerImpl leader = waitForLeader(cluster);
 
     final AtomicBoolean running  = new AtomicBoolean(true);
     final AtomicBoolean success = new AtomicBoolean(false);
@@ -301,7 +301,7 @@ public class TestRaftStream {
 
     // force change the leader
     RaftTestUtil.waitAndKillLeader(cluster, true);
-    final RaftServer newLeader = waitForLeader(cluster);
+    final RaftServerImpl newLeader = waitForLeader(cluster);
     Assert.assertNotEquals(leader.getId(), newLeader.getId());
     Thread.sleep(500);
 

http://git-wip-us.apache.org/repos/asf/incubator-ratis/blob/22f3ee0b/raft-grpc/src/test/java/org/apache/raft/grpc/TestRaftWithGrpc.java
----------------------------------------------------------------------
diff --git a/raft-grpc/src/test/java/org/apache/raft/grpc/TestRaftWithGrpc.java b/raft-grpc/src/test/java/org/apache/raft/grpc/TestRaftWithGrpc.java
index b2d104b..c6667b4 100644
--- a/raft-grpc/src/test/java/org/apache/raft/grpc/TestRaftWithGrpc.java
+++ b/raft-grpc/src/test/java/org/apache/raft/grpc/TestRaftWithGrpc.java
@@ -22,7 +22,7 @@ import org.apache.raft.RaftBasicTests;
 import org.apache.raft.grpc.server.PipelinedLogAppenderFactory;
 import org.apache.raft.server.impl.BlockRequestHandlingInjection;
 import org.apache.raft.server.impl.LogAppenderFactory;
-import org.apache.raft.server.impl.RaftServer;
+import org.apache.raft.server.impl.RaftServerImpl;
 import org.apache.raft.util.RaftUtils;
 import org.junit.Assert;
 import org.junit.BeforeClass;
@@ -34,7 +34,7 @@ import static org.apache.raft.server.RaftServerConfigKeys.RAFT_SERVER_LOG_APPEND
 
 public class TestRaftWithGrpc extends RaftBasicTests {
   static {
-    RaftUtils.setLogLevel(RaftServer.LOG, Level.DEBUG);
+    RaftUtils.setLogLevel(RaftServerImpl.LOG, Level.DEBUG);
   }
 
   private final MiniRaftClusterWithGRpc cluster;

http://git-wip-us.apache.org/repos/asf/incubator-ratis/blob/22f3ee0b/raft-hadoop/src/main/java/org/apache/raft/hadooprpc/server/HadoopRpcService.java
----------------------------------------------------------------------
diff --git a/raft-hadoop/src/main/java/org/apache/raft/hadooprpc/server/HadoopRpcService.java b/raft-hadoop/src/main/java/org/apache/raft/hadooprpc/server/HadoopRpcService.java
index ad4beec..eb92372 100644
--- a/raft-hadoop/src/main/java/org/apache/raft/hadooprpc/server/HadoopRpcService.java
+++ b/raft-hadoop/src/main/java/org/apache/raft/hadooprpc/server/HadoopRpcService.java
@@ -26,7 +26,7 @@ import org.apache.raft.hadooprpc.client.RaftClientProtocolPB;
 import org.apache.raft.hadooprpc.client.RaftClientProtocolServerSideTranslatorPB;
 import org.apache.raft.protocol.RaftPeer;
 import org.apache.raft.server.*;
-import org.apache.raft.server.impl.RaftServer;
+import org.apache.raft.server.impl.RaftServerImpl;
 import org.apache.raft.server.impl.RaftServerRpc;
 import org.apache.raft.server.impl.RequestDispatcher;
 import org.apache.raft.shaded.com.google.protobuf.BlockingService;
@@ -56,7 +56,7 @@ public class HadoopRpcService implements RaftServerRpc {
 
   private final PeerProxyMap<Proxy<RaftServerProtocolPB>> proxies;
 
-  public HadoopRpcService(RaftServer server, final Configuration conf)
+  public HadoopRpcService(RaftServerImpl server, final Configuration conf)
       throws IOException {
     this.proxies = new PeerProxyMap<>(
         p -> new Proxy(RaftServerProtocolPB.class, p.getAddress(), conf));

http://git-wip-us.apache.org/repos/asf/incubator-ratis/blob/22f3ee0b/raft-hadoop/src/test/java/org/apache/raft/hadooprpc/MiniRaftClusterWithHadoopRpc.java
----------------------------------------------------------------------
diff --git a/raft-hadoop/src/test/java/org/apache/raft/hadooprpc/MiniRaftClusterWithHadoopRpc.java b/raft-hadoop/src/test/java/org/apache/raft/hadooprpc/MiniRaftClusterWithHadoopRpc.java
index 7f7ef49..87b4d1c 100644
--- a/raft-hadoop/src/test/java/org/apache/raft/hadooprpc/MiniRaftClusterWithHadoopRpc.java
+++ b/raft-hadoop/src/test/java/org/apache/raft/hadooprpc/MiniRaftClusterWithHadoopRpc.java
@@ -27,7 +27,7 @@ import org.apache.raft.hadooprpc.client.HadoopClientRequestSender;
 import org.apache.raft.hadooprpc.server.HadoopRpcService;
 import org.apache.raft.protocol.RaftPeer;
 import org.apache.raft.server.impl.DelayLocalExecutionInjection;
-import org.apache.raft.server.impl.RaftServer;
+import org.apache.raft.server.impl.RaftServerImpl;
 import org.apache.raft.server.RaftServerConfigKeys;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
@@ -70,10 +70,10 @@ public class MiniRaftClusterWithHadoopRpc extends MiniRaftCluster.RpcBase {
   }
 
   private static Map<RaftPeer, HadoopRpcService> initRpcServices(
-      Collection<RaftServer> servers, Configuration hadoopConf) throws IOException {
+      Collection<RaftServerImpl> servers, Configuration hadoopConf) throws IOException {
     final Map<RaftPeer, HadoopRpcService> peerRpcs = new HashMap<>();
 
-    for(RaftServer s : servers) {
+    for(RaftServerImpl s : servers) {
       final HadoopRpcService rpc = new HadoopRpcService(s, hadoopConf);
       peerRpcs.put(new RaftPeer(s.getId(), rpc.getInetSocketAddress()), rpc);
     }
@@ -81,11 +81,11 @@ public class MiniRaftClusterWithHadoopRpc extends MiniRaftCluster.RpcBase {
   }
 
   @Override
-  protected RaftServer setPeerRpc(RaftPeer peer) throws IOException {
+  protected RaftServerImpl setPeerRpc(RaftPeer peer) throws IOException {
     Configuration hconf = new Configuration(hadoopConf);
     hconf.set(RaftServerConfigKeys.Ipc.ADDRESS_KEY, peer.getAddress());
 
-    RaftServer server = servers.get(peer.getId());
+    RaftServerImpl server = servers.get(peer.getId());
     final HadoopRpcService rpc = new HadoopRpcService(server, hconf);
     Preconditions.checkState(
         rpc.getInetSocketAddress().toString().contains(peer.getAddress()),
@@ -97,7 +97,7 @@ public class MiniRaftClusterWithHadoopRpc extends MiniRaftCluster.RpcBase {
 
   @Override
   public Collection<RaftPeer> addNewPeers(Collection<RaftPeer> newPeers,
-      Collection<RaftServer> newServers, boolean startService)
+                                          Collection<RaftServerImpl> newServers, boolean startService)
       throws IOException {
     return addNewPeers(initRpcServices(newServers, hadoopConf),
         newServers, startService);

http://git-wip-us.apache.org/repos/asf/incubator-ratis/blob/22f3ee0b/raft-hadoop/src/test/java/org/apache/raft/hadooprpc/TestRaftWithHadoopRpc.java
----------------------------------------------------------------------
diff --git a/raft-hadoop/src/test/java/org/apache/raft/hadooprpc/TestRaftWithHadoopRpc.java b/raft-hadoop/src/test/java/org/apache/raft/hadooprpc/TestRaftWithHadoopRpc.java
index 3971274..4848196 100644
--- a/raft-hadoop/src/test/java/org/apache/raft/hadooprpc/TestRaftWithHadoopRpc.java
+++ b/raft-hadoop/src/test/java/org/apache/raft/hadooprpc/TestRaftWithHadoopRpc.java
@@ -22,7 +22,7 @@ import org.apache.log4j.Level;
 import org.apache.raft.RaftBasicTests;
 import org.apache.raft.client.RaftClient;
 import org.apache.raft.server.impl.BlockRequestHandlingInjection;
-import org.apache.raft.server.impl.RaftServer;
+import org.apache.raft.server.impl.RaftServerImpl;
 import org.apache.raft.server.RaftServerConfigKeys;
 import org.apache.raft.util.RaftUtils;
 import org.junit.Test;
@@ -33,7 +33,7 @@ import static org.apache.raft.hadooprpc.MiniRaftClusterWithHadoopRpc.sendServerR
 
 public class TestRaftWithHadoopRpc extends RaftBasicTests {
   static {
-    RaftUtils.setLogLevel(RaftServer.LOG, Level.DEBUG);
+    RaftUtils.setLogLevel(RaftServerImpl.LOG, Level.DEBUG);
     RaftUtils.setLogLevel(RaftClient.LOG, Level.DEBUG);
     RaftUtils.setLogLevel(MiniRaftClusterWithHadoopRpc.LOG, Level.DEBUG);
   }

http://git-wip-us.apache.org/repos/asf/incubator-ratis/blob/22f3ee0b/raft-netty/src/main/java/org/apache/raft/netty/server/NettyRpcService.java
----------------------------------------------------------------------
diff --git a/raft-netty/src/main/java/org/apache/raft/netty/server/NettyRpcService.java b/raft-netty/src/main/java/org/apache/raft/netty/server/NettyRpcService.java
index c0d751d..b5b8550 100644
--- a/raft-netty/src/main/java/org/apache/raft/netty/server/NettyRpcService.java
+++ b/raft-netty/src/main/java/org/apache/raft/netty/server/NettyRpcService.java
@@ -29,7 +29,7 @@ import org.apache.raft.client.impl.ClientProtoUtils;
 import org.apache.raft.netty.NettyRpcProxy;
 import org.apache.raft.protocol.RaftClientReply;
 import org.apache.raft.protocol.RaftPeer;
-import org.apache.raft.server.impl.RaftServer;
+import org.apache.raft.server.impl.RaftServerImpl;
 import org.apache.raft.server.impl.RaftServerRpc;
 import org.apache.raft.server.impl.RequestDispatcher;
 import org.apache.raft.shaded.io.netty.handler.codec.protobuf.ProtobufDecoder;
@@ -75,7 +75,7 @@ public final class NettyRpcService implements RaftServerRpc {
   }
 
   /** Constructs a netty server with the given port. */
-  public NettyRpcService(int port, RaftServer server) {
+  public NettyRpcService(int port, RaftServerImpl server) {
     this.raftService = new RequestDispatcher(server);
     this.id = server.getId();
 

http://git-wip-us.apache.org/repos/asf/incubator-ratis/blob/22f3ee0b/raft-netty/src/test/java/org/apache/raft/netty/MiniRaftClusterWithNetty.java
----------------------------------------------------------------------
diff --git a/raft-netty/src/test/java/org/apache/raft/netty/MiniRaftClusterWithNetty.java b/raft-netty/src/test/java/org/apache/raft/netty/MiniRaftClusterWithNetty.java
index 4958e9a..12cdf13 100644
--- a/raft-netty/src/test/java/org/apache/raft/netty/MiniRaftClusterWithNetty.java
+++ b/raft-netty/src/test/java/org/apache/raft/netty/MiniRaftClusterWithNetty.java
@@ -26,7 +26,7 @@ import org.apache.raft.netty.server.NettyRpcService;
 import org.apache.raft.protocol.RaftPeer;
 import org.apache.raft.server.impl.DelayLocalExecutionInjection;
 import org.apache.raft.server.impl.RaftConfiguration;
-import org.apache.raft.server.impl.RaftServer;
+import org.apache.raft.server.impl.RaftServerImpl;
 import org.apache.raft.util.NetUtils;
 
 import java.io.IOException;
@@ -69,17 +69,17 @@ public class MiniRaftClusterWithNetty extends MiniRaftCluster.RpcBase {
   }
 
   private static NettyRpcService newNettyRpcService(
-      RaftServer s, RaftConfiguration conf) {
+      RaftServerImpl s, RaftConfiguration conf) {
     final String address = getAddress(s.getId(), conf);
     final int port = NetUtils.newInetSocketAddress(address).getPort();
     return new NettyRpcService(port, s);
   }
 
   private static Map<RaftPeer, NettyRpcService> initRpcServices(
-      Collection<RaftServer> servers, RaftConfiguration conf) {
+      Collection<RaftServerImpl> servers, RaftConfiguration conf) {
     final Map<RaftPeer, NettyRpcService> peerRpcs = new HashMap<>();
 
-    for (RaftServer s : servers) {
+    for (RaftServerImpl s : servers) {
       final NettyRpcService rpc = newNettyRpcService(s, conf);
       peerRpcs.put(new RaftPeer(s.getId(), rpc.getInetSocketAddress()), rpc);
     }
@@ -88,8 +88,8 @@ public class MiniRaftClusterWithNetty extends MiniRaftCluster.RpcBase {
   }
 
   @Override
-  protected RaftServer setPeerRpc(RaftPeer peer) throws IOException {
-    final RaftServer s = servers.get(peer.getId());
+  protected RaftServerImpl setPeerRpc(RaftPeer peer) throws IOException {
+    final RaftServerImpl s = servers.get(peer.getId());
     final NettyRpcService rpc = newNettyRpcService(s, conf);
     s.setServerRpc(rpc);
     return s;
@@ -97,7 +97,7 @@ public class MiniRaftClusterWithNetty extends MiniRaftCluster.RpcBase {
 
   @Override
   protected Collection<RaftPeer> addNewPeers(
-      Collection<RaftPeer> newPeers, Collection<RaftServer> newServers,
+      Collection<RaftPeer> newPeers, Collection<RaftServerImpl> newServers,
       boolean startService) throws IOException {
     return addNewPeers(initRpcServices(newServers, conf),
         newServers, startService);

http://git-wip-us.apache.org/repos/asf/incubator-ratis/blob/22f3ee0b/raft-netty/src/test/java/org/apache/raft/netty/TestRaftWithNetty.java
----------------------------------------------------------------------
diff --git a/raft-netty/src/test/java/org/apache/raft/netty/TestRaftWithNetty.java b/raft-netty/src/test/java/org/apache/raft/netty/TestRaftWithNetty.java
index fb75b7b..cba991d 100644
--- a/raft-netty/src/test/java/org/apache/raft/netty/TestRaftWithNetty.java
+++ b/raft-netty/src/test/java/org/apache/raft/netty/TestRaftWithNetty.java
@@ -21,7 +21,7 @@ import org.apache.log4j.Level;
 import org.apache.raft.RaftBasicTests;
 import org.apache.raft.client.RaftClient;
 import org.apache.raft.server.impl.BlockRequestHandlingInjection;
-import org.apache.raft.server.impl.RaftServer;
+import org.apache.raft.server.impl.RaftServerImpl;
 import org.apache.raft.util.RaftUtils;
 import org.junit.Test;
 
@@ -29,7 +29,7 @@ import java.io.IOException;
 
 public class TestRaftWithNetty extends RaftBasicTests {
   static {
-    RaftUtils.setLogLevel(RaftServer.LOG, Level.DEBUG);
+    RaftUtils.setLogLevel(RaftServerImpl.LOG, Level.DEBUG);
     RaftUtils.setLogLevel(RaftClient.LOG, Level.DEBUG);
   }
 

http://git-wip-us.apache.org/repos/asf/incubator-ratis/blob/22f3ee0b/raft-server/src/main/java/org/apache/raft/server/impl/FollowerState.java
----------------------------------------------------------------------
diff --git a/raft-server/src/main/java/org/apache/raft/server/impl/FollowerState.java b/raft-server/src/main/java/org/apache/raft/server/impl/FollowerState.java
index 61b3c92..20f2d8f 100644
--- a/raft-server/src/main/java/org/apache/raft/server/impl/FollowerState.java
+++ b/raft-server/src/main/java/org/apache/raft/server/impl/FollowerState.java
@@ -25,15 +25,15 @@ import org.slf4j.Logger;
  * Used when the peer is a follower. Used to track the election timeout.
  */
 class FollowerState extends Daemon {
-  static final Logger LOG = RaftServer.LOG;
+  static final Logger LOG = RaftServerImpl.LOG;
 
-  private final RaftServer server;
+  private final RaftServerImpl server;
 
   private volatile Timestamp lastRpcTime = new Timestamp();
   private volatile boolean monitorRunning = true;
   private volatile boolean inLogSync = false;
 
-  FollowerState(RaftServer server) {
+  FollowerState(RaftServerImpl server) {
     this.server = server;
   }
 

http://git-wip-us.apache.org/repos/asf/incubator-ratis/blob/22f3ee0b/raft-server/src/main/java/org/apache/raft/server/impl/LeaderElection.java
----------------------------------------------------------------------
diff --git a/raft-server/src/main/java/org/apache/raft/server/impl/LeaderElection.java b/raft-server/src/main/java/org/apache/raft/server/impl/LeaderElection.java
index 10e901f..8552029 100644
--- a/raft-server/src/main/java/org/apache/raft/server/impl/LeaderElection.java
+++ b/raft-server/src/main/java/org/apache/raft/server/impl/LeaderElection.java
@@ -67,7 +67,7 @@ class LeaderElection extends Daemon {
     }
   }
 
-  private final RaftServer server;
+  private final RaftServerImpl server;
   private ExecutorCompletionService<RequestVoteReplyProto> service;
   private ExecutorService executor;
   private volatile boolean running;
@@ -79,7 +79,7 @@ class LeaderElection extends Daemon {
   private final RaftConfiguration conf;
   private final Collection<RaftPeer> others;
 
-  LeaderElection(RaftServer server) {
+  LeaderElection(RaftServerImpl server) {
     this.server = server;
     conf = server.getRaftConf();
     others = conf.getOtherPeers(server.getId());

http://git-wip-us.apache.org/repos/asf/incubator-ratis/blob/22f3ee0b/raft-server/src/main/java/org/apache/raft/server/impl/LeaderState.java
----------------------------------------------------------------------
diff --git a/raft-server/src/main/java/org/apache/raft/server/impl/LeaderState.java b/raft-server/src/main/java/org/apache/raft/server/impl/LeaderState.java
index 79d0d9e..39dc400 100644
--- a/raft-server/src/main/java/org/apache/raft/server/impl/LeaderState.java
+++ b/raft-server/src/main/java/org/apache/raft/server/impl/LeaderState.java
@@ -49,7 +49,7 @@ import static org.apache.raft.server.impl.LeaderState.StateUpdateEventType.*;
  *                           corresponding log entries are committed
  */
 public class LeaderState {
-  private static final Logger LOG = RaftServer.LOG;
+  private static final Logger LOG = RaftServerImpl.LOG;
 
   enum StateUpdateEventType {
     STEPDOWN, UPDATECOMMIT, STAGINGPROGRESS
@@ -74,7 +74,7 @@ public class LeaderState {
   static final StateUpdateEvent STAGING_PROGRESS_EVENT =
       new StateUpdateEvent(StateUpdateEventType.STAGINGPROGRESS, -1);
 
-  private final RaftServer server;
+  private final RaftServerImpl server;
   private final RaftLog raftLog;
   private final long currentTerm;
   private volatile ConfigurationStagingState stagingState;
@@ -94,7 +94,7 @@ public class LeaderState {
   private final int snapshotChunkMaxSize;
   private final int syncInterval;
 
-  LeaderState(RaftServer server, RaftProperties properties) {
+  LeaderState(RaftServerImpl server, RaftProperties properties) {
     this.server = server;
 
     stagingCatchupGap = properties.getInt(

http://git-wip-us.apache.org/repos/asf/incubator-ratis/blob/22f3ee0b/raft-server/src/main/java/org/apache/raft/server/impl/LogAppender.java
----------------------------------------------------------------------
diff --git a/raft-server/src/main/java/org/apache/raft/server/impl/LogAppender.java b/raft-server/src/main/java/org/apache/raft/server/impl/LogAppender.java
index 143da28..3b18f13 100644
--- a/raft-server/src/main/java/org/apache/raft/server/impl/LogAppender.java
+++ b/raft-server/src/main/java/org/apache/raft/server/impl/LogAppender.java
@@ -46,9 +46,9 @@ import static org.apache.raft.server.impl.RaftServerConstants.INVALID_LOG_INDEX;
  * A daemon thread appending log entries to a follower peer.
  */
 public class LogAppender extends Daemon {
-  public static final Logger LOG = RaftServer.LOG;
+  public static final Logger LOG = RaftServerImpl.LOG;
 
-  protected final RaftServer server;
+  protected final RaftServerImpl server;
   private final LeaderState leaderState;
   protected final RaftLog raftLog;
   protected final FollowerInfo follower;
@@ -59,7 +59,7 @@ public class LogAppender extends Daemon {
 
   private volatile boolean sending = true;
 
-  public LogAppender(RaftServer server, LeaderState leaderState, FollowerInfo f) {
+  public LogAppender(RaftServerImpl server, LeaderState leaderState, FollowerInfo f) {
     this.follower = f;
     this.server = server;
     this.leaderState = leaderState;

http://git-wip-us.apache.org/repos/asf/incubator-ratis/blob/22f3ee0b/raft-server/src/main/java/org/apache/raft/server/impl/LogAppenderFactory.java
----------------------------------------------------------------------
diff --git a/raft-server/src/main/java/org/apache/raft/server/impl/LogAppenderFactory.java b/raft-server/src/main/java/org/apache/raft/server/impl/LogAppenderFactory.java
index b5ed775..d77faff 100644
--- a/raft-server/src/main/java/org/apache/raft/server/impl/LogAppenderFactory.java
+++ b/raft-server/src/main/java/org/apache/raft/server/impl/LogAppenderFactory.java
@@ -18,13 +18,13 @@
 package org.apache.raft.server.impl;
 
 public interface LogAppenderFactory {
-  LogAppender getLogAppender(RaftServer server, LeaderState state,
-      FollowerInfo f);
+  LogAppender getLogAppender(RaftServerImpl server, LeaderState state,
+                             FollowerInfo f);
 
   class SynchronousLogAppenderFactory implements LogAppenderFactory {
     @Override
-    public LogAppender getLogAppender(RaftServer server, LeaderState state,
-        FollowerInfo f) {
+    public LogAppender getLogAppender(RaftServerImpl server, LeaderState state,
+                                      FollowerInfo f) {
       return new LogAppender(server, state, f);
     }
   }

http://git-wip-us.apache.org/repos/asf/incubator-ratis/blob/22f3ee0b/raft-server/src/main/java/org/apache/raft/server/impl/PendingRequests.java
----------------------------------------------------------------------
diff --git a/raft-server/src/main/java/org/apache/raft/server/impl/PendingRequests.java b/raft-server/src/main/java/org/apache/raft/server/impl/PendingRequests.java
index ad6ecef..32f127e 100644
--- a/raft-server/src/main/java/org/apache/raft/server/impl/PendingRequests.java
+++ b/raft-server/src/main/java/org/apache/raft/server/impl/PendingRequests.java
@@ -30,14 +30,14 @@ import java.util.concurrent.ConcurrentMap;
 import java.util.stream.Collectors;
 
 class PendingRequests {
-  private static final Logger LOG = RaftServer.LOG;
+  private static final Logger LOG = RaftServerImpl.LOG;
 
   private PendingRequest pendingSetConf;
-  private final RaftServer server;
+  private final RaftServerImpl server;
   private final ConcurrentMap<Long, PendingRequest> pendingRequests = new ConcurrentHashMap<>();
   private PendingRequest last = null;
 
-  PendingRequests(RaftServer server) {
+  PendingRequests(RaftServerImpl server) {
     this.server = server;
   }
 

http://git-wip-us.apache.org/repos/asf/incubator-ratis/blob/22f3ee0b/raft-server/src/main/java/org/apache/raft/server/impl/RaftServer.java
----------------------------------------------------------------------
diff --git a/raft-server/src/main/java/org/apache/raft/server/impl/RaftServer.java b/raft-server/src/main/java/org/apache/raft/server/impl/RaftServer.java
deleted file mode 100644
index c1bf4a9..0000000
--- a/raft-server/src/main/java/org/apache/raft/server/impl/RaftServer.java
+++ /dev/null
@@ -1,749 +0,0 @@
-/**
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- *     http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-package org.apache.raft.server.impl;
-
-import com.google.common.annotations.VisibleForTesting;
-import com.google.common.base.Preconditions;
-import org.apache.raft.conf.RaftProperties;
-import org.apache.raft.protocol.*;
-import org.apache.raft.server.RaftServerConfigKeys;
-import org.apache.raft.server.protocol.RaftServerProtocol;
-import org.apache.raft.server.protocol.TermIndex;
-import org.apache.raft.server.storage.FileInfo;
-import org.apache.raft.shaded.proto.RaftProtos.*;
-import org.apache.raft.statemachine.SnapshotInfo;
-import org.apache.raft.statemachine.StateMachine;
-import org.apache.raft.statemachine.TransactionContext;
-import org.apache.raft.util.CodeInjectionForTesting;
-import org.apache.raft.util.LifeCycle;
-import org.apache.raft.util.ProtoUtils;
-import org.apache.raft.util.RaftUtils;
-import org.slf4j.Logger;
-import org.slf4j.LoggerFactory;
-
-import java.io.Closeable;
-import java.io.IOException;
-import java.io.InterruptedIOException;
-import java.util.Arrays;
-import java.util.Collection;
-import java.util.List;
-import java.util.OptionalLong;
-import java.util.concurrent.CompletableFuture;
-
-import static org.apache.raft.shaded.proto.RaftProtos.AppendEntriesReplyProto.AppendResult.*;
-import static org.apache.raft.util.LifeCycle.State.*;
-
-public class RaftServer implements RaftServerProtocol, Closeable {
-  public static final Logger LOG = LoggerFactory.getLogger(RaftServer.class);
-
-  private static final String CLASS_NAME = RaftServer.class.getSimpleName();
-  static final String REQUEST_VOTE = CLASS_NAME + ".requestVote";
-  static final String APPEND_ENTRIES = CLASS_NAME + ".appendEntries";
-  static final String INSTALL_SNAPSHOT = CLASS_NAME + ".installSnapshot";
-
-
-  private final int minTimeoutMs;
-  private final int maxTimeoutMs;
-
-  private final LifeCycle lifeCycle;
-  private final ServerState state;
-  private final StateMachine stateMachine;
-  private final RaftProperties properties;
-  private volatile Role role;
-
-  /** used when the peer is follower, to monitor election timeout */
-  private volatile FollowerState heartbeatMonitor;
-
-  /** used when the peer is candidate, to request votes from other peers */
-  private volatile LeaderElection electionDaemon;
-
-  /** used when the peer is leader */
-  private volatile LeaderState leaderState;
-
-  private RaftServerRpc serverRpc;
-
-  private final LogAppenderFactory appenderFactory;
-
-  public RaftServer(String id, RaftConfiguration raftConf,
-      RaftProperties properties, StateMachine stateMachine) throws IOException {
-    this.lifeCycle = new LifeCycle(id);
-    minTimeoutMs = properties.getInt(
-        RaftServerConfigKeys.RAFT_SERVER_RPC_TIMEOUT_MIN_MS_KEY,
-        RaftServerConfigKeys.RAFT_SERVER_RPC_TIMEOUT_MIN_MS_DEFAULT);
-    maxTimeoutMs = properties.getInt(
-        RaftServerConfigKeys.RAFT_SERVER_RPC_TIMEOUT_MAX_MS_KEY,
-        RaftServerConfigKeys.RAFT_SERVER_RPC_TIMEOUT_MAX_MS_DEFAULT);
-    Preconditions.checkArgument(maxTimeoutMs > minTimeoutMs,
-        "max timeout: %s, min timeout: %s", maxTimeoutMs, minTimeoutMs);
-    this.properties = properties;
-    this.stateMachine = stateMachine;
-    this.state = new ServerState(id, raftConf, properties, this, stateMachine);
-    appenderFactory = initAppenderFactory();
-  }
-
-  public int getMinTimeoutMs() {
-    return minTimeoutMs;
-  }
-
-  public int getMaxTimeoutMs() {
-    return maxTimeoutMs;
-  }
-
-  public int getRandomTimeoutMs() {
-    return RaftUtils.getRandomBetween(minTimeoutMs, maxTimeoutMs);
-  }
-
-  public StateMachine getStateMachine() {
-    return this.stateMachine;
-  }
-
-  public LogAppenderFactory getLogAppenderFactory() {
-    return appenderFactory;
-  }
-
-  private LogAppenderFactory initAppenderFactory() {
-    Class<? extends LogAppenderFactory> factoryClass = properties.getClass(
-        RaftServerConfigKeys.RAFT_SERVER_LOG_APPENDER_FACTORY_CLASS_KEY,
-        RaftServerConfigKeys.RAFT_SERVER_LOG_APPENDER_FACTORY_CLASS_DEFAULT,
-        LogAppenderFactory.class);
-    return RaftUtils.newInstance(factoryClass);
-  }
-
-  /**
-   * Used by tests to set initial raft configuration with correct port bindings.
-   */
-  @VisibleForTesting
-  public void setInitialConf(RaftConfiguration conf) {
-    this.state.setInitialConf(conf);
-  }
-
-  public void setServerRpc(RaftServerRpc serverRpc) {
-    this.serverRpc = serverRpc;
-    // add peers into rpc service
-    RaftConfiguration conf = getRaftConf();
-    if (conf != null) {
-      addPeersToRPC(conf.getPeers());
-    }
-  }
-
-  public RaftServerRpc getServerRpc() {
-    return serverRpc;
-  }
-
-  public void start() {
-    lifeCycle.transition(STARTING);
-    state.start();
-    RaftConfiguration conf = getRaftConf();
-    if (conf != null && conf.contains(getId())) {
-      LOG.debug("{} starts as a follower", getId());
-      startAsFollower();
-    } else {
-      LOG.debug("{} starts with initializing state", getId());
-      startInitializing();
-    }
-  }
-
-  /**
-   * The peer belongs to the current configuration, should start as a follower
-   */
-  private void startAsFollower() {
-    role = Role.FOLLOWER;
-    heartbeatMonitor = new FollowerState(this);
-    heartbeatMonitor.start();
-
-    serverRpc.start();
-    lifeCycle.transition(RUNNING);
-  }
-
-  /**
-   * The peer does not have any configuration (maybe it will later be included
-   * in some configuration). Start still as a follower but will not vote or
-   * start election.
-   */
-  private void startInitializing() {
-    role = Role.FOLLOWER;
-    // do not start heartbeatMonitoring
-    serverRpc.start();
-  }
-
-  public ServerState getState() {
-    return this.state;
-  }
-
-  public String getId() {
-    return getState().getSelfId();
-  }
-
-  public RaftConfiguration getRaftConf() {
-    return getState().getRaftConf();
-  }
-
-  @Override
-  public void close() {
-    lifeCycle.checkStateAndClose(() -> {
-      try {
-        shutdownHeartbeatMonitor();
-        shutdownElectionDaemon();
-        shutdownLeaderState();
-
-        serverRpc.shutdown();
-        state.close();
-      } catch (Exception ignored) {
-        LOG.warn("Failed to kill " + state.getSelfId(), ignored);
-      }
-    });
-  }
-
-  public boolean isAlive() {
-    return !lifeCycle.getCurrentState().isOneOf(CLOSING, CLOSED);
-  }
-
-  public boolean isFollower() {
-    return role == Role.FOLLOWER;
-  }
-
-  public boolean isCandidate() {
-    return role == Role.CANDIDATE;
-  }
-
-  public boolean isLeader() {
-    return role == Role.LEADER;
-  }
-
-  Role getRole() {
-    return role;
-  }
-
-  /**
-   * Change the server state to Follower if necessary
-   * @param newTerm The new term.
-   * @param sync We will call {@link ServerState#persistMetadata()} if this is
-   *             set to true and term/votedFor get updated.
-   * @return if the term/votedFor should be updated to the new term
-   * @throws IOException if term/votedFor persistence failed.
-   */
-  synchronized boolean changeToFollower(long newTerm, boolean sync)
-      throws IOException {
-    final Role old = role;
-    role = Role.FOLLOWER;
-
-    boolean metadataUpdated = false;
-    if (newTerm > state.getCurrentTerm()) {
-      state.setCurrentTerm(newTerm);
-      state.resetLeaderAndVotedFor();
-      metadataUpdated = true;
-    }
-
-    if (old == Role.LEADER) {
-      assert leaderState != null;
-      shutdownLeaderState();
-    } else if (old == Role.CANDIDATE) {
-      shutdownElectionDaemon();
-    }
-
-    if (old != Role.FOLLOWER) {
-      heartbeatMonitor = new FollowerState(this);
-      heartbeatMonitor.start();
-    }
-
-    if (metadataUpdated && sync) {
-      state.persistMetadata();
-    }
-    return metadataUpdated;
-  }
-
-  private synchronized void shutdownLeaderState() {
-    final LeaderState leader = leaderState;
-    if (leader != null) {
-      leader.stop();
-    }
-    leaderState = null;
-    // TODO: make sure that StateMachineUpdater has applied all transactions that have context
-  }
-
-  private void shutdownElectionDaemon() {
-    final LeaderElection election = electionDaemon;
-    if (election != null) {
-      election.stopRunning();
-      // no need to interrupt the election thread
-    }
-    electionDaemon = null;
-  }
-
-  synchronized void changeToLeader() {
-    Preconditions.checkState(isCandidate());
-    shutdownElectionDaemon();
-    role = Role.LEADER;
-    state.becomeLeader();
-    // start sending AppendEntries RPC to followers
-    leaderState = new LeaderState(this, properties);
-    leaderState.start();
-  }
-
-  private void shutdownHeartbeatMonitor() {
-    final FollowerState hm = heartbeatMonitor;
-    if (hm != null) {
-      hm.stopRunning();
-      hm.interrupt();
-    }
-    heartbeatMonitor = null;
-  }
-
-  synchronized void changeToCandidate() {
-    Preconditions.checkState(isFollower());
-    shutdownHeartbeatMonitor();
-    role = Role.CANDIDATE;
-    // start election
-    electionDaemon = new LeaderElection(this);
-    electionDaemon.start();
-  }
-
-  @Override
-  public String toString() {
-    return role + " " + state + " " + lifeCycle.getCurrentState();
-  }
-
-  /**
-   * @return null if the server is in leader state.
-   */
-  CompletableFuture<RaftClientReply> checkLeaderState(
-      RaftClientRequest request) {
-    if (!isLeader()) {
-      NotLeaderException exception = generateNotLeaderException();
-      CompletableFuture<RaftClientReply> future = new CompletableFuture<>();
-      future.complete(new RaftClientReply(request, exception));
-      return future;
-    }
-    return null;
-  }
-
-  NotLeaderException generateNotLeaderException() {
-    if (lifeCycle.getCurrentState() != RUNNING) {
-      return new NotLeaderException(getId(), null, null);
-    }
-    String leaderId = state.getLeaderId();
-    if (leaderId == null || leaderId.equals(state.getSelfId())) {
-      // No idea about who is the current leader. Or the peer is the current
-      // leader, but it is about to step down
-      RaftPeer suggestedLeader = state.getRaftConf()
-          .getRandomPeer(state.getSelfId());
-      leaderId = suggestedLeader == null ? null : suggestedLeader.getId();
-    }
-    RaftConfiguration conf = getRaftConf();
-    Collection<RaftPeer> peers = conf.getPeers();
-    return new NotLeaderException(getId(), conf.getPeer(leaderId),
-        peers.toArray(new RaftPeer[peers.size()]));
-  }
-
-  /**
-   * Handle a normal update request from client.
-   */
-  public CompletableFuture<RaftClientReply> appendTransaction(
-      RaftClientRequest request, TransactionContext entry)
-      throws RaftException {
-    LOG.debug("{}: receive client request({})", getId(), request);
-    lifeCycle.assertCurrentState(RUNNING);
-    CompletableFuture<RaftClientReply> reply;
-
-    final PendingRequest pending;
-    synchronized (this) {
-      reply = checkLeaderState(request);
-      if (reply != null) {
-        return reply;
-      }
-
-      // append the message to its local log
-      final long entryIndex;
-      try {
-        entryIndex = state.applyLog(entry);
-      } catch (IOException e) {
-        throw new RaftException(e);
-      }
-
-      // put the request into the pending queue
-      pending = leaderState.addPendingRequest(entryIndex, request, entry);
-      leaderState.notifySenders();
-    }
-    return pending.getFuture();
-  }
-
-  /**
-   * Handle a raft configuration change request from client.
-   */
-  public CompletableFuture<RaftClientReply> setConfiguration(
-      SetConfigurationRequest request) throws IOException {
-    LOG.debug("{}: receive setConfiguration({})", getId(), request);
-    lifeCycle.assertCurrentState(RUNNING);
-    CompletableFuture<RaftClientReply> reply = checkLeaderState(request);
-    if (reply != null) {
-      return reply;
-    }
-
-    final RaftPeer[] peersInNewConf = request.getPeersInNewConf();
-    final PendingRequest pending;
-    synchronized (this) {
-      reply = checkLeaderState(request);
-      if (reply != null) {
-        return reply;
-      }
-
-      final RaftConfiguration current = getRaftConf();
-      // make sure there is no other raft reconfiguration in progress
-      if (!current.isStable() || leaderState.inStagingState() ||
-          !state.isCurrentConfCommitted()) {
-        throw new ReconfigurationInProgressException(
-            "Reconfiguration is already in progress: " + current);
-      }
-
-      // return true if the new configuration is the same with the current one
-      if (current.hasNoChange(peersInNewConf)) {
-        pending = leaderState.returnNoConfChange(request);
-        return pending.getFuture();
-      }
-
-      // add new peers into the rpc service
-      addPeersToRPC(Arrays.asList(peersInNewConf));
-      // add staging state into the leaderState
-      pending = leaderState.startSetConfiguration(request);
-    }
-    return pending.getFuture();
-  }
-
-  private boolean shouldWithholdVotes() {
-    return isLeader() || (isFollower() && state.hasLeader()
-        && heartbeatMonitor.shouldWithholdVotes());
-  }
-
-  /**
-   * check if the remote peer is not included in the current conf
-   * and should shutdown. should shutdown if all the following stands:
-   * 1. this is a leader
-   * 2. current conf is stable and has been committed
-   * 3. candidate id is not included in conf
-   * 4. candidate's last entry's index < conf's index
-   */
-  private boolean shouldSendShutdown(String candidateId,
-      TermIndex candidateLastEntry) {
-    return isLeader()
-        && getRaftConf().isStable()
-        && getState().isConfCommitted()
-        && !getRaftConf().containsInConf(candidateId)
-        && candidateLastEntry.getIndex() < getRaftConf().getLogEntryIndex()
-        && !leaderState.isBootStrappingPeer(candidateId);
-  }
-
-  @Override
-  public RequestVoteReplyProto requestVote(RequestVoteRequestProto r)
-      throws IOException {
-    final String candidateId = r.getServerRequest().getRequestorId();
-    return requestVote(candidateId, r.getCandidateTerm(),
-        ServerProtoUtils.toTermIndex(r.getCandidateLastEntry()));
-  }
-
-  private RequestVoteReplyProto requestVote(String candidateId,
-      long candidateTerm, TermIndex candidateLastEntry) throws IOException {
-    CodeInjectionForTesting.execute(REQUEST_VOTE, getId(),
-        candidateId, candidateTerm, candidateLastEntry);
-    LOG.debug("{}: receive requestVote({}, {}, {})",
-        getId(), candidateId, candidateTerm, candidateLastEntry);
-    lifeCycle.assertCurrentState(RUNNING);
-
-    boolean voteGranted = false;
-    boolean shouldShutdown = false;
-    final RequestVoteReplyProto reply;
-    synchronized (this) {
-      if (shouldWithholdVotes()) {
-        LOG.info("{} Withhold vote from server {} with term {}. " +
-            "This server:{}, last rpc time from leader {} is {}", getId(),
-            candidateId, candidateTerm, this, this.getState().getLeaderId(),
-            (isFollower() ? heartbeatMonitor.getLastRpcTime() : -1));
-      } else if (state.recognizeCandidate(candidateId, candidateTerm)) {
-        boolean termUpdated = changeToFollower(candidateTerm, false);
-        // see Section 5.4.1 Election restriction
-        if (state.isLogUpToDate(candidateLastEntry)) {
-          heartbeatMonitor.updateLastRpcTime(false);
-          state.grantVote(candidateId);
-          voteGranted = true;
-        }
-        if (termUpdated || voteGranted) {
-          state.persistMetadata(); // sync metafile
-        }
-      }
-      if (!voteGranted && shouldSendShutdown(candidateId, candidateLastEntry)) {
-        shouldShutdown = true;
-      }
-      reply = ServerProtoUtils.toRequestVoteReplyProto(candidateId, getId(),
-          voteGranted, state.getCurrentTerm(), shouldShutdown);
-      if (LOG.isDebugEnabled()) {
-        LOG.debug("{} replies to vote request: {}. Peer's state: {}",
-            getId(), ProtoUtils.toString(reply), state);
-      }
-    }
-    return reply;
-  }
-
-  private void validateEntries(long expectedTerm, TermIndex previous,
-      LogEntryProto... entries) {
-    if (entries != null && entries.length > 0) {
-      final long index0 = entries[0].getIndex();
-
-      if (previous == null || previous.getTerm() == 0) {
-        Preconditions.checkArgument(index0 == 0,
-            "Unexpected Index: previous is null but entries[%s].getIndex()=%s",
-            0, index0);
-      } else {
-        Preconditions.checkArgument(previous.getIndex() == index0 - 1,
-            "Unexpected Index: previous is %s but entries[%s].getIndex()=%s",
-            previous, 0, index0);
-      }
-
-      for (int i = 0; i < entries.length; i++) {
-        final long t = entries[i].getTerm();
-        Preconditions.checkArgument(expectedTerm >= t,
-            "Unexpected Term: entries[%s].getTerm()=%s but expectedTerm=%s",
-            i, t, expectedTerm);
-
-        final long indexi = entries[i].getIndex();
-        Preconditions.checkArgument(indexi == index0 + i,
-            "Unexpected Index: entries[%s].getIndex()=%s but entries[0].getIndex()=%s",
-            i, indexi, index0);
-      }
-    }
-  }
-
-  @Override
-  public AppendEntriesReplyProto appendEntries(AppendEntriesRequestProto r)
-      throws IOException {
-    // TODO avoid converting list to array
-    final LogEntryProto[] entries = r.getEntriesList()
-        .toArray(new LogEntryProto[r.getEntriesCount()]);
-    final TermIndex previous = r.hasPreviousLog() ?
-        ServerProtoUtils.toTermIndex(r.getPreviousLog()) : null;
-    return appendEntries(r.getServerRequest().getRequestorId(),
-        r.getLeaderTerm(), previous, r.getLeaderCommit(), r.getInitializing(),
-        entries);
-  }
-
-  private AppendEntriesReplyProto appendEntries(String leaderId, long leaderTerm,
-      TermIndex previous, long leaderCommit, boolean initializing,
-      LogEntryProto... entries) throws IOException {
-    CodeInjectionForTesting.execute(APPEND_ENTRIES, getId(),
-        leaderId, leaderTerm, previous, leaderCommit, initializing, entries);
-    if (LOG.isDebugEnabled()) {
-      LOG.debug("{}: receive appendEntries({}, {}, {}, {}, {}, {})", getId(),
-          leaderId, leaderTerm, previous, leaderCommit, initializing,
-          ServerProtoUtils.toString(entries));
-    }
-    lifeCycle.assertCurrentState(STARTING, RUNNING);
-
-    try {
-      validateEntries(leaderTerm, previous, entries);
-    } catch (IllegalArgumentException e) {
-      throw new IOException(e);
-    }
-
-    final long currentTerm;
-    long nextIndex = state.getLog().getNextIndex();
-    synchronized (this) {
-      final boolean recognized = state.recognizeLeader(leaderId, leaderTerm);
-      currentTerm = state.getCurrentTerm();
-      if (!recognized) {
-        final AppendEntriesReplyProto reply = ServerProtoUtils.toAppendEntriesReplyProto(
-            leaderId, getId(), currentTerm, nextIndex, NOT_LEADER);
-        if (LOG.isDebugEnabled()) {
-          LOG.debug("{}: do not recognize leader. Reply: {}",
-              getId(), ProtoUtils.toString(reply));
-        }
-        return reply;
-      }
-      changeToFollower(leaderTerm, true);
-      state.setLeader(leaderId);
-
-      if (!initializing && lifeCycle.compareAndTransition(STARTING, RUNNING)) {
-        heartbeatMonitor = new FollowerState(this);
-        heartbeatMonitor.start();
-      }
-      if (lifeCycle.getCurrentState() == RUNNING) {
-        heartbeatMonitor.updateLastRpcTime(true);
-      }
-
-      // We need to check if "previous" is in the local peer. Note that it is
-      // possible that "previous" is covered by the latest snapshot: e.g.,
-      // it's possible there's no log entries outside of the latest snapshot.
-      // However, it is not possible that "previous" index is smaller than the
-      // last index included in snapshot. This is because indices <= snapshot's
-      // last index should have been committed.
-      if (previous != null && !containPrevious(previous)) {
-        final AppendEntriesReplyProto reply =
-            ServerProtoUtils.toAppendEntriesReplyProto(leaderId, getId(),
-                currentTerm, Math.min(nextIndex, previous.getIndex()), INCONSISTENCY);
-        LOG.debug("{}: inconsistency entries. Leader previous:{}, Reply:{}",
-            getId(), previous, ServerProtoUtils.toString(reply));
-        return reply;
-      }
-
-      state.getLog().append(entries);
-      state.updateConfiguration(entries);
-      state.updateStatemachine(leaderCommit, currentTerm);
-    }
-    if (entries != null && entries.length > 0) {
-      try {
-        state.getLog().logSync();
-      } catch (InterruptedException e) {
-        throw new InterruptedIOException("logSync got interrupted");
-      }
-      nextIndex = entries[entries.length - 1].getIndex() + 1;
-    }
-    synchronized (this) {
-      if (lifeCycle.getCurrentState() == RUNNING && isFollower()
-          && getState().getCurrentTerm() == currentTerm) {
-        // reset election timer to avoid punishing the leader for our own
-        // long disk writes
-        heartbeatMonitor.updateLastRpcTime(false);
-      }
-    }
-    final AppendEntriesReplyProto reply = ServerProtoUtils.toAppendEntriesReplyProto(
-        leaderId, getId(), currentTerm, nextIndex, SUCCESS);
-    LOG.debug("{}: succeeded to handle AppendEntries. Reply: {}", getId(),
-        ServerProtoUtils.toString(reply));
-    return reply;
-  }
-
-  private boolean containPrevious(TermIndex previous) {
-    LOG.debug("{}: prev:{}, latestSnapshot:{}, getLatestInstalledSnapshot:{}",
-        getId(), previous, state.getLatestSnapshot(), state.getLatestInstalledSnapshot());
-    return state.getLog().contains(previous)
-        ||  (state.getLatestSnapshot() != null
-             && state.getLatestSnapshot().getTermIndex().equals(previous))
-        || (state.getLatestInstalledSnapshot() != null)
-             && state.getLatestInstalledSnapshot().equals(previous);
-  }
-
-  @Override
-  public InstallSnapshotReplyProto installSnapshot(
-      InstallSnapshotRequestProto request) throws IOException {
-    final String leaderId = request.getServerRequest().getRequestorId();
-    CodeInjectionForTesting.execute(INSTALL_SNAPSHOT, getId(), leaderId, request);
-    LOG.debug("{}: receive installSnapshot({})", getId(), request);
-
-    lifeCycle.assertCurrentState(STARTING, RUNNING);
-
-    final long currentTerm;
-    final long leaderTerm = request.getLeaderTerm();
-    final TermIndex lastTermIndex = ServerProtoUtils.toTermIndex(
-        request.getTermIndex());
-    final long lastIncludedIndex = lastTermIndex.getIndex();
-    synchronized (this) {
-      final boolean recognized = state.recognizeLeader(leaderId, leaderTerm);
-      currentTerm = state.getCurrentTerm();
-      if (!recognized) {
-        final InstallSnapshotReplyProto reply = ServerProtoUtils
-            .toInstallSnapshotReplyProto(leaderId, getId(), currentTerm,
-                request.getRequestIndex(), InstallSnapshotResult.NOT_LEADER);
-        LOG.debug("{}: do not recognize leader for installing snapshot." +
-            " Reply: {}", getId(), reply);
-        return reply;
-      }
-      changeToFollower(leaderTerm, true);
-      state.setLeader(leaderId);
-
-      if (lifeCycle.getCurrentState() == RUNNING) {
-        heartbeatMonitor.updateLastRpcTime(true);
-      }
-
-      // Check and append the snapshot chunk. We simply put this in lock
-      // considering a follower peer requiring a snapshot installation does not
-      // have a lot of requests
-      Preconditions.checkState(
-          state.getLog().getNextIndex() <= lastIncludedIndex,
-          "%s log's next id is %s, last included index in snapshot is %s",
-          getId(),  state.getLog().getNextIndex(), lastIncludedIndex);
-
-      //TODO: We should only update State with installed snapshot once the request is done.
-      state.installSnapshot(request);
-
-      // update the committed index
-      // re-load the state machine if this is the last chunk
-      if (request.getDone()) {
-        state.reloadStateMachine(lastIncludedIndex, leaderTerm);
-      }
-      if (lifeCycle.getCurrentState() == RUNNING) {
-        heartbeatMonitor.updateLastRpcTime(false);
-      }
-    }
-    if (request.getDone()) {
-      LOG.info("{}: successfully install the whole snapshot-{}", getId(),
-          lastIncludedIndex);
-    }
-    return ServerProtoUtils.toInstallSnapshotReplyProto(leaderId, getId(),
-        currentTerm, request.getRequestIndex(), InstallSnapshotResult.SUCCESS);
-  }
-
-  AppendEntriesRequestProto createAppendEntriesRequest(long leaderTerm,
-      String targetId, TermIndex previous, List<LogEntryProto> entries,
-      boolean initializing) {
-    return ServerProtoUtils.toAppendEntriesRequestProto(getId(), targetId,
-        leaderTerm, entries, state.getLog().getLastCommittedIndex(),
-        initializing, previous);
-  }
-
-  synchronized InstallSnapshotRequestProto createInstallSnapshotRequest(
-      String targetId, String requestId, int requestIndex, SnapshotInfo snapshot,
-      List<FileChunkProto> chunks, boolean done) {
-    OptionalLong totalSize = snapshot.getFiles().stream()
-        .mapToLong(FileInfo::getFileSize).reduce(Long::sum);
-    assert totalSize.isPresent();
-    return ServerProtoUtils.toInstallSnapshotRequestProto(getId(), targetId,
-        requestId, requestIndex, state.getCurrentTerm(), snapshot.getTermIndex(),
-        chunks, totalSize.getAsLong(), done);
-  }
-
-  synchronized RequestVoteRequestProto createRequestVoteRequest(String targetId,
-      long term, TermIndex lastEntry) {
-    return ServerProtoUtils.toRequestVoteRequestProto(getId(), targetId, term,
-        lastEntry);
-  }
-
-  public synchronized void submitLocalSyncEvent() {
-    if (isLeader() && leaderState != null) {
-      leaderState.submitUpdateStateEvent(LeaderState.UPDATE_COMMIT_EVENT);
-    }
-  }
-
-  public void addPeersToRPC(Iterable<RaftPeer> peers) {
-    serverRpc.addPeers(peers);
-  }
-
-  synchronized void replyPendingRequest(long logIndex,
-      CompletableFuture<Message> message) {
-    if (isLeader() && leaderState != null) { // is leader and is running
-      leaderState.replyPendingRequest(logIndex, message);
-    }
-  }
-
-  TransactionContext getTransactionContext(long index) {
-    if (leaderState != null) { // is leader and is running
-      return leaderState.getTransactionContext(index);
-    }
-    return null;
-  }
-
-  public RaftProperties getProperties() {
-    return this.properties;
-  }
-}


[11/54] [abbrv] incubator-ratis git commit: Server code cleanup: remove rarely used methods and reduce method visibility.

Posted by ji...@apache.org.
Server code cleanup: remove rarely used methods and reduce method visibility.


Project: http://git-wip-us.apache.org/repos/asf/incubator-ratis/repo
Commit: http://git-wip-us.apache.org/repos/asf/incubator-ratis/commit/eaadf8e6
Tree: http://git-wip-us.apache.org/repos/asf/incubator-ratis/tree/eaadf8e6
Diff: http://git-wip-us.apache.org/repos/asf/incubator-ratis/diff/eaadf8e6

Branch: refs/heads/master
Commit: eaadf8e66a98e2bc5e254af96a56f6d8f6d9800f
Parents: a38e2f7
Author: Tsz-Wo Nicholas Sze <sz...@hortonworks.com>
Authored: Tue Jan 3 19:42:48 2017 +0800
Committer: Tsz-Wo Nicholas Sze <sz...@hortonworks.com>
Committed: Tue Jan 3 19:42:48 2017 +0800

----------------------------------------------------------------------
 .../apache/raft/server/impl/FollowerState.java  |  4 +-
 .../apache/raft/server/impl/LeaderElection.java |  4 +-
 .../apache/raft/server/impl/LeaderState.java    |  7 +--
 .../raft/server/impl/RaftConfiguration.java     | 12 ++---
 .../apache/raft/server/impl/RaftServerImpl.java | 17 +++---
 .../java/org/apache/raft/server/impl/Role.java  | 25 ---------
 .../raft/server/impl/RaftServerTestUtil.java    |  6 +++
 .../raft/statemachine/RaftSnapshotBaseTest.java | 57 ++++++++++----------
 .../SimpleStateMachine4Testing.java             | 10 ++--
 .../raft/statemachine/TestStateMachine.java     | 11 ++--
 10 files changed, 71 insertions(+), 82 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-ratis/blob/eaadf8e6/raft-server/src/main/java/org/apache/raft/server/impl/FollowerState.java
----------------------------------------------------------------------
diff --git a/raft-server/src/main/java/org/apache/raft/server/impl/FollowerState.java b/raft-server/src/main/java/org/apache/raft/server/impl/FollowerState.java
index 20f2d8f..fbef188 100644
--- a/raft-server/src/main/java/org/apache/raft/server/impl/FollowerState.java
+++ b/raft-server/src/main/java/org/apache/raft/server/impl/FollowerState.java
@@ -67,8 +67,8 @@ class FollowerState extends Daemon {
         }
         synchronized (server) {
           if (!inLogSync && lastRpcTime.elapsedTimeMs() >= electionTimeout) {
-            LOG.info("{} changes to {}, lastRpcTime:{}, electionTimeout:{}",
-                server.getId(), Role.CANDIDATE, lastRpcTime, electionTimeout);
+            LOG.info("{} changes to CANDIDATE, lastRpcTime:{}, electionTimeout:{}ms",
+                server.getId(), lastRpcTime, electionTimeout);
             // election timeout, should become a candidate
             server.changeToCandidate();
             break;

http://git-wip-us.apache.org/repos/asf/incubator-ratis/blob/eaadf8e6/raft-server/src/main/java/org/apache/raft/server/impl/LeaderElection.java
----------------------------------------------------------------------
diff --git a/raft-server/src/main/java/org/apache/raft/server/impl/LeaderElection.java b/raft-server/src/main/java/org/apache/raft/server/impl/LeaderElection.java
index 39bdb13..a326eb5 100644
--- a/raft-server/src/main/java/org/apache/raft/server/impl/LeaderElection.java
+++ b/raft-server/src/main/java/org/apache/raft/server/impl/LeaderElection.java
@@ -104,8 +104,8 @@ class LeaderElection extends Daemon {
     } catch (InterruptedException e) {
       // the leader election thread is interrupted. The peer may already step
       // down to a follower. The leader election should skip.
-      LOG.info("The leader election thread of peer {} is interrupted. " +
-          "Currently role: {}.", server.getId(), server.getRole());
+      LOG.info(server.getId() + " " + getClass().getSimpleName()
+          + " thread is interrupted gracefully; server=" + server);
     } catch (IOException e) {
       LOG.warn("Failed to persist votedFor/term. Exit the leader election.", e);
       stopRunning();

http://git-wip-us.apache.org/repos/asf/incubator-ratis/blob/eaadf8e6/raft-server/src/main/java/org/apache/raft/server/impl/LeaderState.java
----------------------------------------------------------------------
diff --git a/raft-server/src/main/java/org/apache/raft/server/impl/LeaderState.java b/raft-server/src/main/java/org/apache/raft/server/impl/LeaderState.java
index 39dc400..fbbcb85 100644
--- a/raft-server/src/main/java/org/apache/raft/server/impl/LeaderState.java
+++ b/raft-server/src/main/java/org/apache/raft/server/impl/LeaderState.java
@@ -308,11 +308,12 @@ public class LeaderState {
           }
           // the updated configuration does not need to be sync'ed here
         } catch (InterruptedException e) {
+          final String s = server.getId() + " " + getClass().getSimpleName()
+              + " thread is interrupted ";
           if (!running) {
-            LOG.info("The LeaderState gets is stopped");
+            LOG.info(s + " gracefully; server=" + server);
           } else {
-            LOG.warn("The leader election thread of peer {} is interrupted. "
-                + "Currently role: {}.", server.getId(), server.getRole());
+            LOG.warn(s + " UNEXPECTEDLY; server=" + server, e);
             throw new RuntimeException(e);
           }
         } catch (IOException e) {

http://git-wip-us.apache.org/repos/asf/incubator-ratis/blob/eaadf8e6/raft-server/src/main/java/org/apache/raft/server/impl/RaftConfiguration.java
----------------------------------------------------------------------
diff --git a/raft-server/src/main/java/org/apache/raft/server/impl/RaftConfiguration.java b/raft-server/src/main/java/org/apache/raft/server/impl/RaftConfiguration.java
index 28ff330..4879314 100644
--- a/raft-server/src/main/java/org/apache/raft/server/impl/RaftConfiguration.java
+++ b/raft-server/src/main/java/org/apache/raft/server/impl/RaftConfiguration.java
@@ -140,12 +140,12 @@ public class RaftConfiguration {
   }
 
   /** Is this configuration transitional, i.e. in the middle of a peer change? */
-  public boolean isTransitional() {
+  boolean isTransitional() {
     return oldConf != null;
   }
 
   /** Is this configuration stable, i.e. no on-going peer change? */
-  public boolean isStable() {
+  boolean isStable() {
     return oldConf == null;
   }
 
@@ -157,7 +157,7 @@ public class RaftConfiguration {
     return oldConf != null && oldConf.contains(peerId);
   }
 
-  public boolean contains(String peerId) {
+  boolean contains(String peerId) {
     return containsInConf(peerId) && (oldConf == null || containsInOldConf(peerId));
   }
 
@@ -203,7 +203,7 @@ public class RaftConfiguration {
   }
 
   /** @return true if the self id together with the others are in the majority. */
-  public boolean hasMajority(Collection<String> others, String selfId) {
+  boolean hasMajority(Collection<String> others, String selfId) {
     Preconditions.checkArgument(!others.contains(selfId));
     return conf.hasMajority(others, selfId) &&
         (oldConf == null || oldConf.hasMajority(others, selfId));
@@ -251,11 +251,11 @@ public class RaftConfiguration {
     return peers.get(index);
   }
 
-  public Collection<RaftPeer> getPeersInOldConf() {
+  Collection<RaftPeer> getPeersInOldConf() {
     return oldConf != null ? oldConf.getPeers() : Collections.emptyList();
   }
 
-  public Collection<RaftPeer> getPeersInConf() {
+  Collection<RaftPeer> getPeersInConf() {
     return conf.getPeers();
   }
 }

http://git-wip-us.apache.org/repos/asf/incubator-ratis/blob/eaadf8e6/raft-server/src/main/java/org/apache/raft/server/impl/RaftServerImpl.java
----------------------------------------------------------------------
diff --git a/raft-server/src/main/java/org/apache/raft/server/impl/RaftServerImpl.java b/raft-server/src/main/java/org/apache/raft/server/impl/RaftServerImpl.java
index 1ea40f6..c8a2e28 100644
--- a/raft-server/src/main/java/org/apache/raft/server/impl/RaftServerImpl.java
+++ b/raft-server/src/main/java/org/apache/raft/server/impl/RaftServerImpl.java
@@ -58,6 +58,11 @@ public class RaftServerImpl implements RaftServer {
   static final String INSTALL_SNAPSHOT = CLASS_NAME + ".installSnapshot";
 
 
+  /** Role of raft peer */
+  enum Role {
+    LEADER, CANDIDATE, FOLLOWER
+  }
+
   private final int minTimeoutMs;
   private final int maxTimeoutMs;
 
@@ -97,19 +102,19 @@ public class RaftServerImpl implements RaftServer {
     appenderFactory = initAppenderFactory();
   }
 
-  public int getMinTimeoutMs() {
+  int getMinTimeoutMs() {
     return minTimeoutMs;
   }
 
-  public int getMaxTimeoutMs() {
+  int getMaxTimeoutMs() {
     return maxTimeoutMs;
   }
 
-  public int getRandomTimeoutMs() {
+  int getRandomTimeoutMs() {
     return RaftUtils.getRandomBetween(minTimeoutMs, maxTimeoutMs);
   }
 
-  public StateMachine getStateMachine() {
+  StateMachine getStateMachine() {
     return this.stateMachine;
   }
 
@@ -230,10 +235,6 @@ public class RaftServerImpl implements RaftServer {
     return role == Role.LEADER;
   }
 
-  Role getRole() {
-    return role;
-  }
-
   /**
    * Change the server state to Follower if necessary
    * @param newTerm The new term.

http://git-wip-us.apache.org/repos/asf/incubator-ratis/blob/eaadf8e6/raft-server/src/main/java/org/apache/raft/server/impl/Role.java
----------------------------------------------------------------------
diff --git a/raft-server/src/main/java/org/apache/raft/server/impl/Role.java b/raft-server/src/main/java/org/apache/raft/server/impl/Role.java
deleted file mode 100644
index 1413961..0000000
--- a/raft-server/src/main/java/org/apache/raft/server/impl/Role.java
+++ /dev/null
@@ -1,25 +0,0 @@
-/**
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- *     http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-package org.apache.raft.server.impl;
-
-/**
- * Role of Raft peer
- */
-public enum Role {
-  LEADER, CANDIDATE, FOLLOWER
-}

http://git-wip-us.apache.org/repos/asf/incubator-ratis/blob/eaadf8e6/raft-server/src/test/java/org/apache/raft/server/impl/RaftServerTestUtil.java
----------------------------------------------------------------------
diff --git a/raft-server/src/test/java/org/apache/raft/server/impl/RaftServerTestUtil.java b/raft-server/src/test/java/org/apache/raft/server/impl/RaftServerTestUtil.java
index 5103fca..bd1934f 100644
--- a/raft-server/src/test/java/org/apache/raft/server/impl/RaftServerTestUtil.java
+++ b/raft-server/src/test/java/org/apache/raft/server/impl/RaftServerTestUtil.java
@@ -20,6 +20,8 @@ package org.apache.raft.server.impl;
 import org.apache.raft.MiniRaftCluster;
 import org.apache.raft.RaftTestUtil;
 import org.apache.raft.protocol.RaftPeer;
+import org.apache.raft.server.RaftServer;
+import org.apache.raft.statemachine.StateMachine;
 import org.junit.Assert;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
@@ -64,4 +66,8 @@ public class RaftServerTestUtil {
     }
     Assert.assertEquals(peers.length, numIncluded + deadIncluded);
   }
+
+  public static StateMachine getStateMachine(RaftServerImpl s) {
+    return s.getStateMachine();
+  }
 }

http://git-wip-us.apache.org/repos/asf/incubator-ratis/blob/eaadf8e6/raft-server/src/test/java/org/apache/raft/statemachine/RaftSnapshotBaseTest.java
----------------------------------------------------------------------
diff --git a/raft-server/src/test/java/org/apache/raft/statemachine/RaftSnapshotBaseTest.java b/raft-server/src/test/java/org/apache/raft/statemachine/RaftSnapshotBaseTest.java
index 721d12c..fbdcb8b 100644
--- a/raft-server/src/test/java/org/apache/raft/statemachine/RaftSnapshotBaseTest.java
+++ b/raft-server/src/test/java/org/apache/raft/statemachine/RaftSnapshotBaseTest.java
@@ -25,6 +25,7 @@ import org.apache.raft.client.RaftClient;
 import org.apache.raft.conf.RaftProperties;
 import org.apache.raft.protocol.RaftClientReply;
 import org.apache.raft.protocol.SetConfigurationRequest;
+import org.apache.raft.server.RaftServer;
 import org.apache.raft.server.impl.RaftServerImpl;
 import org.apache.raft.server.impl.RaftServerTestUtil;
 import org.apache.raft.server.simulation.RequestHandler;
@@ -61,6 +62,28 @@ public abstract class RaftSnapshotBaseTest {
   static final Logger LOG = LoggerFactory.getLogger(RaftSnapshotBaseTest.class);
   private static final int SNAPSHOT_TRIGGER_THRESHOLD = 10;
 
+  static File getSnapshotFile(MiniRaftCluster cluster, int i) {
+    final RaftServerImpl leader = cluster.getLeader();
+    final SimpleStateMachine4Testing sm = SimpleStateMachine4Testing.get(leader);
+    return sm.getStateMachineStorage().getSnapshotFile(
+        leader.getState().getCurrentTerm(), i);
+  }
+
+  static void assertLeaderContent(MiniRaftCluster cluster)
+      throws InterruptedException {
+    final RaftServerImpl leader = RaftTestUtil.waitForLeader(cluster);
+    Assert.assertEquals(SNAPSHOT_TRIGGER_THRESHOLD * 2,
+        leader.getState().getLog().getLastCommittedIndex());
+    final LogEntryProto[] entries = SimpleStateMachine4Testing.get(leader).getContent();
+
+    for (int i = 1; i < SNAPSHOT_TRIGGER_THRESHOLD * 2 - 1; i++) {
+      Assert.assertEquals(i+1, entries[i].getIndex());
+      Assert.assertArrayEquals(
+          new SimpleMessage("m" + i).getContent().toByteArray(),
+          entries[i].getSmLogEntry().getData().toByteArray());
+    }
+  }
+
   private MiniRaftCluster cluster;
 
   public abstract MiniRaftCluster initCluster(int numServer, RaftProperties prop)
@@ -103,9 +126,7 @@ public abstract class RaftSnapshotBaseTest {
     }
 
     // wait for the snapshot to be done
-    StateMachine sm = cluster.getLeader().getStateMachine();
-    File snapshotFile = ((SimpleStateMachineStorage)sm.getStateMachineStorage())
-        .getSnapshotFile(cluster.getLeader().getState().getCurrentTerm(), i);
+    final File snapshotFile = getSnapshotFile(cluster, i);
 
     int retries = 0;
     do {
@@ -117,19 +138,8 @@ public abstract class RaftSnapshotBaseTest {
     // restart the peer and check if it can correctly load snapshot
     cluster.restart(false);
     try {
-      RaftTestUtil.waitForLeader(cluster);
-
       // 200 messages + two leader elections --> last committed = 201
-      Assert.assertEquals(SNAPSHOT_TRIGGER_THRESHOLD * 2,
-          cluster.getLeader().getState().getLog().getLastCommittedIndex());
-      sm = cluster.getLeader().getStateMachine();
-      LogEntryProto[] entries = ((SimpleStateMachine4Testing) sm).getContent();
-      for (i = 1; i < SNAPSHOT_TRIGGER_THRESHOLD * 2 - 1; i++) {
-        Assert.assertEquals(i+1, entries[i].getIndex());
-        Assert.assertArrayEquals(
-            new SimpleMessage("m" + i).getContent().toByteArray(),
-            entries[i].getSmLogEntry().getData().toByteArray());
-      }
+      assertLeaderContent(cluster);
     } finally {
       cluster.shutdown();
     }
@@ -158,9 +168,7 @@ public abstract class RaftSnapshotBaseTest {
       // wait for the snapshot to be done
       RaftStorageDirectory storageDirectory = cluster.getLeader().getState()
           .getStorage().getStorageDir();
-      StateMachine sm = cluster.getLeader().getStateMachine();
-      File snapshotFile = ((SimpleStateMachineStorage) sm.getStateMachineStorage())
-          .getSnapshotFile(cluster.getLeader().getState().getCurrentTerm(), i);
+      final File snapshotFile = getSnapshotFile(cluster, i);
       logs = storageDirectory.getLogSegmentFiles();
 
       int retries = 0;
@@ -182,18 +190,7 @@ public abstract class RaftSnapshotBaseTest {
     LOG.info("Restarting the cluster");
     cluster.restart(false);
     try {
-      RaftTestUtil.waitForLeader(cluster);
-
-      Assert.assertEquals(SNAPSHOT_TRIGGER_THRESHOLD * 2,
-          cluster.getLeader().getState().getLog().getLastCommittedIndex());
-      StateMachine sm = cluster.getLeader().getStateMachine();
-      LogEntryProto[] entries = ((SimpleStateMachine4Testing) sm).getContent();
-      for (int i = 1; i < SNAPSHOT_TRIGGER_THRESHOLD * 2 - 1; i++) {
-        Assert.assertEquals(i+1, entries[i].getIndex());
-        Assert.assertArrayEquals(
-            new SimpleMessage("m" + i).getContent().toByteArray(),
-            entries[i].getSmLogEntry().getData().toByteArray());
-      }
+      assertLeaderContent(cluster);
 
       // generate some more traffic
       try(final RaftClient client = cluster.createClient("client",

http://git-wip-us.apache.org/repos/asf/incubator-ratis/blob/eaadf8e6/raft-server/src/test/java/org/apache/raft/statemachine/SimpleStateMachine4Testing.java
----------------------------------------------------------------------
diff --git a/raft-server/src/test/java/org/apache/raft/statemachine/SimpleStateMachine4Testing.java b/raft-server/src/test/java/org/apache/raft/statemachine/SimpleStateMachine4Testing.java
index 227ea58..3e0ae15 100644
--- a/raft-server/src/test/java/org/apache/raft/statemachine/SimpleStateMachine4Testing.java
+++ b/raft-server/src/test/java/org/apache/raft/statemachine/SimpleStateMachine4Testing.java
@@ -17,7 +17,6 @@
  */
 package org.apache.raft.statemachine;
 
-import com.google.common.annotations.VisibleForTesting;
 import com.google.common.base.Preconditions;
 import org.apache.raft.RaftTestUtil.SimpleMessage;
 import org.apache.raft.conf.RaftProperties;
@@ -26,6 +25,8 @@ import org.apache.raft.protocol.Message;
 import org.apache.raft.protocol.RaftClientReply;
 import org.apache.raft.protocol.RaftClientRequest;
 import org.apache.raft.server.impl.RaftServerConstants;
+import org.apache.raft.server.impl.RaftServerImpl;
+import org.apache.raft.server.impl.RaftServerTestUtil;
 import org.apache.raft.server.protocol.TermIndex;
 import org.apache.raft.server.storage.LogInputStream;
 import org.apache.raft.server.storage.LogOutputStream;
@@ -60,6 +61,10 @@ public class SimpleStateMachine4Testing extends BaseStateMachine {
       = "raft.test.simple.state.machine.take.snapshot";
   public static final boolean RAFT_TEST_SIMPLE_STATE_MACHINE_TAKE_SNAPSHOT_DEFAULT = false;
 
+  public static SimpleStateMachine4Testing get(RaftServerImpl s) {
+    return (SimpleStateMachine4Testing)RaftServerTestUtil.getStateMachine(s);
+  }
+
   private final List<LogEntryProto> list =
       Collections.synchronizedList(new ArrayList<>());
   private final Daemon checkpointer;
@@ -174,7 +179,7 @@ public class SimpleStateMachine4Testing extends BaseStateMachine {
   }
 
   @Override
-  public StateMachineStorage getStateMachineStorage() {
+  public SimpleStateMachineStorage getStateMachineStorage() {
     return storage;
   }
 
@@ -235,7 +240,6 @@ public class SimpleStateMachine4Testing extends BaseStateMachine {
     });
   }
 
-  @VisibleForTesting
   public LogEntryProto[] getContent() {
     return list.toArray(new LogEntryProto[list.size()]);
   }

http://git-wip-us.apache.org/repos/asf/incubator-ratis/blob/eaadf8e6/raft-server/src/test/java/org/apache/raft/statemachine/TestStateMachine.java
----------------------------------------------------------------------
diff --git a/raft-server/src/test/java/org/apache/raft/statemachine/TestStateMachine.java b/raft-server/src/test/java/org/apache/raft/statemachine/TestStateMachine.java
index 5892c65..546bfb8 100644
--- a/raft-server/src/test/java/org/apache/raft/statemachine/TestStateMachine.java
+++ b/raft-server/src/test/java/org/apache/raft/statemachine/TestStateMachine.java
@@ -26,6 +26,7 @@ import org.apache.raft.protocol.Message;
 import org.apache.raft.protocol.RaftClientRequest;
 import org.apache.raft.server.RaftServerConfigKeys;
 import org.apache.raft.server.impl.RaftServerImpl;
+import org.apache.raft.server.impl.RaftServerTestUtil;
 import org.apache.raft.server.simulation.MiniRaftClusterWithSimulatedRpc;
 import org.apache.raft.shaded.proto.RaftProtos.SMLogEntryProto;
 import org.apache.raft.util.RaftUtils;
@@ -93,7 +94,11 @@ public class TestStateMachine {
     return properties;
   }
 
-  public static class SMTransactionContext extends SimpleStateMachine4Testing {
+  static class SMTransactionContext extends SimpleStateMachine4Testing {
+    public static SMTransactionContext get(RaftServerImpl s) {
+      return (SMTransactionContext)RaftServerTestUtil.getStateMachine(s);
+    }
+
     AtomicReference<Throwable> throwable = new AtomicReference<>(null);
     AtomicLong transactions = new AtomicLong(0);
     AtomicBoolean isLeader = new AtomicBoolean(false);
@@ -162,7 +167,7 @@ public class TestStateMachine {
     Thread.sleep(cluster.getMaxTimeout() + 100);
 
     for (RaftServerImpl raftServer : cluster.getServers()) {
-      SMTransactionContext sm = ((SMTransactionContext)raftServer.getStateMachine());
+      final SMTransactionContext sm = SMTransactionContext.get(raftServer);
       sm.rethrowIfException();
       assertEquals(numTrx, sm.numApplied.get());
     }
@@ -170,7 +175,7 @@ public class TestStateMachine {
     // check leader
     RaftServerImpl raftServer = cluster.getLeader();
     // assert every transaction has obtained context in leader
-    SMTransactionContext sm = ((SMTransactionContext)raftServer.getStateMachine());
+    final SMTransactionContext sm = SMTransactionContext.get(raftServer);
     List<Long> ll = sm.applied.stream().collect(Collectors.toList());
     Collections.sort(ll);
     assertEquals(ll.toString(), ll.size(), numTrx);


[33/54] [abbrv] incubator-ratis git commit: Renamed the packages from raft to ratis in preperation for Apache Incubation - Moved all java packages from org.apache.raft to org.apache.ratis. - Moved native package to org_apache_ratis, and native lib to l

Posted by ji...@apache.org.
http://git-wip-us.apache.org/repos/asf/incubator-ratis/blob/7e71a2e0/raft-server/src/test/java/org/apache/raft/server/storage/TestRaftLogCache.java
----------------------------------------------------------------------
diff --git a/raft-server/src/test/java/org/apache/raft/server/storage/TestRaftLogCache.java b/raft-server/src/test/java/org/apache/raft/server/storage/TestRaftLogCache.java
deleted file mode 100644
index dde6c7a..0000000
--- a/raft-server/src/test/java/org/apache/raft/server/storage/TestRaftLogCache.java
+++ /dev/null
@@ -1,255 +0,0 @@
-/**
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- *     http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-package org.apache.raft.server.storage;
-
-import org.apache.raft.RaftTestUtil.SimpleOperation;
-import org.apache.raft.server.storage.RaftLogCache.TruncationSegments;
-import org.apache.raft.shaded.proto.RaftProtos.LogEntryProto;
-import org.apache.raft.util.ProtoUtils;
-import org.junit.Assert;
-import org.junit.Before;
-import org.junit.Test;
-
-import java.util.Iterator;
-
-public class TestRaftLogCache {
-  private RaftLogCache cache;
-
-  @Before
-  public void setup() {
-    cache = new RaftLogCache();
-  }
-
-  private LogSegment prepareLogSegment(long start, long end, boolean isOpen) {
-    LogSegment s = LogSegment.newOpenSegment(start);
-    for (long i = start; i <= end; i++) {
-      SimpleOperation m = new SimpleOperation("m" + i);
-      LogEntryProto entry = ProtoUtils.toLogEntryProto(m.getLogEntryContent(),
-          0, i);
-      s.appendToOpenSegment(entry);
-    }
-    if (!isOpen) {
-      s.close();
-    }
-    return s;
-  }
-
-  private void checkCache(long start, long end, int segmentSize) {
-    Assert.assertEquals(start, cache.getStartIndex());
-    Assert.assertEquals(end, cache.getEndIndex());
-
-    for (long index = start; index <= end; index++) {
-      LogEntryProto entry = cache.getEntry(index);
-      Assert.assertEquals(index, entry.getIndex());
-    }
-
-    long[] offsets = new long[]{start, start + 1, start + (end - start) / 2,
-        end - 1, end};
-    for (long offset : offsets) {
-      checkCacheEntries(offset, (int) (end - offset + 1), end);
-      checkCacheEntries(offset, 1, end);
-      checkCacheEntries(offset, 20, end);
-      checkCacheEntries(offset, segmentSize, end);
-      checkCacheEntries(offset, segmentSize - 1, end);
-    }
-  }
-
-  private void checkCacheEntries(long offset, int size, long end) {
-    LogEntryProto[] entries = cache.getEntries(offset, offset + size);
-    long realEnd = offset + size > end + 1 ? end + 1 : offset + size;
-    Assert.assertEquals(realEnd - offset, entries.length);
-    for (long i = offset; i < realEnd; i++) {
-      Assert.assertEquals(i, entries[(int) (i - offset)].getIndex());
-    }
-  }
-
-  @Test
-  public void testAddSegments() throws Exception {
-    LogSegment s1 = prepareLogSegment(1, 100, false);
-    cache.addSegment(s1);
-    checkCache(1, 100, 100);
-
-    try {
-      LogSegment s = prepareLogSegment(102, 103, true);
-      cache.addSegment(s);
-      Assert.fail("should fail since there is gap between two segments");
-    } catch (IllegalStateException ignored) {
-    }
-
-    LogSegment s2 = prepareLogSegment(101, 200, true);
-    cache.addSegment(s2);
-    checkCache(1, 200, 100);
-
-    try {
-      LogSegment s = prepareLogSegment(201, 202, true);
-      cache.addSegment(s);
-      Assert.fail("should fail since there is still an open segment in cache");
-    } catch (IllegalStateException ignored) {
-    }
-
-    cache.rollOpenSegment(false);
-    checkCache(1, 200, 100);
-
-    try {
-      LogSegment s = prepareLogSegment(202, 203, true);
-      cache.addSegment(s);
-      Assert.fail("should fail since there is gap between two segments");
-    } catch (IllegalStateException ignored) {
-    }
-
-    LogSegment s3 = prepareLogSegment(201, 300, true);
-    cache.addSegment(s3);
-    Assert.assertNotNull(cache.getOpenSegment());
-    checkCache(1, 300, 100);
-
-    cache.rollOpenSegment(true);
-    Assert.assertNotNull(cache.getOpenSegment());
-    checkCache(1, 300, 100);
-  }
-
-  @Test
-  public void testAppendEntry() throws Exception {
-    LogSegment closedSegment = prepareLogSegment(0, 99, false);
-    cache.addSegment(closedSegment);
-
-    final SimpleOperation m = new SimpleOperation("m");
-    try {
-      LogEntryProto entry = ProtoUtils.toLogEntryProto(m.getLogEntryContent(),
-          0, 0);
-      cache.appendEntry(entry);
-      Assert.fail("the open segment is null");
-    } catch (IllegalStateException ignored) {
-    }
-
-    LogSegment openSegment = prepareLogSegment(100, 100, true);
-    cache.addSegment(openSegment);
-    for (long index = 101; index < 200; index++) {
-      LogEntryProto entry = ProtoUtils.toLogEntryProto(m.getLogEntryContent(),
-          0, index);
-      cache.appendEntry(entry);
-    }
-
-    Assert.assertNotNull(cache.getOpenSegment());
-    checkCache(0, 199, 100);
-  }
-
-  @Test
-  public void testTruncate() throws Exception {
-    long start = 0;
-    for (int i = 0; i < 5; i++) { // 5 closed segments
-      LogSegment s = prepareLogSegment(start, start + 99, false);
-      cache.addSegment(s);
-      start += 100;
-    }
-    // add another open segment
-    LogSegment s = prepareLogSegment(start, start + 99, true);
-    cache.addSegment(s);
-
-    long end = cache.getEndIndex();
-    Assert.assertEquals(599, end);
-    int numOfSegments = 6;
-    // start truncation
-    for (int i = 0; i < 10; i++) { // truncate 10 times
-      // each time truncate 37 entries
-      end -= 37;
-      TruncationSegments ts = cache.truncate(end + 1);
-      checkCache(0, end, 100);
-
-      // check TruncationSegments
-      int currentNum= (int) (end / 100 + 1);
-      if (currentNum < numOfSegments) {
-        Assert.assertEquals(1, ts.toDelete.length);
-        numOfSegments = currentNum;
-      } else {
-        Assert.assertEquals(0, ts.toDelete.length);
-      }
-    }
-
-    // 230 entries remaining. truncate at the segment boundary
-    TruncationSegments ts = cache.truncate(200);
-    checkCache(0, 199, 100);
-    Assert.assertEquals(1, ts.toDelete.length);
-    Assert.assertEquals(200, ts.toDelete[0].startIndex);
-    Assert.assertEquals(229, ts.toDelete[0].endIndex);
-    Assert.assertEquals(0, ts.toDelete[0].targetLength);
-    Assert.assertFalse(ts.toDelete[0].isOpen);
-    Assert.assertNull(ts.toTruncate);
-
-    // add another open segment and truncate it as a whole
-    LogSegment newOpen = prepareLogSegment(200, 249, true);
-    cache.addSegment(newOpen);
-    ts = cache.truncate(200);
-    checkCache(0, 199, 100);
-    Assert.assertEquals(1, ts.toDelete.length);
-    Assert.assertEquals(200, ts.toDelete[0].startIndex);
-    Assert.assertEquals(249, ts.toDelete[0].endIndex);
-    Assert.assertEquals(0, ts.toDelete[0].targetLength);
-    Assert.assertTrue(ts.toDelete[0].isOpen);
-    Assert.assertNull(ts.toTruncate);
-
-    // add another open segment and truncate part of it
-    newOpen = prepareLogSegment(200, 249, true);
-    cache.addSegment(newOpen);
-    ts = cache.truncate(220);
-    checkCache(0, 219, 100);
-    Assert.assertNull(cache.getOpenSegment());
-    Assert.assertEquals(0, ts.toDelete.length);
-    Assert.assertTrue(ts.toTruncate.isOpen);
-    Assert.assertEquals(219, ts.toTruncate.newEndIndex);
-    Assert.assertEquals(200, ts.toTruncate.startIndex);
-    Assert.assertEquals(249, ts.toTruncate.endIndex);
-  }
-
-  private void testIterator(long startIndex) {
-    Iterator<LogEntryProto> iterator = cache.iterator(startIndex);
-    LogEntryProto prev = null;
-    while (iterator.hasNext()) {
-      LogEntryProto entry = iterator.next();
-      Assert.assertEquals(cache.getEntry(entry.getIndex()), entry);
-      if (prev != null) {
-        Assert.assertEquals(prev.getIndex() + 1, entry.getIndex());
-      }
-      prev = entry;
-    }
-    if (startIndex <= cache.getEndIndex()) {
-      Assert.assertNotNull(prev);
-      Assert.assertEquals(cache.getEndIndex(), prev.getIndex());
-    }
-  }
-
-  @Test
-  public void testIterator() throws Exception {
-    long start = 0;
-    for (int i = 0; i < 2; i++) { // 2 closed segments
-      LogSegment s = prepareLogSegment(start, start + 99, false);
-      cache.addSegment(s);
-      start += 100;
-    }
-    // add another open segment
-    LogSegment s = prepareLogSegment(start, start + 99, true);
-    cache.addSegment(s);
-
-    for (long startIndex = 0; startIndex < 300; startIndex += 50) {
-      testIterator(startIndex);
-    }
-    testIterator(299);
-
-    Iterator<LogEntryProto> iterator = cache.iterator(300);
-    Assert.assertFalse(iterator.hasNext());
-  }
-}

http://git-wip-us.apache.org/repos/asf/incubator-ratis/blob/7e71a2e0/raft-server/src/test/java/org/apache/raft/server/storage/TestRaftLogReadWrite.java
----------------------------------------------------------------------
diff --git a/raft-server/src/test/java/org/apache/raft/server/storage/TestRaftLogReadWrite.java b/raft-server/src/test/java/org/apache/raft/server/storage/TestRaftLogReadWrite.java
deleted file mode 100644
index fa17696..0000000
--- a/raft-server/src/test/java/org/apache/raft/server/storage/TestRaftLogReadWrite.java
+++ /dev/null
@@ -1,266 +0,0 @@
-/**
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- *     http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-package org.apache.raft.server.storage;
-
-import org.apache.raft.RaftTestUtil;
-import org.apache.raft.RaftTestUtil.SimpleOperation;
-import org.apache.raft.conf.RaftProperties;
-import org.apache.raft.protocol.ChecksumException;
-import org.apache.raft.server.impl.RaftServerConstants;
-import org.apache.raft.server.impl.RaftServerConstants.StartupOption;
-import org.apache.raft.shaded.com.google.protobuf.CodedOutputStream;
-import org.apache.raft.shaded.proto.RaftProtos.LogEntryProto;
-import org.apache.raft.util.FileUtils;
-import org.apache.raft.util.ProtoUtils;
-import org.junit.After;
-import org.junit.Assert;
-import org.junit.Before;
-import org.junit.Test;
-import org.slf4j.Logger;
-import org.slf4j.LoggerFactory;
-
-import java.io.File;
-import java.io.FileOutputStream;
-import java.io.IOException;
-import java.io.RandomAccessFile;
-import java.nio.ByteBuffer;
-import java.util.ArrayList;
-import java.util.List;
-
-import static org.apache.raft.server.RaftServerConfigKeys.*;
-
-/**
- * Test basic functionality of LogReader, LogInputStream, and LogOutputStream.
- */
-public class TestRaftLogReadWrite {
-  private static final Logger LOG = LoggerFactory.getLogger(TestRaftLogReadWrite.class);
-
-  private File storageDir;
-  private RaftProperties properties;
-  private int segmentMaxSize;
-
-  @Before
-  public void setup() throws Exception {
-    storageDir = RaftTestUtil.getTestDir(TestRaftLogReadWrite.class);
-    properties = new RaftProperties();
-    properties.set(RAFT_SERVER_STORAGE_DIR_KEY,
-        FileUtils.fileAsURI(storageDir).toString());
-  }
-
-  @After
-  public void tearDown() throws Exception {
-    if (storageDir != null) {
-      FileUtils.fullyDelete(storageDir.getParentFile());
-    }
-  }
-
-  private LogEntryProto[] readLog(File file, long startIndex, long endIndex,
-      boolean isOpen) throws IOException {
-    List<LogEntryProto> list = new ArrayList<>();
-    try (LogInputStream in =
-             new LogInputStream(file, startIndex, endIndex, isOpen)) {
-      LogEntryProto entry;
-      while ((entry = in.nextEntry()) != null) {
-        list.add(entry);
-      }
-    }
-    return list.toArray(new LogEntryProto[list.size()]);
-  }
-
-  private long writeMessages(LogEntryProto[] entries, LogOutputStream out)
-      throws IOException {
-    long size = 0;
-    for (int i = 0; i < entries.length; i++) {
-      SimpleOperation m = new SimpleOperation("m" + i);
-      entries[i] = ProtoUtils.toLogEntryProto(m.getLogEntryContent(), 0, i);
-      final int s = entries[i].getSerializedSize();
-      size += CodedOutputStream.computeUInt32SizeNoTag(s) + s + 4;
-      out.write(entries[i]);
-    }
-    return size;
-  }
-
-  /**
-   * Test basic functionality: write several log entries, then read
-   */
-  @Test
-  public void testReadWriteLog() throws IOException {
-    RaftStorage storage = new RaftStorage(properties, StartupOption.REGULAR);
-    File openSegment = storage.getStorageDir().getOpenLogFile(0);
-    long size = SegmentedRaftLog.HEADER_BYTES.length;
-
-    final LogEntryProto[] entries = new LogEntryProto[100];
-    try (LogOutputStream out =
-             new LogOutputStream(openSegment, false, properties)) {
-      size += writeMessages(entries, out);
-    } finally {
-      storage.close();
-    }
-
-    Assert.assertEquals(size, openSegment.length());
-
-    LogEntryProto[] readEntries = readLog(openSegment, 0,
-        RaftServerConstants.INVALID_LOG_INDEX, true);
-    Assert.assertArrayEquals(entries, readEntries);
-  }
-
-  @Test
-  public void testAppendLog() throws IOException {
-    RaftStorage storage = new RaftStorage(properties, StartupOption.REGULAR);
-    File openSegment = storage.getStorageDir().getOpenLogFile(0);
-    LogEntryProto[] entries = new LogEntryProto[200];
-    try (LogOutputStream out =
-             new LogOutputStream(openSegment, false, properties)) {
-      for (int i = 0; i < 100; i++) {
-        SimpleOperation m = new SimpleOperation("m" + i);
-        entries[i] = ProtoUtils.toLogEntryProto(m.getLogEntryContent(), 0, i);
-        out.write(entries[i]);
-      }
-    }
-
-    try (LogOutputStream out =
-             new LogOutputStream(openSegment, true, properties)) {
-      for (int i = 100; i < 200; i++) {
-        SimpleOperation m = new SimpleOperation("m" + i);
-        entries[i] = ProtoUtils.toLogEntryProto(m.getLogEntryContent(), 0, i);
-        out.write(entries[i]);
-      }
-    }
-
-    LogEntryProto[] readEntries = readLog(openSegment, 0,
-        RaftServerConstants.INVALID_LOG_INDEX, true);
-    Assert.assertArrayEquals(entries, readEntries);
-
-    storage.close();
-  }
-
-  /**
-   * Simulate the scenario that the peer is shutdown without truncating
-   * log segment file padding. Make sure the reader can correctly handle this.
-   */
-  @Test
-  public void testReadWithPadding() throws IOException {
-    RaftStorage storage = new RaftStorage(properties, StartupOption.REGULAR);
-    File openSegment = storage.getStorageDir().getOpenLogFile(0);
-    long size = SegmentedRaftLog.HEADER_BYTES.length;
-
-    LogEntryProto[] entries = new LogEntryProto[100];
-    LogOutputStream out = new LogOutputStream(openSegment, false, properties);
-    size += writeMessages(entries, out);
-    out.flush();
-
-    // make sure the file contains padding
-    Assert.assertEquals(RAFT_LOG_SEGMENT_PREALLOCATED_SIZE_DEFAULT,
-        openSegment.length());
-
-    // check if the reader can correctly read the log file
-    LogEntryProto[] readEntries = readLog(openSegment, 0,
-        RaftServerConstants.INVALID_LOG_INDEX, true);
-    Assert.assertArrayEquals(entries, readEntries);
-
-    out.close();
-    Assert.assertEquals(size, openSegment.length());
-  }
-
-  /**
-   * corrupt the padding by inserting non-zero bytes. Make sure the reader
-   * throws exception.
-   */
-  @Test
-  public void testReadWithCorruptPadding() throws IOException {
-    properties.setLong(RAFT_LOG_SEGMENT_PREALLOCATED_SIZE_KEY, 4 * 1024 * 1024);
-    properties.setLong(RAFT_LOG_SEGMENT_MAX_SIZE_KEY, 16 * 1024 * 1024);
-
-    RaftStorage storage = new RaftStorage(properties, StartupOption.REGULAR);
-    File openSegment = storage.getStorageDir().getOpenLogFile(0);
-
-    LogEntryProto[] entries = new LogEntryProto[10];
-    LogOutputStream out = new LogOutputStream(openSegment, false, properties);
-    for (int i = 0; i < 10; i++) {
-      SimpleOperation m = new SimpleOperation("m" + i);
-      entries[i] = ProtoUtils.toLogEntryProto(m.getLogEntryContent(), 0, i);
-      out.write(entries[i]);
-    }
-    out.flush();
-
-    // make sure the file contains padding
-    Assert.assertEquals(4 * 1024 * 1024, openSegment.length());
-
-    try (FileOutputStream fout = new FileOutputStream(openSegment, true)) {
-      ByteBuffer byteBuffer = ByteBuffer.wrap(new byte[]{-1, 1});
-      fout.getChannel()
-          .write(byteBuffer, 16 * 1024 * 1024 - 10);
-    }
-
-    List<LogEntryProto> list = new ArrayList<>();
-    try (LogInputStream in = new LogInputStream(openSegment, 0,
-        RaftServerConstants.INVALID_LOG_INDEX, true)) {
-      LogEntryProto entry;
-      while ((entry = in.nextEntry()) != null) {
-        list.add(entry);
-      }
-      Assert.fail("should fail since we corrupt the padding");
-    } catch (IOException e) {
-      boolean findVerifyTerminator = false;
-      for (StackTraceElement s : e.getStackTrace()) {
-        if (s.getMethodName().equals("verifyTerminator")) {
-          findVerifyTerminator = true;
-          break;
-        }
-      }
-      Assert.assertTrue(findVerifyTerminator);
-    }
-    Assert.assertArrayEquals(entries,
-        list.toArray(new LogEntryProto[list.size()]));
-  }
-
-  /**
-   * Test the log reader to make sure it can detect the checksum mismatch.
-   */
-  @Test
-  public void testReadWithEntryCorruption() throws IOException {
-    RaftStorage storage = new RaftStorage(properties, StartupOption.REGULAR);
-    File openSegment = storage.getStorageDir().getOpenLogFile(0);
-    try (LogOutputStream out =
-             new LogOutputStream(openSegment, false, properties)) {
-      for (int i = 0; i < 100; i++) {
-        LogEntryProto entry = ProtoUtils.toLogEntryProto(
-            new SimpleOperation("m" + i).getLogEntryContent(), 0, i);
-        out.write(entry);
-      }
-    } finally {
-      storage.close();
-    }
-
-    // corrupt the log file
-    try (RandomAccessFile raf = new RandomAccessFile(openSegment.getCanonicalFile(),
-        "rw")) {
-      raf.seek(100);
-      int correctValue = raf.read();
-      raf.seek(100);
-      raf.write(correctValue + 1);
-    }
-
-    try {
-      readLog(openSegment, 0, RaftServerConstants.INVALID_LOG_INDEX, true);
-      Assert.fail("The read of corrupted log file should fail");
-    } catch (ChecksumException e) {
-      LOG.info("Caught ChecksumException as expected", e);
-    }
-  }
-}

http://git-wip-us.apache.org/repos/asf/incubator-ratis/blob/7e71a2e0/raft-server/src/test/java/org/apache/raft/server/storage/TestRaftLogSegment.java
----------------------------------------------------------------------
diff --git a/raft-server/src/test/java/org/apache/raft/server/storage/TestRaftLogSegment.java b/raft-server/src/test/java/org/apache/raft/server/storage/TestRaftLogSegment.java
deleted file mode 100644
index 470f80f..0000000
--- a/raft-server/src/test/java/org/apache/raft/server/storage/TestRaftLogSegment.java
+++ /dev/null
@@ -1,303 +0,0 @@
-/**
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- *     http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-package org.apache.raft.server.storage;
-
-import org.apache.raft.RaftTestUtil;
-import org.apache.raft.RaftTestUtil.SimpleOperation;
-import org.apache.raft.conf.RaftProperties;
-import org.apache.raft.server.RaftServerConfigKeys;
-import org.apache.raft.server.impl.RaftServerConstants.StartupOption;
-import org.apache.raft.shaded.proto.RaftProtos.LogEntryProto;
-import org.apache.raft.shaded.proto.RaftProtos.SMLogEntryProto;
-import org.apache.raft.util.FileUtils;
-import org.apache.raft.util.ProtoUtils;
-import org.junit.After;
-import org.junit.Assert;
-import org.junit.Before;
-import org.junit.Test;
-
-import java.io.File;
-import java.io.IOException;
-import java.util.ArrayList;
-import java.util.Arrays;
-import java.util.List;
-
-import static org.apache.raft.server.RaftServerConfigKeys.*;
-import static org.apache.raft.server.impl.RaftServerConstants.INVALID_LOG_INDEX;
-import static org.apache.raft.server.storage.LogSegment.getEntrySize;
-
-/**
- * Test basic functionality of {@link LogSegment}
- */
-public class TestRaftLogSegment {
-  private File storageDir;
-  private final RaftProperties properties = new RaftProperties();
-
-  @Before
-  public void setup() throws Exception {
-    storageDir = RaftTestUtil.getTestDir(TestRaftLogSegment.class);
-    properties.set(RaftServerConfigKeys.RAFT_SERVER_STORAGE_DIR_KEY,
-        storageDir.getCanonicalPath());
-  }
-
-  @After
-  public void tearDown() throws Exception {
-    if (storageDir != null) {
-      FileUtils.fullyDelete(storageDir.getParentFile());
-    }
-  }
-
-  private File prepareLog(boolean isOpen, long start, int size, long term)
-      throws IOException {
-    RaftStorage storage = new RaftStorage(properties, StartupOption.REGULAR);
-    File file = isOpen ? storage.getStorageDir().getOpenLogFile(start) :
-        storage.getStorageDir().getClosedLogFile(start, start + size - 1);
-
-    LogEntryProto[] entries = new LogEntryProto[size];
-    try (LogOutputStream out = new LogOutputStream(file, false, properties)) {
-      for (int i = 0; i < size; i++) {
-        SimpleOperation op = new SimpleOperation("m" + i);
-        entries[i] = ProtoUtils.toLogEntryProto(op.getLogEntryContent(),
-            term, i + start);
-        out.write(entries[i]);
-      }
-    }
-    storage.close();
-    return file;
-  }
-
-  private void checkLogSegment(LogSegment segment, long start, long end,
-      boolean isOpen, long totalSize, long term) {
-    Assert.assertEquals(start, segment.getStartIndex());
-    Assert.assertEquals(end, segment.getEndIndex());
-    Assert.assertEquals(isOpen, segment.isOpen());
-    Assert.assertEquals(totalSize, segment.getTotalSize());
-
-    long offset = SegmentedRaftLog.HEADER_BYTES.length;
-    for (long i = start; i <= end; i++) {
-      LogSegment.LogRecord record = segment.getLogRecord(i);
-      Assert.assertEquals(i, record.entry.getIndex());
-      Assert.assertEquals(term, record.entry.getTerm());
-      Assert.assertEquals(offset, record.offset);
-
-      offset += getEntrySize(record.entry);
-    }
-  }
-
-  @Test
-  public void testLoadLogSegment() throws Exception {
-    // load an open segment
-    File openSegmentFile = prepareLog(true, 0, 100, 0);
-    LogSegment openSegment = LogSegment.loadSegment(openSegmentFile, 0,
-        INVALID_LOG_INDEX, true, null);
-    checkLogSegment(openSegment, 0, 99, true, openSegmentFile.length(), 0);
-
-    // load a closed segment (1000-1099)
-    File closedSegmentFile = prepareLog(false, 1000, 100, 1);
-    LogSegment closedSegment = LogSegment.loadSegment(closedSegmentFile, 1000,
-        1099, false, null);
-    checkLogSegment(closedSegment, 1000, 1099, false,
-        closedSegment.getTotalSize(), 1);
-  }
-
-  @Test
-  public void testAppendEntries() throws Exception {
-    final long start = 1000;
-    LogSegment segment = LogSegment.newOpenSegment(start);
-    long size = SegmentedRaftLog.HEADER_BYTES.length;
-    final long max = 8 * 1024 * 1024;
-    checkLogSegment(segment, start, start - 1, true, size, 0);
-
-    // append till full
-    long term = 0;
-    int i = 0;
-    List<LogEntryProto> list = new ArrayList<>();
-    while (size < max) {
-      SimpleOperation op = new SimpleOperation("m" + i);
-      LogEntryProto entry = ProtoUtils.toLogEntryProto(op.getLogEntryContent(),
-          term, i++ + start);
-      size += getEntrySize(entry);
-      list.add(entry);
-    }
-
-    segment.appendToOpenSegment(list.toArray(new LogEntryProto[list.size()]));
-    Assert.assertTrue(segment.getTotalSize() >= max);
-    checkLogSegment(segment, start, i - 1 + start, true, size, term);
-  }
-
-  @Test
-  public void testAppendWithGap() throws Exception {
-    LogSegment segment = LogSegment.newOpenSegment(1000);
-    SimpleOperation op = new SimpleOperation("m");
-    final SMLogEntryProto m = op.getLogEntryContent();
-    try {
-      LogEntryProto entry = ProtoUtils.toLogEntryProto(m, 0, 1001);
-      segment.appendToOpenSegment(entry);
-      Assert.fail("should fail since the entry's index needs to be 1000");
-    } catch (Exception e) {
-      Assert.assertTrue(e instanceof IllegalArgumentException);
-    }
-
-    LogEntryProto entry = ProtoUtils.toLogEntryProto(m, 0, 1000);
-    segment.appendToOpenSegment(entry);
-
-    try {
-      entry = ProtoUtils.toLogEntryProto(m, 0, 1002);
-      segment.appendToOpenSegment(entry);
-      Assert.fail("should fail since the entry's index needs to be 1001");
-    } catch (Exception e) {
-      Assert.assertTrue(e instanceof IllegalArgumentException);
-    }
-
-    LogEntryProto[] entries = new LogEntryProto[2];
-    for (int i = 0; i < 2; i++) {
-      entries[i] = ProtoUtils.toLogEntryProto(m, 0, 1001 + i * 2);
-    }
-    try {
-      segment.appendToOpenSegment(entries);
-      Assert.fail("should fail since there is gap between entries");
-    } catch (Exception e) {
-      Assert.assertTrue(e instanceof IllegalArgumentException);
-    }
-  }
-
-  @Test
-  public void testTruncate() throws Exception {
-    final long term = 1;
-    final long start = 1000;
-    LogSegment segment = LogSegment.newOpenSegment(start);
-    for (int i = 0; i < 100; i++) {
-      LogEntryProto entry = ProtoUtils.toLogEntryProto(
-          new SimpleOperation("m" + i).getLogEntryContent(), term, i + start);
-      segment.appendToOpenSegment(entry);
-    }
-
-    // truncate an open segment (remove 1080~1099)
-    long newSize = segment.getLogRecord(start + 80).offset;
-    segment.truncate(start + 80);
-    Assert.assertEquals(80, segment.numOfEntries());
-    checkLogSegment(segment, start, start + 79, false, newSize, term);
-
-    // truncate a closed segment (remove 1050~1079)
-    newSize = segment.getLogRecord(start + 50).offset;
-    segment.truncate(start + 50);
-    Assert.assertEquals(50, segment.numOfEntries());
-    checkLogSegment(segment, start, start + 49, false, newSize, term);
-
-    // truncate all the remaining entries
-    segment.truncate(start);
-    Assert.assertEquals(0, segment.numOfEntries());
-    checkLogSegment(segment, start, start - 1, false,
-        SegmentedRaftLog.HEADER_BYTES.length, term);
-  }
-
-  private RaftProperties getProperties(long maxSegmentSize,
-      long preallocatedSize) {
-    RaftProperties p = new RaftProperties();
-    p.setLong(RAFT_LOG_SEGMENT_MAX_SIZE_KEY,
-        maxSegmentSize);
-    p.setLong(RaftServerConfigKeys.RAFT_LOG_SEGMENT_PREALLOCATED_SIZE_KEY,
-        preallocatedSize);
-    return p;
-  }
-
-  @Test
-  public void testPreallocateSegment() throws Exception {
-    RaftStorage storage = new RaftStorage(properties, StartupOption.REGULAR);
-    final File file = storage.getStorageDir().getOpenLogFile(0);
-    final int[] maxSizes = new int[]{1024, 1025, 1024 * 1024 - 1, 1024 * 1024,
-        1024 * 1024 + 1, 2 * 1024 * 1024 - 1, 2 * 1024 * 1024,
-        2 * 1024 * 1024 + 1, 8 * 1024 * 1024};
-    final int[] preallocated = new int[]{512, 1024, 1025, 1024 * 1024,
-        1024 * 1024 + 1, 2 * 1024 * 1024};
-
-    // make sure preallocation is correct with different max/pre-allocated size
-    for (int max : maxSizes) {
-      for (int a : preallocated) {
-        try (LogOutputStream ignored =
-                 new LogOutputStream(file, false, getProperties(max, a))) {
-          Assert.assertEquals(file.length(), Math.min(max, a));
-        }
-        try (LogInputStream in =
-                 new LogInputStream(file, 0, INVALID_LOG_INDEX, true)) {
-          LogEntryProto entry = in.nextEntry();
-          Assert.assertNull(entry);
-        }
-      }
-    }
-
-    // test the scenario where an entry's size is larger than the max size
-    final byte[] content = new byte[1024 * 2];
-    Arrays.fill(content, (byte) 1);
-    final long size;
-    try (LogOutputStream out = new LogOutputStream(file, false,
-        getProperties(1024, 1024))) {
-      SimpleOperation op = new SimpleOperation(new String(content));
-      LogEntryProto entry = ProtoUtils.toLogEntryProto(op.getLogEntryContent(),
-          0, 0);
-      size = LogSegment.getEntrySize(entry);
-      out.write(entry);
-    }
-    Assert.assertEquals(file.length(),
-        size + SegmentedRaftLog.HEADER_BYTES.length);
-    try (LogInputStream in = new LogInputStream(file, 0,
-        INVALID_LOG_INDEX, true)) {
-      LogEntryProto entry = in.nextEntry();
-      Assert.assertArrayEquals(content,
-          entry.getSmLogEntry().getData().toByteArray());
-      Assert.assertNull(in.nextEntry());
-    }
-  }
-
-  /**
-   * Keep appending and check if pre-allocation is correct
-   */
-  @Test
-  public void testPreallocationAndAppend() throws Exception {
-    final long max = 2 * 1024 * 1024;
-    properties.setLong(RAFT_LOG_SEGMENT_MAX_SIZE_KEY, max);
-    properties.setLong(RAFT_LOG_SEGMENT_PREALLOCATED_SIZE_KEY, 16 * 1024);
-    properties.setLong(RAFT_LOG_WRITE_BUFFER_SIZE_KEY, 10 * 1024);
-    RaftStorage storage = new RaftStorage(properties, StartupOption.REGULAR);
-    final File file = storage.getStorageDir().getOpenLogFile(0);
-
-    final byte[] content = new byte[1024];
-    Arrays.fill(content, (byte) 1);
-    SimpleOperation op = new SimpleOperation(new String(content));
-    LogEntryProto entry = ProtoUtils.toLogEntryProto(op.getLogEntryContent(),
-        0, 0);
-    final long entrySize = LogSegment.getEntrySize(entry);
-
-    long totalSize = SegmentedRaftLog.HEADER_BYTES.length;
-    long preallocated = 16 * 1024;
-    try (LogOutputStream out = new LogOutputStream(file, false, properties)) {
-      Assert.assertEquals(preallocated, file.length());
-      while (totalSize + entrySize < max) {
-        totalSize += entrySize;
-        out.write(entry);
-        if (totalSize > preallocated) {
-          Assert.assertEquals("totalSize==" + totalSize,
-              preallocated + 16 * 1024, file.length());
-          preallocated += 16 * 1024;
-        }
-      }
-    }
-
-    Assert.assertEquals(totalSize, file.length());
-  }
-}

http://git-wip-us.apache.org/repos/asf/incubator-ratis/blob/7e71a2e0/raft-server/src/test/java/org/apache/raft/server/storage/TestRaftStorage.java
----------------------------------------------------------------------
diff --git a/raft-server/src/test/java/org/apache/raft/server/storage/TestRaftStorage.java b/raft-server/src/test/java/org/apache/raft/server/storage/TestRaftStorage.java
deleted file mode 100644
index 1b14199..0000000
--- a/raft-server/src/test/java/org/apache/raft/server/storage/TestRaftStorage.java
+++ /dev/null
@@ -1,212 +0,0 @@
-/**
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- *     http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-package org.apache.raft.server.storage;
-
-import org.apache.raft.RaftTestUtil;
-import org.apache.raft.conf.RaftProperties;
-import org.apache.raft.io.nativeio.NativeIO;
-import org.apache.raft.server.RaftServerConfigKeys;
-import org.apache.raft.server.impl.RaftServerConstants.StartupOption;
-import org.apache.raft.server.protocol.TermIndex;
-import org.apache.raft.server.storage.RaftStorageDirectory.StorageState;
-import org.apache.raft.statemachine.SimpleStateMachineStorage;
-import org.apache.raft.util.FileUtils;
-import org.junit.After;
-import org.junit.Assert;
-import org.junit.Before;
-import org.junit.Test;
-import org.mockito.internal.util.reflection.Whitebox;
-
-import java.io.File;
-import java.io.IOException;
-import java.util.concurrent.ThreadLocalRandom;
-
-/**
- * Test RaftStorage and RaftStorageDirectory
- */
-public class TestRaftStorage {
-  private File storageDir;
-  private final RaftProperties properties = new RaftProperties();
-
-  @Before
-  public void setup() throws Exception {
-    storageDir = RaftTestUtil.getTestDir(TestRaftStorage.class);
-    properties.set(RaftServerConfigKeys.RAFT_SERVER_STORAGE_DIR_KEY,
-        storageDir.getCanonicalPath());
-  }
-
-  @After
-  public void tearDown() throws Exception {
-    if (storageDir != null) {
-      FileUtils.fullyDelete(storageDir.getParentFile());
-    }
-  }
-
-  @Test
-  public void testNotExistent() throws IOException {
-    FileUtils.fullyDelete(storageDir);
-
-    // we will format the empty directory
-    RaftStorage storage = new RaftStorage(properties, StartupOption.REGULAR);
-    Assert.assertEquals(StorageState.NORMAL, storage.getState());
-
-    try {
-      new RaftStorage(properties, StartupOption.FORMAT).close();
-      Assert.fail("the format should fail since the storage is still locked");
-    } catch (IOException e) {
-      Assert.assertTrue(e.getMessage().contains("directory is already locked"));
-    }
-
-    storage.close();
-    FileUtils.fullyDelete(storageDir);
-    Assert.assertTrue(storageDir.createNewFile());
-    try {
-      new RaftStorage(properties, StartupOption.REGULAR);
-      Assert.fail();
-    } catch (IOException e) {
-      Assert.assertTrue(
-          e.getMessage().contains(StorageState.NON_EXISTENT.name()));
-    }
-  }
-
-  /**
-   * make sure the RaftStorage format works
-   */
-  @Test
-  public void testStorage() throws Exception {
-    RaftStorageDirectory sd = new RaftStorageDirectory(storageDir);
-    try {
-      StorageState state = sd.analyzeStorage(true);
-      Assert.assertEquals(StorageState.NOT_FORMATTED, state);
-      Assert.assertTrue(sd.isCurrentEmpty());
-    } finally {
-      sd.unlock();
-    }
-
-    RaftStorage storage = new RaftStorage(properties, StartupOption.REGULAR);
-    Assert.assertEquals(StorageState.NORMAL, storage.getState());
-    storage.close();
-
-    Assert.assertEquals(StorageState.NORMAL, sd.analyzeStorage(false));
-    File m = sd.getMetaFile();
-    Assert.assertTrue(m.exists());
-    MetaFile metaFile = new MetaFile(m);
-    Assert.assertEquals(MetaFile.DEFAULT_TERM, metaFile.getTerm());
-    Assert.assertEquals(MetaFile.EMPTY_VOTEFOR, metaFile.getVotedFor());
-
-    metaFile.set(123, "peer1");
-    metaFile.readFile();
-    Assert.assertEquals(123, metaFile.getTerm());
-    Assert.assertEquals("peer1", metaFile.getVotedFor());
-
-    MetaFile metaFile2 = new MetaFile(m);
-    Assert.assertFalse((Boolean) Whitebox.getInternalState(metaFile2, "loaded"));
-    Assert.assertEquals(123, metaFile.getTerm());
-    Assert.assertEquals("peer1", metaFile.getVotedFor());
-
-    // test format
-    storage = new RaftStorage(properties, StartupOption.FORMAT);
-    Assert.assertEquals(StorageState.NORMAL, storage.getState());
-    metaFile = new MetaFile(sd.getMetaFile());
-    Assert.assertEquals(MetaFile.DEFAULT_TERM, metaFile.getTerm());
-    Assert.assertEquals(MetaFile.EMPTY_VOTEFOR, metaFile.getVotedFor());
-    storage.close();
-  }
-
-  @Test
-  public void testMetaFile() throws Exception {
-    RaftStorage storage = new RaftStorage(properties, StartupOption.FORMAT);
-    File m = storage.getStorageDir().getMetaFile();
-    Assert.assertTrue(m.exists());
-    MetaFile metaFile = new MetaFile(m);
-    Assert.assertEquals(MetaFile.DEFAULT_TERM, metaFile.getTerm());
-    Assert.assertEquals(MetaFile.EMPTY_VOTEFOR, metaFile.getVotedFor());
-
-    metaFile.set(123, "peer1");
-    metaFile.readFile();
-    Assert.assertEquals(123, metaFile.getTerm());
-    Assert.assertEquals("peer1", metaFile.getVotedFor());
-
-    MetaFile metaFile2 = new MetaFile(m);
-    Assert.assertFalse((Boolean) Whitebox.getInternalState(metaFile2, "loaded"));
-    Assert.assertEquals(123, metaFile.getTerm());
-    Assert.assertEquals("peer1", metaFile.getVotedFor());
-
-    storage.close();
-  }
-
-  /**
-   * check if RaftStorage deletes tmp metafile when startup
-   */
-  @Test
-  public void testCleanMetaTmpFile() throws Exception {
-    RaftStorage storage = new RaftStorage(properties, StartupOption.REGULAR);
-    Assert.assertEquals(StorageState.NORMAL, storage.getState());
-    storage.close();
-
-    RaftStorageDirectory sd = new RaftStorageDirectory(storageDir);
-    File metaFile = sd.getMetaFile();
-    NativeIO.renameTo(metaFile, sd.getMetaTmpFile());
-
-    Assert.assertEquals(StorageState.NOT_FORMATTED, sd.analyzeStorage(false));
-
-    try {
-      new RaftStorage(properties, StartupOption.REGULAR);
-      Assert.fail("should throw IOException since storage dir is not formatted");
-    } catch (IOException e) {
-      Assert.assertTrue(
-          e.getMessage().contains(StorageState.NOT_FORMATTED.name()));
-    }
-
-    // let the storage dir contain both raft-meta and raft-meta.tmp
-    new RaftStorage(properties, StartupOption.FORMAT).close();
-    Assert.assertTrue(sd.getMetaFile().exists());
-    Assert.assertTrue(sd.getMetaTmpFile().createNewFile());
-    Assert.assertTrue(sd.getMetaTmpFile().exists());
-    try {
-      storage = new RaftStorage(properties, StartupOption.REGULAR);
-      Assert.assertEquals(StorageState.NORMAL, storage.getState());
-      Assert.assertFalse(sd.getMetaTmpFile().exists());
-      Assert.assertTrue(sd.getMetaFile().exists());
-    } finally {
-      storage.close();
-    }
-  }
-
-  @Test
-  public void testSnapshotFileName() throws Exception {
-    final long term = ThreadLocalRandom.current().nextLong(Long.MAX_VALUE);
-    final long index = ThreadLocalRandom.current().nextLong(Long.MAX_VALUE);
-    final String name = SimpleStateMachineStorage.getSnapshotFileName(term, index);
-    System.out.println("name = " + name);
-    final File file = new File(storageDir, name);
-    final TermIndex ti = SimpleStateMachineStorage.getTermIndexFromSnapshotFile(file);
-    System.out.println("file = " + file);
-    Assert.assertEquals(term, ti.getTerm());
-    Assert.assertEquals(index, ti.getIndex());
-    System.out.println("ti = " + ti);
-
-    final File foo = new File(storageDir, "foo");
-    try {
-      SimpleStateMachineStorage.getTermIndexFromSnapshotFile(foo);
-      Assert.fail();
-    } catch(IllegalArgumentException iae) {
-      System.out.println("Good " + iae);
-    }
-  }
-}

http://git-wip-us.apache.org/repos/asf/incubator-ratis/blob/7e71a2e0/raft-server/src/test/java/org/apache/raft/server/storage/TestSegmentedRaftLog.java
----------------------------------------------------------------------
diff --git a/raft-server/src/test/java/org/apache/raft/server/storage/TestSegmentedRaftLog.java b/raft-server/src/test/java/org/apache/raft/server/storage/TestSegmentedRaftLog.java
deleted file mode 100644
index 264ba8e..0000000
--- a/raft-server/src/test/java/org/apache/raft/server/storage/TestSegmentedRaftLog.java
+++ /dev/null
@@ -1,329 +0,0 @@
-/**
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- *     http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-package org.apache.raft.server.storage;
-
-import org.apache.log4j.Level;
-import org.apache.raft.MiniRaftCluster;
-import org.apache.raft.RaftTestUtil;
-import org.apache.raft.RaftTestUtil.SimpleOperation;
-import org.apache.raft.conf.RaftProperties;
-import org.apache.raft.server.RaftServerConfigKeys;
-import org.apache.raft.server.impl.ConfigurationManager;
-import org.apache.raft.server.impl.RaftServerConstants;
-import org.apache.raft.shaded.proto.RaftProtos.LogEntryProto;
-import org.apache.raft.util.FileUtils;
-import org.apache.raft.util.ProtoUtils;
-import org.apache.raft.util.RaftUtils;
-import org.junit.After;
-import org.junit.Assert;
-import org.junit.Before;
-import org.junit.Test;
-
-import java.io.File;
-import java.io.IOException;
-import java.util.ArrayList;
-import java.util.Arrays;
-import java.util.Collections;
-import java.util.List;
-import java.util.function.Supplier;
-
-import static org.apache.raft.server.RaftServerConfigKeys.RAFT_LOG_SEGMENT_MAX_SIZE_KEY;
-import static org.apache.raft.server.RaftServerConfigKeys.RAFT_LOG_SEGMENT_PREALLOCATED_SIZE_KEY;
-
-public class TestSegmentedRaftLog {
-  static {
-    RaftUtils.setLogLevel(RaftLogWorker.LOG, Level.DEBUG);
-  }
-
-  private static final String peerId = "s0";
-
-  private static class SegmentRange {
-    final long start;
-    final long end;
-    final long term;
-    final boolean isOpen;
-
-    SegmentRange(long s, long e, long term, boolean isOpen) {
-      this.start = s;
-      this.end = e;
-      this.term = term;
-      this.isOpen = isOpen;
-    }
-  }
-
-  private File storageDir;
-  private RaftProperties properties;
-  private RaftStorage storage;
-  private final ConfigurationManager cm = new ConfigurationManager(
-      MiniRaftCluster.initConfiguration(MiniRaftCluster.generateIds(3, 0)));
-
-  @Before
-  public void setup() throws Exception {
-    storageDir = RaftTestUtil.getTestDir(TestSegmentedRaftLog.class);
-    properties = new RaftProperties();
-    properties.set(RaftServerConfigKeys.RAFT_SERVER_STORAGE_DIR_KEY,
-        storageDir.getCanonicalPath());
-    storage = new RaftStorage(properties, RaftServerConstants.StartupOption.REGULAR);
-  }
-
-  @After
-  public void tearDown() throws Exception {
-    if (storageDir != null) {
-      FileUtils.fullyDelete(storageDir.getParentFile());
-    }
-  }
-
-  private LogEntryProto[] prepareLog(List<SegmentRange> list) throws IOException {
-    List<LogEntryProto> entryList = new ArrayList<>();
-    for (SegmentRange range : list) {
-      File file = range.isOpen ?
-          storage.getStorageDir().getOpenLogFile(range.start) :
-          storage.getStorageDir().getClosedLogFile(range.start, range.end);
-
-      final int size = (int) (range.end - range.start + 1);
-      LogEntryProto[] entries = new LogEntryProto[size];
-      try (LogOutputStream out = new LogOutputStream(file, false, properties)) {
-        for (int i = 0; i < size; i++) {
-          SimpleOperation m = new SimpleOperation("m" + (i + range.start));
-          entries[i] = ProtoUtils.toLogEntryProto(m.getLogEntryContent(),
-              range.term, i + range.start);
-          out.write(entries[i]);
-        }
-      }
-      Collections.addAll(entryList, entries);
-    }
-    return entryList.toArray(new LogEntryProto[entryList.size()]);
-  }
-
-  private List<SegmentRange> prepareRanges(int number, int segmentSize,
-      long startIndex) {
-    List<SegmentRange> list = new ArrayList<>(number);
-    for (int i = 0; i < number; i++) {
-      list.add(new SegmentRange(startIndex, startIndex + segmentSize - 1, i,
-          i == number - 1));
-      startIndex += segmentSize;
-    }
-    return list;
-  }
-
-  @Test
-  public void testLoadLogSegments() throws Exception {
-    // first generate log files
-    List<SegmentRange> ranges = prepareRanges(5, 100, 0);
-    LogEntryProto[] entries = prepareLog(ranges);
-
-    // create RaftLog object and load log file
-    try (SegmentedRaftLog raftLog =
-             new SegmentedRaftLog(peerId, null, storage, -1, properties)) {
-      raftLog.open(cm, RaftServerConstants.INVALID_LOG_INDEX);
-      // check if log entries are loaded correctly
-      for (LogEntryProto e : entries) {
-        LogEntryProto entry = raftLog.get(e.getIndex());
-        Assert.assertEquals(e, entry);
-      }
-
-      Assert.assertArrayEquals(entries, raftLog.getEntries(0, 500));
-      Assert.assertEquals(entries[entries.length - 1], raftLog.getLastEntry());
-    }
-  }
-
-  List<LogEntryProto> prepareLogEntries(List<SegmentRange> slist,
-      Supplier<String> stringSupplier) {
-    List<LogEntryProto> eList = new ArrayList<>();
-    for (SegmentRange range : slist) {
-      for (long index = range.start; index <= range.end; index++) {
-        SimpleOperation m = stringSupplier == null ?
-            new SimpleOperation("m" + index) :
-            new SimpleOperation(stringSupplier.get());
-        eList.add(ProtoUtils.toLogEntryProto(m.getLogEntryContent(),
-            range.term, index));
-      }
-    }
-    return eList;
-  }
-
-  /**
-   * Append entry one by one and check if log state is correct.
-   */
-  @Test
-  public void testAppendEntry() throws Exception {
-    List<SegmentRange> ranges = prepareRanges(5, 200, 0);
-    List<LogEntryProto> entries = prepareLogEntries(ranges, null);
-
-    try (SegmentedRaftLog raftLog =
-             new SegmentedRaftLog(peerId, null, storage, -1, properties)) {
-      raftLog.open(cm, RaftServerConstants.INVALID_LOG_INDEX);
-      // append entries to the raftlog
-      entries.forEach(raftLog::appendEntry);
-      raftLog.logSync();
-    }
-
-    try (SegmentedRaftLog raftLog =
-             new SegmentedRaftLog(peerId, null, storage, -1, properties)) {
-      raftLog.open(cm, RaftServerConstants.INVALID_LOG_INDEX);
-      // check if the raft log is correct
-      checkEntries(raftLog, entries, 0, entries.size());
-    }
-  }
-
-  /**
-   * Keep appending entries, make sure the rolling is correct.
-   */
-  @Test
-  public void testAppendAndRoll() throws Exception {
-    properties.setLong(RAFT_LOG_SEGMENT_PREALLOCATED_SIZE_KEY, 16 * 1024);
-    properties.setLong(RAFT_LOG_SEGMENT_MAX_SIZE_KEY, 128 * 1024);
-
-    List<SegmentRange> ranges = prepareRanges(1, 1024, 0);
-    final byte[] content = new byte[1024];
-    List<LogEntryProto> entries = prepareLogEntries(ranges,
-        () -> new String(content));
-
-    try (SegmentedRaftLog raftLog =
-             new SegmentedRaftLog(peerId, null, storage, -1, properties)) {
-      raftLog.open(cm, RaftServerConstants.INVALID_LOG_INDEX);
-      // append entries to the raftlog
-      entries.forEach(raftLog::appendEntry);
-      raftLog.logSync();
-    }
-
-    try (SegmentedRaftLog raftLog =
-             new SegmentedRaftLog(peerId, null, storage, -1, properties)) {
-      raftLog.open(cm, RaftServerConstants.INVALID_LOG_INDEX);
-      // check if the raft log is correct
-      checkEntries(raftLog, entries, 0, entries.size());
-      Assert.assertEquals(9, raftLog.getRaftLogCache().getNumOfSegments());
-    }
-  }
-
-  @Test
-  public void testTruncate() throws Exception {
-    // prepare the log for truncation
-    List<SegmentRange> ranges = prepareRanges(5, 200, 0);
-    List<LogEntryProto> entries = prepareLogEntries(ranges, null);
-
-    try (SegmentedRaftLog raftLog =
-             new SegmentedRaftLog(peerId, null, storage, -1, properties)) {
-      raftLog.open(cm, RaftServerConstants.INVALID_LOG_INDEX);
-      // append entries to the raftlog
-      entries.forEach(raftLog::appendEntry);
-      raftLog.logSync();
-    }
-
-    for (long fromIndex = 900; fromIndex >= 0; fromIndex -= 150) {
-      testTruncate(entries, fromIndex);
-    }
-  }
-
-  private void testTruncate(List<LogEntryProto> entries, long fromIndex)
-      throws Exception {
-    try (SegmentedRaftLog raftLog =
-             new SegmentedRaftLog(peerId, null, storage, -1, properties)) {
-      raftLog.open(cm, RaftServerConstants.INVALID_LOG_INDEX);
-      // truncate the log
-      raftLog.truncate(fromIndex);
-      raftLog.logSync();
-
-      checkEntries(raftLog, entries, 0, (int) fromIndex);
-    }
-
-    try (SegmentedRaftLog raftLog =
-             new SegmentedRaftLog(peerId, null, storage, -1, properties)) {
-      raftLog.open(cm, RaftServerConstants.INVALID_LOG_INDEX);
-      // check if the raft log is correct
-      if (fromIndex > 0) {
-        Assert.assertEquals(entries.get((int) (fromIndex - 1)),
-            raftLog.getLastEntry());
-      } else {
-        Assert.assertNull(raftLog.getLastEntry());
-      }
-      checkEntries(raftLog, entries, 0, (int) fromIndex);
-    }
-  }
-
-  private void checkEntries(RaftLog raftLog, List<LogEntryProto> expected,
-      int offset, int size) {
-    if (size > 0) {
-      for (int i = offset; i < size + offset; i++) {
-        LogEntryProto entry = raftLog.get(expected.get(i).getIndex());
-        Assert.assertEquals(expected.get(i), entry);
-      }
-      LogEntryProto[] entriesFromLog = raftLog.getEntries(
-          expected.get(offset).getIndex(),
-          expected.get(offset + size - 1).getIndex() + 1);
-      LogEntryProto[] expectedArray = expected.subList(offset, offset + size)
-          .toArray(SegmentedRaftLog.EMPTY_LOGENTRY_ARRAY);
-      Assert.assertArrayEquals(expectedArray, entriesFromLog);
-    }
-  }
-
-  /**
-   * Test append with inconsistent entries
-   */
-  @Test
-  public void testAppendEntriesWithInconsistency() throws Exception {
-    // prepare the log for truncation
-    List<SegmentRange> ranges = prepareRanges(5, 200, 0);
-    List<LogEntryProto> entries = prepareLogEntries(ranges, null);
-
-    try (SegmentedRaftLog raftLog =
-             new SegmentedRaftLog(peerId, null, storage, -1, properties)) {
-      raftLog.open(cm, RaftServerConstants.INVALID_LOG_INDEX);
-      // append entries to the raftlog
-      entries.forEach(raftLog::appendEntry);
-      raftLog.logSync();
-    }
-
-    // append entries whose first 100 entries are the same with existing log,
-    // and the next 100 are with different term
-    SegmentRange r1 = new SegmentRange(550, 599, 2, false);
-    SegmentRange r2 = new SegmentRange(600, 649, 3, false);
-    SegmentRange r3 = new SegmentRange(650, 749, 10, false);
-    List<LogEntryProto> newEntries = prepareLogEntries(
-        Arrays.asList(r1, r2, r3), null);
-
-    try (SegmentedRaftLog raftLog =
-             new SegmentedRaftLog(peerId, null, storage, -1, properties)) {
-      raftLog.open(cm, RaftServerConstants.INVALID_LOG_INDEX);
-      raftLog.append(newEntries.toArray(new LogEntryProto[newEntries.size()]));
-      raftLog.logSync();
-
-      checkEntries(raftLog, entries, 0, 650);
-      checkEntries(raftLog, newEntries, 100, 100);
-      Assert.assertEquals(newEntries.get(newEntries.size() - 1),
-          raftLog.getLastEntry());
-      Assert.assertEquals(newEntries.get(newEntries.size() - 1).getIndex(),
-          raftLog.getLatestFlushedIndex());
-    }
-
-    // load the raftlog again and check
-    try (SegmentedRaftLog raftLog =
-             new SegmentedRaftLog(peerId, null, storage, -1, properties)) {
-      raftLog.open(cm, RaftServerConstants.INVALID_LOG_INDEX);
-      checkEntries(raftLog, entries, 0, 650);
-      checkEntries(raftLog, newEntries, 100, 100);
-      Assert.assertEquals(newEntries.get(newEntries.size() - 1),
-          raftLog.getLastEntry());
-      Assert.assertEquals(newEntries.get(newEntries.size() - 1).getIndex(),
-          raftLog.getLatestFlushedIndex());
-
-      RaftLogCache cache = raftLog.getRaftLogCache();
-      Assert.assertEquals(5, cache.getNumOfSegments());
-    }
-  }
-}

http://git-wip-us.apache.org/repos/asf/incubator-ratis/blob/7e71a2e0/raft-server/src/test/java/org/apache/raft/statemachine/RaftSnapshotBaseTest.java
----------------------------------------------------------------------
diff --git a/raft-server/src/test/java/org/apache/raft/statemachine/RaftSnapshotBaseTest.java b/raft-server/src/test/java/org/apache/raft/statemachine/RaftSnapshotBaseTest.java
deleted file mode 100644
index fbdcb8b..0000000
--- a/raft-server/src/test/java/org/apache/raft/statemachine/RaftSnapshotBaseTest.java
+++ /dev/null
@@ -1,215 +0,0 @@
-/**
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- *     http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-package org.apache.raft.statemachine;
-
-import org.apache.log4j.Level;
-import org.apache.raft.MiniRaftCluster;
-import org.apache.raft.RaftTestUtil;
-import org.apache.raft.RaftTestUtil.SimpleMessage;
-import org.apache.raft.client.RaftClient;
-import org.apache.raft.conf.RaftProperties;
-import org.apache.raft.protocol.RaftClientReply;
-import org.apache.raft.protocol.SetConfigurationRequest;
-import org.apache.raft.server.RaftServer;
-import org.apache.raft.server.impl.RaftServerImpl;
-import org.apache.raft.server.impl.RaftServerTestUtil;
-import org.apache.raft.server.simulation.RequestHandler;
-import org.apache.raft.server.storage.RaftLog;
-import org.apache.raft.server.storage.RaftStorageDirectory;
-import org.apache.raft.server.storage.RaftStorageDirectory.LogPathAndIndex;
-import org.apache.raft.shaded.proto.RaftProtos.LogEntryProto;
-import org.apache.raft.util.FileUtils;
-import org.apache.raft.util.RaftUtils;
-import org.junit.After;
-import org.junit.Assert;
-import org.junit.Before;
-import org.junit.Test;
-import org.slf4j.Logger;
-import org.slf4j.LoggerFactory;
-
-import java.io.File;
-import java.io.IOException;
-import java.util.ArrayList;
-import java.util.List;
-
-import static org.apache.raft.server.RaftServerConfigKeys.RAFT_SERVER_AUTO_SNAPSHOT_ENABLED_KEY;
-import static org.apache.raft.server.RaftServerConfigKeys.RAFT_SERVER_SNAPSHOT_TRIGGER_THRESHOLD_KEY;
-import static org.apache.raft.server.impl.RaftServerConstants.DEFAULT_SEQNUM;
-
-public abstract class RaftSnapshotBaseTest {
-  static {
-    RaftUtils.setLogLevel(RaftServerImpl.LOG, Level.DEBUG);
-    RaftUtils.setLogLevel(RaftLog.LOG, Level.DEBUG);
-    RaftUtils.setLogLevel(RequestHandler.LOG, Level.DEBUG);
-    RaftUtils.setLogLevel(RaftClient.LOG, Level.DEBUG);
-  }
-
-  static final Logger LOG = LoggerFactory.getLogger(RaftSnapshotBaseTest.class);
-  private static final int SNAPSHOT_TRIGGER_THRESHOLD = 10;
-
-  static File getSnapshotFile(MiniRaftCluster cluster, int i) {
-    final RaftServerImpl leader = cluster.getLeader();
-    final SimpleStateMachine4Testing sm = SimpleStateMachine4Testing.get(leader);
-    return sm.getStateMachineStorage().getSnapshotFile(
-        leader.getState().getCurrentTerm(), i);
-  }
-
-  static void assertLeaderContent(MiniRaftCluster cluster)
-      throws InterruptedException {
-    final RaftServerImpl leader = RaftTestUtil.waitForLeader(cluster);
-    Assert.assertEquals(SNAPSHOT_TRIGGER_THRESHOLD * 2,
-        leader.getState().getLog().getLastCommittedIndex());
-    final LogEntryProto[] entries = SimpleStateMachine4Testing.get(leader).getContent();
-
-    for (int i = 1; i < SNAPSHOT_TRIGGER_THRESHOLD * 2 - 1; i++) {
-      Assert.assertEquals(i+1, entries[i].getIndex());
-      Assert.assertArrayEquals(
-          new SimpleMessage("m" + i).getContent().toByteArray(),
-          entries[i].getSmLogEntry().getData().toByteArray());
-    }
-  }
-
-  private MiniRaftCluster cluster;
-
-  public abstract MiniRaftCluster initCluster(int numServer, RaftProperties prop)
-      throws IOException;
-
-  @Before
-  public void setup() throws IOException {
-    final RaftProperties prop = new RaftProperties();
-    prop.setClass(MiniRaftCluster.STATEMACHINE_CLASS_KEY,
-        SimpleStateMachine4Testing.class, StateMachine.class);
-    prop.setLong(RAFT_SERVER_SNAPSHOT_TRIGGER_THRESHOLD_KEY,
-        SNAPSHOT_TRIGGER_THRESHOLD);
-    prop.setBoolean(RAFT_SERVER_AUTO_SNAPSHOT_ENABLED_KEY, true);
-    this.cluster = initCluster(1, prop);
-    cluster.start();
-  }
-
-  @After
-  public void tearDown() {
-    if (cluster != null) {
-      cluster.shutdown();
-    }
-  }
-
-  /**
-   * Keep generating writing traffic and make sure snapshots are taken.
-   * We then restart the whole raft peer and check if it can correctly load
-   * snapshots + raft log.
-   */
-  @Test
-  public void testRestartPeer() throws Exception {
-    RaftTestUtil.waitForLeader(cluster);
-    final String leaderId = cluster.getLeader().getId();
-    int i = 0;
-    try(final RaftClient client = cluster.createClient("client", leaderId)) {
-      for (; i < SNAPSHOT_TRIGGER_THRESHOLD * 2 - 1; i++) {
-        RaftClientReply reply = client.send(new SimpleMessage("m" + i));
-        Assert.assertTrue(reply.isSuccess());
-      }
-    }
-
-    // wait for the snapshot to be done
-    final File snapshotFile = getSnapshotFile(cluster, i);
-
-    int retries = 0;
-    do {
-      Thread.sleep(1000);
-    } while (!snapshotFile.exists() && retries++ < 10);
-
-    Assert.assertTrue(snapshotFile + " does not exist", snapshotFile.exists());
-
-    // restart the peer and check if it can correctly load snapshot
-    cluster.restart(false);
-    try {
-      // 200 messages + two leader elections --> last committed = 201
-      assertLeaderContent(cluster);
-    } finally {
-      cluster.shutdown();
-    }
-  }
-
-  /**
-   * Basic test for install snapshot: start a one node cluster and let it
-   * generate a snapshot. Then delete the log and restart the node, and add more
-   * nodes as followers.
-   */
-  @Test
-  public void testBasicInstallSnapshot() throws Exception {
-    List<LogPathAndIndex> logs = new ArrayList<>();
-    try {
-      RaftTestUtil.waitForLeader(cluster);
-      final String leaderId = cluster.getLeader().getId();
-
-      int i = 0;
-      try(final RaftClient client = cluster.createClient("client", leaderId)) {
-        for (; i < SNAPSHOT_TRIGGER_THRESHOLD * 2 - 1; i++) {
-          RaftClientReply reply = client.send(new SimpleMessage("m" + i));
-          Assert.assertTrue(reply.isSuccess());
-        }
-      }
-
-      // wait for the snapshot to be done
-      RaftStorageDirectory storageDirectory = cluster.getLeader().getState()
-          .getStorage().getStorageDir();
-      final File snapshotFile = getSnapshotFile(cluster, i);
-      logs = storageDirectory.getLogSegmentFiles();
-
-      int retries = 0;
-      do {
-        Thread.sleep(1000);
-      } while (!snapshotFile.exists() && retries++ < 10);
-
-      Assert.assertTrue(snapshotFile + " does not exist", snapshotFile.exists());
-    } finally {
-      cluster.shutdown();
-    }
-
-    // delete the log segments from the leader
-    for (LogPathAndIndex path : logs) {
-      FileUtils.deleteFile(path.path.toFile());
-    }
-
-    // restart the peer
-    LOG.info("Restarting the cluster");
-    cluster.restart(false);
-    try {
-      assertLeaderContent(cluster);
-
-      // generate some more traffic
-      try(final RaftClient client = cluster.createClient("client",
-          cluster.getLeader().getId())) {
-        Assert.assertTrue(client.send(new SimpleMessage("test")).isSuccess());
-      }
-
-      // add two more peers
-      MiniRaftCluster.PeerChanges change = cluster.addNewPeers(
-          new String[]{"s3", "s4"}, true);
-      // trigger setConfiguration
-      SetConfigurationRequest request = new SetConfigurationRequest("client",
-          cluster.getLeader().getId(), DEFAULT_SEQNUM, change.allPeersInNewConf);
-      LOG.info("Start changing the configuration: {}", request);
-      cluster.getLeader().setConfiguration(request);
-
-      RaftServerTestUtil.waitAndCheckNewConf(cluster, change.allPeersInNewConf, 0, null);
-    } finally {
-      cluster.shutdown();
-    }
-  }
-}

http://git-wip-us.apache.org/repos/asf/incubator-ratis/blob/7e71a2e0/raft-server/src/test/java/org/apache/raft/statemachine/SimpleStateMachine4Testing.java
----------------------------------------------------------------------
diff --git a/raft-server/src/test/java/org/apache/raft/statemachine/SimpleStateMachine4Testing.java b/raft-server/src/test/java/org/apache/raft/statemachine/SimpleStateMachine4Testing.java
deleted file mode 100644
index b8dd3f3..0000000
--- a/raft-server/src/test/java/org/apache/raft/statemachine/SimpleStateMachine4Testing.java
+++ /dev/null
@@ -1,245 +0,0 @@
-/**
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- *     http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-package org.apache.raft.statemachine;
-
-import com.google.common.base.Preconditions;
-import org.apache.raft.RaftTestUtil.SimpleMessage;
-import org.apache.raft.conf.RaftProperties;
-import org.apache.raft.io.MD5Hash;
-import org.apache.raft.protocol.Message;
-import org.apache.raft.protocol.RaftClientReply;
-import org.apache.raft.protocol.RaftClientRequest;
-import org.apache.raft.server.impl.RaftServerConstants;
-import org.apache.raft.server.impl.RaftServerImpl;
-import org.apache.raft.server.impl.RaftServerTestUtil;
-import org.apache.raft.server.impl.ServerProtoUtils;
-import org.apache.raft.server.protocol.TermIndex;
-import org.apache.raft.server.storage.LogInputStream;
-import org.apache.raft.server.storage.LogOutputStream;
-import org.apache.raft.server.storage.RaftStorage;
-import org.apache.raft.shaded.proto.RaftProtos.LogEntryProto;
-import org.apache.raft.shaded.proto.RaftProtos.SMLogEntryProto;
-import org.apache.raft.util.Daemon;
-import org.apache.raft.util.LifeCycle;
-import org.apache.raft.util.MD5FileUtil;
-import org.slf4j.Logger;
-import org.slf4j.LoggerFactory;
-
-import java.io.File;
-import java.io.IOException;
-import java.util.ArrayList;
-import java.util.Collection;
-import java.util.Collections;
-import java.util.List;
-import java.util.concurrent.CompletableFuture;
-
-/**
- * A {@link StateMachine} implementation example that simply stores all the log
- * entries in a list. Mainly used for test.
- *
- * For snapshot it simply merges all the log segments together.
- */
-public class SimpleStateMachine4Testing extends BaseStateMachine {
-  static volatile int SNAPSHOT_THRESHOLD = 100;
-  static final Logger LOG = LoggerFactory.getLogger(SimpleStateMachine4Testing.class);
-  public static final String RAFT_TEST_SIMPLE_STATE_MACHINE_TAKE_SNAPSHOT_KEY
-      = "raft.test.simple.state.machine.take.snapshot";
-  public static final boolean RAFT_TEST_SIMPLE_STATE_MACHINE_TAKE_SNAPSHOT_DEFAULT = false;
-
-  public static SimpleStateMachine4Testing get(RaftServerImpl s) {
-    return (SimpleStateMachine4Testing)RaftServerTestUtil.getStateMachine(s);
-  }
-
-  private final List<LogEntryProto> list =
-      Collections.synchronizedList(new ArrayList<>());
-  private final Daemon checkpointer;
-  private final SimpleStateMachineStorage storage = new SimpleStateMachineStorage();
-  private final TermIndexTracker termIndexTracker = new TermIndexTracker();
-  private final RaftProperties properties = new RaftProperties();
-
-  private volatile boolean running = true;
-  private long endIndexLastCkpt = RaftServerConstants.INVALID_LOG_INDEX;
-
-  SimpleStateMachine4Testing() {
-    checkpointer = new Daemon(() -> {
-      while (running) {
-        try {
-          if (list.get(list.size() - 1).getIndex() - endIndexLastCkpt >=
-              SNAPSHOT_THRESHOLD) {
-            endIndexLastCkpt = takeSnapshot();
-          }
-          try {
-            Thread.sleep(1000);
-          } catch (InterruptedException ignored) {
-          }
-        } catch (IOException ioe) {
-          LOG.warn("Received IOException in Checkpointer", ioe);
-        }
-      }
-    });
-  }
-
-  @Override
-  public synchronized void initialize(String id, RaftProperties properties,
-      RaftStorage raftStorage) throws IOException {
-    LOG.info("Initializing " + getClass().getSimpleName() + ":" + id);
-    lifeCycle.startAndTransition(() -> {
-      super.initialize(id, properties, raftStorage);
-      storage.init(raftStorage);
-      loadSnapshot(storage.findLatestSnapshot());
-
-      if (properties.getBoolean(
-          RAFT_TEST_SIMPLE_STATE_MACHINE_TAKE_SNAPSHOT_KEY,
-          RAFT_TEST_SIMPLE_STATE_MACHINE_TAKE_SNAPSHOT_DEFAULT)) {
-        checkpointer.start();
-      }
-    });
-  }
-
-  @Override
-  public synchronized void pause() {
-    lifeCycle.transition(LifeCycle.State.PAUSING);
-    lifeCycle.transition(LifeCycle.State.PAUSED);
-  }
-
-  @Override
-  public synchronized void reinitialize(String id, RaftProperties properties,
-      RaftStorage storage) throws IOException {
-    LOG.info("Reinitializing " + getClass().getSimpleName() + ":" + id);
-    initialize(id, properties, storage);
-  }
-
-  @Override
-  public CompletableFuture<Message> applyTransaction(TransactionContext trx) {
-    LogEntryProto entry = trx.getLogEntry().get();
-    Preconditions.checkNotNull(entry);
-    list.add(entry);
-    termIndexTracker.update(ServerProtoUtils.toTermIndex(entry));
-    return CompletableFuture.completedFuture(
-        new SimpleMessage(entry.getIndex() + " OK"));
-  }
-
-  @Override
-  public long takeSnapshot() throws IOException {
-    TermIndex termIndex = termIndexTracker.getLatestTermIndex();
-    if (termIndex.getTerm() <= 0 || termIndex.getIndex() <= 0) {
-      return RaftServerConstants.INVALID_LOG_INDEX;
-    }
-    final long endIndex = termIndex.getIndex();
-
-    // TODO: snapshot should be written to a tmp file, then renamed
-    File snapshotFile = storage.getSnapshotFile(termIndex.getTerm(),
-        termIndex.getIndex());
-    LOG.debug("Taking a snapshot with t:{}, i:{}, file:{}", termIndex.getTerm(),
-        termIndex.getIndex(), snapshotFile);
-    try (LogOutputStream out = new LogOutputStream(snapshotFile, false, properties)) {
-      for (final LogEntryProto entry : list) {
-        if (entry.getIndex() > endIndex) {
-          break;
-        } else {
-          out.write(entry);
-        }
-      }
-      out.flush();
-    } catch (IOException e) {
-      LOG.warn("Failed to take snapshot", e);
-    }
-
-    try {
-      final MD5Hash digest = MD5FileUtil.computeMd5ForFile(snapshotFile);
-      MD5FileUtil.saveMD5File(snapshotFile, digest);
-    } catch (IOException e) {
-      LOG.warn("Hit IOException when computing MD5 for snapshot file "
-          + snapshotFile, e);
-    }
-
-    try {
-      this.storage.loadLatestSnapshot();
-    } catch (IOException e) {
-      LOG.warn("Hit IOException when loading latest snapshot for snapshot file "
-          + snapshotFile, e);
-    }
-    // TODO: purge log segments
-    return endIndex;
-  }
-
-  @Override
-  public SimpleStateMachineStorage getStateMachineStorage() {
-    return storage;
-  }
-
-  public synchronized long loadSnapshot(SingleFileSnapshotInfo snapshot)
-      throws IOException {
-    if (snapshot == null || !snapshot.getFile().getPath().toFile().exists()) {
-      LOG.info("The snapshot file {} does not exist",
-          snapshot == null ? null : snapshot.getFile());
-      return RaftServerConstants.INVALID_LOG_INDEX;
-    } else {
-      LOG.info("Loading snapshot with t:{}, i:{}, file:{}", snapshot.getTerm(),
-          snapshot.getIndex(), snapshot.getFile().getPath());
-      final long endIndex = snapshot.getIndex();
-      try (LogInputStream in = new LogInputStream(
-          snapshot.getFile().getPath().toFile(), 0, endIndex, false)) {
-        LogEntryProto entry;
-        while ((entry = in.nextEntry()) != null) {
-          list.add(entry);
-          termIndexTracker.update(ServerProtoUtils.toTermIndex(entry));
-        }
-      }
-      Preconditions.checkState(
-          !list.isEmpty() && endIndex == list.get(list.size() - 1).getIndex(),
-          "endIndex=%s, list=%s", endIndex, list);
-      this.endIndexLastCkpt = endIndex;
-      termIndexTracker.init(snapshot.getTermIndex());
-      this.storage.loadLatestSnapshot();
-      return endIndex;
-    }
-  }
-
-  @Override
-  public CompletableFuture<RaftClientReply> query(
-      RaftClientRequest request) {
-    return CompletableFuture.completedFuture(
-        new RaftClientReply(request, new SimpleMessage("query success")));
-  }
-
-  @Override
-  public TransactionContext startTransaction(RaftClientRequest request)
-      throws IOException {
-    return new TransactionContext(this, request, SMLogEntryProto.newBuilder()
-        .setData(request.getMessage().getContent())
-        .build());
-  }
-
-  @Override
-  public void notifyNotLeader(Collection<TransactionContext> pendingEntries) {
-    // do nothing
-  }
-
-  @Override
-  public void close() {
-    lifeCycle.checkStateAndClose(() -> {
-      running = false;
-      checkpointer.interrupt();
-    });
-  }
-
-  public LogEntryProto[] getContent() {
-    return list.toArray(new LogEntryProto[list.size()]);
-  }
-}

http://git-wip-us.apache.org/repos/asf/incubator-ratis/blob/7e71a2e0/raft-server/src/test/java/org/apache/raft/statemachine/TermIndexTracker.java
----------------------------------------------------------------------
diff --git a/raft-server/src/test/java/org/apache/raft/statemachine/TermIndexTracker.java b/raft-server/src/test/java/org/apache/raft/statemachine/TermIndexTracker.java
deleted file mode 100644
index b08fe11..0000000
--- a/raft-server/src/test/java/org/apache/raft/statemachine/TermIndexTracker.java
+++ /dev/null
@@ -1,66 +0,0 @@
-/**
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- *     http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-package org.apache.raft.statemachine;
-
-import com.google.common.base.Preconditions;
-import org.apache.raft.server.protocol.TermIndex;
-
-import static org.apache.raft.server.impl.RaftServerConstants.INVALID_LOG_INDEX;
-
-/**
- * Tracks the term index that is applied to the StateMachine for simple state machines with
- * no concurrent snapshoting capabilities.
- */
-class TermIndexTracker {
-  static final TermIndex INIT_TERMINDEX =
-      TermIndex.newTermIndex(INVALID_LOG_INDEX, INVALID_LOG_INDEX);
-
-  private TermIndex current = INIT_TERMINDEX;
-
-  //TODO: developer note: everything is synchronized for now for convenience.
-
-  /**
-   * Initialize the tracker with a term index (likely from a snapshot).
-   */
-  public synchronized void init(TermIndex termIndex) {
-    this.current = termIndex;
-  }
-
-  public synchronized void reset() {
-    init(INIT_TERMINDEX);
-  }
-
-  /**
-   * Update the tracker with a new TermIndex. It means that the StateMachine has
-   * this index in memory.
-   */
-  public synchronized void update(TermIndex termIndex) {
-    Preconditions.checkArgument(termIndex != null &&
-        termIndex.compareTo(current) >= 0);
-    this.current = termIndex;
-  }
-
-  /**
-   * Return latest term and index that is inserted to this tracker as an atomic
-   * entity.
-   */
-  public synchronized TermIndex getLatestTermIndex() {
-    return current;
-  }
-
-}

http://git-wip-us.apache.org/repos/asf/incubator-ratis/blob/7e71a2e0/raft-server/src/test/java/org/apache/raft/statemachine/TestStateMachine.java
----------------------------------------------------------------------
diff --git a/raft-server/src/test/java/org/apache/raft/statemachine/TestStateMachine.java b/raft-server/src/test/java/org/apache/raft/statemachine/TestStateMachine.java
deleted file mode 100644
index 546bfb8..0000000
--- a/raft-server/src/test/java/org/apache/raft/statemachine/TestStateMachine.java
+++ /dev/null
@@ -1,186 +0,0 @@
-/**
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- *     http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-package org.apache.raft.statemachine;
-
-import org.apache.log4j.Level;
-import org.apache.raft.MiniRaftCluster;
-import org.apache.raft.RaftTestUtil;
-import org.apache.raft.client.RaftClient;
-import org.apache.raft.conf.RaftProperties;
-import org.apache.raft.protocol.Message;
-import org.apache.raft.protocol.RaftClientRequest;
-import org.apache.raft.server.RaftServerConfigKeys;
-import org.apache.raft.server.impl.RaftServerImpl;
-import org.apache.raft.server.impl.RaftServerTestUtil;
-import org.apache.raft.server.simulation.MiniRaftClusterWithSimulatedRpc;
-import org.apache.raft.shaded.proto.RaftProtos.SMLogEntryProto;
-import org.apache.raft.util.RaftUtils;
-import org.junit.*;
-import org.junit.rules.Timeout;
-
-import java.io.IOException;
-import java.util.Collections;
-import java.util.List;
-import java.util.Optional;
-import java.util.concurrent.CompletableFuture;
-import java.util.concurrent.ConcurrentLinkedQueue;
-import java.util.concurrent.atomic.AtomicBoolean;
-import java.util.concurrent.atomic.AtomicLong;
-import java.util.concurrent.atomic.AtomicReference;
-import java.util.stream.Collectors;
-
-import static org.junit.Assert.*;
-
-/**
- * Test StateMachine related functionality
- */
-public class TestStateMachine {
-  static {
-    RaftUtils.setLogLevel(RaftServerImpl.LOG, Level.DEBUG);
-    RaftUtils.setLogLevel(RaftClient.LOG, Level.DEBUG);
-  }
-
-  public static final int NUM_SERVERS = 5;
-
-  private final RaftProperties properties = new RaftProperties();
-  {
-    // TODO: fix and run with in-memory log. It fails with NPE
-    properties.setBoolean(RaftServerConfigKeys.RAFT_SERVER_USE_MEMORY_LOG_KEY, false);
-  }
-
-  private MiniRaftClusterWithSimulatedRpc cluster;
-
-  @Rule
-  public Timeout globalTimeout = new Timeout(60 * 1000);
-
-  @Before
-  public void setup() throws IOException {
-  }
-
-  private void startCluster() {
-    cluster = new MiniRaftClusterWithSimulatedRpc(NUM_SERVERS, properties);
-    Assert.assertNull(getCluster().getLeader());
-    getCluster().start();
-  }
-
-  @After
-  public void tearDown() {
-    final MiniRaftCluster cluster = getCluster();
-    if (cluster != null) {
-      cluster.shutdown();
-    }
-  }
-
-  public MiniRaftClusterWithSimulatedRpc getCluster() {
-    return cluster;
-  }
-
-  public RaftProperties getProperties() {
-    return properties;
-  }
-
-  static class SMTransactionContext extends SimpleStateMachine4Testing {
-    public static SMTransactionContext get(RaftServerImpl s) {
-      return (SMTransactionContext)RaftServerTestUtil.getStateMachine(s);
-    }
-
-    AtomicReference<Throwable> throwable = new AtomicReference<>(null);
-    AtomicLong transactions = new AtomicLong(0);
-    AtomicBoolean isLeader = new AtomicBoolean(false);
-    AtomicLong numApplied = new AtomicLong(0);
-    ConcurrentLinkedQueue<Long> applied = new ConcurrentLinkedQueue<>();
-
-    @Override
-    public TransactionContext startTransaction(RaftClientRequest request) throws IOException {
-      // only leader will get this call
-      isLeader.set(true);
-      // send the next transaction id as the "context" from SM
-      return new TransactionContext(this, request, SMLogEntryProto.newBuilder()
-          .setData(request.getMessage().getContent())
-          .build(), transactions.incrementAndGet());
-    }
-
-    @Override
-    public CompletableFuture<Message> applyTransaction(TransactionContext trx) {
-      try {
-        assertTrue(trx.getLogEntry().isPresent());
-        assertTrue(trx.getSMLogEntry().isPresent());
-        Optional<Object> context = trx.getStateMachineContext();
-        if (isLeader.get()) {
-          assertTrue(trx.getClientRequest().isPresent());
-          assertTrue(context.isPresent());
-          assertTrue(context.get() instanceof Long);
-          Long val = (Long)context.get();
-          assertTrue(val <= transactions.get());
-          applied.add(val);
-        } else {
-          assertFalse(trx.getClientRequest().isPresent());
-          assertFalse(context.isPresent());
-        }
-        numApplied.incrementAndGet();
-      } catch (Throwable t) {
-        throwable.set(t);
-      }
-      return CompletableFuture.completedFuture(null);
-    }
-
-    void rethrowIfException() throws Throwable {
-      Throwable t = throwable.get();
-      if (t != null) {
-        throw t;
-      }
-    }
-  }
-
-  @Test
-  public void testTransactionContextIsPassedBack() throws Throwable {
-    // tests that the TrxContext set by the StateMachine in Leader is passed back to the SM
-    properties.setClass(
-        MiniRaftCluster.STATEMACHINE_CLASS_KEY,
-        SMTransactionContext.class, StateMachine.class);
-    startCluster();
-
-    int numTrx = 100;
-    final RaftTestUtil.SimpleMessage[] messages = RaftTestUtil.SimpleMessage.create(numTrx);
-    try(final RaftClient client = cluster.createClient("client", null)) {
-      for (RaftTestUtil.SimpleMessage message : messages) {
-        client.send(message);
-      }
-    }
-
-    // TODO: there eshould be a better way to ensure all data is replicated and applied
-    Thread.sleep(cluster.getMaxTimeout() + 100);
-
-    for (RaftServerImpl raftServer : cluster.getServers()) {
-      final SMTransactionContext sm = SMTransactionContext.get(raftServer);
-      sm.rethrowIfException();
-      assertEquals(numTrx, sm.numApplied.get());
-    }
-
-    // check leader
-    RaftServerImpl raftServer = cluster.getLeader();
-    // assert every transaction has obtained context in leader
-    final SMTransactionContext sm = SMTransactionContext.get(raftServer);
-    List<Long> ll = sm.applied.stream().collect(Collectors.toList());
-    Collections.sort(ll);
-    assertEquals(ll.toString(), ll.size(), numTrx);
-    for (int i=0; i < numTrx; i++) {
-      assertEquals(ll.toString(), Long.valueOf(i+1), ll.get(i));
-    }
-  }
-}

http://git-wip-us.apache.org/repos/asf/incubator-ratis/blob/7e71a2e0/raft-server/src/test/resources/log4j.properties
----------------------------------------------------------------------
diff --git a/raft-server/src/test/resources/log4j.properties b/raft-server/src/test/resources/log4j.properties
deleted file mode 100644
index ced0687..0000000
--- a/raft-server/src/test/resources/log4j.properties
+++ /dev/null
@@ -1,18 +0,0 @@
-#   Licensed 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.
-# log4j configuration used during build and unit tests
-
-log4j.rootLogger=info,stdout
-log4j.threshold=ALL
-log4j.appender.stdout=org.apache.log4j.ConsoleAppender
-log4j.appender.stdout.layout=org.apache.log4j.PatternLayout
-log4j.appender.stdout.layout.ConversionPattern=%d{ISO8601} %-5p %c{2} (%F:%M(%L)) - %m%n

http://git-wip-us.apache.org/repos/asf/incubator-ratis/blob/7e71a2e0/ratis-client/pom.xml
----------------------------------------------------------------------
diff --git a/ratis-client/pom.xml b/ratis-client/pom.xml
new file mode 100644
index 0000000..58d9817
--- /dev/null
+++ b/ratis-client/pom.xml
@@ -0,0 +1,54 @@
+<?xml version="1.0" encoding="UTF-8"?>
+<!--
+  Licensed 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. See accompanying LICENSE file.
+-->
+<project xmlns="http://maven.apache.org/POM/4.0.0"
+         xmlns:xsi="http://www.w3.org/2001/XMLSchema-instance"
+         xsi:schemaLocation="http://maven.apache.org/POM/4.0.0 http://maven.apache.org/xsd/maven-4.0.0.xsd">
+  <modelVersion>4.0.0</modelVersion>
+  <parent>
+    <artifactId>ratis-project-dist</artifactId>
+    <groupId>org.apache.ratis</groupId>
+    <version>1.0-SNAPSHOT</version>
+    <relativePath>../ratis-project-dist</relativePath>
+  </parent>
+
+  <artifactId>ratis-client</artifactId>
+  <name>Ratis Client</name>
+
+  <dependencies>
+    <dependency>
+      <artifactId>ratis-proto-shaded</artifactId>
+      <groupId>org.apache.ratis</groupId>
+      <scope>provided</scope>
+    </dependency>
+
+    <dependency>
+      <artifactId>ratis-common</artifactId>
+      <groupId>org.apache.ratis</groupId>
+      <scope>provided</scope>
+    </dependency>
+    
+    <dependency>
+      <groupId>org.slf4j</groupId>
+      <artifactId>slf4j-api</artifactId>
+      <scope>compile</scope>
+    </dependency>
+
+    <dependency>
+      <groupId>com.google.guava</groupId>
+      <artifactId>guava</artifactId>
+      <scope>compile</scope>
+    </dependency>
+  </dependencies>
+</project>

http://git-wip-us.apache.org/repos/asf/incubator-ratis/blob/7e71a2e0/ratis-client/src/main/java/org/apache/ratis/client/RaftClient.java
----------------------------------------------------------------------
diff --git a/ratis-client/src/main/java/org/apache/ratis/client/RaftClient.java b/ratis-client/src/main/java/org/apache/ratis/client/RaftClient.java
new file mode 100644
index 0000000..e4e0b84
--- /dev/null
+++ b/ratis-client/src/main/java/org/apache/ratis/client/RaftClient.java
@@ -0,0 +1,49 @@
+/**
+ * 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.ratis.client;
+
+import org.apache.ratis.protocol.Message;
+import org.apache.ratis.protocol.RaftClientReply;
+import org.apache.ratis.protocol.RaftPeer;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.io.Closeable;
+import java.io.IOException;
+
+/** A client who sends requests to a raft service. */
+public interface RaftClient extends Closeable {
+  Logger LOG = LoggerFactory.getLogger(RaftClient.class);
+  long DEFAULT_SEQNUM = 0;
+
+  /** @return the id of this client. */
+  String getId();
+
+  /**
+   * Send the given message to the raft service.
+   * The message may change the state of the service.
+   * For readonly messages, use {@link #sendReadOnly(Message)} instead.
+   */
+  RaftClientReply send(Message message) throws IOException;
+
+  /** Send the given readonly message to the raft service. */
+  RaftClientReply sendReadOnly(Message message) throws IOException;
+
+  /** Send set configuration request to the raft service. */
+  RaftClientReply setConfiguration(RaftPeer[] peersInNewConf) throws IOException;
+}

http://git-wip-us.apache.org/repos/asf/incubator-ratis/blob/7e71a2e0/ratis-client/src/main/java/org/apache/ratis/client/RaftClientConfigKeys.java
----------------------------------------------------------------------
diff --git a/ratis-client/src/main/java/org/apache/ratis/client/RaftClientConfigKeys.java b/ratis-client/src/main/java/org/apache/ratis/client/RaftClientConfigKeys.java
new file mode 100644
index 0000000..e1e1593
--- /dev/null
+++ b/ratis-client/src/main/java/org/apache/ratis/client/RaftClientConfigKeys.java
@@ -0,0 +1,23 @@
+/**
+ * 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.ratis.client;
+
+public interface RaftClientConfigKeys {
+  String RAFT_RPC_TIMEOUT_MS_KEY = "raft.rpc.timeout.ms";
+  int RAFT_RPC_TIMEOUT_MS_DEFAULT = 300;
+}



[38/54] [abbrv] incubator-ratis git commit: Renamed the packages from raft to ratis in preperation for Apache Incubation - Moved all java packages from org.apache.raft to org.apache.ratis. - Moved native package to org_apache_ratis, and native lib to l

Posted by ji...@apache.org.
http://git-wip-us.apache.org/repos/asf/incubator-ratis/blob/7e71a2e0/raft-server/src/main/java/org/apache/raft/server/impl/LeaderState.java
----------------------------------------------------------------------
diff --git a/raft-server/src/main/java/org/apache/raft/server/impl/LeaderState.java b/raft-server/src/main/java/org/apache/raft/server/impl/LeaderState.java
deleted file mode 100644
index fbbcb85..0000000
--- a/raft-server/src/main/java/org/apache/raft/server/impl/LeaderState.java
+++ /dev/null
@@ -1,582 +0,0 @@
-/**
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- *     http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-package org.apache.raft.server.impl;
-
-import com.google.common.base.Preconditions;
-import org.apache.raft.conf.RaftProperties;
-import org.apache.raft.protocol.*;
-import org.apache.raft.server.storage.RaftLog;
-import org.apache.raft.shaded.proto.RaftProtos.LeaderNoOp;
-import org.apache.raft.shaded.proto.RaftProtos.LogEntryProto;
-import org.apache.raft.statemachine.TransactionContext;
-import org.apache.raft.util.Daemon;
-import org.apache.raft.util.ProtoUtils;
-import org.apache.raft.util.Timestamp;
-import org.slf4j.Logger;
-
-import java.io.IOException;
-import java.util.*;
-import java.util.concurrent.ArrayBlockingQueue;
-import java.util.concurrent.BlockingQueue;
-import java.util.concurrent.CompletableFuture;
-import java.util.concurrent.TimeUnit;
-import java.util.stream.Collectors;
-
-import static org.apache.raft.server.RaftServerConfigKeys.*;
-import static org.apache.raft.server.impl.LeaderState.StateUpdateEventType.*;
-
-/**
- * States for leader only. It contains three different types of processors:
- * 1. RPC senders: each thread is appending log to a follower
- * 2. EventProcessor: a single thread updating the raft server's state based on
- *                    status of log appending response
- * 3. PendingRequestHandler: a handler sending back responses to clients when
- *                           corresponding log entries are committed
- */
-public class LeaderState {
-  private static final Logger LOG = RaftServerImpl.LOG;
-
-  enum StateUpdateEventType {
-    STEPDOWN, UPDATECOMMIT, STAGINGPROGRESS
-  }
-
-  enum BootStrapProgress {
-    NOPROGRESS, PROGRESSING, CAUGHTUP
-  }
-
-  static class StateUpdateEvent {
-    final StateUpdateEventType type;
-    final long newTerm;
-
-    StateUpdateEvent(StateUpdateEventType type, long newTerm) {
-      this.type = type;
-      this.newTerm = newTerm;
-    }
-  }
-
-  static final StateUpdateEvent UPDATE_COMMIT_EVENT =
-      new StateUpdateEvent(StateUpdateEventType.UPDATECOMMIT, -1);
-  static final StateUpdateEvent STAGING_PROGRESS_EVENT =
-      new StateUpdateEvent(StateUpdateEventType.STAGINGPROGRESS, -1);
-
-  private final RaftServerImpl server;
-  private final RaftLog raftLog;
-  private final long currentTerm;
-  private volatile ConfigurationStagingState stagingState;
-  private List<List<FollowerInfo>> voterLists;
-
-  /**
-   * The list of threads appending entries to followers.
-   * The list is protected by the RaftServer's lock.
-   */
-  private final List<LogAppender> senders;
-  private final BlockingQueue<StateUpdateEvent> eventQ;
-  private final EventProcessor processor;
-  private final PendingRequests pendingRequests;
-  private volatile boolean running = true;
-
-  private final int stagingCatchupGap;
-  private final int snapshotChunkMaxSize;
-  private final int syncInterval;
-
-  LeaderState(RaftServerImpl server, RaftProperties properties) {
-    this.server = server;
-
-    stagingCatchupGap = properties.getInt(
-        RAFT_SERVER_STAGING_CATCHUP_GAP_KEY,
-        RAFT_SERVER_STAGING_CATCHUP_GAP_DEFAULT);
-    snapshotChunkMaxSize = properties.getInt(
-        RAFT_SNAPSHOT_CHUNK_MAX_SIZE_KEY,
-        RAFT_SNAPSHOT_CHUNK_MAX_SIZE_DEFAULT);
-    syncInterval = properties.getInt(
-        RAFT_SERVER_RPC_SLEEP_TIME_MS_KEY,
-        RAFT_SERVER_RPC_SLEEP_TIME_MS_DEFAULT);
-
-    final ServerState state = server.getState();
-    this.raftLog = state.getLog();
-    this.currentTerm = state.getCurrentTerm();
-    eventQ = new ArrayBlockingQueue<>(4096);
-    processor = new EventProcessor();
-    pendingRequests = new PendingRequests(server);
-
-    final RaftConfiguration conf = server.getRaftConf();
-    Collection<RaftPeer> others = conf.getOtherPeers(state.getSelfId());
-    final Timestamp t = new Timestamp().addTimeMs(-server.getMaxTimeoutMs());
-    final long nextIndex = raftLog.getNextIndex();
-    senders = new ArrayList<>(others.size());
-    for (RaftPeer p : others) {
-      FollowerInfo f = new FollowerInfo(p, t, nextIndex, true);
-      senders.add(server.getLogAppenderFactory().getLogAppender(server, this, f));
-    }
-    voterLists = divideFollowers(conf);
-  }
-
-  void start() {
-    // In the beginning of the new term, replicate an empty entry in order
-    // to finally commit entries in the previous term.
-    // Also this message can help identify the last committed index when
-    // the leader peer is just started.
-    final LogEntryProto placeHolder = LogEntryProto.newBuilder()
-        .setTerm(server.getState().getCurrentTerm())
-        .setIndex(raftLog.getNextIndex())
-        .setNoOp(LeaderNoOp.newBuilder()).build();
-    raftLog.append(placeHolder);
-
-    processor.start();
-    startSenders();
-  }
-
-  private void startSenders() {
-    senders.forEach(Thread::start);
-  }
-
-  void stop() {
-    this.running = false;
-    // do not interrupt event processor since it may be in the middle of logSync
-    for (LogAppender sender : senders) {
-      sender.stopSender();
-      sender.interrupt();
-    }
-    try {
-      pendingRequests.sendNotLeaderResponses();
-    } catch (IOException e) {
-      LOG.warn("Caught exception in sendNotLeaderResponses", e);
-    }
-  }
-
-  void notifySenders() {
-    senders.forEach(LogAppender::notifyAppend);
-  }
-
-  boolean inStagingState() {
-    return stagingState != null;
-  }
-
-  ConfigurationStagingState getStagingState() {
-    return stagingState;
-  }
-
-  long getCurrentTerm() {
-    return currentTerm;
-  }
-
-  int getSnapshotChunkMaxSize() {
-    return snapshotChunkMaxSize;
-  }
-
-  int getSyncInterval() {
-    return syncInterval;
-  }
-
-  /**
-   * Start bootstrapping new peers
-   */
-  PendingRequest startSetConfiguration(SetConfigurationRequest request) {
-    Preconditions.checkState(running && !inStagingState());
-
-    RaftPeer[] peersInNewConf = request.getPeersInNewConf();
-    Collection<RaftPeer> peersToBootStrap = RaftConfiguration
-        .computeNewPeers(peersInNewConf, server.getRaftConf());
-
-    // add the request to the pending queue
-    final PendingRequest pending = pendingRequests.addConfRequest(request);
-
-    ConfigurationStagingState stagingState = new ConfigurationStagingState(
-        peersToBootStrap, new PeerConfiguration(Arrays.asList(peersInNewConf)));
-    Collection<RaftPeer> newPeers = stagingState.getNewPeers();
-    // set the staging state
-    this.stagingState = stagingState;
-
-    if (newPeers.isEmpty()) {
-      applyOldNewConf();
-    } else {
-      // update the LeaderState's sender list
-      addSenders(newPeers);
-    }
-    return pending;
-  }
-
-  PendingRequest addPendingRequest(long index, RaftClientRequest request,
-      TransactionContext entry) {
-    return pendingRequests.addPendingRequest(index, request, entry);
-  }
-
-  private void applyOldNewConf() {
-    final ServerState state = server.getState();
-    final RaftConfiguration current = server.getRaftConf();
-    final RaftConfiguration oldNewConf= stagingState.generateOldNewConf(current,
-        state.getLog().getNextIndex());
-    // apply the (old, new) configuration to log, and use it as the current conf
-    long index = state.getLog().append(state.getCurrentTerm(), oldNewConf);
-    updateConfiguration(index, oldNewConf);
-
-    this.stagingState = null;
-    notifySenders();
-  }
-
-  private void updateConfiguration(long logIndex, RaftConfiguration newConf) {
-    voterLists = divideFollowers(newConf);
-    server.getState().setRaftConf(logIndex, newConf);
-  }
-
-  /**
-   * After receiving a setConfiguration request, the leader should update its
-   * RpcSender list.
-   */
-  void addSenders(Collection<RaftPeer> newMembers) {
-    final Timestamp t = new Timestamp().addTimeMs(-server.getMaxTimeoutMs());
-    final long nextIndex = raftLog.getNextIndex();
-    for (RaftPeer peer : newMembers) {
-      FollowerInfo f = new FollowerInfo(peer, t, nextIndex, false);
-      LogAppender sender = server.getLogAppenderFactory()
-          .getLogAppender(server, this, f);
-      senders.add(sender);
-      sender.start();
-    }
-  }
-
-  /**
-   * Update the RpcSender list based on the current configuration
-   */
-  private void updateSenders(RaftConfiguration conf) {
-    Preconditions.checkState(conf.isStable() && !inStagingState());
-    Iterator<LogAppender> iterator = senders.iterator();
-    while (iterator.hasNext()) {
-      LogAppender sender = iterator.next();
-      if (!conf.containsInConf(sender.getFollower().getPeer().getId())) {
-        iterator.remove();
-        sender.stopSender();
-        sender.interrupt();
-      }
-    }
-  }
-
-  void submitUpdateStateEvent(StateUpdateEvent event) {
-    try {
-      eventQ.put(event);
-    } catch (InterruptedException e) {
-      LOG.info("Interrupted when adding event {} into the queue", event);
-    }
-  }
-
-  private void prepare() {
-    synchronized (server) {
-      if (running) {
-        final RaftConfiguration conf = server.getRaftConf();
-        if (conf.isTransitional() && server.getState().isConfCommitted()) {
-          // the configuration is in transitional state, and has been committed
-          // so it is time to generate and replicate (new) conf.
-          replicateNewConf();
-        }
-      }
-    }
-  }
-
-  /**
-   * The processor thread takes the responsibility to update the raft server's
-   * state, such as changing to follower, or updating the committed index.
-   */
-  private class EventProcessor extends Daemon {
-    @Override
-    public void run() {
-      // apply an empty message; check if necessary to replicate (new) conf
-      prepare();
-
-      while (running) {
-        try {
-          StateUpdateEvent event = eventQ.poll(server.getMaxTimeoutMs(),
-              TimeUnit.MILLISECONDS);
-          synchronized (server) {
-            if (running) {
-              handleEvent(event);
-            }
-          }
-          // the updated configuration does not need to be sync'ed here
-        } catch (InterruptedException e) {
-          final String s = server.getId() + " " + getClass().getSimpleName()
-              + " thread is interrupted ";
-          if (!running) {
-            LOG.info(s + " gracefully; server=" + server);
-          } else {
-            LOG.warn(s + " UNEXPECTEDLY; server=" + server, e);
-            throw new RuntimeException(e);
-          }
-        } catch (IOException e) {
-          LOG.warn("Failed to persist new votedFor/term.", e);
-          // the failure should happen while changing the state to follower
-          // thus the in-memory state should have been updated
-          Preconditions.checkState(!running);
-        }
-      }
-    }
-  }
-
-  private void handleEvent(StateUpdateEvent e) throws IOException {
-    if (e == null) {
-      if (inStagingState()) {
-        checkNewPeers();
-      }
-    } else {
-      if (e.type == STEPDOWN) {
-        server.changeToFollower(e.newTerm, true);
-      } else if (e.type == UPDATECOMMIT) {
-        updateLastCommitted();
-      } else if (e.type == STAGINGPROGRESS) {
-        checkNewPeers();
-      }
-    }
-  }
-
-  /**
-   * So far we use a simple implementation for catchup checking:
-   * 1. If the latest rpc time of the remote peer is before 3 * max_timeout,
-   *    the peer made no progress for that long. We should fail the whole
-   *    setConfiguration request.
-   * 2. If the peer's matching index is just behind for a small gap, and the
-   *    peer was updated recently (within max_timeout), declare the peer as
-   *    caught-up.
-   * 3. Otherwise the peer is making progressing. Keep waiting.
-   */
-  private BootStrapProgress checkProgress(FollowerInfo follower,
-      long committed) {
-    Preconditions.checkArgument(!follower.isAttendingVote());
-    final Timestamp progressTime = new Timestamp().addTimeMs(-server.getMaxTimeoutMs());
-    final Timestamp timeoutTime = new Timestamp().addTimeMs(-3*server.getMaxTimeoutMs());
-    if (follower.getLastRpcResponseTime().compareTo(timeoutTime) < 0) {
-      LOG.debug("{} detects a follower {} timeout for bootstrapping," +
-              " timeoutTime: {}", server.getId(), follower, timeoutTime);
-      return BootStrapProgress.NOPROGRESS;
-    } else if (follower.getMatchIndex() + stagingCatchupGap > committed
-        && follower.getLastRpcResponseTime().compareTo(progressTime) > 0) {
-      return BootStrapProgress.CAUGHTUP;
-    } else {
-      return BootStrapProgress.PROGRESSING;
-    }
-  }
-
-  private Collection<BootStrapProgress> checkAllProgress(long committed) {
-    Preconditions.checkState(inStagingState());
-    return senders.stream()
-        .filter(sender -> !sender.getFollower().isAttendingVote())
-        .map(sender -> checkProgress(sender.getFollower(), committed))
-        .collect(Collectors.toCollection(ArrayList::new));
-  }
-
-  private void checkNewPeers() {
-    if (!inStagingState()) {
-      // it is possible that the bootstrapping is done and we still have
-      // remaining STAGINGPROGRESS event to handle.
-      updateLastCommitted();
-    } else {
-      final long committedIndex = server.getState().getLog()
-          .getLastCommittedIndex();
-      Collection<BootStrapProgress> reports = checkAllProgress(committedIndex);
-      if (reports.contains(BootStrapProgress.NOPROGRESS)) {
-        LOG.debug("{} fails the setConfiguration request", server.getId());
-        stagingState.fail();
-      } else if (!reports.contains(BootStrapProgress.PROGRESSING)) {
-        // all caught up!
-        applyOldNewConf();
-        for (LogAppender sender : senders) {
-          sender.getFollower().startAttendVote();
-        }
-      }
-    }
-  }
-
-  boolean isBootStrappingPeer(String peerId) {
-    return inStagingState() && getStagingState().contains(peerId);
-  }
-
-  private void updateLastCommitted() {
-    final String selfId = server.getId();
-    final RaftConfiguration conf = server.getRaftConf();
-    long majorityInNewConf = computeLastCommitted(voterLists.get(0),
-        conf.containsInConf(selfId));
-    final long oldLastCommitted = raftLog.getLastCommittedIndex();
-    final LogEntryProto[] entriesToCommit;
-    if (!conf.isTransitional()) {
-      // copy the entries that may get committed out of the raftlog, to prevent
-      // the possible race that the log gets purged after the statemachine does
-      // a snapshot
-      entriesToCommit = raftLog.getEntries(oldLastCommitted + 1,
-          Math.max(majorityInNewConf, oldLastCommitted) + 1);
-      server.getState().updateStatemachine(majorityInNewConf, currentTerm);
-    } else { // configuration is in transitional state
-      long majorityInOldConf = computeLastCommitted(voterLists.get(1),
-          conf.containsInOldConf(selfId));
-      final long majority = Math.min(majorityInNewConf, majorityInOldConf);
-      entriesToCommit = raftLog.getEntries(oldLastCommitted + 1,
-          Math.max(majority, oldLastCommitted) + 1);
-      server.getState().updateStatemachine(majority, currentTerm);
-    }
-    checkAndUpdateConfiguration(entriesToCommit);
-  }
-
-  private boolean committedConf(LogEntryProto[] entries) {
-    final long currentCommitted = raftLog.getLastCommittedIndex();
-    for (LogEntryProto entry : entries) {
-      if (entry.getIndex() <= currentCommitted &&
-          ProtoUtils.isConfigurationLogEntry(entry)) {
-        return true;
-      }
-    }
-    return false;
-  }
-
-  private void checkAndUpdateConfiguration(LogEntryProto[] entriesToCheck) {
-    final RaftConfiguration conf = server.getRaftConf();
-    if (committedConf(entriesToCheck)) {
-      if (conf.isTransitional()) {
-        replicateNewConf();
-      } else { // the (new) log entry has been committed
-        LOG.debug("{} sends success to setConfiguration request", server.getId());
-        pendingRequests.replySetConfiguration();
-        // if the leader is not included in the current configuration, step down
-        if (!conf.containsInConf(server.getId())) {
-          LOG.info("{} is not included in the new configuration {}. Step down.",
-              server.getId(), conf);
-          try {
-            // leave some time for all RPC senders to send out new conf entry
-            Thread.sleep(server.getMinTimeoutMs());
-          } catch (InterruptedException ignored) {
-          }
-          // the pending request handler will send NotLeaderException for
-          // pending client requests when it stops
-          server.close();
-        }
-      }
-    }
-  }
-
-  /**
-   * when the (old, new) log entry has been committed, should replicate (new):
-   * 1) append (new) to log
-   * 2) update conf to (new)
-   * 3) update RpcSenders list
-   * 4) start replicating the log entry
-   */
-  private void replicateNewConf() {
-    final RaftConfiguration conf = server.getRaftConf();
-    final RaftConfiguration newConf = RaftConfiguration.newBuilder()
-        .setConf(conf)
-        .setLogEntryIndex(raftLog.getNextIndex())
-        .build();
-    // stop the LogAppender if the corresponding follower is no longer in the conf
-    updateSenders(newConf);
-    long index = raftLog.append(server.getState().getCurrentTerm(), newConf);
-    updateConfiguration(index, newConf);
-    notifySenders();
-  }
-
-  private long computeLastCommitted(List<FollowerInfo> followers,
-      boolean includeSelf) {
-    final int length = includeSelf ? followers.size() + 1 : followers.size();
-    final long[] indices = new long[length];
-    for (int i = 0; i < followers.size(); i++) {
-      indices[i] = followers.get(i).getMatchIndex();
-    }
-    if (includeSelf) {
-      // note that we also need to wait for the local disk I/O
-      indices[length - 1] = raftLog.getLatestFlushedIndex();
-    }
-
-    Arrays.sort(indices);
-    return indices[(indices.length - 1) / 2];
-  }
-
-  private List<List<FollowerInfo>> divideFollowers(RaftConfiguration conf) {
-    List<List<FollowerInfo>> lists = new ArrayList<>(2);
-    List<FollowerInfo> listForNew = senders.stream()
-        .filter(sender -> conf.containsInConf(sender.getFollower().getPeer().getId()))
-        .map(LogAppender::getFollower)
-        .collect(Collectors.toList());
-    lists.add(listForNew);
-    if (conf.isTransitional()) {
-      List<FollowerInfo> listForOld = senders.stream()
-          .filter(sender -> conf.containsInOldConf(sender.getFollower().getPeer().getId()))
-          .map(LogAppender::getFollower)
-          .collect(Collectors.toList());
-      lists.add(listForOld);
-    }
-    return lists;
-  }
-
-  PendingRequest returnNoConfChange(SetConfigurationRequest r) {
-    PendingRequest pending = new PendingRequest(r);
-    pending.setSuccessReply(null);
-    return pending;
-  }
-
-  void replyPendingRequest(long logIndex, CompletableFuture<Message> message) {
-    pendingRequests.replyPendingRequest(logIndex, message);
-  }
-
-  TransactionContext getTransactionContext(long index) {
-    return pendingRequests.getTransactionContext(index);
-  }
-
-  private class ConfigurationStagingState {
-    private final Map<String, RaftPeer> newPeers;
-    private final PeerConfiguration newConf;
-
-    ConfigurationStagingState(Collection<RaftPeer> newPeers,
-        PeerConfiguration newConf) {
-      Map<String, RaftPeer> map = new HashMap<>();
-      for (RaftPeer peer : newPeers) {
-        map.put(peer.getId(), peer);
-      }
-      this.newPeers = Collections.unmodifiableMap(map);
-      this.newConf = newConf;
-    }
-
-    RaftConfiguration generateOldNewConf(RaftConfiguration current,
-        long logIndex) {
-      return RaftConfiguration.newBuilder()
-          .setConf(newConf)
-          .setOldConf(current)
-          .setLogEntryIndex(logIndex)
-          .build();
-    }
-
-    Collection<RaftPeer> getNewPeers() {
-      return newPeers.values();
-    }
-
-    boolean contains(String peerId) {
-      return newPeers.containsKey(peerId);
-    }
-
-    void fail() {
-      Iterator<LogAppender> iterator = senders.iterator();
-      while (iterator.hasNext()) {
-        LogAppender sender = iterator.next();
-        if (!sender.getFollower().isAttendingVote()) {
-          iterator.remove();
-          sender.stopSender();
-          sender.interrupt();
-        }
-      }
-      LeaderState.this.stagingState = null;
-      // send back failure response to client's request
-      pendingRequests.failSetConfiguration(
-          new ReconfigurationTimeoutException("Fail to set configuration "
-              + newConf + ". Timeout when bootstrapping new peers."));
-    }
-  }
-}

http://git-wip-us.apache.org/repos/asf/incubator-ratis/blob/7e71a2e0/raft-server/src/main/java/org/apache/raft/server/impl/LogAppender.java
----------------------------------------------------------------------
diff --git a/raft-server/src/main/java/org/apache/raft/server/impl/LogAppender.java b/raft-server/src/main/java/org/apache/raft/server/impl/LogAppender.java
deleted file mode 100644
index cf613ca..0000000
--- a/raft-server/src/main/java/org/apache/raft/server/impl/LogAppender.java
+++ /dev/null
@@ -1,480 +0,0 @@
-/**
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- *     http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-package org.apache.raft.server.impl;
-
-import com.google.common.base.Preconditions;
-import com.google.common.collect.Lists;
-import org.apache.commons.io.IOUtils;
-import org.apache.raft.server.impl.LeaderState.StateUpdateEventType;
-import org.apache.raft.server.protocol.TermIndex;
-import org.apache.raft.server.storage.FileInfo;
-import org.apache.raft.server.storage.RaftLog;
-import org.apache.raft.shaded.com.google.protobuf.ByteString;
-import org.apache.raft.shaded.proto.RaftProtos.*;
-import org.apache.raft.statemachine.SnapshotInfo;
-import org.apache.raft.util.Daemon;
-import org.apache.raft.util.ProtoUtils;
-import org.apache.raft.util.Timestamp;
-import org.slf4j.Logger;
-
-import java.io.File;
-import java.io.FileInputStream;
-import java.io.IOException;
-import java.io.InterruptedIOException;
-import java.nio.file.Path;
-import java.util.*;
-
-import static org.apache.raft.server.RaftServerConfigKeys.*;
-import static org.apache.raft.server.impl.RaftServerConstants.INVALID_LOG_INDEX;
-
-/**
- * A daemon thread appending log entries to a follower peer.
- */
-public class LogAppender extends Daemon {
-  public static final Logger LOG = RaftServerImpl.LOG;
-
-  protected final RaftServerImpl server;
-  private final LeaderState leaderState;
-  protected final RaftLog raftLog;
-  protected final FollowerInfo follower;
-  private final int maxBufferSize;
-  private final boolean batchSending;
-  private final LogEntryBuffer buffer;
-  private final long leaderTerm;
-
-  private volatile boolean sending = true;
-
-  public LogAppender(RaftServerImpl server, LeaderState leaderState, FollowerInfo f) {
-    this.follower = f;
-    this.server = server;
-    this.leaderState = leaderState;
-    this.raftLog = server.getState().getLog();
-    this.maxBufferSize = server.getProperties().getInt(
-        RAFT_SERVER_LOG_APPENDER_BUFFER_CAPACITY_KEY,
-        RAFT_SERVER_LOG_APPENDER_BUFFER_CAPACITY_DEFAULT);
-    this.batchSending = server.getProperties().getBoolean(
-        RAFT_SERVER_LOG_APPENDER_BATCH_ENABLED_KEY,
-        RAFT_SERVER_LOG_APPENDER_BATCH_ENABLED_DEFAULT);
-    this.buffer = new LogEntryBuffer();
-    this.leaderTerm = server.getState().getCurrentTerm();
-  }
-
-  @Override
-  public String toString() {
-    return getClass().getSimpleName() + "(" + server.getId() + " -> " +
-        follower.getPeer().getId() + ")";
-  }
-
-  @Override
-  public void run() {
-    try {
-      checkAndSendAppendEntries();
-    } catch (InterruptedException | InterruptedIOException e) {
-      LOG.info(this + " was interrupted: " + e);
-    }
-  }
-
-  protected boolean isAppenderRunning() {
-    return sending;
-  }
-
-  public void stopSender() {
-    this.sending = false;
-  }
-
-  public FollowerInfo getFollower() {
-    return follower;
-  }
-
-  /**
-   * A buffer for log entries with size limitation.
-   */
-  private class LogEntryBuffer {
-    private final List<LogEntryProto> buf = new ArrayList<>();
-    private int totalSize = 0;
-
-    void addEntry(LogEntryProto entry) {
-      buf.add(entry);
-      totalSize += entry.getSerializedSize();
-    }
-
-    boolean isFull() {
-      return totalSize >= maxBufferSize;
-    }
-
-    boolean isEmpty() {
-      return buf.isEmpty();
-    }
-
-    AppendEntriesRequestProto getAppendRequest(TermIndex previous) {
-      final AppendEntriesRequestProto request = server
-          .createAppendEntriesRequest(leaderTerm, follower.getPeer().getId(),
-              previous, buf, !follower.isAttendingVote());
-      buf.clear();
-      totalSize = 0;
-      return request;
-    }
-
-    int getPendingEntryNum() {
-      return buf.size();
-    }
-  }
-
-  private TermIndex getPrevious() {
-    TermIndex previous = ServerProtoUtils.toTermIndex(
-        raftLog.get(follower.getNextIndex() - 1));
-    if (previous == null) {
-      // if previous is null, nextIndex must be equal to the log start
-      // index (otherwise we will install snapshot).
-      Preconditions.checkState(follower.getNextIndex() == raftLog.getStartIndex(),
-          "follower's next index %s, local log start index %s",
-          follower.getNextIndex(), raftLog.getStartIndex());
-      SnapshotInfo snapshot = server.getState().getLatestSnapshot();
-      previous = snapshot == null ? null : snapshot.getTermIndex();
-    }
-    return previous;
-  }
-
-  protected AppendEntriesRequestProto createRequest() {
-    final TermIndex previous = getPrevious();
-    final long leaderNext = raftLog.getNextIndex();
-    long next = follower.getNextIndex() + buffer.getPendingEntryNum();
-    boolean toSend = false;
-
-    if (leaderNext == next && !buffer.isEmpty()) {
-      // no new entries, then send out the entries in the buffer
-      toSend = true;
-    } else if (leaderNext > next) {
-      while (leaderNext > next && !buffer.isFull()) {
-        // stop adding entry once the buffer size is >= the max size
-        buffer.addEntry(raftLog.get(next++));
-      }
-      if (buffer.isFull() || !batchSending) {
-        // buffer is full or batch sending is disabled, send out a request
-        toSend = true;
-      }
-    }
-
-    if (toSend || shouldHeartbeat()) {
-      return buffer.getAppendRequest(previous);
-    }
-    return null;
-  }
-
-  /** Send an appendEntries RPC; retry indefinitely. */
-  private AppendEntriesReplyProto sendAppendEntriesWithRetries()
-      throws InterruptedException, InterruptedIOException {
-    int retry = 0;
-    AppendEntriesRequestProto request = null;
-    while (isAppenderRunning()) { // keep retrying for IOException
-      try {
-        if (request == null || request.getEntriesCount() == 0) {
-          request = createRequest();
-        }
-
-        if (request == null) {
-          LOG.trace("{} need not send AppendEntries now." +
-              " Wait for more entries.", server.getId());
-          return null;
-        } else if (!isAppenderRunning()) {
-          LOG.debug("LogAppender {} has been stopped. Skip the request.", this);
-          return null;
-        }
-
-        follower.updateLastRpcSendTime();
-        final AppendEntriesReplyProto r = server.getServerRpc()
-            .appendEntries(request);
-        follower.updateLastRpcResponseTime();
-
-        return r;
-      } catch (InterruptedIOException iioe) {
-        throw iioe;
-      } catch (IOException ioe) {
-        LOG.debug(this + ": failed to send appendEntries; retry " + retry++, ioe);
-      }
-      if (isAppenderRunning()) {
-        Thread.sleep(leaderState.getSyncInterval());
-      }
-    }
-    return null;
-  }
-
-  protected class SnapshotRequestIter
-      implements Iterable<InstallSnapshotRequestProto> {
-    private final SnapshotInfo snapshot;
-    private final List<FileInfo> files;
-    private FileInputStream in;
-    private int fileIndex = 0;
-
-    private FileInfo currentFileInfo;
-    private byte[] currentBuf;
-    private long currentFileSize;
-    private long currentOffset = 0;
-    private int chunkIndex = 0;
-
-    private final String requestId;
-    private int requestIndex = 0;
-
-    public SnapshotRequestIter(SnapshotInfo snapshot, String requestId)
-        throws IOException {
-      this.snapshot = snapshot;
-      this.requestId = requestId;
-      this.files = snapshot.getFiles();
-      if (files.size() > 0) {
-        startReadFile();
-      }
-    }
-
-    private void startReadFile() throws IOException {
-      currentFileInfo = files.get(fileIndex);
-      File snapshotFile = currentFileInfo.getPath().toFile();
-      currentFileSize = snapshotFile.length();
-      final int bufLength =
-          (int) Math.min(leaderState.getSnapshotChunkMaxSize(), currentFileSize);
-      currentBuf = new byte[bufLength];
-      currentOffset = 0;
-      chunkIndex = 0;
-      in = new FileInputStream(snapshotFile);
-    }
-
-    @Override
-    public Iterator<InstallSnapshotRequestProto> iterator() {
-      return new Iterator<InstallSnapshotRequestProto>() {
-        @Override
-        public boolean hasNext() {
-          return fileIndex < files.size();
-        }
-
-        @Override
-        public InstallSnapshotRequestProto next() {
-          if (fileIndex >= files.size()) {
-            throw new NoSuchElementException();
-          }
-          int targetLength = (int) Math.min(currentFileSize - currentOffset,
-              leaderState.getSnapshotChunkMaxSize());
-          FileChunkProto chunk;
-          try {
-            chunk = readFileChunk(currentFileInfo, in, currentBuf,
-                targetLength, currentOffset, chunkIndex);
-            boolean done = (fileIndex == files.size() - 1) &&
-                chunk.getDone();
-            InstallSnapshotRequestProto request =
-                server.createInstallSnapshotRequest(follower.getPeer().getId(),
-                    requestId, requestIndex++, snapshot,
-                    Lists.newArrayList(chunk), done);
-            currentOffset += targetLength;
-            chunkIndex++;
-
-            if (currentOffset >= currentFileSize) {
-              in.close();
-              fileIndex++;
-              if (fileIndex < files.size()) {
-                startReadFile();
-              }
-            }
-
-            return request;
-          } catch (IOException e) {
-            if (in != null) {
-              try {
-                in.close();
-              } catch (IOException ignored) {
-              }
-            }
-            LOG.warn("Got exception when preparing InstallSnapshot request", e);
-            throw new RuntimeException(e);
-          }
-        }
-      };
-    }
-  }
-
-  private FileChunkProto readFileChunk(FileInfo fileInfo,
-      FileInputStream in, byte[] buf, int length, long offset, int chunkIndex)
-      throws IOException {
-    FileChunkProto.Builder builder = FileChunkProto.newBuilder()
-        .setOffset(offset).setChunkIndex(chunkIndex);
-    IOUtils.readFully(in, buf, 0, length);
-    Path relativePath = server.getState().getStorage().getStorageDir()
-        .relativizeToRoot(fileInfo.getPath());
-    builder.setFilename(relativePath.toString());
-    builder.setDone(offset + length == fileInfo.getFileSize());
-    builder.setFileDigest(
-        ByteString.copyFrom(fileInfo.getFileDigest().getDigest()));
-    builder.setData(ByteString.copyFrom(buf, 0, length));
-    return builder.build();
-  }
-
-  private InstallSnapshotReplyProto installSnapshot(SnapshotInfo snapshot)
-      throws InterruptedException, InterruptedIOException {
-    String requestId = UUID.randomUUID().toString();
-    InstallSnapshotReplyProto reply = null;
-    try {
-      for (InstallSnapshotRequestProto request :
-          new SnapshotRequestIter(snapshot, requestId)) {
-        follower.updateLastRpcSendTime();
-        reply = server.getServerRpc().installSnapshot(request);
-        follower.updateLastRpcResponseTime();
-
-        if (!reply.getServerReply().getSuccess()) {
-          return reply;
-        }
-      }
-    } catch (InterruptedIOException iioe) {
-      throw iioe;
-    } catch (Exception ioe) {
-      LOG.warn(this + ": failed to install SnapshotInfo " + snapshot.getFiles(),
-          ioe);
-      return null;
-    }
-
-    if (reply != null) {
-      follower.updateMatchIndex(snapshot.getTermIndex().getIndex());
-      follower.updateNextIndex(snapshot.getTermIndex().getIndex() + 1);
-      LOG.info("{}: install snapshot-{} successfully on follower {}",
-          server.getId(), snapshot.getTermIndex().getIndex(), follower.getPeer());
-    }
-    return reply;
-  }
-
-  protected SnapshotInfo shouldInstallSnapshot() {
-    final long logStartIndex = raftLog.getStartIndex();
-    // we should install snapshot if the follower needs to catch up and:
-    // 1. there is no local log entry but there is snapshot
-    // 2. or the follower's next index is smaller than the log start index
-    if (follower.getNextIndex() < raftLog.getNextIndex()) {
-      SnapshotInfo snapshot = server.getState().getLatestSnapshot();
-      if (follower.getNextIndex() < logStartIndex ||
-          (logStartIndex == INVALID_LOG_INDEX && snapshot != null)) {
-        return snapshot;
-      }
-    }
-    return null;
-  }
-
-  /** Check and send appendEntries RPC */
-  private void checkAndSendAppendEntries()
-      throws InterruptedException, InterruptedIOException {
-    while (isAppenderRunning()) {
-      if (shouldSendRequest()) {
-        SnapshotInfo snapshot = shouldInstallSnapshot();
-        if (snapshot != null) {
-          LOG.info("{}: follower {}'s next index is {}," +
-              " log's start index is {}, need to install snapshot",
-              server.getId(), follower.getPeer(), follower.getNextIndex(),
-              raftLog.getStartIndex());
-
-          final InstallSnapshotReplyProto r = installSnapshot(snapshot);
-          if (r != null && r.getResult() == InstallSnapshotResult.NOT_LEADER) {
-            checkResponseTerm(r.getTerm());
-          } // otherwise if r is null, retry the snapshot installation
-        } else {
-          final AppendEntriesReplyProto r = sendAppendEntriesWithRetries();
-          if (r != null) {
-            handleReply(r);
-          }
-        }
-      }
-      if (isAppenderRunning() && !shouldAppendEntries(
-          follower.getNextIndex() + buffer.getPendingEntryNum())) {
-        final long waitTime = getHeartbeatRemainingTime(
-            follower.getLastRpcTime());
-        if (waitTime > 0) {
-          synchronized (this) {
-            wait(waitTime);
-          }
-        }
-      }
-    }
-  }
-
-  private void handleReply(AppendEntriesReplyProto reply) {
-    if (reply != null) {
-      switch (reply.getResult()) {
-        case SUCCESS:
-          final long oldNextIndex = follower.getNextIndex();
-          final long nextIndex = reply.getNextIndex();
-          if (nextIndex < oldNextIndex) {
-            throw new IllegalStateException("nextIndex=" + nextIndex
-                + " < oldNextIndex=" + oldNextIndex
-                + ", reply=" + ProtoUtils.toString(reply));
-          }
-
-          if (nextIndex > oldNextIndex) {
-            follower.updateMatchIndex(nextIndex - 1);
-            follower.updateNextIndex(nextIndex);
-            submitEventOnSuccessAppend();
-          }
-          break;
-        case NOT_LEADER:
-          // check if should step down
-          checkResponseTerm(reply.getTerm());
-          break;
-        case INCONSISTENCY:
-          follower.decreaseNextIndex(reply.getNextIndex());
-          break;
-        case UNRECOGNIZED:
-          LOG.warn("{} received UNRECOGNIZED AppendResult from {}",
-              server.getId(), follower.getPeer().getId());
-          break;
-      }
-    }
-  }
-
-  protected void submitEventOnSuccessAppend() {
-    LeaderState.StateUpdateEvent e = follower.isAttendingVote() ?
-        LeaderState.UPDATE_COMMIT_EVENT :
-        LeaderState.STAGING_PROGRESS_EVENT;
-    leaderState.submitUpdateStateEvent(e);
-  }
-
-  public synchronized void notifyAppend() {
-    this.notify();
-  }
-
-  /** Should the leader send appendEntries RPC to this follower? */
-  protected boolean shouldSendRequest() {
-    return shouldAppendEntries(follower.getNextIndex()) || shouldHeartbeat();
-  }
-
-  private boolean shouldAppendEntries(long followerIndex) {
-    return followerIndex < raftLog.getNextIndex();
-  }
-
-  private boolean shouldHeartbeat() {
-    return getHeartbeatRemainingTime(follower.getLastRpcTime()) <= 0;
-  }
-
-  /**
-   * @return the time in milliseconds that the leader should send a heartbeat.
-   */
-  protected long getHeartbeatRemainingTime(Timestamp lastTime) {
-    return server.getMinTimeoutMs() / 2 - lastTime.elapsedTimeMs();
-  }
-
-  protected void checkResponseTerm(long responseTerm) {
-    synchronized (server) {
-      if (isAppenderRunning() && follower.isAttendingVote()
-          && responseTerm > leaderState.getCurrentTerm()) {
-        leaderState.submitUpdateStateEvent(
-            new LeaderState.StateUpdateEvent(StateUpdateEventType.STEPDOWN,
-                responseTerm));
-      }
-    }
-  }
-}

http://git-wip-us.apache.org/repos/asf/incubator-ratis/blob/7e71a2e0/raft-server/src/main/java/org/apache/raft/server/impl/LogAppenderFactory.java
----------------------------------------------------------------------
diff --git a/raft-server/src/main/java/org/apache/raft/server/impl/LogAppenderFactory.java b/raft-server/src/main/java/org/apache/raft/server/impl/LogAppenderFactory.java
deleted file mode 100644
index d77faff..0000000
--- a/raft-server/src/main/java/org/apache/raft/server/impl/LogAppenderFactory.java
+++ /dev/null
@@ -1,31 +0,0 @@
-/**
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- *     http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-package org.apache.raft.server.impl;
-
-public interface LogAppenderFactory {
-  LogAppender getLogAppender(RaftServerImpl server, LeaderState state,
-                             FollowerInfo f);
-
-  class SynchronousLogAppenderFactory implements LogAppenderFactory {
-    @Override
-    public LogAppender getLogAppender(RaftServerImpl server, LeaderState state,
-                                      FollowerInfo f) {
-      return new LogAppender(server, state, f);
-    }
-  }
-}

http://git-wip-us.apache.org/repos/asf/incubator-ratis/blob/7e71a2e0/raft-server/src/main/java/org/apache/raft/server/impl/PeerConfiguration.java
----------------------------------------------------------------------
diff --git a/raft-server/src/main/java/org/apache/raft/server/impl/PeerConfiguration.java b/raft-server/src/main/java/org/apache/raft/server/impl/PeerConfiguration.java
deleted file mode 100644
index 774a0c5..0000000
--- a/raft-server/src/main/java/org/apache/raft/server/impl/PeerConfiguration.java
+++ /dev/null
@@ -1,90 +0,0 @@
-/**
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- *     http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-package org.apache.raft.server.impl;
-
-import com.google.common.base.Preconditions;
-import org.apache.raft.protocol.RaftPeer;
-
-import java.util.*;
-
-/**
- * The peer configuration of a raft cluster.
- *
- * The objects of this class are immutable.
- */
-class PeerConfiguration {
-  private final Map<String, RaftPeer> peers;
-
-  PeerConfiguration(Iterable<RaftPeer> peers) {
-    Preconditions.checkNotNull(peers);
-    Map<String, RaftPeer> map = new HashMap<>();
-    for(RaftPeer p : peers) {
-      map.put(p.getId(), p);
-    }
-    this.peers = Collections.unmodifiableMap(map);
-    Preconditions.checkState(!this.peers.isEmpty());
-  }
-
-  Collection<RaftPeer> getPeers() {
-    return Collections.unmodifiableCollection(peers.values());
-  }
-
-  int size() {
-    return peers.size();
-  }
-
-  @Override
-  public String toString() {
-    return peers.values().toString();
-  }
-
-  RaftPeer getPeer(String id) {
-    return peers.get(id);
-  }
-
-  boolean contains(String id) {
-    return peers.containsKey(id);
-  }
-
-  List<RaftPeer> getOtherPeers(String selfId) {
-    List<RaftPeer> others = new ArrayList<>();
-    for (Map.Entry<String, RaftPeer> entry : peers.entrySet()) {
-      if (!selfId.equals(entry.getValue().getId())) {
-        others.add(entry.getValue());
-      }
-    }
-    return others;
-  }
-
-  boolean hasMajority(Collection<String> others, String selfId) {
-    Preconditions.checkArgument(!others.contains(selfId));
-    int num = 0;
-    if (contains(selfId)) {
-      num++;
-    }
-    for (String other : others) {
-      if (contains(other)) {
-        num++;
-      }
-      if (num > size() / 2) {
-        return true;
-      }
-    }
-    return false;
-  }
-}

http://git-wip-us.apache.org/repos/asf/incubator-ratis/blob/7e71a2e0/raft-server/src/main/java/org/apache/raft/server/impl/PendingRequest.java
----------------------------------------------------------------------
diff --git a/raft-server/src/main/java/org/apache/raft/server/impl/PendingRequest.java b/raft-server/src/main/java/org/apache/raft/server/impl/PendingRequest.java
deleted file mode 100644
index 689566a..0000000
--- a/raft-server/src/main/java/org/apache/raft/server/impl/PendingRequest.java
+++ /dev/null
@@ -1,87 +0,0 @@
-/**
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- * <p>
- * http://www.apache.org/licenses/LICENSE-2.0
- * <p>
- * 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.raft.server.impl;
-
-import com.google.common.base.Preconditions;
-import org.apache.raft.protocol.Message;
-import org.apache.raft.protocol.RaftClientReply;
-import org.apache.raft.protocol.RaftClientRequest;
-import org.apache.raft.protocol.SetConfigurationRequest;
-import org.apache.raft.statemachine.TransactionContext;
-
-import java.util.concurrent.CompletableFuture;
-
-public class PendingRequest implements Comparable<PendingRequest> {
-  private final Long index;
-  private final RaftClientRequest request;
-  private final TransactionContext entry;
-  private final CompletableFuture<RaftClientReply> future;
-
-  PendingRequest(long index, RaftClientRequest request,
-                 TransactionContext entry) {
-    this.index = index;
-    this.request = request;
-    this.entry = entry;
-    this.future = new CompletableFuture<>();
-  }
-
-  PendingRequest(SetConfigurationRequest request) {
-    this(RaftServerConstants.INVALID_LOG_INDEX, request, null);
-  }
-
-  long getIndex() {
-    return index;
-  }
-
-  RaftClientRequest getRequest() {
-    return request;
-  }
-
-  public CompletableFuture<RaftClientReply> getFuture() {
-    return future;
-  }
-
-  TransactionContext getEntry() {
-    return entry;
-  }
-
-  synchronized void setException(Throwable e) {
-    Preconditions.checkArgument(e != null);
-    future.completeExceptionally(e);
-  }
-
-  synchronized void setReply(RaftClientReply r) {
-    Preconditions.checkArgument(r != null);
-    future.complete(r);
-  }
-
-  void setSuccessReply(Message message) {
-    setReply(new RaftClientReply(getRequest(), message));
-  }
-
-  @Override
-  public int compareTo(PendingRequest that) {
-    return Long.compare(this.index, that.index);
-  }
-
-  @Override
-  public String toString() {
-    return getClass().getSimpleName() + "(index=" + index
-        + ", request=" + request;
-  }
-}

http://git-wip-us.apache.org/repos/asf/incubator-ratis/blob/7e71a2e0/raft-server/src/main/java/org/apache/raft/server/impl/PendingRequests.java
----------------------------------------------------------------------
diff --git a/raft-server/src/main/java/org/apache/raft/server/impl/PendingRequests.java b/raft-server/src/main/java/org/apache/raft/server/impl/PendingRequests.java
deleted file mode 100644
index 32f127e..0000000
--- a/raft-server/src/main/java/org/apache/raft/server/impl/PendingRequests.java
+++ /dev/null
@@ -1,129 +0,0 @@
-/**
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- *     http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-package org.apache.raft.server.impl;
-
-import com.google.common.base.Preconditions;
-import org.apache.raft.protocol.*;
-import org.apache.raft.statemachine.TransactionContext;
-import org.slf4j.Logger;
-
-import java.io.IOException;
-import java.util.Collection;
-import java.util.concurrent.CompletableFuture;
-import java.util.concurrent.ConcurrentHashMap;
-import java.util.concurrent.ConcurrentMap;
-import java.util.stream.Collectors;
-
-class PendingRequests {
-  private static final Logger LOG = RaftServerImpl.LOG;
-
-  private PendingRequest pendingSetConf;
-  private final RaftServerImpl server;
-  private final ConcurrentMap<Long, PendingRequest> pendingRequests = new ConcurrentHashMap<>();
-  private PendingRequest last = null;
-
-  PendingRequests(RaftServerImpl server) {
-    this.server = server;
-  }
-
-  PendingRequest addPendingRequest(long index, RaftClientRequest request,
-      TransactionContext entry) {
-    // externally synced for now
-    Preconditions.checkArgument(!request.isReadOnly());
-    Preconditions.checkState(last == null || index == last.getIndex() + 1);
-    return add(index, request, entry);
-  }
-
-  private PendingRequest add(long index, RaftClientRequest request,
-      TransactionContext entry) {
-    final PendingRequest pending = new PendingRequest(index, request, entry);
-    pendingRequests.put(index, pending);
-    last = pending;
-    return pending;
-  }
-
-  PendingRequest addConfRequest(SetConfigurationRequest request) {
-    Preconditions.checkState(pendingSetConf == null);
-    pendingSetConf = new PendingRequest(request);
-    return pendingSetConf;
-  }
-
-  void replySetConfiguration() {
-    // we allow the pendingRequest to be null in case that the new leader
-    // commits the new configuration while it has not received the retry
-    // request from the client
-    if (pendingSetConf != null) {
-      // for setConfiguration we do not need to wait for statemachine. send back
-      // reply after it's committed.
-      pendingSetConf.setSuccessReply(null);
-      pendingSetConf = null;
-    }
-  }
-
-  void failSetConfiguration(RaftException e) {
-    Preconditions.checkState(pendingSetConf != null);
-    pendingSetConf.setException(e);
-    pendingSetConf = null;
-  }
-
-  TransactionContext getTransactionContext(long index) {
-    PendingRequest pendingRequest = pendingRequests.get(index);
-    // it is possible that the pendingRequest is null if this peer just becomes
-    // the new leader and commits transactions received by the previous leader
-    return pendingRequest != null ? pendingRequest.getEntry() : null;
-  }
-
-  void replyPendingRequest(long index, CompletableFuture<Message> messageFuture) {
-    final PendingRequest pending = pendingRequests.get(index);
-    if (pending != null) {
-      Preconditions.checkState(pending.getIndex() == index);
-
-      messageFuture.whenComplete((reply, exception) -> {
-        if (exception == null) {
-          pending.setSuccessReply(reply);
-        } else {
-          pending.setException(exception);
-        }
-      });
-    }
-  }
-
-  /**
-   * The leader state is stopped. Send NotLeaderException to all the pending
-   * requests since they have not got applied to the state machine yet.
-   */
-  void sendNotLeaderResponses() throws IOException {
-    LOG.info("{} sends responses before shutting down PendingRequestsHandler",
-        server.getId());
-
-    Collection<TransactionContext> pendingEntries = pendingRequests.values().stream()
-        .map(PendingRequest::getEntry).collect(Collectors.toList());
-    // notify the state machine about stepping down
-    server.getStateMachine().notifyNotLeader(pendingEntries);
-    pendingRequests.values().forEach(this::setNotLeaderException);
-    if (pendingSetConf != null) {
-      setNotLeaderException(pendingSetConf);
-    }
-  }
-
-  private void setNotLeaderException(PendingRequest pending) {
-    RaftClientReply reply = new RaftClientReply(pending.getRequest(),
-        server.generateNotLeaderException());
-    pending.setReply(reply);
-  }
-}

http://git-wip-us.apache.org/repos/asf/incubator-ratis/blob/7e71a2e0/raft-server/src/main/java/org/apache/raft/server/impl/RaftConfiguration.java
----------------------------------------------------------------------
diff --git a/raft-server/src/main/java/org/apache/raft/server/impl/RaftConfiguration.java b/raft-server/src/main/java/org/apache/raft/server/impl/RaftConfiguration.java
deleted file mode 100644
index 4879314..0000000
--- a/raft-server/src/main/java/org/apache/raft/server/impl/RaftConfiguration.java
+++ /dev/null
@@ -1,261 +0,0 @@
-/**
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- *     http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-package org.apache.raft.server.impl;
-
-import com.google.common.annotations.VisibleForTesting;
-import com.google.common.base.Preconditions;
-import org.apache.raft.protocol.RaftPeer;
-
-import java.util.*;
-import java.util.concurrent.ThreadLocalRandom;
-
-/**
- * The configuration of the raft cluster.
- *
- * The configuration is stable if there is no on-going peer change. Otherwise,
- * the configuration is transitional, i.e. in the middle of a peer change.
- *
- * The objects of this class are immutable.
- */
-public class RaftConfiguration {
-  /** Create a {@link Builder}. */
-  public static Builder newBuilder() {
-    return new Builder();
-  }
-
-  /** To build {@link RaftConfiguration} objects. */
-  public static class Builder {
-    private PeerConfiguration oldConf;
-    private PeerConfiguration conf;
-    private long logEntryIndex = RaftServerConstants.INVALID_LOG_INDEX;
-
-    private boolean forceStable = false;
-    private boolean forceTransitional = false;
-
-    private Builder() {}
-
-    public Builder setConf(PeerConfiguration conf) {
-      Preconditions.checkNotNull(conf);
-      Preconditions.checkState(this.conf == null, "conf is already set.");
-      this.conf = conf;
-      return this;
-    }
-
-    public Builder setConf(Iterable<RaftPeer> peers) {
-      return setConf(new PeerConfiguration(peers));
-    }
-
-    public Builder setConf(RaftPeer[] peers) {
-      return setConf(Arrays.asList(peers));
-    }
-
-    Builder setConf(RaftConfiguration transitionalConf) {
-      Preconditions.checkNotNull(transitionalConf);
-      Preconditions.checkState(transitionalConf.isTransitional());
-
-      Preconditions.checkState(!forceTransitional);
-      forceStable = true;
-      return setConf(transitionalConf.conf);
-    }
-
-
-    public Builder setOldConf(PeerConfiguration oldConf) {
-      Preconditions.checkNotNull(oldConf);
-      Preconditions.checkState(this.oldConf == null, "oldConf is already set.");
-      this.oldConf = oldConf;
-      return this;
-    }
-
-    public Builder setOldConf(Iterable<RaftPeer> oldPeers) {
-      return setOldConf(new PeerConfiguration(oldPeers));
-    }
-
-    public Builder setOldConf(RaftPeer[] oldPeers) {
-      return setOldConf(Arrays.asList(oldPeers));
-    }
-
-    Builder setOldConf(RaftConfiguration stableConf) {
-      Preconditions.checkNotNull(stableConf);
-      Preconditions.checkState(stableConf.isStable());
-
-      Preconditions.checkState(!forceStable);
-      forceTransitional = true;
-      return setOldConf(stableConf.conf);
-    }
-
-    public Builder setLogEntryIndex(long logEntryIndex) {
-      Preconditions.checkArgument(
-          logEntryIndex != RaftServerConstants.INVALID_LOG_INDEX);
-      Preconditions.checkState(
-          this.logEntryIndex == RaftServerConstants.INVALID_LOG_INDEX,
-          "logEntryIndex is already set.");
-      this.logEntryIndex = logEntryIndex;
-      return this;
-    }
-
-    /** Build a {@link RaftConfiguration}. */
-    public RaftConfiguration build() {
-      if (forceTransitional) {
-        Preconditions.checkState(oldConf != null);
-      }
-      if (forceStable) {
-        Preconditions.checkState(oldConf == null);
-      }
-      return new RaftConfiguration(conf, oldConf, logEntryIndex);
-    }
-  }
-
-  /** Non-null only if this configuration is transitional. */
-  private final PeerConfiguration oldConf;
-  /**
-   * The current peer configuration while this configuration is stable;
-   * or the new peer configuration while this configuration is transitional.
-   */
-  private final PeerConfiguration conf;
-
-  /** The index of the corresponding log entry for this configuration. */
-  private final long logEntryIndex;
-
-  private RaftConfiguration(PeerConfiguration conf, PeerConfiguration oldConf,
-      long logEntryIndex) {
-    Preconditions.checkNotNull(conf);
-    this.conf = conf;
-    this.oldConf = oldConf;
-    this.logEntryIndex = logEntryIndex;
-  }
-
-  /** Is this configuration transitional, i.e. in the middle of a peer change? */
-  boolean isTransitional() {
-    return oldConf != null;
-  }
-
-  /** Is this configuration stable, i.e. no on-going peer change? */
-  boolean isStable() {
-    return oldConf == null;
-  }
-
-  boolean containsInConf(String peerId) {
-    return conf.contains(peerId);
-  }
-
-  boolean containsInOldConf(String peerId) {
-    return oldConf != null && oldConf.contains(peerId);
-  }
-
-  boolean contains(String peerId) {
-    return containsInConf(peerId) && (oldConf == null || containsInOldConf(peerId));
-  }
-
-  /**
-   * @return the peer corresponding to the given id;
-   *         or return null if the peer is not in this configuration.
-   */
-  public RaftPeer getPeer(String id) {
-    if (id == null) {
-      return null;
-    }
-    RaftPeer peer = conf.getPeer(id);
-    if (peer != null) {
-      return peer;
-    } else if (oldConf != null) {
-      return oldConf.getPeer(id);
-    }
-    return null;
-  }
-
-  /** @return all the peers from the conf, and the old conf if it exists. */
-  public Collection<RaftPeer> getPeers() {
-    final Collection<RaftPeer> peers = new ArrayList<>(conf.getPeers());
-    if (oldConf != null) {
-      oldConf.getPeers().stream().filter(p -> !peers.contains(p))
-          .forEach(peers::add);
-    }
-    return peers;
-  }
-
-  /**
-   * @return all the peers other than the given self id from the conf,
-   *         and the old conf if it exists.
-   */
-  public Collection<RaftPeer> getOtherPeers(String selfId) {
-    Collection<RaftPeer> others = conf.getOtherPeers(selfId);
-    if (oldConf != null) {
-      oldConf.getOtherPeers(selfId).stream()
-          .filter(p -> !others.contains(p))
-          .forEach(others::add);
-    }
-    return others;
-  }
-
-  /** @return true if the self id together with the others are in the majority. */
-  boolean hasMajority(Collection<String> others, String selfId) {
-    Preconditions.checkArgument(!others.contains(selfId));
-    return conf.hasMajority(others, selfId) &&
-        (oldConf == null || oldConf.hasMajority(others, selfId));
-  }
-
-  @Override
-  public String toString() {
-    return conf + (oldConf != null ? "old:" + oldConf : "");
-  }
-
-  @VisibleForTesting
-  boolean hasNoChange(RaftPeer[] newMembers) {
-    if (!isStable() || conf.size() != newMembers.length) {
-      return false;
-    }
-    for (RaftPeer peer : newMembers) {
-      if (!conf.contains(peer.getId())) {
-        return false;
-      }
-    }
-    return true;
-  }
-
-  long getLogEntryIndex() {
-    return logEntryIndex;
-  }
-
-  static Collection<RaftPeer> computeNewPeers(RaftPeer[] newMembers,
-      RaftConfiguration old) {
-    List<RaftPeer> peers = new ArrayList<>();
-    for (RaftPeer p : newMembers) {
-      if (!old.containsInConf(p.getId())) {
-        peers.add(p);
-      }
-    }
-    return peers;
-  }
-
-  RaftPeer getRandomPeer(String exclusiveId) {
-    final List<RaftPeer> peers = conf.getOtherPeers(exclusiveId);
-    if (peers.isEmpty()) {
-      return null;
-    }
-    final int index = ThreadLocalRandom.current().nextInt(peers.size());
-    return peers.get(index);
-  }
-
-  Collection<RaftPeer> getPeersInOldConf() {
-    return oldConf != null ? oldConf.getPeers() : Collections.emptyList();
-  }
-
-  Collection<RaftPeer> getPeersInConf() {
-    return conf.getPeers();
-  }
-}

http://git-wip-us.apache.org/repos/asf/incubator-ratis/blob/7e71a2e0/raft-server/src/main/java/org/apache/raft/server/impl/RaftServerConstants.java
----------------------------------------------------------------------
diff --git a/raft-server/src/main/java/org/apache/raft/server/impl/RaftServerConstants.java b/raft-server/src/main/java/org/apache/raft/server/impl/RaftServerConstants.java
deleted file mode 100644
index 6634152..0000000
--- a/raft-server/src/main/java/org/apache/raft/server/impl/RaftServerConstants.java
+++ /dev/null
@@ -1,46 +0,0 @@
-/**
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- *     http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-package org.apache.raft.server.impl;
-
-import org.apache.raft.client.RaftClient;
-
-public interface RaftServerConstants {
-  long INVALID_LOG_INDEX = -1;
-  byte LOG_TERMINATE_BYTE = 0;
-  long DEFAULT_SEQNUM = RaftClient.DEFAULT_SEQNUM;
-
-  enum StartupOption {
-    FORMAT("format"),
-    REGULAR("regular");
-
-    private final String option;
-
-    StartupOption(String arg) {
-      this.option = arg;
-    }
-
-    public static StartupOption getOption(String arg) {
-      for (StartupOption s : StartupOption.values()) {
-        if (s.option.equals(arg)) {
-          return s;
-        }
-      }
-      return REGULAR;
-    }
-  }
-}


[41/54] [abbrv] incubator-ratis git commit: Renamed the packages from raft to ratis in preperation for Apache Incubation - Moved all java packages from org.apache.raft to org.apache.ratis. - Moved native package to org_apache_ratis, and native lib to l

Posted by ji...@apache.org.
http://git-wip-us.apache.org/repos/asf/incubator-ratis/blob/7e71a2e0/raft-grpc/src/main/java/org/apache/raft/grpc/server/GRpcLogAppender.java
----------------------------------------------------------------------
diff --git a/raft-grpc/src/main/java/org/apache/raft/grpc/server/GRpcLogAppender.java b/raft-grpc/src/main/java/org/apache/raft/grpc/server/GRpcLogAppender.java
deleted file mode 100644
index a8a39bb..0000000
--- a/raft-grpc/src/main/java/org/apache/raft/grpc/server/GRpcLogAppender.java
+++ /dev/null
@@ -1,415 +0,0 @@
-/**
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- *     http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-package org.apache.raft.grpc.server;
-
-import com.google.common.base.Preconditions;
-import org.apache.raft.grpc.RaftGRpcService;
-import org.apache.raft.grpc.RaftGrpcConfigKeys;
-import org.apache.raft.server.impl.FollowerInfo;
-import org.apache.raft.server.impl.LeaderState;
-import org.apache.raft.server.impl.LogAppender;
-import org.apache.raft.server.impl.RaftServerImpl;
-import org.apache.raft.shaded.io.grpc.Status;
-import org.apache.raft.shaded.io.grpc.stub.StreamObserver;
-import org.apache.raft.shaded.proto.RaftProtos.AppendEntriesReplyProto;
-import org.apache.raft.shaded.proto.RaftProtos.AppendEntriesRequestProto;
-import org.apache.raft.shaded.proto.RaftProtos.InstallSnapshotReplyProto;
-import org.apache.raft.shaded.proto.RaftProtos.InstallSnapshotRequestProto;
-import org.apache.raft.statemachine.SnapshotInfo;
-import org.apache.raft.util.CodeInjectionForTesting;
-
-import java.util.LinkedList;
-import java.util.Queue;
-import java.util.UUID;
-import java.util.concurrent.ConcurrentLinkedQueue;
-import java.util.concurrent.atomic.AtomicBoolean;
-
-import static org.apache.raft.grpc.RaftGRpcService.GRPC_SEND_SERVER_REQUEST;
-
-/**
- * A new log appender implementation using grpc bi-directional stream API.
- */
-public class GRpcLogAppender extends LogAppender {
-  private final RaftServerProtocolClient client;
-  private final Queue<AppendEntriesRequestProto> pendingRequests;
-  private final int maxPendingRequestsNum;
-  private volatile boolean firstResponseReceived = false;
-
-  private final AppendLogResponseHandler appendResponseHandler;
-  private final InstallSnapshotResponseHandler snapshotResponseHandler;
-
-  private volatile StreamObserver<AppendEntriesRequestProto> appendLogRequestObserver;
-  private StreamObserver<InstallSnapshotRequestProto> snapshotRequestObserver;
-
-  public GRpcLogAppender(RaftServerImpl server, LeaderState leaderState,
-                         FollowerInfo f) {
-    super(server, leaderState, f);
-
-    RaftGRpcService rpcService = (RaftGRpcService) server.getServerRpc();
-    client = rpcService.getRpcClient(f.getPeer());
-    maxPendingRequestsNum = server.getProperties().getInt(
-        RaftGrpcConfigKeys.RAFT_GRPC_LEADER_MAX_OUTSTANDING_APPENDS_KEY,
-        RaftGrpcConfigKeys.RAFT_GRPC_LEADER_MAX_OUTSTANDING_APPENDS_DEFAULT);
-    pendingRequests = new ConcurrentLinkedQueue<>();
-
-    appendResponseHandler = new AppendLogResponseHandler();
-    snapshotResponseHandler = new InstallSnapshotResponseHandler();
-  }
-
-  @Override
-  public void run() {
-    while (isAppenderRunning()) {
-      if (shouldSendRequest()) {
-        SnapshotInfo snapshot = shouldInstallSnapshot();
-        if (snapshot != null) {
-          installSnapshot(snapshot, snapshotResponseHandler);
-        } else {
-          // keep appending log entries or sending heartbeats
-          appendLog();
-        }
-      }
-
-      if (isAppenderRunning() && !shouldSendRequest()) {
-        // use lastSend time instead of lastResponse time
-        final long waitTime = getHeartbeatRemainingTime(
-            follower.getLastRpcTime());
-        if (waitTime > 0) {
-          synchronized (this) {
-            try {
-              LOG.debug("{} decides to wait {}ms before appending to {}",
-                  server.getId(), waitTime, follower.getPeer());
-              wait(waitTime);
-            } catch (InterruptedException ignored) {
-            }
-          }
-        }
-      }
-    }
-    appendLogRequestObserver.onCompleted();
-  }
-
-  private boolean shouldWait() {
-    return pendingRequests.size() >= maxPendingRequestsNum ||
-        shouldWaitForFirstResponse();
-  }
-
-  private void appendLog() {
-    if (appendLogRequestObserver == null) {
-      appendLogRequestObserver = client.appendEntries(appendResponseHandler);
-    }
-    AppendEntriesRequestProto pending = null;
-    final StreamObserver<AppendEntriesRequestProto> s;
-    synchronized (this) {
-      // if the queue's size >= maxSize, wait
-      while (isAppenderRunning() && shouldWait()) {
-        try {
-          LOG.debug("{} wait to send the next AppendEntries to {}",
-              server.getId(), follower.getPeer());
-          this.wait();
-        } catch (InterruptedException ignored) {
-        }
-      }
-
-      if (isAppenderRunning()) {
-        // prepare and enqueue the append request. note changes on follower's
-        // nextIndex and ops on pendingRequests should always be associated
-        // together and protected by the lock
-        pending = createRequest();
-        if (pending != null) {
-          Preconditions.checkState(pendingRequests.offer(pending));
-          updateNextIndex(pending);
-        }
-      }
-      s = appendLogRequestObserver;
-    }
-
-    if (pending != null && isAppenderRunning()) {
-      sendRequest(pending, s);
-    }
-  }
-
-  private void sendRequest(AppendEntriesRequestProto request,
-      StreamObserver<AppendEntriesRequestProto> s) {
-    CodeInjectionForTesting.execute(GRPC_SEND_SERVER_REQUEST, server.getId(),
-        null, request);
-
-    s.onNext(request);
-    follower.updateLastRpcSendTime();
-  }
-
-  private void updateNextIndex(AppendEntriesRequestProto request) {
-    final int count = request.getEntriesCount();
-    if (count > 0) {
-      follower.updateNextIndex(request.getEntries(count - 1).getIndex() + 1);
-    }
-  }
-
-  /**
-   * if this is the first append, wait for the response of the first append so
-   * that we can get the correct next index.
-   */
-  private boolean shouldWaitForFirstResponse() {
-    return pendingRequests.size() > 0 && !firstResponseReceived;
-  }
-
-  /**
-   * StreamObserver for handling responses from the follower
-   */
-  private class AppendLogResponseHandler
-      implements StreamObserver<AppendEntriesReplyProto> {
-    /**
-     * After receiving a appendEntries reply, do the following:
-     * 1. If the reply is success, update the follower's match index and submit
-     *    an event to leaderState
-     * 2. If the reply is NOT_LEADER, step down
-     * 3. If the reply is INCONSISTENCY, decrease the follower's next index
-     *    based on the response
-     */
-    @Override
-    public void onNext(AppendEntriesReplyProto reply) {
-      LOG.debug("{} received {} response from {}", server.getId(),
-          (!firstResponseReceived ? "the first" : "a"),
-          follower.getPeer());
-
-      // update the last rpc time
-      follower.updateLastRpcResponseTime();
-
-      if (!firstResponseReceived) {
-        firstResponseReceived = true;
-      }
-      switch (reply.getResult()) {
-        case SUCCESS:
-          onSuccess(reply);
-          break;
-        case NOT_LEADER:
-          onNotLeader(reply);
-          break;
-        case INCONSISTENCY:
-          onInconsistency(reply);
-          break;
-        default:
-          break;
-      }
-      notifyAppend();
-    }
-
-    /**
-     * for now we simply retry the first pending request
-     */
-    @Override
-    public void onError(Throwable t) {
-      if (!isAppenderRunning()) {
-        LOG.info("{} is stopped", GRpcLogAppender.this);
-        return;
-      }
-      LOG.warn("{} got error when appending entries to {}, exception: {}.",
-          server.getId(), follower.getPeer().getId(), t);
-
-      synchronized (this) {
-        final Status cause = Status.fromThrowable(t);
-        if (cause != null && cause.getCode() == Status.Code.INTERNAL) {
-          // TODO check other Status. Add sleep to avoid tight loop
-          LOG.debug("{} restarts Append call to {} due to error {}",
-              server.getId(), follower.getPeer(), t);
-          // recreate the StreamObserver
-          appendLogRequestObserver = client.appendEntries(appendResponseHandler);
-          // reset firstResponseReceived to false
-          firstResponseReceived = false;
-        }
-
-        // clear the pending requests queue and reset the next index of follower
-        AppendEntriesRequestProto request = pendingRequests.peek();
-        if (request != null) {
-          final long nextIndex = request.hasPreviousLog() ?
-              request.getPreviousLog().getIndex() + 1 : raftLog.getStartIndex();
-          clearPendingRequests(nextIndex);
-        }
-      }
-    }
-
-    @Override
-    public void onCompleted() {
-      LOG.info("{} stops appending log entries to follower {}", server.getId(),
-          follower);
-    }
-  }
-
-  private void clearPendingRequests(long newNextIndex) {
-    pendingRequests.clear();
-    follower.decreaseNextIndex(newNextIndex);
-  }
-
-  private void onSuccess(AppendEntriesReplyProto reply) {
-    AppendEntriesRequestProto request = pendingRequests.poll();
-    final long replyNextIndex = reply.getNextIndex();
-    Preconditions.checkNotNull(request,
-        "Got reply with next index %s but the pending queue is empty",
-        replyNextIndex);
-
-    if (request.getEntriesCount() == 0) {
-      Preconditions.checkState(!request.hasPreviousLog() ||
-              replyNextIndex - 1 == request.getPreviousLog().getIndex(),
-          "reply's next index is %s, request's previous is %s",
-          replyNextIndex, request.getPreviousLog());
-    } else {
-      // check if the reply and the pending request is consistent
-      final long lastEntryIndex = request
-          .getEntries(request.getEntriesCount() - 1).getIndex();
-      Preconditions.checkState(replyNextIndex == lastEntryIndex + 1,
-          "reply's next index is %s, request's last entry index is %s",
-          replyNextIndex, lastEntryIndex);
-      follower.updateMatchIndex(lastEntryIndex);
-      submitEventOnSuccessAppend();
-    }
-  }
-
-  private void onNotLeader(AppendEntriesReplyProto reply) {
-    checkResponseTerm(reply.getTerm());
-    // the running loop will end and the connection will onComplete
-  }
-
-  private synchronized void onInconsistency(AppendEntriesReplyProto reply) {
-    AppendEntriesRequestProto request = pendingRequests.peek();
-    Preconditions.checkState(request.hasPreviousLog());
-    if (request.getPreviousLog().getIndex() >= reply.getNextIndex()) {
-      clearPendingRequests(reply.getNextIndex());
-    }
-  }
-
-  private class InstallSnapshotResponseHandler
-      implements StreamObserver<InstallSnapshotReplyProto> {
-    private final Queue<Integer> pending;
-    private final AtomicBoolean done = new AtomicBoolean(false);
-
-    InstallSnapshotResponseHandler() {
-      pending = new LinkedList<>();
-    }
-
-    synchronized void addPending(InstallSnapshotRequestProto request) {
-      pending.offer(request.getRequestIndex());
-    }
-
-    synchronized void removePending(InstallSnapshotReplyProto reply) {
-      int index = pending.poll();
-      Preconditions.checkState(index == reply.getRequestIndex());
-    }
-
-    boolean isDone() {
-      return done.get();
-    }
-
-    void close() {
-      done.set(true);
-      GRpcLogAppender.this.notifyAppend();
-    }
-
-    synchronized boolean hasAllResponse() {
-      return pending.isEmpty();
-    }
-
-    @Override
-    public void onNext(InstallSnapshotReplyProto reply) {
-      LOG.debug("{} received {} response from {}", server.getId(),
-          (!firstResponseReceived ? "the first" : "a"),
-          follower.getPeer());
-
-      // update the last rpc time
-      follower.updateLastRpcResponseTime();
-
-      if (!firstResponseReceived) {
-        firstResponseReceived = true;
-      }
-
-      switch (reply.getResult()) {
-        case SUCCESS:
-          removePending(reply);
-          break;
-        case NOT_LEADER:
-          checkResponseTerm(reply.getTerm());
-          break;
-        case UNRECOGNIZED:
-          break;
-      }
-    }
-
-    @Override
-    public void onError(Throwable t) {
-      if (!isAppenderRunning()) {
-        LOG.info("{} is stopped", GRpcLogAppender.this);
-        return;
-      }
-      LOG.info("{} got error when installing snapshot to {}, exception: {}",
-          server.getId(), follower.getPeer(), t);
-      close();
-    }
-
-    @Override
-    public void onCompleted() {
-      LOG.info("{} stops sending snapshots to follower {}", server.getId(),
-          follower);
-      close();
-    }
-  }
-
-  private void installSnapshot(SnapshotInfo snapshot,
-      InstallSnapshotResponseHandler responseHandler) {
-    LOG.info("{}: follower {}'s next index is {}," +
-            " log's start index is {}, need to install snapshot",
-        server.getId(), follower.getPeer(), follower.getNextIndex(),
-        raftLog.getStartIndex());
-
-    snapshotRequestObserver = client.installSnapshot(snapshotResponseHandler);
-    final String requestId = UUID.randomUUID().toString();
-    try {
-      for (InstallSnapshotRequestProto request :
-          new SnapshotRequestIter(snapshot, requestId)) {
-        if (isAppenderRunning()) {
-          snapshotRequestObserver.onNext(request);
-          follower.updateLastRpcSendTime();
-          responseHandler.addPending(request);
-        } else {
-          break;
-        }
-      }
-      snapshotRequestObserver.onCompleted();
-    } catch (Exception e) {
-      LOG.warn("{} failed to install snapshot {}. Exception: {}", this,
-          snapshot.getFiles(), e);
-      snapshotRequestObserver.onError(e);
-      return;
-    } finally {
-      snapshotRequestObserver = null;
-    }
-
-    synchronized (this) {
-      while (isAppenderRunning() && !responseHandler.isDone()) {
-        try {
-          wait();
-        } catch (InterruptedException ignored) {
-        }
-      }
-    }
-
-    if (responseHandler.hasAllResponse()) {
-      follower.updateMatchIndex(snapshot.getTermIndex().getIndex());
-      follower.updateNextIndex(snapshot.getTermIndex().getIndex() + 1);
-      LOG.info("{}: install snapshot-{} successfully on follower {}",
-          server.getId(), snapshot.getTermIndex().getIndex(), follower.getPeer());
-    }
-  }
-}

http://git-wip-us.apache.org/repos/asf/incubator-ratis/blob/7e71a2e0/raft-grpc/src/main/java/org/apache/raft/grpc/server/PipelinedLogAppenderFactory.java
----------------------------------------------------------------------
diff --git a/raft-grpc/src/main/java/org/apache/raft/grpc/server/PipelinedLogAppenderFactory.java b/raft-grpc/src/main/java/org/apache/raft/grpc/server/PipelinedLogAppenderFactory.java
deleted file mode 100644
index cc2e513..0000000
--- a/raft-grpc/src/main/java/org/apache/raft/grpc/server/PipelinedLogAppenderFactory.java
+++ /dev/null
@@ -1,32 +0,0 @@
-/**
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- *     http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-package org.apache.raft.grpc.server;
-
-import org.apache.raft.server.impl.FollowerInfo;
-import org.apache.raft.server.impl.LeaderState;
-import org.apache.raft.server.impl.LogAppender;
-import org.apache.raft.server.impl.LogAppenderFactory;
-import org.apache.raft.server.impl.RaftServerImpl;
-
-public class PipelinedLogAppenderFactory implements LogAppenderFactory {
-  @Override
-  public LogAppender getLogAppender(RaftServerImpl server, LeaderState state,
-                                    FollowerInfo f) {
-    return new GRpcLogAppender(server, state, f);
-  }
-}

http://git-wip-us.apache.org/repos/asf/incubator-ratis/blob/7e71a2e0/raft-grpc/src/main/java/org/apache/raft/grpc/server/RaftServerProtocolClient.java
----------------------------------------------------------------------
diff --git a/raft-grpc/src/main/java/org/apache/raft/grpc/server/RaftServerProtocolClient.java b/raft-grpc/src/main/java/org/apache/raft/grpc/server/RaftServerProtocolClient.java
deleted file mode 100644
index 437e1f4..0000000
--- a/raft-grpc/src/main/java/org/apache/raft/grpc/server/RaftServerProtocolClient.java
+++ /dev/null
@@ -1,63 +0,0 @@
-/**
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- *     http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-package org.apache.raft.grpc.server;
-
-import org.apache.raft.protocol.RaftPeer;
-import org.apache.raft.shaded.io.grpc.ManagedChannel;
-import org.apache.raft.shaded.io.grpc.ManagedChannelBuilder;
-import org.apache.raft.shaded.io.grpc.stub.StreamObserver;
-import org.apache.raft.shaded.proto.RaftProtos.*;
-import org.apache.raft.shaded.proto.grpc.RaftServerProtocolServiceGrpc;
-import org.apache.raft.shaded.proto.grpc.RaftServerProtocolServiceGrpc.RaftServerProtocolServiceBlockingStub;
-import org.apache.raft.shaded.proto.grpc.RaftServerProtocolServiceGrpc.RaftServerProtocolServiceStub;
-
-/**
- * This is a RaftClient implementation that supports streaming data to the raft
- * ring. The stream implementation utilizes gRPC.
- */
-public class RaftServerProtocolClient {
-  private final ManagedChannel channel;
-  private final RaftServerProtocolServiceBlockingStub blockingStub;
-  private final RaftServerProtocolServiceStub asyncStub;
-
-  public RaftServerProtocolClient(RaftPeer target) {
-    channel = ManagedChannelBuilder.forTarget(target.getAddress())
-        .usePlaintext(true).build();
-    blockingStub = RaftServerProtocolServiceGrpc.newBlockingStub(channel);
-    asyncStub = RaftServerProtocolServiceGrpc.newStub(channel);
-  }
-
-  public void shutdown() {
-    channel.shutdownNow();
-  }
-
-  public RequestVoteReplyProto requestVote(RequestVoteRequestProto request) {
-    // the StatusRuntimeException will be handled by the caller
-    return blockingStub.requestVote(request);
-  }
-
-  StreamObserver<AppendEntriesRequestProto> appendEntries(
-      StreamObserver<AppendEntriesReplyProto> responseHandler) {
-    return asyncStub.appendEntries(responseHandler);
-  }
-
-  StreamObserver<InstallSnapshotRequestProto> installSnapshot(
-      StreamObserver<InstallSnapshotReplyProto> responseHandler) {
-    return asyncStub.installSnapshot(responseHandler);
-  }
-}

http://git-wip-us.apache.org/repos/asf/incubator-ratis/blob/7e71a2e0/raft-grpc/src/main/java/org/apache/raft/grpc/server/RaftServerProtocolService.java
----------------------------------------------------------------------
diff --git a/raft-grpc/src/main/java/org/apache/raft/grpc/server/RaftServerProtocolService.java b/raft-grpc/src/main/java/org/apache/raft/grpc/server/RaftServerProtocolService.java
deleted file mode 100644
index 53dbb6a..0000000
--- a/raft-grpc/src/main/java/org/apache/raft/grpc/server/RaftServerProtocolService.java
+++ /dev/null
@@ -1,111 +0,0 @@
-/**
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- *     http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-package org.apache.raft.grpc.server;
-
-import org.apache.raft.grpc.RaftGrpcUtil;
-import org.apache.raft.server.protocol.RaftServerProtocol;
-import org.apache.raft.shaded.io.grpc.stub.StreamObserver;
-import org.apache.raft.shaded.proto.RaftProtos.*;
-import org.apache.raft.shaded.proto.grpc.RaftServerProtocolServiceGrpc.RaftServerProtocolServiceImplBase;
-import org.slf4j.Logger;
-import org.slf4j.LoggerFactory;
-
-public class RaftServerProtocolService extends RaftServerProtocolServiceImplBase {
-  public static final Logger LOG = LoggerFactory.getLogger(RaftServerProtocolService.class);
-
-  private final String id;
-  private final RaftServerProtocol server;
-
-  public RaftServerProtocolService(String id, RaftServerProtocol server) {
-    this.id = id;
-    this.server = server;
-  }
-
-  @Override
-  public void requestVote(RequestVoteRequestProto request,
-      StreamObserver<RequestVoteReplyProto> responseObserver) {
-    try {
-      final RequestVoteReplyProto reply = server.requestVote(request);
-      responseObserver.onNext(reply);
-      responseObserver.onCompleted();
-    } catch (Throwable e) {
-      LOG.info("{} got exception when handling requestVote {}: {}",
-          id, request.getServerRequest(), e);
-      responseObserver.onError(RaftGrpcUtil.wrapException(e));
-    }
-  }
-
-  @Override
-  public StreamObserver<AppendEntriesRequestProto> appendEntries(
-      StreamObserver<AppendEntriesReplyProto> responseObserver) {
-    return new StreamObserver<AppendEntriesRequestProto>() {
-      @Override
-      public void onNext(AppendEntriesRequestProto request) {
-        try {
-          final AppendEntriesReplyProto reply = server.appendEntries(request);
-          responseObserver.onNext(reply);
-        } catch (Throwable e) {
-          LOG.info("{} got exception when handling appendEntries {}: {}",
-              id, request.getServerRequest(), e);
-          responseObserver.onError(RaftGrpcUtil.wrapException(e));
-        }
-      }
-
-      @Override
-      public void onError(Throwable t) {
-        // for now we just log a msg
-        LOG.info("{}: appendEntries on error. Exception: {}", id, t);
-      }
-
-      @Override
-      public void onCompleted() {
-        LOG.info("{}: appendEntries completed", id);
-        responseObserver.onCompleted();
-      }
-    };
-  }
-
-  @Override
-  public StreamObserver<InstallSnapshotRequestProto> installSnapshot(
-      StreamObserver<InstallSnapshotReplyProto> responseObserver) {
-    return new StreamObserver<InstallSnapshotRequestProto>() {
-      @Override
-      public void onNext(InstallSnapshotRequestProto request) {
-        try {
-          final InstallSnapshotReplyProto reply = server.installSnapshot(request);
-          responseObserver.onNext(reply);
-        } catch (Throwable e) {
-          LOG.info("{} got exception when handling installSnapshot {}: {}",
-              id, request.getServerRequest(), e);
-          responseObserver.onError(RaftGrpcUtil.wrapException(e));
-        }
-      }
-
-      @Override
-      public void onError(Throwable t) {
-        LOG.info("{}: installSnapshot on error. Exception: {}", id, t);
-      }
-
-      @Override
-      public void onCompleted() {
-        LOG.info("{}: installSnapshot completed", id);
-        responseObserver.onCompleted();
-      }
-    };
-  }
-}

http://git-wip-us.apache.org/repos/asf/incubator-ratis/blob/7e71a2e0/raft-grpc/src/test/java/org/apache/raft/grpc/MiniRaftClusterWithGRpc.java
----------------------------------------------------------------------
diff --git a/raft-grpc/src/test/java/org/apache/raft/grpc/MiniRaftClusterWithGRpc.java b/raft-grpc/src/test/java/org/apache/raft/grpc/MiniRaftClusterWithGRpc.java
deleted file mode 100644
index 359dabd..0000000
--- a/raft-grpc/src/test/java/org/apache/raft/grpc/MiniRaftClusterWithGRpc.java
+++ /dev/null
@@ -1,135 +0,0 @@
-/**
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- *     http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-package org.apache.raft.grpc;
-
-import com.google.common.base.Preconditions;
-import org.apache.raft.MiniRaftCluster;
-import org.apache.raft.RaftTestUtil;
-import org.apache.raft.client.RaftClientRequestSender;
-import org.apache.raft.conf.RaftProperties;
-import org.apache.raft.grpc.client.RaftClientSenderWithGrpc;
-import org.apache.raft.grpc.server.PipelinedLogAppenderFactory;
-import org.apache.raft.protocol.RaftPeer;
-import org.apache.raft.server.impl.BlockRequestHandlingInjection;
-import org.apache.raft.server.impl.DelayLocalExecutionInjection;
-import org.apache.raft.server.impl.LogAppenderFactory;
-import org.apache.raft.server.impl.RaftServerImpl;
-import org.apache.raft.util.NetUtils;
-
-import java.io.IOException;
-import java.util.ArrayList;
-import java.util.Collection;
-import java.util.HashMap;
-import java.util.Map;
-
-import static org.apache.raft.server.RaftServerConfigKeys.RAFT_SERVER_LOG_APPENDER_FACTORY_CLASS_KEY;
-
-public class MiniRaftClusterWithGRpc extends MiniRaftCluster.RpcBase {
-  public static final Factory<MiniRaftClusterWithGRpc> FACTORY
-      = new Factory<MiniRaftClusterWithGRpc>() {
-    @Override
-    public MiniRaftClusterWithGRpc newCluster(
-        String[] ids, RaftProperties prop, boolean formatted) throws IOException {
-      return new MiniRaftClusterWithGRpc(ids, prop, formatted);
-    }
-  };
-
-  public static final DelayLocalExecutionInjection sendServerRequestInjection =
-      new DelayLocalExecutionInjection(RaftGRpcService.GRPC_SEND_SERVER_REQUEST);
-
-  public MiniRaftClusterWithGRpc(int numServers, RaftProperties properties)
-      throws IOException {
-    this(generateIds(numServers, 0), properties, true);
-  }
-
-  public MiniRaftClusterWithGRpc(String[] ids, RaftProperties properties,
-      boolean formatted) throws IOException {
-    super(ids, getPropForGrpc(properties), formatted);
-    init(initRpcServices(getServers(), properties));
-  }
-
-  private static RaftProperties getPropForGrpc(RaftProperties prop) {
-    RaftProperties newProp = new RaftProperties(prop);
-    newProp.setClass(RAFT_SERVER_LOG_APPENDER_FACTORY_CLASS_KEY,
-        PipelinedLogAppenderFactory.class, LogAppenderFactory.class);
-    return newProp;
-  }
-
-  private static Map<RaftPeer, RaftGRpcService> initRpcServices(
-      Collection<RaftServerImpl> servers, RaftProperties prop) throws IOException {
-    final Map<RaftPeer, RaftGRpcService> peerRpcs = new HashMap<>();
-
-    for (RaftServerImpl s : servers) {
-      final RaftGRpcService rpc = new RaftGRpcService(s, prop);
-      peerRpcs.put(new RaftPeer(s.getId(), rpc.getInetSocketAddress()), rpc);
-    }
-    return peerRpcs;
-  }
-
-  @Override
-  public RaftClientRequestSender getRaftClientRequestSender() {
-    return new RaftClientSenderWithGrpc(getPeers());
-  }
-
-  @Override
-  protected Collection<RaftPeer> addNewPeers(Collection<RaftPeer> newPeers,
-                                             Collection<RaftServerImpl> newServers, boolean startService)
-      throws IOException {
-    final Map<RaftPeer, RaftGRpcService> peers = initRpcServices(newServers, properties);
-    for (Map.Entry<RaftPeer, RaftGRpcService> entry : peers.entrySet()) {
-      RaftServerImpl server = servers.get(entry.getKey().getId());
-      server.setServerRpc(entry.getValue());
-      if (!startService) {
-        BlockRequestHandlingInjection.getInstance().blockReplier(server.getId());
-      } else {
-        server.start();
-      }
-    }
-    return new ArrayList<>(peers.keySet());
-  }
-
-  @Override
-  protected RaftServerImpl setPeerRpc(RaftPeer peer) throws IOException {
-    RaftServerImpl server = servers.get(peer.getId());
-    int port = NetUtils.newInetSocketAddress(peer.getAddress()).getPort();
-    int oldPort = properties.getInt(RaftGrpcConfigKeys.RAFT_GRPC_SERVER_PORT_KEY,
-        RaftGrpcConfigKeys.RAFT_GRPC_SERVER_PORT_DEFAULT);
-    properties.setInt(RaftGrpcConfigKeys.RAFT_GRPC_SERVER_PORT_KEY, port);
-    final RaftGRpcService rpc = new RaftGRpcService(server, properties);
-    Preconditions.checkState(
-        rpc.getInetSocketAddress().toString().contains(peer.getAddress()),
-        "address in the raft conf: %s, address in rpc server: %s",
-        peer.getAddress(), rpc.getInetSocketAddress().toString());
-    server.setServerRpc(rpc);
-    properties.setInt(RaftGrpcConfigKeys.RAFT_GRPC_SERVER_PORT_KEY, oldPort);
-    return server;
-  }
-
-  @Override
-  public void startServer(String id) {
-    super.startServer(id);
-    BlockRequestHandlingInjection.getInstance().unblockReplier(id);
-  }
-
-  @Override
-  protected void blockQueueAndSetDelay(String leaderId, int delayMs)
-      throws InterruptedException {
-    RaftTestUtil.blockQueueAndSetDelay(getServers(), sendServerRequestInjection,
-        leaderId, delayMs, getMaxTimeout());
-  }
-}

http://git-wip-us.apache.org/repos/asf/incubator-ratis/blob/7e71a2e0/raft-grpc/src/test/java/org/apache/raft/grpc/TestNotLeaderExceptionWithGrpc.java
----------------------------------------------------------------------
diff --git a/raft-grpc/src/test/java/org/apache/raft/grpc/TestNotLeaderExceptionWithGrpc.java b/raft-grpc/src/test/java/org/apache/raft/grpc/TestNotLeaderExceptionWithGrpc.java
deleted file mode 100644
index a8357c9..0000000
--- a/raft-grpc/src/test/java/org/apache/raft/grpc/TestNotLeaderExceptionWithGrpc.java
+++ /dev/null
@@ -1,39 +0,0 @@
-/**
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- *     http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-package org.apache.raft.grpc;
-
-import org.apache.raft.MiniRaftCluster;
-import org.apache.raft.RaftNotLeaderExceptionBaseTest;
-import org.apache.raft.conf.RaftProperties;
-import org.apache.raft.grpc.server.PipelinedLogAppenderFactory;
-import org.apache.raft.server.impl.LogAppenderFactory;
-
-import java.io.IOException;
-
-import static org.apache.raft.server.RaftServerConfigKeys.RAFT_SERVER_LOG_APPENDER_FACTORY_CLASS_KEY;
-
-public class TestNotLeaderExceptionWithGrpc extends RaftNotLeaderExceptionBaseTest {
-  @Override
-  public MiniRaftCluster initCluster() throws IOException {
-    String[] s = MiniRaftCluster.generateIds(NUM_PEERS, 0);
-    RaftProperties prop = new RaftProperties();
-    prop.setClass(RAFT_SERVER_LOG_APPENDER_FACTORY_CLASS_KEY,
-        PipelinedLogAppenderFactory.class, LogAppenderFactory.class);
-    return new MiniRaftClusterWithGRpc(s, prop, true);
-  }
-}

http://git-wip-us.apache.org/repos/asf/incubator-ratis/blob/7e71a2e0/raft-grpc/src/test/java/org/apache/raft/grpc/TestRaftReconfigurationWithGRpc.java
----------------------------------------------------------------------
diff --git a/raft-grpc/src/test/java/org/apache/raft/grpc/TestRaftReconfigurationWithGRpc.java b/raft-grpc/src/test/java/org/apache/raft/grpc/TestRaftReconfigurationWithGRpc.java
deleted file mode 100644
index 83e6c62..0000000
--- a/raft-grpc/src/test/java/org/apache/raft/grpc/TestRaftReconfigurationWithGRpc.java
+++ /dev/null
@@ -1,47 +0,0 @@
-/**
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- *     http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-package org.apache.raft.grpc;
-
-import org.apache.log4j.Level;
-import org.apache.raft.grpc.server.PipelinedLogAppenderFactory;
-import org.apache.raft.grpc.server.RaftServerProtocolService;
-import org.apache.raft.server.impl.LogAppenderFactory;
-import org.apache.raft.server.impl.RaftReconfigurationBaseTest;
-import org.apache.raft.util.RaftUtils;
-import org.junit.BeforeClass;
-
-import java.io.IOException;
-
-import static org.apache.raft.server.RaftServerConfigKeys.RAFT_SERVER_LOG_APPENDER_FACTORY_CLASS_KEY;
-
-public class TestRaftReconfigurationWithGRpc extends RaftReconfigurationBaseTest {
-  static {
-    RaftUtils.setLogLevel(RaftServerProtocolService.LOG, Level.DEBUG);
-  }
-
-  @BeforeClass
-  public static void setProp() {
-    prop.setClass(RAFT_SERVER_LOG_APPENDER_FACTORY_CLASS_KEY,
-        PipelinedLogAppenderFactory.class, LogAppenderFactory.class);
-  }
-
-  @Override
-  public MiniRaftClusterWithGRpc getCluster(int peerNum) throws IOException {
-    return new MiniRaftClusterWithGRpc(peerNum, prop);
-  }
-}

http://git-wip-us.apache.org/repos/asf/incubator-ratis/blob/7e71a2e0/raft-grpc/src/test/java/org/apache/raft/grpc/TestRaftSnapshotWithGrpc.java
----------------------------------------------------------------------
diff --git a/raft-grpc/src/test/java/org/apache/raft/grpc/TestRaftSnapshotWithGrpc.java b/raft-grpc/src/test/java/org/apache/raft/grpc/TestRaftSnapshotWithGrpc.java
deleted file mode 100644
index 74b2c63..0000000
--- a/raft-grpc/src/test/java/org/apache/raft/grpc/TestRaftSnapshotWithGrpc.java
+++ /dev/null
@@ -1,32 +0,0 @@
-/**
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- *     http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-package org.apache.raft.grpc;
-
-import org.apache.raft.MiniRaftCluster;
-import org.apache.raft.conf.RaftProperties;
-import org.apache.raft.statemachine.RaftSnapshotBaseTest;
-
-import java.io.IOException;
-
-public class TestRaftSnapshotWithGrpc extends RaftSnapshotBaseTest {
-  @Override
-  public MiniRaftCluster initCluster(int numServer, RaftProperties prop)
-      throws IOException {
-    return MiniRaftClusterWithGRpc.FACTORY.newCluster(numServer, prop, true);
-  }
-}

http://git-wip-us.apache.org/repos/asf/incubator-ratis/blob/7e71a2e0/raft-grpc/src/test/java/org/apache/raft/grpc/TestRaftStream.java
----------------------------------------------------------------------
diff --git a/raft-grpc/src/test/java/org/apache/raft/grpc/TestRaftStream.java b/raft-grpc/src/test/java/org/apache/raft/grpc/TestRaftStream.java
deleted file mode 100644
index 82a4e13..0000000
--- a/raft-grpc/src/test/java/org/apache/raft/grpc/TestRaftStream.java
+++ /dev/null
@@ -1,319 +0,0 @@
-/**
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- *     http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-package org.apache.raft.grpc;
-
-import org.apache.log4j.Level;
-import org.apache.raft.RaftTestUtil;
-import org.apache.raft.conf.RaftProperties;
-import org.apache.raft.grpc.client.AppendStreamer;
-import org.apache.raft.grpc.client.RaftOutputStream;
-import org.apache.raft.grpc.server.PipelinedLogAppenderFactory;
-import org.apache.raft.server.impl.LogAppenderFactory;
-import org.apache.raft.server.impl.RaftServerImpl;
-import org.apache.raft.server.storage.RaftLog;
-import org.apache.raft.shaded.proto.RaftProtos.LogEntryProto;
-import org.apache.raft.util.RaftUtils;
-import org.junit.After;
-import org.junit.Assert;
-import org.junit.BeforeClass;
-import org.junit.Test;
-import org.slf4j.Logger;
-import org.slf4j.LoggerFactory;
-
-import java.io.IOException;
-import java.util.*;
-import java.util.concurrent.CountDownLatch;
-import java.util.concurrent.TimeUnit;
-import java.util.concurrent.atomic.AtomicBoolean;
-import java.util.concurrent.atomic.AtomicInteger;
-import java.util.function.Supplier;
-
-import static org.apache.raft.RaftTestUtil.waitForLeader;
-import static org.apache.raft.grpc.RaftGrpcConfigKeys.RAFT_OUTPUTSTREAM_BUFFER_SIZE_KEY;
-import static org.apache.raft.server.RaftServerConfigKeys.RAFT_SERVER_LOG_APPENDER_FACTORY_CLASS_KEY;
-import static org.junit.Assert.fail;
-
-public class TestRaftStream {
-  static {
-    RaftUtils.setLogLevel(AppendStreamer.LOG, Level.ALL);
-  }
-  static final Logger LOG = LoggerFactory.getLogger(TestRaftStream.class);
-
-  private static final RaftProperties prop = new RaftProperties();
-  private static final int NUM_SERVERS = 3;
-
-  private MiniRaftClusterWithGRpc cluster;
-
-
-  @BeforeClass
-  public static void setProp() {
-    prop.setClass(RAFT_SERVER_LOG_APPENDER_FACTORY_CLASS_KEY,
-        PipelinedLogAppenderFactory.class, LogAppenderFactory.class);
-  }
-
-  @After
-  public void tearDown() {
-    if (cluster != null) {
-      cluster.shutdown();
-    }
-  }
-
-  private byte[] genContent(int count) {
-    return toBytes(count);
-  }
-
-  private byte[] toBytes(int i) {
-    byte[] b = new byte[4];
-    b[0] = (byte) ((i >>> 24) & 0xFF);
-    b[1] = (byte) ((i >>> 16) & 0xFF);
-    b[2] = (byte) ((i >>> 8) & 0xFF);
-    b[3] = (byte) (i & 0xFF);
-    return b;
-  }
-
-  @Test
-  public void testSimpleWrite() throws Exception {
-    LOG.info("Running testSimpleWrite");
-
-    // default 64K is too large for a test
-    prop.setInt(RAFT_OUTPUTSTREAM_BUFFER_SIZE_KEY, 4);
-    cluster = new MiniRaftClusterWithGRpc(NUM_SERVERS, prop);
-
-    cluster.start();
-    RaftServerImpl leader = waitForLeader(cluster);
-
-    int count = 1;
-    try (RaftOutputStream out = new RaftOutputStream(prop, "writer-1",
-        cluster.getPeers(), leader.getId())) {
-      for (int i = 0; i < 500; i++) { // generate 500 requests
-        out.write(genContent(count++));
-      }
-    }
-
-    // check the leader's raft log
-    final RaftLog raftLog = leader.getState().getLog();
-    final AtomicInteger currentNum = new AtomicInteger(1);
-    checkLog(raftLog, 500, () -> {
-      int value = currentNum.getAndIncrement();
-      return toBytes(value);
-    });
-  }
-
-  private void checkLog(RaftLog raftLog, long expectedCommittedIndex,
-      Supplier<byte[]> s) {
-    long committedIndex = raftLog.getLastCommittedIndex();
-    Assert.assertEquals(expectedCommittedIndex, committedIndex);
-    // check the log content
-    LogEntryProto[] entries = raftLog.getEntries(1, expectedCommittedIndex + 1);
-    for (LogEntryProto entry : entries) {
-      byte[] logData = entry.getSmLogEntry().getData().toByteArray();
-      byte[] expected = s.get();
-      Assert.assertEquals("log entry: " + entry,
-          expected.length, logData.length);
-      Assert.assertArrayEquals(expected, logData);
-    }
-  }
-
-  @Test
-  public void testWriteAndFlush() throws Exception {
-    LOG.info("Running testWriteAndFlush");
-
-    prop.setInt(RAFT_OUTPUTSTREAM_BUFFER_SIZE_KEY, ByteValue.BUFFERSIZE);
-    cluster = new MiniRaftClusterWithGRpc(NUM_SERVERS, prop);
-    cluster.start();
-
-    RaftServerImpl leader = waitForLeader(cluster);
-    RaftOutputStream out = new RaftOutputStream(prop, "writer",
-        cluster.getPeers(), leader.getId());
-
-    int[] lengths = new int[]{1, 500, 1023, 1024, 1025, 2048, 3000, 3072};
-    ByteValue[] values = new ByteValue[lengths.length];
-    for (int i = 0; i < values.length; i++) {
-      values[i] = new ByteValue(lengths[i], (byte) 9);
-    }
-
-    List<byte[]> expectedTxs = new ArrayList<>();
-    for (ByteValue v : values) {
-      byte[] data = v.genData();
-      expectedTxs.addAll(v.getTransactions());
-      out.write(data);
-      out.flush();
-
-      // make sure after the flush the data has been committed
-      Assert.assertEquals(expectedTxs.size(),
-          leader.getState().getLastAppliedIndex());
-    }
-    out.close();
-
-    try {
-      out.write(0);
-      fail("The OutputStream has been closed");
-    } catch (IOException ignored) {
-    }
-
-    LOG.info("Start to check leader's log");
-    final AtomicInteger index = new AtomicInteger(0);
-    checkLog(leader.getState().getLog(), expectedTxs.size(),
-        () -> expectedTxs.get(index.getAndIncrement()));
-  }
-
-  private static class ByteValue {
-    final static int BUFFERSIZE = 1024;
-
-    final int length;
-    final byte value;
-    final int numTx;
-    byte[] data;
-
-    ByteValue(int length, byte value) {
-      this.length = length;
-      this.value = value;
-      numTx = (length - 1) / BUFFERSIZE + 1;
-    }
-
-    byte[] genData() {
-      data = new byte[length];
-      Arrays.fill(data, value);
-      return data;
-    }
-
-    Collection<byte[]> getTransactions() {
-      if (data.length <= BUFFERSIZE) {
-        return Collections.singletonList(data);
-      } else {
-        List<byte[]> list = new ArrayList<>();
-        for (int i = 0; i < numTx; i++) {
-          int txSize = Math.min(BUFFERSIZE, length - BUFFERSIZE * i);
-          byte[] t = new byte[txSize];
-          Arrays.fill(t, value);
-          list.add(t);
-        }
-        return list;
-      }
-    }
-  }
-
-  @Test
-  public void testWriteWithOffset() throws Exception {
-    LOG.info("Running testWriteWithOffset");
-    prop.setInt(RAFT_OUTPUTSTREAM_BUFFER_SIZE_KEY, ByteValue.BUFFERSIZE);
-
-    cluster = new MiniRaftClusterWithGRpc(NUM_SERVERS, prop);
-    cluster.start();
-    RaftServerImpl leader = waitForLeader(cluster);
-
-    RaftOutputStream out = new RaftOutputStream(prop, "writer",
-        cluster.getPeers(), leader.getId());
-
-    byte[] b1 = new byte[ByteValue.BUFFERSIZE / 2];
-    Arrays.fill(b1, (byte) 1);
-    byte[] b2 = new byte[ByteValue.BUFFERSIZE];
-    Arrays.fill(b2, (byte) 2);
-    byte[] b3 = new byte[ByteValue.BUFFERSIZE * 2 + ByteValue.BUFFERSIZE / 2];
-    Arrays.fill(b3, (byte) 3);
-    byte[] b4 = new byte[ByteValue.BUFFERSIZE * 4];
-    Arrays.fill(b3, (byte) 4);
-
-    byte[] expected = new byte[ByteValue.BUFFERSIZE * 8];
-    byte[][] data = new byte[][]{b1, b2, b3, b4};
-    final Random random = new Random();
-    int totalSize = 0;
-    for (byte[] b : data) {
-      System.arraycopy(b, 0, expected, totalSize, b.length);
-      totalSize += b.length;
-
-      int written = 0;
-      while (written < b.length) {
-        int toWrite = random.nextInt(b.length - written) + 1;
-        LOG.info("write {} bytes", toWrite);
-        out.write(b, written, toWrite);
-        written += toWrite;
-      }
-    }
-    out.close();
-
-    final RaftLog log = leader.getState().getLog();
-    // 0.5 + 1 + 2.5 + 4 = 8
-    Assert.assertEquals(8, leader.getState().getLastAppliedIndex());
-    Assert.assertEquals(8, log.getLastCommittedIndex());
-    LogEntryProto[] entries = log.getEntries(1, 9);
-    byte[] actual = new byte[ByteValue.BUFFERSIZE * 8];
-    totalSize = 0;
-    for (LogEntryProto e : entries) {
-      byte[] eValue = e.getSmLogEntry().getData().toByteArray();
-      Assert.assertEquals(ByteValue.BUFFERSIZE, eValue.length);
-      System.arraycopy(eValue, 0, actual, totalSize, eValue.length);
-      totalSize += eValue.length;
-    }
-    Assert.assertArrayEquals(expected, actual);
-  }
-
-  /**
-   * Write while leader is killed
-   */
-  @Test
-  public void testKillLeader() throws Exception {
-    LOG.info("Running testChangeLeader");
-
-    prop.setInt(RAFT_OUTPUTSTREAM_BUFFER_SIZE_KEY, 4);
-    cluster = new MiniRaftClusterWithGRpc(NUM_SERVERS, prop);
-    cluster.start();
-    final RaftServerImpl leader = waitForLeader(cluster);
-
-    final AtomicBoolean running  = new AtomicBoolean(true);
-    final AtomicBoolean success = new AtomicBoolean(false);
-    final AtomicInteger result = new AtomicInteger(0);
-    final CountDownLatch latch = new CountDownLatch(1);
-
-    new Thread(() -> {
-      LOG.info("Writer thread starts");
-      int count = 0;
-      try (RaftOutputStream out = new RaftOutputStream(prop, "writer",
-          cluster.getPeers(), leader.getId())) {
-        while (running.get()) {
-          out.write(toBytes(count++));
-          Thread.sleep(10);
-        }
-        success.set(true);
-        result.set(count);
-      } catch (Exception e) {
-        LOG.info("Got exception when writing", e);
-        success.set(false);
-      } finally {
-        latch.countDown();
-      }
-    }).start();
-
-    // force change the leader
-    RaftTestUtil.waitAndKillLeader(cluster, true);
-    final RaftServerImpl newLeader = waitForLeader(cluster);
-    Assert.assertNotEquals(leader.getId(), newLeader.getId());
-    Thread.sleep(500);
-
-    running.set(false);
-    latch.await(5, TimeUnit.SECONDS);
-    Assert.assertTrue(success.get());
-    // total number of tx should be >= result + 2, where 2 means two NoOp from
-    // leaders. It may be larger than result+2 because the client may resend
-    // requests and we do not have retry cache on servers yet.
-    LOG.info("last applied index: {}. total number of requests: {}",
-        newLeader.getState().getLastAppliedIndex(), result.get());
-    Assert.assertTrue(
-        newLeader.getState().getLastAppliedIndex() >= result.get() + 1);
-  }
-}

http://git-wip-us.apache.org/repos/asf/incubator-ratis/blob/7e71a2e0/raft-grpc/src/test/java/org/apache/raft/grpc/TestRaftWithGrpc.java
----------------------------------------------------------------------
diff --git a/raft-grpc/src/test/java/org/apache/raft/grpc/TestRaftWithGrpc.java b/raft-grpc/src/test/java/org/apache/raft/grpc/TestRaftWithGrpc.java
deleted file mode 100644
index c6667b4..0000000
--- a/raft-grpc/src/test/java/org/apache/raft/grpc/TestRaftWithGrpc.java
+++ /dev/null
@@ -1,73 +0,0 @@
-/**
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- *     http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-package org.apache.raft.grpc;
-
-import org.apache.log4j.Level;
-import org.apache.raft.RaftBasicTests;
-import org.apache.raft.grpc.server.PipelinedLogAppenderFactory;
-import org.apache.raft.server.impl.BlockRequestHandlingInjection;
-import org.apache.raft.server.impl.LogAppenderFactory;
-import org.apache.raft.server.impl.RaftServerImpl;
-import org.apache.raft.util.RaftUtils;
-import org.junit.Assert;
-import org.junit.BeforeClass;
-import org.junit.Test;
-
-import java.io.IOException;
-
-import static org.apache.raft.server.RaftServerConfigKeys.RAFT_SERVER_LOG_APPENDER_FACTORY_CLASS_KEY;
-
-public class TestRaftWithGrpc extends RaftBasicTests {
-  static {
-    RaftUtils.setLogLevel(RaftServerImpl.LOG, Level.DEBUG);
-  }
-
-  private final MiniRaftClusterWithGRpc cluster;
-
-  @BeforeClass
-  public static void setProp() {
-    properties.setClass(RAFT_SERVER_LOG_APPENDER_FACTORY_CLASS_KEY,
-        PipelinedLogAppenderFactory.class, LogAppenderFactory.class);
-  }
-
-  public TestRaftWithGrpc() throws IOException {
-    cluster = new MiniRaftClusterWithGRpc(NUM_SERVERS, properties);
-    Assert.assertNull(cluster.getLeader());
-  }
-
-  @Override
-  public MiniRaftClusterWithGRpc getCluster() {
-    return cluster;
-  }
-
-  @Override
-  @Test
-  public void testEnforceLeader() throws Exception {
-    super.testEnforceLeader();
-
-    MiniRaftClusterWithGRpc.sendServerRequestInjection.clear();
-    BlockRequestHandlingInjection.getInstance().unblockAll();
-  }
-
-  @Override
-  @Test
-  public void testWithLoad() throws Exception {
-    super.testWithLoad();
-    BlockRequestHandlingInjection.getInstance().unblockAll();
-  }
-}

http://git-wip-us.apache.org/repos/asf/incubator-ratis/blob/7e71a2e0/raft-grpc/src/test/resources/log4j.properties
----------------------------------------------------------------------
diff --git a/raft-grpc/src/test/resources/log4j.properties b/raft-grpc/src/test/resources/log4j.properties
deleted file mode 100644
index ced0687..0000000
--- a/raft-grpc/src/test/resources/log4j.properties
+++ /dev/null
@@ -1,18 +0,0 @@
-#   Licensed 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.
-# log4j configuration used during build and unit tests
-
-log4j.rootLogger=info,stdout
-log4j.threshold=ALL
-log4j.appender.stdout=org.apache.log4j.ConsoleAppender
-log4j.appender.stdout.layout=org.apache.log4j.PatternLayout
-log4j.appender.stdout.layout.ConversionPattern=%d{ISO8601} %-5p %c{2} (%F:%M(%L)) - %m%n

http://git-wip-us.apache.org/repos/asf/incubator-ratis/blob/7e71a2e0/raft-hadoop/pom.xml
----------------------------------------------------------------------
diff --git a/raft-hadoop/pom.xml b/raft-hadoop/pom.xml
deleted file mode 100644
index da84fc9..0000000
--- a/raft-hadoop/pom.xml
+++ /dev/null
@@ -1,99 +0,0 @@
-<?xml version="1.0" encoding="UTF-8"?>
-<!--
-  Licensed 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. See accompanying LICENSE file.
--->
-<project xmlns="http://maven.apache.org/POM/4.0.0"
-         xmlns:xsi="http://www.w3.org/2001/XMLSchema-instance"
-         xsi:schemaLocation="http://maven.apache.org/POM/4.0.0 http://maven.apache.org/xsd/maven-4.0.0.xsd">
-  <modelVersion>4.0.0</modelVersion>
-  <parent>
-    <artifactId>raft-project-dist</artifactId>
-    <groupId>com.hortonworks.raft</groupId>
-    <version>1.0-SNAPSHOT</version>
-    <relativePath>../raft-project-dist</relativePath>
-  </parent>
-
-  <artifactId>raft-hadoop</artifactId>
-  <name>Raft Hadoop Support</name>
-
-  <dependencies>
-    <dependency>
-      <artifactId>raft-proto-shaded</artifactId>
-      <groupId>com.hortonworks.raft</groupId>
-      <scope>provided</scope>
-    </dependency>
-
-    <dependency>
-      <artifactId>raft-common</artifactId>
-      <groupId>com.hortonworks.raft</groupId>
-      <scope>provided</scope>
-    </dependency>
-    <dependency>
-      <artifactId>raft-common</artifactId>
-      <groupId>com.hortonworks.raft</groupId>
-      <scope>test</scope>
-      <type>test-jar</type>
-    </dependency>
-    
-    <dependency>
-      <artifactId>raft-client</artifactId>
-      <groupId>com.hortonworks.raft</groupId>
-      <scope>provided</scope>
-    </dependency>
-    <dependency>
-      <artifactId>raft-client</artifactId>
-      <groupId>com.hortonworks.raft</groupId>
-      <scope>test</scope>
-      <type>test-jar</type>
-    </dependency>
-    
-    <dependency>
-      <artifactId>raft-server</artifactId>
-      <groupId>com.hortonworks.raft</groupId>
-      <scope>provided</scope>
-    </dependency>
-    <dependency>
-      <artifactId>raft-server</artifactId>
-      <groupId>com.hortonworks.raft</groupId>
-      <scope>test</scope>
-      <type>test-jar</type>
-    </dependency>
-    
-    <dependency>
-      <groupId>org.slf4j</groupId>
-      <artifactId>slf4j-api</artifactId>
-    </dependency>
-
-    <dependency>
-      <groupId>com.google.guava</groupId>
-      <artifactId>guava</artifactId>
-    </dependency>
-
-    <dependency>
-      <groupId>junit</groupId>
-      <artifactId>junit</artifactId>
-      <scope>test</scope>
-    </dependency>
-    <dependency>
-      <groupId>org.mockito</groupId>
-      <artifactId>mockito-all</artifactId>
-      <scope>test</scope>
-    </dependency>
-
-    <dependency>
-      <groupId>org.apache.hadoop</groupId>
-      <artifactId>hadoop-common</artifactId>
-      <version>${hadoop.version}</version>
-    </dependency>    
-  </dependencies>
-</project>

http://git-wip-us.apache.org/repos/asf/incubator-ratis/blob/7e71a2e0/raft-hadoop/src/main/java/org/apache/hadoop/ipc/ProtobufRpcEngineShaded.java
----------------------------------------------------------------------
diff --git a/raft-hadoop/src/main/java/org/apache/hadoop/ipc/ProtobufRpcEngineShaded.java b/raft-hadoop/src/main/java/org/apache/hadoop/ipc/ProtobufRpcEngineShaded.java
deleted file mode 100644
index 824b19c..0000000
--- a/raft-hadoop/src/main/java/org/apache/hadoop/ipc/ProtobufRpcEngineShaded.java
+++ /dev/null
@@ -1,623 +0,0 @@
-/**
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- *     http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-
-package org.apache.hadoop.ipc;
-
-import com.google.common.annotations.VisibleForTesting;
-import org.apache.commons.logging.Log;
-import org.apache.commons.logging.LogFactory;
-import org.apache.hadoop.classification.InterfaceAudience;
-import org.apache.hadoop.classification.InterfaceStability;
-import org.apache.hadoop.conf.Configuration;
-import org.apache.hadoop.io.DataOutputOutputStream;
-import org.apache.hadoop.io.Writable;
-import org.apache.hadoop.io.retry.RetryPolicy;
-import org.apache.hadoop.ipc.Client.ConnectionId;
-import org.apache.hadoop.ipc.RPC.RpcInvoker;
-import org.apache.hadoop.security.UserGroupInformation;
-import org.apache.hadoop.security.token.SecretManager;
-import org.apache.hadoop.security.token.TokenIdentifier;
-import org.apache.hadoop.util.ProtoUtil;
-import org.apache.hadoop.util.Time;
-import org.apache.raft.shaded.com.google.protobuf.*;
-import org.apache.raft.shaded.com.google.protobuf.Descriptors.MethodDescriptor;
-import org.apache.raft.shaded.org.apache.hadoop.ipc.protobuf.ProtobufRpcEngineProtos.RequestHeaderProto;
-import org.apache.raft.shaded.org.apache.hadoop.ipc.protobuf.RpcHeaderProtos.RpcRequestHeaderProto;
-import org.apache.raft.shaded.org.apache.hadoop.ipc.protobuf.RpcHeaderProtos.RpcResponseHeaderProto;
-
-import javax.net.SocketFactory;
-import java.io.DataInput;
-import java.io.DataOutput;
-import java.io.IOException;
-import java.io.OutputStream;
-import java.lang.reflect.Method;
-import java.lang.reflect.Proxy;
-import java.net.InetSocketAddress;
-import java.util.Map;
-import java.util.concurrent.ConcurrentHashMap;
-import java.util.concurrent.atomic.AtomicBoolean;
-
-/**
- * Copied from {@link org.apache.hadoop.ipc.ProtobufRpcEngine}
- * and replaced the protobuf classes with the shaded classes.
- */
-@InterfaceStability.Evolving
-public class ProtobufRpcEngineShaded implements RpcEngine {
-  public static final Log LOG = LogFactory.getLog(ProtobufRpcEngineShaded.class);
-  
-  static { // Register the rpcRequest deserializer for WritableRpcEngine 
-    org.apache.hadoop.ipc.Server.registerProtocolEngine(
-        RPC.RpcKind.RPC_PROTOCOL_BUFFER, RpcRequestWrapper.class,
-        new Server.ProtoBufRpcInvoker());
-  }
-
-  private static final ClientCache CLIENTS = new ClientCache();
-
-  public <T> ProtocolProxy<T> getProxy(Class<T> protocol, long clientVersion,
-      InetSocketAddress addr, UserGroupInformation ticket, Configuration conf,
-      SocketFactory factory, int rpcTimeout) throws IOException {
-    return getProxy(protocol, clientVersion, addr, ticket, conf, factory,
-        rpcTimeout, null);
-  }
-
-  @Override
-  public <T> ProtocolProxy<T> getProxy(Class<T> protocol, long clientVersion,
-      InetSocketAddress addr, UserGroupInformation ticket, Configuration conf,
-      SocketFactory factory, int rpcTimeout, RetryPolicy connectionRetryPolicy
-      ) throws IOException {
-    return getProxy(protocol, clientVersion, addr, ticket, conf, factory,
-      rpcTimeout, connectionRetryPolicy, null);
-  }
-
-  @Override
-  @SuppressWarnings("unchecked")
-  public <T> ProtocolProxy<T> getProxy(Class<T> protocol, long clientVersion,
-      InetSocketAddress addr, UserGroupInformation ticket, Configuration conf,
-      SocketFactory factory, int rpcTimeout, RetryPolicy connectionRetryPolicy,
-      AtomicBoolean fallbackToSimpleAuth) throws IOException {
-
-    final Invoker invoker = new Invoker(protocol, addr, ticket, conf, factory,
-        rpcTimeout, connectionRetryPolicy, fallbackToSimpleAuth);
-    return new ProtocolProxy<T>(protocol, (T) Proxy.newProxyInstance(
-        protocol.getClassLoader(), new Class[]{protocol}, invoker), false);
-  }
-  
-  @Override
-  public ProtocolProxy<ProtocolMetaInfoPB> getProtocolMetaInfoProxy(
-      ConnectionId connId, Configuration conf, SocketFactory factory)
-      throws IOException {
-    Class<ProtocolMetaInfoPB> protocol = ProtocolMetaInfoPB.class;
-    return new ProtocolProxy<ProtocolMetaInfoPB>(protocol,
-        (ProtocolMetaInfoPB) Proxy.newProxyInstance(protocol.getClassLoader(),
-            new Class[] { protocol }, new Invoker(protocol, connId, conf,
-                factory)), false);
-  }
-
-  private static class Invoker implements RpcInvocationHandler {
-    private final Map<String, Message> returnTypes = 
-        new ConcurrentHashMap<String, Message>();
-    private boolean isClosed = false;
-    private final Client.ConnectionId remoteId;
-    private final Client client;
-    private final long clientProtocolVersion;
-    private final String protocolName;
-    private AtomicBoolean fallbackToSimpleAuth;
-
-    private Invoker(Class<?> protocol, InetSocketAddress addr,
-        UserGroupInformation ticket, Configuration conf, SocketFactory factory,
-        int rpcTimeout, RetryPolicy connectionRetryPolicy,
-        AtomicBoolean fallbackToSimpleAuth) throws IOException {
-      this(protocol, Client.ConnectionId.getConnectionId(
-          addr, protocol, ticket, rpcTimeout, connectionRetryPolicy, conf),
-          conf, factory);
-      this.fallbackToSimpleAuth = fallbackToSimpleAuth;
-    }
-    
-    /**
-     * This constructor takes a connectionId, instead of creating a new one.
-     */
-    private Invoker(Class<?> protocol, Client.ConnectionId connId,
-        Configuration conf, SocketFactory factory) {
-      this.remoteId = connId;
-      this.client = CLIENTS.getClient(conf, factory, RpcResponseWrapper.class);
-      this.protocolName = RPC.getProtocolName(protocol);
-      this.clientProtocolVersion = RPC
-          .getProtocolVersion(protocol);
-    }
-
-    private RequestHeaderProto constructRpcRequestHeader(Method method) {
-      RequestHeaderProto.Builder builder = RequestHeaderProto
-          .newBuilder();
-      builder.setMethodName(method.getName());
-     
-
-      // For protobuf, {@code protocol} used when creating client side proxy is
-      // the interface extending BlockingInterface, which has the annotations 
-      // such as ProtocolName etc.
-      //
-      // Using Method.getDeclaringClass(), as in WritableEngine to get at
-      // the protocol interface will return BlockingInterface, from where 
-      // the annotation ProtocolName and Version cannot be
-      // obtained.
-      //
-      // Hence we simply use the protocol class used to create the proxy.
-      // For PB this may limit the use of mixins on client side.
-      builder.setDeclaringClassProtocolName(protocolName);
-      builder.setClientProtocolVersion(clientProtocolVersion);
-      return builder.build();
-    }
-
-    /**
-     * This is the client side invoker of RPC method. It only throws
-     * ServiceException, since the invocation proxy expects only
-     * ServiceException to be thrown by the method in case protobuf service.
-     * 
-     * ServiceException has the following causes:
-     * <ol>
-     * <li>Exceptions encountered on the client side in this method are 
-     * set as cause in ServiceException as is.</li>
-     * <li>Exceptions from the server are wrapped in RemoteException and are
-     * set as cause in ServiceException</li>
-     * </ol>
-     * 
-     * Note that the client calling protobuf RPC methods, must handle
-     * ServiceException by getting the cause from the ServiceException. If the
-     * cause is RemoteException, then unwrap it to get the exception thrown by
-     * the server.
-     */
-    @Override
-    public Object invoke(Object proxy, Method method, Object[] args)
-        throws ServiceException {
-      long startTime = 0;
-      if (LOG.isDebugEnabled()) {
-        startTime = Time.now();
-      }
-      
-      if (args.length != 2) { // RpcController + Message
-        throw new ServiceException("Too many parameters for request. Method: ["
-            + method.getName() + "]" + ", Expected: 2, Actual: "
-            + args.length);
-      }
-      if (args[1] == null) {
-        throw new ServiceException("null param while calling Method: ["
-            + method.getName() + "]");
-      }
-
-      RequestHeaderProto rpcRequestHeader = constructRpcRequestHeader(method);
-      
-      if (LOG.isTraceEnabled()) {
-        LOG.trace(Thread.currentThread().getId() + ": Call -> " +
-            remoteId + ": " + method.getName() +
-            " {" + TextFormat.shortDebugString((Message) args[1]) + "}");
-      }
-
-
-      Message theRequest = (Message) args[1];
-      final RpcResponseWrapper val;
-      try {
-        val = (RpcResponseWrapper) client.call(RPC.RpcKind.RPC_PROTOCOL_BUFFER,
-            new RpcRequestWrapper(rpcRequestHeader, theRequest), remoteId,
-            fallbackToSimpleAuth);
-
-      } catch (Throwable e) {
-        if (LOG.isTraceEnabled()) {
-          LOG.trace(Thread.currentThread().getId() + ": Exception <- " +
-              remoteId + ": " + method.getName() +
-                " {" + e + "}");
-        }
-        throw new ServiceException(e);
-      }
-
-      if (LOG.isDebugEnabled()) {
-        long callTime = Time.now() - startTime;
-        LOG.debug("Call: " + method.getName() + " took " + callTime + "ms");
-      }
-      
-      Message prototype = null;
-      try {
-        prototype = getReturnProtoType(method);
-      } catch (Exception e) {
-        throw new ServiceException(e);
-      }
-      Message returnMessage;
-      try {
-        returnMessage = prototype.newBuilderForType()
-            .mergeFrom(val.theResponseRead).build();
-
-        if (LOG.isTraceEnabled()) {
-          LOG.trace(Thread.currentThread().getId() + ": Response <- " +
-              remoteId + ": " + method.getName() +
-                " {" + TextFormat.shortDebugString(returnMessage) + "}");
-        }
-
-      } catch (Throwable e) {
-        throw new ServiceException(e);
-      }
-      return returnMessage;
-    }
-
-    @Override
-    public void close() throws IOException {
-      if (!isClosed) {
-        isClosed = true;
-        CLIENTS.stopClient(client);
-      }
-    }
-
-    private Message getReturnProtoType(Method method) throws Exception {
-      if (returnTypes.containsKey(method.getName())) {
-        return returnTypes.get(method.getName());
-      }
-      
-      Class<?> returnType = method.getReturnType();
-      Method newInstMethod = returnType.getMethod("getDefaultInstance");
-      newInstMethod.setAccessible(true);
-      Message prototype = (Message) newInstMethod.invoke(null, (Object[]) null);
-      returnTypes.put(method.getName(), prototype);
-      return prototype;
-    }
-
-    @Override //RpcInvocationHandler
-    public ConnectionId getConnectionId() {
-      return remoteId;
-    }
-  }
-
-  interface RpcWrapper extends Writable {
-    int getLength();
-  }
-  /**
-   * Wrapper for Protocol Buffer Requests
-   * 
-   * Note while this wrapper is writable, the request on the wire is in
-   * Protobuf. Several methods on {@link org.apache.hadoop.ipc.Server and RPC} 
-   * use type Writable as a wrapper to work across multiple RpcEngine kinds.
-   */
-  private static abstract class RpcMessageWithHeader<T extends GeneratedMessage>
-    implements RpcWrapper {
-    T requestHeader;
-    Message theRequest; // for clientSide, the request is here
-    byte[] theRequestRead; // for server side, the request is here
-
-    public RpcMessageWithHeader() {
-    }
-
-    public RpcMessageWithHeader(T requestHeader, Message theRequest) {
-      this.requestHeader = requestHeader;
-      this.theRequest = theRequest;
-    }
-
-    @Override
-    public void write(DataOutput out) throws IOException {
-      OutputStream os = DataOutputOutputStream.constructOutputStream(out);
-      
-      ((Message)requestHeader).writeDelimitedTo(os);
-      theRequest.writeDelimitedTo(os);
-    }
-
-    @Override
-    public void readFields(DataInput in) throws IOException {
-      requestHeader = parseHeaderFrom(readVarintBytes(in));
-      theRequestRead = readMessageRequest(in);
-    }
-
-    abstract T parseHeaderFrom(byte[] bytes) throws IOException;
-
-    byte[] readMessageRequest(DataInput in) throws IOException {
-      return readVarintBytes(in);
-    }
-
-    private static byte[] readVarintBytes(DataInput in) throws IOException {
-      final int length = ProtoUtil.readRawVarint32(in);
-      final byte[] bytes = new byte[length];
-      in.readFully(bytes);
-      return bytes;
-    }
-
-    public T getMessageHeader() {
-      return requestHeader;
-    }
-
-    public byte[] getMessageBytes() {
-      return theRequestRead;
-    }
-    
-    @Override
-    public int getLength() {
-      int headerLen = requestHeader.getSerializedSize();
-      int reqLen;
-      if (theRequest != null) {
-        reqLen = theRequest.getSerializedSize();
-      } else if (theRequestRead != null ) {
-        reqLen = theRequestRead.length;
-      } else {
-        throw new IllegalArgumentException(
-            "getLength on uninitialized RpcWrapper");      
-      }
-      return CodedOutputStream.computeRawVarint32Size(headerLen) +  headerLen
-          + CodedOutputStream.computeRawVarint32Size(reqLen) + reqLen;
-    }
-  }
-  
-  private static class RpcRequestWrapper
-  extends RpcMessageWithHeader<RequestHeaderProto> {
-    @SuppressWarnings("unused")
-    public RpcRequestWrapper() {}
-    
-    public RpcRequestWrapper(
-        RequestHeaderProto requestHeader, Message theRequest) {
-      super(requestHeader, theRequest);
-    }
-    
-    @Override
-    RequestHeaderProto parseHeaderFrom(byte[] bytes) throws IOException {
-      return RequestHeaderProto.parseFrom(bytes);
-    }
-    
-    @Override
-    public String toString() {
-      return requestHeader.getDeclaringClassProtocolName() + "." +
-          requestHeader.getMethodName();
-    }
-  }
-
-  @InterfaceAudience.LimitedPrivate({"RPC"})
-  public static class RpcRequestMessageWrapper
-  extends RpcMessageWithHeader<RpcRequestHeaderProto> {
-    public RpcRequestMessageWrapper() {}
-    
-    public RpcRequestMessageWrapper(
-        RpcRequestHeaderProto requestHeader, Message theRequest) {
-      super(requestHeader, theRequest);
-    }
-    
-    @Override
-    RpcRequestHeaderProto parseHeaderFrom(byte[] bytes) throws IOException {
-      return RpcRequestHeaderProto.parseFrom(bytes);
-    }
-  }
-
-  @InterfaceAudience.LimitedPrivate({"RPC"})
-  public static class RpcResponseMessageWrapper
-  extends RpcMessageWithHeader<RpcResponseHeaderProto> {
-    public RpcResponseMessageWrapper() {}
-    
-    public RpcResponseMessageWrapper(
-        RpcResponseHeaderProto responseHeader, Message theRequest) {
-      super(responseHeader, theRequest);
-    }
-    
-    @Override
-    byte[] readMessageRequest(DataInput in) throws IOException {
-      // error message contain no message body
-      switch (requestHeader.getStatus()) {
-        case ERROR:
-        case FATAL:
-          return null;
-        default:
-          return super.readMessageRequest(in);
-      }
-    }
-    
-    @Override
-    RpcResponseHeaderProto parseHeaderFrom(byte[] bytes) throws IOException {
-      return RpcResponseHeaderProto.parseFrom(bytes);
-    }
-  }
-
-  /**
-   *  Wrapper for Protocol Buffer Responses
-   * 
-   * Note while this wrapper is writable, the request on the wire is in
-   * Protobuf. Several methods on {@link org.apache.hadoop.ipc.Server and RPC} 
-   * use type Writable as a wrapper to work across multiple RpcEngine kinds.
-   */
-  @InterfaceAudience.LimitedPrivate({"RPC"}) // temporarily exposed 
-  public static class RpcResponseWrapper implements RpcWrapper {
-    Message theResponse; // for senderSide, the response is here
-    byte[] theResponseRead; // for receiver side, the response is here
-
-    public RpcResponseWrapper() {
-    }
-
-    public RpcResponseWrapper(Message message) {
-      this.theResponse = message;
-    }
-
-    @Override
-    public void write(DataOutput out) throws IOException {
-      OutputStream os = DataOutputOutputStream.constructOutputStream(out);
-      theResponse.writeDelimitedTo(os);   
-    }
-
-    @Override
-    public void readFields(DataInput in) throws IOException {
-      int length = ProtoUtil.readRawVarint32(in);
-      theResponseRead = new byte[length];
-      in.readFully(theResponseRead);
-    }
-    
-    @Override
-    public int getLength() {
-      int resLen;
-      if (theResponse != null) {
-        resLen = theResponse.getSerializedSize();
-      } else if (theResponseRead != null ) {
-        resLen = theResponseRead.length;
-      } else {
-        throw new IllegalArgumentException(
-            "getLength on uninitialized RpcWrapper");      
-      }
-      return CodedOutputStream.computeRawVarint32Size(resLen) + resLen;
-    }
-  }
-
-  @VisibleForTesting
-  @InterfaceAudience.Private
-  @InterfaceStability.Unstable
-  static Client getClient(Configuration conf) {
-    return CLIENTS.getClient(conf, SocketFactory.getDefault(),
-        RpcResponseWrapper.class);
-  }
-  
- 
-
-  @Override
-  public RPC.Server getServer(Class<?> protocol, Object protocolImpl,
-      String bindAddress, int port, int numHandlers, int numReaders,
-      int queueSizePerHandler, boolean verbose, Configuration conf,
-      SecretManager<? extends TokenIdentifier> secretManager,
-      String portRangeConfig)
-      throws IOException {
-    return new Server(protocol, protocolImpl, conf, bindAddress, port,
-        numHandlers, numReaders, queueSizePerHandler, verbose, secretManager,
-        portRangeConfig);
-  }
-  
-  public static class Server extends RPC.Server {
-    /**
-     * Construct an RPC server.
-     * 
-     * @param protocolClass the class of protocol
-     * @param protocolImpl the protocolImpl whose methods will be called
-     * @param conf the configuration to use
-     * @param bindAddress the address to bind on to listen for connection
-     * @param port the port to listen for connections on
-     * @param numHandlers the number of method handler threads to run
-     * @param verbose whether each call should be logged
-     * @param portRangeConfig A config parameter that can be used to restrict
-     * the range of ports used when port is 0 (an ephemeral port)
-     */
-    public Server(Class<?> protocolClass, Object protocolImpl,
-        Configuration conf, String bindAddress, int port, int numHandlers,
-        int numReaders, int queueSizePerHandler, boolean verbose,
-        SecretManager<? extends TokenIdentifier> secretManager, 
-        String portRangeConfig)
-        throws IOException {
-      super(bindAddress, port, null, numHandlers,
-          numReaders, queueSizePerHandler, conf, classNameBase(protocolImpl
-              .getClass().getName()), secretManager, portRangeConfig);
-      this.verbose = verbose;  
-      registerProtocolAndImpl(RPC.RpcKind.RPC_PROTOCOL_BUFFER, protocolClass,
-          protocolImpl);
-    }
-    
-    /**
-     * Protobuf invoker for {@link RpcInvoker}
-     */
-    static class ProtoBufRpcInvoker implements RpcInvoker {
-      private static ProtoClassProtoImpl getProtocolImpl(RPC.Server server,
-          String protoName, long clientVersion) throws RpcServerException {
-        ProtoNameVer pv = new ProtoNameVer(protoName, clientVersion);
-        ProtoClassProtoImpl impl = 
-            server.getProtocolImplMap(RPC.RpcKind.RPC_PROTOCOL_BUFFER).get(pv);
-        if (impl == null) { // no match for Protocol AND Version
-          VerProtocolImpl highest = 
-              server.getHighestSupportedProtocol(RPC.RpcKind.RPC_PROTOCOL_BUFFER, 
-                  protoName);
-          if (highest == null) {
-            throw new RpcNoSuchProtocolException(
-                "Unknown protocol: " + protoName);
-          }
-          // protocol supported but not the version that client wants
-          throw new RPC.VersionMismatch(protoName, clientVersion,
-              highest.version);
-        }
-        return impl;
-      }
-
-      @Override 
-      /**
-       * This is a server side method, which is invoked over RPC. On success
-       * the return response has protobuf response payload. On failure, the
-       * exception name and the stack trace are return in the resposne.
-       * See {@link HadoopRpcResponseProto}
-       * 
-       * In this method there three types of exceptions possible and they are
-       * returned in response as follows.
-       * <ol>
-       * <li> Exceptions encountered in this method that are returned 
-       * as {@link RpcServerException} </li>
-       * <li> Exceptions thrown by the service is wrapped in ServiceException. 
-       * In that this method returns in response the exception thrown by the 
-       * service.</li>
-       * <li> Other exceptions thrown by the service. They are returned as
-       * it is.</li>
-       * </ol>
-       */
-      public Writable call(RPC.Server server, String protocol,
-          Writable writableRequest, long receiveTime) throws Exception {
-        RpcRequestWrapper request = (RpcRequestWrapper) writableRequest;
-        RequestHeaderProto rpcRequest = request.requestHeader;
-        String methodName = rpcRequest.getMethodName();
-        String protoName = rpcRequest.getDeclaringClassProtocolName();
-        long clientVersion = rpcRequest.getClientProtocolVersion();
-        if (server.verbose)
-          LOG.info("Call: protocol=" + protocol + ", method=" + methodName);
-        
-        ProtoClassProtoImpl protocolImpl = getProtocolImpl(server, protoName,
-            clientVersion);
-        BlockingService service = (BlockingService) protocolImpl.protocolImpl;
-        MethodDescriptor methodDescriptor = service.getDescriptorForType()
-            .findMethodByName(methodName);
-        if (methodDescriptor == null) {
-          String msg = "Unknown method " + methodName + " called on " + protocol
-              + " protocol.";
-          LOG.warn(msg);
-          throw new RpcNoSuchMethodException(msg);
-        }
-        Message prototype = service.getRequestPrototype(methodDescriptor);
-        Message param = prototype.newBuilderForType()
-            .mergeFrom(request.theRequestRead).build();
-        
-        Message result;
-        long startTime = Time.now();
-        int qTime = (int) (startTime - receiveTime);
-        Exception exception = null;
-        try {
-          server.rpcDetailedMetrics.init(protocolImpl.protocolClass);
-          result = service.callBlockingMethod(methodDescriptor, null, param);
-        } catch (ServiceException e) {
-          exception = (Exception) e.getCause();
-          throw (Exception) e.getCause();
-        } catch (Exception e) {
-          exception = e;
-          throw e;
-        } finally {
-          int processingTime = (int) (Time.now() - startTime);
-          if (LOG.isDebugEnabled()) {
-            String msg = "Served: " + methodName + " queueTime= " + qTime +
-                " procesingTime= " + processingTime;
-            if (exception != null) {
-              msg += " exception= " + exception.getClass().getSimpleName();
-            }
-            LOG.debug(msg);
-          }
-          String detailedMetricsName = (exception == null) ?
-              methodName :
-              exception.getClass().getSimpleName();
-          server.rpcMetrics.addRpcQueueTime(qTime);
-          server.rpcMetrics.addRpcProcessingTime(processingTime);
-          server.rpcDetailedMetrics.addProcessingTime(detailedMetricsName,
-              processingTime);
-        }
-        return new RpcResponseWrapper(result);
-      }
-    }
-  }
-}

http://git-wip-us.apache.org/repos/asf/incubator-ratis/blob/7e71a2e0/raft-hadoop/src/main/java/org/apache/raft/hadooprpc/HadoopConstants.java
----------------------------------------------------------------------
diff --git a/raft-hadoop/src/main/java/org/apache/raft/hadooprpc/HadoopConstants.java b/raft-hadoop/src/main/java/org/apache/raft/hadooprpc/HadoopConstants.java
deleted file mode 100644
index b8b0497..0000000
--- a/raft-hadoop/src/main/java/org/apache/raft/hadooprpc/HadoopConstants.java
+++ /dev/null
@@ -1,29 +0,0 @@
-/**
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- *     http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-package org.apache.raft.hadooprpc;
-
-public interface HadoopConstants {
-  String RAFT_SERVER_KERBEROS_PRINCIPAL_KEY
-      = "raft.server.kerberos.principal";
-  String RAFT_CLIENT_KERBEROS_PRINCIPAL_KEY
-      = "raft.client.kerberos.principal";
-  String RAFT_SERVER_PROTOCOL_NAME
-      = "org.apache.hadoop.raft.server.protocol.RaftServerProtocol";
-  String RAFT_CLIENT_PROTOCOL_NAME
-      = "org.apache.hadoop.raft.protocol.RaftClientProtocol";
-}

http://git-wip-us.apache.org/repos/asf/incubator-ratis/blob/7e71a2e0/raft-hadoop/src/main/java/org/apache/raft/hadooprpc/Proxy.java
----------------------------------------------------------------------
diff --git a/raft-hadoop/src/main/java/org/apache/raft/hadooprpc/Proxy.java b/raft-hadoop/src/main/java/org/apache/raft/hadooprpc/Proxy.java
deleted file mode 100644
index 60c8be5..0000000
--- a/raft-hadoop/src/main/java/org/apache/raft/hadooprpc/Proxy.java
+++ /dev/null
@@ -1,55 +0,0 @@
-/**
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- *     http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-package org.apache.raft.hadooprpc;
-
-import org.apache.hadoop.conf.Configuration;
-import org.apache.hadoop.ipc.ProtobufRpcEngineShaded;
-import org.apache.hadoop.ipc.RPC;
-import org.apache.hadoop.net.NetUtils;
-import org.apache.hadoop.security.UserGroupInformation;
-
-import java.io.Closeable;
-import java.io.IOException;
-
-public class Proxy<PROTOCOL> implements Closeable {
-  public static <PROTOCOL> PROTOCOL getProxy(
-      Class<PROTOCOL> clazz, String addressStr, Configuration conf)
-      throws IOException {
-    RPC.setProtocolEngine(conf, clazz, ProtobufRpcEngineShaded.class);
-    return RPC.getProxy(clazz, RPC.getProtocolVersion(clazz),
-        org.apache.raft.util.NetUtils.newInetSocketAddress(addressStr),
-        UserGroupInformation.getCurrentUser(),
-        conf, NetUtils.getSocketFactory(conf, clazz));
-  }
-
-  private final PROTOCOL protocol;
-
-  public Proxy(Class<PROTOCOL> clazz, String addressStr, Configuration conf)
-      throws IOException {
-    this.protocol = getProxy(clazz, addressStr, conf);
-  }
-
-  public PROTOCOL getProtocol() {
-    return protocol;
-  }
-
-  @Override
-  public void close() {
-    RPC.stopProxy(protocol);
-  }
-}

http://git-wip-us.apache.org/repos/asf/incubator-ratis/blob/7e71a2e0/raft-hadoop/src/main/java/org/apache/raft/hadooprpc/client/HadoopClientRequestSender.java
----------------------------------------------------------------------
diff --git a/raft-hadoop/src/main/java/org/apache/raft/hadooprpc/client/HadoopClientRequestSender.java b/raft-hadoop/src/main/java/org/apache/raft/hadooprpc/client/HadoopClientRequestSender.java
deleted file mode 100644
index 29372ea..0000000
--- a/raft-hadoop/src/main/java/org/apache/raft/hadooprpc/client/HadoopClientRequestSender.java
+++ /dev/null
@@ -1,68 +0,0 @@
-/**
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- *     http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-package org.apache.raft.hadooprpc.client;
-
-import org.apache.hadoop.conf.Configuration;
-import org.apache.hadoop.ipc.RemoteException;
-import org.apache.raft.client.RaftClientRequestSender;
-import org.apache.raft.protocol.*;
-import org.apache.raft.util.PeerProxyMap;
-
-import java.io.IOException;
-import java.util.Collection;
-
-public class HadoopClientRequestSender implements RaftClientRequestSender {
-
-  private final PeerProxyMap<RaftClientProtocolClientSideTranslatorPB> proxies;
-
-  public HadoopClientRequestSender(
-      Collection<RaftPeer> peers, final Configuration conf) {
-    this.proxies  = new PeerProxyMap<>(
-        p -> new RaftClientProtocolClientSideTranslatorPB(p.getAddress(), conf));
-    proxies.addPeers(peers);
-  }
-
-  @Override
-  public RaftClientReply sendRequest(RaftClientRequest request)
-      throws IOException {
-    final String serverId = request.getReplierId();
-    final RaftClientProtocolClientSideTranslatorPB proxy =
-        proxies.getProxy(serverId);
-    try {
-      if (request instanceof SetConfigurationRequest) {
-        return proxy.setConfiguration((SetConfigurationRequest) request);
-      } else {
-        return proxy.submitClientRequest(request);
-      }
-    } catch (RemoteException e) {
-      throw e.unwrapRemoteException(StateMachineException.class,
-          ReconfigurationTimeoutException.class,
-          ReconfigurationInProgressException.class, RaftException.class);
-    }
-  }
-
-  @Override
-  public void addServers(Iterable<RaftPeer> servers) {
-    proxies.addPeers(servers);
-  }
-
-  @Override
-  public void close() {
-    proxies.close();
-  }
-}



[47/54] [abbrv] incubator-ratis git commit: Renamed the packages from raft to ratis in preperation for Apache Incubation - Moved all java packages from org.apache.raft to org.apache.ratis. - Moved native package to org_apache_ratis, and native lib to l

Posted by ji...@apache.org.
http://git-wip-us.apache.org/repos/asf/incubator-ratis/blob/7e71a2e0/raft-common/src/main/java/org/apache/raft/io/nativeio/NativeIO.java
----------------------------------------------------------------------
diff --git a/raft-common/src/main/java/org/apache/raft/io/nativeio/NativeIO.java b/raft-common/src/main/java/org/apache/raft/io/nativeio/NativeIO.java
deleted file mode 100644
index 8416941..0000000
--- a/raft-common/src/main/java/org/apache/raft/io/nativeio/NativeIO.java
+++ /dev/null
@@ -1,804 +0,0 @@
-/**
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- *     http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-package org.apache.raft.io.nativeio;
-
-import com.google.common.annotations.VisibleForTesting;
-import org.apache.raft.protocol.AlreadyExistsException;
-import org.apache.raft.util.NativeCodeLoader;
-import org.apache.raft.util.RaftUtils;
-import org.slf4j.Logger;
-import org.slf4j.LoggerFactory;
-import sun.misc.Unsafe;
-
-import java.io.File;
-import java.io.FileDescriptor;
-import java.io.FileInputStream;
-import java.io.FileOutputStream;
-import java.io.IOException;
-import java.io.RandomAccessFile;
-import java.lang.reflect.Field;
-import java.nio.ByteBuffer;
-import java.nio.MappedByteBuffer;
-import java.nio.channels.FileChannel;
-
-/**
- * JNI wrappers for various native IO-related calls not available in Java.
- * These functions should generally be used alongside a fallback to another
- * more portable mechanism.
- */
-public class NativeIO {
-  private static final Logger LOG = LoggerFactory.getLogger(NativeIO.class);
-
-  public static class POSIX {
-    // Flags for open() call from bits/fcntl.h - Set by JNI
-    public static int O_RDONLY = -1;
-    public static int O_WRONLY = -1;
-    public static int O_RDWR = -1;
-    public static int O_CREAT = -1;
-    public static int O_EXCL = -1;
-    public static int O_NOCTTY = -1;
-    public static int O_TRUNC = -1;
-    public static int O_APPEND = -1;
-    public static int O_NONBLOCK = -1;
-    public static int O_SYNC = -1;
-
-    // Flags for posix_fadvise() from bits/fcntl.h - Set by JNI
-    /* No further special treatment.  */
-    public static int POSIX_FADV_NORMAL = -1;
-    /* Expect random page references.  */
-    public static int POSIX_FADV_RANDOM = -1;
-    /* Expect sequential page references.  */
-    public static int POSIX_FADV_SEQUENTIAL = -1;
-    /* Will need these pages.  */
-    public static int POSIX_FADV_WILLNEED = -1;
-    /* Don't need these pages.  */
-    public static int POSIX_FADV_DONTNEED = -1;
-    /* Data will be accessed once.  */
-    public static int POSIX_FADV_NOREUSE = -1;
-
-
-    // Updated by JNI when supported by glibc.  Leave defaults in case kernel
-    // supports sync_file_range, but glibc does not.
-    /* Wait upon writeout of all pages
-       in the range before performing the
-       write.  */
-    public static int SYNC_FILE_RANGE_WAIT_BEFORE = 1;
-    /* Initiate writeout of all those
-       dirty pages in the range which are
-       not presently under writeback.  */
-    public static int SYNC_FILE_RANGE_WRITE = 2;
-    /* Wait upon writeout of all pages in
-       the range after performing the
-       write.  */
-    public static int SYNC_FILE_RANGE_WAIT_AFTER = 4;
-
-    // Set to true via JNI if possible
-    public static boolean fadvisePossible = false;
-
-    private static boolean nativeLoaded = false;
-    private static boolean syncFileRangePossible = true;
-
-    private static long cacheTimeout = -1;
-
-    private static CacheManipulator cacheManipulator = new CacheManipulator();
-
-    public static CacheManipulator getCacheManipulator() {
-      return cacheManipulator;
-    }
-
-    public static void setCacheManipulator(CacheManipulator cacheManipulator) {
-      POSIX.cacheManipulator = cacheManipulator;
-    }
-
-    /**
-     * Used to manipulate the operating system cache.
-     */
-    @VisibleForTesting
-    public static class CacheManipulator {
-      public void mlock(String identifier, ByteBuffer buffer,
-          long len) throws IOException {
-        POSIX.mlock(buffer, len);
-      }
-
-      public long getMemlockLimit() {
-        return NativeIO.getMemlockLimit();
-      }
-
-      public long getOperatingSystemPageSize() {
-        return NativeIO.getOperatingSystemPageSize();
-      }
-
-      public void posixFadviseIfPossible(String identifier,
-        FileDescriptor fd, long offset, long len, int flags)
-            throws NativeIOException {
-        NativeIO.POSIX.posixFadviseIfPossible(identifier, fd, offset,
-            len, flags);
-      }
-
-      public boolean verifyCanMlock() {
-        return NativeIO.isAvailable();
-      }
-    }
-
-    /**
-     * A CacheManipulator used for testing which does not actually call mlock.
-     * This allows many tests to be run even when the operating system does not
-     * allow mlock, or only allows limited mlocking.
-     */
-    @VisibleForTesting
-    public static class NoMlockCacheManipulator extends CacheManipulator {
-      public void mlock(String identifier, ByteBuffer buffer,
-          long len) throws IOException {
-        LOG.info("mlocking " + identifier);
-      }
-
-      public long getMemlockLimit() {
-        return 1125899906842624L;
-      }
-
-      public long getOperatingSystemPageSize() {
-        return 4096;
-      }
-
-      public boolean verifyCanMlock() {
-        return true;
-      }
-    }
-
-    static {
-      initNativeLib();
-    }
-
-    /**
-     * Return true if the JNI-based native IO extensions are available.
-     */
-    public static boolean isAvailable() {
-      return NativeCodeLoader.isNativeCodeLoaded() && nativeLoaded;
-    }
-
-    private static void assertCodeLoaded() throws IOException {
-      if (!isAvailable()) {
-        throw new IOException("NativeIO was not loaded");
-      }
-    }
-
-    /** Wrapper around open(2) */
-    public static native FileDescriptor open(String path, int flags, int mode) throws IOException;
-    /** Wrapper around fstat(2) */
-    private static native Stat fstat(FileDescriptor fd) throws IOException;
-
-    /** Native chmod implementation. On UNIX, it is a wrapper around chmod(2) */
-    private static native void chmodImpl(String path, int mode) throws IOException;
-
-    public static void chmod(String path, int mode) throws IOException {
-      if (!RaftUtils.WINDOWS) {
-        chmodImpl(path, mode);
-      } else {
-        try {
-          chmodImpl(path, mode);
-        } catch (NativeIOException nioe) {
-          if (nioe.getErrorCode() == 3) {
-            throw new NativeIOException("No such file or directory",
-                Errno.ENOENT);
-          } else {
-            LOG.warn(String.format("NativeIO.chmod error (%d): %s",
-                nioe.getErrorCode(), nioe.getMessage()));
-            throw new NativeIOException("Unknown error", Errno.UNKNOWN);
-          }
-        }
-      }
-    }
-
-    /** Wrapper around posix_fadvise(2) */
-    static native void posix_fadvise(
-      FileDescriptor fd, long offset, long len, int flags) throws NativeIOException;
-
-    /** Wrapper around sync_file_range(2) */
-    static native void sync_file_range(
-      FileDescriptor fd, long offset, long nbytes, int flags) throws NativeIOException;
-
-    /**
-     * Call posix_fadvise on the given file descriptor. See the manpage
-     * for this syscall for more information. On systems where this
-     * call is not available, does nothing.
-     *
-     * @throws NativeIOException if there is an error with the syscall
-     */
-    static void posixFadviseIfPossible(String identifier,
-        FileDescriptor fd, long offset, long len, int flags)
-        throws NativeIOException {
-      if (nativeLoaded && fadvisePossible) {
-        try {
-          posix_fadvise(fd, offset, len, flags);
-        } catch (UnsatisfiedLinkError ule) {
-          fadvisePossible = false;
-        }
-      }
-    }
-
-    /**
-     * Call sync_file_range on the given file descriptor. See the manpage
-     * for this syscall for more information. On systems where this
-     * call is not available, does nothing.
-     *
-     * @throws NativeIOException if there is an error with the syscall
-     */
-    public static void syncFileRangeIfPossible(
-        FileDescriptor fd, long offset, long nbytes, int flags)
-        throws NativeIOException {
-      if (nativeLoaded && syncFileRangePossible) {
-        try {
-          sync_file_range(fd, offset, nbytes, flags);
-        } catch (UnsupportedOperationException | UnsatisfiedLinkError uoe) {
-          syncFileRangePossible = false;
-        }
-      }
-    }
-
-    static native void mlock_native(
-        ByteBuffer buffer, long len) throws NativeIOException;
-
-    /**
-     * Locks the provided direct ByteBuffer into memory, preventing it from
-     * swapping out. After a buffer is locked, future accesses will not incur
-     * a page fault.
-     *
-     * See the mlock(2) man page for more information.
-     */
-    static void mlock(ByteBuffer buffer, long len)
-        throws IOException {
-      assertCodeLoaded();
-      if (!buffer.isDirect()) {
-        throw new IOException("Cannot mlock a non-direct ByteBuffer");
-      }
-      mlock_native(buffer, len);
-    }
-
-    /**
-     * Unmaps the block from memory. See munmap(2).
-     *
-     * There isn't any portable way to unmap a memory region in Java.
-     * So we use the sun.nio method here.
-     * Note that unmapping a memory region could cause crashes if code
-     * continues to reference the unmapped code.  However, if we don't
-     * manually unmap the memory, we are dependent on the finalizer to
-     * do it, and we have no idea when the finalizer will run.
-     *
-     * @param buffer    The buffer to unmap.
-     */
-    public static void munmap(MappedByteBuffer buffer) {
-      if (buffer instanceof sun.nio.ch.DirectBuffer) {
-        sun.misc.Cleaner cleaner =
-            ((sun.nio.ch.DirectBuffer)buffer).cleaner();
-        cleaner.clean();
-      }
-    }
-
-    /** Linux only methods used for getOwner() implementation */
-    private static native long getUIDforFDOwnerforOwner(FileDescriptor fd) throws IOException;
-    private static native String getUserName(long uid) throws IOException;
-
-    /**
-     * Result type of the fstat call
-     */
-    public static class Stat {
-      private int ownerId, groupId;
-      private String owner, group;
-      private int mode;
-
-      // Mode constants - Set by JNI
-      public static int S_IFMT = -1;    /* type of file */
-      public static int S_IFIFO  = -1;  /* named pipe (fifo) */
-      public static int S_IFCHR  = -1;  /* character special */
-      public static int S_IFDIR  = -1;  /* directory */
-      public static int S_IFBLK  = -1;  /* block special */
-      public static int S_IFREG  = -1;  /* regular */
-      public static int S_IFLNK  = -1;  /* symbolic link */
-      public static int S_IFSOCK = -1;  /* socket */
-      public static int S_ISUID = -1;  /* set user id on execution */
-      public static int S_ISGID = -1;  /* set group id on execution */
-      public static int S_ISVTX = -1;  /* save swapped text even after use */
-      public static int S_IRUSR = -1;  /* read permission, owner */
-      public static int S_IWUSR = -1;  /* write permission, owner */
-      public static int S_IXUSR = -1;  /* execute/search permission, owner */
-
-      Stat(int ownerId, int groupId, int mode) {
-        this.ownerId = ownerId;
-        this.groupId = groupId;
-        this.mode = mode;
-      }
-
-      Stat(String owner, String group, int mode) {
-        if (!RaftUtils.WINDOWS) {
-          this.owner = owner;
-        } else {
-          this.owner = stripDomain(owner);
-        }
-        if (!RaftUtils.WINDOWS) {
-          this.group = group;
-        } else {
-          this.group = stripDomain(group);
-        }
-        this.mode = mode;
-      }
-
-      @Override
-      public String toString() {
-        return "Stat(owner='" + owner + "', group='" + group + "'" +
-          ", mode=" + mode + ")";
-      }
-
-      public String getOwner() {
-        return owner;
-      }
-      public String getGroup() {
-        return group;
-      }
-      public int getMode() {
-        return mode;
-      }
-    }
-
-    private static class CachedName {
-      final long timestamp;
-      final String name;
-
-      public CachedName(String name, long timestamp) {
-        this.name = name;
-        this.timestamp = timestamp;
-      }
-    }
-
-    public final static int MMAP_PROT_READ = 0x1;
-    public final static int MMAP_PROT_WRITE = 0x2;
-    public final static int MMAP_PROT_EXEC = 0x4;
-
-    public static native long mmap(FileDescriptor fd, int prot,
-        boolean shared, long length) throws IOException;
-
-    public static native void munmap(long addr, long length)
-        throws IOException;
-  }
-
-  private static boolean workaroundNonThreadSafePasswdCalls = false;
-
-
-  public static class Windows {
-    // Flags for CreateFile() call on Windows
-    public static final long GENERIC_READ = 0x80000000L;
-    public static final long GENERIC_WRITE = 0x40000000L;
-
-    public static final long FILE_SHARE_READ = 0x00000001L;
-    public static final long FILE_SHARE_WRITE = 0x00000002L;
-    public static final long FILE_SHARE_DELETE = 0x00000004L;
-
-    public static final long CREATE_NEW = 1;
-    public static final long CREATE_ALWAYS = 2;
-    public static final long OPEN_EXISTING = 3;
-    public static final long OPEN_ALWAYS = 4;
-    public static final long TRUNCATE_EXISTING = 5;
-
-    public static final long FILE_BEGIN = 0;
-    public static final long FILE_CURRENT = 1;
-    public static final long FILE_END = 2;
-
-    public static final long FILE_ATTRIBUTE_NORMAL = 0x00000080L;
-
-    /**
-     * Create a directory with permissions set to the specified mode.  By setting
-     * permissions at creation time, we avoid issues related to the user lacking
-     * WRITE_DAC rights on subsequent chmod calls.  One example where this can
-     * occur is writing to an SMB share where the user does not have Full Control
-     * rights, and therefore WRITE_DAC is denied.
-     *
-     * @param path directory to create
-     * @param mode permissions of new directory
-     * @throws IOException if there is an I/O error
-     */
-    public static void createDirectoryWithMode(File path, int mode)
-        throws IOException {
-      createDirectoryWithMode0(path.getAbsolutePath(), mode);
-    }
-
-    /** Wrapper around CreateDirectory() on Windows */
-    private static native void createDirectoryWithMode0(String path, int mode)
-        throws NativeIOException;
-
-    /** Wrapper around CreateFile() on Windows */
-    public static native FileDescriptor createFile(String path,
-        long desiredAccess, long shareMode, long creationDisposition)
-        throws IOException;
-
-    /**
-     * Create a file for write with permissions set to the specified mode.  By
-     * setting permissions at creation time, we avoid issues related to the user
-     * lacking WRITE_DAC rights on subsequent chmod calls.  One example where
-     * this can occur is writing to an SMB share where the user does not have
-     * Full Control rights, and therefore WRITE_DAC is denied.
-     *
-     * This method mimics the semantics implemented by the JDK in
-     * {@link FileOutputStream}.  The file is opened for truncate or
-     * append, the sharing mode allows other readers and writers, and paths
-     * longer than MAX_PATH are supported.  (See io_util_md.c in the JDK.)
-     *
-     * @param path file to create
-     * @param append if true, then open file for append
-     * @param mode permissions of new directory
-     * @return FileOutputStream of opened file
-     * @throws IOException if there is an I/O error
-     */
-    public static FileOutputStream createFileOutputStreamWithMode(File path,
-        boolean append, int mode) throws IOException {
-      long shareMode = FILE_SHARE_READ | FILE_SHARE_WRITE;
-      long creationDisposition = append ? OPEN_ALWAYS : CREATE_ALWAYS;
-      return new FileOutputStream(createFileWithMode0(path.getAbsolutePath(),
-          GENERIC_WRITE, shareMode, creationDisposition, mode));
-    }
-
-    /** Wrapper around CreateFile() with security descriptor on Windows */
-    private static native FileDescriptor createFileWithMode0(String path,
-        long desiredAccess, long shareMode, long creationDisposition, int mode)
-        throws NativeIOException;
-
-    /** Wrapper around SetFilePointer() on Windows */
-    public static native long setFilePointer(FileDescriptor fd,
-        long distanceToMove, long moveMethod) throws IOException;
-
-    /** Windows only methods used for getOwner() implementation */
-    private static native String getOwner(FileDescriptor fd) throws IOException;
-
-    /** Supported list of Windows access right flags */
-    public enum AccessRight {
-      ACCESS_READ (0x0001),      // FILE_READ_DATA
-      ACCESS_WRITE (0x0002),     // FILE_WRITE_DATA
-      ACCESS_EXECUTE (0x0020);   // FILE_EXECUTE
-
-      private final int accessRight;
-      AccessRight(int access) {
-        accessRight = access;
-      }
-
-      public int accessRight() {
-        return accessRight;
-      }
-    }
-
-    /** Windows only method used to check if the current process has requested
-     *  access rights on the given path. */
-    private static native boolean access0(String path, int requestedAccess);
-
-    /**
-     * Checks whether the current process has desired access rights on
-     * the given path.
-     *
-     * Longer term this native function can be substituted with JDK7
-     * function Files#isReadable, isWritable, isExecutable.
-     *
-     * @param path input path
-     * @param desiredAccess ACCESS_READ, ACCESS_WRITE or ACCESS_EXECUTE
-     * @return true if access is allowed
-     * @throws IOException I/O exception on error
-     */
-    public static boolean access(String path, AccessRight desiredAccess)
-        throws IOException {
-      return access0(path, desiredAccess.accessRight());
-    }
-
-    /**
-     * Extends both the minimum and maximum working set size of the current
-     * process.  This method gets the current minimum and maximum working set
-     * size, adds the requested amount to each and then sets the minimum and
-     * maximum working set size to the new values.  Controlling the working set
-     * size of the process also controls the amount of memory it can lock.
-     *
-     * @param delta amount to increment minimum and maximum working set size
-     * @throws IOException for any error
-     * @see POSIX#mlock(ByteBuffer, long)
-     */
-    public static native void extendWorkingSetSize(long delta) throws IOException;
-
-    static {
-      initNativeLib();
-    }
-  }
-
-  private static boolean nativeLoaded = false;
-
-  static {
-    initNativeLib();
-  }
-
-  private static void initNativeLib() {
-    if (NativeCodeLoader.isNativeCodeLoaded()) {
-      try {
-        initNative();
-        nativeLoaded = true;
-      } catch (Throwable t) {
-        LOG.debug("Unable to initialize NativeIO libraries", t);
-      }
-    }
-  }
-
-  /**
-   * Return true if the JNI-based native IO extensions are available.
-   */
-  public static boolean isAvailable() {
-    return NativeCodeLoader.isNativeCodeLoaded() && nativeLoaded;
-  }
-
-  /** Initialize the JNI method ID and class ID cache */
-  private static native void initNative();
-
-  /**
-   * Get the maximum number of bytes that can be locked into memory at any
-   * given point.
-   *
-   * @return 0 if no bytes can be locked into memory;
-   *         Long.MAX_VALUE if there is no limit;
-   *         The number of bytes that can be locked into memory otherwise.
-   */
-  static long getMemlockLimit() {
-    return isAvailable() ? getMemlockLimit0() : 0;
-  }
-
-  private static native long getMemlockLimit0();
-
-  /**
-   * @return the operating system's page size.
-   */
-  static long getOperatingSystemPageSize() {
-    try {
-      Field f = Unsafe.class.getDeclaredField("theUnsafe");
-      f.setAccessible(true);
-      Unsafe unsafe = (Unsafe)f.get(null);
-      return unsafe.pageSize();
-    } catch (Throwable e) {
-      LOG.warn("Unable to get operating system page size.  Guessing 4096.", e);
-      return 4096;
-    }
-  }
-
-  private static class CachedUid {
-    final long timestamp;
-    final String username;
-    public CachedUid(String username, long timestamp) {
-      this.timestamp = timestamp;
-      this.username = username;
-    }
-  }
-
-  private static boolean initialized = false;
-
-  /**
-   * The Windows logon name has two part, NetBIOS domain name and
-   * user account name, of the format DOMAIN\UserName. This method
-   * will remove the domain part of the full logon name.
-   *
-   * @param name full principal name containing the domain
-   * @return name with domain removed
-   */
-  private static String stripDomain(String name) {
-    int i = name.indexOf('\\');
-    if (i != -1)
-      name = name.substring(i + 1);
-    return name;
-  }
-
-  /**
-   * Create a FileInputStream that shares delete permission on the
-   * file opened, i.e. other process can delete the file the
-   * FileInputStream is reading. Only Windows implementation uses
-   * the native interface.
-   */
-  public static FileInputStream getShareDeleteFileInputStream(File f)
-      throws IOException {
-    if (!RaftUtils.WINDOWS) {
-      // On Linux the default FileInputStream shares delete permission
-      // on the file opened.
-      //
-      return new FileInputStream(f);
-    } else {
-      // Use Windows native interface to create a FileInputStream that
-      // shares delete permission on the file opened.
-      //
-      FileDescriptor fd = Windows.createFile(
-          f.getAbsolutePath(),
-          Windows.GENERIC_READ,
-          Windows.FILE_SHARE_READ |
-              Windows.FILE_SHARE_WRITE |
-              Windows.FILE_SHARE_DELETE,
-          Windows.OPEN_EXISTING);
-      return new FileInputStream(fd);
-    }
-  }
-
-  /**
-   * Create a FileInputStream that shares delete permission on the
-   * file opened at a given offset, i.e. other process can delete
-   * the file the FileInputStream is reading. Only Windows implementation
-   * uses the native interface.
-   */
-  public static FileInputStream getShareDeleteFileInputStream(File f, long seekOffset)
-      throws IOException {
-    if (!RaftUtils.WINDOWS) {
-      RandomAccessFile rf = new RandomAccessFile(f, "r");
-      if (seekOffset > 0) {
-        rf.seek(seekOffset);
-      }
-      return new FileInputStream(rf.getFD());
-    } else {
-      // Use Windows native interface to create a FileInputStream that
-      // shares delete permission on the file opened, and set it to the
-      // given offset.
-      //
-      FileDescriptor fd = NativeIO.Windows.createFile(
-          f.getAbsolutePath(),
-          NativeIO.Windows.GENERIC_READ,
-          NativeIO.Windows.FILE_SHARE_READ |
-              NativeIO.Windows.FILE_SHARE_WRITE |
-              NativeIO.Windows.FILE_SHARE_DELETE,
-          NativeIO.Windows.OPEN_EXISTING);
-      if (seekOffset > 0)
-        NativeIO.Windows.setFilePointer(fd, seekOffset, NativeIO.Windows.FILE_BEGIN);
-      return new FileInputStream(fd);
-    }
-  }
-
-  /**
-   * Create the specified File for write access, ensuring that it does not exist.
-   * @param f the file that we want to create
-   * @param permissions we want to have on the file (if security is enabled)
-   *
-   * @throws AlreadyExistsException if the file already exists
-   * @throws IOException if any other error occurred
-   */
-  public static FileOutputStream getCreateForWriteFileOutputStream(File f, int permissions)
-      throws IOException {
-    if (!RaftUtils.WINDOWS) {
-      // Use the native wrapper around open(2)
-      try {
-        FileDescriptor fd = NativeIO.POSIX.open(f.getAbsolutePath(),
-            NativeIO.POSIX.O_WRONLY | NativeIO.POSIX.O_CREAT
-                | NativeIO.POSIX.O_EXCL, permissions);
-        return new FileOutputStream(fd);
-      } catch (NativeIOException nioe) {
-        if (nioe.getErrno() == Errno.EEXIST) {
-          throw new AlreadyExistsException(nioe);
-        }
-        throw nioe;
-      }
-    } else {
-      // Use the Windows native APIs to create equivalent FileOutputStream
-      try {
-        FileDescriptor fd = NativeIO.Windows.createFile(f.getCanonicalPath(),
-            NativeIO.Windows.GENERIC_WRITE,
-            NativeIO.Windows.FILE_SHARE_DELETE
-                | NativeIO.Windows.FILE_SHARE_READ
-                | NativeIO.Windows.FILE_SHARE_WRITE,
-            NativeIO.Windows.CREATE_NEW);
-        NativeIO.POSIX.chmod(f.getCanonicalPath(), permissions);
-        return new FileOutputStream(fd);
-      } catch (NativeIOException nioe) {
-        if (nioe.getErrorCode() == 80) {
-          // ERROR_FILE_EXISTS
-          // 80 (0x50)
-          // The file exists
-          throw new AlreadyExistsException(nioe);
-        }
-        throw nioe;
-      }
-    }
-  }
-  
-  /**
-   * A version of renameTo that throws a descriptive exception when it fails.
-   *
-   * @param src                  The source path
-   * @param dst                  The destination path
-   * 
-   * @throws NativeIOException   On failure.
-   */
-  public static void renameTo(File src, File dst)
-      throws IOException {
-    if (!nativeLoaded) {
-      if (!src.renameTo(dst)) {
-        throw new IOException("renameTo(src=" + src + ", dst=" +
-          dst + ") failed.");
-      }
-    } else {
-      renameTo0(src.getAbsolutePath(), dst.getAbsolutePath());
-    }
-  }
-
-  /**
-   * A version of renameTo that throws a descriptive exception when it fails.
-   *
-   * @param src                  The source path
-   * @param dst                  The destination path
-   * 
-   * @throws NativeIOException   On failure.
-   */
-  private static native void renameTo0(String src, String dst)
-      throws NativeIOException;
-
-  private static native void link0(String src, String dst)
-      throws NativeIOException;
-
-  /**
-   * Unbuffered file copy from src to dst without tainting OS buffer cache
-   *
-   * In POSIX platform:
-   * It uses FileChannel#transferTo() which internally attempts
-   * unbuffered IO on OS with native sendfile64() support and falls back to
-   * buffered IO otherwise.
-   *
-   * It minimizes the number of FileChannel#transferTo call by passing the the
-   * src file size directly instead of a smaller size as the 3rd parameter.
-   * This saves the number of sendfile64() system call when native sendfile64()
-   * is supported. In the two fall back cases where sendfile is not supported,
-   * FileChannle#transferTo already has its own batching of size 8 MB and 8 KB,
-   * respectively.
-   *
-   * In Windows Platform:
-   * It uses its own native wrapper of CopyFileEx with COPY_FILE_NO_BUFFERING
-   * flag, which is supported on Windows Server 2008 and above.
-   *
-   * Ideally, we should use FileChannel#transferTo() across both POSIX and Windows
-   * platform. Unfortunately, the wrapper(Java_sun_nio_ch_FileChannelImpl_transferTo0)
-   * used by FileChannel#transferTo for unbuffered IO is not implemented on Windows.
-   * Based on OpenJDK 6/7/8 source code, Java_sun_nio_ch_FileChannelImpl_transferTo0
-   * on Windows simply returns IOS_UNSUPPORTED.
-   *
-   * Note: This simple native wrapper does minimal parameter checking before copy and
-   * consistency check (e.g., size) after copy.
-   * It is recommended to use wrapper function like
-   * the Storage#nativeCopyFileUnbuffered() function with pre/post copy checks.
-   *
-   * @param src                  The source path
-   * @param dst                  The destination path
-   */
-  public static void copyFileUnbuffered(File src, File dst) throws IOException {
-    if (nativeLoaded && RaftUtils.WINDOWS) {
-      copyFileUnbuffered0(src.getAbsolutePath(), dst.getAbsolutePath());
-    } else {
-      FileInputStream fis = null;
-      FileOutputStream fos = null;
-      FileChannel input = null;
-      FileChannel output = null;
-      try {
-        fis = new FileInputStream(src);
-        fos = new FileOutputStream(dst);
-        input = fis.getChannel();
-        output = fos.getChannel();
-        long remaining = input.size();
-        long position = 0;
-        long transferred;
-        while (remaining > 0) {
-          transferred = input.transferTo(position, remaining, output);
-          remaining -= transferred;
-          position += transferred;
-        }
-      } finally {
-        RaftUtils.cleanup(LOG, output, fos, input, fis);
-      }
-    }
-  }
-
-  private static native void copyFileUnbuffered0(String src, String dst)
-      throws NativeIOException;
-}

http://git-wip-us.apache.org/repos/asf/incubator-ratis/blob/7e71a2e0/raft-common/src/main/java/org/apache/raft/io/nativeio/NativeIOException.java
----------------------------------------------------------------------
diff --git a/raft-common/src/main/java/org/apache/raft/io/nativeio/NativeIOException.java b/raft-common/src/main/java/org/apache/raft/io/nativeio/NativeIOException.java
deleted file mode 100644
index c1536d4..0000000
--- a/raft-common/src/main/java/org/apache/raft/io/nativeio/NativeIOException.java
+++ /dev/null
@@ -1,70 +0,0 @@
-/**
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- *     http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-package org.apache.raft.io.nativeio;
-
-import org.apache.raft.util.RaftUtils;
-
-import java.io.IOException;
-
-
-/**
- * An exception generated by a call to the native IO code.
- *
- * These exceptions simply wrap <i>errno</i> result codes on Linux,
- * or the System Error Code on Windows.
- */
-public class NativeIOException extends IOException {
-  private static final long serialVersionUID = 1L;
-
-  private Errno errno;
-
-  // Java has no unsigned primitive error code. Use a signed 32-bit
-  // integer to hold the unsigned 32-bit integer.
-  private int errorCode;
-
-  public NativeIOException(String msg, Errno errno) {
-    super(msg);
-    this.errno = errno;
-    // Windows error code is always set to ERROR_SUCCESS on Linux,
-    // i.e. no failure on Windows
-    this.errorCode = 0;
-  }
-
-  public NativeIOException(String msg, int errorCode) {
-    super(msg);
-    this.errorCode = errorCode;
-    this.errno = Errno.UNKNOWN;
-  }
-
-  public long getErrorCode() {
-    return errorCode;
-  }
-
-  public Errno getErrno() {
-    return errno;
-  }
-
-  @Override
-  public String toString() {
-    if (RaftUtils.WINDOWS)
-      return errorCode + ": " + super.getMessage();
-    else
-      return errno.toString() + ": " + super.getMessage();
-  }
-}
-

http://git-wip-us.apache.org/repos/asf/incubator-ratis/blob/7e71a2e0/raft-common/src/main/java/org/apache/raft/protocol/AlreadyExistsException.java
----------------------------------------------------------------------
diff --git a/raft-common/src/main/java/org/apache/raft/protocol/AlreadyExistsException.java b/raft-common/src/main/java/org/apache/raft/protocol/AlreadyExistsException.java
deleted file mode 100644
index 760e6e8..0000000
--- a/raft-common/src/main/java/org/apache/raft/protocol/AlreadyExistsException.java
+++ /dev/null
@@ -1,36 +0,0 @@
-/**
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- *     http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-package org.apache.raft.protocol;
-
-import java.io.IOException;
-
-/**
- * Signals that an attempt to create a file at a given pathname has failed
- * because another file already existed at that path.
- */
-public class AlreadyExistsException extends IOException {
-  private static final long serialVersionUID = 1L;
-
-  public AlreadyExistsException(String msg) {
-    super(msg);
-  }
-
-  public AlreadyExistsException(Throwable cause) {
-    super(cause);
-  }
-}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/incubator-ratis/blob/7e71a2e0/raft-common/src/main/java/org/apache/raft/protocol/ChecksumException.java
----------------------------------------------------------------------
diff --git a/raft-common/src/main/java/org/apache/raft/protocol/ChecksumException.java b/raft-common/src/main/java/org/apache/raft/protocol/ChecksumException.java
deleted file mode 100644
index 801a428..0000000
--- a/raft-common/src/main/java/org/apache/raft/protocol/ChecksumException.java
+++ /dev/null
@@ -1,35 +0,0 @@
-/**
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- *     http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-
-package org.apache.raft.protocol;
-
-import java.io.IOException;
-
-/** Thrown for checksum errors. */
-public class ChecksumException extends IOException {
-  private static final long serialVersionUID = 1L;
-  private long pos;
-  public ChecksumException(String description, long pos) {
-    super(description);
-    this.pos = pos;
-  }
-
-  public long getPos() {
-    return pos;
-  }
-}

http://git-wip-us.apache.org/repos/asf/incubator-ratis/blob/7e71a2e0/raft-common/src/main/java/org/apache/raft/protocol/Message.java
----------------------------------------------------------------------
diff --git a/raft-common/src/main/java/org/apache/raft/protocol/Message.java b/raft-common/src/main/java/org/apache/raft/protocol/Message.java
deleted file mode 100644
index b6d8381..0000000
--- a/raft-common/src/main/java/org/apache/raft/protocol/Message.java
+++ /dev/null
@@ -1,30 +0,0 @@
-/**
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- *     http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-package org.apache.raft.protocol;
-
-import org.apache.raft.shaded.com.google.protobuf.ByteString;
-
-/**
- * The information clients append to the raft ring.
- */
-public interface Message {
-  /**
-   * @return the content of the message
-   */
-  ByteString getContent();
-}

http://git-wip-us.apache.org/repos/asf/incubator-ratis/blob/7e71a2e0/raft-common/src/main/java/org/apache/raft/protocol/NotLeaderException.java
----------------------------------------------------------------------
diff --git a/raft-common/src/main/java/org/apache/raft/protocol/NotLeaderException.java b/raft-common/src/main/java/org/apache/raft/protocol/NotLeaderException.java
deleted file mode 100644
index 6aa486f..0000000
--- a/raft-common/src/main/java/org/apache/raft/protocol/NotLeaderException.java
+++ /dev/null
@@ -1,40 +0,0 @@
-/**
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- *     http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-package org.apache.raft.protocol;
-
-public class NotLeaderException extends RaftException {
-  private final RaftPeer suggestedLeader;
-  /** the client may need to update its RaftPeer list */
-  private final RaftPeer[] peers;
-
-  public NotLeaderException(String id, RaftPeer suggestedLeader,
-      RaftPeer[] peers) {
-    super("Server " + id + " is not the leader (" + suggestedLeader
-        + "). Request must be sent to leader.");
-    this.suggestedLeader = suggestedLeader;
-    this.peers = peers == null ? RaftPeer.EMPTY_PEERS : peers;
-  }
-
-  public RaftPeer getSuggestedLeader() {
-    return suggestedLeader;
-  }
-
-  public RaftPeer[] getPeers() {
-    return peers;
-  }
-}

http://git-wip-us.apache.org/repos/asf/incubator-ratis/blob/7e71a2e0/raft-common/src/main/java/org/apache/raft/protocol/RaftClientAsynchronousProtocol.java
----------------------------------------------------------------------
diff --git a/raft-common/src/main/java/org/apache/raft/protocol/RaftClientAsynchronousProtocol.java b/raft-common/src/main/java/org/apache/raft/protocol/RaftClientAsynchronousProtocol.java
deleted file mode 100644
index 3572b7e..0000000
--- a/raft-common/src/main/java/org/apache/raft/protocol/RaftClientAsynchronousProtocol.java
+++ /dev/null
@@ -1,30 +0,0 @@
-/**
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- *     http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-package org.apache.raft.protocol;
-
-import java.io.IOException;
-import java.util.concurrent.CompletableFuture;
-
-/** Asynchronous version of {@link RaftClientProtocol}. */
-public interface RaftClientAsynchronousProtocol {
-  CompletableFuture<RaftClientReply> submitClientRequestAsync(
-      RaftClientRequest request) throws IOException;
-
-  CompletableFuture<RaftClientReply> setConfigurationAsync(
-      SetConfigurationRequest request) throws IOException;
-}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/incubator-ratis/blob/7e71a2e0/raft-common/src/main/java/org/apache/raft/protocol/RaftClientProtocol.java
----------------------------------------------------------------------
diff --git a/raft-common/src/main/java/org/apache/raft/protocol/RaftClientProtocol.java b/raft-common/src/main/java/org/apache/raft/protocol/RaftClientProtocol.java
deleted file mode 100644
index 5f8ed8d..0000000
--- a/raft-common/src/main/java/org/apache/raft/protocol/RaftClientProtocol.java
+++ /dev/null
@@ -1,26 +0,0 @@
-/**
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- *     http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-package org.apache.raft.protocol;
-
-import java.io.IOException;
-
-public interface RaftClientProtocol {
-  RaftClientReply submitClientRequest(RaftClientRequest request) throws IOException;
-
-  RaftClientReply setConfiguration(SetConfigurationRequest request) throws IOException;
-}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/incubator-ratis/blob/7e71a2e0/raft-common/src/main/java/org/apache/raft/protocol/RaftClientReply.java
----------------------------------------------------------------------
diff --git a/raft-common/src/main/java/org/apache/raft/protocol/RaftClientReply.java b/raft-common/src/main/java/org/apache/raft/protocol/RaftClientReply.java
deleted file mode 100644
index 4390ddf..0000000
--- a/raft-common/src/main/java/org/apache/raft/protocol/RaftClientReply.java
+++ /dev/null
@@ -1,91 +0,0 @@
-/**
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- *     http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-package org.apache.raft.protocol;
-
-public class RaftClientReply extends RaftRpcMessage {
-  private final String requestorId;
-  private final String replierId;
-  private final boolean success;
-  private final long seqNum;
-
-  /** non-null if the server is not leader */
-  private final NotLeaderException notLeaderException;
-  private final Message message;
-
-  public RaftClientReply(String requestorId, String replierId, long seqNum,
-      boolean success, Message message, NotLeaderException notLeaderException) {
-    this.requestorId = requestorId;
-    this.replierId = replierId;
-    this.success = success;
-    this.seqNum = seqNum;
-    this.message = message;
-    this.notLeaderException = notLeaderException;
-  }
-
-  public RaftClientReply(RaftClientRequest request,
-      NotLeaderException notLeaderException) {
-    this(request.getRequestorId(), request.getReplierId(), request.getSeqNum(),
-        false, null, notLeaderException);
-  }
-
-  public RaftClientReply(RaftClientRequest request, Message message) {
-    this(request.getRequestorId(), request.getReplierId(), request.getSeqNum(),
-        true, message, null);
-  }
-
-  @Override
-  public final boolean isRequest() {
-    return false;
-  }
-
-  @Override
-  public String getRequestorId() {
-    return requestorId;
-  }
-
-  @Override
-  public String getReplierId() {
-    return replierId;
-  }
-
-  public long getSeqNum() {
-    return seqNum;
-  }
-
-  @Override
-  public String toString() {
-    return super.toString() + ", seqNum: " + getSeqNum()
-        + ", success: " + isSuccess();
-  }
-
-  public boolean isSuccess() {
-    return success;
-  }
-
-  public Message getMessage() {
-    return message;
-  }
-
-  public NotLeaderException getNotLeaderException() {
-    return notLeaderException;
-  }
-
-  public boolean isNotLeader() {
-    return notLeaderException != null;
-  }
-}

http://git-wip-us.apache.org/repos/asf/incubator-ratis/blob/7e71a2e0/raft-common/src/main/java/org/apache/raft/protocol/RaftClientRequest.java
----------------------------------------------------------------------
diff --git a/raft-common/src/main/java/org/apache/raft/protocol/RaftClientRequest.java b/raft-common/src/main/java/org/apache/raft/protocol/RaftClientRequest.java
deleted file mode 100644
index 1955c89..0000000
--- a/raft-common/src/main/java/org/apache/raft/protocol/RaftClientRequest.java
+++ /dev/null
@@ -1,73 +0,0 @@
-/**
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- *     http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-package org.apache.raft.protocol;
-
-public class RaftClientRequest extends RaftRpcMessage {
-  private final String requestorId;
-  private final String replierId;
-  private final long seqNum;
-  private final Message message;
-  private final boolean readOnly;
-
-  public RaftClientRequest(String  requestorId, String replierId, long seqNum,
-                           Message message) {
-    this(requestorId, replierId, seqNum, message, false);
-  }
-
-  public RaftClientRequest(String requestorId, String replierId, long seqNum,
-       Message message, boolean readOnly) {
-    this.requestorId = requestorId;
-    this.replierId = replierId;
-    this.seqNum = seqNum;
-    this.message = message;
-    this.readOnly = readOnly;
-  }
-
-  @Override
-  public final boolean isRequest() {
-    return true;
-  }
-
-  @Override
-  public String getRequestorId() {
-    return requestorId;
-  }
-
-  @Override
-  public String getReplierId() {
-    return replierId;
-  }
-
-  public long getSeqNum() {
-    return seqNum;
-  }
-
-  public Message getMessage() {
-    return message;
-  }
-
-  public boolean isReadOnly() {
-    return readOnly;
-  }
-
-  @Override
-  public String toString() {
-    return super.toString() + ", seqNum: " + seqNum + ", "
-        + (isReadOnly()? "RO": "RW");
-  }
-}

http://git-wip-us.apache.org/repos/asf/incubator-ratis/blob/7e71a2e0/raft-common/src/main/java/org/apache/raft/protocol/RaftException.java
----------------------------------------------------------------------
diff --git a/raft-common/src/main/java/org/apache/raft/protocol/RaftException.java b/raft-common/src/main/java/org/apache/raft/protocol/RaftException.java
deleted file mode 100644
index 94db20f..0000000
--- a/raft-common/src/main/java/org/apache/raft/protocol/RaftException.java
+++ /dev/null
@@ -1,36 +0,0 @@
-/**
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- *     http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-package org.apache.raft.protocol;
-
-import java.io.IOException;
-
-public class RaftException extends IOException {
-  private static final long serialVersionUID = 1L;
-
-  public RaftException(String message) {
-    super(message);
-  }
-
-  public RaftException(Throwable cause) {
-    super(cause);
-  }
-
-  public RaftException(String message, Throwable cause) {
-    super(message, cause);
-  }
-}

http://git-wip-us.apache.org/repos/asf/incubator-ratis/blob/7e71a2e0/raft-common/src/main/java/org/apache/raft/protocol/RaftPeer.java
----------------------------------------------------------------------
diff --git a/raft-common/src/main/java/org/apache/raft/protocol/RaftPeer.java b/raft-common/src/main/java/org/apache/raft/protocol/RaftPeer.java
deleted file mode 100644
index c33412b..0000000
--- a/raft-common/src/main/java/org/apache/raft/protocol/RaftPeer.java
+++ /dev/null
@@ -1,79 +0,0 @@
-/**
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- *     http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-package org.apache.raft.protocol;
-
-import com.google.common.net.HostAndPort;
-
-import java.net.InetSocketAddress;
-
-/**
- * A {@link RaftPeer} is a server in a Raft cluster.
- *
- * The objects of this class are immutable.
- */
-public class RaftPeer {
-  public static final RaftPeer[] EMPTY_PEERS = {};
-
-  /** The id of the peer. */
-  private final String id;
-  /** The address of the peer. */
-  private final String address;
-
-  /** Construct a peer with the given id and a null address. */
-  public RaftPeer(String id) {
-    this(id, (String)null);
-  }
-
-  /** Construct a peer with the given id and address. */
-  public RaftPeer(String id, InetSocketAddress address) {
-    this(id, address == null ? null :
-        HostAndPort.fromParts(address.getAddress().getHostAddress(),
-            address.getPort()).toString());
-  }
-
-  /** Construct a peer with the given id and address. */
-  public RaftPeer(String id, String address) {
-    this.id = id;
-    this.address = address;
-  }
-
-  /** @return The id of the peer. */
-  public String getId() {
-    return id;
-  }
-
-  /** @return The address of the peer. */
-  public String getAddress() {
-    return address;
-  }
-
-  @Override
-  public String toString() {
-    return id + ":" + address;
-  }
-
-  @Override
-  public boolean equals(Object o) {
-    return (o instanceof RaftPeer) && id.equals(((RaftPeer) o).getId());
-  }
-
-  @Override
-  public int hashCode() {
-    return id.hashCode();
-  }
-}

http://git-wip-us.apache.org/repos/asf/incubator-ratis/blob/7e71a2e0/raft-common/src/main/java/org/apache/raft/protocol/RaftRpcMessage.java
----------------------------------------------------------------------
diff --git a/raft-common/src/main/java/org/apache/raft/protocol/RaftRpcMessage.java b/raft-common/src/main/java/org/apache/raft/protocol/RaftRpcMessage.java
deleted file mode 100644
index c48e75c..0000000
--- a/raft-common/src/main/java/org/apache/raft/protocol/RaftRpcMessage.java
+++ /dev/null
@@ -1,33 +0,0 @@
-/**
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- *     http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-package org.apache.raft.protocol;
-
-public abstract class RaftRpcMessage {
-
-  public abstract boolean isRequest();
-
-  public abstract String getRequestorId();
-
-  public abstract String getReplierId();
-
-  @Override
-  public String toString() {
-    return getClass().getSimpleName() + "(" + getRequestorId()
-        + (isRequest()? "->": "<-") + getReplierId() + ")";
-  }
-}

http://git-wip-us.apache.org/repos/asf/incubator-ratis/blob/7e71a2e0/raft-common/src/main/java/org/apache/raft/protocol/ReconfigurationInProgressException.java
----------------------------------------------------------------------
diff --git a/raft-common/src/main/java/org/apache/raft/protocol/ReconfigurationInProgressException.java b/raft-common/src/main/java/org/apache/raft/protocol/ReconfigurationInProgressException.java
deleted file mode 100644
index a27e97c..0000000
--- a/raft-common/src/main/java/org/apache/raft/protocol/ReconfigurationInProgressException.java
+++ /dev/null
@@ -1,24 +0,0 @@
-/**
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- *     http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-package org.apache.raft.protocol;
-
-public class ReconfigurationInProgressException extends RaftException {
-  public ReconfigurationInProgressException(String message) {
-    super(message);
-  }
-}

http://git-wip-us.apache.org/repos/asf/incubator-ratis/blob/7e71a2e0/raft-common/src/main/java/org/apache/raft/protocol/ReconfigurationTimeoutException.java
----------------------------------------------------------------------
diff --git a/raft-common/src/main/java/org/apache/raft/protocol/ReconfigurationTimeoutException.java b/raft-common/src/main/java/org/apache/raft/protocol/ReconfigurationTimeoutException.java
deleted file mode 100644
index d02d3ff..0000000
--- a/raft-common/src/main/java/org/apache/raft/protocol/ReconfigurationTimeoutException.java
+++ /dev/null
@@ -1,24 +0,0 @@
-/**
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- *     http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-package org.apache.raft.protocol;
-
-public class ReconfigurationTimeoutException extends RaftException {
-  public ReconfigurationTimeoutException(String message) {
-    super(message);
-  }
-}

http://git-wip-us.apache.org/repos/asf/incubator-ratis/blob/7e71a2e0/raft-common/src/main/java/org/apache/raft/protocol/SetConfigurationRequest.java
----------------------------------------------------------------------
diff --git a/raft-common/src/main/java/org/apache/raft/protocol/SetConfigurationRequest.java b/raft-common/src/main/java/org/apache/raft/protocol/SetConfigurationRequest.java
deleted file mode 100644
index 2b6b1c2..0000000
--- a/raft-common/src/main/java/org/apache/raft/protocol/SetConfigurationRequest.java
+++ /dev/null
@@ -1,39 +0,0 @@
-/**
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- *     http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-package org.apache.raft.protocol;
-
-import java.util.Arrays;
-
-public class SetConfigurationRequest extends RaftClientRequest {
-  private final RaftPeer[] peers;
-
-  public SetConfigurationRequest(String requestorId, String replierId,
-      long seqNum, RaftPeer[] peers) {
-    super(requestorId, replierId, seqNum, null);
-    this.peers = peers;
-  }
-
-  public RaftPeer[] getPeersInNewConf() {
-    return peers;
-  }
-
-  @Override
-  public String toString() {
-    return super.toString() + ", peers:" + Arrays.asList(getPeersInNewConf());
-  }
-}

http://git-wip-us.apache.org/repos/asf/incubator-ratis/blob/7e71a2e0/raft-common/src/main/java/org/apache/raft/protocol/StateMachineException.java
----------------------------------------------------------------------
diff --git a/raft-common/src/main/java/org/apache/raft/protocol/StateMachineException.java b/raft-common/src/main/java/org/apache/raft/protocol/StateMachineException.java
deleted file mode 100644
index 5228f87..0000000
--- a/raft-common/src/main/java/org/apache/raft/protocol/StateMachineException.java
+++ /dev/null
@@ -1,28 +0,0 @@
-/**
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- *     http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-package org.apache.raft.protocol;
-
-public class StateMachineException extends RaftException {
-  public StateMachineException(String serverId, Exception cause) {
-    super(cause.getClass().getName() + " from Server " + serverId, cause);
-  }
-
-  public StateMachineException(String msg) {
-    super(msg);
-  }
-}

http://git-wip-us.apache.org/repos/asf/incubator-ratis/blob/7e71a2e0/raft-common/src/main/java/org/apache/raft/util/AtomicFileOutputStream.java
----------------------------------------------------------------------
diff --git a/raft-common/src/main/java/org/apache/raft/util/AtomicFileOutputStream.java b/raft-common/src/main/java/org/apache/raft/util/AtomicFileOutputStream.java
deleted file mode 100644
index 28a5ea5..0000000
--- a/raft-common/src/main/java/org/apache/raft/util/AtomicFileOutputStream.java
+++ /dev/null
@@ -1,112 +0,0 @@
-/**
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- *     http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-package org.apache.raft.util;
-
-import org.apache.raft.io.nativeio.NativeIO;
-import org.apache.raft.io.nativeio.NativeIOException;
-import org.slf4j.Logger;
-import org.slf4j.LoggerFactory;
-
-import java.io.*;
-
-/**
- * A FileOutputStream that has the property that it will only show
- * up at its destination once it has been entirely written and flushed
- * to disk. While being written, it will use a .tmp suffix.
- *
- * When the output stream is closed, it is flushed, fsynced, and
- * will be moved into place, overwriting any file that already
- * exists at that location.
- *
- * <b>NOTE</b>: on Windows platforms, it will not atomically
- * replace the target file - instead the target file is deleted
- * before this one is moved into place.
- */
-public class AtomicFileOutputStream extends FilterOutputStream {
-
-  public static final String TMP_EXTENSION = ".tmp";
-
-  public static final Logger LOG = LoggerFactory.getLogger(AtomicFileOutputStream.class);
-
-  private final File origFile;
-  private final File tmpFile;
-
-  public AtomicFileOutputStream(File f) throws FileNotFoundException {
-    // Code unfortunately must be duplicated below since we can't assign anything
-    // before calling super
-    super(new FileOutputStream(new File(f.getParentFile(), f.getName() + TMP_EXTENSION)));
-    origFile = f.getAbsoluteFile();
-    tmpFile = new File(f.getParentFile(), f.getName() + TMP_EXTENSION).getAbsoluteFile();
-  }
-
-  @Override
-  public void close() throws IOException {
-    boolean triedToClose = false, success = false;
-    try {
-      flush();
-      ((FileOutputStream)out).getChannel().force(true);
-
-      triedToClose = true;
-      super.close();
-      success = true;
-    } finally {
-      if (success) {
-        boolean renamed = tmpFile.renameTo(origFile);
-        if (!renamed) {
-          // On windows, renameTo does not replace.
-          if (origFile.exists() && !origFile.delete()) {
-            throw new IOException("Could not delete original file " + origFile);
-          }
-          try {
-            NativeIO.renameTo(tmpFile, origFile);
-          } catch (NativeIOException e) {
-            throw new IOException("Could not rename temporary file " + tmpFile
-                + " to " + origFile + " due to failure in native rename. "
-                + e.toString());
-          }
-        }
-      } else {
-        if (!triedToClose) {
-          // If we failed when flushing, try to close it to not leak an FD
-          RaftUtils.cleanup(LOG, out);
-        }
-        // close wasn't successful, try to delete the tmp file
-        if (!tmpFile.delete()) {
-          LOG.warn("Unable to delete tmp file " + tmpFile);
-        }
-      }
-    }
-  }
-
-  /**
-   * Close the atomic file, but do not "commit" the temporary file
-   * on top of the destination. This should be used if there is a failure
-   * in writing.
-   */
-  public void abort() {
-    try {
-      super.close();
-    } catch (IOException ioe) {
-      LOG.warn("Unable to abort file " + tmpFile, ioe);
-    }
-    if (!tmpFile.delete()) {
-      LOG.warn("Unable to delete tmp file during abort " + tmpFile);
-    }
-  }
-
-}

http://git-wip-us.apache.org/repos/asf/incubator-ratis/blob/7e71a2e0/raft-common/src/main/java/org/apache/raft/util/AutoCloseableLock.java
----------------------------------------------------------------------
diff --git a/raft-common/src/main/java/org/apache/raft/util/AutoCloseableLock.java b/raft-common/src/main/java/org/apache/raft/util/AutoCloseableLock.java
deleted file mode 100644
index 8276689..0000000
--- a/raft-common/src/main/java/org/apache/raft/util/AutoCloseableLock.java
+++ /dev/null
@@ -1,57 +0,0 @@
-/**
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- *     http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-package org.apache.raft.util;
-
-import java.util.concurrent.atomic.AtomicBoolean;
-import java.util.concurrent.locks.Lock;
-
-/**
- * Wrap a lock with the {@link AutoCloseable} interface
- * so that the {@link #close()} method will unlock the lock.
- */
-public class AutoCloseableLock implements AutoCloseable {
-  /**
-   * Acquire the given lock and then wrap it with {@link AutoCloseableLock}
-   * so that the given lock can be released by calling {@link #close()},
-   * or by using a {@code try}-with-resources statement as shown below.
-   *
-   * <pre> {@code
-   * try(AutoCloseableLock acl = AutoCloseableLock.acquire(lock)) {
-   *   ...
-   * }}</pre>
-   */
-  public static AutoCloseableLock acquire(final Lock lock) {
-    lock.lock();
-    return new AutoCloseableLock(lock);
-  }
-
-  private final Lock underlying;
-  private final AtomicBoolean closed = new AtomicBoolean(false);
-
-  private AutoCloseableLock(Lock underlying) {
-    this.underlying = underlying;
-  }
-
-  /** Unlock the underlying lock.  This method is idempotent. */
-  @Override
-  public void close() {
-    if (closed.compareAndSet(false, true)) {
-      underlying.unlock();
-    }
-  }
-}

http://git-wip-us.apache.org/repos/asf/incubator-ratis/blob/7e71a2e0/raft-common/src/main/java/org/apache/raft/util/CheckedFunction.java
----------------------------------------------------------------------
diff --git a/raft-common/src/main/java/org/apache/raft/util/CheckedFunction.java b/raft-common/src/main/java/org/apache/raft/util/CheckedFunction.java
deleted file mode 100644
index 696e8e6..0000000
--- a/raft-common/src/main/java/org/apache/raft/util/CheckedFunction.java
+++ /dev/null
@@ -1,30 +0,0 @@
-/**
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- *     http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-package org.apache.raft.util;
-
-import java.util.function.Function;
-
-/** Function with a throws-clause. */
-@FunctionalInterface
-public interface CheckedFunction<INPUT, OUTPUT, THROWABLE extends Throwable> {
-  /**
-   * The same as {@link Function#apply(Object)}
-   * except that this method is declared with a throws-clause.
-   */
-  OUTPUT apply(INPUT input) throws THROWABLE;
-}

http://git-wip-us.apache.org/repos/asf/incubator-ratis/blob/7e71a2e0/raft-common/src/main/java/org/apache/raft/util/CheckedRunnable.java
----------------------------------------------------------------------
diff --git a/raft-common/src/main/java/org/apache/raft/util/CheckedRunnable.java b/raft-common/src/main/java/org/apache/raft/util/CheckedRunnable.java
deleted file mode 100644
index e7752a8..0000000
--- a/raft-common/src/main/java/org/apache/raft/util/CheckedRunnable.java
+++ /dev/null
@@ -1,28 +0,0 @@
-/**
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- *     http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-package org.apache.raft.util;
-
-/** Runnable with a throws-clause. */
-@FunctionalInterface
-public interface CheckedRunnable<THROWABLE extends Throwable> {
-  /**
-   * The same as {@link Runnable#run()}
-   * except that this method is declared with a throws-clause.
-   */
-  void run() throws THROWABLE;
-}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/incubator-ratis/blob/7e71a2e0/raft-common/src/main/java/org/apache/raft/util/CodeInjectionForTesting.java
----------------------------------------------------------------------
diff --git a/raft-common/src/main/java/org/apache/raft/util/CodeInjectionForTesting.java b/raft-common/src/main/java/org/apache/raft/util/CodeInjectionForTesting.java
deleted file mode 100644
index 8de7af4..0000000
--- a/raft-common/src/main/java/org/apache/raft/util/CodeInjectionForTesting.java
+++ /dev/null
@@ -1,67 +0,0 @@
-/**
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- *     http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-package org.apache.raft.util;
-
-import org.slf4j.Logger;
-import org.slf4j.LoggerFactory;
-
-import java.util.Arrays;
-import java.util.Map;
-import java.util.concurrent.ConcurrentHashMap;
-
-/** Inject code for testing. */
-public class CodeInjectionForTesting {
-  public static final Logger LOG = LoggerFactory.getLogger(CodeInjectionForTesting.class);
-
-  /** Code to be injected. */
-  public interface Code {
-    Logger LOG = CodeInjectionForTesting.LOG;
-
-    /**
-     * Execute the injected code for testing.
-     * @param localId the id of the local peer
-     * @param remoteId the id of the remote peer if handling a request
-     * @param args other possible args
-     * @return if the injected code is executed
-     */
-    boolean execute(String localId, String remoteId, Object... args);
-  }
-
-  private static final Map<String, Code> INJECTION_POINTS
-      = new ConcurrentHashMap<>();
-
-  /** Put an injection point. */
-  public static void put(String injectionPoint, Code code) {
-    LOG.debug("put: {}, {}", injectionPoint, code);
-    INJECTION_POINTS.put(injectionPoint, code);
-  }
-
-  /** Execute the injected code, if there is any. */
-  public static boolean execute(String injectionPoint, String localId,
-      String remoteId, Object... args) {
-    final Code code = INJECTION_POINTS.get(injectionPoint);
-    if (code == null) {
-      return false;
-    }
-    if (LOG.isDebugEnabled()) {
-      LOG.debug("execute: {}, {}, localId={}, remoteId={}, args={}",
-          injectionPoint, code, localId, remoteId, Arrays.toString(args));
-    }
-    return code.execute(localId, remoteId, args);
-  }
-}

http://git-wip-us.apache.org/repos/asf/incubator-ratis/blob/7e71a2e0/raft-common/src/main/java/org/apache/raft/util/Daemon.java
----------------------------------------------------------------------
diff --git a/raft-common/src/main/java/org/apache/raft/util/Daemon.java b/raft-common/src/main/java/org/apache/raft/util/Daemon.java
deleted file mode 100644
index ccb81ec..0000000
--- a/raft-common/src/main/java/org/apache/raft/util/Daemon.java
+++ /dev/null
@@ -1,35 +0,0 @@
-/**
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- *     http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-package org.apache.raft.util;
-
-public class Daemon extends Thread {
-  {
-    setDaemon(true);
-  }
-
-  /** Construct a daemon thread. */
-  public Daemon() {
-    super();
-  }
-
-  /** Construct a daemon thread with the given runnable. */
-  public Daemon(Runnable runnable) {
-    super(runnable);
-    this.setName(runnable.toString());
-  }
-}

http://git-wip-us.apache.org/repos/asf/incubator-ratis/blob/7e71a2e0/raft-common/src/main/java/org/apache/raft/util/ExitUtils.java
----------------------------------------------------------------------
diff --git a/raft-common/src/main/java/org/apache/raft/util/ExitUtils.java b/raft-common/src/main/java/org/apache/raft/util/ExitUtils.java
deleted file mode 100644
index 6526b5a..0000000
--- a/raft-common/src/main/java/org/apache/raft/util/ExitUtils.java
+++ /dev/null
@@ -1,98 +0,0 @@
-/**
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- *     http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-package org.apache.raft.util;
-
-import org.slf4j.Logger;
-
-/** Facilitates hooking process termination for tests and debugging. */
-public class ExitUtils {
-  public static class ExitException extends RuntimeException {
-    private static final long serialVersionUID = 1L;
-
-    public final int status;
-
-    public ExitException(int status, String message, Throwable throwable) {
-      super(message, throwable);
-      this.status = status;
-    }
-  }
-
-  private static volatile boolean systemExitDisabled = false;
-  private static volatile ExitException firstExitException;
-
-  /**
-   * @return the first {@link ExitException} thrown, or null if none thrown yet.
-   */
-  public static ExitException getFirstExitException() {
-    return firstExitException;
-  }
-
-  /**
-   * Reset the tracking of process termination.
-   * This is useful when some tests expect an exit but the others do not.
-   */
-  public static void resetFirstExitException() {
-    firstExitException = null;
-  }
-
-  /** @return true if {@link #terminate(int, String, Throwable, Logger)} has been invoked. */
-  public static boolean isTerminated() {
-    // Either this member is set or System.exit is actually invoked.
-    return firstExitException != null;
-  }
-
-  /** Disable the use of {@link System#exit(int)} for testing. */
-  public static void disableSystemExit() {
-    systemExitDisabled = true;
-  }
-
-  /**
-   * Terminate the current process. Note that terminate is the *only* method
-   * that should be used to terminate the daemon processes.
-   *
-   * @param status Exit status
-   * @param message message used to create the {@code ExitException}
-   * @throws ExitException if System.exit is disabled for test purposes
-   */
-  public static void terminate(
-      int status, String message, Throwable throwable, Logger log)
-      throws ExitException {
-    if (log != null) {
-      final String s = "Terminating with exit status " + status + ": " + message;
-      if (status == 0) {
-        log.info(s, throwable);
-      } else {
-        log.error(s, throwable);
-      }
-    }
-
-    if (!systemExitDisabled) {
-      System.exit(status);
-    }
-
-    final ExitException ee = new ExitException(status, message, throwable);
-    if (firstExitException == null) {
-      firstExitException = ee;
-    }
-    throw ee;
-  }
-
-  public static void terminate(int status, String message, Logger log) {
-    terminate(status, message, null, log);
-  }
-}

http://git-wip-us.apache.org/repos/asf/incubator-ratis/blob/7e71a2e0/raft-common/src/main/java/org/apache/raft/util/FileUtils.java
----------------------------------------------------------------------
diff --git a/raft-common/src/main/java/org/apache/raft/util/FileUtils.java b/raft-common/src/main/java/org/apache/raft/util/FileUtils.java
deleted file mode 100644
index 603078c..0000000
--- a/raft-common/src/main/java/org/apache/raft/util/FileUtils.java
+++ /dev/null
@@ -1,207 +0,0 @@
-/**
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- *     http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-package org.apache.raft.util;
-
-import org.apache.raft.io.nativeio.NativeIO;
-import org.slf4j.Logger;
-import org.slf4j.LoggerFactory;
-
-import java.io.File;
-import java.io.FileOutputStream;
-import java.io.IOException;
-import java.net.URI;
-import java.net.URISyntaxException;
-import java.nio.file.Files;
-
-public class FileUtils {
-  public static final Logger LOG = LoggerFactory.getLogger(FileUtils.class);
-
-  public static void truncateFile(File f, long target) throws IOException {
-    try (FileOutputStream out = new FileOutputStream(f, true)) {
-      out.getChannel().truncate(target);
-    }
-  }
-
-  public static void deleteFile(File f) throws IOException {
-    try {
-      Files.delete(f.toPath());
-    } catch (IOException e) {
-      LOG.warn("Could not delete " + f);
-      throw e;
-    }
-  }
-
-  /**
-   * Delete a directory and all its contents.  If
-   * we return false, the directory may be partially-deleted.
-   * (1) If dir is symlink to a file, the symlink is deleted. The file pointed
-   *     to by the symlink is not deleted.
-   * (2) If dir is symlink to a directory, symlink is deleted. The directory
-   *     pointed to by symlink is not deleted.
-   * (3) If dir is a normal file, it is deleted.
-   * (4) If dir is a normal directory, then dir and all its contents recursively
-   *     are deleted.
-   */
-  public static boolean fullyDelete(final File dir) {
-    if (deleteImpl(dir, false)) {
-      // dir is (a) normal file, (b) symlink to a file, (c) empty directory or
-      // (d) symlink to a directory
-      return true;
-    }
-    // handle nonempty directory deletion
-    return fullyDeleteContents(dir) && deleteImpl(dir, true);
-  }
-
-  private static boolean deleteImpl(final File f, final boolean doLog) {
-    if (f == null) {
-      LOG.warn("null file argument.");
-      return false;
-    }
-    final boolean wasDeleted = f.delete();
-    if (wasDeleted) {
-      return true;
-    }
-    final boolean ex = f.exists();
-    if (doLog && ex) {
-      LOG.warn("Failed to delete file or dir ["
-          + f.getAbsolutePath() + "]: it still exists.");
-    }
-    return !ex;
-  }
-
-  /**
-   * Delete the contents of a directory, not the directory itself.  If
-   * we return false, the directory may be partially-deleted.
-   * If dir is a symlink to a directory, all the contents of the actual
-   * directory pointed to by dir will be deleted.
-   */
-  private static boolean fullyDeleteContents(final File dir) {
-    boolean deletionSucceeded = true;
-    final File[] contents = dir.listFiles();
-    if (contents != null) {
-      for (File content : contents) {
-        if (content.isFile()) {
-          if (!deleteImpl(content, true)) {
-            deletionSucceeded = false;
-          }
-        } else {
-          // Either directory or symlink to another directory.
-          // Try deleting the directory as this might be a symlink
-          if (deleteImpl(content, false)) {
-            // this was indeed a symlink or an empty directory
-            continue;
-          }
-          // if not an empty directory or symlink let
-          // fullyDelete handle it.
-          if (!fullyDelete(content)) {
-            deletionSucceeded = false;
-            // continue deletion of other files/dirs under dir
-          }
-        }
-      }
-    }
-    return deletionSucceeded;
-  }
-
-  /**
-   * Interprets the passed string as a URI. In case of error it
-   * assumes the specified string is a file.
-   *
-   * @param s the string to interpret
-   * @return the resulting URI
-   */
-  public static URI stringAsURI(String s) throws IOException {
-    URI u = null;
-    // try to make a URI
-    try {
-      u = new URI(s);
-    } catch (URISyntaxException e){
-      LOG.error("Syntax error in URI " + s
-          + ". Please check hdfs configuration.", e);
-    }
-
-    // if URI is null or scheme is undefined, then assume it's file://
-    if(u == null || u.getScheme() == null){
-      LOG.warn("Path " + s + " should be specified as a URI "
-          + "in configuration files. Please update configuration.");
-      u = fileAsURI(new File(s));
-    }
-    return u;
-  }
-
-  /**
-   * Converts the passed File to a URI. This method trims the trailing slash if
-   * one is appended because the underlying file is in fact a directory that
-   * exists.
-   *
-   * @param f the file to convert
-   * @return the resulting URI
-   */
-  public static URI fileAsURI(File f) throws IOException {
-    URI u = f.getCanonicalFile().toURI();
-
-    // trim the trailing slash, if it's present
-    if (u.getPath().endsWith("/")) {
-      String uriAsString = u.toString();
-      try {
-        u = new URI(uriAsString.substring(0, uriAsString.length() - 1));
-      } catch (URISyntaxException e) {
-        throw new IOException(e);
-      }
-    }
-    return u;
-  }
-
-  /**
-   * A wrapper for {@link File#listFiles()}. This java.io API returns null
-   * when a dir is not a directory or for any I/O error. Instead of having
-   * null check everywhere File#listFiles() is used, we will add utility API
-   * to get around this problem. For the majority of cases where we prefer
-   * an IOException to be thrown.
-   * @param dir directory for which listing should be performed
-   * @return list of files or empty list
-   * @exception IOException for invalid directory or for a bad disk.
-   */
-  public static File[] listFiles(File dir) throws IOException {
-    File[] files = dir.listFiles();
-    if(files == null) {
-      throw new IOException("Invalid directory or I/O error occurred for dir: "
-          + dir.toString());
-    }
-    return files;
-  }
-
-  /**
-   * Platform independent implementation for {@link File#canWrite()}
-   * @param f input file
-   * @return On Unix, same as {@link File#canWrite()}
-   *         On Windows, true if process has write access on the path
-   */
-  public static boolean canWrite(File f) {
-    if (RaftUtils.WINDOWS) {
-      try {
-        return NativeIO.Windows.access(f.getCanonicalPath(),
-            NativeIO.Windows.AccessRight.ACCESS_WRITE);
-      } catch (IOException e) {
-        return false;
-      }
-    } else {
-      return f.canWrite();
-    }
-  }
-}



[42/54] [abbrv] incubator-ratis git commit: Renamed the packages from raft to ratis in preperation for Apache Incubation - Moved all java packages from org.apache.raft to org.apache.ratis. - Moved native package to org_apache_ratis, and native lib to l

Posted by ji...@apache.org.
http://git-wip-us.apache.org/repos/asf/incubator-ratis/blob/7e71a2e0/raft-examples/src/test/java/org/apache/raft/TestRestartRaftPeer.java
----------------------------------------------------------------------
diff --git a/raft-examples/src/test/java/org/apache/raft/TestRestartRaftPeer.java b/raft-examples/src/test/java/org/apache/raft/TestRestartRaftPeer.java
deleted file mode 100644
index e04d141..0000000
--- a/raft-examples/src/test/java/org/apache/raft/TestRestartRaftPeer.java
+++ /dev/null
@@ -1,114 +0,0 @@
-/**
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- *     http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-package org.apache.raft;
-
-import org.apache.log4j.Level;
-import org.apache.raft.RaftTestUtil.SimpleMessage;
-import org.apache.raft.client.RaftClient;
-import org.apache.raft.conf.RaftProperties;
-import org.apache.raft.examples.RaftExamplesTestUtil;
-import org.apache.raft.server.impl.RaftServerImpl;
-import org.apache.raft.server.RaftServerConfigKeys;
-import org.apache.raft.server.simulation.RequestHandler;
-import org.apache.raft.server.storage.RaftLog;
-import org.apache.raft.statemachine.SimpleStateMachine4Testing;
-import org.apache.raft.statemachine.StateMachine;
-import org.apache.raft.util.RaftUtils;
-import org.junit.Assert;
-import org.junit.Rule;
-import org.junit.Test;
-import org.junit.rules.Timeout;
-import org.junit.runner.RunWith;
-import org.junit.runners.Parameterized;
-import org.slf4j.Logger;
-import org.slf4j.LoggerFactory;
-
-import java.io.IOException;
-import java.util.Arrays;
-import java.util.Collection;
-
-/**
- * Test restarting raft peers.
- */
-@RunWith(Parameterized.class)
-public class TestRestartRaftPeer {
-  static Logger LOG = LoggerFactory.getLogger(TestRestartRaftPeer.class);
-  static {
-    RaftUtils.setLogLevel(RaftServerImpl.LOG, Level.DEBUG);
-    RaftUtils.setLogLevel(RaftLog.LOG, Level.DEBUG);
-    RaftUtils.setLogLevel(RequestHandler.LOG, Level.DEBUG);
-    RaftUtils.setLogLevel(RaftClient.LOG, Level.DEBUG);
-  }
-
-  @Parameterized.Parameters
-  public static Collection<Object[]> data() throws IOException {
-    RaftProperties prop = new RaftProperties();
-    prop.setClass(MiniRaftCluster.STATEMACHINE_CLASS_KEY,
-        SimpleStateMachine4Testing.class, StateMachine.class);
-    prop.setInt(RaftServerConfigKeys.RAFT_LOG_SEGMENT_MAX_SIZE_KEY, 1024 * 8);
-    return RaftExamplesTestUtil.getMiniRaftClusters(prop, 3);
-  }
-
-  @Parameterized.Parameter
-  public MiniRaftCluster cluster;
-
-  @Rule
-  public Timeout globalTimeout = new Timeout(60 * 1000);
-
-  @Test
-  public void restartFollower() throws Exception {
-    cluster.start();
-    RaftTestUtil.waitForLeader(cluster);
-    final String leaderId = cluster.getLeader().getId();
-    final RaftClient client = cluster.createClient("client", leaderId);
-
-    // write some messages
-    final byte[] content = new byte[1024];
-    Arrays.fill(content, (byte) 1);
-    final SimpleMessage message = new SimpleMessage(new String(content));
-    for (int i = 0; i < 10; i++) {
-      Assert.assertTrue(client.send(message).isSuccess());
-    }
-
-    // restart a follower
-    String followerId = cluster.getFollowers().get(0).getId();
-    LOG.info("Restart follower {}", followerId);
-    cluster.restartServer(followerId, false);
-
-    // write some more messages
-    for (int i = 0; i < 10; i++) {
-      Assert.assertTrue(client.send(message).isSuccess());
-    }
-    client.close();
-
-    // make sure the restarted follower can catchup
-    boolean catchup = false;
-    long lastAppliedIndex = 0;
-    for (int i = 0; i < 10 && !catchup; i++) {
-      Thread.sleep(500);
-      lastAppliedIndex = cluster.getServer(followerId).getState().getLastAppliedIndex();
-      catchup = lastAppliedIndex >= 20;
-    }
-    Assert.assertTrue("lastAppliedIndex=" + lastAppliedIndex, catchup);
-
-    // make sure the restarted peer's log segments is correct
-    cluster.restartServer(followerId, false);
-    Assert.assertTrue(cluster.getServer(followerId).getState().getLog()
-        .getLastEntry().getIndex() >= 20);
-  }
-}

http://git-wip-us.apache.org/repos/asf/incubator-ratis/blob/7e71a2e0/raft-examples/src/test/java/org/apache/raft/examples/RaftExamplesTestUtil.java
----------------------------------------------------------------------
diff --git a/raft-examples/src/test/java/org/apache/raft/examples/RaftExamplesTestUtil.java b/raft-examples/src/test/java/org/apache/raft/examples/RaftExamplesTestUtil.java
deleted file mode 100644
index ec622f4..0000000
--- a/raft-examples/src/test/java/org/apache/raft/examples/RaftExamplesTestUtil.java
+++ /dev/null
@@ -1,81 +0,0 @@
-/**
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- *     http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-package org.apache.raft.examples;
-
-import org.apache.raft.MiniRaftCluster;
-import org.apache.raft.conf.RaftProperties;
-import org.apache.raft.grpc.MiniRaftClusterWithGRpc;
-import org.apache.raft.hadooprpc.MiniRaftClusterWithHadoopRpc;
-import org.apache.raft.netty.MiniRaftClusterWithNetty;
-import org.apache.raft.server.simulation.MiniRaftClusterWithSimulatedRpc;
-import org.apache.raft.statemachine.StateMachine;
-
-import java.io.IOException;
-import java.util.*;
-
-public class RaftExamplesTestUtil {
-  private static void add(
-      Collection<Object[]> clusters, MiniRaftCluster.Factory factory,
-      String[] ids, RaftProperties properties)
-      throws IOException {
-    clusters.add(new Object[]{factory.newCluster(ids, properties, true)});
-  }
-
-  public static Collection<Object[]> getMiniRaftClusters(
-      RaftProperties prop, int clusterSize, Class<?>... clusterClasses)
-      throws IOException {
-    final List<Class<?>> classes = Arrays.asList(clusterClasses);
-    final boolean isAll = classes.isEmpty(); //empty means all
-
-    final Iterator<String[]> ids = new Iterator<String[]>() {
-      private int i = 0;
-      @Override
-      public boolean hasNext() {
-        return true;
-      }
-      @Override
-      public String[] next() {
-        return MiniRaftCluster.generateIds(clusterSize, i++*clusterSize);
-      }
-    };
-
-    final List<Object[]> clusters = new ArrayList<>();
-
-    if (isAll || classes.contains(MiniRaftClusterWithSimulatedRpc.class)) {
-      add(clusters, MiniRaftClusterWithSimulatedRpc.FACTORY, ids.next(), prop);
-    }
-    if (isAll || classes.contains(MiniRaftClusterWithHadoopRpc.class)) {
-      add(clusters, MiniRaftClusterWithHadoopRpc.FACTORY, ids.next(), prop);
-    }
-    if (isAll || classes.contains(MiniRaftClusterWithNetty.class)) {
-      add(clusters, MiniRaftClusterWithNetty.FACTORY, ids.next(), prop);
-    }
-    if (isAll || classes.contains(MiniRaftClusterWithGRpc.class)) {
-      add(clusters, MiniRaftClusterWithGRpc.FACTORY, ids.next(), prop);
-    }
-    return clusters;
-  }
-
-  public static <S extends StateMachine> Collection<Object[]> getMiniRaftClusters(
-      Class<S> stateMachineClass, Class<?>... clusterClasses) throws IOException {
-    final RaftProperties prop = new RaftProperties();
-    prop.setClass(MiniRaftCluster.STATEMACHINE_CLASS_KEY,
-        stateMachineClass, StateMachine.class);
-    return getMiniRaftClusters(prop, 3, clusterClasses);
-  }
-}

http://git-wip-us.apache.org/repos/asf/incubator-ratis/blob/7e71a2e0/raft-examples/src/test/java/org/apache/raft/examples/arithmetic/TestArithmetic.java
----------------------------------------------------------------------
diff --git a/raft-examples/src/test/java/org/apache/raft/examples/arithmetic/TestArithmetic.java b/raft-examples/src/test/java/org/apache/raft/examples/arithmetic/TestArithmetic.java
deleted file mode 100644
index 27b3814..0000000
--- a/raft-examples/src/test/java/org/apache/raft/examples/arithmetic/TestArithmetic.java
+++ /dev/null
@@ -1,104 +0,0 @@
-/**
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- *     http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-package org.apache.raft.examples.arithmetic;
-
-
-import org.apache.log4j.Level;
-import org.apache.raft.MiniRaftCluster;
-import org.apache.raft.RaftTestUtil;
-import org.apache.raft.client.RaftClient;
-import org.apache.raft.examples.RaftExamplesTestUtil;
-import org.apache.raft.examples.arithmetic.expression.*;
-import org.apache.raft.protocol.RaftClientReply;
-import org.apache.raft.util.RaftUtils;
-import org.junit.Assert;
-import org.junit.Test;
-import org.junit.runner.RunWith;
-import org.junit.runners.Parameterized;
-
-import java.io.IOException;
-import java.util.Collection;
-
-@RunWith(Parameterized.class)
-public class TestArithmetic {
-  static {
-    RaftUtils.setLogLevel(ArithmeticStateMachine.LOG, Level.ALL);
-  }
-
-  @Parameterized.Parameters
-  public static Collection<Object[]> data() throws IOException {
-    return RaftExamplesTestUtil.getMiniRaftClusters(ArithmeticStateMachine.class);
-  }
-
-  @Parameterized.Parameter
-  public MiniRaftCluster cluster;
-
-  @Test
-  public void testPythagorean() throws Exception {
-    cluster.start();
-    RaftTestUtil.waitForLeader(cluster);
-    final String leaderId = cluster.getLeader().getId();
-    final RaftClient client = cluster.createClient("pythagorean", leaderId);
-
-    final Variable a = new Variable("a");
-    final Variable b = new Variable("b");
-    final Variable c = new Variable("c");
-    final BinaryExpression a2 = new BinaryExpression(BinaryExpression.Op.MULT, a, a);
-    final BinaryExpression b2 = new BinaryExpression(BinaryExpression.Op.MULT, b, b);
-    final BinaryExpression c2 = new BinaryExpression(BinaryExpression.Op.ADD, a2, b2);
-    final AssignmentMessage pythagorean = new AssignmentMessage(c,
-        new UnaryExpression(UnaryExpression.Op.SQRT, c2));
-
-    final AssignmentMessage nullA = new AssignmentMessage(a, NullValue.getInstance());
-    final AssignmentMessage nullB = new AssignmentMessage(b, NullValue.getInstance());
-    final AssignmentMessage nullC = new AssignmentMessage(c, NullValue.getInstance());
-
-    for(int n = 3; n < 100; n += 2) {
-      int n2 = n*n;
-      int half_n2 = n2/2;
-
-      RaftClientReply r;
-      r = client.send(new AssignmentMessage(a, new DoubleValue(n)));
-      assertRaftClientReply(r, (double)n);
-      r = client.sendReadOnly(Expression.Utils.toMessage(a2));
-      assertRaftClientReply(r, (double)n2);
-      r = client.send(new AssignmentMessage(b, new DoubleValue(half_n2)));
-      assertRaftClientReply(r, (double)half_n2);
-      r = client.sendReadOnly(Expression.Utils.toMessage(b2));
-      assertRaftClientReply(r, (double)half_n2*half_n2);
-      r = client.send(pythagorean);
-      assertRaftClientReply(r, (double)half_n2 + 1);
-
-      r = client.send(nullA);
-      assertRaftClientReply(r, null);
-      r = client.send(nullB);
-      assertRaftClientReply(r, null);
-      r = client.send(nullC);
-      assertRaftClientReply(r, null);
-    }
-    client.close();
-    cluster.shutdown();
-  }
-
-  static void assertRaftClientReply(RaftClientReply reply, Double expected) {
-    Assert.assertTrue(reply.isSuccess());
-    final Expression e = Expression.Utils.bytes2Expression(
-        reply.getMessage().getContent().toByteArray(), 0);
-    Assert.assertEquals(expected, e.evaluate(null));
-  }
-}

http://git-wip-us.apache.org/repos/asf/incubator-ratis/blob/7e71a2e0/raft-examples/src/test/java/org/apache/raft/examples/arithmetic/expression/TestExpression.java
----------------------------------------------------------------------
diff --git a/raft-examples/src/test/java/org/apache/raft/examples/arithmetic/expression/TestExpression.java b/raft-examples/src/test/java/org/apache/raft/examples/arithmetic/expression/TestExpression.java
deleted file mode 100644
index f06c88e..0000000
--- a/raft-examples/src/test/java/org/apache/raft/examples/arithmetic/expression/TestExpression.java
+++ /dev/null
@@ -1,102 +0,0 @@
-/**
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- *     http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-package org.apache.raft.examples.arithmetic.expression;
-
-
-import org.junit.Assert;
-import org.junit.Test;
-import org.slf4j.Logger;
-import org.slf4j.LoggerFactory;
-
-import java.util.Random;
-import java.util.concurrent.ThreadLocalRandom;
-
-public class TestExpression {
-  static final Logger LOG = LoggerFactory.getLogger(TestExpression.class);
-
-  @Test
-  public void testArithmeticUtils() throws Exception {
-    final Random ran = ThreadLocalRandom.current();
-    final byte[] buf = new byte[1024];
-    int offset = 0;
-
-    for(int i = 0; i < 10; i++) {
-      {
-        final int n = ran.nextInt();
-        Expression.Utils.int2bytes(n, buf, offset);
-        final int m = Expression.Utils.bytes2int(buf, offset);
-        Assert.assertEquals(n, m);
-        offset += 4;
-      }
-      {
-        final long n = ran.nextLong();
-        Expression.Utils.long2bytes(n, buf, offset);
-        final long m = Expression.Utils.bytes2long(buf, offset);
-        Assert.assertEquals(n, m);
-        offset += 8;
-      }
-      {
-        final double n = ran.nextDouble();
-        Expression.Utils.double2bytes(n, buf, offset);
-        final double m = Expression.Utils.bytes2double(buf, offset);
-        Assert.assertTrue(n == m);
-        offset += 8;
-      }
-    }
-  }
-  @Test
-  public void testOp() throws Exception {
-    for(BinaryExpression.Op op : BinaryExpression.Op.values()) {
-      final byte b = op.byteValue();
-      Assert.assertEquals(op, BinaryExpression.Op.valueOf(b));
-    }
-    for(UnaryExpression.Op op : UnaryExpression.Op.values()) {
-      final byte b = op.byteValue();
-      Assert.assertEquals(op, UnaryExpression.Op.valueOf(b));
-    }
-  }
-
-  @Test
-  public void testExpression() throws Exception {
-    final byte[] buf = new byte[1024];
-    int offset = 0;
-
-    {
-      final Variable a = new Variable("pi");
-      LOG.info("var a: " + a);
-      final int len = a.toBytes(buf, offset);
-      final Variable a2 = new Variable(buf, offset);
-      LOG.info("var a2: " + a2);
-      Assert.assertEquals(a.getName(), a2.getName());
-      Assert.assertEquals(len, a.length());
-      Assert.assertEquals(len, a2.length());
-      offset += len;
-    }
-
-    {
-      final DoubleValue three = new DoubleValue(3);
-      LOG.info("double three: " + three.evaluate(null));
-      final int len = three.toBytes(buf, offset);
-      final DoubleValue three2 = new DoubleValue(buf, offset);
-      LOG.info("double three2: " + three2.evaluate(null));
-      Assert.assertTrue(three.evaluate(null).equals(three2.evaluate(null)));
-      Assert.assertEquals(len, three.length());
-      Assert.assertEquals(len, three2.length());
-    }
-  }
-}

http://git-wip-us.apache.org/repos/asf/incubator-ratis/blob/7e71a2e0/raft-examples/src/test/java/org/apache/raft/statemachine/TestRaftStateMachineException.java
----------------------------------------------------------------------
diff --git a/raft-examples/src/test/java/org/apache/raft/statemachine/TestRaftStateMachineException.java b/raft-examples/src/test/java/org/apache/raft/statemachine/TestRaftStateMachineException.java
deleted file mode 100644
index 0832579..0000000
--- a/raft-examples/src/test/java/org/apache/raft/statemachine/TestRaftStateMachineException.java
+++ /dev/null
@@ -1,86 +0,0 @@
-/**
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- *     http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-package org.apache.raft.statemachine;
-
-import org.apache.log4j.Level;
-import org.apache.raft.MiniRaftCluster;
-import org.apache.raft.RaftTestUtil;
-import org.apache.raft.client.RaftClient;
-import org.apache.raft.examples.RaftExamplesTestUtil;
-import org.apache.raft.protocol.Message;
-import org.apache.raft.protocol.StateMachineException;
-import org.apache.raft.server.impl.RaftServerImpl;
-import org.apache.raft.server.simulation.RequestHandler;
-import org.apache.raft.server.storage.RaftLog;
-import org.apache.raft.util.RaftUtils;
-import org.junit.Assert;
-import org.junit.Test;
-import org.junit.runner.RunWith;
-import org.junit.runners.Parameterized;
-
-import java.io.IOException;
-import java.util.Collection;
-import java.util.concurrent.CompletableFuture;
-
-import static org.junit.Assert.fail;
-
-@RunWith(Parameterized.class)
-public class TestRaftStateMachineException {
-  static {
-    RaftUtils.setLogLevel(RaftServerImpl.LOG, Level.DEBUG);
-    RaftUtils.setLogLevel(RaftLog.LOG, Level.DEBUG);
-    RaftUtils.setLogLevel(RequestHandler.LOG, Level.DEBUG);
-    RaftUtils.setLogLevel(RaftClient.LOG, Level.DEBUG);
-  }
-
-  protected static class StateMachineWithException extends SimpleStateMachine4Testing {
-    @Override
-    public CompletableFuture<Message> applyTransaction(TransactionContext trx) {
-      CompletableFuture<Message> future = new CompletableFuture<>();
-      future.completeExceptionally(new StateMachineException("Fake Exception"));
-      return future;
-    }
-  }
-
-  @Parameterized.Parameters
-  public static Collection<Object[]> data() throws IOException {
-    return RaftExamplesTestUtil.getMiniRaftClusters(
-        StateMachineWithException.class);
-  }
-
-  @Parameterized.Parameter
-  public MiniRaftCluster cluster;
-
-  @Test
-  public void testHandleStateMachineException() throws Exception {
-    cluster.start();
-    RaftTestUtil.waitForLeader(cluster);
-
-    final String leaderId = cluster.getLeader().getId();
-
-    try(final RaftClient client = cluster.createClient("client", leaderId)) {
-      client.send(new RaftTestUtil.SimpleMessage("m"));
-      fail("Exception expected");
-    } catch (StateMachineException e) {
-      e.printStackTrace();
-      Assert.assertTrue(e.getMessage().contains("Fake Exception"));
-    }
-
-    cluster.shutdown();
-  }
-}

http://git-wip-us.apache.org/repos/asf/incubator-ratis/blob/7e71a2e0/raft-examples/src/test/resources/log4j.properties
----------------------------------------------------------------------
diff --git a/raft-examples/src/test/resources/log4j.properties b/raft-examples/src/test/resources/log4j.properties
deleted file mode 100644
index ced0687..0000000
--- a/raft-examples/src/test/resources/log4j.properties
+++ /dev/null
@@ -1,18 +0,0 @@
-#   Licensed 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.
-# log4j configuration used during build and unit tests
-
-log4j.rootLogger=info,stdout
-log4j.threshold=ALL
-log4j.appender.stdout=org.apache.log4j.ConsoleAppender
-log4j.appender.stdout.layout=org.apache.log4j.PatternLayout
-log4j.appender.stdout.layout.ConversionPattern=%d{ISO8601} %-5p %c{2} (%F:%M(%L)) - %m%n

http://git-wip-us.apache.org/repos/asf/incubator-ratis/blob/7e71a2e0/raft-grpc/pom.xml
----------------------------------------------------------------------
diff --git a/raft-grpc/pom.xml b/raft-grpc/pom.xml
deleted file mode 100644
index 9b712fd..0000000
--- a/raft-grpc/pom.xml
+++ /dev/null
@@ -1,93 +0,0 @@
-<?xml version="1.0" encoding="UTF-8"?>
-<!--
-  Licensed 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. See accompanying LICENSE file.
--->
-<project xmlns="http://maven.apache.org/POM/4.0.0"
-         xmlns:xsi="http://www.w3.org/2001/XMLSchema-instance"
-         xsi:schemaLocation="http://maven.apache.org/POM/4.0.0 http://maven.apache.org/xsd/maven-4.0.0.xsd">
-  <modelVersion>4.0.0</modelVersion>
-  <parent>
-    <artifactId>raft-project-dist</artifactId>
-    <groupId>com.hortonworks.raft</groupId>
-    <version>1.0-SNAPSHOT</version>
-    <relativePath>../raft-project-dist</relativePath>
-  </parent>
-
-  <artifactId>raft-grpc</artifactId>
-  <name>Raft gRPC Support</name>
-
-  <dependencies>
-    <dependency>
-      <artifactId>raft-proto-shaded</artifactId>
-      <groupId>com.hortonworks.raft</groupId>
-      <scope>provided</scope>
-    </dependency>
-
-    <dependency>
-      <artifactId>raft-common</artifactId>
-      <groupId>com.hortonworks.raft</groupId>
-      <scope>provided</scope>
-    </dependency>
-    <dependency>
-      <artifactId>raft-common</artifactId>
-      <groupId>com.hortonworks.raft</groupId>
-      <scope>test</scope>
-      <type>test-jar</type>
-    </dependency>
-    
-    <dependency>
-      <artifactId>raft-client</artifactId>
-      <groupId>com.hortonworks.raft</groupId>
-      <scope>provided</scope>
-    </dependency>
-    <dependency>
-      <artifactId>raft-client</artifactId>
-      <groupId>com.hortonworks.raft</groupId>
-      <scope>test</scope>
-      <type>test-jar</type>
-    </dependency>
-    
-    <dependency>
-      <artifactId>raft-server</artifactId>
-      <groupId>com.hortonworks.raft</groupId>
-      <scope>provided</scope>
-    </dependency>
-    <dependency>
-      <artifactId>raft-server</artifactId>
-      <groupId>com.hortonworks.raft</groupId>
-      <scope>test</scope>
-      <type>test-jar</type>
-    </dependency>
-    
-    <dependency>
-      <groupId>org.slf4j</groupId>
-      <artifactId>slf4j-api</artifactId>
-    </dependency>
-
-    <dependency>
-      <groupId>com.google.guava</groupId>
-      <artifactId>guava</artifactId>
-    </dependency>
-
-    <dependency>
-      <groupId>junit</groupId>
-      <artifactId>junit</artifactId>
-      <scope>test</scope>
-    </dependency>
-    <dependency>
-      <groupId>org.mockito</groupId>
-      <artifactId>mockito-all</artifactId>
-      <scope>test</scope>
-    </dependency>
-  </dependencies>
-</project>

http://git-wip-us.apache.org/repos/asf/incubator-ratis/blob/7e71a2e0/raft-grpc/src/main/java/org/apache/raft/grpc/RaftGRpcService.java
----------------------------------------------------------------------
diff --git a/raft-grpc/src/main/java/org/apache/raft/grpc/RaftGRpcService.java b/raft-grpc/src/main/java/org/apache/raft/grpc/RaftGRpcService.java
deleted file mode 100644
index 1184e2e..0000000
--- a/raft-grpc/src/main/java/org/apache/raft/grpc/RaftGRpcService.java
+++ /dev/null
@@ -1,151 +0,0 @@
-/**
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- *     http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-package org.apache.raft.grpc;
-
-import com.google.common.base.Preconditions;
-import org.apache.raft.conf.RaftProperties;
-import org.apache.raft.grpc.client.RaftClientProtocolService;
-import org.apache.raft.grpc.server.RaftServerProtocolClient;
-import org.apache.raft.grpc.server.RaftServerProtocolService;
-import org.apache.raft.protocol.RaftPeer;
-import org.apache.raft.server.RaftServer;
-import org.apache.raft.server.RaftServerRpc;
-import org.apache.raft.shaded.io.grpc.Server;
-import org.apache.raft.shaded.io.grpc.ServerBuilder;
-import org.apache.raft.shaded.io.grpc.netty.NettyServerBuilder;
-import org.apache.raft.shaded.proto.RaftProtos.*;
-import org.apache.raft.util.CodeInjectionForTesting;
-import org.slf4j.Logger;
-import org.slf4j.LoggerFactory;
-
-import java.io.IOException;
-import java.net.InetSocketAddress;
-import java.util.Collections;
-import java.util.HashMap;
-import java.util.Map;
-
-import static org.apache.raft.grpc.RaftGrpcConfigKeys.RAFT_GRPC_SERVER_PORT_DEFAULT;
-import static org.apache.raft.grpc.RaftGrpcConfigKeys.RAFT_GRPC_SERVER_PORT_KEY;
-
-public class RaftGRpcService implements RaftServerRpc {
-  static final Logger LOG = LoggerFactory.getLogger(RaftGRpcService.class);
-  public static final String GRPC_SEND_SERVER_REQUEST =
-      RaftGRpcService.class.getSimpleName() + ".sendRequest";
-
-  private final Server server;
-  private final InetSocketAddress address;
-  private final Map<String, RaftServerProtocolClient> peers =
-      Collections.synchronizedMap(new HashMap<>());
-  private final String selfId;
-
-  public RaftGRpcService(RaftServer raftServer, RaftProperties properties) {
-    int port = properties.getInt(RAFT_GRPC_SERVER_PORT_KEY,
-        RAFT_GRPC_SERVER_PORT_DEFAULT);
-    int maxMessageSize = properties.getInt(
-        RaftGrpcConfigKeys.RAFT_GRPC_MESSAGE_MAXSIZE_KEY,
-        RaftGrpcConfigKeys.RAFT_GRPC_MESSAGE_MAXSIZE_DEFAULT);
-    ServerBuilder serverBuilder = ServerBuilder.forPort(port);
-    selfId = raftServer.getId();
-    server = ((NettyServerBuilder) serverBuilder).maxMessageSize(maxMessageSize)
-        .addService(new RaftServerProtocolService(selfId, raftServer))
-        .addService(new RaftClientProtocolService(selfId, raftServer))
-        .build();
-
-    // start service to determine the port (in case port is configured as 0)
-    startService();
-    address = new InetSocketAddress(server.getPort());
-    LOG.info("Server started, listening on " + address.getPort());
-  }
-
-  @Override
-  public void start() {
-    // do nothing
-  }
-
-  private void startService() {
-    try {
-      server.start();
-    } catch (IOException e) {
-      LOG.error("Failed to start Grpc server", e);
-      System.exit(1);
-    }
-    Runtime.getRuntime().addShutdownHook(new Thread() {
-      @Override
-      public void run() {
-        System.err.println("*** shutting down gRPC server since JVM is shutting down");
-        RaftGRpcService.this.close();
-        System.err.println("*** server shut down");
-      }
-    });
-  }
-
-  @Override
-  public void close() {
-    if (server != null) {
-      server.shutdown();
-    }
-    shutdownClients();
-  }
-
-  @Override
-  public InetSocketAddress getInetSocketAddress() {
-    return address;
-  }
-
-  @Override
-  public AppendEntriesReplyProto appendEntries(
-      AppendEntriesRequestProto request) throws IOException {
-    throw new UnsupportedOperationException(
-        "Blocking AppendEntries call is not supported");
-  }
-
-  @Override
-  public InstallSnapshotReplyProto installSnapshot(
-      InstallSnapshotRequestProto request) throws IOException {
-    throw new UnsupportedOperationException(
-        "Blocking InstallSnapshot call is not supported");
-  }
-
-  @Override
-  public RequestVoteReplyProto requestVote(RequestVoteRequestProto request)
-      throws IOException {
-    CodeInjectionForTesting.execute(GRPC_SEND_SERVER_REQUEST, selfId,
-        null, request);
-
-    RaftServerProtocolClient target = Preconditions.checkNotNull(
-        peers.get(request.getServerRequest().getReplyId()));
-    return target.requestVote(request);
-  }
-
-  @Override
-  public void addPeers(Iterable<RaftPeer> newPeers) {
-    for (RaftPeer p : newPeers) {
-      if (!peers.containsKey(p.getId())) {
-        peers.put(p.getId(), new RaftServerProtocolClient(p));
-      }
-    }
-  }
-
-  private void shutdownClients() {
-    peers.values().forEach(RaftServerProtocolClient::shutdown);
-  }
-
-  public RaftServerProtocolClient getRpcClient(RaftPeer peer) {
-    return peers.get(peer.getId());
-  }
-}

http://git-wip-us.apache.org/repos/asf/incubator-ratis/blob/7e71a2e0/raft-grpc/src/main/java/org/apache/raft/grpc/RaftGrpcConfigKeys.java
----------------------------------------------------------------------
diff --git a/raft-grpc/src/main/java/org/apache/raft/grpc/RaftGrpcConfigKeys.java b/raft-grpc/src/main/java/org/apache/raft/grpc/RaftGrpcConfigKeys.java
deleted file mode 100644
index 395848b..0000000
--- a/raft-grpc/src/main/java/org/apache/raft/grpc/RaftGrpcConfigKeys.java
+++ /dev/null
@@ -1,47 +0,0 @@
-/**
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- *     http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-package org.apache.raft.grpc;
-
-import org.apache.raft.client.RaftClientConfigKeys;
-
-public interface RaftGrpcConfigKeys {
-  String PREFIX = "raft.grpc";
-
-  String RAFT_GRPC_SERVER_PORT_KEY = PREFIX + ".server.port";
-  int RAFT_GRPC_SERVER_PORT_DEFAULT = 0;
-
-  String RAFT_GRPC_MESSAGE_MAXSIZE_KEY = PREFIX + ".message.maxsize";
-  int RAFT_GRPC_MESSAGE_MAXSIZE_DEFAULT = 64 * 1024 * 1024; // 64 MB
-
-  String RAFT_GRPC_LEADER_MAX_OUTSTANDING_APPENDS_KEY =
-      PREFIX + "leader.max.outstanding.appends";
-  int RAFT_GRPC_LEADER_MAX_OUTSTANDING_APPENDS_DEFAULT = 128;
-
-  String RAFT_GRPC_CLIENT_MAX_OUTSTANDING_APPENDS_KEY =
-      PREFIX + "client.max.outstanding.appends";
-  int RAFT_GRPC_CLIENT_MAX_OUTSTANDING_APPENDS_DEFAULT = 128;
-
-  String RAFT_OUTPUTSTREAM_BUFFER_SIZE_KEY = "raft.outputstream.buffer.size";
-  int RAFT_OUTPUTSTREAM_BUFFER_SIZE_DEFAULT = 64 * 1024;
-
-  String RAFT_OUTPUTSTREAM_MAX_RETRY_TIMES_KEY = "raft.outputstream.max.retry.times";
-  int RAFT_OUTPUTSTREAM_MAX_RETRY_TIMES_DEFAULT = 5;
-
-  String RAFT_OUTPUTSTREAM_RETRY_INTERVAL_KEY = "raft.outputstream.retry.interval";
-  long RAFT_OUTPUTSTREAM_RETRY_INTERVAL_DEFAULT = RaftClientConfigKeys.RAFT_RPC_TIMEOUT_MS_DEFAULT;
-}

http://git-wip-us.apache.org/repos/asf/incubator-ratis/blob/7e71a2e0/raft-grpc/src/main/java/org/apache/raft/grpc/RaftGrpcUtil.java
----------------------------------------------------------------------
diff --git a/raft-grpc/src/main/java/org/apache/raft/grpc/RaftGrpcUtil.java b/raft-grpc/src/main/java/org/apache/raft/grpc/RaftGrpcUtil.java
deleted file mode 100644
index 6afb39b..0000000
--- a/raft-grpc/src/main/java/org/apache/raft/grpc/RaftGrpcUtil.java
+++ /dev/null
@@ -1,87 +0,0 @@
-/**
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- *     http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-package org.apache.raft.grpc;
-
-import org.apache.raft.shaded.io.grpc.Metadata;
-import org.apache.raft.shaded.io.grpc.Status;
-import org.apache.raft.shaded.io.grpc.StatusRuntimeException;
-import org.apache.raft.util.RaftUtils;
-
-import java.io.IOException;
-import java.io.PrintWriter;
-import java.io.StringWriter;
-import java.lang.reflect.Constructor;
-
-public class RaftGrpcUtil {
-  public static final Metadata.Key<String> EXCEPTION_TYPE_KEY =
-      Metadata.Key.of("exception-type", Metadata.ASCII_STRING_MARSHALLER);
-
-  public static String stringifyException(Throwable e) {
-    StringWriter stm = new StringWriter();
-    PrintWriter wrt = new PrintWriter(stm);
-    e.printStackTrace(wrt);
-    wrt.close();
-    return stm.toString();
-  }
-
-  public static StatusRuntimeException wrapException(Throwable t) {
-    Metadata trailers = new Metadata();
-    trailers.put(EXCEPTION_TYPE_KEY, t.getClass().getCanonicalName());
-    return new StatusRuntimeException(
-        Status.INTERNAL.withDescription(RaftGrpcUtil.stringifyException(t)),
-        trailers);
-  }
-
-  public static IOException unwrapException(StatusRuntimeException se) {
-    final Metadata trailers = se.getTrailers();
-    final Status status = se.getStatus();
-    if (trailers != null && status != null) {
-      final String className = trailers.get(EXCEPTION_TYPE_KEY);
-      if (className != null) {
-        try {
-          Class<?> clazz = Class.forName(className);
-          final Exception unwrapped = instantiateException(
-              clazz.asSubclass(Exception.class), status.getDescription(), se);
-          return RaftUtils.asIOException(unwrapped);
-        } catch (Exception e) {
-          return new IOException(se);
-        }
-      }
-    }
-    return new IOException(se);
-  }
-
-  public static IOException unwrapIOException(Throwable t) {
-    final IOException e;
-    if (t instanceof StatusRuntimeException) {
-      e = RaftGrpcUtil.unwrapException((StatusRuntimeException) t);
-    } else {
-      e = RaftUtils.asIOException(t);
-    }
-    return e;
-  }
-
-  private static Exception instantiateException(Class<? extends Exception> cls,
-      String message, Exception from) throws Exception {
-    Constructor<? extends Exception> cn = cls.getConstructor(String.class);
-    cn.setAccessible(true);
-    Exception ex = cn.newInstance(message);
-    ex.initCause(from);
-    return ex;
-  }
-}

http://git-wip-us.apache.org/repos/asf/incubator-ratis/blob/7e71a2e0/raft-grpc/src/main/java/org/apache/raft/grpc/client/AppendStreamer.java
----------------------------------------------------------------------
diff --git a/raft-grpc/src/main/java/org/apache/raft/grpc/client/AppendStreamer.java b/raft-grpc/src/main/java/org/apache/raft/grpc/client/AppendStreamer.java
deleted file mode 100644
index 9cf8cd5..0000000
--- a/raft-grpc/src/main/java/org/apache/raft/grpc/client/AppendStreamer.java
+++ /dev/null
@@ -1,395 +0,0 @@
-/**
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- *     http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-package org.apache.raft.grpc.client;
-
-import com.google.common.base.Preconditions;
-import org.apache.raft.conf.RaftProperties;
-import org.apache.raft.grpc.RaftGrpcConfigKeys;
-import org.apache.raft.grpc.RaftGrpcUtil;
-import org.apache.raft.protocol.NotLeaderException;
-import org.apache.raft.protocol.RaftClientReply;
-import org.apache.raft.protocol.RaftPeer;
-import org.apache.raft.shaded.com.google.protobuf.ByteString;
-import org.apache.raft.shaded.proto.RaftProtos.RaftClientReplyProto;
-import org.apache.raft.shaded.proto.RaftProtos.RaftClientRequestProto;
-import org.apache.raft.shaded.proto.RaftProtos.RaftRpcRequestProto;
-import org.apache.raft.util.Daemon;
-import org.apache.raft.util.PeerProxyMap;
-import org.apache.raft.util.RaftUtils;
-import org.apache.raft.util.StringUtils;
-import org.slf4j.Logger;
-import org.slf4j.LoggerFactory;
-
-import java.io.Closeable;
-import java.io.IOException;
-import java.util.*;
-import java.util.concurrent.ConcurrentLinkedDeque;
-import java.util.concurrent.TimeUnit;
-import java.util.concurrent.atomic.AtomicInteger;
-import java.util.function.Function;
-import java.util.stream.Collectors;
-
-import static org.apache.raft.client.impl.ClientProtoUtils.*;
-import static org.apache.raft.grpc.RaftGrpcConfigKeys.RAFT_GRPC_CLIENT_MAX_OUTSTANDING_APPENDS_DEFAULT;
-import static org.apache.raft.grpc.RaftGrpcConfigKeys.RAFT_GRPC_CLIENT_MAX_OUTSTANDING_APPENDS_KEY;
-
-public class AppendStreamer implements Closeable {
-  public static final Logger LOG = LoggerFactory.getLogger(AppendStreamer.class);
-
-  enum RunningState {RUNNING, LOOK_FOR_LEADER, CLOSED, ERROR}
-
-  private static class ExceptionAndRetry {
-    private final Map<String, IOException> exceptionMap = new HashMap<>();
-    private final AtomicInteger retryTimes = new AtomicInteger(0);
-    private final int maxRetryTimes;
-    private final long retryInterval;
-
-    ExceptionAndRetry(RaftProperties prop) {
-      maxRetryTimes = prop.getInt(
-          RaftGrpcConfigKeys.RAFT_OUTPUTSTREAM_MAX_RETRY_TIMES_KEY,
-          RaftGrpcConfigKeys.RAFT_OUTPUTSTREAM_MAX_RETRY_TIMES_DEFAULT);
-      retryInterval = prop.getTimeDuration(
-          RaftGrpcConfigKeys.RAFT_OUTPUTSTREAM_RETRY_INTERVAL_KEY,
-          RaftGrpcConfigKeys.RAFT_OUTPUTSTREAM_RETRY_INTERVAL_DEFAULT,
-          TimeUnit.MILLISECONDS);
-    }
-
-    void addException(String peer, IOException e) {
-      exceptionMap.put(peer, e);
-      retryTimes.incrementAndGet();
-    }
-
-    IOException getCombinedException() {
-      return new IOException("Exceptions: " + exceptionMap);
-    }
-
-    boolean shouldRetry() {
-      return retryTimes.get() <= maxRetryTimes;
-    }
-  }
-
-  private final Deque<RaftClientRequestProto> dataQueue;
-  private final Deque<RaftClientRequestProto> ackQueue;
-  private final int maxPendingNum;
-
-  private final PeerProxyMap<RaftClientProtocolProxy> proxyMap;
-  private final Map<String, RaftPeer> peers;
-  private String leaderId;
-  private volatile RaftClientProtocolProxy leaderProxy;
-  private final String clientId;
-
-  private volatile RunningState running = RunningState.RUNNING;
-  private final ExceptionAndRetry exceptionAndRetry;
-  private final Sender senderThread;
-
-  AppendStreamer(RaftProperties prop, Collection<RaftPeer> peers,
-      String leaderId, String clientId) {
-    this.clientId = clientId;
-    maxPendingNum = prop.getInt(
-        RAFT_GRPC_CLIENT_MAX_OUTSTANDING_APPENDS_KEY,
-        RAFT_GRPC_CLIENT_MAX_OUTSTANDING_APPENDS_DEFAULT);
-    dataQueue = new ConcurrentLinkedDeque<>();
-    ackQueue = new ConcurrentLinkedDeque<>();
-    exceptionAndRetry = new ExceptionAndRetry(prop);
-
-    this.peers = peers.stream().collect(
-        Collectors.toMap(RaftPeer::getId, Function.identity()));
-    proxyMap = new PeerProxyMap<>(
-        raftPeer -> new RaftClientProtocolProxy(raftPeer, ResponseHandler::new));
-    proxyMap.addPeers(peers);
-    refreshLeaderProxy(leaderId, null);
-
-    senderThread = new Sender();
-    senderThread.setName(this.toString() + "-sender");
-    senderThread.start();
-  }
-
-  private synchronized void refreshLeaderProxy(String suggested,
-      String oldLeader) {
-    if (suggested != null) {
-      leaderId = suggested;
-    } else {
-      if (oldLeader == null) {
-        leaderId = peers.keySet().iterator().next();
-      } else {
-        leaderId = StringUtils.next(oldLeader, peers.keySet());
-      }
-    }
-    LOG.debug("{} switches leader from {} to {}. suggested leader: {}", this,
-          oldLeader, leaderId, suggested);
-    if (leaderProxy != null) {
-      leaderProxy.closeCurrentSession();
-    }
-    try {
-      leaderProxy = proxyMap.getProxy(leaderId);
-    } catch (IOException e) {
-      LOG.error("Should not hit IOException here", e);
-      refreshLeader(null, leaderId);
-    }
-  }
-
-  private boolean isRunning() {
-    return running == RunningState.RUNNING ||
-        running == RunningState.LOOK_FOR_LEADER;
-  }
-
-  private void checkState() throws IOException {
-    if (!isRunning()) {
-      throwException("The AppendStreamer has been closed");
-    }
-  }
-
-  synchronized void write(ByteString content, long seqNum)
-      throws IOException {
-    checkState();
-    while (isRunning() && dataQueue.size() >= maxPendingNum) {
-      try {
-        wait();
-      } catch (InterruptedException ignored) {
-      }
-    }
-    if (isRunning()) {
-      // wrap the current buffer into a RaftClientRequestProto
-      final RaftClientRequestProto request = genRaftClientRequestProto(
-          clientId, leaderId, seqNum, content, false);
-      dataQueue.offer(request);
-      this.notifyAll();
-    } else {
-      throwException(this + " got closed.");
-    }
-  }
-
-  synchronized void flush() throws IOException {
-    checkState();
-    if (dataQueue.isEmpty() && ackQueue.isEmpty()) {
-      return;
-    }
-    // wait for the pending Q to become empty
-    while (isRunning() && (!dataQueue.isEmpty() || !ackQueue.isEmpty())) {
-      try {
-        wait();
-      } catch (InterruptedException ignored) {
-      }
-    }
-    if (!isRunning() && (!dataQueue.isEmpty() || !ackQueue.isEmpty())) {
-      throwException(this + " got closed before finishing flush");
-    }
-  }
-
-  @Override
-  public void close() throws IOException {
-    if (!isRunning()) {
-      return;
-    }
-    flush();
-
-    running = RunningState.CLOSED;
-    senderThread.interrupt();
-    try {
-      senderThread.join();
-    } catch (InterruptedException ignored) {
-    }
-    proxyMap.close();
-  }
-
-  @Override
-  public String toString() {
-    return this.getClass().getSimpleName() + "-" + clientId;
-  }
-
-  private class Sender extends Daemon {
-    @Override
-    public void run() {
-      while (isRunning()) {
-
-        synchronized (AppendStreamer.this) {
-          while (isRunning() && shouldWait()) {
-            try {
-              AppendStreamer.this.wait();
-            } catch (InterruptedException ignored) {
-            }
-          }
-          if (running == RunningState.RUNNING) {
-            RaftClientRequestProto next = dataQueue.poll();
-            leaderProxy.onNext(next);
-            ackQueue.offer(next);
-          }
-        }
-      }
-    }
-
-    private boolean shouldWait() {
-      // the sender should wait if any of the following is true
-      // 1) there is no data to send
-      // 2) there are too many outstanding pending requests
-      // 3) Error/NotLeaderException just happened, we're still waiting for
-      //    the first response to confirm the new leader
-      return dataQueue.isEmpty() || ackQueue.size() >= maxPendingNum ||
-          running == RunningState.LOOK_FOR_LEADER;
-    }
-  }
-
-  /** the response handler for stream RPC */
-  private class ResponseHandler implements
-      RaftClientProtocolProxy.CloseableStreamObserver {
-    private final String targetId;
-    // once handled the first NotLeaderException or Error, the handler should
-    // be inactive and should not make any further action.
-    private volatile boolean active = true;
-
-    ResponseHandler(RaftPeer target) {
-      targetId = target.getId();
-    }
-
-    @Override
-    public String toString() {
-      return AppendStreamer.this + "-ResponseHandler-" + targetId;
-    }
-
-    @Override
-    public void onNext(RaftClientReplyProto reply) {
-      if (!active) {
-        return;
-      }
-      synchronized (AppendStreamer.this) {
-        RaftClientRequestProto pending = Preconditions.checkNotNull(
-            ackQueue.peek());
-        if (reply.getRpcReply().getSuccess()) {
-          Preconditions.checkState(pending.getRpcRequest().getSeqNum() ==
-              reply.getRpcReply().getSeqNum());
-          ackQueue.poll();
-          LOG.trace("{} received success ack for request {}", this,
-              pending.getRpcRequest());
-          // we've identified the correct leader
-          if (running == RunningState.LOOK_FOR_LEADER) {
-            running = RunningState.RUNNING;
-          }
-        } else {
-          // this may be a NotLeaderException
-          RaftClientReply r = toRaftClientReply(reply);
-          if (r.isNotLeader()) {
-            LOG.debug("{} received a NotLeaderException from {}", this,
-                r.getReplierId());
-            handleNotLeader(r.getNotLeaderException(), targetId);
-          }
-        }
-        AppendStreamer.this.notifyAll();
-      }
-    }
-
-    @Override
-    public void onError(Throwable t) {
-      if (active) {
-        synchronized (AppendStreamer.this) {
-          handleError(t, this);
-          AppendStreamer.this.notifyAll();
-        }
-      }
-    }
-
-    @Override
-    public void onCompleted() {
-      LOG.info("{} onCompleted, pending requests #: {}", this,
-          ackQueue.size());
-    }
-
-    @Override // called by handleError and handleNotLeader
-    public void close() throws IOException {
-      active = false;
-    }
-  }
-
-  private void throwException(String msg) throws IOException {
-    if (running == RunningState.ERROR) {
-      throw exceptionAndRetry.getCombinedException();
-    } else {
-      throw new IOException(msg);
-    }
-  }
-
-  private void handleNotLeader(NotLeaderException nle,
-      String oldLeader) {
-    Preconditions.checkState(Thread.holdsLock(AppendStreamer.this));
-    // handle NotLeaderException: refresh leader and RaftConfiguration
-    refreshPeers(nle.getPeers());
-
-    refreshLeader(nle.getSuggestedLeader().getId(), oldLeader);
-  }
-
-  private void handleError(Throwable t, ResponseHandler handler) {
-    Preconditions.checkState(Thread.holdsLock(AppendStreamer.this));
-    final IOException e = RaftGrpcUtil.unwrapIOException(t);
-
-    exceptionAndRetry.addException(handler.targetId, e);
-    LOG.debug("{} got error: {}. Total retry times {}, max retry times {}.",
-        handler, e, exceptionAndRetry.retryTimes.get(),
-        exceptionAndRetry.maxRetryTimes);
-
-    leaderProxy.onError();
-    if (exceptionAndRetry.shouldRetry()) {
-      refreshLeader(null, leaderId);
-    } else {
-      running = RunningState.ERROR;
-    }
-  }
-
-  private void refreshLeader(String suggestedLeader, String oldLeader) {
-    running = RunningState.LOOK_FOR_LEADER;
-    refreshLeaderProxy(suggestedLeader, oldLeader);
-    reQueuePendingRequests(leaderId);
-
-    final RaftClientRequestProto request = Preconditions.checkNotNull(
-        dataQueue.poll());
-    ackQueue.offer(request);
-    try {
-      Thread.sleep(exceptionAndRetry.retryInterval);
-    } catch (InterruptedException ignored) {
-    }
-    leaderProxy.onNext(request);
-  }
-
-  private void reQueuePendingRequests(String newLeader) {
-    if (isRunning()) {
-      // resend all the pending requests
-      while (!ackQueue.isEmpty()) {
-        RaftClientRequestProto oldRequest = ackQueue.pollLast();
-        RaftRpcRequestProto r = oldRequest.getRpcRequest();
-        RaftClientRequestProto newRequest = RaftClientRequestProto.newBuilder()
-            .setMessage(oldRequest.getMessage())
-            .setReadOnly(oldRequest.getReadOnly())
-            .setRpcRequest(toRaftRpcRequestProtoBuilder(
-                clientId, newLeader, r.getSeqNum()))
-            .build();
-        dataQueue.offerFirst(newRequest);
-      }
-    }
-  }
-
-  private void refreshPeers(RaftPeer[] newPeers) {
-    if (newPeers != null && newPeers.length > 0) {
-      // we only add new peers, we do not remove any peer even if it no longer
-      // belongs to the current raft conf
-      Arrays.stream(newPeers).forEach(peer -> {
-        peers.putIfAbsent(peer.getId(), peer);
-        proxyMap.putIfAbsent(peer);
-      });
-
-      LOG.debug("refreshed peers: {}", peers);
-    }
-  }
-}

http://git-wip-us.apache.org/repos/asf/incubator-ratis/blob/7e71a2e0/raft-grpc/src/main/java/org/apache/raft/grpc/client/RaftClientProtocolClient.java
----------------------------------------------------------------------
diff --git a/raft-grpc/src/main/java/org/apache/raft/grpc/client/RaftClientProtocolClient.java b/raft-grpc/src/main/java/org/apache/raft/grpc/client/RaftClientProtocolClient.java
deleted file mode 100644
index a8372a3..0000000
--- a/raft-grpc/src/main/java/org/apache/raft/grpc/client/RaftClientProtocolClient.java
+++ /dev/null
@@ -1,73 +0,0 @@
-/**
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- *     http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-package org.apache.raft.grpc.client;
-
-import org.apache.raft.grpc.RaftGrpcUtil;
-import org.apache.raft.protocol.RaftPeer;
-import org.apache.raft.shaded.io.grpc.ManagedChannel;
-import org.apache.raft.shaded.io.grpc.ManagedChannelBuilder;
-import org.apache.raft.shaded.io.grpc.StatusRuntimeException;
-import org.apache.raft.shaded.io.grpc.stub.StreamObserver;
-import org.apache.raft.shaded.proto.RaftProtos.RaftClientReplyProto;
-import org.apache.raft.shaded.proto.RaftProtos.RaftClientRequestProto;
-import org.apache.raft.shaded.proto.RaftProtos.SetConfigurationRequestProto;
-import org.apache.raft.shaded.proto.grpc.RaftClientProtocolServiceGrpc;
-import org.apache.raft.shaded.proto.grpc.RaftClientProtocolServiceGrpc.RaftClientProtocolServiceBlockingStub;
-import org.apache.raft.shaded.proto.grpc.RaftClientProtocolServiceGrpc.RaftClientProtocolServiceStub;
-
-import java.io.Closeable;
-import java.io.IOException;
-
-public class RaftClientProtocolClient implements Closeable {
-  private final RaftPeer target;
-  private final ManagedChannel channel;
-  private final RaftClientProtocolServiceBlockingStub blockingStub;
-  private final RaftClientProtocolServiceStub asyncStub;
-
-  public RaftClientProtocolClient(RaftPeer target) {
-    this.target = target;
-    channel = ManagedChannelBuilder.forTarget(target.getAddress())
-        .usePlaintext(true).build();
-    blockingStub = RaftClientProtocolServiceGrpc.newBlockingStub(channel);
-    asyncStub = RaftClientProtocolServiceGrpc.newStub(channel);
-  }
-
-  @Override
-  public void close() {
-    channel.shutdownNow();
-  }
-
-  public RaftClientReplyProto setConfiguration(
-      SetConfigurationRequestProto request) throws IOException {
-    try {
-      return blockingStub.setConfiguration(request);
-    } catch (StatusRuntimeException e) {
-      // unwrap StatusRuntimeException
-      throw RaftGrpcUtil.unwrapException(e);
-    }
-  }
-
-  StreamObserver<RaftClientRequestProto> append(
-      StreamObserver<RaftClientReplyProto> responseHandler) {
-    return asyncStub.append(responseHandler);
-  }
-
-  public RaftPeer getTarget() {
-    return target;
-  }
-}

http://git-wip-us.apache.org/repos/asf/incubator-ratis/blob/7e71a2e0/raft-grpc/src/main/java/org/apache/raft/grpc/client/RaftClientProtocolProxy.java
----------------------------------------------------------------------
diff --git a/raft-grpc/src/main/java/org/apache/raft/grpc/client/RaftClientProtocolProxy.java b/raft-grpc/src/main/java/org/apache/raft/grpc/client/RaftClientProtocolProxy.java
deleted file mode 100644
index 01ec023..0000000
--- a/raft-grpc/src/main/java/org/apache/raft/grpc/client/RaftClientProtocolProxy.java
+++ /dev/null
@@ -1,104 +0,0 @@
-/**
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- *     http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-package org.apache.raft.grpc.client;
-
-import org.apache.raft.protocol.RaftPeer;
-import org.apache.raft.shaded.io.grpc.stub.StreamObserver;
-import org.apache.raft.shaded.proto.RaftProtos.RaftClientReplyProto;
-import org.apache.raft.shaded.proto.RaftProtos.RaftClientRequestProto;
-
-import java.io.Closeable;
-import java.io.IOException;
-import java.util.function.Function;
-
-public class RaftClientProtocolProxy implements Closeable {
-  private final RaftClientProtocolClient proxy;
-  private final Function<RaftPeer, CloseableStreamObserver> responseHandlerCreation;
-  private RpcSession currentSession;
-
-  public RaftClientProtocolProxy(RaftPeer target,
-      Function<RaftPeer, CloseableStreamObserver> responseHandlerCreation) {
-    proxy = new RaftClientProtocolClient(target);
-    this.responseHandlerCreation = responseHandlerCreation;
-  }
-
-  @Override
-  public void close() throws IOException {
-    closeCurrentSession();
-    proxy.close();
-  }
-
-  @Override
-  public String toString() {
-    return "ProxyTo:" + proxy.getTarget();
-  }
-
-  public void closeCurrentSession() {
-    if (currentSession != null) {
-      currentSession.close();
-      currentSession = null;
-    }
-  }
-
-  public void onNext(RaftClientRequestProto request) {
-    if (currentSession == null) {
-      currentSession = new RpcSession(
-          responseHandlerCreation.apply(proxy.getTarget()));
-    }
-    currentSession.requestObserver.onNext(request);
-  }
-
-  public void onError() {
-    if (currentSession != null) {
-      currentSession.onError();
-    }
-  }
-
-  public interface CloseableStreamObserver
-      extends StreamObserver<RaftClientReplyProto>, Closeable {
-  }
-
-  class RpcSession implements Closeable {
-    private final StreamObserver<RaftClientRequestProto> requestObserver;
-    private final CloseableStreamObserver responseHandler;
-    private boolean hasError = false;
-
-    RpcSession(CloseableStreamObserver responseHandler) {
-      this.responseHandler = responseHandler;
-      this.requestObserver = proxy.append(responseHandler);
-    }
-
-    void onError() {
-      hasError = true;
-    }
-
-    @Override
-    public void close() {
-      if (!hasError) {
-        try {
-          requestObserver.onCompleted();
-        } catch (Exception ignored) {
-        }
-      }
-      try {
-        responseHandler.close();
-      } catch (IOException ignored) {
-      }
-    }
-  }
-}

http://git-wip-us.apache.org/repos/asf/incubator-ratis/blob/7e71a2e0/raft-grpc/src/main/java/org/apache/raft/grpc/client/RaftClientProtocolService.java
----------------------------------------------------------------------
diff --git a/raft-grpc/src/main/java/org/apache/raft/grpc/client/RaftClientProtocolService.java b/raft-grpc/src/main/java/org/apache/raft/grpc/client/RaftClientProtocolService.java
deleted file mode 100644
index 8f41bdc..0000000
--- a/raft-grpc/src/main/java/org/apache/raft/grpc/client/RaftClientProtocolService.java
+++ /dev/null
@@ -1,194 +0,0 @@
-/**
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- *     http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-package org.apache.raft.grpc.client;
-
-import com.google.common.base.Preconditions;
-import org.apache.raft.client.impl.ClientProtoUtils;
-import org.apache.raft.grpc.RaftGrpcUtil;
-import org.apache.raft.protocol.RaftClientAsynchronousProtocol;
-import org.apache.raft.protocol.RaftClientReply;
-import org.apache.raft.shaded.io.grpc.stub.StreamObserver;
-import org.apache.raft.shaded.proto.RaftProtos.RaftClientReplyProto;
-import org.apache.raft.shaded.proto.RaftProtos.RaftClientRequestProto;
-import org.apache.raft.shaded.proto.RaftProtos.SetConfigurationRequestProto;
-import org.apache.raft.shaded.proto.grpc.RaftClientProtocolServiceGrpc.RaftClientProtocolServiceImplBase;
-import org.slf4j.Logger;
-import org.slf4j.LoggerFactory;
-
-import java.util.*;
-import java.util.concurrent.CompletableFuture;
-
-public class RaftClientProtocolService extends RaftClientProtocolServiceImplBase {
-  static final Logger LOG = LoggerFactory.getLogger(RaftClientProtocolService.class);
-
-  private static class PendingAppend implements Comparable<PendingAppend> {
-    private final long seqNum;
-    private volatile RaftClientReply reply;
-
-    PendingAppend(long seqNum) {
-      this.seqNum = seqNum;
-    }
-
-    boolean isReady() {
-      return reply != null || this == COMPLETED;
-    }
-
-    void setReply(RaftClientReply reply) {
-      this.reply = reply;
-    }
-
-    @Override
-    public int compareTo(PendingAppend p) {
-      return seqNum == p.seqNum ? 0 : (seqNum < p.seqNum ? -1 : 1);
-    }
-
-    @Override
-    public String toString() {
-      return seqNum + ", reply:" + (reply == null ? "null" : reply.toString());
-    }
-  }
-  private static final PendingAppend COMPLETED = new PendingAppend(Long.MAX_VALUE);
-
-  private final String id;
-  private final RaftClientAsynchronousProtocol client;
-
-  public RaftClientProtocolService(String id, RaftClientAsynchronousProtocol client) {
-    this.id = id;
-    this.client = client;
-  }
-
-  @Override
-  public void setConfiguration(SetConfigurationRequestProto request,
-      StreamObserver<RaftClientReplyProto> responseObserver) {
-    try {
-      CompletableFuture<RaftClientReply> future = client.setConfigurationAsync(
-          ClientProtoUtils.toSetConfigurationRequest(request));
-      future.whenCompleteAsync((reply, exception) -> {
-        if (exception != null) {
-          responseObserver.onError(RaftGrpcUtil.wrapException(exception));
-        } else {
-          responseObserver.onNext(ClientProtoUtils.toRaftClientReplyProto(reply));
-          responseObserver.onCompleted();
-        }
-      });
-    } catch (Exception e) {
-      responseObserver.onError(RaftGrpcUtil.wrapException(e));
-    }
-  }
-
-  @Override
-  public StreamObserver<RaftClientRequestProto> append(
-      StreamObserver<RaftClientReplyProto> responseObserver) {
-    return new AppendRequestStreamObserver(responseObserver);
-  }
-
-  private class AppendRequestStreamObserver implements
-      StreamObserver<RaftClientRequestProto> {
-    private final List<PendingAppend> pendingList = new LinkedList<>();
-    private final StreamObserver<RaftClientReplyProto> responseObserver;
-
-    AppendRequestStreamObserver(StreamObserver<RaftClientReplyProto> ro) {
-      this.responseObserver = ro;
-    }
-
-    @Override
-    public void onNext(RaftClientRequestProto request) {
-      try {
-        PendingAppend p = new PendingAppend(request.getRpcRequest().getSeqNum());
-        synchronized (pendingList) {
-          pendingList.add(p);
-        }
-
-        CompletableFuture<RaftClientReply> future = client.submitClientRequestAsync(
-            ClientProtoUtils.toRaftClientRequest(request));
-        future.whenCompleteAsync((reply, exception) -> {
-          if (exception != null) {
-            // TODO: the exception may be from either raft or state machine.
-            // Currently we skip all the following responses when getting an
-            // exception from the state machine.
-            responseObserver.onError(RaftGrpcUtil.wrapException(exception));
-          } else {
-            final long replySeq = reply.getSeqNum();
-            synchronized (pendingList) {
-              Preconditions.checkState(!pendingList.isEmpty(),
-                  "PendingList is empty when handling onNext for seqNum %s",
-                  replySeq);
-              final long headSeqNum = pendingList.get(0).seqNum;
-              // we assume the seqNum is consecutive for a stream RPC call
-              final PendingAppend pendingForReply = pendingList.get(
-                  (int) (replySeq - headSeqNum));
-              Preconditions.checkState(pendingForReply != null &&
-                      pendingForReply.seqNum == replySeq,
-                  "pending for reply is: %s, the pending list: %s",
-                  pendingForReply, pendingList);
-              pendingForReply.setReply(reply);
-
-              if (headSeqNum == replySeq) {
-                Collection<PendingAppend> readySet = new ArrayList<>();
-                // if this is head, we send back all the ready responses
-                Iterator<PendingAppend> iter = pendingList.iterator();
-                PendingAppend pending;
-                while (iter.hasNext() && ((pending = iter.next()).isReady())) {
-                  readySet.add(pending);
-                  iter.remove();
-                }
-                sendReadyReplies(readySet);
-              }
-            }
-          }
-        });
-      } catch (Throwable e) {
-        LOG.info("{} got exception when handling client append request {}: {}",
-            id, request.getRpcRequest(), e);
-        responseObserver.onError(RaftGrpcUtil.wrapException(e));
-      }
-    }
-
-    private void sendReadyReplies(Collection<PendingAppend> readySet) {
-      readySet.forEach(ready -> {
-        Preconditions.checkState(ready.isReady());
-        if (ready == COMPLETED) {
-          responseObserver.onCompleted();
-        } else {
-          responseObserver.onNext(
-              ClientProtoUtils.toRaftClientReplyProto(ready.reply));
-        }
-      });
-    }
-
-    @Override
-    public void onError(Throwable t) {
-      // for now we just log a msg
-      LOG.warn("{} onError: client Append cancelled", id, t);
-      synchronized (pendingList) {
-        pendingList.clear();
-      }
-    }
-
-    @Override
-    public void onCompleted() {
-      synchronized (pendingList) {
-        if (pendingList.isEmpty()) {
-          responseObserver.onCompleted();
-        } else {
-          pendingList.add(COMPLETED);
-        }
-      }
-    }
-  }
-}

http://git-wip-us.apache.org/repos/asf/incubator-ratis/blob/7e71a2e0/raft-grpc/src/main/java/org/apache/raft/grpc/client/RaftClientSenderWithGrpc.java
----------------------------------------------------------------------
diff --git a/raft-grpc/src/main/java/org/apache/raft/grpc/client/RaftClientSenderWithGrpc.java b/raft-grpc/src/main/java/org/apache/raft/grpc/client/RaftClientSenderWithGrpc.java
deleted file mode 100644
index 7351e1a..0000000
--- a/raft-grpc/src/main/java/org/apache/raft/grpc/client/RaftClientSenderWithGrpc.java
+++ /dev/null
@@ -1,119 +0,0 @@
-/**
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- *     http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-package org.apache.raft.grpc.client;
-
-import org.apache.raft.client.RaftClientRequestSender;
-import org.apache.raft.grpc.RaftGrpcUtil;
-import org.apache.raft.protocol.RaftClientReply;
-import org.apache.raft.protocol.RaftClientRequest;
-import org.apache.raft.protocol.RaftPeer;
-import org.apache.raft.protocol.SetConfigurationRequest;
-import org.apache.raft.shaded.io.grpc.StatusRuntimeException;
-import org.apache.raft.shaded.io.grpc.stub.StreamObserver;
-import org.apache.raft.shaded.proto.RaftProtos.RaftClientReplyProto;
-import org.apache.raft.shaded.proto.RaftProtos.RaftClientRequestProto;
-import org.apache.raft.shaded.proto.RaftProtos.SetConfigurationRequestProto;
-import org.apache.raft.util.PeerProxyMap;
-import org.apache.raft.util.RaftUtils;
-import org.slf4j.Logger;
-import org.slf4j.LoggerFactory;
-
-import java.io.IOException;
-import java.io.InterruptedIOException;
-import java.util.Collection;
-import java.util.concurrent.CompletableFuture;
-import java.util.concurrent.ExecutionException;
-
-import static org.apache.raft.client.impl.ClientProtoUtils.*;
-
-public class RaftClientSenderWithGrpc implements RaftClientRequestSender {
-  public static final Logger LOG = LoggerFactory.getLogger(RaftClientSenderWithGrpc.class);
-
-  private final PeerProxyMap<RaftClientProtocolClient> proxies
-      = new PeerProxyMap<>(RaftClientProtocolClient::new);
-
-  public RaftClientSenderWithGrpc(Collection<RaftPeer> peers) {
-    addServers(peers);
-  }
-
-  @Override
-  public RaftClientReply sendRequest(RaftClientRequest request)
-      throws IOException {
-    final String serverId = request.getReplierId();
-    final RaftClientProtocolClient proxy = proxies.getProxy(serverId);
-    if (request instanceof SetConfigurationRequest) {
-      SetConfigurationRequestProto setConf =
-          toSetConfigurationRequestProto((SetConfigurationRequest) request);
-      return toRaftClientReply(proxy.setConfiguration(setConf));
-    } else {
-      RaftClientRequestProto requestProto = toRaftClientRequestProto(request);
-      CompletableFuture<RaftClientReplyProto> replyFuture =
-          new CompletableFuture<>();
-      final StreamObserver<RaftClientRequestProto> requestObserver =
-          proxy.append(new StreamObserver<RaftClientReplyProto>() {
-            @Override
-            public void onNext(RaftClientReplyProto value) {
-              replyFuture.complete(value);
-            }
-
-            @Override
-            public void onError(Throwable t) {
-              // This implementation is used as RaftClientRequestSender. Retry
-              // logic on Exception is in RaftClient.
-              final IOException e;
-              if (t instanceof StatusRuntimeException) {
-                e = RaftGrpcUtil.unwrapException((StatusRuntimeException) t);
-              } else {
-                e = RaftUtils.asIOException(t);
-              }
-              replyFuture.completeExceptionally(e);
-            }
-
-            @Override
-            public void onCompleted() {
-              if (!replyFuture.isDone()) {
-                replyFuture.completeExceptionally(
-                    new IOException("No reply for request " + request));
-              }
-            }
-          });
-      requestObserver.onNext(requestProto);
-      requestObserver.onCompleted();
-
-      // TODO: timeout support
-      try {
-        return toRaftClientReply(replyFuture.get());
-      } catch (InterruptedException e) {
-        throw new InterruptedIOException(
-            "Interrupted while waiting for response of request " + request);
-      } catch (ExecutionException e) {
-        throw RaftUtils.toIOException(e);
-      }
-    }
-  }
-
-  @Override
-  public void addServers(Iterable<RaftPeer> servers) {
-    proxies.addPeers(servers);
-  }
-
-  @Override
-  public void close() throws IOException {
-    proxies.close();
-  }
-}

http://git-wip-us.apache.org/repos/asf/incubator-ratis/blob/7e71a2e0/raft-grpc/src/main/java/org/apache/raft/grpc/client/RaftOutputStream.java
----------------------------------------------------------------------
diff --git a/raft-grpc/src/main/java/org/apache/raft/grpc/client/RaftOutputStream.java b/raft-grpc/src/main/java/org/apache/raft/grpc/client/RaftOutputStream.java
deleted file mode 100644
index 7edcab9..0000000
--- a/raft-grpc/src/main/java/org/apache/raft/grpc/client/RaftOutputStream.java
+++ /dev/null
@@ -1,112 +0,0 @@
-/**
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- *     http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-package org.apache.raft.grpc.client;
-
-import org.apache.raft.conf.RaftProperties;
-import org.apache.raft.protocol.RaftPeer;
-import org.apache.raft.util.ProtoUtils;
-
-import java.io.IOException;
-import java.io.OutputStream;
-import java.util.Collection;
-
-import static org.apache.raft.grpc.RaftGrpcConfigKeys.RAFT_OUTPUTSTREAM_BUFFER_SIZE_DEFAULT;
-import static org.apache.raft.grpc.RaftGrpcConfigKeys.RAFT_OUTPUTSTREAM_BUFFER_SIZE_KEY;
-
-public class RaftOutputStream extends OutputStream {
-  /** internal buffer */
-  private final byte buf[];
-  private int count;
-  private long seqNum = 0;
-  private final String clientId;
-  private final AppendStreamer streamer;
-
-  private boolean closed = false;
-
-  public RaftOutputStream(RaftProperties prop, String clientId,
-      Collection<RaftPeer> peers, String leaderId) {
-    final int bufferSize = prop.getInt(RAFT_OUTPUTSTREAM_BUFFER_SIZE_KEY,
-        RAFT_OUTPUTSTREAM_BUFFER_SIZE_DEFAULT);
-    buf = new byte[bufferSize];
-    count = 0;
-    this.clientId = clientId;
-    streamer = new AppendStreamer(prop, peers, leaderId, clientId);
-  }
-
-  @Override
-  public void write(int b) throws IOException {
-    checkClosed();
-    buf[count++] = (byte)b;
-    flushIfNecessary();
-  }
-
-  private void flushIfNecessary() throws IOException {
-    if(count == buf.length) {
-      flushToStreamer();
-    }
-  }
-
-  @Override
-  public void write(byte b[], int off, int len) throws IOException {
-    checkClosed();
-    if (off < 0 || len < 0 || off > b.length - len) {
-      throw new ArrayIndexOutOfBoundsException();
-    }
-
-    int total = 0;
-    while (total < len) {
-      int toWrite = Math.min(len - total, buf.length - count);
-      System.arraycopy(b, off + total, buf, count, toWrite);
-      count += toWrite;
-      total += toWrite;
-      flushIfNecessary();
-    }
-  }
-
-  private void flushToStreamer() throws IOException {
-    if (count > 0) {
-      streamer.write(ProtoUtils.toByteString(buf, 0, count), seqNum++);
-      count = 0;
-    }
-  }
-
-  @Override
-  public void flush() throws IOException {
-    checkClosed();
-    flushToStreamer();
-    streamer.flush();
-  }
-
-  @Override
-  public void close() throws IOException {
-    flushToStreamer();
-    streamer.close(); // streamer will flush
-    this.closed = true;
-  }
-
-  @Override
-  public String toString() {
-    return "RaftOutputStream-" + clientId;
-  }
-
-  private void checkClosed() throws IOException {
-    if (closed) {
-      throw new IOException(this.toString() + " was closed.");
-    }
-  }
-}


[25/54] [abbrv] incubator-ratis git commit: Renamed the packages from raft to ratis in preperation for Apache Incubation - Moved all java packages from org.apache.raft to org.apache.ratis. - Moved native package to org_apache_ratis, and native lib to l

Posted by ji...@apache.org.
http://git-wip-us.apache.org/repos/asf/incubator-ratis/blob/7e71a2e0/ratis-examples/src/test/java/org/apache/ratis/TestRestartRaftPeer.java
----------------------------------------------------------------------
diff --git a/ratis-examples/src/test/java/org/apache/ratis/TestRestartRaftPeer.java b/ratis-examples/src/test/java/org/apache/ratis/TestRestartRaftPeer.java
new file mode 100644
index 0000000..37b4064
--- /dev/null
+++ b/ratis-examples/src/test/java/org/apache/ratis/TestRestartRaftPeer.java
@@ -0,0 +1,116 @@
+/**
+ * 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.ratis;
+
+import org.apache.log4j.Level;
+import org.apache.ratis.MiniRaftCluster;
+import org.apache.ratis.RaftTestUtil;
+import org.apache.ratis.RaftTestUtil.SimpleMessage;
+import org.apache.ratis.client.RaftClient;
+import org.apache.ratis.conf.RaftProperties;
+import org.apache.ratis.examples.RaftExamplesTestUtil;
+import org.apache.ratis.server.RaftServerConfigKeys;
+import org.apache.ratis.server.impl.RaftServerImpl;
+import org.apache.ratis.server.simulation.RequestHandler;
+import org.apache.ratis.server.storage.RaftLog;
+import org.apache.ratis.statemachine.SimpleStateMachine4Testing;
+import org.apache.ratis.statemachine.StateMachine;
+import org.apache.ratis.util.RaftUtils;
+import org.junit.Assert;
+import org.junit.Rule;
+import org.junit.Test;
+import org.junit.rules.Timeout;
+import org.junit.runner.RunWith;
+import org.junit.runners.Parameterized;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.io.IOException;
+import java.util.Arrays;
+import java.util.Collection;
+
+/**
+ * Test restarting raft peers.
+ */
+@RunWith(Parameterized.class)
+public class TestRestartRaftPeer {
+  static Logger LOG = LoggerFactory.getLogger(TestRestartRaftPeer.class);
+  static {
+    RaftUtils.setLogLevel(RaftServerImpl.LOG, Level.DEBUG);
+    RaftUtils.setLogLevel(RaftLog.LOG, Level.DEBUG);
+    RaftUtils.setLogLevel(RequestHandler.LOG, Level.DEBUG);
+    RaftUtils.setLogLevel(RaftClient.LOG, Level.DEBUG);
+  }
+
+  @Parameterized.Parameters
+  public static Collection<Object[]> data() throws IOException {
+    RaftProperties prop = new RaftProperties();
+    prop.setClass(MiniRaftCluster.STATEMACHINE_CLASS_KEY,
+        SimpleStateMachine4Testing.class, StateMachine.class);
+    prop.setInt(RaftServerConfigKeys.RAFT_LOG_SEGMENT_MAX_SIZE_KEY, 1024 * 8);
+    return RaftExamplesTestUtil.getMiniRaftClusters(prop, 3);
+  }
+
+  @Parameterized.Parameter
+  public MiniRaftCluster cluster;
+
+  @Rule
+  public Timeout globalTimeout = new Timeout(60 * 1000);
+
+  @Test
+  public void restartFollower() throws Exception {
+    cluster.start();
+    RaftTestUtil.waitForLeader(cluster);
+    final String leaderId = cluster.getLeader().getId();
+    final RaftClient client = cluster.createClient("client", leaderId);
+
+    // write some messages
+    final byte[] content = new byte[1024];
+    Arrays.fill(content, (byte) 1);
+    final SimpleMessage message = new SimpleMessage(new String(content));
+    for (int i = 0; i < 10; i++) {
+      Assert.assertTrue(client.send(message).isSuccess());
+    }
+
+    // restart a follower
+    String followerId = cluster.getFollowers().get(0).getId();
+    LOG.info("Restart follower {}", followerId);
+    cluster.restartServer(followerId, false);
+
+    // write some more messages
+    for (int i = 0; i < 10; i++) {
+      Assert.assertTrue(client.send(message).isSuccess());
+    }
+    client.close();
+
+    // make sure the restarted follower can catchup
+    boolean catchup = false;
+    long lastAppliedIndex = 0;
+    for (int i = 0; i < 10 && !catchup; i++) {
+      Thread.sleep(500);
+      lastAppliedIndex = cluster.getServer(followerId).getState().getLastAppliedIndex();
+      catchup = lastAppliedIndex >= 20;
+    }
+    Assert.assertTrue("lastAppliedIndex=" + lastAppliedIndex, catchup);
+
+    // make sure the restarted peer's log segments is correct
+    cluster.restartServer(followerId, false);
+    Assert.assertTrue(cluster.getServer(followerId).getState().getLog()
+        .getLastEntry().getIndex() >= 20);
+  }
+}

http://git-wip-us.apache.org/repos/asf/incubator-ratis/blob/7e71a2e0/ratis-examples/src/test/java/org/apache/ratis/examples/RaftExamplesTestUtil.java
----------------------------------------------------------------------
diff --git a/ratis-examples/src/test/java/org/apache/ratis/examples/RaftExamplesTestUtil.java b/ratis-examples/src/test/java/org/apache/ratis/examples/RaftExamplesTestUtil.java
new file mode 100644
index 0000000..ff936bd
--- /dev/null
+++ b/ratis-examples/src/test/java/org/apache/ratis/examples/RaftExamplesTestUtil.java
@@ -0,0 +1,81 @@
+/**
+ * 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.ratis.examples;
+
+import org.apache.ratis.MiniRaftCluster;
+import org.apache.ratis.conf.RaftProperties;
+import org.apache.ratis.grpc.MiniRaftClusterWithGRpc;
+import org.apache.ratis.hadooprpc.MiniRaftClusterWithHadoopRpc;
+import org.apache.ratis.netty.MiniRaftClusterWithNetty;
+import org.apache.ratis.server.simulation.MiniRaftClusterWithSimulatedRpc;
+import org.apache.ratis.statemachine.StateMachine;
+
+import java.io.IOException;
+import java.util.*;
+
+public class RaftExamplesTestUtil {
+  private static void add(
+      Collection<Object[]> clusters, MiniRaftCluster.Factory factory,
+      String[] ids, RaftProperties properties)
+      throws IOException {
+    clusters.add(new Object[]{factory.newCluster(ids, properties, true)});
+  }
+
+  public static Collection<Object[]> getMiniRaftClusters(
+      RaftProperties prop, int clusterSize, Class<?>... clusterClasses)
+      throws IOException {
+    final List<Class<?>> classes = Arrays.asList(clusterClasses);
+    final boolean isAll = classes.isEmpty(); //empty means all
+
+    final Iterator<String[]> ids = new Iterator<String[]>() {
+      private int i = 0;
+      @Override
+      public boolean hasNext() {
+        return true;
+      }
+      @Override
+      public String[] next() {
+        return MiniRaftCluster.generateIds(clusterSize, i++*clusterSize);
+      }
+    };
+
+    final List<Object[]> clusters = new ArrayList<>();
+
+    if (isAll || classes.contains(MiniRaftClusterWithSimulatedRpc.class)) {
+      add(clusters, MiniRaftClusterWithSimulatedRpc.FACTORY, ids.next(), prop);
+    }
+    if (isAll || classes.contains(MiniRaftClusterWithHadoopRpc.class)) {
+      add(clusters, MiniRaftClusterWithHadoopRpc.FACTORY, ids.next(), prop);
+    }
+    if (isAll || classes.contains(MiniRaftClusterWithNetty.class)) {
+      add(clusters, MiniRaftClusterWithNetty.FACTORY, ids.next(), prop);
+    }
+    if (isAll || classes.contains(MiniRaftClusterWithGRpc.class)) {
+      add(clusters, MiniRaftClusterWithGRpc.FACTORY, ids.next(), prop);
+    }
+    return clusters;
+  }
+
+  public static <S extends StateMachine> Collection<Object[]> getMiniRaftClusters(
+      Class<S> stateMachineClass, Class<?>... clusterClasses) throws IOException {
+    final RaftProperties prop = new RaftProperties();
+    prop.setClass(MiniRaftCluster.STATEMACHINE_CLASS_KEY,
+        stateMachineClass, StateMachine.class);
+    return getMiniRaftClusters(prop, 3, clusterClasses);
+  }
+}

http://git-wip-us.apache.org/repos/asf/incubator-ratis/blob/7e71a2e0/ratis-examples/src/test/java/org/apache/ratis/examples/arithmetic/TestArithmetic.java
----------------------------------------------------------------------
diff --git a/ratis-examples/src/test/java/org/apache/ratis/examples/arithmetic/TestArithmetic.java b/ratis-examples/src/test/java/org/apache/ratis/examples/arithmetic/TestArithmetic.java
new file mode 100644
index 0000000..44cf894
--- /dev/null
+++ b/ratis-examples/src/test/java/org/apache/ratis/examples/arithmetic/TestArithmetic.java
@@ -0,0 +1,106 @@
+/**
+ * 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.ratis.examples.arithmetic;
+
+
+import org.apache.log4j.Level;
+import org.apache.ratis.MiniRaftCluster;
+import org.apache.ratis.RaftTestUtil;
+import org.apache.ratis.client.RaftClient;
+import org.apache.ratis.examples.RaftExamplesTestUtil;
+import org.apache.ratis.examples.arithmetic.ArithmeticStateMachine;
+import org.apache.ratis.examples.arithmetic.AssignmentMessage;
+import org.apache.ratis.examples.arithmetic.expression.*;
+import org.apache.ratis.protocol.RaftClientReply;
+import org.apache.ratis.util.RaftUtils;
+import org.junit.Assert;
+import org.junit.Test;
+import org.junit.runner.RunWith;
+import org.junit.runners.Parameterized;
+
+import java.io.IOException;
+import java.util.Collection;
+
+@RunWith(Parameterized.class)
+public class TestArithmetic {
+  static {
+    RaftUtils.setLogLevel(ArithmeticStateMachine.LOG, Level.ALL);
+  }
+
+  @Parameterized.Parameters
+  public static Collection<Object[]> data() throws IOException {
+    return RaftExamplesTestUtil.getMiniRaftClusters(ArithmeticStateMachine.class);
+  }
+
+  @Parameterized.Parameter
+  public MiniRaftCluster cluster;
+
+  @Test
+  public void testPythagorean() throws Exception {
+    cluster.start();
+    RaftTestUtil.waitForLeader(cluster);
+    final String leaderId = cluster.getLeader().getId();
+    final RaftClient client = cluster.createClient("pythagorean", leaderId);
+
+    final Variable a = new Variable("a");
+    final Variable b = new Variable("b");
+    final Variable c = new Variable("c");
+    final BinaryExpression a2 = new BinaryExpression(BinaryExpression.Op.MULT, a, a);
+    final BinaryExpression b2 = new BinaryExpression(BinaryExpression.Op.MULT, b, b);
+    final BinaryExpression c2 = new BinaryExpression(BinaryExpression.Op.ADD, a2, b2);
+    final AssignmentMessage pythagorean = new AssignmentMessage(c,
+        new UnaryExpression(UnaryExpression.Op.SQRT, c2));
+
+    final AssignmentMessage nullA = new AssignmentMessage(a, NullValue.getInstance());
+    final AssignmentMessage nullB = new AssignmentMessage(b, NullValue.getInstance());
+    final AssignmentMessage nullC = new AssignmentMessage(c, NullValue.getInstance());
+
+    for(int n = 3; n < 100; n += 2) {
+      int n2 = n*n;
+      int half_n2 = n2/2;
+
+      RaftClientReply r;
+      r = client.send(new AssignmentMessage(a, new DoubleValue(n)));
+      assertRaftClientReply(r, (double)n);
+      r = client.sendReadOnly(Expression.Utils.toMessage(a2));
+      assertRaftClientReply(r, (double)n2);
+      r = client.send(new AssignmentMessage(b, new DoubleValue(half_n2)));
+      assertRaftClientReply(r, (double)half_n2);
+      r = client.sendReadOnly(Expression.Utils.toMessage(b2));
+      assertRaftClientReply(r, (double)half_n2*half_n2);
+      r = client.send(pythagorean);
+      assertRaftClientReply(r, (double)half_n2 + 1);
+
+      r = client.send(nullA);
+      assertRaftClientReply(r, null);
+      r = client.send(nullB);
+      assertRaftClientReply(r, null);
+      r = client.send(nullC);
+      assertRaftClientReply(r, null);
+    }
+    client.close();
+    cluster.shutdown();
+  }
+
+  static void assertRaftClientReply(RaftClientReply reply, Double expected) {
+    Assert.assertTrue(reply.isSuccess());
+    final Expression e = Expression.Utils.bytes2Expression(
+        reply.getMessage().getContent().toByteArray(), 0);
+    Assert.assertEquals(expected, e.evaluate(null));
+  }
+}

http://git-wip-us.apache.org/repos/asf/incubator-ratis/blob/7e71a2e0/ratis-examples/src/test/java/org/apache/ratis/examples/arithmetic/expression/TestExpression.java
----------------------------------------------------------------------
diff --git a/ratis-examples/src/test/java/org/apache/ratis/examples/arithmetic/expression/TestExpression.java b/ratis-examples/src/test/java/org/apache/ratis/examples/arithmetic/expression/TestExpression.java
new file mode 100644
index 0000000..a2d6e29
--- /dev/null
+++ b/ratis-examples/src/test/java/org/apache/ratis/examples/arithmetic/expression/TestExpression.java
@@ -0,0 +1,107 @@
+/**
+ * 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.ratis.examples.arithmetic.expression;
+
+
+import org.apache.ratis.examples.arithmetic.expression.BinaryExpression;
+import org.apache.ratis.examples.arithmetic.expression.DoubleValue;
+import org.apache.ratis.examples.arithmetic.expression.Expression;
+import org.apache.ratis.examples.arithmetic.expression.UnaryExpression;
+import org.apache.ratis.examples.arithmetic.expression.Variable;
+import org.junit.Assert;
+import org.junit.Test;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.util.Random;
+import java.util.concurrent.ThreadLocalRandom;
+
+public class TestExpression {
+  static final Logger LOG = LoggerFactory.getLogger(TestExpression.class);
+
+  @Test
+  public void testArithmeticUtils() throws Exception {
+    final Random ran = ThreadLocalRandom.current();
+    final byte[] buf = new byte[1024];
+    int offset = 0;
+
+    for(int i = 0; i < 10; i++) {
+      {
+        final int n = ran.nextInt();
+        Expression.Utils.int2bytes(n, buf, offset);
+        final int m = Expression.Utils.bytes2int(buf, offset);
+        Assert.assertEquals(n, m);
+        offset += 4;
+      }
+      {
+        final long n = ran.nextLong();
+        Expression.Utils.long2bytes(n, buf, offset);
+        final long m = Expression.Utils.bytes2long(buf, offset);
+        Assert.assertEquals(n, m);
+        offset += 8;
+      }
+      {
+        final double n = ran.nextDouble();
+        Expression.Utils.double2bytes(n, buf, offset);
+        final double m = Expression.Utils.bytes2double(buf, offset);
+        Assert.assertTrue(n == m);
+        offset += 8;
+      }
+    }
+  }
+  @Test
+  public void testOp() throws Exception {
+    for(BinaryExpression.Op op : BinaryExpression.Op.values()) {
+      final byte b = op.byteValue();
+      Assert.assertEquals(op, BinaryExpression.Op.valueOf(b));
+    }
+    for(UnaryExpression.Op op : UnaryExpression.Op.values()) {
+      final byte b = op.byteValue();
+      Assert.assertEquals(op, UnaryExpression.Op.valueOf(b));
+    }
+  }
+
+  @Test
+  public void testExpression() throws Exception {
+    final byte[] buf = new byte[1024];
+    int offset = 0;
+
+    {
+      final Variable a = new Variable("pi");
+      LOG.info("var a: " + a);
+      final int len = a.toBytes(buf, offset);
+      final Variable a2 = new Variable(buf, offset);
+      LOG.info("var a2: " + a2);
+      Assert.assertEquals(a.getName(), a2.getName());
+      Assert.assertEquals(len, a.length());
+      Assert.assertEquals(len, a2.length());
+      offset += len;
+    }
+
+    {
+      final DoubleValue three = new DoubleValue(3);
+      LOG.info("double three: " + three.evaluate(null));
+      final int len = three.toBytes(buf, offset);
+      final DoubleValue three2 = new DoubleValue(buf, offset);
+      LOG.info("double three2: " + three2.evaluate(null));
+      Assert.assertTrue(three.evaluate(null).equals(three2.evaluate(null)));
+      Assert.assertEquals(len, three.length());
+      Assert.assertEquals(len, three2.length());
+    }
+  }
+}

http://git-wip-us.apache.org/repos/asf/incubator-ratis/blob/7e71a2e0/ratis-examples/src/test/java/org/apache/ratis/statemachine/TestRaftStateMachineException.java
----------------------------------------------------------------------
diff --git a/ratis-examples/src/test/java/org/apache/ratis/statemachine/TestRaftStateMachineException.java b/ratis-examples/src/test/java/org/apache/ratis/statemachine/TestRaftStateMachineException.java
new file mode 100644
index 0000000..1f885ea
--- /dev/null
+++ b/ratis-examples/src/test/java/org/apache/ratis/statemachine/TestRaftStateMachineException.java
@@ -0,0 +1,88 @@
+/**
+ * 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.ratis.statemachine;
+
+import org.apache.log4j.Level;
+import org.apache.ratis.MiniRaftCluster;
+import org.apache.ratis.RaftTestUtil;
+import org.apache.ratis.client.RaftClient;
+import org.apache.ratis.examples.RaftExamplesTestUtil;
+import org.apache.ratis.protocol.Message;
+import org.apache.ratis.protocol.StateMachineException;
+import org.apache.ratis.server.impl.RaftServerImpl;
+import org.apache.ratis.server.simulation.RequestHandler;
+import org.apache.ratis.server.storage.RaftLog;
+import org.apache.ratis.statemachine.SimpleStateMachine4Testing;
+import org.apache.ratis.statemachine.TransactionContext;
+import org.apache.ratis.util.RaftUtils;
+import org.junit.Assert;
+import org.junit.Test;
+import org.junit.runner.RunWith;
+import org.junit.runners.Parameterized;
+
+import java.io.IOException;
+import java.util.Collection;
+import java.util.concurrent.CompletableFuture;
+
+import static org.junit.Assert.fail;
+
+@RunWith(Parameterized.class)
+public class TestRaftStateMachineException {
+  static {
+    RaftUtils.setLogLevel(RaftServerImpl.LOG, Level.DEBUG);
+    RaftUtils.setLogLevel(RaftLog.LOG, Level.DEBUG);
+    RaftUtils.setLogLevel(RequestHandler.LOG, Level.DEBUG);
+    RaftUtils.setLogLevel(RaftClient.LOG, Level.DEBUG);
+  }
+
+  protected static class StateMachineWithException extends SimpleStateMachine4Testing {
+    @Override
+    public CompletableFuture<Message> applyTransaction(TransactionContext trx) {
+      CompletableFuture<Message> future = new CompletableFuture<>();
+      future.completeExceptionally(new StateMachineException("Fake Exception"));
+      return future;
+    }
+  }
+
+  @Parameterized.Parameters
+  public static Collection<Object[]> data() throws IOException {
+    return RaftExamplesTestUtil.getMiniRaftClusters(
+        StateMachineWithException.class);
+  }
+
+  @Parameterized.Parameter
+  public MiniRaftCluster cluster;
+
+  @Test
+  public void testHandleStateMachineException() throws Exception {
+    cluster.start();
+    RaftTestUtil.waitForLeader(cluster);
+
+    final String leaderId = cluster.getLeader().getId();
+
+    try(final RaftClient client = cluster.createClient("client", leaderId)) {
+      client.send(new RaftTestUtil.SimpleMessage("m"));
+      fail("Exception expected");
+    } catch (StateMachineException e) {
+      e.printStackTrace();
+      Assert.assertTrue(e.getMessage().contains("Fake Exception"));
+    }
+
+    cluster.shutdown();
+  }
+}

http://git-wip-us.apache.org/repos/asf/incubator-ratis/blob/7e71a2e0/ratis-examples/src/test/resources/log4j.properties
----------------------------------------------------------------------
diff --git a/ratis-examples/src/test/resources/log4j.properties b/ratis-examples/src/test/resources/log4j.properties
new file mode 100644
index 0000000..ced0687
--- /dev/null
+++ b/ratis-examples/src/test/resources/log4j.properties
@@ -0,0 +1,18 @@
+#   Licensed 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.
+# log4j configuration used during build and unit tests
+
+log4j.rootLogger=info,stdout
+log4j.threshold=ALL
+log4j.appender.stdout=org.apache.log4j.ConsoleAppender
+log4j.appender.stdout.layout=org.apache.log4j.PatternLayout
+log4j.appender.stdout.layout.ConversionPattern=%d{ISO8601} %-5p %c{2} (%F:%M(%L)) - %m%n

http://git-wip-us.apache.org/repos/asf/incubator-ratis/blob/7e71a2e0/ratis-grpc/pom.xml
----------------------------------------------------------------------
diff --git a/ratis-grpc/pom.xml b/ratis-grpc/pom.xml
new file mode 100644
index 0000000..6a46be5
--- /dev/null
+++ b/ratis-grpc/pom.xml
@@ -0,0 +1,93 @@
+<?xml version="1.0" encoding="UTF-8"?>
+<!--
+  Licensed 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. See accompanying LICENSE file.
+-->
+<project xmlns="http://maven.apache.org/POM/4.0.0"
+         xmlns:xsi="http://www.w3.org/2001/XMLSchema-instance"
+         xsi:schemaLocation="http://maven.apache.org/POM/4.0.0 http://maven.apache.org/xsd/maven-4.0.0.xsd">
+  <modelVersion>4.0.0</modelVersion>
+  <parent>
+    <artifactId>ratis-project-dist</artifactId>
+    <groupId>org.apache.ratis</groupId>
+    <version>1.0-SNAPSHOT</version>
+    <relativePath>../ratis-project-dist</relativePath>
+  </parent>
+
+  <artifactId>ratis-grpc</artifactId>
+  <name>Ratis gRPC Support</name>
+
+  <dependencies>
+    <dependency>
+      <artifactId>ratis-proto-shaded</artifactId>
+      <groupId>org.apache.ratis</groupId>
+      <scope>provided</scope>
+    </dependency>
+
+    <dependency>
+      <artifactId>ratis-common</artifactId>
+      <groupId>org.apache.ratis</groupId>
+      <scope>provided</scope>
+    </dependency>
+    <dependency>
+      <artifactId>ratis-common</artifactId>
+      <groupId>org.apache.ratis</groupId>
+      <scope>test</scope>
+      <type>test-jar</type>
+    </dependency>
+    
+    <dependency>
+      <artifactId>ratis-client</artifactId>
+      <groupId>org.apache.ratis</groupId>
+      <scope>provided</scope>
+    </dependency>
+    <dependency>
+      <artifactId>ratis-client</artifactId>
+      <groupId>org.apache.ratis</groupId>
+      <scope>test</scope>
+      <type>test-jar</type>
+    </dependency>
+    
+    <dependency>
+      <artifactId>ratis-server</artifactId>
+      <groupId>org.apache.ratis</groupId>
+      <scope>provided</scope>
+    </dependency>
+    <dependency>
+      <artifactId>ratis-server</artifactId>
+      <groupId>org.apache.ratis</groupId>
+      <scope>test</scope>
+      <type>test-jar</type>
+    </dependency>
+    
+    <dependency>
+      <groupId>org.slf4j</groupId>
+      <artifactId>slf4j-api</artifactId>
+    </dependency>
+
+    <dependency>
+      <groupId>com.google.guava</groupId>
+      <artifactId>guava</artifactId>
+    </dependency>
+
+    <dependency>
+      <groupId>junit</groupId>
+      <artifactId>junit</artifactId>
+      <scope>test</scope>
+    </dependency>
+    <dependency>
+      <groupId>org.mockito</groupId>
+      <artifactId>mockito-all</artifactId>
+      <scope>test</scope>
+    </dependency>
+  </dependencies>
+</project>

http://git-wip-us.apache.org/repos/asf/incubator-ratis/blob/7e71a2e0/ratis-grpc/src/main/java/org/apache/ratis/grpc/RaftGRpcService.java
----------------------------------------------------------------------
diff --git a/ratis-grpc/src/main/java/org/apache/ratis/grpc/RaftGRpcService.java b/ratis-grpc/src/main/java/org/apache/ratis/grpc/RaftGRpcService.java
new file mode 100644
index 0000000..b61e70e
--- /dev/null
+++ b/ratis-grpc/src/main/java/org/apache/ratis/grpc/RaftGRpcService.java
@@ -0,0 +1,152 @@
+/**
+ * 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.ratis.grpc;
+
+import com.google.common.base.Preconditions;
+
+import org.apache.ratis.shaded.io.grpc.Server;
+import org.apache.ratis.shaded.io.grpc.ServerBuilder;
+import org.apache.ratis.shaded.io.grpc.netty.NettyServerBuilder;
+import org.apache.ratis.shaded.proto.RaftProtos.*;
+import org.apache.ratis.conf.RaftProperties;
+import org.apache.ratis.grpc.client.RaftClientProtocolService;
+import org.apache.ratis.grpc.server.RaftServerProtocolClient;
+import org.apache.ratis.grpc.server.RaftServerProtocolService;
+import org.apache.ratis.protocol.RaftPeer;
+import org.apache.ratis.server.RaftServer;
+import org.apache.ratis.server.RaftServerRpc;
+import org.apache.ratis.util.CodeInjectionForTesting;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import static org.apache.ratis.grpc.RaftGrpcConfigKeys.RAFT_GRPC_SERVER_PORT_DEFAULT;
+import static org.apache.ratis.grpc.RaftGrpcConfigKeys.RAFT_GRPC_SERVER_PORT_KEY;
+
+import java.io.IOException;
+import java.net.InetSocketAddress;
+import java.util.Collections;
+import java.util.HashMap;
+import java.util.Map;
+
+public class RaftGRpcService implements RaftServerRpc {
+  static final Logger LOG = LoggerFactory.getLogger(RaftGRpcService.class);
+  public static final String GRPC_SEND_SERVER_REQUEST =
+      RaftGRpcService.class.getSimpleName() + ".sendRequest";
+
+  private final Server server;
+  private final InetSocketAddress address;
+  private final Map<String, RaftServerProtocolClient> peers =
+      Collections.synchronizedMap(new HashMap<>());
+  private final String selfId;
+
+  public RaftGRpcService(RaftServer raftServer, RaftProperties properties) {
+    int port = properties.getInt(RAFT_GRPC_SERVER_PORT_KEY,
+        RAFT_GRPC_SERVER_PORT_DEFAULT);
+    int maxMessageSize = properties.getInt(
+        RaftGrpcConfigKeys.RAFT_GRPC_MESSAGE_MAXSIZE_KEY,
+        RaftGrpcConfigKeys.RAFT_GRPC_MESSAGE_MAXSIZE_DEFAULT);
+    ServerBuilder serverBuilder = ServerBuilder.forPort(port);
+    selfId = raftServer.getId();
+    server = ((NettyServerBuilder) serverBuilder).maxMessageSize(maxMessageSize)
+        .addService(new RaftServerProtocolService(selfId, raftServer))
+        .addService(new RaftClientProtocolService(selfId, raftServer))
+        .build();
+
+    // start service to determine the port (in case port is configured as 0)
+    startService();
+    address = new InetSocketAddress(server.getPort());
+    LOG.info("Server started, listening on " + address.getPort());
+  }
+
+  @Override
+  public void start() {
+    // do nothing
+  }
+
+  private void startService() {
+    try {
+      server.start();
+    } catch (IOException e) {
+      LOG.error("Failed to start Grpc server", e);
+      System.exit(1);
+    }
+    Runtime.getRuntime().addShutdownHook(new Thread() {
+      @Override
+      public void run() {
+        System.err.println("*** shutting down gRPC server since JVM is shutting down");
+        RaftGRpcService.this.close();
+        System.err.println("*** server shut down");
+      }
+    });
+  }
+
+  @Override
+  public void close() {
+    if (server != null) {
+      server.shutdown();
+    }
+    shutdownClients();
+  }
+
+  @Override
+  public InetSocketAddress getInetSocketAddress() {
+    return address;
+  }
+
+  @Override
+  public AppendEntriesReplyProto appendEntries(
+      AppendEntriesRequestProto request) throws IOException {
+    throw new UnsupportedOperationException(
+        "Blocking AppendEntries call is not supported");
+  }
+
+  @Override
+  public InstallSnapshotReplyProto installSnapshot(
+      InstallSnapshotRequestProto request) throws IOException {
+    throw new UnsupportedOperationException(
+        "Blocking InstallSnapshot call is not supported");
+  }
+
+  @Override
+  public RequestVoteReplyProto requestVote(RequestVoteRequestProto request)
+      throws IOException {
+    CodeInjectionForTesting.execute(GRPC_SEND_SERVER_REQUEST, selfId,
+        null, request);
+
+    RaftServerProtocolClient target = Preconditions.checkNotNull(
+        peers.get(request.getServerRequest().getReplyId()));
+    return target.requestVote(request);
+  }
+
+  @Override
+  public void addPeers(Iterable<RaftPeer> newPeers) {
+    for (RaftPeer p : newPeers) {
+      if (!peers.containsKey(p.getId())) {
+        peers.put(p.getId(), new RaftServerProtocolClient(p));
+      }
+    }
+  }
+
+  private void shutdownClients() {
+    peers.values().forEach(RaftServerProtocolClient::shutdown);
+  }
+
+  public RaftServerProtocolClient getRpcClient(RaftPeer peer) {
+    return peers.get(peer.getId());
+  }
+}

http://git-wip-us.apache.org/repos/asf/incubator-ratis/blob/7e71a2e0/ratis-grpc/src/main/java/org/apache/ratis/grpc/RaftGrpcConfigKeys.java
----------------------------------------------------------------------
diff --git a/ratis-grpc/src/main/java/org/apache/ratis/grpc/RaftGrpcConfigKeys.java b/ratis-grpc/src/main/java/org/apache/ratis/grpc/RaftGrpcConfigKeys.java
new file mode 100644
index 0000000..ffec8ff
--- /dev/null
+++ b/ratis-grpc/src/main/java/org/apache/ratis/grpc/RaftGrpcConfigKeys.java
@@ -0,0 +1,47 @@
+/**
+ * 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.ratis.grpc;
+
+import org.apache.ratis.client.RaftClientConfigKeys;
+
+public interface RaftGrpcConfigKeys {
+  String PREFIX = "raft.grpc";
+
+  String RAFT_GRPC_SERVER_PORT_KEY = PREFIX + ".server.port";
+  int RAFT_GRPC_SERVER_PORT_DEFAULT = 0;
+
+  String RAFT_GRPC_MESSAGE_MAXSIZE_KEY = PREFIX + ".message.maxsize";
+  int RAFT_GRPC_MESSAGE_MAXSIZE_DEFAULT = 64 * 1024 * 1024; // 64 MB
+
+  String RAFT_GRPC_LEADER_MAX_OUTSTANDING_APPENDS_KEY =
+      PREFIX + "leader.max.outstanding.appends";
+  int RAFT_GRPC_LEADER_MAX_OUTSTANDING_APPENDS_DEFAULT = 128;
+
+  String RAFT_GRPC_CLIENT_MAX_OUTSTANDING_APPENDS_KEY =
+      PREFIX + "client.max.outstanding.appends";
+  int RAFT_GRPC_CLIENT_MAX_OUTSTANDING_APPENDS_DEFAULT = 128;
+
+  String RAFT_OUTPUTSTREAM_BUFFER_SIZE_KEY = "raft.outputstream.buffer.size";
+  int RAFT_OUTPUTSTREAM_BUFFER_SIZE_DEFAULT = 64 * 1024;
+
+  String RAFT_OUTPUTSTREAM_MAX_RETRY_TIMES_KEY = "raft.outputstream.max.retry.times";
+  int RAFT_OUTPUTSTREAM_MAX_RETRY_TIMES_DEFAULT = 5;
+
+  String RAFT_OUTPUTSTREAM_RETRY_INTERVAL_KEY = "raft.outputstream.retry.interval";
+  long RAFT_OUTPUTSTREAM_RETRY_INTERVAL_DEFAULT = RaftClientConfigKeys.RAFT_RPC_TIMEOUT_MS_DEFAULT;
+}

http://git-wip-us.apache.org/repos/asf/incubator-ratis/blob/7e71a2e0/ratis-grpc/src/main/java/org/apache/ratis/grpc/RaftGrpcUtil.java
----------------------------------------------------------------------
diff --git a/ratis-grpc/src/main/java/org/apache/ratis/grpc/RaftGrpcUtil.java b/ratis-grpc/src/main/java/org/apache/ratis/grpc/RaftGrpcUtil.java
new file mode 100644
index 0000000..52ed851
--- /dev/null
+++ b/ratis-grpc/src/main/java/org/apache/ratis/grpc/RaftGrpcUtil.java
@@ -0,0 +1,87 @@
+/**
+ * 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.ratis.grpc;
+
+import org.apache.ratis.shaded.io.grpc.Metadata;
+import org.apache.ratis.shaded.io.grpc.Status;
+import org.apache.ratis.shaded.io.grpc.StatusRuntimeException;
+import org.apache.ratis.util.RaftUtils;
+
+import java.io.IOException;
+import java.io.PrintWriter;
+import java.io.StringWriter;
+import java.lang.reflect.Constructor;
+
+public class RaftGrpcUtil {
+  public static final Metadata.Key<String> EXCEPTION_TYPE_KEY =
+      Metadata.Key.of("exception-type", Metadata.ASCII_STRING_MARSHALLER);
+
+  public static String stringifyException(Throwable e) {
+    StringWriter stm = new StringWriter();
+    PrintWriter wrt = new PrintWriter(stm);
+    e.printStackTrace(wrt);
+    wrt.close();
+    return stm.toString();
+  }
+
+  public static StatusRuntimeException wrapException(Throwable t) {
+    Metadata trailers = new Metadata();
+    trailers.put(EXCEPTION_TYPE_KEY, t.getClass().getCanonicalName());
+    return new StatusRuntimeException(
+        Status.INTERNAL.withDescription(RaftGrpcUtil.stringifyException(t)),
+        trailers);
+  }
+
+  public static IOException unwrapException(StatusRuntimeException se) {
+    final Metadata trailers = se.getTrailers();
+    final Status status = se.getStatus();
+    if (trailers != null && status != null) {
+      final String className = trailers.get(EXCEPTION_TYPE_KEY);
+      if (className != null) {
+        try {
+          Class<?> clazz = Class.forName(className);
+          final Exception unwrapped = instantiateException(
+              clazz.asSubclass(Exception.class), status.getDescription(), se);
+          return RaftUtils.asIOException(unwrapped);
+        } catch (Exception e) {
+          return new IOException(se);
+        }
+      }
+    }
+    return new IOException(se);
+  }
+
+  public static IOException unwrapIOException(Throwable t) {
+    final IOException e;
+    if (t instanceof StatusRuntimeException) {
+      e = RaftGrpcUtil.unwrapException((StatusRuntimeException) t);
+    } else {
+      e = RaftUtils.asIOException(t);
+    }
+    return e;
+  }
+
+  private static Exception instantiateException(Class<? extends Exception> cls,
+      String message, Exception from) throws Exception {
+    Constructor<? extends Exception> cn = cls.getConstructor(String.class);
+    cn.setAccessible(true);
+    Exception ex = cn.newInstance(message);
+    ex.initCause(from);
+    return ex;
+  }
+}

http://git-wip-us.apache.org/repos/asf/incubator-ratis/blob/7e71a2e0/ratis-grpc/src/main/java/org/apache/ratis/grpc/client/AppendStreamer.java
----------------------------------------------------------------------
diff --git a/ratis-grpc/src/main/java/org/apache/ratis/grpc/client/AppendStreamer.java b/ratis-grpc/src/main/java/org/apache/ratis/grpc/client/AppendStreamer.java
new file mode 100644
index 0000000..cb05b33
--- /dev/null
+++ b/ratis-grpc/src/main/java/org/apache/ratis/grpc/client/AppendStreamer.java
@@ -0,0 +1,396 @@
+/**
+ * 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.ratis.grpc.client;
+
+import com.google.common.base.Preconditions;
+
+import org.apache.ratis.shaded.com.google.protobuf.ByteString;
+import org.apache.ratis.shaded.proto.RaftProtos.RaftClientReplyProto;
+import org.apache.ratis.shaded.proto.RaftProtos.RaftClientRequestProto;
+import org.apache.ratis.shaded.proto.RaftProtos.RaftRpcRequestProto;
+import org.apache.ratis.conf.RaftProperties;
+import org.apache.ratis.grpc.RaftGrpcConfigKeys;
+import org.apache.ratis.grpc.RaftGrpcUtil;
+import org.apache.ratis.protocol.NotLeaderException;
+import org.apache.ratis.protocol.RaftClientReply;
+import org.apache.ratis.protocol.RaftPeer;
+import org.apache.ratis.util.Daemon;
+import org.apache.ratis.util.PeerProxyMap;
+import org.apache.ratis.util.RaftUtils;
+import org.apache.ratis.util.StringUtils;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.io.Closeable;
+import java.io.IOException;
+import java.util.*;
+import java.util.concurrent.ConcurrentLinkedDeque;
+import java.util.concurrent.TimeUnit;
+import java.util.concurrent.atomic.AtomicInteger;
+import java.util.function.Function;
+import java.util.stream.Collectors;
+
+import static org.apache.ratis.client.impl.ClientProtoUtils.*;
+import static org.apache.ratis.grpc.RaftGrpcConfigKeys.RAFT_GRPC_CLIENT_MAX_OUTSTANDING_APPENDS_DEFAULT;
+import static org.apache.ratis.grpc.RaftGrpcConfigKeys.RAFT_GRPC_CLIENT_MAX_OUTSTANDING_APPENDS_KEY;
+
+public class AppendStreamer implements Closeable {
+  public static final Logger LOG = LoggerFactory.getLogger(AppendStreamer.class);
+
+  enum RunningState {RUNNING, LOOK_FOR_LEADER, CLOSED, ERROR}
+
+  private static class ExceptionAndRetry {
+    private final Map<String, IOException> exceptionMap = new HashMap<>();
+    private final AtomicInteger retryTimes = new AtomicInteger(0);
+    private final int maxRetryTimes;
+    private final long retryInterval;
+
+    ExceptionAndRetry(RaftProperties prop) {
+      maxRetryTimes = prop.getInt(
+          RaftGrpcConfigKeys.RAFT_OUTPUTSTREAM_MAX_RETRY_TIMES_KEY,
+          RaftGrpcConfigKeys.RAFT_OUTPUTSTREAM_MAX_RETRY_TIMES_DEFAULT);
+      retryInterval = prop.getTimeDuration(
+          RaftGrpcConfigKeys.RAFT_OUTPUTSTREAM_RETRY_INTERVAL_KEY,
+          RaftGrpcConfigKeys.RAFT_OUTPUTSTREAM_RETRY_INTERVAL_DEFAULT,
+          TimeUnit.MILLISECONDS);
+    }
+
+    void addException(String peer, IOException e) {
+      exceptionMap.put(peer, e);
+      retryTimes.incrementAndGet();
+    }
+
+    IOException getCombinedException() {
+      return new IOException("Exceptions: " + exceptionMap);
+    }
+
+    boolean shouldRetry() {
+      return retryTimes.get() <= maxRetryTimes;
+    }
+  }
+
+  private final Deque<RaftClientRequestProto> dataQueue;
+  private final Deque<RaftClientRequestProto> ackQueue;
+  private final int maxPendingNum;
+
+  private final PeerProxyMap<RaftClientProtocolProxy> proxyMap;
+  private final Map<String, RaftPeer> peers;
+  private String leaderId;
+  private volatile RaftClientProtocolProxy leaderProxy;
+  private final String clientId;
+
+  private volatile RunningState running = RunningState.RUNNING;
+  private final ExceptionAndRetry exceptionAndRetry;
+  private final Sender senderThread;
+
+  AppendStreamer(RaftProperties prop, Collection<RaftPeer> peers,
+      String leaderId, String clientId) {
+    this.clientId = clientId;
+    maxPendingNum = prop.getInt(
+        RAFT_GRPC_CLIENT_MAX_OUTSTANDING_APPENDS_KEY,
+        RAFT_GRPC_CLIENT_MAX_OUTSTANDING_APPENDS_DEFAULT);
+    dataQueue = new ConcurrentLinkedDeque<>();
+    ackQueue = new ConcurrentLinkedDeque<>();
+    exceptionAndRetry = new ExceptionAndRetry(prop);
+
+    this.peers = peers.stream().collect(
+        Collectors.toMap(RaftPeer::getId, Function.identity()));
+    proxyMap = new PeerProxyMap<>(
+        raftPeer -> new RaftClientProtocolProxy(raftPeer, ResponseHandler::new));
+    proxyMap.addPeers(peers);
+    refreshLeaderProxy(leaderId, null);
+
+    senderThread = new Sender();
+    senderThread.setName(this.toString() + "-sender");
+    senderThread.start();
+  }
+
+  private synchronized void refreshLeaderProxy(String suggested,
+      String oldLeader) {
+    if (suggested != null) {
+      leaderId = suggested;
+    } else {
+      if (oldLeader == null) {
+        leaderId = peers.keySet().iterator().next();
+      } else {
+        leaderId = StringUtils.next(oldLeader, peers.keySet());
+      }
+    }
+    LOG.debug("{} switches leader from {} to {}. suggested leader: {}", this,
+          oldLeader, leaderId, suggested);
+    if (leaderProxy != null) {
+      leaderProxy.closeCurrentSession();
+    }
+    try {
+      leaderProxy = proxyMap.getProxy(leaderId);
+    } catch (IOException e) {
+      LOG.error("Should not hit IOException here", e);
+      refreshLeader(null, leaderId);
+    }
+  }
+
+  private boolean isRunning() {
+    return running == RunningState.RUNNING ||
+        running == RunningState.LOOK_FOR_LEADER;
+  }
+
+  private void checkState() throws IOException {
+    if (!isRunning()) {
+      throwException("The AppendStreamer has been closed");
+    }
+  }
+
+  synchronized void write(ByteString content, long seqNum)
+      throws IOException {
+    checkState();
+    while (isRunning() && dataQueue.size() >= maxPendingNum) {
+      try {
+        wait();
+      } catch (InterruptedException ignored) {
+      }
+    }
+    if (isRunning()) {
+      // wrap the current buffer into a RaftClientRequestProto
+      final RaftClientRequestProto request = genRaftClientRequestProto(
+          clientId, leaderId, seqNum, content, false);
+      dataQueue.offer(request);
+      this.notifyAll();
+    } else {
+      throwException(this + " got closed.");
+    }
+  }
+
+  synchronized void flush() throws IOException {
+    checkState();
+    if (dataQueue.isEmpty() && ackQueue.isEmpty()) {
+      return;
+    }
+    // wait for the pending Q to become empty
+    while (isRunning() && (!dataQueue.isEmpty() || !ackQueue.isEmpty())) {
+      try {
+        wait();
+      } catch (InterruptedException ignored) {
+      }
+    }
+    if (!isRunning() && (!dataQueue.isEmpty() || !ackQueue.isEmpty())) {
+      throwException(this + " got closed before finishing flush");
+    }
+  }
+
+  @Override
+  public void close() throws IOException {
+    if (!isRunning()) {
+      return;
+    }
+    flush();
+
+    running = RunningState.CLOSED;
+    senderThread.interrupt();
+    try {
+      senderThread.join();
+    } catch (InterruptedException ignored) {
+    }
+    proxyMap.close();
+  }
+
+  @Override
+  public String toString() {
+    return this.getClass().getSimpleName() + "-" + clientId;
+  }
+
+  private class Sender extends Daemon {
+    @Override
+    public void run() {
+      while (isRunning()) {
+
+        synchronized (AppendStreamer.this) {
+          while (isRunning() && shouldWait()) {
+            try {
+              AppendStreamer.this.wait();
+            } catch (InterruptedException ignored) {
+            }
+          }
+          if (running == RunningState.RUNNING) {
+            RaftClientRequestProto next = dataQueue.poll();
+            leaderProxy.onNext(next);
+            ackQueue.offer(next);
+          }
+        }
+      }
+    }
+
+    private boolean shouldWait() {
+      // the sender should wait if any of the following is true
+      // 1) there is no data to send
+      // 2) there are too many outstanding pending requests
+      // 3) Error/NotLeaderException just happened, we're still waiting for
+      //    the first response to confirm the new leader
+      return dataQueue.isEmpty() || ackQueue.size() >= maxPendingNum ||
+          running == RunningState.LOOK_FOR_LEADER;
+    }
+  }
+
+  /** the response handler for stream RPC */
+  private class ResponseHandler implements
+      RaftClientProtocolProxy.CloseableStreamObserver {
+    private final String targetId;
+    // once handled the first NotLeaderException or Error, the handler should
+    // be inactive and should not make any further action.
+    private volatile boolean active = true;
+
+    ResponseHandler(RaftPeer target) {
+      targetId = target.getId();
+    }
+
+    @Override
+    public String toString() {
+      return AppendStreamer.this + "-ResponseHandler-" + targetId;
+    }
+
+    @Override
+    public void onNext(RaftClientReplyProto reply) {
+      if (!active) {
+        return;
+      }
+      synchronized (AppendStreamer.this) {
+        RaftClientRequestProto pending = Preconditions.checkNotNull(
+            ackQueue.peek());
+        if (reply.getRpcReply().getSuccess()) {
+          Preconditions.checkState(pending.getRpcRequest().getSeqNum() ==
+              reply.getRpcReply().getSeqNum());
+          ackQueue.poll();
+          LOG.trace("{} received success ack for request {}", this,
+              pending.getRpcRequest());
+          // we've identified the correct leader
+          if (running == RunningState.LOOK_FOR_LEADER) {
+            running = RunningState.RUNNING;
+          }
+        } else {
+          // this may be a NotLeaderException
+          RaftClientReply r = toRaftClientReply(reply);
+          if (r.isNotLeader()) {
+            LOG.debug("{} received a NotLeaderException from {}", this,
+                r.getReplierId());
+            handleNotLeader(r.getNotLeaderException(), targetId);
+          }
+        }
+        AppendStreamer.this.notifyAll();
+      }
+    }
+
+    @Override
+    public void onError(Throwable t) {
+      if (active) {
+        synchronized (AppendStreamer.this) {
+          handleError(t, this);
+          AppendStreamer.this.notifyAll();
+        }
+      }
+    }
+
+    @Override
+    public void onCompleted() {
+      LOG.info("{} onCompleted, pending requests #: {}", this,
+          ackQueue.size());
+    }
+
+    @Override // called by handleError and handleNotLeader
+    public void close() throws IOException {
+      active = false;
+    }
+  }
+
+  private void throwException(String msg) throws IOException {
+    if (running == RunningState.ERROR) {
+      throw exceptionAndRetry.getCombinedException();
+    } else {
+      throw new IOException(msg);
+    }
+  }
+
+  private void handleNotLeader(NotLeaderException nle,
+      String oldLeader) {
+    Preconditions.checkState(Thread.holdsLock(AppendStreamer.this));
+    // handle NotLeaderException: refresh leader and RaftConfiguration
+    refreshPeers(nle.getPeers());
+
+    refreshLeader(nle.getSuggestedLeader().getId(), oldLeader);
+  }
+
+  private void handleError(Throwable t, ResponseHandler handler) {
+    Preconditions.checkState(Thread.holdsLock(AppendStreamer.this));
+    final IOException e = RaftGrpcUtil.unwrapIOException(t);
+
+    exceptionAndRetry.addException(handler.targetId, e);
+    LOG.debug("{} got error: {}. Total retry times {}, max retry times {}.",
+        handler, e, exceptionAndRetry.retryTimes.get(),
+        exceptionAndRetry.maxRetryTimes);
+
+    leaderProxy.onError();
+    if (exceptionAndRetry.shouldRetry()) {
+      refreshLeader(null, leaderId);
+    } else {
+      running = RunningState.ERROR;
+    }
+  }
+
+  private void refreshLeader(String suggestedLeader, String oldLeader) {
+    running = RunningState.LOOK_FOR_LEADER;
+    refreshLeaderProxy(suggestedLeader, oldLeader);
+    reQueuePendingRequests(leaderId);
+
+    final RaftClientRequestProto request = Preconditions.checkNotNull(
+        dataQueue.poll());
+    ackQueue.offer(request);
+    try {
+      Thread.sleep(exceptionAndRetry.retryInterval);
+    } catch (InterruptedException ignored) {
+    }
+    leaderProxy.onNext(request);
+  }
+
+  private void reQueuePendingRequests(String newLeader) {
+    if (isRunning()) {
+      // resend all the pending requests
+      while (!ackQueue.isEmpty()) {
+        RaftClientRequestProto oldRequest = ackQueue.pollLast();
+        RaftRpcRequestProto r = oldRequest.getRpcRequest();
+        RaftClientRequestProto newRequest = RaftClientRequestProto.newBuilder()
+            .setMessage(oldRequest.getMessage())
+            .setReadOnly(oldRequest.getReadOnly())
+            .setRpcRequest(toRaftRpcRequestProtoBuilder(
+                clientId, newLeader, r.getSeqNum()))
+            .build();
+        dataQueue.offerFirst(newRequest);
+      }
+    }
+  }
+
+  private void refreshPeers(RaftPeer[] newPeers) {
+    if (newPeers != null && newPeers.length > 0) {
+      // we only add new peers, we do not remove any peer even if it no longer
+      // belongs to the current raft conf
+      Arrays.stream(newPeers).forEach(peer -> {
+        peers.putIfAbsent(peer.getId(), peer);
+        proxyMap.putIfAbsent(peer);
+      });
+
+      LOG.debug("refreshed peers: {}", peers);
+    }
+  }
+}

http://git-wip-us.apache.org/repos/asf/incubator-ratis/blob/7e71a2e0/ratis-grpc/src/main/java/org/apache/ratis/grpc/client/RaftClientProtocolClient.java
----------------------------------------------------------------------
diff --git a/ratis-grpc/src/main/java/org/apache/ratis/grpc/client/RaftClientProtocolClient.java b/ratis-grpc/src/main/java/org/apache/ratis/grpc/client/RaftClientProtocolClient.java
new file mode 100644
index 0000000..74fb253
--- /dev/null
+++ b/ratis-grpc/src/main/java/org/apache/ratis/grpc/client/RaftClientProtocolClient.java
@@ -0,0 +1,73 @@
+/**
+ * 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.ratis.grpc.client;
+
+import org.apache.ratis.shaded.io.grpc.ManagedChannel;
+import org.apache.ratis.shaded.io.grpc.ManagedChannelBuilder;
+import org.apache.ratis.shaded.io.grpc.StatusRuntimeException;
+import org.apache.ratis.shaded.io.grpc.stub.StreamObserver;
+import org.apache.ratis.shaded.proto.RaftProtos.RaftClientReplyProto;
+import org.apache.ratis.shaded.proto.RaftProtos.RaftClientRequestProto;
+import org.apache.ratis.shaded.proto.RaftProtos.SetConfigurationRequestProto;
+import org.apache.ratis.shaded.proto.grpc.RaftClientProtocolServiceGrpc;
+import org.apache.ratis.shaded.proto.grpc.RaftClientProtocolServiceGrpc.RaftClientProtocolServiceBlockingStub;
+import org.apache.ratis.shaded.proto.grpc.RaftClientProtocolServiceGrpc.RaftClientProtocolServiceStub;
+import org.apache.ratis.grpc.RaftGrpcUtil;
+import org.apache.ratis.protocol.RaftPeer;
+
+import java.io.Closeable;
+import java.io.IOException;
+
+public class RaftClientProtocolClient implements Closeable {
+  private final RaftPeer target;
+  private final ManagedChannel channel;
+  private final RaftClientProtocolServiceBlockingStub blockingStub;
+  private final RaftClientProtocolServiceStub asyncStub;
+
+  public RaftClientProtocolClient(RaftPeer target) {
+    this.target = target;
+    channel = ManagedChannelBuilder.forTarget(target.getAddress())
+        .usePlaintext(true).build();
+    blockingStub = RaftClientProtocolServiceGrpc.newBlockingStub(channel);
+    asyncStub = RaftClientProtocolServiceGrpc.newStub(channel);
+  }
+
+  @Override
+  public void close() {
+    channel.shutdownNow();
+  }
+
+  public RaftClientReplyProto setConfiguration(
+      SetConfigurationRequestProto request) throws IOException {
+    try {
+      return blockingStub.setConfiguration(request);
+    } catch (StatusRuntimeException e) {
+      // unwrap StatusRuntimeException
+      throw RaftGrpcUtil.unwrapException(e);
+    }
+  }
+
+  StreamObserver<RaftClientRequestProto> append(
+      StreamObserver<RaftClientReplyProto> responseHandler) {
+    return asyncStub.append(responseHandler);
+  }
+
+  public RaftPeer getTarget() {
+    return target;
+  }
+}

http://git-wip-us.apache.org/repos/asf/incubator-ratis/blob/7e71a2e0/ratis-grpc/src/main/java/org/apache/ratis/grpc/client/RaftClientProtocolProxy.java
----------------------------------------------------------------------
diff --git a/ratis-grpc/src/main/java/org/apache/ratis/grpc/client/RaftClientProtocolProxy.java b/ratis-grpc/src/main/java/org/apache/ratis/grpc/client/RaftClientProtocolProxy.java
new file mode 100644
index 0000000..6892c71
--- /dev/null
+++ b/ratis-grpc/src/main/java/org/apache/ratis/grpc/client/RaftClientProtocolProxy.java
@@ -0,0 +1,104 @@
+/**
+ * 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.ratis.grpc.client;
+
+import org.apache.ratis.shaded.io.grpc.stub.StreamObserver;
+import org.apache.ratis.shaded.proto.RaftProtos.RaftClientReplyProto;
+import org.apache.ratis.shaded.proto.RaftProtos.RaftClientRequestProto;
+import org.apache.ratis.protocol.RaftPeer;
+
+import java.io.Closeable;
+import java.io.IOException;
+import java.util.function.Function;
+
+public class RaftClientProtocolProxy implements Closeable {
+  private final RaftClientProtocolClient proxy;
+  private final Function<RaftPeer, CloseableStreamObserver> responseHandlerCreation;
+  private RpcSession currentSession;
+
+  public RaftClientProtocolProxy(RaftPeer target,
+      Function<RaftPeer, CloseableStreamObserver> responseHandlerCreation) {
+    proxy = new RaftClientProtocolClient(target);
+    this.responseHandlerCreation = responseHandlerCreation;
+  }
+
+  @Override
+  public void close() throws IOException {
+    closeCurrentSession();
+    proxy.close();
+  }
+
+  @Override
+  public String toString() {
+    return "ProxyTo:" + proxy.getTarget();
+  }
+
+  public void closeCurrentSession() {
+    if (currentSession != null) {
+      currentSession.close();
+      currentSession = null;
+    }
+  }
+
+  public void onNext(RaftClientRequestProto request) {
+    if (currentSession == null) {
+      currentSession = new RpcSession(
+          responseHandlerCreation.apply(proxy.getTarget()));
+    }
+    currentSession.requestObserver.onNext(request);
+  }
+
+  public void onError() {
+    if (currentSession != null) {
+      currentSession.onError();
+    }
+  }
+
+  public interface CloseableStreamObserver
+      extends StreamObserver<RaftClientReplyProto>, Closeable {
+  }
+
+  class RpcSession implements Closeable {
+    private final StreamObserver<RaftClientRequestProto> requestObserver;
+    private final CloseableStreamObserver responseHandler;
+    private boolean hasError = false;
+
+    RpcSession(CloseableStreamObserver responseHandler) {
+      this.responseHandler = responseHandler;
+      this.requestObserver = proxy.append(responseHandler);
+    }
+
+    void onError() {
+      hasError = true;
+    }
+
+    @Override
+    public void close() {
+      if (!hasError) {
+        try {
+          requestObserver.onCompleted();
+        } catch (Exception ignored) {
+        }
+      }
+      try {
+        responseHandler.close();
+      } catch (IOException ignored) {
+      }
+    }
+  }
+}

http://git-wip-us.apache.org/repos/asf/incubator-ratis/blob/7e71a2e0/ratis-grpc/src/main/java/org/apache/ratis/grpc/client/RaftClientProtocolService.java
----------------------------------------------------------------------
diff --git a/ratis-grpc/src/main/java/org/apache/ratis/grpc/client/RaftClientProtocolService.java b/ratis-grpc/src/main/java/org/apache/ratis/grpc/client/RaftClientProtocolService.java
new file mode 100644
index 0000000..bb212e1
--- /dev/null
+++ b/ratis-grpc/src/main/java/org/apache/ratis/grpc/client/RaftClientProtocolService.java
@@ -0,0 +1,195 @@
+/**
+ * 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.ratis.grpc.client;
+
+import com.google.common.base.Preconditions;
+
+import org.apache.ratis.shaded.io.grpc.stub.StreamObserver;
+import org.apache.ratis.shaded.proto.RaftProtos.RaftClientReplyProto;
+import org.apache.ratis.shaded.proto.RaftProtos.RaftClientRequestProto;
+import org.apache.ratis.shaded.proto.RaftProtos.SetConfigurationRequestProto;
+import org.apache.ratis.shaded.proto.grpc.RaftClientProtocolServiceGrpc.RaftClientProtocolServiceImplBase;
+import org.apache.ratis.client.impl.ClientProtoUtils;
+import org.apache.ratis.grpc.RaftGrpcUtil;
+import org.apache.ratis.protocol.RaftClientAsynchronousProtocol;
+import org.apache.ratis.protocol.RaftClientReply;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.util.*;
+import java.util.concurrent.CompletableFuture;
+
+public class RaftClientProtocolService extends RaftClientProtocolServiceImplBase {
+  static final Logger LOG = LoggerFactory.getLogger(RaftClientProtocolService.class);
+
+  private static class PendingAppend implements Comparable<PendingAppend> {
+    private final long seqNum;
+    private volatile RaftClientReply reply;
+
+    PendingAppend(long seqNum) {
+      this.seqNum = seqNum;
+    }
+
+    boolean isReady() {
+      return reply != null || this == COMPLETED;
+    }
+
+    void setReply(RaftClientReply reply) {
+      this.reply = reply;
+    }
+
+    @Override
+    public int compareTo(PendingAppend p) {
+      return seqNum == p.seqNum ? 0 : (seqNum < p.seqNum ? -1 : 1);
+    }
+
+    @Override
+    public String toString() {
+      return seqNum + ", reply:" + (reply == null ? "null" : reply.toString());
+    }
+  }
+  private static final PendingAppend COMPLETED = new PendingAppend(Long.MAX_VALUE);
+
+  private final String id;
+  private final RaftClientAsynchronousProtocol client;
+
+  public RaftClientProtocolService(String id, RaftClientAsynchronousProtocol client) {
+    this.id = id;
+    this.client = client;
+  }
+
+  @Override
+  public void setConfiguration(SetConfigurationRequestProto request,
+      StreamObserver<RaftClientReplyProto> responseObserver) {
+    try {
+      CompletableFuture<RaftClientReply> future = client.setConfigurationAsync(
+          ClientProtoUtils.toSetConfigurationRequest(request));
+      future.whenCompleteAsync((reply, exception) -> {
+        if (exception != null) {
+          responseObserver.onError(RaftGrpcUtil.wrapException(exception));
+        } else {
+          responseObserver.onNext(ClientProtoUtils.toRaftClientReplyProto(reply));
+          responseObserver.onCompleted();
+        }
+      });
+    } catch (Exception e) {
+      responseObserver.onError(RaftGrpcUtil.wrapException(e));
+    }
+  }
+
+  @Override
+  public StreamObserver<RaftClientRequestProto> append(
+      StreamObserver<RaftClientReplyProto> responseObserver) {
+    return new AppendRequestStreamObserver(responseObserver);
+  }
+
+  private class AppendRequestStreamObserver implements
+      StreamObserver<RaftClientRequestProto> {
+    private final List<PendingAppend> pendingList = new LinkedList<>();
+    private final StreamObserver<RaftClientReplyProto> responseObserver;
+
+    AppendRequestStreamObserver(StreamObserver<RaftClientReplyProto> ro) {
+      this.responseObserver = ro;
+    }
+
+    @Override
+    public void onNext(RaftClientRequestProto request) {
+      try {
+        PendingAppend p = new PendingAppend(request.getRpcRequest().getSeqNum());
+        synchronized (pendingList) {
+          pendingList.add(p);
+        }
+
+        CompletableFuture<RaftClientReply> future = client.submitClientRequestAsync(
+            ClientProtoUtils.toRaftClientRequest(request));
+        future.whenCompleteAsync((reply, exception) -> {
+          if (exception != null) {
+            // TODO: the exception may be from either raft or state machine.
+            // Currently we skip all the following responses when getting an
+            // exception from the state machine.
+            responseObserver.onError(RaftGrpcUtil.wrapException(exception));
+          } else {
+            final long replySeq = reply.getSeqNum();
+            synchronized (pendingList) {
+              Preconditions.checkState(!pendingList.isEmpty(),
+                  "PendingList is empty when handling onNext for seqNum %s",
+                  replySeq);
+              final long headSeqNum = pendingList.get(0).seqNum;
+              // we assume the seqNum is consecutive for a stream RPC call
+              final PendingAppend pendingForReply = pendingList.get(
+                  (int) (replySeq - headSeqNum));
+              Preconditions.checkState(pendingForReply != null &&
+                      pendingForReply.seqNum == replySeq,
+                  "pending for reply is: %s, the pending list: %s",
+                  pendingForReply, pendingList);
+              pendingForReply.setReply(reply);
+
+              if (headSeqNum == replySeq) {
+                Collection<PendingAppend> readySet = new ArrayList<>();
+                // if this is head, we send back all the ready responses
+                Iterator<PendingAppend> iter = pendingList.iterator();
+                PendingAppend pending;
+                while (iter.hasNext() && ((pending = iter.next()).isReady())) {
+                  readySet.add(pending);
+                  iter.remove();
+                }
+                sendReadyReplies(readySet);
+              }
+            }
+          }
+        });
+      } catch (Throwable e) {
+        LOG.info("{} got exception when handling client append request {}: {}",
+            id, request.getRpcRequest(), e);
+        responseObserver.onError(RaftGrpcUtil.wrapException(e));
+      }
+    }
+
+    private void sendReadyReplies(Collection<PendingAppend> readySet) {
+      readySet.forEach(ready -> {
+        Preconditions.checkState(ready.isReady());
+        if (ready == COMPLETED) {
+          responseObserver.onCompleted();
+        } else {
+          responseObserver.onNext(
+              ClientProtoUtils.toRaftClientReplyProto(ready.reply));
+        }
+      });
+    }
+
+    @Override
+    public void onError(Throwable t) {
+      // for now we just log a msg
+      LOG.warn("{} onError: client Append cancelled", id, t);
+      synchronized (pendingList) {
+        pendingList.clear();
+      }
+    }
+
+    @Override
+    public void onCompleted() {
+      synchronized (pendingList) {
+        if (pendingList.isEmpty()) {
+          responseObserver.onCompleted();
+        } else {
+          pendingList.add(COMPLETED);
+        }
+      }
+    }
+  }
+}

http://git-wip-us.apache.org/repos/asf/incubator-ratis/blob/7e71a2e0/ratis-grpc/src/main/java/org/apache/ratis/grpc/client/RaftClientSenderWithGrpc.java
----------------------------------------------------------------------
diff --git a/ratis-grpc/src/main/java/org/apache/ratis/grpc/client/RaftClientSenderWithGrpc.java b/ratis-grpc/src/main/java/org/apache/ratis/grpc/client/RaftClientSenderWithGrpc.java
new file mode 100644
index 0000000..6d9e11f
--- /dev/null
+++ b/ratis-grpc/src/main/java/org/apache/ratis/grpc/client/RaftClientSenderWithGrpc.java
@@ -0,0 +1,119 @@
+/**
+ * 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.ratis.grpc.client;
+
+import org.apache.ratis.shaded.io.grpc.StatusRuntimeException;
+import org.apache.ratis.shaded.io.grpc.stub.StreamObserver;
+import org.apache.ratis.shaded.proto.RaftProtos.RaftClientReplyProto;
+import org.apache.ratis.shaded.proto.RaftProtos.RaftClientRequestProto;
+import org.apache.ratis.shaded.proto.RaftProtos.SetConfigurationRequestProto;
+import org.apache.ratis.client.RaftClientRequestSender;
+import org.apache.ratis.grpc.RaftGrpcUtil;
+import org.apache.ratis.protocol.RaftClientReply;
+import org.apache.ratis.protocol.RaftClientRequest;
+import org.apache.ratis.protocol.RaftPeer;
+import org.apache.ratis.protocol.SetConfigurationRequest;
+import org.apache.ratis.util.PeerProxyMap;
+import org.apache.ratis.util.RaftUtils;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import static org.apache.ratis.client.impl.ClientProtoUtils.*;
+
+import java.io.IOException;
+import java.io.InterruptedIOException;
+import java.util.Collection;
+import java.util.concurrent.CompletableFuture;
+import java.util.concurrent.ExecutionException;
+
+public class RaftClientSenderWithGrpc implements RaftClientRequestSender {
+  public static final Logger LOG = LoggerFactory.getLogger(RaftClientSenderWithGrpc.class);
+
+  private final PeerProxyMap<RaftClientProtocolClient> proxies
+      = new PeerProxyMap<>(RaftClientProtocolClient::new);
+
+  public RaftClientSenderWithGrpc(Collection<RaftPeer> peers) {
+    addServers(peers);
+  }
+
+  @Override
+  public RaftClientReply sendRequest(RaftClientRequest request)
+      throws IOException {
+    final String serverId = request.getReplierId();
+    final RaftClientProtocolClient proxy = proxies.getProxy(serverId);
+    if (request instanceof SetConfigurationRequest) {
+      SetConfigurationRequestProto setConf =
+          toSetConfigurationRequestProto((SetConfigurationRequest) request);
+      return toRaftClientReply(proxy.setConfiguration(setConf));
+    } else {
+      RaftClientRequestProto requestProto = toRaftClientRequestProto(request);
+      CompletableFuture<RaftClientReplyProto> replyFuture =
+          new CompletableFuture<>();
+      final StreamObserver<RaftClientRequestProto> requestObserver =
+          proxy.append(new StreamObserver<RaftClientReplyProto>() {
+            @Override
+            public void onNext(RaftClientReplyProto value) {
+              replyFuture.complete(value);
+            }
+
+            @Override
+            public void onError(Throwable t) {
+              // This implementation is used as RaftClientRequestSender. Retry
+              // logic on Exception is in RaftClient.
+              final IOException e;
+              if (t instanceof StatusRuntimeException) {
+                e = RaftGrpcUtil.unwrapException((StatusRuntimeException) t);
+              } else {
+                e = RaftUtils.asIOException(t);
+              }
+              replyFuture.completeExceptionally(e);
+            }
+
+            @Override
+            public void onCompleted() {
+              if (!replyFuture.isDone()) {
+                replyFuture.completeExceptionally(
+                    new IOException("No reply for request " + request));
+              }
+            }
+          });
+      requestObserver.onNext(requestProto);
+      requestObserver.onCompleted();
+
+      // TODO: timeout support
+      try {
+        return toRaftClientReply(replyFuture.get());
+      } catch (InterruptedException e) {
+        throw new InterruptedIOException(
+            "Interrupted while waiting for response of request " + request);
+      } catch (ExecutionException e) {
+        throw RaftUtils.toIOException(e);
+      }
+    }
+  }
+
+  @Override
+  public void addServers(Iterable<RaftPeer> servers) {
+    proxies.addPeers(servers);
+  }
+
+  @Override
+  public void close() throws IOException {
+    proxies.close();
+  }
+}

http://git-wip-us.apache.org/repos/asf/incubator-ratis/blob/7e71a2e0/ratis-grpc/src/main/java/org/apache/ratis/grpc/client/RaftOutputStream.java
----------------------------------------------------------------------
diff --git a/ratis-grpc/src/main/java/org/apache/ratis/grpc/client/RaftOutputStream.java b/ratis-grpc/src/main/java/org/apache/ratis/grpc/client/RaftOutputStream.java
new file mode 100644
index 0000000..a3905f8
--- /dev/null
+++ b/ratis-grpc/src/main/java/org/apache/ratis/grpc/client/RaftOutputStream.java
@@ -0,0 +1,112 @@
+/**
+ * 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.ratis.grpc.client;
+
+import static org.apache.ratis.grpc.RaftGrpcConfigKeys.RAFT_OUTPUTSTREAM_BUFFER_SIZE_DEFAULT;
+import static org.apache.ratis.grpc.RaftGrpcConfigKeys.RAFT_OUTPUTSTREAM_BUFFER_SIZE_KEY;
+
+import java.io.IOException;
+import java.io.OutputStream;
+import java.util.Collection;
+
+import org.apache.ratis.conf.RaftProperties;
+import org.apache.ratis.protocol.RaftPeer;
+import org.apache.ratis.util.ProtoUtils;
+
+public class RaftOutputStream extends OutputStream {
+  /** internal buffer */
+  private final byte buf[];
+  private int count;
+  private long seqNum = 0;
+  private final String clientId;
+  private final AppendStreamer streamer;
+
+  private boolean closed = false;
+
+  public RaftOutputStream(RaftProperties prop, String clientId,
+      Collection<RaftPeer> peers, String leaderId) {
+    final int bufferSize = prop.getInt(RAFT_OUTPUTSTREAM_BUFFER_SIZE_KEY,
+        RAFT_OUTPUTSTREAM_BUFFER_SIZE_DEFAULT);
+    buf = new byte[bufferSize];
+    count = 0;
+    this.clientId = clientId;
+    streamer = new AppendStreamer(prop, peers, leaderId, clientId);
+  }
+
+  @Override
+  public void write(int b) throws IOException {
+    checkClosed();
+    buf[count++] = (byte)b;
+    flushIfNecessary();
+  }
+
+  private void flushIfNecessary() throws IOException {
+    if(count == buf.length) {
+      flushToStreamer();
+    }
+  }
+
+  @Override
+  public void write(byte b[], int off, int len) throws IOException {
+    checkClosed();
+    if (off < 0 || len < 0 || off > b.length - len) {
+      throw new ArrayIndexOutOfBoundsException();
+    }
+
+    int total = 0;
+    while (total < len) {
+      int toWrite = Math.min(len - total, buf.length - count);
+      System.arraycopy(b, off + total, buf, count, toWrite);
+      count += toWrite;
+      total += toWrite;
+      flushIfNecessary();
+    }
+  }
+
+  private void flushToStreamer() throws IOException {
+    if (count > 0) {
+      streamer.write(ProtoUtils.toByteString(buf, 0, count), seqNum++);
+      count = 0;
+    }
+  }
+
+  @Override
+  public void flush() throws IOException {
+    checkClosed();
+    flushToStreamer();
+    streamer.flush();
+  }
+
+  @Override
+  public void close() throws IOException {
+    flushToStreamer();
+    streamer.close(); // streamer will flush
+    this.closed = true;
+  }
+
+  @Override
+  public String toString() {
+    return "RaftOutputStream-" + clientId;
+  }
+
+  private void checkClosed() throws IOException {
+    if (closed) {
+      throw new IOException(this.toString() + " was closed.");
+    }
+  }
+}


[20/54] [abbrv] incubator-ratis git commit: Renamed the packages from raft to ratis in preperation for Apache Incubation - Moved all java packages from org.apache.raft to org.apache.ratis. - Moved native package to org_apache_ratis, and native lib to l

Posted by ji...@apache.org.
http://git-wip-us.apache.org/repos/asf/incubator-ratis/blob/7e71a2e0/ratis-server/src/main/java/org/apache/ratis/server/impl/RaftServerImpl.java
----------------------------------------------------------------------
diff --git a/ratis-server/src/main/java/org/apache/ratis/server/impl/RaftServerImpl.java b/ratis-server/src/main/java/org/apache/ratis/server/impl/RaftServerImpl.java
new file mode 100644
index 0000000..d079abf
--- /dev/null
+++ b/ratis-server/src/main/java/org/apache/ratis/server/impl/RaftServerImpl.java
@@ -0,0 +1,832 @@
+/**
+ * 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.ratis.server.impl;
+
+import static org.apache.ratis.util.LifeCycle.State.CLOSED;
+import static org.apache.ratis.util.LifeCycle.State.CLOSING;
+import static org.apache.ratis.util.LifeCycle.State.RUNNING;
+import static org.apache.ratis.util.LifeCycle.State.STARTING;
+
+import static org.apache.ratis.shaded.proto.RaftProtos.AppendEntriesReplyProto.AppendResult.*;
+import java.io.IOException;
+import java.io.InterruptedIOException;
+import java.util.Arrays;
+import java.util.Collection;
+import java.util.List;
+import java.util.OptionalLong;
+import java.util.concurrent.CompletableFuture;
+import java.util.concurrent.ExecutionException;
+
+import org.apache.ratis.conf.RaftProperties;
+import org.apache.ratis.protocol.Message;
+import org.apache.ratis.protocol.NotLeaderException;
+import org.apache.ratis.protocol.RaftClientReply;
+import org.apache.ratis.protocol.RaftClientRequest;
+import org.apache.ratis.protocol.RaftException;
+import org.apache.ratis.protocol.RaftPeer;
+import org.apache.ratis.protocol.ReconfigurationInProgressException;
+import org.apache.ratis.protocol.SetConfigurationRequest;
+import org.apache.ratis.server.RaftServer;
+import org.apache.ratis.server.RaftServerConfigKeys;
+import org.apache.ratis.server.RaftServerRpc;
+import org.apache.ratis.server.protocol.TermIndex;
+import org.apache.ratis.server.storage.FileInfo;
+import org.apache.ratis.shaded.proto.RaftProtos.AppendEntriesReplyProto;
+import org.apache.ratis.shaded.proto.RaftProtos.AppendEntriesRequestProto;
+import org.apache.ratis.shaded.proto.RaftProtos.FileChunkProto;
+import org.apache.ratis.shaded.proto.RaftProtos.InstallSnapshotReplyProto;
+import org.apache.ratis.shaded.proto.RaftProtos.InstallSnapshotRequestProto;
+import org.apache.ratis.shaded.proto.RaftProtos.InstallSnapshotResult;
+import org.apache.ratis.shaded.proto.RaftProtos.LogEntryProto;
+import org.apache.ratis.shaded.proto.RaftProtos.RequestVoteReplyProto;
+import org.apache.ratis.shaded.proto.RaftProtos.RequestVoteRequestProto;
+import org.apache.ratis.statemachine.SnapshotInfo;
+import org.apache.ratis.statemachine.StateMachine;
+import org.apache.ratis.statemachine.TransactionContext;
+import org.apache.ratis.util.CodeInjectionForTesting;
+import org.apache.ratis.util.LifeCycle;
+import org.apache.ratis.util.ProtoUtils;
+import org.apache.ratis.util.RaftUtils;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import com.google.common.annotations.VisibleForTesting;
+import com.google.common.base.Preconditions;
+
+public class RaftServerImpl implements RaftServer {
+  public static final Logger LOG = LoggerFactory.getLogger(RaftServerImpl.class);
+
+  private static final String CLASS_NAME = RaftServerImpl.class.getSimpleName();
+  static final String REQUEST_VOTE = CLASS_NAME + ".requestVote";
+  static final String APPEND_ENTRIES = CLASS_NAME + ".appendEntries";
+  static final String INSTALL_SNAPSHOT = CLASS_NAME + ".installSnapshot";
+
+
+  /** Role of raft peer */
+  enum Role {
+    LEADER, CANDIDATE, FOLLOWER
+  }
+
+  private final int minTimeoutMs;
+  private final int maxTimeoutMs;
+
+  private final LifeCycle lifeCycle;
+  private final ServerState state;
+  private final StateMachine stateMachine;
+  private final RaftProperties properties;
+  private volatile Role role;
+
+  /** used when the peer is follower, to monitor election timeout */
+  private volatile FollowerState heartbeatMonitor;
+
+  /** used when the peer is candidate, to request votes from other peers */
+  private volatile LeaderElection electionDaemon;
+
+  /** used when the peer is leader */
+  private volatile LeaderState leaderState;
+
+  private RaftServerRpc serverRpc;
+
+  private final LogAppenderFactory appenderFactory;
+
+  public RaftServerImpl(String id, RaftConfiguration raftConf,
+                        RaftProperties properties, StateMachine stateMachine) throws IOException {
+    this.lifeCycle = new LifeCycle(id);
+    minTimeoutMs = properties.getInt(
+        RaftServerConfigKeys.RAFT_SERVER_RPC_TIMEOUT_MIN_MS_KEY,
+        RaftServerConfigKeys.RAFT_SERVER_RPC_TIMEOUT_MIN_MS_DEFAULT);
+    maxTimeoutMs = properties.getInt(
+        RaftServerConfigKeys.RAFT_SERVER_RPC_TIMEOUT_MAX_MS_KEY,
+        RaftServerConfigKeys.RAFT_SERVER_RPC_TIMEOUT_MAX_MS_DEFAULT);
+    Preconditions.checkArgument(maxTimeoutMs > minTimeoutMs,
+        "max timeout: %s, min timeout: %s", maxTimeoutMs, minTimeoutMs);
+    this.properties = properties;
+    this.stateMachine = stateMachine;
+    this.state = new ServerState(id, raftConf, properties, this, stateMachine);
+    appenderFactory = initAppenderFactory();
+  }
+
+  int getMinTimeoutMs() {
+    return minTimeoutMs;
+  }
+
+  int getMaxTimeoutMs() {
+    return maxTimeoutMs;
+  }
+
+  int getRandomTimeoutMs() {
+    return RaftUtils.getRandomBetween(minTimeoutMs, maxTimeoutMs);
+  }
+
+  @Override
+  public StateMachine getStateMachine() {
+    return this.stateMachine;
+  }
+
+  public LogAppenderFactory getLogAppenderFactory() {
+    return appenderFactory;
+  }
+
+  private LogAppenderFactory initAppenderFactory() {
+    Class<? extends LogAppenderFactory> factoryClass = properties.getClass(
+        RaftServerConfigKeys.RAFT_SERVER_LOG_APPENDER_FACTORY_CLASS_KEY,
+        RaftServerConfigKeys.RAFT_SERVER_LOG_APPENDER_FACTORY_CLASS_DEFAULT,
+        LogAppenderFactory.class);
+    return RaftUtils.newInstance(factoryClass);
+  }
+
+  /**
+   * Used by tests to set initial raft configuration with correct port bindings.
+   */
+  @VisibleForTesting
+  public void setInitialConf(RaftConfiguration conf) {
+    this.state.setInitialConf(conf);
+  }
+
+  @Override
+  public void setServerRpc(RaftServerRpc serverRpc) {
+    this.serverRpc = serverRpc;
+    // add peers into rpc service
+    RaftConfiguration conf = getRaftConf();
+    if (conf != null) {
+      serverRpc.addPeers(conf.getPeers());
+    }
+  }
+
+  public RaftServerRpc getServerRpc() {
+    return serverRpc;
+  }
+
+  @Override
+  public void start() {
+    lifeCycle.transition(STARTING);
+    state.start();
+    RaftConfiguration conf = getRaftConf();
+    if (conf != null && conf.contains(getId())) {
+      LOG.debug("{} starts as a follower", getId());
+      startAsFollower();
+    } else {
+      LOG.debug("{} starts with initializing state", getId());
+      startInitializing();
+    }
+  }
+
+  /**
+   * The peer belongs to the current configuration, should start as a follower
+   */
+  private void startAsFollower() {
+    role = Role.FOLLOWER;
+    heartbeatMonitor = new FollowerState(this);
+    heartbeatMonitor.start();
+
+    serverRpc.start();
+    lifeCycle.transition(RUNNING);
+  }
+
+  /**
+   * The peer does not have any configuration (maybe it will later be included
+   * in some configuration). Start still as a follower but will not vote or
+   * start election.
+   */
+  private void startInitializing() {
+    role = Role.FOLLOWER;
+    // do not start heartbeatMonitoring
+    serverRpc.start();
+  }
+
+  public ServerState getState() {
+    return this.state;
+  }
+
+  @Override
+  public String getId() {
+    return getState().getSelfId();
+  }
+
+  RaftConfiguration getRaftConf() {
+    return getState().getRaftConf();
+  }
+
+  @Override
+  public void close() {
+    lifeCycle.checkStateAndClose(() -> {
+      try {
+        shutdownHeartbeatMonitor();
+        shutdownElectionDaemon();
+        shutdownLeaderState();
+
+        serverRpc.close();
+        state.close();
+      } catch (Exception ignored) {
+        LOG.warn("Failed to kill " + state.getSelfId(), ignored);
+      }
+    });
+  }
+
+  @VisibleForTesting
+  public boolean isAlive() {
+    return !lifeCycle.getCurrentState().isOneOf(CLOSING, CLOSED);
+  }
+
+  public boolean isFollower() {
+    return role == Role.FOLLOWER;
+  }
+
+  public boolean isCandidate() {
+    return role == Role.CANDIDATE;
+  }
+
+  public boolean isLeader() {
+    return role == Role.LEADER;
+  }
+
+  /**
+   * Change the server state to Follower if necessary
+   * @param newTerm The new term.
+   * @param sync We will call {@link ServerState#persistMetadata()} if this is
+   *             set to true and term/votedFor get updated.
+   * @return if the term/votedFor should be updated to the new term
+   * @throws IOException if term/votedFor persistence failed.
+   */
+  synchronized boolean changeToFollower(long newTerm, boolean sync)
+      throws IOException {
+    final Role old = role;
+    role = Role.FOLLOWER;
+
+    boolean metadataUpdated = false;
+    if (newTerm > state.getCurrentTerm()) {
+      state.setCurrentTerm(newTerm);
+      state.resetLeaderAndVotedFor();
+      metadataUpdated = true;
+    }
+
+    if (old == Role.LEADER) {
+      assert leaderState != null;
+      shutdownLeaderState();
+    } else if (old == Role.CANDIDATE) {
+      shutdownElectionDaemon();
+    }
+
+    if (old != Role.FOLLOWER) {
+      heartbeatMonitor = new FollowerState(this);
+      heartbeatMonitor.start();
+    }
+
+    if (metadataUpdated && sync) {
+      state.persistMetadata();
+    }
+    return metadataUpdated;
+  }
+
+  private synchronized void shutdownLeaderState() {
+    final LeaderState leader = leaderState;
+    if (leader != null) {
+      leader.stop();
+    }
+    leaderState = null;
+    // TODO: make sure that StateMachineUpdater has applied all transactions that have context
+  }
+
+  private void shutdownElectionDaemon() {
+    final LeaderElection election = electionDaemon;
+    if (election != null) {
+      election.stopRunning();
+      // no need to interrupt the election thread
+    }
+    electionDaemon = null;
+  }
+
+  synchronized void changeToLeader() {
+    Preconditions.checkState(isCandidate());
+    shutdownElectionDaemon();
+    role = Role.LEADER;
+    state.becomeLeader();
+    // start sending AppendEntries RPC to followers
+    leaderState = new LeaderState(this, properties);
+    leaderState.start();
+  }
+
+  private void shutdownHeartbeatMonitor() {
+    final FollowerState hm = heartbeatMonitor;
+    if (hm != null) {
+      hm.stopRunning();
+      hm.interrupt();
+    }
+    heartbeatMonitor = null;
+  }
+
+  synchronized void changeToCandidate() {
+    Preconditions.checkState(isFollower());
+    shutdownHeartbeatMonitor();
+    role = Role.CANDIDATE;
+    // start election
+    electionDaemon = new LeaderElection(this);
+    electionDaemon.start();
+  }
+
+  @Override
+  public String toString() {
+    return role + " " + state + " " + lifeCycle.getCurrentState();
+  }
+
+  /**
+   * @return null if the server is in leader state.
+   */
+  private CompletableFuture<RaftClientReply> checkLeaderState(
+      RaftClientRequest request) {
+    if (!isLeader()) {
+      NotLeaderException exception = generateNotLeaderException();
+      CompletableFuture<RaftClientReply> future = new CompletableFuture<>();
+      future.complete(new RaftClientReply(request, exception));
+      return future;
+    }
+    return null;
+  }
+
+  NotLeaderException generateNotLeaderException() {
+    if (lifeCycle.getCurrentState() != RUNNING) {
+      return new NotLeaderException(getId(), null, null);
+    }
+    String leaderId = state.getLeaderId();
+    if (leaderId == null || leaderId.equals(state.getSelfId())) {
+      // No idea about who is the current leader. Or the peer is the current
+      // leader, but it is about to step down
+      RaftPeer suggestedLeader = state.getRaftConf()
+          .getRandomPeer(state.getSelfId());
+      leaderId = suggestedLeader == null ? null : suggestedLeader.getId();
+    }
+    RaftConfiguration conf = getRaftConf();
+    Collection<RaftPeer> peers = conf.getPeers();
+    return new NotLeaderException(getId(), conf.getPeer(leaderId),
+        peers.toArray(new RaftPeer[peers.size()]));
+  }
+
+  /**
+   * Handle a normal update request from client.
+   */
+  private CompletableFuture<RaftClientReply> appendTransaction(
+      RaftClientRequest request, TransactionContext entry)
+      throws RaftException {
+    LOG.debug("{}: receive client request({})", getId(), request);
+    lifeCycle.assertCurrentState(RUNNING);
+    CompletableFuture<RaftClientReply> reply;
+
+    final PendingRequest pending;
+    synchronized (this) {
+      reply = checkLeaderState(request);
+      if (reply != null) {
+        return reply;
+      }
+
+      // append the message to its local log
+      final long entryIndex;
+      try {
+        entryIndex = state.applyLog(entry);
+      } catch (IOException e) {
+        throw new RaftException(e);
+      }
+
+      // put the request into the pending queue
+      pending = leaderState.addPendingRequest(entryIndex, request, entry);
+      leaderState.notifySenders();
+    }
+    return pending.getFuture();
+  }
+
+  @Override
+  public CompletableFuture<RaftClientReply> submitClientRequestAsync(
+      RaftClientRequest request) throws IOException {
+    // first check the server's leader state
+    CompletableFuture<RaftClientReply> reply = checkLeaderState(request);
+    if (reply != null) {
+      return reply;
+    }
+
+    // let the state machine handle read-only request from client
+    if (request.isReadOnly()) {
+      // TODO: We might not be the leader anymore by the time this completes. See the RAFT paper,
+      // section 8 (last part)
+      return stateMachine.query(request);
+    }
+
+    // TODO: this client request will not be added to pending requests
+    // until later which means that any failure in between will leave partial state in the
+    // state machine. We should call cancelTransaction() for failed requests
+    TransactionContext entry = stateMachine.startTransaction(request);
+    if (entry.getException().isPresent()) {
+      throw RaftUtils.asIOException(entry.getException().get());
+    }
+
+    return appendTransaction(request, entry);
+  }
+
+  @Override
+  public RaftClientReply submitClientRequest(RaftClientRequest request)
+      throws IOException {
+    return waitForReply(getId(), request, submitClientRequestAsync(request));
+  }
+
+  private static RaftClientReply waitForReply(String id, RaftClientRequest request,
+      CompletableFuture<RaftClientReply> future) throws IOException {
+    try {
+      return future.get();
+    } catch (InterruptedException e) {
+      final String s = id + ": Interrupted when waiting for reply, request=" + request;
+      LOG.info(s, e);
+      throw RaftUtils.toInterruptedIOException(s, e);
+    } catch (ExecutionException e) {
+      final Throwable cause = e.getCause();
+      if (cause == null) {
+        throw new IOException(e);
+      }
+      if (cause instanceof NotLeaderException) {
+        return new RaftClientReply(request, (NotLeaderException)cause);
+      } else {
+        throw RaftUtils.asIOException(cause);
+      }
+    }
+  }
+
+  @Override
+  public RaftClientReply setConfiguration(SetConfigurationRequest request)
+      throws IOException {
+    return waitForReply(getId(), request, setConfigurationAsync(request));
+  }
+
+  /**
+   * Handle a raft configuration change request from client.
+   */
+  @Override
+  public CompletableFuture<RaftClientReply> setConfigurationAsync(
+      SetConfigurationRequest request) throws IOException {
+    LOG.debug("{}: receive setConfiguration({})", getId(), request);
+    lifeCycle.assertCurrentState(RUNNING);
+    CompletableFuture<RaftClientReply> reply = checkLeaderState(request);
+    if (reply != null) {
+      return reply;
+    }
+
+    final RaftPeer[] peersInNewConf = request.getPeersInNewConf();
+    final PendingRequest pending;
+    synchronized (this) {
+      reply = checkLeaderState(request);
+      if (reply != null) {
+        return reply;
+      }
+
+      final RaftConfiguration current = getRaftConf();
+      // make sure there is no other raft reconfiguration in progress
+      if (!current.isStable() || leaderState.inStagingState() ||
+          !state.isCurrentConfCommitted()) {
+        throw new ReconfigurationInProgressException(
+            "Reconfiguration is already in progress: " + current);
+      }
+
+      // return true if the new configuration is the same with the current one
+      if (current.hasNoChange(peersInNewConf)) {
+        pending = leaderState.returnNoConfChange(request);
+        return pending.getFuture();
+      }
+
+      // add new peers into the rpc service
+      getServerRpc().addPeers(Arrays.asList(peersInNewConf));
+      // add staging state into the leaderState
+      pending = leaderState.startSetConfiguration(request);
+    }
+    return pending.getFuture();
+  }
+
+  private boolean shouldWithholdVotes() {
+    return isLeader() || (isFollower() && state.hasLeader()
+        && heartbeatMonitor.shouldWithholdVotes());
+  }
+
+  /**
+   * check if the remote peer is not included in the current conf
+   * and should shutdown. should shutdown if all the following stands:
+   * 1. this is a leader
+   * 2. current conf is stable and has been committed
+   * 3. candidate id is not included in conf
+   * 4. candidate's last entry's index < conf's index
+   */
+  private boolean shouldSendShutdown(String candidateId,
+      TermIndex candidateLastEntry) {
+    return isLeader()
+        && getRaftConf().isStable()
+        && getState().isConfCommitted()
+        && !getRaftConf().containsInConf(candidateId)
+        && candidateLastEntry.getIndex() < getRaftConf().getLogEntryIndex()
+        && !leaderState.isBootStrappingPeer(candidateId);
+  }
+
+  @Override
+  public RequestVoteReplyProto requestVote(RequestVoteRequestProto r)
+      throws IOException {
+    final String candidateId = r.getServerRequest().getRequestorId();
+    return requestVote(candidateId, r.getCandidateTerm(),
+        ServerProtoUtils.toTermIndex(r.getCandidateLastEntry()));
+  }
+
+  private RequestVoteReplyProto requestVote(String candidateId,
+      long candidateTerm, TermIndex candidateLastEntry) throws IOException {
+    CodeInjectionForTesting.execute(REQUEST_VOTE, getId(),
+        candidateId, candidateTerm, candidateLastEntry);
+    LOG.debug("{}: receive requestVote({}, {}, {})",
+        getId(), candidateId, candidateTerm, candidateLastEntry);
+    lifeCycle.assertCurrentState(RUNNING);
+
+    boolean voteGranted = false;
+    boolean shouldShutdown = false;
+    final RequestVoteReplyProto reply;
+    synchronized (this) {
+      if (shouldWithholdVotes()) {
+        LOG.info("{} Withhold vote from server {} with term {}. " +
+            "This server:{}, last rpc time from leader {} is {}", getId(),
+            candidateId, candidateTerm, this, this.getState().getLeaderId(),
+            (isFollower() ? heartbeatMonitor.getLastRpcTime() : -1));
+      } else if (state.recognizeCandidate(candidateId, candidateTerm)) {
+        boolean termUpdated = changeToFollower(candidateTerm, false);
+        // see Section 5.4.1 Election restriction
+        if (state.isLogUpToDate(candidateLastEntry)) {
+          heartbeatMonitor.updateLastRpcTime(false);
+          state.grantVote(candidateId);
+          voteGranted = true;
+        }
+        if (termUpdated || voteGranted) {
+          state.persistMetadata(); // sync metafile
+        }
+      }
+      if (!voteGranted && shouldSendShutdown(candidateId, candidateLastEntry)) {
+        shouldShutdown = true;
+      }
+      reply = ServerProtoUtils.toRequestVoteReplyProto(candidateId, getId(),
+          voteGranted, state.getCurrentTerm(), shouldShutdown);
+      if (LOG.isDebugEnabled()) {
+        LOG.debug("{} replies to vote request: {}. Peer's state: {}",
+            getId(), ProtoUtils.toString(reply), state);
+      }
+    }
+    return reply;
+  }
+
+  private void validateEntries(long expectedTerm, TermIndex previous,
+      LogEntryProto... entries) {
+    if (entries != null && entries.length > 0) {
+      final long index0 = entries[0].getIndex();
+
+      if (previous == null || previous.getTerm() == 0) {
+        Preconditions.checkArgument(index0 == 0,
+            "Unexpected Index: previous is null but entries[%s].getIndex()=%s",
+            0, index0);
+      } else {
+        Preconditions.checkArgument(previous.getIndex() == index0 - 1,
+            "Unexpected Index: previous is %s but entries[%s].getIndex()=%s",
+            previous, 0, index0);
+      }
+
+      for (int i = 0; i < entries.length; i++) {
+        final long t = entries[i].getTerm();
+        Preconditions.checkArgument(expectedTerm >= t,
+            "Unexpected Term: entries[%s].getTerm()=%s but expectedTerm=%s",
+            i, t, expectedTerm);
+
+        final long indexi = entries[i].getIndex();
+        Preconditions.checkArgument(indexi == index0 + i,
+            "Unexpected Index: entries[%s].getIndex()=%s but entries[0].getIndex()=%s",
+            i, indexi, index0);
+      }
+    }
+  }
+
+  @Override
+  public AppendEntriesReplyProto appendEntries(AppendEntriesRequestProto r)
+      throws IOException {
+    // TODO avoid converting list to array
+    final LogEntryProto[] entries = r.getEntriesList()
+        .toArray(new LogEntryProto[r.getEntriesCount()]);
+    final TermIndex previous = r.hasPreviousLog() ?
+        ServerProtoUtils.toTermIndex(r.getPreviousLog()) : null;
+    return appendEntries(r.getServerRequest().getRequestorId(),
+        r.getLeaderTerm(), previous, r.getLeaderCommit(), r.getInitializing(),
+        entries);
+  }
+
+  private AppendEntriesReplyProto appendEntries(String leaderId, long leaderTerm,
+      TermIndex previous, long leaderCommit, boolean initializing,
+      LogEntryProto... entries) throws IOException {
+    CodeInjectionForTesting.execute(APPEND_ENTRIES, getId(),
+        leaderId, leaderTerm, previous, leaderCommit, initializing, entries);
+    if (LOG.isDebugEnabled()) {
+      LOG.debug("{}: receive appendEntries({}, {}, {}, {}, {}, {})", getId(),
+          leaderId, leaderTerm, previous, leaderCommit, initializing,
+          ServerProtoUtils.toString(entries));
+    }
+    lifeCycle.assertCurrentState(STARTING, RUNNING);
+
+    try {
+      validateEntries(leaderTerm, previous, entries);
+    } catch (IllegalArgumentException e) {
+      throw new IOException(e);
+    }
+
+    final long currentTerm;
+    long nextIndex = state.getLog().getNextIndex();
+    synchronized (this) {
+      final boolean recognized = state.recognizeLeader(leaderId, leaderTerm);
+      currentTerm = state.getCurrentTerm();
+      if (!recognized) {
+        final AppendEntriesReplyProto reply = ServerProtoUtils.toAppendEntriesReplyProto(
+            leaderId, getId(), currentTerm, nextIndex, NOT_LEADER);
+        if (LOG.isDebugEnabled()) {
+          LOG.debug("{}: do not recognize leader. Reply: {}",
+              getId(), ProtoUtils.toString(reply));
+        }
+        return reply;
+      }
+      changeToFollower(leaderTerm, true);
+      state.setLeader(leaderId);
+
+      if (!initializing && lifeCycle.compareAndTransition(STARTING, RUNNING)) {
+        heartbeatMonitor = new FollowerState(this);
+        heartbeatMonitor.start();
+      }
+      if (lifeCycle.getCurrentState() == RUNNING) {
+        heartbeatMonitor.updateLastRpcTime(true);
+      }
+
+      // We need to check if "previous" is in the local peer. Note that it is
+      // possible that "previous" is covered by the latest snapshot: e.g.,
+      // it's possible there's no log entries outside of the latest snapshot.
+      // However, it is not possible that "previous" index is smaller than the
+      // last index included in snapshot. This is because indices <= snapshot's
+      // last index should have been committed.
+      if (previous != null && !containPrevious(previous)) {
+        final AppendEntriesReplyProto reply =
+            ServerProtoUtils.toAppendEntriesReplyProto(leaderId, getId(),
+                currentTerm, Math.min(nextIndex, previous.getIndex()), INCONSISTENCY);
+        LOG.debug("{}: inconsistency entries. Leader previous:{}, Reply:{}",
+            getId(), previous, ServerProtoUtils.toString(reply));
+        return reply;
+      }
+
+      state.getLog().append(entries);
+      state.updateConfiguration(entries);
+      state.updateStatemachine(leaderCommit, currentTerm);
+    }
+    if (entries != null && entries.length > 0) {
+      try {
+        state.getLog().logSync();
+      } catch (InterruptedException e) {
+        throw new InterruptedIOException("logSync got interrupted");
+      }
+      nextIndex = entries[entries.length - 1].getIndex() + 1;
+    }
+    synchronized (this) {
+      if (lifeCycle.getCurrentState() == RUNNING && isFollower()
+          && getState().getCurrentTerm() == currentTerm) {
+        // reset election timer to avoid punishing the leader for our own
+        // long disk writes
+        heartbeatMonitor.updateLastRpcTime(false);
+      }
+    }
+    final AppendEntriesReplyProto reply = ServerProtoUtils.toAppendEntriesReplyProto(
+        leaderId, getId(), currentTerm, nextIndex, SUCCESS);
+    LOG.debug("{}: succeeded to handle AppendEntries. Reply: {}", getId(),
+        ServerProtoUtils.toString(reply));
+    return reply;
+  }
+
+  private boolean containPrevious(TermIndex previous) {
+    LOG.debug("{}: prev:{}, latestSnapshot:{}, getLatestInstalledSnapshot:{}",
+        getId(), previous, state.getLatestSnapshot(), state.getLatestInstalledSnapshot());
+    return state.getLog().contains(previous)
+        ||  (state.getLatestSnapshot() != null
+             && state.getLatestSnapshot().getTermIndex().equals(previous))
+        || (state.getLatestInstalledSnapshot() != null)
+             && state.getLatestInstalledSnapshot().equals(previous);
+  }
+
+  @Override
+  public InstallSnapshotReplyProto installSnapshot(
+      InstallSnapshotRequestProto request) throws IOException {
+    final String leaderId = request.getServerRequest().getRequestorId();
+    CodeInjectionForTesting.execute(INSTALL_SNAPSHOT, getId(), leaderId, request);
+    LOG.debug("{}: receive installSnapshot({})", getId(), request);
+
+    lifeCycle.assertCurrentState(STARTING, RUNNING);
+
+    final long currentTerm;
+    final long leaderTerm = request.getLeaderTerm();
+    final TermIndex lastTermIndex = ServerProtoUtils.toTermIndex(
+        request.getTermIndex());
+    final long lastIncludedIndex = lastTermIndex.getIndex();
+    synchronized (this) {
+      final boolean recognized = state.recognizeLeader(leaderId, leaderTerm);
+      currentTerm = state.getCurrentTerm();
+      if (!recognized) {
+        final InstallSnapshotReplyProto reply = ServerProtoUtils
+            .toInstallSnapshotReplyProto(leaderId, getId(), currentTerm,
+                request.getRequestIndex(), InstallSnapshotResult.NOT_LEADER);
+        LOG.debug("{}: do not recognize leader for installing snapshot." +
+            " Reply: {}", getId(), reply);
+        return reply;
+      }
+      changeToFollower(leaderTerm, true);
+      state.setLeader(leaderId);
+
+      if (lifeCycle.getCurrentState() == RUNNING) {
+        heartbeatMonitor.updateLastRpcTime(true);
+      }
+
+      // Check and append the snapshot chunk. We simply put this in lock
+      // considering a follower peer requiring a snapshot installation does not
+      // have a lot of requests
+      Preconditions.checkState(
+          state.getLog().getNextIndex() <= lastIncludedIndex,
+          "%s log's next id is %s, last included index in snapshot is %s",
+          getId(),  state.getLog().getNextIndex(), lastIncludedIndex);
+
+      //TODO: We should only update State with installed snapshot once the request is done.
+      state.installSnapshot(request);
+
+      // update the committed index
+      // re-load the state machine if this is the last chunk
+      if (request.getDone()) {
+        state.reloadStateMachine(lastIncludedIndex, leaderTerm);
+      }
+      if (lifeCycle.getCurrentState() == RUNNING) {
+        heartbeatMonitor.updateLastRpcTime(false);
+      }
+    }
+    if (request.getDone()) {
+      LOG.info("{}: successfully install the whole snapshot-{}", getId(),
+          lastIncludedIndex);
+    }
+    return ServerProtoUtils.toInstallSnapshotReplyProto(leaderId, getId(),
+        currentTerm, request.getRequestIndex(), InstallSnapshotResult.SUCCESS);
+  }
+
+  AppendEntriesRequestProto createAppendEntriesRequest(long leaderTerm,
+      String targetId, TermIndex previous, List<LogEntryProto> entries,
+      boolean initializing) {
+    return ServerProtoUtils.toAppendEntriesRequestProto(getId(), targetId,
+        leaderTerm, entries, state.getLog().getLastCommittedIndex(),
+        initializing, previous);
+  }
+
+  synchronized InstallSnapshotRequestProto createInstallSnapshotRequest(
+      String targetId, String requestId, int requestIndex, SnapshotInfo snapshot,
+      List<FileChunkProto> chunks, boolean done) {
+    OptionalLong totalSize = snapshot.getFiles().stream()
+        .mapToLong(FileInfo::getFileSize).reduce(Long::sum);
+    assert totalSize.isPresent();
+    return ServerProtoUtils.toInstallSnapshotRequestProto(getId(), targetId,
+        requestId, requestIndex, state.getCurrentTerm(), snapshot.getTermIndex(),
+        chunks, totalSize.getAsLong(), done);
+  }
+
+  synchronized RequestVoteRequestProto createRequestVoteRequest(String targetId,
+      long term, TermIndex lastEntry) {
+    return ServerProtoUtils.toRequestVoteRequestProto(getId(), targetId, term,
+        lastEntry);
+  }
+
+  public synchronized void submitLocalSyncEvent() {
+    if (isLeader() && leaderState != null) {
+      leaderState.submitUpdateStateEvent(LeaderState.UPDATE_COMMIT_EVENT);
+    }
+  }
+
+  synchronized void replyPendingRequest(long logIndex,
+      CompletableFuture<Message> message) {
+    if (isLeader() && leaderState != null) { // is leader and is running
+      leaderState.replyPendingRequest(logIndex, message);
+    }
+  }
+
+  TransactionContext getTransactionContext(long index) {
+    if (leaderState != null) { // is leader and is running
+      return leaderState.getTransactionContext(index);
+    }
+    return null;
+  }
+
+  public RaftProperties getProperties() {
+    return this.properties;
+  }
+}

http://git-wip-us.apache.org/repos/asf/incubator-ratis/blob/7e71a2e0/ratis-server/src/main/java/org/apache/ratis/server/impl/ServerProtoUtils.java
----------------------------------------------------------------------
diff --git a/ratis-server/src/main/java/org/apache/ratis/server/impl/ServerProtoUtils.java b/ratis-server/src/main/java/org/apache/ratis/server/impl/ServerProtoUtils.java
new file mode 100644
index 0000000..4d4371d
--- /dev/null
+++ b/ratis-server/src/main/java/org/apache/ratis/server/impl/ServerProtoUtils.java
@@ -0,0 +1,191 @@
+/**
+ * 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.ratis.server.impl;
+
+import static org.apache.ratis.server.impl.RaftServerConstants.DEFAULT_SEQNUM;
+import static org.apache.ratis.shaded.proto.RaftProtos.AppendEntriesReplyProto.AppendResult.SUCCESS;
+
+import java.util.Arrays;
+import java.util.List;
+import java.util.stream.Collectors;
+
+import org.apache.ratis.client.impl.ClientProtoUtils;
+import org.apache.ratis.server.protocol.TermIndex;
+import org.apache.ratis.shaded.proto.RaftProtos.AppendEntriesReplyProto;
+import org.apache.ratis.shaded.proto.RaftProtos.AppendEntriesRequestProto;
+import org.apache.ratis.shaded.proto.RaftProtos.FileChunkProto;
+import org.apache.ratis.shaded.proto.RaftProtos.InstallSnapshotReplyProto;
+import org.apache.ratis.shaded.proto.RaftProtos.InstallSnapshotRequestProto;
+import org.apache.ratis.shaded.proto.RaftProtos.InstallSnapshotResult;
+import org.apache.ratis.shaded.proto.RaftProtos.LogEntryProto;
+import org.apache.ratis.shaded.proto.RaftProtos.RaftConfigurationProto;
+import org.apache.ratis.shaded.proto.RaftProtos.RaftRpcReplyProto;
+import org.apache.ratis.shaded.proto.RaftProtos.RequestVoteReplyProto;
+import org.apache.ratis.shaded.proto.RaftProtos.RequestVoteRequestProto;
+import org.apache.ratis.shaded.proto.RaftProtos.TermIndexProto;
+import org.apache.ratis.util.ProtoUtils;
+
+
+/** Server proto utilities for internal use. */
+public class ServerProtoUtils {
+  public static TermIndex toTermIndex(TermIndexProto p) {
+    return p == null? null: TermIndex.newTermIndex(p.getTerm(), p.getIndex());
+  }
+
+  public static TermIndexProto toTermIndexProto(TermIndex ti) {
+    return ti == null? null: TermIndexProto.newBuilder()
+        .setTerm(ti.getTerm())
+        .setIndex(ti.getIndex())
+        .build();
+  }
+
+  public static TermIndex toTermIndex(LogEntryProto entry) {
+    return entry == null ? null :
+        TermIndex.newTermIndex(entry.getTerm(), entry.getIndex());
+  }
+
+  public static String toString(LogEntryProto... entries) {
+    return entries == null? "null"
+        : entries.length == 0 ? "[]"
+        : entries.length == 1? "" + toTermIndex(entries[0])
+        : "" + Arrays.stream(entries).map(ServerProtoUtils::toTermIndex)
+            .collect(Collectors.toList());
+  }
+
+  public static String toString(AppendEntriesReplyProto reply) {
+    return toString(reply.getServerReply()) + "," + reply.getResult()
+        + ",nextIndex:" + reply.getNextIndex() + ",term:" + reply.getTerm();
+  }
+
+  private static String toString(RaftRpcReplyProto reply) {
+    return reply.getRequestorId() + "->" + reply.getReplyId() + ","
+        + reply.getSuccess();
+  }
+
+  public static RaftConfigurationProto toRaftConfigurationProto(
+      RaftConfiguration conf) {
+    return RaftConfigurationProto.newBuilder()
+        .addAllPeers(ProtoUtils.toRaftPeerProtos(conf.getPeersInConf()))
+        .addAllOldPeers(ProtoUtils.toRaftPeerProtos(conf.getPeersInOldConf()))
+        .build();
+  }
+
+  public static RaftConfiguration toRaftConfiguration(
+      long index, RaftConfigurationProto proto) {
+    final RaftConfiguration.Builder b = RaftConfiguration.newBuilder()
+        .setConf(ProtoUtils.toRaftPeerArray(proto.getPeersList()))
+        .setLogEntryIndex(index);
+    if (proto.getOldPeersCount() > 0) {
+      b.setOldConf(ProtoUtils.toRaftPeerArray(proto.getOldPeersList()));
+    }
+    return b.build();
+  }
+
+  public static LogEntryProto toLogEntryProto(
+      RaftConfiguration conf, long term, long index) {
+    return LogEntryProto.newBuilder()
+        .setTerm(term)
+        .setIndex(index)
+        .setConfigurationEntry(toRaftConfigurationProto(conf))
+        .build();
+  }
+
+  public static RequestVoteReplyProto toRequestVoteReplyProto(
+      String requestorId, String replyId, boolean success, long term,
+      boolean shouldShutdown) {
+    final RequestVoteReplyProto.Builder b = RequestVoteReplyProto.newBuilder();
+    b.setServerReply(ClientProtoUtils.toRaftRpcReplyProtoBuilder(requestorId, replyId,
+        DEFAULT_SEQNUM, success))
+        .setTerm(term)
+        .setShouldShutdown(shouldShutdown);
+    return b.build();
+  }
+
+  public static RequestVoteRequestProto toRequestVoteRequestProto(
+      String requestorId, String replyId, long term, TermIndex lastEntry) {
+    final RequestVoteRequestProto.Builder b = RequestVoteRequestProto.newBuilder()
+        .setServerRequest(
+            ClientProtoUtils.toRaftRpcRequestProtoBuilder(requestorId, replyId, DEFAULT_SEQNUM))
+        .setCandidateTerm(term);
+    if (lastEntry != null) {
+      b.setCandidateLastEntry(toTermIndexProto(lastEntry));
+    }
+    return b.build();
+  }
+
+  public static InstallSnapshotReplyProto toInstallSnapshotReplyProto(
+      String requestorId, String replyId, long term, int requestIndex,
+      InstallSnapshotResult result) {
+    final RaftRpcReplyProto.Builder rb = ClientProtoUtils.toRaftRpcReplyProtoBuilder(requestorId,
+        replyId, DEFAULT_SEQNUM, result == InstallSnapshotResult.SUCCESS);
+    final InstallSnapshotReplyProto.Builder builder = InstallSnapshotReplyProto
+        .newBuilder().setServerReply(rb).setTerm(term).setResult(result)
+        .setRequestIndex(requestIndex);
+    return builder.build();
+  }
+
+  public static InstallSnapshotRequestProto toInstallSnapshotRequestProto(
+      String requestorId, String replyId, String requestId, int requestIndex,
+      long term, TermIndex lastTermIndex, List<FileChunkProto> chunks,
+      long totalSize, boolean done) {
+    return InstallSnapshotRequestProto.newBuilder()
+        .setServerRequest(
+            ClientProtoUtils.toRaftRpcRequestProtoBuilder(requestorId, replyId, DEFAULT_SEQNUM))
+        .setRequestId(requestId)
+        .setRequestIndex(requestIndex)
+        // .setRaftConfiguration()  TODO: save and pass RaftConfiguration
+        .setLeaderTerm(term)
+        .setTermIndex(toTermIndexProto(lastTermIndex))
+        .addAllFileChunks(chunks)
+        .setTotalSize(totalSize)
+        .setDone(done).build();
+  }
+
+  public static AppendEntriesReplyProto toAppendEntriesReplyProto(
+      String requestorId, String replyId, long term,
+      long nextIndex, AppendEntriesReplyProto.AppendResult appendResult) {
+    RaftRpcReplyProto.Builder rb = ClientProtoUtils.toRaftRpcReplyProtoBuilder(requestorId,
+        replyId, DEFAULT_SEQNUM, appendResult == SUCCESS);
+    final AppendEntriesReplyProto.Builder b = AppendEntriesReplyProto.newBuilder();
+    b.setServerReply(rb).setTerm(term).setNextIndex(nextIndex)
+        .setResult(appendResult);
+    return b.build();
+  }
+
+  public static AppendEntriesRequestProto toAppendEntriesRequestProto(
+      String requestorId, String replyId, long leaderTerm,
+      List<LogEntryProto> entries, long leaderCommit, boolean initializing,
+      TermIndex previous) {
+    final AppendEntriesRequestProto.Builder b = AppendEntriesRequestProto
+        .newBuilder()
+        .setServerRequest(
+            ClientProtoUtils.toRaftRpcRequestProtoBuilder(requestorId, replyId, DEFAULT_SEQNUM))
+        .setLeaderTerm(leaderTerm)
+        .setLeaderCommit(leaderCommit)
+        .setInitializing(initializing);
+    if (entries != null && !entries.isEmpty()) {
+      b.addAllEntries(entries);
+    }
+
+    if (previous != null) {
+      b.setPreviousLog(toTermIndexProto(previous));
+    }
+    return b.build();
+  }
+
+}

http://git-wip-us.apache.org/repos/asf/incubator-ratis/blob/7e71a2e0/ratis-server/src/main/java/org/apache/ratis/server/impl/ServerState.java
----------------------------------------------------------------------
diff --git a/ratis-server/src/main/java/org/apache/ratis/server/impl/ServerState.java b/ratis-server/src/main/java/org/apache/ratis/server/impl/ServerState.java
new file mode 100644
index 0000000..8608fc4
--- /dev/null
+++ b/ratis-server/src/main/java/org/apache/ratis/server/impl/ServerState.java
@@ -0,0 +1,350 @@
+/**
+ * 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.ratis.server.impl;
+
+import static org.apache.ratis.server.RaftServerConfigKeys.RAFT_SERVER_USE_MEMORY_LOG_DEFAULT;
+import static org.apache.ratis.server.RaftServerConfigKeys.RAFT_SERVER_USE_MEMORY_LOG_KEY;
+
+import java.io.Closeable;
+import java.io.IOException;
+
+import org.apache.ratis.conf.RaftProperties;
+import org.apache.ratis.server.protocol.TermIndex;
+import org.apache.ratis.server.storage.MemoryRaftLog;
+import org.apache.ratis.server.storage.RaftLog;
+import org.apache.ratis.server.storage.RaftStorage;
+import org.apache.ratis.server.storage.SegmentedRaftLog;
+import org.apache.ratis.server.storage.SnapshotManager;
+import org.apache.ratis.shaded.proto.RaftProtos.InstallSnapshotRequestProto;
+import org.apache.ratis.shaded.proto.RaftProtos.LogEntryProto;
+import org.apache.ratis.statemachine.SnapshotInfo;
+import org.apache.ratis.statemachine.StateMachine;
+import org.apache.ratis.statemachine.TransactionContext;
+import org.apache.ratis.util.ProtoUtils;
+
+import com.google.common.annotations.VisibleForTesting;
+import com.google.common.base.Preconditions;
+
+/**
+ * Common states of a raft peer. Protected by RaftServer's lock.
+ */
+public class ServerState implements Closeable {
+  private final String selfId;
+  private final RaftServerImpl server;
+  /** Raft log */
+  private final RaftLog log;
+  /** Raft configuration */
+  private final ConfigurationManager configurationManager;
+  /** The thread that applies committed log entries to the state machine */
+  private final StateMachineUpdater stateMachineUpdater;
+  /** local storage for log and snapshot */
+  private final RaftStorage storage;
+  private final SnapshotManager snapshotManager;
+
+  /**
+   * Latest term server has seen. initialized to 0 on first boot, increases
+   * monotonically.
+   */
+  private long currentTerm;
+  /**
+   * The server ID of the leader for this term. Null means either there is
+   * no leader for this term yet or this server does not know who it is yet.
+   */
+  private String leaderId;
+  /**
+   * Candidate that this peer granted vote for in current term (or null if none)
+   */
+  private String votedFor;
+
+  /**
+   * Latest installed snapshot for this server. This maybe different than StateMachine's latest
+   * snapshot. Once we successfully install a snapshot, the SM may not pick it up immediately.
+   * Further, this will not get updated when SM does snapshots itself.
+   */
+  private TermIndex latestInstalledSnapshot;
+
+  ServerState(String id, RaftConfiguration conf, RaftProperties prop,
+              RaftServerImpl server, StateMachine stateMachine) throws IOException {
+    this.selfId = id;
+    this.server = server;
+    configurationManager = new ConfigurationManager(conf);
+    storage = new RaftStorage(prop, RaftServerConstants.StartupOption.REGULAR);
+    snapshotManager = new SnapshotManager(storage, id);
+
+    long lastApplied = initStatemachine(stateMachine, prop);
+
+    leaderId = null;
+    log = initLog(id, prop, server, lastApplied);
+    RaftLog.Metadata metadata = log.loadMetadata();
+    currentTerm = metadata.getTerm();
+    votedFor = metadata.getVotedFor();
+
+    stateMachineUpdater = new StateMachineUpdater(stateMachine, server, log,
+         lastApplied, prop);
+  }
+
+  /**
+   * Used by tests to set initial raft configuration with correct port bindings.
+   */
+  @VisibleForTesting
+  public void setInitialConf(RaftConfiguration initialConf) {
+    configurationManager.setInitialConf(initialConf);
+  }
+
+  private long initStatemachine(StateMachine sm, RaftProperties properties)
+      throws IOException {
+    sm.initialize(selfId, properties, storage);
+    storage.setStateMachineStorage(sm.getStateMachineStorage());
+    SnapshotInfo snapshot = sm.getLatestSnapshot();
+
+    if (snapshot == null || snapshot.getTermIndex().getIndex() < 0) {
+      return RaftServerConstants.INVALID_LOG_INDEX;
+    }
+
+    // get the raft configuration from the snapshot
+    RaftConfiguration raftConf = sm.getRaftConfiguration();
+    if (raftConf != null) {
+      configurationManager.addConfiguration(raftConf.getLogEntryIndex(),
+          raftConf);
+    }
+    return snapshot.getIndex();
+  }
+
+  void start() {
+    stateMachineUpdater.start();
+  }
+
+  /**
+   * note we do not apply log entries to the state machine here since we do not
+   * know whether they have been committed.
+   */
+  private RaftLog initLog(String id, RaftProperties prop, RaftServerImpl server,
+      long lastIndexInSnapshot) throws IOException {
+    final RaftLog log;
+    if (prop.getBoolean(RAFT_SERVER_USE_MEMORY_LOG_KEY,
+        RAFT_SERVER_USE_MEMORY_LOG_DEFAULT)) {
+      log = new MemoryRaftLog(id);
+    } else {
+      log = new SegmentedRaftLog(id, server, this.storage,
+          lastIndexInSnapshot, prop);
+    }
+    log.open(configurationManager, lastIndexInSnapshot);
+    return log;
+  }
+
+  public RaftConfiguration getRaftConf() {
+    return configurationManager.getCurrent();
+  }
+
+  @VisibleForTesting
+
+  public String getSelfId() {
+    return this.selfId;
+  }
+
+  public long getCurrentTerm() {
+    return currentTerm;
+  }
+
+  void setCurrentTerm(long term) {
+    currentTerm = term;
+  }
+
+  String getLeaderId() {
+    return leaderId;
+  }
+
+  boolean hasLeader() {
+    return leaderId != null;
+  }
+
+  /**
+   * Become a candidate and start leader election
+   */
+  long initElection() {
+    votedFor = selfId;
+    leaderId = null;
+    return ++currentTerm;
+  }
+
+  void persistMetadata() throws IOException {
+    this.log.writeMetadata(currentTerm, votedFor);
+  }
+
+  void resetLeaderAndVotedFor() {
+    votedFor = null;
+    leaderId = null;
+  }
+
+  /**
+   * Vote for a candidate and update the local state.
+   */
+  void grantVote(String candidateId) {
+    votedFor = candidateId;
+    leaderId = null;
+  }
+
+  void setLeader(String leaderId) {
+    this.leaderId = leaderId;
+  }
+
+  void becomeLeader() {
+    leaderId = selfId;
+  }
+
+  public RaftLog getLog() {
+    return log;
+  }
+
+  long applyLog(TransactionContext operation) throws IOException {
+    return log.append(currentTerm, operation);
+  }
+
+  /**
+   * Check if accept the leader selfId and term from the incoming AppendEntries rpc.
+   * If accept, update the current state.
+   * @return true if the check passes
+   */
+  boolean recognizeLeader(String leaderId, long leaderTerm) {
+    if (leaderTerm < currentTerm) {
+      return false;
+    } else if (leaderTerm > currentTerm || this.leaderId == null) {
+      // If the request indicates a term that is greater than the current term
+      // or no leader has been set for the current term, make sure to update
+      // leader and term later
+      return true;
+    }
+    Preconditions.checkArgument(this.leaderId.equals(leaderId),
+        "selfId:%s, this.leaderId:%s, received leaderId:%s",
+        selfId, this.leaderId, leaderId);
+    return true;
+  }
+
+  /**
+   * Check if the candidate's term is acceptable
+   */
+  boolean recognizeCandidate(String candidateId,
+      long candidateTerm) {
+    if (candidateTerm > currentTerm) {
+      return true;
+    } else if (candidateTerm == currentTerm) {
+      // has not voted yet or this is a retry
+      return votedFor == null || votedFor.equals(candidateId);
+    }
+    return false;
+  }
+
+  boolean isLogUpToDate(TermIndex candidateLastEntry) {
+    LogEntryProto lastEntry = log.getLastEntry();
+    // need to take into account snapshot
+    SnapshotInfo snapshot = server.getStateMachine().getLatestSnapshot();
+     if (lastEntry == null && snapshot == null) {
+      return true;
+    } else if (candidateLastEntry == null) {
+      return false;
+    }
+    TermIndex local = ServerProtoUtils.toTermIndex(lastEntry);
+    if (local == null || (snapshot != null && snapshot.getIndex() > lastEntry.getIndex())) {
+      local = snapshot.getTermIndex();
+    }
+    return local.compareTo(candidateLastEntry) <= 0;
+  }
+
+  @Override
+  public String toString() {
+    return selfId + ":t" + currentTerm + ", leader=" + leaderId
+        + ", voted=" + votedFor + ", raftlog=" + log + ", conf=" + getRaftConf();
+  }
+
+  boolean isConfCommitted() {
+    return getLog().getLastCommittedIndex() >=
+        getRaftConf().getLogEntryIndex();
+  }
+
+  public void setRaftConf(long logIndex, RaftConfiguration conf) {
+    configurationManager.addConfiguration(logIndex, conf);
+    RaftServerImpl.LOG.info("{}: successfully update the configuration {}",
+        getSelfId(), conf);
+  }
+
+  void updateConfiguration(LogEntryProto[] entries) {
+    if (entries != null && entries.length > 0) {
+      configurationManager.removeConfigurations(entries[0].getIndex());
+      for (LogEntryProto entry : entries) {
+        if (ProtoUtils.isConfigurationLogEntry(entry)) {
+          final RaftConfiguration conf = ServerProtoUtils.toRaftConfiguration(
+              entry.getIndex(), entry.getConfigurationEntry());
+          configurationManager.addConfiguration(entry.getIndex(), conf);
+          server.getServerRpc().addPeers(conf.getPeers());
+        }
+      }
+    }
+  }
+
+  void updateStatemachine(long majorityIndex, long currentTerm) {
+    log.updateLastCommitted(majorityIndex, currentTerm);
+    stateMachineUpdater.notifyUpdater();
+  }
+
+  void reloadStateMachine(long lastIndexInSnapshot, long currentTerm)
+      throws IOException {
+    log.updateLastCommitted(lastIndexInSnapshot, currentTerm);
+
+    stateMachineUpdater.reloadStateMachine();
+  }
+
+  @Override
+  public void close() throws IOException {
+    stateMachineUpdater.stop();
+    RaftServerImpl.LOG.info("{} closes. The last applied log index is {}",
+        getSelfId(), getLastAppliedIndex());
+    storage.close();
+  }
+
+  @VisibleForTesting
+  public RaftStorage getStorage() {
+    return storage;
+  }
+
+  void installSnapshot(InstallSnapshotRequestProto request) throws IOException {
+    // TODO: verify that we need to install the snapshot
+    StateMachine sm = server.getStateMachine();
+    sm.pause(); // pause the SM to prepare for install snapshot
+    snapshotManager.installSnapshot(sm, request);
+    log.syncWithSnapshot(request.getTermIndex().getIndex());
+    this.latestInstalledSnapshot = ServerProtoUtils.toTermIndex(
+        request.getTermIndex());
+  }
+
+  SnapshotInfo getLatestSnapshot() {
+    return server.getStateMachine().getStateMachineStorage().getLatestSnapshot();
+  }
+
+  public TermIndex getLatestInstalledSnapshot() {
+    return latestInstalledSnapshot;
+  }
+
+  @VisibleForTesting
+  public long getLastAppliedIndex() {
+    return stateMachineUpdater.getLastAppliedIndex();
+  }
+
+  boolean isCurrentConfCommitted() {
+    return getRaftConf().getLogEntryIndex() <= getLog().getLastCommittedIndex();
+  }
+}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/incubator-ratis/blob/7e71a2e0/ratis-server/src/main/java/org/apache/ratis/server/impl/ServerUtils.java
----------------------------------------------------------------------
diff --git a/ratis-server/src/main/java/org/apache/ratis/server/impl/ServerUtils.java b/ratis-server/src/main/java/org/apache/ratis/server/impl/ServerUtils.java
new file mode 100644
index 0000000..3dc9ab4
--- /dev/null
+++ b/ratis-server/src/main/java/org/apache/ratis/server/impl/ServerUtils.java
@@ -0,0 +1,81 @@
+/**
+ * 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.ratis.server.impl;
+
+import org.apache.commons.lang.builder.HashCodeBuilder;
+import org.apache.ratis.server.protocol.TermIndex;
+
+/** Server utilities for internal use. */
+public class ServerUtils {
+  public static TermIndex newTermIndex(long term, long index) {
+    return new TermIndexImpl(term, index);
+  }
+
+  private static class TermIndexImpl implements TermIndex {
+    private final long term;
+    private final long index; //log index; first index is 1.
+
+    TermIndexImpl(long term, long logIndex) {
+      this.term = term;
+      this.index = logIndex;
+    }
+
+    @Override
+    public long getTerm() {
+      return term;
+    }
+
+    @Override
+    public long getIndex() {
+      return index;
+    }
+
+    @Override
+    public int compareTo(TermIndex that) {
+      final int d = Long.compare(this.getTerm(), that.getTerm());
+      return d != 0 ? d : Long.compare(this.getIndex(), that.getIndex());
+    }
+
+    @Override
+    public boolean equals(Object obj) {
+      if (obj == this) {
+        return true;
+      } else if (obj == null || !(obj instanceof TermIndexImpl)) {
+        return false;
+      }
+
+      final TermIndexImpl that = (TermIndexImpl) obj;
+      return this.getTerm() == that.getTerm()
+          && this.getIndex() == that.getIndex();
+    }
+
+    @Override
+    public int hashCode() {
+      return new HashCodeBuilder().append(term).append(index).hashCode();
+    }
+
+    private static String toString(long n) {
+      return n < 0 ? "~" : "" + n;
+    }
+
+    @Override
+    public String toString() {
+      return "(t:" + toString(term) + ", i:" + toString(index) + ")";
+    }
+  }
+}

http://git-wip-us.apache.org/repos/asf/incubator-ratis/blob/7e71a2e0/ratis-server/src/main/java/org/apache/ratis/server/impl/StateMachineUpdater.java
----------------------------------------------------------------------
diff --git a/ratis-server/src/main/java/org/apache/ratis/server/impl/StateMachineUpdater.java b/ratis-server/src/main/java/org/apache/ratis/server/impl/StateMachineUpdater.java
new file mode 100644
index 0000000..a2a4149
--- /dev/null
+++ b/ratis-server/src/main/java/org/apache/ratis/server/impl/StateMachineUpdater.java
@@ -0,0 +1,214 @@
+/**
+ * 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.ratis.server.impl;
+
+import static org.apache.ratis.shaded.proto.RaftProtos.LogEntryProto.LogEntryBodyCase.CONFIGURATIONENTRY;
+import static org.apache.ratis.shaded.proto.RaftProtos.LogEntryProto.LogEntryBodyCase.SMLOGENTRY;
+
+import java.io.IOException;
+import java.util.concurrent.CompletableFuture;
+
+import org.apache.ratis.conf.RaftProperties;
+import org.apache.ratis.protocol.Message;
+import org.apache.ratis.server.RaftServerConfigKeys;
+import org.apache.ratis.server.storage.RaftLog;
+import org.apache.ratis.server.storage.RaftStorage;
+import org.apache.ratis.shaded.proto.RaftProtos.LogEntryProto;
+import org.apache.ratis.statemachine.SnapshotInfo;
+import org.apache.ratis.statemachine.StateMachine;
+import org.apache.ratis.statemachine.TransactionContext;
+import org.apache.ratis.util.Daemon;
+import org.apache.ratis.util.ExitUtils;
+import org.apache.ratis.util.LifeCycle;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import com.google.common.base.Preconditions;
+
+/**
+ * This class tracks the log entries that have been committed in a quorum and
+ * applies them to the state machine. We let a separate thread do this work
+ * asynchronously so that this will not block normal raft protocol.
+ *
+ * If the auto log compaction is enabled, the state machine updater thread will
+ * trigger a snapshot of the state machine by calling
+ * {@link StateMachine#takeSnapshot} when the log size exceeds a limit.
+ */
+class StateMachineUpdater implements Runnable {
+  static final Logger LOG = LoggerFactory.getLogger(StateMachineUpdater.class);
+
+  enum State {
+    RUNNING, STOP, RELOAD
+  }
+
+  private final RaftProperties properties;
+  private final StateMachine stateMachine;
+  private final RaftServerImpl server;
+  private final RaftLog raftLog;
+
+  private volatile long lastAppliedIndex;
+
+  private final boolean autoSnapshotEnabled;
+  private final long snapshotThreshold;
+  private long lastSnapshotIndex;
+
+  private final Thread updater;
+  private volatile State state = State.RUNNING;
+
+  StateMachineUpdater(StateMachine stateMachine, RaftServerImpl server,
+      RaftLog raftLog, long lastAppliedIndex, RaftProperties properties) {
+    this.properties = properties;
+    this.stateMachine = stateMachine;
+    this.server = server;
+    this.raftLog = raftLog;
+
+    this.lastAppliedIndex = lastAppliedIndex;
+    lastSnapshotIndex = lastAppliedIndex;
+
+    autoSnapshotEnabled = properties.getBoolean(
+        RaftServerConfigKeys.RAFT_SERVER_AUTO_SNAPSHOT_ENABLED_KEY,
+        RaftServerConfigKeys.RAFT_SERVER_AUTO_SNAPSHOT_ENABLED_DEFAULT);
+    snapshotThreshold = properties.getLong(
+        RaftServerConfigKeys.RAFT_SERVER_SNAPSHOT_TRIGGER_THRESHOLD_KEY,
+        RaftServerConfigKeys.RAFT_SERVER_SNAPSHOT_TRIGGER_THRESHOLD_DEFAULT);
+    updater = new Daemon(this);
+  }
+
+  void start() {
+    updater.start();
+  }
+
+  void stop() {
+    state = State.STOP;
+    updater.interrupt();
+    try {
+      stateMachine.close();
+    } catch (IOException ignored) {
+    }
+  }
+
+  void reloadStateMachine() {
+    state = State.RELOAD;
+    notifyUpdater();
+  }
+
+  synchronized void notifyUpdater() {
+    notifyAll();
+  }
+
+  @Override
+  public String toString() {
+    return this.getClass().getSimpleName() + "-" + raftLog.getSelfId();
+  }
+
+  @Override
+  public void run() {
+    final RaftStorage storage = server.getState().getStorage();
+    while (isRunning()) {
+      try {
+        synchronized (this) {
+          // when the peers just start, the committedIndex is initialized as 0
+          // and will be updated only after the leader contacts other peers.
+          // Thus initially lastAppliedIndex can be greater than lastCommitted.
+          while (lastAppliedIndex >= raftLog.getLastCommittedIndex()) {
+            wait();
+          }
+        }
+
+        final long committedIndex = raftLog.getLastCommittedIndex();
+        Preconditions.checkState(lastAppliedIndex < committedIndex);
+
+        if (state == State.RELOAD) {
+          Preconditions.checkState(stateMachine.getLifeCycleState() == LifeCycle.State.PAUSED);
+
+          stateMachine.reinitialize(server.getId(), properties, storage);
+
+          SnapshotInfo snapshot = stateMachine.getLatestSnapshot();
+          Preconditions.checkState(snapshot != null && snapshot.getIndex() > lastAppliedIndex,
+              "Snapshot: %s, lastAppliedIndex: %s", snapshot, lastAppliedIndex);
+
+          lastAppliedIndex = snapshot.getIndex();
+          lastSnapshotIndex = snapshot.getIndex();
+          state = State.RUNNING;
+        }
+
+        while (lastAppliedIndex < committedIndex) {
+          final LogEntryProto next = raftLog.get(lastAppliedIndex + 1);
+          if (next != null) {
+            if (next.getLogEntryBodyCase() == CONFIGURATIONENTRY) {
+              // the reply should have already been set. only need to record
+              // the new conf in the state machine.
+              stateMachine.setRaftConfiguration(
+                  ServerProtoUtils.toRaftConfiguration(next.getIndex(),
+                      next.getConfigurationEntry()));
+            } else if (next.getLogEntryBodyCase() == SMLOGENTRY) {
+              // check whether there is a TransactionContext because we are the leader.
+              TransactionContext trx = server.getTransactionContext(next.getIndex());
+              if (trx == null) {
+                trx = new TransactionContext(stateMachine, next);
+              }
+
+              // Let the StateMachine inject logic for committed transactions in sequential order.
+              trx = stateMachine.applyTransactionSerial(trx);
+
+              // TODO: This step can be parallelized
+              CompletableFuture<Message> messageFuture =
+                  stateMachine.applyTransaction(trx);
+              server.replyPendingRequest(next.getIndex(), messageFuture);
+            }
+            lastAppliedIndex++;
+          } else {
+            LOG.debug("{}: logEntry {} is null. There may be snapshot to load. state:{}",
+                this, lastAppliedIndex + 1, state);
+            break;
+          }
+        }
+
+        // check if need to trigger a snapshot
+        if (shouldTakeSnapshot(lastAppliedIndex)) {
+          stateMachine.takeSnapshot();
+          // TODO purge logs, including log cache. but should keep log for leader's RPCSenders
+          lastSnapshotIndex = lastAppliedIndex;
+        }
+      } catch (InterruptedException e) {
+        if (!isRunning()) {
+          LOG.info("{}: the StateMachineUpdater is interrupted and will exit.", this);
+        } else {
+          final String s = this + ": the StateMachineUpdater is wrongly interrupted";
+          ExitUtils.terminate(1, s, e, LOG);
+        }
+      } catch (Throwable t) {
+        final String s = this + ": the StateMachineUpdater hits Throwable";
+        ExitUtils.terminate(2, s, t, LOG);
+      }
+    }
+  }
+
+  private boolean isRunning() {
+    return state != State.STOP;
+  }
+
+  private boolean shouldTakeSnapshot(long currentAppliedIndex) {
+    return autoSnapshotEnabled && (state != State.RELOAD) &&
+        (currentAppliedIndex - lastSnapshotIndex >= snapshotThreshold);
+  }
+
+  long getLastAppliedIndex() {
+    return lastAppliedIndex;
+  }
+}

http://git-wip-us.apache.org/repos/asf/incubator-ratis/blob/7e71a2e0/ratis-server/src/main/java/org/apache/ratis/server/protocol/RaftServerProtocol.java
----------------------------------------------------------------------
diff --git a/ratis-server/src/main/java/org/apache/ratis/server/protocol/RaftServerProtocol.java b/ratis-server/src/main/java/org/apache/ratis/server/protocol/RaftServerProtocol.java
new file mode 100644
index 0000000..f1f5512
--- /dev/null
+++ b/ratis-server/src/main/java/org/apache/ratis/server/protocol/RaftServerProtocol.java
@@ -0,0 +1,36 @@
+/**
+ * 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.ratis.server.protocol;
+
+import java.io.IOException;
+
+import org.apache.ratis.shaded.proto.RaftProtos.AppendEntriesReplyProto;
+import org.apache.ratis.shaded.proto.RaftProtos.AppendEntriesRequestProto;
+import org.apache.ratis.shaded.proto.RaftProtos.InstallSnapshotReplyProto;
+import org.apache.ratis.shaded.proto.RaftProtos.InstallSnapshotRequestProto;
+import org.apache.ratis.shaded.proto.RaftProtos.RequestVoteReplyProto;
+import org.apache.ratis.shaded.proto.RaftProtos.RequestVoteRequestProto;
+
+public interface RaftServerProtocol {
+
+  RequestVoteReplyProto requestVote(RequestVoteRequestProto request) throws IOException;
+
+  AppendEntriesReplyProto appendEntries(AppendEntriesRequestProto request) throws IOException;
+
+  InstallSnapshotReplyProto installSnapshot(InstallSnapshotRequestProto request) throws IOException;
+}

http://git-wip-us.apache.org/repos/asf/incubator-ratis/blob/7e71a2e0/ratis-server/src/main/java/org/apache/ratis/server/protocol/TermIndex.java
----------------------------------------------------------------------
diff --git a/ratis-server/src/main/java/org/apache/ratis/server/protocol/TermIndex.java b/ratis-server/src/main/java/org/apache/ratis/server/protocol/TermIndex.java
new file mode 100644
index 0000000..665f5d5
--- /dev/null
+++ b/ratis-server/src/main/java/org/apache/ratis/server/protocol/TermIndex.java
@@ -0,0 +1,36 @@
+/**
+ * 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.ratis.server.protocol;
+
+import org.apache.ratis.server.impl.ServerUtils;
+
+/** The term and the log index defined in the Raft consensus algorithm. */
+public interface TermIndex extends Comparable<TermIndex> {
+  /** @return the term. */
+  long getTerm();
+
+  /** @return the index. */
+  long getIndex();
+
+  /** Create a new {@link TermIndex} instance. */
+  static TermIndex newTermIndex(long term, long index) {
+    return ServerUtils.newTermIndex(term, index);
+  }
+}
+
+

http://git-wip-us.apache.org/repos/asf/incubator-ratis/blob/7e71a2e0/ratis-server/src/main/java/org/apache/ratis/server/storage/BufferedChannelBase.java
----------------------------------------------------------------------
diff --git a/ratis-server/src/main/java/org/apache/ratis/server/storage/BufferedChannelBase.java b/ratis-server/src/main/java/org/apache/ratis/server/storage/BufferedChannelBase.java
new file mode 100644
index 0000000..0dfc6f0
--- /dev/null
+++ b/ratis-server/src/main/java/org/apache/ratis/server/storage/BufferedChannelBase.java
@@ -0,0 +1,52 @@
+/**
+ * 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.ratis.server.storage;
+
+import java.io.Closeable;
+import java.io.IOException;
+import java.nio.channels.FileChannel;
+
+public abstract class BufferedChannelBase implements Closeable {
+  protected final FileChannel fileChannel;
+
+  protected BufferedChannelBase(FileChannel fc) {
+    this.fileChannel = fc;
+  }
+
+  protected FileChannel validateAndGetFileChannel() throws IOException {
+    if (!fileChannel.isOpen()) {
+      throw new IOException(
+          "Attempting to access a file channel that has already been closed");
+    }
+    return fileChannel;
+  }
+
+  /**
+   * Get the current size of the underlying FileChannel.
+   */
+  public long size() throws IOException {
+    return validateAndGetFileChannel().size();
+  }
+
+  /**
+   * Get the {@link FileChannel} that this BufferedChannel wraps around.
+   */
+  public FileChannel getFileChannel() {
+    return fileChannel;
+  }
+}

http://git-wip-us.apache.org/repos/asf/incubator-ratis/blob/7e71a2e0/ratis-server/src/main/java/org/apache/ratis/server/storage/BufferedWriteChannel.java
----------------------------------------------------------------------
diff --git a/ratis-server/src/main/java/org/apache/ratis/server/storage/BufferedWriteChannel.java b/ratis-server/src/main/java/org/apache/ratis/server/storage/BufferedWriteChannel.java
new file mode 100644
index 0000000..e61a4d3
--- /dev/null
+++ b/ratis-server/src/main/java/org/apache/ratis/server/storage/BufferedWriteChannel.java
@@ -0,0 +1,159 @@
+/**
+ * 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.ratis.server.storage;
+
+import java.io.IOException;
+import java.nio.ByteBuffer;
+import java.nio.channels.FileChannel;
+import java.util.concurrent.atomic.AtomicLong;
+
+/**
+ * Provides a buffering layer in front of a FileChannel for writing.
+ */
+public class BufferedWriteChannel extends BufferedChannelBase {
+  // The capacity of the write buffer.
+  private final int writeCapacity;
+  // The position of the file channel's write pointer.
+  private AtomicLong writeBufferStartPosition = new AtomicLong(0);
+  // The buffer used to write operations.
+  private final ByteBuffer writeBuffer;
+  // The absolute position of the next write operation.
+  private volatile long position;
+
+  public BufferedWriteChannel(FileChannel fc, int writeCapacity)
+      throws IOException {
+    super(fc);
+    this.writeCapacity = writeCapacity;
+    this.position = fc.position();
+    this.writeBufferStartPosition.set(position);
+    this.writeBuffer = ByteBuffer.allocateDirect(writeCapacity);
+  }
+
+  /**
+   * Write all the data in src to the {@link FileChannel}. Note that this function can
+   * buffer or re-order writes based on the implementation. These writes will be flushed
+   * to the disk only when flush() is invoked.
+   *
+   * @param src The source ByteBuffer which contains the data to be written.
+   * @throws IOException if a write operation fails.
+   */
+  public void write(ByteBuffer src) throws IOException {
+    int copied = 0;
+    while (src.remaining() > 0) {
+      int truncated = 0;
+      if (writeBuffer.remaining() < src.remaining()) {
+        truncated = src.remaining() - writeBuffer.remaining();
+        src.limit(src.limit() - truncated);
+      }
+      copied += src.remaining();
+      writeBuffer.put(src);
+      src.limit(src.limit() + truncated);
+      // if we have run out of buffer space, we should flush to the file
+      if (writeBuffer.remaining() == 0) {
+        flushInternal();
+      }
+    }
+    position += copied;
+  }
+
+  /**
+   * Write the specified byte.
+   * @param b the byte to be written
+   */
+  public void write(int b) throws IOException {
+    writeBuffer.put((byte) b);
+    if (writeBuffer.remaining() == 0) {
+      flushInternal();
+    }
+    position++;
+  }
+
+  public void write(byte[] b) throws IOException {
+    int offset = 0;
+    while (offset < b.length) {
+      int toPut = Math.min(b.length - offset, writeBuffer.remaining());
+      writeBuffer.put(b, offset, toPut);
+      offset += toPut;
+      if (writeBuffer.remaining() == 0) {
+        flushInternal();
+      }
+    }
+    position += b.length;
+  }
+
+  /**
+   * Get the position where the next write operation will begin writing from.
+   */
+  public long position() {
+    return position;
+  }
+
+  /**
+   * Get the position of the file channel's write pointer.
+   */
+  public long getFileChannelPosition() {
+    return writeBufferStartPosition.get();
+  }
+
+
+  /**
+   * Write any data in the buffer to the file. If sync is set to true, force a
+   * sync operation so that data is persisted to the disk.
+   *
+   * @throws IOException if the write or sync operation fails.
+   */
+  public void flush(boolean shouldForceWrite) throws IOException {
+    synchronized (this) {
+      flushInternal();
+    }
+    if (shouldForceWrite) {
+      forceWrite(false);
+    }
+  }
+
+  /**
+   * Write any data in the buffer to the file and advance the writeBufferPosition
+   * Callers are expected to synchronize appropriately
+   *
+   * @throws IOException if the write fails.
+   */
+  private void flushInternal() throws IOException {
+    writeBuffer.flip();
+    do {
+      fileChannel.write(writeBuffer);
+    } while (writeBuffer.hasRemaining());
+    writeBuffer.clear();
+    writeBufferStartPosition.set(fileChannel.position());
+  }
+
+  public long forceWrite(boolean forceMetadata) throws IOException {
+    // This is the point up to which we had flushed to the file system page cache
+    // before issuing this force write hence is guaranteed to be made durable by
+    // the force write, any flush that happens after this may or may
+    // not be flushed
+    long positionForceWrite = writeBufferStartPosition.get();
+    fileChannel.force(forceMetadata);
+    return positionForceWrite;
+  }
+
+  @Override
+  public void close() throws IOException {
+    fileChannel.close();
+    writeBuffer.clear();
+  }
+}

http://git-wip-us.apache.org/repos/asf/incubator-ratis/blob/7e71a2e0/ratis-server/src/main/java/org/apache/ratis/server/storage/FileInfo.java
----------------------------------------------------------------------
diff --git a/ratis-server/src/main/java/org/apache/ratis/server/storage/FileInfo.java b/ratis-server/src/main/java/org/apache/ratis/server/storage/FileInfo.java
new file mode 100644
index 0000000..9673596
--- /dev/null
+++ b/ratis-server/src/main/java/org/apache/ratis/server/storage/FileInfo.java
@@ -0,0 +1,59 @@
+/**
+ * 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.ratis.server.storage;
+
+import java.nio.file.Path;
+
+import org.apache.ratis.io.MD5Hash;
+
+/**
+ * Metadata about a file.
+ *
+ * The objects of this class are immutable.
+ */
+public class FileInfo {
+  private final Path path;
+  private final MD5Hash fileDigest;
+  private final long fileSize;
+
+  public FileInfo(Path path, MD5Hash fileDigest) {
+    this.path = path;
+    this.fileDigest = fileDigest;
+    this.fileSize = path.toFile().length();
+  }
+
+  @Override
+  public String toString() {
+    return path.toString();
+  }
+
+  /** @return the path of the file. */
+  public Path getPath() {
+    return path;
+  }
+
+  /** @return the MD5 file digest of the file. */
+  public MD5Hash getFileDigest() {
+    return fileDigest;
+  }
+
+  /** @return the size of the file. */
+  public long getFileSize() {
+    return fileSize;
+  }
+}

http://git-wip-us.apache.org/repos/asf/incubator-ratis/blob/7e71a2e0/ratis-server/src/main/java/org/apache/ratis/server/storage/LogInputStream.java
----------------------------------------------------------------------
diff --git a/ratis-server/src/main/java/org/apache/ratis/server/storage/LogInputStream.java b/ratis-server/src/main/java/org/apache/ratis/server/storage/LogInputStream.java
new file mode 100644
index 0000000..60572c6
--- /dev/null
+++ b/ratis-server/src/main/java/org/apache/ratis/server/storage/LogInputStream.java
@@ -0,0 +1,259 @@
+/**
+ * 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.ratis.server.storage;
+
+import static org.apache.ratis.server.impl.RaftServerConstants.INVALID_LOG_INDEX;
+
+import java.io.Closeable;
+import java.io.EOFException;
+import java.io.File;
+import java.io.IOException;
+
+import org.apache.ratis.shaded.proto.RaftProtos.LogEntryProto;
+import org.apache.ratis.util.RaftUtils;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import com.google.common.base.Preconditions;
+import com.google.common.base.Throwables;
+
+public class LogInputStream implements Closeable {
+  static final Logger LOG = LoggerFactory.getLogger(LogInputStream.class);
+
+  static class LogValidation {
+    private final long validLength;
+    private final long endIndex;
+    private final boolean hasCorruptHeader;
+
+    LogValidation(long validLength, long endIndex, boolean hasCorruptHeader) {
+      this.validLength = validLength;
+      this.endIndex = endIndex;
+      this.hasCorruptHeader = hasCorruptHeader;
+    }
+
+    long getValidLength() {
+      return validLength;
+    }
+
+    long getEndIndex() {
+      return endIndex;
+    }
+
+    boolean hasCorruptHeader() {
+      return hasCorruptHeader;
+    }
+  }
+
+  private enum State {
+    UNINIT,
+    OPEN,
+    CLOSED
+  }
+
+  private final File logFile;
+  private final long startIndex;
+  private final long endIndex;
+  private final boolean isOpen;
+  private State state = State.UNINIT;
+  private LogReader reader;
+
+  public LogInputStream(File log, long startIndex, long endIndex,
+      boolean isOpen) {
+    if (isOpen) {
+      Preconditions.checkArgument(endIndex == INVALID_LOG_INDEX);
+    } else {
+      Preconditions.checkArgument(endIndex >= startIndex);
+    }
+
+    this.logFile = log;
+    this.startIndex = startIndex;
+    this.endIndex = endIndex;
+    this.isOpen = isOpen;
+  }
+
+  private void init() throws IOException {
+    Preconditions.checkState(state == State.UNINIT);
+    try {
+      reader = new LogReader(logFile);
+      // read the log header
+      String header = reader.readLogHeader();
+      Preconditions.checkState(SegmentedRaftLog.HEADER_STR.equals(header),
+          "Corrupted log header: %s", header);
+      state = State.OPEN;
+    } finally {
+      if (reader == null) {
+        state = State.CLOSED;
+      }
+    }
+  }
+
+  long getStartIndex() {
+    return startIndex;
+  }
+
+  long getEndIndex() {
+    return endIndex;
+  }
+
+  String getName() {
+    return logFile.getName();
+  }
+
+  public LogEntryProto nextEntry() throws IOException {
+    LogEntryProto entry = null;
+    switch (state) {
+      case UNINIT:
+        try {
+          init();
+        } catch (Throwable e) {
+          LOG.error("caught exception initializing " + this, e);
+          Throwables.propagateIfPossible(e, IOException.class);
+        }
+        Preconditions.checkState(state != State.UNINIT);
+        return nextEntry();
+      case OPEN:
+        entry = reader.readEntry();
+        if (entry != null) {
+          long index = entry.getIndex();
+          if (!isOpen() && index >= endIndex) {
+            /**
+             * The end index may be derived from the segment recovery
+             * process. It is possible that we still have some uncleaned garbage
+             * in the end. We should skip them.
+             */
+            long skipAmt = logFile.length() - reader.getPos();
+            if (skipAmt > 0) {
+              LOG.debug("skipping {} bytes at the end of log '{}': reached" +
+                  " entry {} out of {}", skipAmt, getName(), index, endIndex);
+              reader.skipFully(skipAmt);
+            }
+          }
+        }
+        break;
+      case CLOSED:
+        break; // return null
+    }
+    return entry;
+  }
+
+  long scanNextEntry() throws IOException {
+    Preconditions.checkState(state == State.OPEN);
+    return reader.scanEntry();
+  }
+
+  long getPosition() {
+    if (state == State.OPEN) {
+      return reader.getPos();
+    } else {
+      return 0;
+    }
+  }
+
+  @Override
+  public void close() throws IOException {
+    if (state == State.OPEN) {
+      reader.close();
+    }
+    state = State.CLOSED;
+  }
+
+  boolean isOpen() {
+    return isOpen;
+  }
+
+  @Override
+  public String toString() {
+    return getName();
+  }
+
+  /**
+   * @param file          File being scanned and validated.
+   * @param maxTxIdToScan Maximum Tx ID to try to scan.
+   *                      The scan returns after reading this or a higher
+   *                      ID. The file portion beyond this ID is
+   *                      potentially being updated.
+   * @return Result of the validation
+   * @throws IOException
+   */
+  static LogValidation scanEditLog(File file, long maxTxIdToScan)
+      throws IOException {
+    LogInputStream in;
+    try {
+      in = new LogInputStream(file, INVALID_LOG_INDEX, INVALID_LOG_INDEX, false);
+      // read the header, initialize the inputstream
+      in.init();
+    } catch (EOFException e) {
+      LOG.warn("Log file " + file + " has no valid header", e);
+      return new LogValidation(0, INVALID_LOG_INDEX, true);
+    }
+
+    try {
+      return scanEditLog(in, maxTxIdToScan);
+    } finally {
+      RaftUtils.cleanup(LOG, in);
+    }
+  }
+
+  /**
+   * Find the last valid entry index in the stream.
+   * If there are invalid or corrupt entries in the middle of the stream,
+   * scanEditLog will skip over them.
+   *
+   * This reads through the stream but does not close it.
+   *
+   * @param maxIndexToScan Maximum entry index to try to scan. The scan returns
+   *                       after reading this or a higher index. The file
+   *                       portion beyond this index is potentially being
+   *                       updated.
+   */
+  static LogValidation scanEditLog(LogInputStream in, long maxIndexToScan) {
+    long lastPos = 0;
+    long end = INVALID_LOG_INDEX;
+    long numValid = 0;
+    boolean hitError = false;
+    while (end < maxIndexToScan) {
+      long index;
+      lastPos = in.getPosition();
+      try {
+        if (hitError) {
+          LogEntryProto entry = in.nextEntry();
+          index = entry != null ? entry.getIndex() : INVALID_LOG_INDEX;
+          LOG.warn("After resync, position is " + in.getPosition());
+        } else {
+          index = in.scanNextEntry();
+        }
+        if (index == INVALID_LOG_INDEX) {
+          break;
+        } else {
+          hitError = false;
+        }
+      } catch (Throwable t) {
+        LOG.warn("Caught exception after scanning through {} ops from {}"
+            + " while determining its valid length. Position was "
+            + lastPos, numValid, in, t);
+        hitError = true;
+        continue;
+      }
+      if (end == INVALID_LOG_INDEX || index > end) {
+        end = index;
+      }
+      numValid++;
+    }
+    return new LogValidation(lastPos, end, false);
+  }
+}


[08/54] [abbrv] incubator-ratis git commit: Add a new RaftServer interface.

Posted by ji...@apache.org.
Add a new RaftServer interface.


Project: http://git-wip-us.apache.org/repos/asf/incubator-ratis/repo
Commit: http://git-wip-us.apache.org/repos/asf/incubator-ratis/commit/56e9b719
Tree: http://git-wip-us.apache.org/repos/asf/incubator-ratis/tree/56e9b719
Diff: http://git-wip-us.apache.org/repos/asf/incubator-ratis/diff/56e9b719

Branch: refs/heads/master
Commit: 56e9b719003e7d7024b05d9edec5a95937b3d3b3
Parents: 22f3ee0
Author: Tsz-Wo Nicholas Sze <sz...@hortonworks.com>
Authored: Mon Jan 2 21:10:11 2017 +0800
Committer: Tsz-Wo Nicholas Sze <sz...@hortonworks.com>
Committed: Mon Jan 2 21:10:11 2017 +0800

----------------------------------------------------------------------
 .../org/apache/raft/grpc/RaftGRpcService.java   |  6 +--
 .../raft/hadooprpc/server/HadoopRpcService.java |  8 ++--
 .../raft/netty/server/NettyRpcService.java      |  6 +--
 .../java/org/apache/raft/server/RaftServer.java | 34 +++++++++++++++
 .../org/apache/raft/server/RaftServerRpc.java   | 44 ++++++++++++++++++++
 .../apache/raft/server/impl/RaftServerImpl.java |  7 ++--
 .../apache/raft/server/impl/RaftServerRpc.java  | 44 --------------------
 .../raft/server/impl/RequestDispatcher.java     |  9 ++--
 .../java/org/apache/raft/MiniRaftCluster.java   |  2 +-
 .../server/simulation/SimulatedServerRpc.java   |  6 +--
 10 files changed, 102 insertions(+), 64 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-ratis/blob/56e9b719/raft-grpc/src/main/java/org/apache/raft/grpc/RaftGRpcService.java
----------------------------------------------------------------------
diff --git a/raft-grpc/src/main/java/org/apache/raft/grpc/RaftGRpcService.java b/raft-grpc/src/main/java/org/apache/raft/grpc/RaftGRpcService.java
index d465724..f3d894a 100644
--- a/raft-grpc/src/main/java/org/apache/raft/grpc/RaftGRpcService.java
+++ b/raft-grpc/src/main/java/org/apache/raft/grpc/RaftGRpcService.java
@@ -23,8 +23,8 @@ import org.apache.raft.grpc.client.RaftClientProtocolService;
 import org.apache.raft.grpc.server.RaftServerProtocolClient;
 import org.apache.raft.grpc.server.RaftServerProtocolService;
 import org.apache.raft.protocol.RaftPeer;
-import org.apache.raft.server.impl.RaftServerImpl;
-import org.apache.raft.server.impl.RaftServerRpc;
+import org.apache.raft.server.RaftServer;
+import org.apache.raft.server.RaftServerRpc;
 import org.apache.raft.server.impl.RequestDispatcher;
 import org.apache.raft.shaded.io.grpc.Server;
 import org.apache.raft.shaded.io.grpc.ServerBuilder;
@@ -54,7 +54,7 @@ public class RaftGRpcService implements RaftServerRpc {
       Collections.synchronizedMap(new HashMap<>());
   private final String selfId;
 
-  public RaftGRpcService(RaftServerImpl raftServer, RaftProperties properties) {
+  public RaftGRpcService(RaftServer raftServer, RaftProperties properties) {
     int port = properties.getInt(RAFT_GRPC_SERVER_PORT_KEY,
         RAFT_GRPC_SERVER_PORT_DEFAULT);
     int maxMessageSize = properties.getInt(

http://git-wip-us.apache.org/repos/asf/incubator-ratis/blob/56e9b719/raft-hadoop/src/main/java/org/apache/raft/hadooprpc/server/HadoopRpcService.java
----------------------------------------------------------------------
diff --git a/raft-hadoop/src/main/java/org/apache/raft/hadooprpc/server/HadoopRpcService.java b/raft-hadoop/src/main/java/org/apache/raft/hadooprpc/server/HadoopRpcService.java
index eb92372..3330d78 100644
--- a/raft-hadoop/src/main/java/org/apache/raft/hadooprpc/server/HadoopRpcService.java
+++ b/raft-hadoop/src/main/java/org/apache/raft/hadooprpc/server/HadoopRpcService.java
@@ -25,9 +25,9 @@ import org.apache.raft.hadooprpc.Proxy;
 import org.apache.raft.hadooprpc.client.RaftClientProtocolPB;
 import org.apache.raft.hadooprpc.client.RaftClientProtocolServerSideTranslatorPB;
 import org.apache.raft.protocol.RaftPeer;
-import org.apache.raft.server.*;
-import org.apache.raft.server.impl.RaftServerImpl;
-import org.apache.raft.server.impl.RaftServerRpc;
+import org.apache.raft.server.RaftServer;
+import org.apache.raft.server.RaftServerConfigKeys;
+import org.apache.raft.server.RaftServerRpc;
 import org.apache.raft.server.impl.RequestDispatcher;
 import org.apache.raft.shaded.com.google.protobuf.BlockingService;
 import org.apache.raft.shaded.com.google.protobuf.ServiceException;
@@ -56,7 +56,7 @@ public class HadoopRpcService implements RaftServerRpc {
 
   private final PeerProxyMap<Proxy<RaftServerProtocolPB>> proxies;
 
-  public HadoopRpcService(RaftServerImpl server, final Configuration conf)
+  public HadoopRpcService(RaftServer server, final Configuration conf)
       throws IOException {
     this.proxies = new PeerProxyMap<>(
         p -> new Proxy(RaftServerProtocolPB.class, p.getAddress(), conf));

http://git-wip-us.apache.org/repos/asf/incubator-ratis/blob/56e9b719/raft-netty/src/main/java/org/apache/raft/netty/server/NettyRpcService.java
----------------------------------------------------------------------
diff --git a/raft-netty/src/main/java/org/apache/raft/netty/server/NettyRpcService.java b/raft-netty/src/main/java/org/apache/raft/netty/server/NettyRpcService.java
index b5b8550..19f5979 100644
--- a/raft-netty/src/main/java/org/apache/raft/netty/server/NettyRpcService.java
+++ b/raft-netty/src/main/java/org/apache/raft/netty/server/NettyRpcService.java
@@ -29,8 +29,8 @@ import org.apache.raft.client.impl.ClientProtoUtils;
 import org.apache.raft.netty.NettyRpcProxy;
 import org.apache.raft.protocol.RaftClientReply;
 import org.apache.raft.protocol.RaftPeer;
-import org.apache.raft.server.impl.RaftServerImpl;
-import org.apache.raft.server.impl.RaftServerRpc;
+import org.apache.raft.server.RaftServer;
+import org.apache.raft.server.RaftServerRpc;
 import org.apache.raft.server.impl.RequestDispatcher;
 import org.apache.raft.shaded.io.netty.handler.codec.protobuf.ProtobufDecoder;
 import org.apache.raft.shaded.io.netty.handler.codec.protobuf.ProtobufEncoder;
@@ -75,7 +75,7 @@ public final class NettyRpcService implements RaftServerRpc {
   }
 
   /** Constructs a netty server with the given port. */
-  public NettyRpcService(int port, RaftServerImpl server) {
+  public NettyRpcService(int port, RaftServer server) {
     this.raftService = new RequestDispatcher(server);
     this.id = server.getId();
 

http://git-wip-us.apache.org/repos/asf/incubator-ratis/blob/56e9b719/raft-server/src/main/java/org/apache/raft/server/RaftServer.java
----------------------------------------------------------------------
diff --git a/raft-server/src/main/java/org/apache/raft/server/RaftServer.java b/raft-server/src/main/java/org/apache/raft/server/RaftServer.java
new file mode 100644
index 0000000..7141eca
--- /dev/null
+++ b/raft-server/src/main/java/org/apache/raft/server/RaftServer.java
@@ -0,0 +1,34 @@
+/**
+ * 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.raft.server;
+
+import org.apache.raft.server.protocol.RaftServerProtocol;
+
+import java.io.Closeable;
+
+/** Raft server interface */
+public interface RaftServer extends RaftServerProtocol, Closeable {
+  /** @return the server ID. */
+  String getId();
+
+  /** Set server RPC service. */
+  void setServerRpc(RaftServerRpc serverRpc);
+
+  /** Start this server. */
+  void start();
+}

http://git-wip-us.apache.org/repos/asf/incubator-ratis/blob/56e9b719/raft-server/src/main/java/org/apache/raft/server/RaftServerRpc.java
----------------------------------------------------------------------
diff --git a/raft-server/src/main/java/org/apache/raft/server/RaftServerRpc.java b/raft-server/src/main/java/org/apache/raft/server/RaftServerRpc.java
new file mode 100644
index 0000000..de81ec2
--- /dev/null
+++ b/raft-server/src/main/java/org/apache/raft/server/RaftServerRpc.java
@@ -0,0 +1,44 @@
+/**
+ * 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.raft.server;
+
+import org.apache.raft.protocol.RaftPeer;
+import org.apache.raft.shaded.proto.RaftProtos.*;
+
+import java.io.IOException;
+import java.net.InetSocketAddress;
+
+public interface RaftServerRpc {
+  void start();
+
+  void shutdown();
+
+  InetSocketAddress getInetSocketAddress();
+
+  AppendEntriesReplyProto sendAppendEntries(
+      AppendEntriesRequestProto request) throws IOException;
+
+  InstallSnapshotReplyProto sendInstallSnapshot(
+      InstallSnapshotRequestProto request) throws IOException;
+
+  RequestVoteReplyProto sendRequestVote(RequestVoteRequestProto request)
+      throws IOException;
+
+  /** add information of the given peers */
+  void addPeers(Iterable<RaftPeer> peers);
+}

http://git-wip-us.apache.org/repos/asf/incubator-ratis/blob/56e9b719/raft-server/src/main/java/org/apache/raft/server/impl/RaftServerImpl.java
----------------------------------------------------------------------
diff --git a/raft-server/src/main/java/org/apache/raft/server/impl/RaftServerImpl.java b/raft-server/src/main/java/org/apache/raft/server/impl/RaftServerImpl.java
index 6778683..131e002 100644
--- a/raft-server/src/main/java/org/apache/raft/server/impl/RaftServerImpl.java
+++ b/raft-server/src/main/java/org/apache/raft/server/impl/RaftServerImpl.java
@@ -21,8 +21,9 @@ import com.google.common.annotations.VisibleForTesting;
 import com.google.common.base.Preconditions;
 import org.apache.raft.conf.RaftProperties;
 import org.apache.raft.protocol.*;
+import org.apache.raft.server.RaftServer;
 import org.apache.raft.server.RaftServerConfigKeys;
-import org.apache.raft.server.protocol.RaftServerProtocol;
+import org.apache.raft.server.RaftServerRpc;
 import org.apache.raft.server.protocol.TermIndex;
 import org.apache.raft.server.storage.FileInfo;
 import org.apache.raft.shaded.proto.RaftProtos.*;
@@ -36,7 +37,6 @@ import org.apache.raft.util.RaftUtils;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
-import java.io.Closeable;
 import java.io.IOException;
 import java.io.InterruptedIOException;
 import java.util.Arrays;
@@ -48,7 +48,7 @@ import java.util.concurrent.CompletableFuture;
 import static org.apache.raft.shaded.proto.RaftProtos.AppendEntriesReplyProto.AppendResult.*;
 import static org.apache.raft.util.LifeCycle.State.*;
 
-public class RaftServerImpl implements RaftServerProtocol, Closeable {
+public class RaftServerImpl implements RaftServer {
   public static final Logger LOG = LoggerFactory.getLogger(RaftServerImpl.class);
 
   private static final String CLASS_NAME = RaftServerImpl.class.getSimpleName();
@@ -209,6 +209,7 @@ public class RaftServerImpl implements RaftServerProtocol, Closeable {
     });
   }
 
+  @VisibleForTesting
   public boolean isAlive() {
     return !lifeCycle.getCurrentState().isOneOf(CLOSING, CLOSED);
   }

http://git-wip-us.apache.org/repos/asf/incubator-ratis/blob/56e9b719/raft-server/src/main/java/org/apache/raft/server/impl/RaftServerRpc.java
----------------------------------------------------------------------
diff --git a/raft-server/src/main/java/org/apache/raft/server/impl/RaftServerRpc.java b/raft-server/src/main/java/org/apache/raft/server/impl/RaftServerRpc.java
deleted file mode 100644
index fb74a0e..0000000
--- a/raft-server/src/main/java/org/apache/raft/server/impl/RaftServerRpc.java
+++ /dev/null
@@ -1,44 +0,0 @@
-/**
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- *     http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-package org.apache.raft.server.impl;
-
-import org.apache.raft.protocol.RaftPeer;
-import org.apache.raft.shaded.proto.RaftProtos.*;
-
-import java.io.IOException;
-import java.net.InetSocketAddress;
-
-public interface RaftServerRpc {
-  void start();
-
-  void shutdown();
-
-  InetSocketAddress getInetSocketAddress();
-
-  AppendEntriesReplyProto sendAppendEntries(
-      AppendEntriesRequestProto request) throws IOException;
-
-  InstallSnapshotReplyProto sendInstallSnapshot(
-      InstallSnapshotRequestProto request) throws IOException;
-
-  RequestVoteReplyProto sendRequestVote(RequestVoteRequestProto request)
-      throws IOException;
-
-  /** add information of the given peers */
-  void addPeers(Iterable<RaftPeer> peers);
-}

http://git-wip-us.apache.org/repos/asf/incubator-ratis/blob/56e9b719/raft-server/src/main/java/org/apache/raft/server/impl/RequestDispatcher.java
----------------------------------------------------------------------
diff --git a/raft-server/src/main/java/org/apache/raft/server/impl/RequestDispatcher.java b/raft-server/src/main/java/org/apache/raft/server/impl/RequestDispatcher.java
index b897afd..39a4ac8 100644
--- a/raft-server/src/main/java/org/apache/raft/server/impl/RequestDispatcher.java
+++ b/raft-server/src/main/java/org/apache/raft/server/impl/RequestDispatcher.java
@@ -17,7 +17,9 @@
  */
 package org.apache.raft.server.impl;
 
+import com.google.common.base.Preconditions;
 import org.apache.raft.protocol.*;
+import org.apache.raft.server.RaftServer;
 import org.apache.raft.server.protocol.RaftServerProtocol;
 import org.apache.raft.shaded.proto.RaftProtos.*;
 import org.apache.raft.statemachine.StateMachine;
@@ -47,9 +49,10 @@ public class RequestDispatcher implements RaftClientProtocol, RaftServerProtocol
   private final RaftServerImpl server;
   private final StateMachine stateMachine;
 
-  public RequestDispatcher(RaftServerImpl server) {
-    this.server = server;
-    this.stateMachine = server.getStateMachine();
+  public RequestDispatcher(RaftServer server) {
+    Preconditions.checkArgument(server instanceof RaftServerImpl);
+    this.server = (RaftServerImpl)server;
+    this.stateMachine = this.server.getStateMachine();
   }
 
   public CompletableFuture<RaftClientReply> handleClientRequest(

http://git-wip-us.apache.org/repos/asf/incubator-ratis/blob/56e9b719/raft-server/src/test/java/org/apache/raft/MiniRaftCluster.java
----------------------------------------------------------------------
diff --git a/raft-server/src/test/java/org/apache/raft/MiniRaftCluster.java b/raft-server/src/test/java/org/apache/raft/MiniRaftCluster.java
index 45cec15..c66ef8f 100644
--- a/raft-server/src/test/java/org/apache/raft/MiniRaftCluster.java
+++ b/raft-server/src/test/java/org/apache/raft/MiniRaftCluster.java
@@ -27,7 +27,7 @@ import org.apache.raft.server.RaftServerConfigKeys;
 import org.apache.raft.server.impl.DelayLocalExecutionInjection;
 import org.apache.raft.server.impl.RaftConfiguration;
 import org.apache.raft.server.impl.RaftServerImpl;
-import org.apache.raft.server.impl.RaftServerRpc;
+import org.apache.raft.server.RaftServerRpc;
 import org.apache.raft.server.storage.MemoryRaftLog;
 import org.apache.raft.server.storage.RaftLog;
 import org.apache.raft.statemachine.BaseStateMachine;

http://git-wip-us.apache.org/repos/asf/incubator-ratis/blob/56e9b719/raft-server/src/test/java/org/apache/raft/server/simulation/SimulatedServerRpc.java
----------------------------------------------------------------------
diff --git a/raft-server/src/test/java/org/apache/raft/server/simulation/SimulatedServerRpc.java b/raft-server/src/test/java/org/apache/raft/server/simulation/SimulatedServerRpc.java
index 93e3f5c..cc3fb35 100644
--- a/raft-server/src/test/java/org/apache/raft/server/simulation/SimulatedServerRpc.java
+++ b/raft-server/src/test/java/org/apache/raft/server/simulation/SimulatedServerRpc.java
@@ -23,7 +23,7 @@ import org.apache.raft.protocol.RaftClientRequest;
 import org.apache.raft.protocol.RaftPeer;
 import org.apache.raft.protocol.SetConfigurationRequest;
 import org.apache.raft.server.impl.RaftServerImpl;
-import org.apache.raft.server.impl.RaftServerRpc;
+import org.apache.raft.server.RaftServerRpc;
 import org.apache.raft.server.impl.RequestDispatcher;
 import org.apache.raft.shaded.proto.RaftProtos.*;
 import org.slf4j.Logger;
@@ -36,7 +36,7 @@ import java.util.concurrent.ExecutorService;
 import java.util.concurrent.Executors;
 import java.util.concurrent.TimeUnit;
 
-public class SimulatedServerRpc implements RaftServerRpc {
+class SimulatedServerRpc implements RaftServerRpc {
   static final Logger LOG = LoggerFactory.getLogger(SimulatedServerRpc.class);
 
   private final RaftServerImpl server;
@@ -46,7 +46,7 @@ public class SimulatedServerRpc implements RaftServerRpc {
   private final ExecutorService executor = Executors.newFixedThreadPool(3,
       new ThreadFactoryBuilder().setDaemon(true).build());
 
-  public SimulatedServerRpc(RaftServerImpl server,
+  SimulatedServerRpc(RaftServerImpl server,
       SimulatedRequestReply<RaftServerRequest, RaftServerReply> serverRequestReply,
       SimulatedRequestReply<RaftClientRequest, RaftClientReply> clientRequestReply) {
     this.server = server;


[36/54] [abbrv] incubator-ratis git commit: Renamed the packages from raft to ratis in preperation for Apache Incubation - Moved all java packages from org.apache.raft to org.apache.ratis. - Moved native package to org_apache_ratis, and native lib to l

Posted by ji...@apache.org.
http://git-wip-us.apache.org/repos/asf/incubator-ratis/blob/7e71a2e0/raft-server/src/main/java/org/apache/raft/server/storage/LogOutputStream.java
----------------------------------------------------------------------
diff --git a/raft-server/src/main/java/org/apache/raft/server/storage/LogOutputStream.java b/raft-server/src/main/java/org/apache/raft/server/storage/LogOutputStream.java
deleted file mode 100644
index 0dc8029..0000000
--- a/raft-server/src/main/java/org/apache/raft/server/storage/LogOutputStream.java
+++ /dev/null
@@ -1,176 +0,0 @@
-/**
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- *     http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-package org.apache.raft.server.storage;
-
-import org.apache.raft.conf.RaftProperties;
-import org.apache.raft.server.impl.RaftServerConstants;
-import org.apache.raft.shaded.com.google.protobuf.CodedOutputStream;
-import org.apache.raft.shaded.proto.RaftProtos.LogEntryProto;
-import org.apache.raft.util.PureJavaCrc32C;
-import org.apache.raft.util.RaftUtils;
-import org.slf4j.Logger;
-import org.slf4j.LoggerFactory;
-
-import java.io.Closeable;
-import java.io.File;
-import java.io.IOException;
-import java.io.RandomAccessFile;
-import java.nio.ByteBuffer;
-import java.nio.channels.FileChannel;
-import java.util.zip.Checksum;
-
-import static org.apache.raft.server.RaftServerConfigKeys.*;
-
-public class LogOutputStream implements Closeable {
-  private static final Logger LOG = LoggerFactory.getLogger(LogOutputStream.class);
-
-  private static final ByteBuffer fill;
-  private static final int BUFFER_SIZE = 1024 * 1024; // 1 MB
-  static {
-    fill = ByteBuffer.allocateDirect(BUFFER_SIZE);
-    fill.position(0);
-    for (int i = 0; i < fill.capacity(); i++) {
-      fill.put(RaftServerConstants.LOG_TERMINATE_BYTE);
-    }
-  }
-
-  private File file;
-  private FileChannel fc; // channel of the file stream for sync
-  private BufferedWriteChannel out; // buffered FileChannel for writing
-  private final Checksum checksum;
-
-  private final long segmentMaxSize;
-  private final long preallocatedSize;
-  private long preallocatedPos;
-
-  public LogOutputStream(File file, boolean append, RaftProperties properties)
-      throws IOException {
-    this.file = file;
-    this.checksum = new PureJavaCrc32C();
-    this.segmentMaxSize = properties.getLong(RAFT_LOG_SEGMENT_MAX_SIZE_KEY,
-        RAFT_LOG_SEGMENT_MAX_SIZE_DEFAULT);
-    this.preallocatedSize = properties.getLong(
-        RAFT_LOG_SEGMENT_PREALLOCATED_SIZE_KEY,
-        RAFT_LOG_SEGMENT_PREALLOCATED_SIZE_DEFAULT);
-    RandomAccessFile rp = new RandomAccessFile(file, "rw");
-    fc = rp.getChannel();
-    fc.position(fc.size());
-    preallocatedPos = fc.size();
-
-    int bufferSize = properties.getInt(RAFT_LOG_WRITE_BUFFER_SIZE_KEY,
-        RAFT_LOG_WRITE_BUFFER_SIZE_DEFAULT);
-    out = new BufferedWriteChannel(fc, bufferSize);
-
-    if (!append) {
-      create();
-    }
-  }
-
-  /**
-   * Format:
-   * LogEntryProto's protobuf
-   * 4-byte checksum of the above protobuf
-   */
-  public void write(LogEntryProto entry) throws IOException {
-    final int serialized = entry.getSerializedSize();
-    final int bufferSize = CodedOutputStream.computeUInt32SizeNoTag(serialized)
-        + serialized;
-
-    preallocateIfNecessary(bufferSize + 4);
-
-    byte[] buf = new byte[bufferSize];
-    CodedOutputStream cout = CodedOutputStream.newInstance(buf);
-    cout.writeUInt32NoTag(serialized);
-    entry.writeTo(cout);
-
-    checksum.reset();
-    checksum.update(buf, 0, buf.length);
-    final int sum = (int) checksum.getValue();
-
-    out.write(buf);
-    writeInt(sum);
-  }
-
-  private void writeInt(int v) throws IOException {
-    out.write((v >>> 24) & 0xFF);
-    out.write((v >>> 16) & 0xFF);
-    out.write((v >>>  8) & 0xFF);
-    out.write((v) & 0xFF);
-  }
-
-  private void create() throws IOException {
-    fc.truncate(0);
-    fc.position(0);
-    preallocatedPos = 0;
-    preallocate(); // preallocate file
-
-    out.write(SegmentedRaftLog.HEADER_BYTES);
-    flush();
-  }
-
-  @Override
-  public void close() throws IOException {
-    try {
-      out.flush(false);
-      if (fc != null && fc.isOpen()) {
-        fc.truncate(fc.position());
-      }
-    } finally {
-      RaftUtils.cleanup(LOG, fc, out);
-      fc = null;
-      out = null;
-    }
-  }
-
-  /**
-   * Flush data to persistent store.
-   * Collect sync metrics.
-   */
-  public void flush() throws IOException {
-    if (out == null) {
-      throw new IOException("Trying to use aborted output stream");
-    }
-    out.flush(true);
-  }
-
-  private void preallocate() throws IOException {
-    fill.position(0);
-    long targetSize = Math.min(segmentMaxSize - fc.size(), preallocatedSize);
-    int allocated = 0;
-    while (allocated < targetSize) {
-      int size = (int) Math.min(BUFFER_SIZE, targetSize - allocated);
-      ByteBuffer buffer = fill.slice();
-      buffer.limit(size);
-      RaftUtils.writeFully(fc, buffer, preallocatedPos);
-      preallocatedPos += size;
-      allocated += size;
-    }
-    LOG.debug("Pre-allocated {} bytes for the log segment", allocated);
-  }
-
-  private void preallocateIfNecessary(int size) throws IOException {
-    if (out.position() + size > preallocatedPos) {
-      preallocate();
-    }
-  }
-
-  @Override
-  public String toString() {
-    return this.getClass().getSimpleName() + "(" + file + ")";
-  }
-}

http://git-wip-us.apache.org/repos/asf/incubator-ratis/blob/7e71a2e0/raft-server/src/main/java/org/apache/raft/server/storage/LogReader.java
----------------------------------------------------------------------
diff --git a/raft-server/src/main/java/org/apache/raft/server/storage/LogReader.java b/raft-server/src/main/java/org/apache/raft/server/storage/LogReader.java
deleted file mode 100644
index 9523cac..0000000
--- a/raft-server/src/main/java/org/apache/raft/server/storage/LogReader.java
+++ /dev/null
@@ -1,292 +0,0 @@
-/**
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- *     http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-package org.apache.raft.server.storage;
-
-import com.google.common.base.Preconditions;
-import org.apache.commons.io.Charsets;
-import org.apache.raft.protocol.ChecksumException;
-import org.apache.raft.server.impl.RaftServerConstants;
-import org.apache.raft.shaded.com.google.protobuf.CodedInputStream;
-import org.apache.raft.shaded.com.google.protobuf.CodedOutputStream;
-import org.apache.raft.shaded.proto.RaftProtos.LogEntryProto;
-import org.apache.raft.util.PureJavaCrc32C;
-import org.apache.raft.util.RaftUtils;
-
-import java.io.*;
-import java.util.zip.Checksum;
-
-public class LogReader implements Closeable {
-  /**
-   * InputStream wrapper that keeps track of the current stream position.
-   *
-   * This stream also allows us to set a limit on how many bytes we can read
-   * without getting an exception.
-   */
-  public static class LimitedInputStream extends FilterInputStream {
-    private long curPos = 0;
-    private long markPos = -1;
-    private long limitPos = Long.MAX_VALUE;
-
-    public LimitedInputStream(InputStream is) {
-      super(is);
-    }
-
-    private void checkLimit(long amt) throws IOException {
-      long extra = (curPos + amt) - limitPos;
-      if (extra > 0) {
-        throw new IOException("Tried to read " + amt + " byte(s) past " +
-            "the limit at offset " + limitPos);
-      }
-    }
-
-    @Override
-    public int read() throws IOException {
-      checkLimit(1);
-      int ret = super.read();
-      if (ret != -1) curPos++;
-      return ret;
-    }
-
-    @Override
-    public int read(byte[] data) throws IOException {
-      checkLimit(data.length);
-      int ret = super.read(data);
-      if (ret > 0) curPos += ret;
-      return ret;
-    }
-
-    @Override
-    public int read(byte[] data, int offset, int length) throws IOException {
-      checkLimit(length);
-      int ret = super.read(data, offset, length);
-      if (ret > 0) curPos += ret;
-      return ret;
-    }
-
-    public void setLimit(long limit) {
-      limitPos = curPos + limit;
-    }
-
-    public void clearLimit() {
-      limitPos = Long.MAX_VALUE;
-    }
-
-    @Override
-    public void mark(int limit) {
-      super.mark(limit);
-      markPos = curPos;
-    }
-
-    @Override
-    public void reset() throws IOException {
-      if (markPos == -1) {
-        throw new IOException("Not marked!");
-      }
-      super.reset();
-      curPos = markPos;
-      markPos = -1;
-    }
-
-    public long getPos() {
-      return curPos;
-    }
-
-    @Override
-    public long skip(long amt) throws IOException {
-      long extra = (curPos + amt) - limitPos;
-      if (extra > 0) {
-        throw new IOException("Tried to skip " + extra + " bytes past " +
-            "the limit at offset " + limitPos);
-      }
-      long ret = super.skip(amt);
-      curPos += ret;
-      return ret;
-    }
-  }
-
-  private static final int maxOpSize = 32 * 1024 * 1024;
-
-  private final LimitedInputStream limiter;
-  private final DataInputStream in;
-  private byte[] temp = new byte[4096];
-  private final Checksum checksum;
-
-  LogReader(File file) throws FileNotFoundException {
-    this.limiter = new LimitedInputStream(
-        new BufferedInputStream(new FileInputStream(file)));
-    in = new DataInputStream(limiter);
-    checksum = new PureJavaCrc32C();
-  }
-
-  String readLogHeader() throws IOException {
-    byte[] header = new byte[SegmentedRaftLog.HEADER_BYTES.length];
-    int num = in.read(header);
-    if (num < header.length) {
-      throw new EOFException("EOF before reading a complete log header");
-    }
-    return new String(header, Charsets.UTF_8);
-  }
-
-  /**
-   * Read a log entry from the input stream.
-   *
-   * @return the operation read from the stream, or null at the end of the
-   *         file
-   * @throws IOException on error.  This function should only throw an
-   *         exception when skipBrokenEdits is false.
-   */
-  LogEntryProto readEntry() throws IOException {
-    try {
-      return decodeEntry();
-    } catch (IOException e) {
-      in.reset();
-
-      throw e;
-    } catch (Throwable e) {
-      // raft log requires no gap between any two entries. thus if an entry is
-      // broken, throw the exception instead of skipping broken entries
-      in.reset();
-      throw new IOException("got unexpected exception " + e.getMessage(), e);
-    }
-  }
-
-  /**
-   * Scan and validate a log entry.
-   * @return the index of the log entry
-   */
-  long scanEntry() throws IOException {
-    LogEntryProto entry = decodeEntry();
-    return entry != null ? entry.getIndex() : RaftServerConstants.INVALID_LOG_INDEX;
-  }
-
-  void verifyTerminator() throws IOException {
-     // The end of the log should contain 0x00 bytes.
-     // If it contains other bytes, the log itself may be corrupt.
-    limiter.clearLimit();
-    int numRead = -1, idx = 0;
-    while (true) {
-      try {
-        numRead = -1;
-        numRead = in.read(temp);
-        if (numRead == -1) {
-          return;
-        }
-        for (idx = 0; idx < numRead; idx++) {
-          if (temp[idx] != RaftServerConstants.LOG_TERMINATE_BYTE) {
-            throw new IOException("Read extra bytes after the terminator!");
-          }
-        }
-      } finally {
-        // After reading each group of bytes, we reposition the mark one
-        // byte before the next group. Similarly, if there is an error, we
-        // want to reposition the mark one byte before the error
-        if (numRead != -1) {
-          in.reset();
-          RaftUtils.skipFully(in, idx);
-          in.mark(temp.length + 1);
-          RaftUtils.skipFully(in, 1);
-        }
-      }
-    }
-  }
-
-  /**
-   * Decode the log entry "frame". This includes reading the log entry, and
-   * validating the checksum.
-   *
-   * The input stream will be advanced to the end of the op at the end of this
-   * function.
-   *
-   * @return The log entry, or null if we hit EOF.
-   */
-  private LogEntryProto decodeEntry() throws IOException {
-    limiter.setLimit(maxOpSize);
-    in.mark(maxOpSize);
-
-    byte nextByte;
-    try {
-      nextByte = in.readByte();
-    } catch (EOFException eof) {
-      // EOF at an opcode boundary is expected.
-      return null;
-    }
-    // Each log entry starts with a var-int. Thus a valid entry's first byte
-    // should not be 0. So if the terminate byte is 0, we should hit the end
-    // of the segment.
-    if (nextByte == RaftServerConstants.LOG_TERMINATE_BYTE) {
-      verifyTerminator();
-      return null;
-    }
-
-    // Here, we verify that the Op size makes sense and that the
-    // data matches its checksum before attempting to construct an Op.
-    int entryLength = CodedInputStream.readRawVarint32(nextByte, in);
-    if (entryLength > maxOpSize) {
-      throw new IOException("Entry has size " + entryLength
-          + ", but maxOpSize = " + maxOpSize);
-    }
-
-    final int varintLength = CodedOutputStream.computeUInt32SizeNoTag(
-        entryLength);
-    final int totalLength = varintLength + entryLength;
-    checkBufferSize(totalLength);
-    in.reset();
-    in.mark(maxOpSize);
-    RaftUtils.readFully(in, temp, 0, totalLength);
-
-    // verify checksum
-    checksum.reset();
-    checksum.update(temp, 0, totalLength);
-    int expectedChecksum = in.readInt();
-    int calculatedChecksum = (int) checksum.getValue();
-    if (expectedChecksum != calculatedChecksum) {
-      throw new ChecksumException("LogEntry is corrupt. Calculated checksum is "
-          + calculatedChecksum + " but read checksum " + expectedChecksum,
-          limiter.markPos);
-    }
-
-    // parse the buffer
-    return LogEntryProto.parseFrom(
-        CodedInputStream.newInstance(temp, varintLength, entryLength));
-  }
-
-  private void checkBufferSize(int entryLength) {
-    Preconditions.checkArgument(entryLength <= maxOpSize);
-    int length = temp.length;
-    if (length < entryLength) {
-      while (length < entryLength) {
-        length = Math.min(length * 2, maxOpSize);
-      }
-      temp = new byte[length];
-    }
-  }
-
-  long getPos() {
-    return limiter.getPos();
-  }
-
-  void skipFully(long length) throws IOException {
-    limiter.clearLimit();
-    RaftUtils.skipFully(limiter, length);
-  }
-
-  @Override
-  public void close() throws IOException {
-    RaftUtils.cleanup(null, in);
-  }
-}

http://git-wip-us.apache.org/repos/asf/incubator-ratis/blob/7e71a2e0/raft-server/src/main/java/org/apache/raft/server/storage/LogSegment.java
----------------------------------------------------------------------
diff --git a/raft-server/src/main/java/org/apache/raft/server/storage/LogSegment.java b/raft-server/src/main/java/org/apache/raft/server/storage/LogSegment.java
deleted file mode 100644
index 987cc6c..0000000
--- a/raft-server/src/main/java/org/apache/raft/server/storage/LogSegment.java
+++ /dev/null
@@ -1,232 +0,0 @@
-/**
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- *     http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-package org.apache.raft.server.storage;
-
-import com.google.common.base.Preconditions;
-import org.apache.raft.server.impl.ConfigurationManager;
-import org.apache.raft.server.impl.ServerProtoUtils;
-import org.apache.raft.shaded.com.google.protobuf.CodedOutputStream;
-import org.apache.raft.shaded.proto.RaftProtos.LogEntryProto;
-import org.apache.raft.util.FileUtils;
-
-import java.io.File;
-import java.io.IOException;
-import java.util.ArrayList;
-import java.util.List;
-
-import static org.apache.raft.shaded.proto.RaftProtos.LogEntryProto.LogEntryBodyCase.CONFIGURATIONENTRY;
-
-/**
- * In-memory cache for a log segment file. All the updates will be first written
- * into LogSegment then into corresponding files in the same order.
- *
- * This class will be protected by the RaftServer's lock.
- */
-class LogSegment implements Comparable<Long> {
-  static class LogRecord {
-    /** starting offset in the file */
-    final long offset;
-    final LogEntryProto entry;
-
-    LogRecord(long offset, LogEntryProto entry) {
-      this.offset = offset;
-      this.entry = entry;
-    }
-  }
-
-  static class SegmentFileInfo {
-    final long startIndex; // start index of the
-    final long endIndex; // original end index
-    final boolean isOpen;
-    final long targetLength; // position for truncation
-    final long newEndIndex; // new end index after the truncation
-
-    SegmentFileInfo(long start, long end, boolean isOpen, long targetLength,
-        long newEndIndex) {
-      this.startIndex = start;
-      this.endIndex = end;
-      this.isOpen = isOpen;
-      this.targetLength = targetLength;
-      this.newEndIndex = newEndIndex;
-    }
-  }
-
-  static long getEntrySize(LogEntryProto entry) {
-    final int serialized = entry.getSerializedSize();
-    return serialized + CodedOutputStream.computeUInt32SizeNoTag(serialized) + 4;
-  }
-
-  private boolean isOpen;
-  private final List<LogRecord> records = new ArrayList<>();
-  private long totalSize;
-  private final long startIndex;
-  private long endIndex;
-
-  private LogSegment(boolean isOpen, long start, long end) {
-    this.isOpen = isOpen;
-    this.startIndex = start;
-    this.endIndex = end;
-    totalSize = SegmentedRaftLog.HEADER_BYTES.length;
-  }
-
-  static LogSegment newOpenSegment(long start) {
-    Preconditions.checkArgument(start >= 0);
-    return new LogSegment(true, start, start - 1);
-  }
-
-  private static LogSegment newCloseSegment(long start, long end) {
-    Preconditions.checkArgument(start >= 0 && end >= start);
-    return new LogSegment(false, start, end);
-  }
-
-  static LogSegment loadSegment(File file, long start, long end, boolean isOpen,
-      ConfigurationManager confManager) throws IOException {
-    final LogSegment segment;
-    try (LogInputStream in = new LogInputStream(file, start, end, isOpen)) {
-      segment = isOpen ? LogSegment.newOpenSegment(start) :
-          LogSegment.newCloseSegment(start, end);
-      LogEntryProto next;
-      LogEntryProto prev = null;
-      while ((next = in.nextEntry()) != null) {
-        if (prev != null) {
-          Preconditions.checkState(next.getIndex() == prev.getIndex() + 1,
-              "gap between entry %s and entry %s", prev, next);
-        }
-        segment.append(next);
-        if (confManager != null &&
-            next.getLogEntryBodyCase() == CONFIGURATIONENTRY) {
-          confManager.addConfiguration(next.getIndex(),
-              ServerProtoUtils.toRaftConfiguration(next.getIndex(),
-                  next.getConfigurationEntry()));
-        }
-        prev = next;
-      }
-    }
-
-    // truncate padding if necessary
-    if (file.length() > segment.getTotalSize()) {
-      FileUtils.truncateFile(file, segment.getTotalSize());
-    }
-
-    Preconditions.checkState(start == segment.records.get(0).entry.getIndex());
-    if (!isOpen) {
-      Preconditions.checkState(segment.getEndIndex() == end);
-    }
-    return segment;
-  }
-
-  long getStartIndex() {
-    return startIndex;
-  }
-
-  long getEndIndex() {
-    return endIndex;
-  }
-
-  boolean isOpen() {
-    return isOpen;
-  }
-
-  int numOfEntries() {
-    return (int) (endIndex - startIndex + 1);
-  }
-
-  void appendToOpenSegment(LogEntryProto... entries) {
-    Preconditions.checkState(isOpen(),
-        "The log segment %s is not open for append", this.toString());
-    append(entries);
-  }
-
-  private void append(LogEntryProto... entries) {
-    Preconditions.checkArgument(entries != null && entries.length > 0);
-    final long term = entries[0].getTerm();
-    if (records.isEmpty()) {
-      Preconditions.checkArgument(entries[0].getIndex() == startIndex,
-          "gap between start index %s and first entry to append %s",
-          startIndex, entries[0].getIndex());
-    }
-    for (LogEntryProto entry : entries) {
-      // all these entries should be of the same term
-      Preconditions.checkArgument(entry.getTerm() == term,
-          "expected term:%s, term of the entry:%s", term, entry.getTerm());
-      final LogRecord currentLast = getLastRecord();
-      if (currentLast != null) {
-        Preconditions.checkArgument(
-            entry.getIndex() == currentLast.entry.getIndex() + 1,
-            "gap between entries %s and %s", entry.getIndex(),
-            currentLast.entry.getIndex());
-      }
-
-      final LogRecord record = new LogRecord(totalSize, entry);
-      records.add(record);
-      totalSize += getEntrySize(entry);
-      endIndex = entry.getIndex();
-    }
-  }
-
-  LogRecord getLogRecord(long index) {
-    if (index >= startIndex && index <= endIndex) {
-      return records.get((int) (index - startIndex));
-    }
-    return null;
-  }
-
-  LogRecord getLastRecord() {
-    return records.isEmpty() ? null : records.get(records.size() - 1);
-  }
-
-  long getTotalSize() {
-    return totalSize;
-  }
-
-  /**
-   * Remove records from the given index (inclusive)
-   */
-  void truncate(long fromIndex) {
-    Preconditions.checkArgument(fromIndex >= startIndex && fromIndex <= endIndex);
-    LogRecord record = records.get((int) (fromIndex - startIndex));
-    for (long index = endIndex; index >= fromIndex; index--) {
-      records.remove((int)(index - startIndex));
-    }
-    totalSize = record.offset;
-    isOpen = false;
-    this.endIndex = fromIndex - 1;
-  }
-
-  void close() {
-    Preconditions.checkState(isOpen());
-    isOpen = false;
-  }
-
-  @Override
-  public String toString() {
-    return isOpen() ? "log-" + startIndex + "-inprogress" :
-        "log-" + startIndex + "-" + endIndex;
-  }
-
-  @Override
-  public int compareTo(Long l) {
-    return (l >= getStartIndex() && l <= getEndIndex()) ? 0 :
-        (this.getEndIndex() < l ? -1 : 1);
-  }
-
-  void clear() {
-    records.clear();
-    endIndex = startIndex - 1;
-  }
-}

http://git-wip-us.apache.org/repos/asf/incubator-ratis/blob/7e71a2e0/raft-server/src/main/java/org/apache/raft/server/storage/MemoryRaftLog.java
----------------------------------------------------------------------
diff --git a/raft-server/src/main/java/org/apache/raft/server/storage/MemoryRaftLog.java b/raft-server/src/main/java/org/apache/raft/server/storage/MemoryRaftLog.java
deleted file mode 100644
index c12e1aa..0000000
--- a/raft-server/src/main/java/org/apache/raft/server/storage/MemoryRaftLog.java
+++ /dev/null
@@ -1,182 +0,0 @@
-/**
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- *     http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-package org.apache.raft.server.storage;
-
-import com.google.common.base.Preconditions;
-import org.apache.raft.server.impl.RaftConfiguration;
-import org.apache.raft.server.impl.RaftServerConstants;
-import org.apache.raft.server.impl.ServerProtoUtils;
-import org.apache.raft.shaded.proto.RaftProtos.LogEntryProto;
-import org.apache.raft.util.AutoCloseableLock;
-import org.apache.raft.util.CodeInjectionForTesting;
-
-import java.util.ArrayList;
-import java.util.List;
-
-/**
- * A simple RaftLog implementation in memory. Used only for testing.
- */
-public class MemoryRaftLog extends RaftLog {
-  private final List<LogEntryProto> entries = new ArrayList<>();
-
-  public MemoryRaftLog(String selfId) {
-    super(selfId);
-  }
-
-  @Override
-  public LogEntryProto get(long index) {
-    checkLogState();
-    try(AutoCloseableLock readLock = readLock()) {
-      final int i = (int) index;
-      return i >= 0 && i < entries.size() ? entries.get(i) : null;
-    }
-  }
-
-  @Override
-  public LogEntryProto[] getEntries(long startIndex, long endIndex) {
-    checkLogState();
-    try(AutoCloseableLock readLock = readLock()) {
-      final int i = (int) startIndex;
-      if (startIndex >= entries.size()) {
-        return null;
-      }
-      final int toIndex = (int) Math.min(entries.size(), endIndex);
-      return entries.subList(i, toIndex).toArray(EMPTY_LOGENTRY_ARRAY);
-    }
-  }
-
-  @Override
-  void truncate(long index) {
-    checkLogState();
-    try(AutoCloseableLock writeLock = writeLock()) {
-      Preconditions.checkArgument(index >= 0);
-      final int truncateIndex = (int) index;
-      for (int i = entries.size() - 1; i >= truncateIndex; i--) {
-        entries.remove(i);
-      }
-    }
-  }
-
-  @Override
-  public LogEntryProto getLastEntry() {
-    checkLogState();
-    try(AutoCloseableLock readLock = readLock()) {
-      final int size = entries.size();
-      return size == 0 ? null : entries.get(size - 1);
-    }
-  }
-
-  @Override
-  void appendEntry(LogEntryProto entry) {
-    checkLogState();
-    try(AutoCloseableLock writeLock = writeLock()) {
-      entries.add(entry);
-    }
-  }
-
-  @Override
-  public long append(long term, RaftConfiguration newConf) {
-    checkLogState();
-    try(AutoCloseableLock writeLock = writeLock()) {
-      final long nextIndex = getNextIndex();
-      final LogEntryProto e = ServerProtoUtils.toLogEntryProto(newConf, term,
-          nextIndex);
-      entries.add(e);
-      return nextIndex;
-    }
-  }
-
-  @Override
-  public long getStartIndex() {
-    return entries.isEmpty() ? RaftServerConstants.INVALID_LOG_INDEX :
-        entries.get(0).getIndex();
-  }
-
-  @Override
-  public void append(LogEntryProto... entries) {
-    checkLogState();
-    try(AutoCloseableLock writeLock = writeLock()) {
-      if (entries == null || entries.length == 0) {
-        return;
-      }
-      // Before truncating the entries, we first need to check if some
-      // entries are duplicated. If the leader sends entry 6, entry 7, then
-      // entry 6 again, without this check the follower may truncate entry 7
-      // when receiving entry 6 again. Then before the leader detects this
-      // truncation in the next appendEntries RPC, leader may think entry 7 has
-      // been committed but in the system the entry has not been committed to
-      // the quorum of peers' disks.
-      // TODO add a unit test for this
-      boolean toTruncate = false;
-      int truncateIndex = (int) entries[0].getIndex();
-      int index = 0;
-      for (; truncateIndex < getNextIndex() && index < entries.length;
-           index++, truncateIndex++) {
-        if (this.entries.get(truncateIndex).getTerm() !=
-            entries[index].getTerm()) {
-          toTruncate = true;
-          break;
-        }
-      }
-      if (toTruncate) {
-        truncate(truncateIndex);
-      }
-      //  Collections.addAll(this.entries, entries);
-      for (int i = index; i < entries.length; i++) {
-        this.entries.add(entries[i]);
-      }
-    }
-  }
-
-  @Override
-  public String toString() {
-    return "last=" + ServerProtoUtils.toString(getLastEntry())
-        + ", committed="
-        + ServerProtoUtils.toString(get(getLastCommittedIndex()));
-  }
-
-  public String getEntryString() {
-    return "entries=" + entries;
-  }
-
-  @Override
-  public void logSync() {
-    CodeInjectionForTesting.execute(LOG_SYNC, getSelfId(), null);
-    // do nothing
-  }
-
-  @Override
-  public long getLatestFlushedIndex() {
-    return getNextIndex() - 1;
-  }
-
-  @Override
-  public void writeMetadata(long term, String votedFor) {
-    // do nothing
-  }
-
-  @Override
-  public Metadata loadMetadata() {
-    return new Metadata(null, 0);
-  }
-
-  @Override
-  public void syncWithSnapshot(long lastSnapshotIndex) {
-    // do nothing
-  }
-}

http://git-wip-us.apache.org/repos/asf/incubator-ratis/blob/7e71a2e0/raft-server/src/main/java/org/apache/raft/server/storage/MetaFile.java
----------------------------------------------------------------------
diff --git a/raft-server/src/main/java/org/apache/raft/server/storage/MetaFile.java b/raft-server/src/main/java/org/apache/raft/server/storage/MetaFile.java
deleted file mode 100644
index b2b6f04..0000000
--- a/raft-server/src/main/java/org/apache/raft/server/storage/MetaFile.java
+++ /dev/null
@@ -1,130 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- *  with the License.  You may obtain a copy of the License at
- *
- *      http://www.apache.org/licenses/LICENSE-2.0
- *
- *  Unless required by applicable law or agreed to in writing, software
- *  distributed under the License is distributed on an "AS IS" BASIS,
- *  WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- *  See the License for the specific language governing permissions and
- *  limitations under the License.
- */
-package org.apache.raft.server.storage;
-
-import com.google.common.base.Charsets;
-import org.apache.raft.util.AtomicFileOutputStream;
-import org.apache.raft.util.RaftUtils;
-import org.slf4j.Logger;
-import org.slf4j.LoggerFactory;
-
-import java.io.*;
-import java.util.Properties;
-
-/**
- * Class that represents a file on disk which persistently stores
- * a single <code>long</code> value. The file is updated atomically
- * and durably (i.e fsynced).
- */
-class MetaFile {
-  private static final Logger LOG = LoggerFactory.getLogger(MetaFile.class);
-  private static final String TERM_KEY = "term";
-  private static final String VOTEDFOR_KEY = "votedFor";
-  static final long DEFAULT_TERM = 0;
-  static final String EMPTY_VOTEFOR = "";
-
-  private final File file;
-  private boolean loaded = false;
-  private long term;
-  private String votedFor;
-
-  MetaFile(File file) {
-    this.file = file;
-    term = DEFAULT_TERM;
-    votedFor = EMPTY_VOTEFOR;
-  }
-
-  boolean exists() {
-    return this.file.exists();
-  }
-
-  long getTerm() throws IOException {
-    if (!loaded) {
-      readFile();
-      loaded = true;
-    }
-    return term;
-  }
-
-  String getVotedFor() throws IOException {
-    if (!loaded) {
-      readFile();
-      loaded = true;
-    }
-    return votedFor;
-  }
-
-  void set(long newTerm, String newVotedFor) throws IOException {
-    newVotedFor = newVotedFor == null ? EMPTY_VOTEFOR : newVotedFor;
-    if (!loaded || (newTerm != term || !newVotedFor.equals(votedFor))) {
-      writeFile(newTerm, newVotedFor);
-    }
-    term = newTerm;
-    votedFor = newVotedFor;
-    loaded = true;
-  }
-
-  /**
-   * Atomically write the given term and votedFor information to the given file,
-   * including fsyncing.
-   *
-   * @throws IOException if the file cannot be written
-   */
-  void writeFile(long term, String votedFor) throws IOException {
-    AtomicFileOutputStream fos = new AtomicFileOutputStream(file);
-    Properties properties = new Properties();
-    properties.setProperty(TERM_KEY, Long.toString(term));
-    properties.setProperty(VOTEDFOR_KEY, votedFor);
-    try {
-      properties.store(
-          new BufferedWriter(new OutputStreamWriter(fos, Charsets.UTF_8)), "");
-      fos.close();
-      fos = null;
-    } finally {
-      if (fos != null) {
-        fos.abort();
-      }
-    }
-  }
-
-  void readFile() throws IOException {
-    term = DEFAULT_TERM;
-    votedFor = EMPTY_VOTEFOR;
-    if (file.exists()) {
-      BufferedReader br = new BufferedReader(
-          new InputStreamReader(new FileInputStream(file), Charsets.UTF_8));
-      try {
-        Properties properties = new Properties();
-        properties.load(br);
-        if (properties.containsKey(TERM_KEY) &&
-            properties.containsKey(VOTEDFOR_KEY)) {
-          term = Long.parseLong((String) properties.get(TERM_KEY));
-          votedFor = (String) properties.get(VOTEDFOR_KEY);
-        } else {
-          throw new IOException("Corrupted term/votedFor properties: "
-              + properties);
-        }
-      } catch(IOException e) {
-        LOG.warn("Cannot load term/votedFor properties from {}", file, e);
-        throw e;
-      } finally {
-        RaftUtils.cleanup(LOG, br);
-      }
-    }
-  }
-}

http://git-wip-us.apache.org/repos/asf/incubator-ratis/blob/7e71a2e0/raft-server/src/main/java/org/apache/raft/server/storage/RaftLog.java
----------------------------------------------------------------------
diff --git a/raft-server/src/main/java/org/apache/raft/server/storage/RaftLog.java b/raft-server/src/main/java/org/apache/raft/server/storage/RaftLog.java
deleted file mode 100644
index de79911..0000000
--- a/raft-server/src/main/java/org/apache/raft/server/storage/RaftLog.java
+++ /dev/null
@@ -1,292 +0,0 @@
-/**
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- *     http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-package org.apache.raft.server.storage;
-
-import com.google.common.base.Preconditions;
-import org.apache.raft.server.impl.ConfigurationManager;
-import org.apache.raft.server.impl.RaftConfiguration;
-import org.apache.raft.server.impl.RaftServerConstants;
-import org.apache.raft.server.impl.ServerProtoUtils;
-import org.apache.raft.server.protocol.TermIndex;
-import org.apache.raft.shaded.proto.RaftProtos.LogEntryProto;
-import org.apache.raft.statemachine.TransactionContext;
-import org.apache.raft.util.AutoCloseableLock;
-import org.apache.raft.util.ProtoUtils;
-import org.slf4j.Logger;
-import org.slf4j.LoggerFactory;
-
-import java.io.Closeable;
-import java.io.IOException;
-import java.util.concurrent.atomic.AtomicLong;
-import java.util.concurrent.locks.ReentrantReadWriteLock;
-
-/**
- * Base class of RaftLog. Currently we provide two types of RaftLog
- * implementation:
- * 1. MemoryRaftLog: all the log entries are stored in memory. This is only used
- *    for testing.
- * 2. Segmented RaftLog: the log entries are persisted on disk, and are stored
- *    in segments.
- */
-public abstract class RaftLog implements Closeable {
-  public static final Logger LOG = LoggerFactory.getLogger(RaftLog.class);
-  public static final LogEntryProto[] EMPTY_LOGENTRY_ARRAY = new LogEntryProto[0];
-  public static final String LOG_SYNC = RaftLog.class.getSimpleName() + ".logSync";
-
-  /**
-   * The largest committed index. Note the last committed log may be included
-   * in the latest snapshot file.
-   */
-  protected final AtomicLong lastCommitted =
-      new AtomicLong(RaftServerConstants.INVALID_LOG_INDEX);
-  private final String selfId;
-
-  private final ReentrantReadWriteLock lock = new ReentrantReadWriteLock(true);
-  private volatile boolean isOpen = false;
-
-  public RaftLog(String selfId) {
-    this.selfId = selfId;
-  }
-
-  public long getLastCommittedIndex() {
-    return lastCommitted.get();
-  }
-
-  public void checkLogState() {
-    Preconditions.checkState(isOpen,
-        "The RaftLog has not been opened or has been closed");
-  }
-
-  /**
-   * Update the last committed index.
-   * @param majorityIndex the index that has achieved majority.
-   * @param currentTerm the current term.
-   */
-  public void updateLastCommitted(long majorityIndex, long currentTerm) {
-    try(AutoCloseableLock writeLock = writeLock()) {
-      if (lastCommitted.get() < majorityIndex) {
-        // Only update last committed index for current term. See �5.4.2 in
-        // paper for details.
-        final LogEntryProto entry = get(majorityIndex);
-        if (entry != null && entry.getTerm() == currentTerm) {
-          LOG.debug("{}: Updating lastCommitted to {}", selfId, majorityIndex);
-          lastCommitted.set(majorityIndex);
-        }
-      }
-    }
-  }
-
-  /**
-   * Does the log contains the given term and index? Used to check the
-   * consistency between the local log of a follower and the log entries sent
-   * by the leader.
-   */
-  public boolean contains(TermIndex ti) {
-    if (ti == null) {
-      return false;
-    }
-    LogEntryProto entry = get(ti.getIndex());
-    TermIndex local = ServerProtoUtils.toTermIndex(entry);
-    return ti.equals(local);
-  }
-
-  /**
-   * @return the index of the next log entry to append.
-   */
-  public long getNextIndex() {
-    final LogEntryProto last = getLastEntry();
-    if (last == null) {
-      // if the log is empty, the last committed index should be consistent with
-      // the last index included in the latest snapshot.
-      return getLastCommittedIndex() + 1;
-    }
-    return last.getIndex() + 1;
-  }
-
-  /**
-   * Generate a log entry for the given term and message, and append the entry.
-   * Used by the leader.
-   * @return the index of the new log entry.
-   */
-  public long append(long term, TransactionContext operation) throws IOException {
-    checkLogState();
-    try(AutoCloseableLock writeLock = writeLock()) {
-      final long nextIndex = getNextIndex();
-
-      // This is called here to guarantee strict serialization of callback executions in case
-      // the SM wants to attach a logic depending on ordered execution in the log commit order.
-      operation = operation.preAppendTransaction();
-
-      // build the log entry after calling the StateMachine
-      final LogEntryProto e = ProtoUtils.toLogEntryProto(
-          operation.getSMLogEntry().get(), term, nextIndex);
-
-      appendEntry(e);
-      operation.setLogEntry(e);
-      return nextIndex;
-    }
-  }
-
-  /**
-   * Generate a log entry for the given term and configurations,
-   * and append the entry. Used by the leader.
-   * @return the index of the new log entry.
-   */
-  public long append(long term, RaftConfiguration newConf) {
-    checkLogState();
-    try(AutoCloseableLock writeLock = writeLock()) {
-      final long nextIndex = getNextIndex();
-      final LogEntryProto e = ServerProtoUtils.toLogEntryProto(newConf, term,
-          nextIndex);
-      appendEntry(e);
-      return nextIndex;
-    }
-  }
-
-  public void open(ConfigurationManager confManager, long lastIndexInSnapshot)
-      throws IOException {
-    isOpen = true;
-  }
-
-  public abstract long getStartIndex();
-
-  /**
-   * Get the log entry of the given index.
-   *
-   * @param index The given index.
-   * @return The log entry associated with the given index.
-   *         Null if there is no log entry with the index.
-   */
-  public abstract LogEntryProto get(long index);
-
-  /**
-   * @param startIndex the starting log index (inclusive)
-   * @param endIndex the ending log index (exclusive)
-   * @return all log entries within the given index range. Null if startIndex
-   *         is greater than the smallest available index.
-   */
-  public abstract LogEntryProto[] getEntries(long startIndex, long endIndex);
-
-  /**
-   * @return the last log entry.
-   */
-  public abstract LogEntryProto getLastEntry();
-
-  /**
-   * Truncate the log entries till the given index. The log with the given index
-   * will also be truncated (i.e., inclusive).
-   */
-  abstract void truncate(long index);
-
-  /**
-   * Used by the leader when appending a new entry based on client's request
-   * or configuration change.
-   */
-  abstract void appendEntry(LogEntryProto entry);
-
-  /**
-   * Append all the given log entries. Used by the followers.
-   *
-   * If an existing entry conflicts with a new one (same index but different
-   * terms), delete the existing entry and all entries that follow it (�5.3).
-   *
-   * This method, {@link #append(long, TransactionContext)},
-   * {@link #append(long, RaftConfiguration)}, and {@link #truncate(long)},
-   * do not guarantee the changes are persisted.
-   * Need to call {@link #logSync()} to persist the changes.
-   */
-  public abstract void append(LogEntryProto... entries);
-
-  /**
-   * Flush and sync the log.
-   * It is triggered by AppendEntries RPC request from the leader.
-   */
-  public abstract void logSync() throws InterruptedException;
-
-  /**
-   * @return the index of the latest entry that has been flushed to the local
-   *         storage.
-   */
-  public abstract long getLatestFlushedIndex();
-
-  /**
-   * Write and flush the metadata (votedFor and term) into the meta file.
-   *
-   * We need to guarantee that the order of writeMetadata calls is the same with
-   * that when we change the in-memory term/votedFor. Otherwise we may persist
-   * stale term/votedFor in file.
-   *
-   * Since the leader change is not frequent, currently we simply put this call
-   * in the RaftPeer's lock. Later we can use an IO task queue to enforce the
-   * order.
-   */
-  public abstract void writeMetadata(long term, String votedFor)
-      throws IOException;
-
-  public abstract Metadata loadMetadata() throws IOException;
-
-  public abstract void syncWithSnapshot(long lastSnapshotIndex);
-
-  @Override
-  public String toString() {
-    return ServerProtoUtils.toString(getLastEntry());
-  }
-
-  public static class Metadata {
-    private final String votedFor;
-    private final long term;
-
-    public Metadata(String votedFor, long term) {
-      this.votedFor = votedFor;
-      this.term = term;
-    }
-
-    public String getVotedFor() {
-      return votedFor;
-    }
-
-    public long getTerm() {
-      return term;
-    }
-  }
-
-  public AutoCloseableLock readLock() {
-    return AutoCloseableLock.acquire(lock.readLock());
-  }
-
-  public AutoCloseableLock writeLock() {
-    return AutoCloseableLock.acquire(lock.writeLock());
-  }
-
-  public boolean hasWriteLock() {
-    return this.lock.isWriteLockedByCurrentThread();
-  }
-
-  public boolean hasReadLock() {
-    return this.lock.getReadHoldCount() > 0 || hasWriteLock();
-  }
-
-  @Override
-  public void close() throws IOException {
-    isOpen = false;
-  }
-
-  public String getSelfId() {
-    return selfId;
-  }
-}

http://git-wip-us.apache.org/repos/asf/incubator-ratis/blob/7e71a2e0/raft-server/src/main/java/org/apache/raft/server/storage/RaftLogCache.java
----------------------------------------------------------------------
diff --git a/raft-server/src/main/java/org/apache/raft/server/storage/RaftLogCache.java b/raft-server/src/main/java/org/apache/raft/server/storage/RaftLogCache.java
deleted file mode 100644
index d022a91..0000000
--- a/raft-server/src/main/java/org/apache/raft/server/storage/RaftLogCache.java
+++ /dev/null
@@ -1,323 +0,0 @@
-/**
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- *     http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-package org.apache.raft.server.storage;
-
-import com.google.common.annotations.VisibleForTesting;
-import com.google.common.base.Preconditions;
-import org.apache.raft.server.impl.RaftServerConstants;
-import org.apache.raft.server.storage.LogSegment.LogRecord;
-import org.apache.raft.server.storage.LogSegment.SegmentFileInfo;
-import org.apache.raft.shaded.proto.RaftProtos.LogEntryProto;
-
-import java.util.*;
-
-import static org.apache.raft.server.impl.RaftServerConstants.INVALID_LOG_INDEX;
-
-/**
- * In-memory RaftLog Cache. Currently we provide a simple implementation that
- * caches all the segments in the memory. The cache is not thread-safe and
- * requires external lock protection.
- */
-class RaftLogCache {
-  private LogSegment openSegment;
-  private final List<LogSegment> closedSegments;
-
-  RaftLogCache() {
-    closedSegments = new ArrayList<>();
-  }
-
-  private boolean areConsecutiveSegments(LogSegment prev, LogSegment segment) {
-    return !prev.isOpen() && prev.getEndIndex() + 1 == segment.getStartIndex();
-  }
-
-  private LogSegment getLastClosedSegment() {
-    return closedSegments.isEmpty() ?
-        null : closedSegments.get(closedSegments.size() - 1);
-  }
-
-  private void validateAdding(LogSegment segment) {
-    final LogSegment lastClosed = getLastClosedSegment();
-    if (!segment.isOpen()) {
-      Preconditions.checkState(lastClosed == null ||
-          areConsecutiveSegments(lastClosed, segment));
-    } else {
-      Preconditions.checkState(openSegment == null &&
-          (lastClosed == null || areConsecutiveSegments(lastClosed, segment)));
-    }
-  }
-
-  void addSegment(LogSegment segment) {
-    validateAdding(segment);
-    if (segment.isOpen()) {
-      openSegment = segment;
-    } else {
-      closedSegments.add(segment);
-    }
-  }
-
-  LogEntryProto getEntry(long index) {
-    if (openSegment != null && index >= openSegment.getStartIndex()) {
-      final LogRecord record = openSegment.getLogRecord(index);
-      return record == null ? null : record.entry;
-    } else {
-      int segmentIndex = Collections.binarySearch(closedSegments, index);
-      if (segmentIndex < 0) {
-        return null;
-      } else {
-        return closedSegments.get(segmentIndex).getLogRecord(index).entry;
-      }
-    }
-  }
-
-  /**
-   * @param startIndex inclusive
-   * @param endIndex exclusive
-   */
-  LogEntryProto[] getEntries(final long startIndex, final long endIndex) {
-    if (startIndex < 0 || startIndex < getStartIndex()) {
-      throw new IndexOutOfBoundsException("startIndex = " + startIndex
-          + ", log cache starts from index " + getStartIndex());
-    }
-    if (startIndex > endIndex) {
-      throw new IndexOutOfBoundsException("startIndex(" + startIndex
-          + ") > endIndex(" + endIndex + ")");
-    }
-    final long realEnd = Math.min(getEndIndex() + 1, endIndex);
-    if (startIndex >= realEnd) {
-      return RaftLog.EMPTY_LOGENTRY_ARRAY;
-    }
-
-    LogEntryProto[] entries = new LogEntryProto[(int) (realEnd - startIndex)];
-    int segmentIndex = Collections.binarySearch(closedSegments, startIndex);
-    if (segmentIndex < 0) {
-      getEntriesFromSegment(openSegment, startIndex, entries, 0, entries.length);
-    } else {
-      long index = startIndex;
-      for (int i = segmentIndex; i < closedSegments.size() && index < realEnd; i++) {
-        LogSegment s = closedSegments.get(i);
-        int numberFromSegment = (int) Math.min(realEnd - index,
-            s.getEndIndex() - index + 1);
-        getEntriesFromSegment(s, index, entries, (int) (index - startIndex),
-            numberFromSegment);
-        index += numberFromSegment;
-      }
-      if (index < realEnd) {
-        getEntriesFromSegment(openSegment, index, entries,
-            (int) (index - startIndex), (int) (realEnd - index));
-      }
-    }
-    return entries;
-  }
-
-  private void getEntriesFromSegment(LogSegment segment, long startIndex,
-      LogEntryProto[] entries, int offset, int size) {
-    long endIndex = segment.getEndIndex();
-    endIndex = Math.min(endIndex, startIndex + size - 1);
-    int index = offset;
-    for (long i = startIndex; i <= endIndex; i++) {
-      entries[index++] = segment.getLogRecord(i).entry;
-    }
-  }
-
-  long getStartIndex() {
-    if (closedSegments.isEmpty()) {
-      return openSegment != null ? openSegment.getStartIndex() :
-          RaftServerConstants.INVALID_LOG_INDEX;
-    } else {
-      return closedSegments.get(0).getStartIndex();
-    }
-  }
-
-  @VisibleForTesting
-  long getEndIndex() {
-    return openSegment != null ? openSegment.getEndIndex() :
-        (closedSegments.isEmpty() ?
-            INVALID_LOG_INDEX :
-            closedSegments.get(closedSegments.size() - 1).getEndIndex());
-  }
-
-  LogEntryProto getLastEntry() {
-    return (openSegment != null && openSegment.numOfEntries() > 0) ?
-        openSegment.getLastRecord().entry :
-        (closedSegments.isEmpty() ? null :
-            closedSegments.get(closedSegments.size() - 1).getLastRecord().entry);
-  }
-
-  LogSegment getOpenSegment() {
-    return openSegment;
-  }
-
-  void appendEntry(LogEntryProto entry) {
-    // SegmentedRaftLog does the segment creation/rolling work. Here we just
-    // simply append the entry into the open segment.
-    Preconditions.checkState(openSegment != null);
-    openSegment.appendToOpenSegment(entry);
-  }
-
-  /**
-   * finalize the current open segment, and start a new open segment
-   */
-  void rollOpenSegment(boolean createNewOpen) {
-    Preconditions.checkState(openSegment != null
-        && openSegment.numOfEntries() > 0);
-    final long nextIndex = openSegment.getEndIndex() + 1;
-    openSegment.close();
-    closedSegments.add(openSegment);
-    if (createNewOpen) {
-      openSegment = LogSegment.newOpenSegment(nextIndex);
-    } else {
-      openSegment = null;
-    }
-  }
-
-  private SegmentFileInfo deleteOpenSegment() {
-    final long oldEnd = openSegment.getEndIndex();
-    openSegment.clear();
-    SegmentFileInfo info = new SegmentFileInfo(openSegment.getStartIndex(),
-        oldEnd, true, 0, openSegment.getEndIndex());
-    openSegment = null;
-    return info;
-  }
-
-  /**
-   * truncate log entries starting from the given index (inclusive)
-   */
-  TruncationSegments truncate(long index) {
-    int segmentIndex = Collections.binarySearch(closedSegments, index);
-    if (segmentIndex == -closedSegments.size() - 1) {
-      if (openSegment != null && openSegment.getEndIndex() >= index) {
-        final long oldEnd = openSegment.getEndIndex();
-        if (index == openSegment.getStartIndex()) {
-          // the open segment should be deleted
-          return new TruncationSegments(null,
-              Collections.singletonList(deleteOpenSegment()));
-        } else {
-          openSegment.truncate(index);
-          Preconditions.checkState(!openSegment.isOpen());
-          SegmentFileInfo info = new SegmentFileInfo(openSegment.getStartIndex(),
-              oldEnd, true, openSegment.getTotalSize(),
-              openSegment.getEndIndex());
-          closedSegments.add(openSegment);
-          openSegment = null;
-          return new TruncationSegments(info, Collections.emptyList());
-        }
-      }
-    } else if (segmentIndex >= 0) {
-      LogSegment ts = closedSegments.get(segmentIndex);
-      final long oldEnd = ts.getEndIndex();
-      List<SegmentFileInfo> list = new ArrayList<>();
-      ts.truncate(index);
-      final int size = closedSegments.size();
-      for (int i = size - 1;
-           i >= (ts.numOfEntries() == 0 ? segmentIndex : segmentIndex + 1);
-           i-- ) {
-        LogSegment s = closedSegments.remove(i);
-        final long endOfS = i == segmentIndex ? oldEnd : s.getEndIndex();
-        s.clear();
-        list.add(new SegmentFileInfo(s.getStartIndex(), endOfS, false, 0,
-            s.getEndIndex()));
-      }
-      if (openSegment != null) {
-        list.add(deleteOpenSegment());
-      }
-      SegmentFileInfo t = ts.numOfEntries() == 0 ? null :
-          new SegmentFileInfo(ts.getStartIndex(), oldEnd, false,
-              ts.getTotalSize(), ts.getEndIndex());
-      return new TruncationSegments(t, list);
-    }
-    return null;
-  }
-
-  static class TruncationSegments {
-    final SegmentFileInfo toTruncate; // name of the file to be truncated
-    final SegmentFileInfo[] toDelete; // names of the files to be deleted
-
-    TruncationSegments(SegmentFileInfo toTruncate,
-        List<SegmentFileInfo> toDelete) {
-      this.toDelete = toDelete == null ? null :
-          toDelete.toArray(new SegmentFileInfo[toDelete.size()]);
-      this.toTruncate = toTruncate;
-    }
-  }
-
-  Iterator<LogEntryProto> iterator(long startIndex) {
-    return new EntryIterator(startIndex);
-  }
-
-  private class EntryIterator implements Iterator<LogEntryProto> {
-    private long nextIndex;
-    private LogSegment currentSegment;
-    private int segmentIndex;
-
-    EntryIterator(long start) {
-      this.nextIndex = start;
-      segmentIndex = Collections.binarySearch(closedSegments, nextIndex);
-      if (segmentIndex >= 0) {
-        currentSegment = closedSegments.get(segmentIndex);
-      } else {
-        segmentIndex = -segmentIndex - 1;
-        if (segmentIndex == closedSegments.size()) {
-          currentSegment = openSegment;
-        } else {
-          // the start index is smaller than the first closed segment's start
-          // index. We no longer keep the log entry (because of the snapshot) or
-          // the start index is invalid.
-          Preconditions.checkState(segmentIndex == 0);
-          throw new IndexOutOfBoundsException();
-        }
-      }
-    }
-
-    @Override
-    public boolean hasNext() {
-      return currentSegment != null &&
-          currentSegment.getLogRecord(nextIndex) != null;
-    }
-
-    @Override
-    public LogEntryProto next() {
-      LogRecord record;
-      if (currentSegment == null ||
-          (record = currentSegment.getLogRecord(nextIndex)) == null) {
-        throw new NoSuchElementException();
-      }
-      if (++nextIndex > currentSegment.getEndIndex()) {
-        if (currentSegment != openSegment) {
-          segmentIndex++;
-          currentSegment = segmentIndex == closedSegments.size() ?
-              openSegment : closedSegments.get(segmentIndex);
-        }
-      }
-      return record.entry;
-    }
-  }
-
-  @VisibleForTesting
-  int getNumOfSegments() {
-    return closedSegments.size() + (openSegment == null ? 0 : 1);
-  }
-
-  boolean isEmpty() {
-    return closedSegments.isEmpty() && openSegment == null;
-  }
-
-  void clear() {
-    openSegment = null;
-    closedSegments.clear();
-  }
-}

http://git-wip-us.apache.org/repos/asf/incubator-ratis/blob/7e71a2e0/raft-server/src/main/java/org/apache/raft/server/storage/RaftLogWorker.java
----------------------------------------------------------------------
diff --git a/raft-server/src/main/java/org/apache/raft/server/storage/RaftLogWorker.java b/raft-server/src/main/java/org/apache/raft/server/storage/RaftLogWorker.java
deleted file mode 100644
index 6cef212..0000000
--- a/raft-server/src/main/java/org/apache/raft/server/storage/RaftLogWorker.java
+++ /dev/null
@@ -1,371 +0,0 @@
-/**
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- *     http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-package org.apache.raft.server.storage;
-
-import com.google.common.base.Preconditions;
-import org.apache.raft.conf.RaftProperties;
-import org.apache.raft.io.nativeio.NativeIO;
-import org.apache.raft.server.impl.RaftServerImpl;
-import org.apache.raft.server.impl.RaftServerConstants;
-import org.apache.raft.server.storage.LogSegment.SegmentFileInfo;
-import org.apache.raft.server.storage.RaftLogCache.TruncationSegments;
-import org.apache.raft.server.storage.SegmentedRaftLog.Task;
-import org.apache.raft.shaded.proto.RaftProtos.LogEntryProto;
-import org.apache.raft.util.ExitUtils;
-import org.apache.raft.util.FileUtils;
-import org.apache.raft.util.RaftUtils;
-import org.slf4j.Logger;
-import org.slf4j.LoggerFactory;
-
-import java.io.File;
-import java.io.IOException;
-import java.util.concurrent.ArrayBlockingQueue;
-import java.util.concurrent.BlockingQueue;
-import java.util.concurrent.TimeUnit;
-
-import static org.apache.raft.server.RaftServerConfigKeys.RAFT_LOG_FORCE_SYNC_NUM_DEFAULT;
-import static org.apache.raft.server.RaftServerConfigKeys.RAFT_LOG_FORCE_SYNC_NUM_KEY;
-
-/**
- * This class takes the responsibility of all the raft log related I/O ops for a
- * raft peer.
- */
-class RaftLogWorker implements Runnable {
-  static final Logger LOG = LoggerFactory.getLogger(RaftLogWorker.class);
-  /**
-   * The task queue accessed by rpc handler threads and the io worker thread.
-   */
-  private final BlockingQueue<Task> queue = new ArrayBlockingQueue<>(4096);
-  private volatile boolean running = true;
-  private final Thread workerThread;
-
-  private final RaftStorage storage;
-  private LogOutputStream out;
-  private final RaftServerImpl raftServer;
-
-  /**
-   * The number of entries that have been written into the LogOutputStream but
-   * has not been flushed.
-   */
-  private int pendingFlushNum = 0;
-  /** the index of the last entry that has been written */
-  private long lastWrittenIndex;
-  /** the largest index of the entry that has been flushed */
-  private volatile long flushedIndex;
-
-  private final int forceSyncNum;
-
-  private final  RaftProperties properties;
-
-  RaftLogWorker(RaftServerImpl raftServer, RaftStorage storage,
-                RaftProperties properties) {
-    this.raftServer = raftServer;
-    this.storage = storage;
-    this.properties = properties;
-    this.forceSyncNum = properties.getInt(RAFT_LOG_FORCE_SYNC_NUM_KEY,
-        RAFT_LOG_FORCE_SYNC_NUM_DEFAULT);
-    workerThread = new Thread(this,
-        getClass().getSimpleName() + " for " + storage);
-  }
-
-  void start(long latestIndex, File openSegmentFile) throws IOException {
-    lastWrittenIndex = latestIndex;
-    flushedIndex = latestIndex;
-    if (openSegmentFile != null) {
-      Preconditions.checkArgument(openSegmentFile.exists());
-      out = new LogOutputStream(openSegmentFile, true, properties);
-    }
-    workerThread.start();
-  }
-
-  void close() {
-    this.running = false;
-    workerThread.interrupt();
-    try {
-      workerThread.join();
-    } catch (InterruptedException ignored) {
-    }
-  }
-
-  /**
-   * A snapshot has just been installed on the follower. Need to update the IO
-   * worker's state accordingly.
-   */
-  void syncWithSnapshot(long lastSnapshotIndex) {
-    queue.clear();
-    lastWrittenIndex = lastSnapshotIndex;
-    flushedIndex = lastSnapshotIndex;
-    pendingFlushNum = 0;
-  }
-
-  @Override
-  public String toString() {
-    return this.getClass().getSimpleName() + "-"
-        + (raftServer != null ? raftServer.getId() : "");
-  }
-
-  /**
-   * This is protected by the RaftServer and RaftLog's lock.
-   */
-  private Task addIOTask(Task task) {
-    LOG.debug("add task {}", task);
-    try {
-      if (!queue.offer(task, 1, TimeUnit.SECONDS)) {
-        Preconditions.checkState(isAlive(),
-            "the worker thread is not alive");
-        queue.put(task);
-      }
-    } catch (Throwable t) {
-      if (t instanceof InterruptedException && !running) {
-        LOG.info("Got InterruptedException when adding task " + task
-            + ". The RaftLogWorker already stopped.");
-      } else {
-        ExitUtils.terminate(2, "Failed to add IO task " + task, t, LOG);
-      }
-    }
-    return task;
-  }
-
-  boolean isAlive() {
-    return running && workerThread.isAlive();
-  }
-
-  @Override
-  public void run() {
-    while (running) {
-      try {
-        Task task = queue.poll(1, TimeUnit.SECONDS);
-        if (task != null) {
-          try {
-            task.execute();
-          } catch (IOException e) {
-            if (task.getEndIndex() < lastWrittenIndex) {
-              LOG.info("Ignore IOException when handling task " + task
-                  + " which is smaller than the lastWrittenIndex."
-                  + " There should be a snapshot installed.", e);
-            } else {
-              throw e;
-            }
-          }
-          task.done();
-        }
-      } catch (InterruptedException e) {
-        LOG.info(Thread.currentThread().getName()
-            + " was interrupted, exiting. There are " + queue.size()
-            + " tasks remaining in the queue.");
-      } catch (Throwable t) {
-        // TODO avoid terminating the jvm by supporting multiple log directories
-        ExitUtils.terminate(1, Thread.currentThread().getName() + " failed.", t, LOG);
-      }
-    }
-  }
-
-  private boolean shouldFlush() {
-    return pendingFlushNum >= forceSyncNum ||
-        (pendingFlushNum > 0 && queue.isEmpty());
-  }
-
-  private void flushWrites() throws IOException {
-    if (out != null) {
-      LOG.debug("flush data to " + out + ", reset pending_sync_number to 0");
-      out.flush();
-      updateFlushedIndex();
-    }
-  }
-
-  private void updateFlushedIndex() {
-    flushedIndex = lastWrittenIndex;
-    pendingFlushNum = 0;
-    if (raftServer != null) {
-      raftServer.submitLocalSyncEvent();
-    }
-  }
-
-  /**
-   * The following several methods (startLogSegment, rollLogSegment,
-   * writeLogEntry, and truncate) are only called by SegmentedRaftLog which is
-   * protected by RaftServer's lock.
-   *
-   * Thus all the tasks are created and added sequentially.
-   */
-  Task startLogSegment(long startIndex) {
-    return addIOTask(new StartLogSegment(startIndex));
-  }
-
-  Task rollLogSegment(LogSegment segmentToClose) {
-    addIOTask(new FinalizeLogSegment(segmentToClose));
-    return addIOTask(new StartLogSegment(segmentToClose.getEndIndex() + 1));
-  }
-
-  Task writeLogEntry(LogEntryProto entry) {
-    return addIOTask(new WriteLog(entry));
-  }
-
-  Task truncate(TruncationSegments ts) {
-    return addIOTask(new TruncateLog(ts));
-  }
-
-  // TODO we can add another level of buffer for writing here
-  private class WriteLog extends Task {
-    private final LogEntryProto entry;
-
-    WriteLog(LogEntryProto entry) {
-      this.entry = entry;
-    }
-
-    @Override
-    public void execute() throws IOException {
-      Preconditions.checkState(out != null);
-      Preconditions.checkState(lastWrittenIndex + 1 == entry.getIndex(),
-          "lastWrittenIndex == %s, entry == %s", lastWrittenIndex, entry);
-      out.write(entry);
-      lastWrittenIndex = entry.getIndex();
-      pendingFlushNum++;
-      if (shouldFlush()) {
-        flushWrites();
-      }
-    }
-
-    @Override
-    long getEndIndex() {
-      return entry.getIndex();
-    }
-  }
-
-  private class FinalizeLogSegment extends Task {
-    private final LogSegment segmentToClose;
-
-    FinalizeLogSegment(LogSegment segmentToClose) {
-      this.segmentToClose = segmentToClose;
-    }
-
-    @Override
-    public void execute() throws IOException {
-      RaftUtils.cleanup(null, out);
-      out = null;
-      Preconditions.checkState(segmentToClose != null);
-
-      File openFile = storage.getStorageDir()
-          .getOpenLogFile(segmentToClose.getStartIndex());
-      Preconditions.checkState(openFile.exists(),
-          "File %s does not exist.", openFile);
-      if (segmentToClose.numOfEntries() > 0) {
-        // finalize the current open segment
-        File dstFile = storage.getStorageDir().getClosedLogFile(
-            segmentToClose.getStartIndex(), segmentToClose.getEndIndex());
-        Preconditions.checkState(!dstFile.exists());
-
-        NativeIO.renameTo(openFile, dstFile);
-      } else { // delete the file of the empty segment
-        FileUtils.deleteFile(openFile);
-      }
-      updateFlushedIndex();
-    }
-
-    @Override
-    long getEndIndex() {
-      return segmentToClose.getEndIndex();
-    }
-  }
-
-  private class StartLogSegment extends Task {
-    private final long newStartIndex;
-
-    StartLogSegment(long newStartIndex) {
-      this.newStartIndex = newStartIndex;
-    }
-
-    @Override
-    void execute() throws IOException {
-      File openFile = storage.getStorageDir().getOpenLogFile(newStartIndex);
-      Preconditions.checkState(!openFile.exists(), "open file %s exists for %s",
-          openFile.getAbsolutePath(), RaftLogWorker.this.toString());
-      Preconditions.checkState(out == null && pendingFlushNum == 0);
-      out = new LogOutputStream(openFile, false, properties);
-    }
-
-    @Override
-    long getEndIndex() {
-      return newStartIndex;
-    }
-  }
-
-  private class TruncateLog extends Task {
-    private final TruncationSegments segments;
-
-    TruncateLog(TruncationSegments ts) {
-      this.segments = ts;
-    }
-
-    @Override
-    void execute() throws IOException {
-      RaftUtils.cleanup(null, out);
-      out = null;
-      if (segments.toTruncate != null) {
-        File fileToTruncate = segments.toTruncate.isOpen ?
-            storage.getStorageDir().getOpenLogFile(
-                segments.toTruncate.startIndex) :
-            storage.getStorageDir().getClosedLogFile(
-                segments.toTruncate.startIndex,
-                segments.toTruncate.endIndex);
-        FileUtils.truncateFile(fileToTruncate, segments.toTruncate.targetLength);
-
-        // rename the file
-        File dstFile = storage.getStorageDir().getClosedLogFile(
-            segments.toTruncate.startIndex, segments.toTruncate.newEndIndex);
-        Preconditions.checkState(!dstFile.exists());
-        NativeIO.renameTo(fileToTruncate, dstFile);
-
-        // update lastWrittenIndex
-        lastWrittenIndex = segments.toTruncate.newEndIndex;
-      }
-      if (segments.toDelete != null && segments.toDelete.length > 0) {
-        long minStart = segments.toDelete[0].startIndex;
-        for (SegmentFileInfo del : segments.toDelete) {
-          final File delFile;
-          if (del.isOpen) {
-            delFile = storage.getStorageDir().getOpenLogFile(del.startIndex);
-          } else {
-            delFile = storage.getStorageDir()
-                .getClosedLogFile(del.startIndex, del.endIndex);
-          }
-          FileUtils.deleteFile(delFile);
-          minStart = Math.min(minStart, del.startIndex);
-        }
-        if (segments.toTruncate == null) {
-          lastWrittenIndex = minStart - 1;
-        }
-      }
-      updateFlushedIndex();
-    }
-
-    @Override
-    long getEndIndex() {
-      if (segments.toTruncate != null) {
-        return segments.toTruncate.newEndIndex;
-      } else if (segments.toDelete.length > 0) {
-        return segments.toDelete[segments.toDelete.length - 1].endIndex;
-      }
-      return RaftServerConstants.INVALID_LOG_INDEX;
-    }
-  }
-
-  long getFlushedIndex() {
-    return flushedIndex;
-  }
-}

http://git-wip-us.apache.org/repos/asf/incubator-ratis/blob/7e71a2e0/raft-server/src/main/java/org/apache/raft/server/storage/RaftStorage.java
----------------------------------------------------------------------
diff --git a/raft-server/src/main/java/org/apache/raft/server/storage/RaftStorage.java b/raft-server/src/main/java/org/apache/raft/server/storage/RaftStorage.java
deleted file mode 100644
index 434f505..0000000
--- a/raft-server/src/main/java/org/apache/raft/server/storage/RaftStorage.java
+++ /dev/null
@@ -1,144 +0,0 @@
-/**
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- *     http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-package org.apache.raft.server.storage;
-
-import com.google.common.base.Preconditions;
-import org.apache.raft.conf.RaftProperties;
-import org.apache.raft.server.impl.RaftServerConstants;
-import org.apache.raft.server.storage.RaftStorageDirectory.StorageState;
-import org.apache.raft.statemachine.SnapshotInfo;
-import org.apache.raft.statemachine.StateMachineStorage;
-import org.apache.raft.util.FileUtils;
-import org.slf4j.Logger;
-import org.slf4j.LoggerFactory;
-
-import java.io.Closeable;
-import java.io.File;
-import java.io.IOException;
-import java.nio.file.Files;
-
-import static org.apache.raft.server.RaftServerConfigKeys.RAFT_SERVER_STORAGE_DIR_DEFAULT;
-import static org.apache.raft.server.RaftServerConfigKeys.RAFT_SERVER_STORAGE_DIR_KEY;
-
-public class RaftStorage implements Closeable {
-  private static final Logger LOG = LoggerFactory.getLogger(RaftStorage.class);
-
-  // TODO support multiple storage directories
-  private final RaftStorageDirectory storageDir;
-  private final StorageState state;
-  private volatile MetaFile metaFile;
-  private StateMachineStorage stateMachineStorage;
-
-  public RaftStorage(RaftProperties prop, RaftServerConstants.StartupOption option)
-      throws IOException {
-    final String dir = prop.get(RAFT_SERVER_STORAGE_DIR_KEY,
-        RAFT_SERVER_STORAGE_DIR_DEFAULT);
-    storageDir = new RaftStorageDirectory(
-        new File(FileUtils.stringAsURI(dir).getPath()));
-    if (option == RaftServerConstants.StartupOption.FORMAT) {
-      if (storageDir.analyzeStorage(false) == StorageState.NON_EXISTENT) {
-        throw new IOException("Cannot format " + storageDir);
-      }
-      storageDir.lock();
-      format();
-      state = storageDir.analyzeStorage(false);
-      Preconditions.checkState(state == StorageState.NORMAL);
-    } else {
-      state = analyzeAndRecoverStorage(true); // metaFile is initialized here
-      if (state != StorageState.NORMAL) {
-        storageDir.unlock();
-        throw new IOException("Cannot load " + storageDir
-            + ". Its state: " + state);
-      }
-    }
-  }
-
-  StorageState getState() {
-    return state;
-  }
-
-  private void format() throws IOException {
-    storageDir.clearDirectory();
-    metaFile = writeMetaFile(MetaFile.DEFAULT_TERM, MetaFile.EMPTY_VOTEFOR);
-    LOG.info("Storage directory " + storageDir.getRoot()
-        + " has been successfully formatted.");
-  }
-
-  private MetaFile writeMetaFile(long term, String votedFor) throws IOException {
-    MetaFile metaFile = new MetaFile(storageDir.getMetaFile());
-    metaFile.set(term, votedFor);
-    return metaFile;
-  }
-
-  private void cleanMetaTmpFile() throws IOException {
-    Files.deleteIfExists(storageDir.getMetaTmpFile().toPath());
-  }
-
-  private StorageState analyzeAndRecoverStorage(boolean toLock)
-      throws IOException {
-    StorageState storageState = storageDir.analyzeStorage(toLock);
-    if (storageState == StorageState.NORMAL) {
-      metaFile = new MetaFile(storageDir.getMetaFile());
-      assert metaFile.exists();
-      metaFile.readFile();
-      // Existence of raft-meta.tmp means the change of votedFor/term has not
-      // been committed. Thus we should delete the tmp file.
-      cleanMetaTmpFile();
-      return StorageState.NORMAL;
-    } else if (storageState == StorageState.NOT_FORMATTED &&
-        storageDir.isCurrentEmpty()) {
-      format();
-      return StorageState.NORMAL;
-    } else {
-      return storageState;
-    }
-  }
-
-  public RaftStorageDirectory getStorageDir() {
-    return storageDir;
-  }
-
-  @Override
-  public void close() throws IOException {
-    storageDir.unlock();
-  }
-
-  MetaFile getMetaFile() {
-    return metaFile;
-  }
-
-  public SnapshotInfo getLastestSnapshot() throws IOException {
-    return getStateMachineStorage().getLatestSnapshot();
-  }
-
-  /**
-   * Called by the state machine after it has initialized the StateMachineStorage.
-   */
-  public void setStateMachineStorage(StateMachineStorage smStorage) {
-    this.stateMachineStorage = smStorage;
-  }
-
-  public StateMachineStorage getStateMachineStorage() {
-    return stateMachineStorage;
-  }
-
-  @Override
-  public String toString() {
-    return getStorageDir() + "";
-  }
-}

http://git-wip-us.apache.org/repos/asf/incubator-ratis/blob/7e71a2e0/raft-server/src/main/java/org/apache/raft/server/storage/RaftStorageDirectory.java
----------------------------------------------------------------------
diff --git a/raft-server/src/main/java/org/apache/raft/server/storage/RaftStorageDirectory.java b/raft-server/src/main/java/org/apache/raft/server/storage/RaftStorageDirectory.java
deleted file mode 100644
index 662e4ec..0000000
--- a/raft-server/src/main/java/org/apache/raft/server/storage/RaftStorageDirectory.java
+++ /dev/null
@@ -1,360 +0,0 @@
-/**
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- *     http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-package org.apache.raft.server.storage;
-
-import com.google.common.annotations.VisibleForTesting;
-import com.google.common.base.Charsets;
-import com.google.common.base.Preconditions;
-import com.google.common.collect.ImmutableList;
-import org.apache.raft.util.AtomicFileOutputStream;
-import org.apache.raft.util.FileUtils;
-import org.slf4j.Logger;
-import org.slf4j.LoggerFactory;
-
-import java.io.File;
-import java.io.IOException;
-import java.io.RandomAccessFile;
-import java.lang.management.ManagementFactory;
-import java.nio.channels.FileLock;
-import java.nio.channels.OverlappingFileLockException;
-import java.nio.file.DirectoryStream;
-import java.nio.file.Files;
-import java.nio.file.Path;
-import java.util.*;
-import java.util.regex.Matcher;
-import java.util.regex.Pattern;
-
-import static java.nio.file.Files.newDirectoryStream;
-import static org.apache.raft.server.impl.RaftServerConstants.INVALID_LOG_INDEX;
-
-public class RaftStorageDirectory {
-  static final Logger LOG = LoggerFactory.getLogger(RaftStorageDirectory.class);
-
-  static final String STORAGE_DIR_CURRENT = "current";
-  static final String STORAGE_FILE_LOCK = "in_use.lock";
-  static final String META_FILE_NAME = "raft-meta";
-  static final String LOG_FILE_INPROGRESS = "inprogress";
-  static final String LOG_FILE_PREFIX = "log";
-  static final String STATE_MACHINE = "sm"; // directory containing state machine snapshots
-  static final String TEMP = "tmp";
-  static final Pattern CLOSED_SEGMENT_REGEX = Pattern.compile("log_(\\d+)-(\\d+)");
-  static final Pattern OPEN_SEGMENT_REGEX = Pattern.compile("log_inprogress_(\\d+)(?:\\..*)?");
-
-  private static final List<Pattern> LOGSEGMENTS_REGEXES =
-      ImmutableList.of(CLOSED_SEGMENT_REGEX, OPEN_SEGMENT_REGEX);
-
-  enum StorageState {
-    NON_EXISTENT,
-    NOT_FORMATTED,
-    NORMAL
-  }
-
-  public static class LogPathAndIndex {
-    public final Path path;
-    public final long startIndex;
-    public final long endIndex;
-
-    LogPathAndIndex(Path path, long startIndex, long endIndex) {
-      this.path = path;
-      this.startIndex = startIndex;
-      this.endIndex = endIndex;
-    }
-
-    @Override
-    public String toString() {
-      return path + "-" + startIndex + "-" + endIndex;
-    }
-  }
-
-  private final File root; // root directory
-  private FileLock lock;   // storage lock
-
-  /**
-   * Constructor
-   * @param dir directory corresponding to the storage
-   */
-  RaftStorageDirectory(File dir) {
-    this.root = dir;
-    this.lock = null;
-  }
-
-  /**
-   * Get root directory of this storage
-   */
-  //TODO
-  public File getRoot() {
-    return root;
-  }
-
-  /**
-   * Clear and re-create storage directory.
-   * <p>
-   * Removes contents of the current directory and creates an empty directory.
-   *
-   * This does not fully format storage directory.
-   * It cannot write the version file since it should be written last after
-   * all other storage type dependent files are written.
-   * Derived storage is responsible for setting specific storage values and
-   * writing the version file to disk.
-   */
-  void clearDirectory() throws IOException {
-    File curDir = this.getCurrentDir();
-    clearDirectory(curDir);
-    clearDirectory(getStateMachineDir());
-  }
-
-  void clearDirectory(File dir) throws IOException {
-    if (dir.exists()) {
-      File[] files = FileUtils.listFiles(dir);
-      LOG.info("Will remove files: " + Arrays.toString(files));
-      if (!(FileUtils.fullyDelete(dir)))
-        throw new IOException("Cannot remove directory: " + dir);
-    }
-    if (!dir.mkdirs())
-      throw new IOException("Cannot create directory " + dir);
-  }
-
-  /**
-   * Directory {@code current} contains latest files defining
-   * the file system meta-data.
-   *
-   * @return the directory path
-   */
-  File getCurrentDir() {
-    return new File(root, STORAGE_DIR_CURRENT);
-  }
-
-  File getMetaFile() {
-    return new File(getCurrentDir(), META_FILE_NAME);
-  }
-
-  File getMetaTmpFile() {
-    return new File(getCurrentDir(), META_FILE_NAME
-        + AtomicFileOutputStream.TMP_EXTENSION);
-  }
-
-  File getOpenLogFile(long startIndex) {
-    return new File(getCurrentDir(), getOpenLogFileName(startIndex));
-  }
-
-  static String getOpenLogFileName(long startIndex) {
-    return LOG_FILE_PREFIX + "_" + LOG_FILE_INPROGRESS + "_" + startIndex;
-  }
-
-  File getClosedLogFile(long startIndex, long endIndex) {
-    return new File(getCurrentDir(), getClosedLogFileName(startIndex, endIndex));
-  }
-
-  static String getClosedLogFileName(long startIndex, long endIndex) {
-    return LOG_FILE_PREFIX + "_" + startIndex + "-" + endIndex;
-  }
-
-  public File getStateMachineDir() {
-    return new File(getRoot(), STATE_MACHINE);
-  }
-
-  /** Returns a uniquely named temporary directory under $rootdir/tmp/ */
-  public File getNewTempDir() {
-    return new File(new File(getRoot(), TEMP), UUID.randomUUID().toString());
-  }
-
-  public Path relativizeToRoot(Path p) {
-    if (p.isAbsolute()) {
-      return getRoot().toPath().relativize(p);
-    }
-    return p;
-  }
-
-  /**
-   * @return log segment files sorted based on their index.
-   */
-  @VisibleForTesting
-  public List<LogPathAndIndex> getLogSegmentFiles() throws IOException {
-    List<LogPathAndIndex> list = new ArrayList<>();
-    try (DirectoryStream<Path> stream =
-             Files.newDirectoryStream(getCurrentDir().toPath())) {
-      for (Path path : stream) {
-        for (Pattern pattern : LOGSEGMENTS_REGEXES) {
-          Matcher matcher = pattern.matcher(path.getFileName().toString());
-          if (matcher.matches()) {
-            final long startIndex = Long.parseLong(matcher.group(1));
-            final long endIndex = matcher.groupCount() == 2 ?
-                Long.parseLong(matcher.group(2)) : INVALID_LOG_INDEX;
-            list.add(new LogPathAndIndex(path, startIndex, endIndex));
-          }
-        }
-      }
-    }
-    Collections.sort(list,
-        (o1, o2) -> o1.startIndex == o2.startIndex ?
-            0 : (o1.startIndex < o2.startIndex ? -1 : 1));
-    return list;
-  }
-
-  /**
-   * Check to see if current/ directory is empty.
-   */
-  boolean isCurrentEmpty() throws IOException {
-    File currentDir = getCurrentDir();
-    if(!currentDir.exists()) {
-      // if current/ does not exist, it's safe to format it.
-      return true;
-    }
-    try(DirectoryStream<Path> dirStream =
-            newDirectoryStream(currentDir.toPath())) {
-      if (dirStream.iterator().hasNext()) {
-        return false;
-      }
-    }
-    return true;
-  }
-
-  /**
-   * Check consistency of the storage directory.
-   *
-   * @return state {@link StorageState} of the storage directory
-   */
-  StorageState analyzeStorage(boolean toLock) throws IOException {
-    Preconditions.checkState(root != null, "root directory is null");
-
-    String rootPath = root.getCanonicalPath();
-    try { // check that storage exists
-      if (!root.exists()) {
-        LOG.info(rootPath + " does not exist. Creating ...");
-        if (!root.mkdirs()) {
-          throw new IOException("Cannot create directory " + rootPath);
-        }
-      }
-      // or is inaccessible
-      if (!root.isDirectory()) {
-        LOG.warn(rootPath + "is not a directory");
-        return StorageState.NON_EXISTENT;
-      }
-      if (!FileUtils.canWrite(root)) {
-        LOG.warn("Cannot access storage directory " + rootPath);
-        return StorageState.NON_EXISTENT;
-      }
-    } catch(SecurityException ex) {
-      LOG.warn("Cannot access storage directory " + rootPath, ex);
-      return StorageState.NON_EXISTENT;
-    }
-
-    if (toLock) {
-      this.lock(); // lock storage if it exists
-    }
-
-    // check whether current directory is valid
-    if (hasMetaFile()) {
-      return StorageState.NORMAL;
-    } else {
-      return StorageState.NOT_FORMATTED;
-    }
-  }
-
-  boolean hasMetaFile() throws IOException {
-    return getMetaFile().exists();
-  }
-
-  /**
-   * Lock storage to provide exclusive access.
-   *
-   * <p> Locking is not supported by all file systems.
-   * E.g., NFS does not consistently support exclusive locks.
-   *
-   * <p> If locking is supported we guarantee exclusive access to the
-   * storage directory. Otherwise, no guarantee is given.
-   *
-   * @throws IOException if locking fails
-   */
-  public void lock() throws IOException {
-    FileLock newLock = tryLock();
-    if (newLock == null) {
-      String msg = "Cannot lock storage " + this.root
-          + ". The directory is already locked";
-      LOG.info(msg);
-      throw new IOException(msg);
-    }
-    // Don't overwrite lock until success - this way if we accidentally
-    // call lock twice, the internal state won't be cleared by the second
-    // (failed) lock attempt
-    lock = newLock;
-  }
-
-  /**
-   * Attempts to acquire an exclusive lock on the storage.
-   *
-   * @return A lock object representing the newly-acquired lock or
-   * <code>null</code> if storage is already locked.
-   * @throws IOException if locking fails.
-   */
-  private FileLock tryLock() throws IOException {
-    boolean deletionHookAdded = false;
-    File lockF = new File(root, STORAGE_FILE_LOCK);
-    if (!lockF.exists()) {
-      lockF.deleteOnExit();
-      deletionHookAdded = true;
-    }
-    RandomAccessFile file = new RandomAccessFile(lockF, "rws");
-    String jvmName = ManagementFactory.getRuntimeMXBean().getName();
-    FileLock res;
-    try {
-      res = file.getChannel().tryLock();
-      if (null == res) {
-        LOG.error("Unable to acquire file lock on path " + lockF.toString());
-        throw new OverlappingFileLockException();
-      }
-      file.write(jvmName.getBytes(Charsets.UTF_8));
-      LOG.info("Lock on " + lockF + " acquired by nodename " + jvmName);
-    } catch (OverlappingFileLockException oe) {
-      // Cannot read from the locked file on Windows.
-      LOG.error("It appears that another process "
-          + "has already locked the storage directory: " + root, oe);
-      file.close();
-      return null;
-    } catch(IOException e) {
-      LOG.error("Failed to acquire lock on " + lockF
-          + ". If this storage directory is mounted via NFS, "
-          + "ensure that the appropriate nfs lock services are running.", e);
-      file.close();
-      throw e;
-    }
-    if (!deletionHookAdded) {
-      // If the file existed prior to our startup, we didn't
-      // call deleteOnExit above. But since we successfully locked
-      // the dir, we can take care of cleaning it up.
-      lockF.deleteOnExit();
-    }
-    return res;
-  }
-
-  /**
-   * Unlock storage.
-   */
-  public void unlock() throws IOException {
-    if (this.lock == null)
-      return;
-    this.lock.release();
-    lock.channel().close();
-    lock = null;
-  }
-
-  @Override
-  public String toString() {
-    return "Storage Directory " + this.root;
-  }
-}



[15/54] [abbrv] incubator-ratis git commit: Small fixes to RaftServer and StateMachine related interfaces after recent refactorings

Posted by ji...@apache.org.
Small fixes to RaftServer and StateMachine related interfaces after recent refactorings


Project: http://git-wip-us.apache.org/repos/asf/incubator-ratis/repo
Commit: http://git-wip-us.apache.org/repos/asf/incubator-ratis/commit/bdd51643
Tree: http://git-wip-us.apache.org/repos/asf/incubator-ratis/tree/bdd51643
Diff: http://git-wip-us.apache.org/repos/asf/incubator-ratis/diff/bdd51643

Branch: refs/heads/master
Commit: bdd516434a39714632dc4e563ed055348ef84c4f
Parents: f58b5ef
Author: Enis Soztutar <en...@apache.org>
Authored: Fri Jan 6 14:15:14 2017 -0800
Committer: Enis Soztutar <en...@apache.org>
Committed: Fri Jan 6 14:15:23 2017 -0800

----------------------------------------------------------------------
 raft-server/pom.xml                                           | 1 -
 .../src/main/java/org/apache/raft/server/RaftServer.java      | 7 +++++++
 .../main/java/org/apache/raft/server/impl/RaftServerImpl.java | 3 ++-
 .../main/java/org/apache/raft/statemachine/StateMachine.java  | 5 +++++
 .../java/org/apache/raft/statemachine/TransactionContext.java | 2 +-
 5 files changed, 15 insertions(+), 3 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-ratis/blob/bdd51643/raft-server/pom.xml
----------------------------------------------------------------------
diff --git a/raft-server/pom.xml b/raft-server/pom.xml
index 584dc9a..d992306 100644
--- a/raft-server/pom.xml
+++ b/raft-server/pom.xml
@@ -30,7 +30,6 @@
     <dependency>
       <artifactId>raft-proto-shaded</artifactId>
       <groupId>com.hortonworks.raft</groupId>
-      <scope>provided</scope>
     </dependency>
 
     <dependency>

http://git-wip-us.apache.org/repos/asf/incubator-ratis/blob/bdd51643/raft-server/src/main/java/org/apache/raft/server/RaftServer.java
----------------------------------------------------------------------
diff --git a/raft-server/src/main/java/org/apache/raft/server/RaftServer.java b/raft-server/src/main/java/org/apache/raft/server/RaftServer.java
index aa4dfbf..bb49c5a 100644
--- a/raft-server/src/main/java/org/apache/raft/server/RaftServer.java
+++ b/raft-server/src/main/java/org/apache/raft/server/RaftServer.java
@@ -20,6 +20,7 @@ package org.apache.raft.server;
 import org.apache.raft.protocol.RaftClientAsynchronousProtocol;
 import org.apache.raft.protocol.RaftClientProtocol;
 import org.apache.raft.server.protocol.RaftServerProtocol;
+import org.apache.raft.statemachine.StateMachine;
 
 import java.io.Closeable;
 
@@ -34,4 +35,10 @@ public interface RaftServer extends Closeable, RaftServerProtocol,
 
   /** Start this server. */
   void start();
+
+  /**
+   * Returns the StateMachine instance.
+   * @return the StateMachine instance.
+   */
+  StateMachine getStateMachine();
 }

http://git-wip-us.apache.org/repos/asf/incubator-ratis/blob/bdd51643/raft-server/src/main/java/org/apache/raft/server/impl/RaftServerImpl.java
----------------------------------------------------------------------
diff --git a/raft-server/src/main/java/org/apache/raft/server/impl/RaftServerImpl.java b/raft-server/src/main/java/org/apache/raft/server/impl/RaftServerImpl.java
index c8a2e28..4b8c442 100644
--- a/raft-server/src/main/java/org/apache/raft/server/impl/RaftServerImpl.java
+++ b/raft-server/src/main/java/org/apache/raft/server/impl/RaftServerImpl.java
@@ -114,7 +114,8 @@ public class RaftServerImpl implements RaftServer {
     return RaftUtils.getRandomBetween(minTimeoutMs, maxTimeoutMs);
   }
 
-  StateMachine getStateMachine() {
+  @Override
+  public StateMachine getStateMachine() {
     return this.stateMachine;
   }
 

http://git-wip-us.apache.org/repos/asf/incubator-ratis/blob/bdd51643/raft-server/src/main/java/org/apache/raft/statemachine/StateMachine.java
----------------------------------------------------------------------
diff --git a/raft-server/src/main/java/org/apache/raft/statemachine/StateMachine.java b/raft-server/src/main/java/org/apache/raft/statemachine/StateMachine.java
index 935a83a..3dedf88 100644
--- a/raft-server/src/main/java/org/apache/raft/statemachine/StateMachine.java
+++ b/raft-server/src/main/java/org/apache/raft/statemachine/StateMachine.java
@@ -31,6 +31,11 @@ import java.io.IOException;
 import java.util.Collection;
 import java.util.concurrent.CompletableFuture;
 
+/**
+ * StateMachine is the entry point for the custom implementation of replicated state as defined in
+ * the "State Machine Approach" in the literature
+ * (see https://en.wikipedia.org/wiki/State_machine_replication).
+ */
 public interface StateMachine extends Closeable {
   /**
    * Initializes the State Machine with the given properties and storage. The state machine is

http://git-wip-us.apache.org/repos/asf/incubator-ratis/blob/bdd51643/raft-server/src/main/java/org/apache/raft/statemachine/TransactionContext.java
----------------------------------------------------------------------
diff --git a/raft-server/src/main/java/org/apache/raft/statemachine/TransactionContext.java b/raft-server/src/main/java/org/apache/raft/statemachine/TransactionContext.java
index 19b8250..675ada9 100644
--- a/raft-server/src/main/java/org/apache/raft/statemachine/TransactionContext.java
+++ b/raft-server/src/main/java/org/apache/raft/statemachine/TransactionContext.java
@@ -97,7 +97,7 @@ public class TransactionContext {
       SMLogEntryProto smLogEntryProto, Object stateMachineContext) {
     this(stateMachine);
     this.clientRequest = Optional.of(clientRequest);
-    this.smLogEntryProto = Optional.of(smLogEntryProto);
+    this.smLogEntryProto = Optional.ofNullable(smLogEntryProto);
     this.stateMachineContext = Optional.ofNullable(stateMachineContext);
   }
 


[49/54] [abbrv] incubator-ratis git commit: Renamed the packages from raft to ratis in preperation for Apache Incubation - Moved all java packages from org.apache.raft to org.apache.ratis. - Moved native package to org_apache_ratis, and native lib to l

Posted by ji...@apache.org.
Renamed the packages from raft to ratis in preperation for Apache Incubation
 - Moved all java packages from org.apache.raft to org.apache.ratis.
 - Moved native package to org_apache_ratis, and native lib to libratis.
 - Moved all modules raft-foo to ratis-foo.
 - Moved artifact classes from com.hortonworks.raft to org.apache.ratis.
 - Individual names for classes (RaftServer, etc) is kept as it is.
 - Usage of "raft" in referring to the protocol is kept as it is.
 - Changed references to previously used name Concur to Ratis.


Project: http://git-wip-us.apache.org/repos/asf/incubator-ratis/repo
Commit: http://git-wip-us.apache.org/repos/asf/incubator-ratis/commit/7e71a2e0
Tree: http://git-wip-us.apache.org/repos/asf/incubator-ratis/tree/7e71a2e0
Diff: http://git-wip-us.apache.org/repos/asf/incubator-ratis/diff/7e71a2e0

Branch: refs/heads/master
Commit: 7e71a2e0e8ffb01a2168fabdf1d2f66f6ba7c4b9
Parents: bdd5164
Author: Enis Soztutar <en...@apache.org>
Authored: Fri Jan 6 15:36:48 2017 -0800
Committer: Enis Soztutar <en...@apache.org>
Committed: Tue Jan 10 15:24:09 2017 -0800

----------------------------------------------------------------------
 README.md                                       |    6 +-
 pom.xml                                         |   14 +-
 raft-client/pom.xml                             |   54 -
 .../java/org/apache/raft/client/RaftClient.java |   49 -
 .../raft/client/RaftClientConfigKeys.java       |   23 -
 .../raft/client/RaftClientRequestSender.java    |   34 -
 .../raft/client/impl/ClientProtoUtils.java      |  131 --
 .../apache/raft/client/impl/RaftClientImpl.java |  171 --
 raft-common/RaftCommon.cmake                    |  208 ---
 raft-common/RaftJNI.cmake                       |   97 -
 raft-common/pom.xml                             |  143 --
 raft-common/src/CMakeLists.txt                  |  108 --
 raft-common/src/config.h.cmake                  |   24 -
 raft-common/src/main/conf/log4j.properties      |   74 -
 .../org/apache/raft/conf/RaftProperties.java    | 1657 -----------------
 .../main/java/org/apache/raft/io/MD5Hash.java   |  205 ---
 .../java/org/apache/raft/io/nativeio/Errno.java |   64 -
 .../org/apache/raft/io/nativeio/NativeIO.java   |  804 ---------
 .../raft/io/nativeio/NativeIOException.java     |   70 -
 .../raft/protocol/AlreadyExistsException.java   |   36 -
 .../apache/raft/protocol/ChecksumException.java |   35 -
 .../java/org/apache/raft/protocol/Message.java  |   30 -
 .../raft/protocol/NotLeaderException.java       |   40 -
 .../RaftClientAsynchronousProtocol.java         |   30 -
 .../raft/protocol/RaftClientProtocol.java       |   26 -
 .../apache/raft/protocol/RaftClientReply.java   |   91 -
 .../apache/raft/protocol/RaftClientRequest.java |   73 -
 .../org/apache/raft/protocol/RaftException.java |   36 -
 .../java/org/apache/raft/protocol/RaftPeer.java |   79 -
 .../apache/raft/protocol/RaftRpcMessage.java    |   33 -
 .../ReconfigurationInProgressException.java     |   24 -
 .../ReconfigurationTimeoutException.java        |   24 -
 .../raft/protocol/SetConfigurationRequest.java  |   39 -
 .../raft/protocol/StateMachineException.java    |   28 -
 .../raft/util/AtomicFileOutputStream.java       |  112 --
 .../org/apache/raft/util/AutoCloseableLock.java |   57 -
 .../org/apache/raft/util/CheckedFunction.java   |   30 -
 .../org/apache/raft/util/CheckedRunnable.java   |   28 -
 .../raft/util/CodeInjectionForTesting.java      |   67 -
 .../main/java/org/apache/raft/util/Daemon.java  |   35 -
 .../java/org/apache/raft/util/ExitUtils.java    |   98 --
 .../java/org/apache/raft/util/FileUtils.java    |  207 ---
 .../java/org/apache/raft/util/LifeCycle.java    |  210 ---
 .../java/org/apache/raft/util/MD5FileUtil.java  |  176 --
 .../org/apache/raft/util/NativeCodeLoader.java  |   70 -
 .../java/org/apache/raft/util/NativeCrc32.java  |  142 --
 .../apache/raft/util/NativeLibraryChecker.java  |   64 -
 .../java/org/apache/raft/util/NetUtils.java     |  151 --
 .../java/org/apache/raft/util/PeerProxyMap.java |  127 --
 .../java/org/apache/raft/util/ProtoUtils.java   |  141 --
 .../org/apache/raft/util/PureJavaCrc32.java     |  619 -------
 .../org/apache/raft/util/PureJavaCrc32C.java    |  632 -------
 .../java/org/apache/raft/util/RaftUtils.java    |  240 ---
 .../java/org/apache/raft/util/StringUtils.java  |  108 --
 .../java/org/apache/raft/util/Timestamp.java    |   84 -
 raft-common/src/main/native/src/exception.c     |  124 --
 raft-common/src/main/native/src/exception.h     |  104 --
 .../src/org/apache/raft/io/nativeio/NativeIO.c  | 1061 -----------
 .../org/apache/raft/io/nativeio/errno_enum.c    |  123 --
 .../org/apache/raft/io/nativeio/errno_enum.h    |   27 -
 .../apache/raft/io/nativeio/file_descriptor.c   |  115 --
 .../apache/raft/io/nativeio/file_descriptor.h   |   36 -
 .../src/org/apache/raft/util/NativeCodeLoader.c |   56 -
 .../src/org/apache/raft/util/NativeCrc32.c      |  276 ---
 .../src/org/apache/raft/util/bulk_crc32.c       |  244 ---
 .../src/org/apache/raft/util/bulk_crc32.h       |   73 -
 .../org/apache/raft/util/bulk_crc32_aarch64.c   |  362 ----
 .../src/org/apache/raft/util/bulk_crc32_x86.c   |  345 ----
 .../raft/util/crc32_zlib_polynomial_tables.h    |  552 ------
 .../src/org/apache/raft/util/crc32c_tables.h    |  550 ------
 .../org/apache/raft/util/gcc_optimizations.h    |   30 -
 .../src/main/native/src/org_apache_raft.h       |  189 --
 .../test/org/apache/raft/util/test_bulk_crc32.c |  113 --
 .../org/apache/raft/util/TestLifeCycle.java     |   53 -
 raft-examples/pom.xml                           |  130 --
 .../arithmetic/ArithmeticStateMachine.java      |  180 --
 .../examples/arithmetic/AssignmentMessage.java  |   83 -
 .../raft/examples/arithmetic/Evaluable.java     |   24 -
 .../arithmetic/expression/BinaryExpression.java |  103 --
 .../arithmetic/expression/DoubleValue.java      |   61 -
 .../arithmetic/expression/Expression.java       |  111 --
 .../arithmetic/expression/NullValue.java        |   55 -
 .../arithmetic/expression/UnaryExpression.java  |   95 -
 .../arithmetic/expression/Variable.java         |  125 --
 .../java/org/apache/raft/TestBatchAppend.java   |  170 --
 .../org/apache/raft/TestRestartRaftPeer.java    |  114 --
 .../raft/examples/RaftExamplesTestUtil.java     |   81 -
 .../examples/arithmetic/TestArithmetic.java     |  104 --
 .../arithmetic/expression/TestExpression.java   |  102 --
 .../TestRaftStateMachineException.java          |   86 -
 .../src/test/resources/log4j.properties         |   18 -
 raft-grpc/pom.xml                               |   93 -
 .../org/apache/raft/grpc/RaftGRpcService.java   |  151 --
 .../apache/raft/grpc/RaftGrpcConfigKeys.java    |   47 -
 .../java/org/apache/raft/grpc/RaftGrpcUtil.java |   87 -
 .../apache/raft/grpc/client/AppendStreamer.java |  395 -----
 .../grpc/client/RaftClientProtocolClient.java   |   73 -
 .../grpc/client/RaftClientProtocolProxy.java    |  104 --
 .../grpc/client/RaftClientProtocolService.java  |  194 --
 .../grpc/client/RaftClientSenderWithGrpc.java   |  119 --
 .../raft/grpc/client/RaftOutputStream.java      |  112 --
 .../raft/grpc/server/GRpcLogAppender.java       |  415 -----
 .../server/PipelinedLogAppenderFactory.java     |   32 -
 .../grpc/server/RaftServerProtocolClient.java   |   63 -
 .../grpc/server/RaftServerProtocolService.java  |  111 --
 .../raft/grpc/MiniRaftClusterWithGRpc.java      |  135 --
 .../grpc/TestNotLeaderExceptionWithGrpc.java    |   39 -
 .../grpc/TestRaftReconfigurationWithGRpc.java   |   47 -
 .../raft/grpc/TestRaftSnapshotWithGrpc.java     |   32 -
 .../org/apache/raft/grpc/TestRaftStream.java    |  319 ----
 .../org/apache/raft/grpc/TestRaftWithGrpc.java  |   73 -
 raft-grpc/src/test/resources/log4j.properties   |   18 -
 raft-hadoop/pom.xml                             |   99 --
 .../hadoop/ipc/ProtobufRpcEngineShaded.java     |  623 -------
 .../apache/raft/hadooprpc/HadoopConstants.java  |   29 -
 .../java/org/apache/raft/hadooprpc/Proxy.java   |   55 -
 .../client/HadoopClientRequestSender.java       |   68 -
 ...aftClientProtocolClientSideTranslatorPB.java |   70 -
 .../hadooprpc/client/RaftClientProtocolPB.java  |   37 -
 ...aftClientProtocolServerSideTranslatorPB.java |   69 -
 .../raft/hadooprpc/server/HadoopRpcService.java |  178 --
 .../hadooprpc/server/RaftServerProtocolPB.java  |   37 -
 ...aftServerProtocolServerSideTranslatorPB.java |   68 -
 .../hadooprpc/MiniRaftClusterWithHadoopRpc.java |  117 --
 .../TestNotLeaderExceptionWithHadoopRpc.java    |   37 -
 .../TestRaftReconfigurationWithHadoopRpc.java   |   40 -
 .../TestRaftSnapshotWithHadoopRpc.java          |   32 -
 .../raft/hadooprpc/TestRaftWithHadoopRpc.java   |   69 -
 raft-hadoop/src/test/resources/log4j.properties |   18 -
 raft-netty/pom.xml                              |   99 --
 .../java/org/apache/raft/netty/NettyClient.java |   72 -
 .../org/apache/raft/netty/NettyRpcProxy.java    |  180 --
 .../netty/client/NettyClientRequestSender.java  |   71 -
 .../raft/netty/server/NettyRpcService.java      |  253 ---
 .../raft/netty/MiniRaftClusterWithNetty.java    |  117 --
 .../netty/TestNotLeaderExceptionWithNetty.java  |   33 -
 .../netty/TestRaftReconfigurationWithNetty.java |   31 -
 .../raft/netty/TestRaftSnapshotWithNetty.java   |   32 -
 .../apache/raft/netty/TestRaftWithNetty.java    |   62 -
 raft-netty/src/test/resources/log4j.properties  |   18 -
 raft-project-dist/pom.xml                       |  169 --
 raft-project/pom.xml                            |  409 -----
 raft-proto-shaded/.gitignore                    |    2 -
 raft-proto-shaded/README.md                     |   23 -
 raft-proto-shaded/pom.xml                       |  426 -----
 raft-proto-shaded/src/main/proto/GRpc.proto     |   45 -
 raft-proto-shaded/src/main/proto/Hadoop.proto   |   44 -
 raft-proto-shaded/src/main/proto/Netty.proto    |   49 -
 raft-proto-shaded/src/main/proto/Raft.proto     |  165 --
 ...e.raft.shaded.io.grpc.ManagedChannelProvider |   16 -
 ...che.raft.shaded.io.grpc.NameResolverProvider |   16 -
 ...rg.apache.raft.shaded.io.grpc.ServerProvider |   16 -
 raft-server/pom.xml                             |   80 -
 .../java/org/apache/raft/server/RaftServer.java |   44 -
 .../raft/server/RaftServerConfigKeys.java       |  150 --
 .../org/apache/raft/server/RaftServerRpc.java   |   39 -
 .../raft/server/impl/ConfigurationManager.java  |   91 -
 .../apache/raft/server/impl/FollowerInfo.java   |  103 --
 .../apache/raft/server/impl/FollowerState.java  |   91 -
 .../apache/raft/server/impl/LeaderElection.java |  241 ---
 .../apache/raft/server/impl/LeaderState.java    |  582 ------
 .../apache/raft/server/impl/LogAppender.java    |  480 -----
 .../raft/server/impl/LogAppenderFactory.java    |   31 -
 .../raft/server/impl/PeerConfiguration.java     |   90 -
 .../apache/raft/server/impl/PendingRequest.java |   87 -
 .../raft/server/impl/PendingRequests.java       |  129 --
 .../raft/server/impl/RaftConfiguration.java     |  261 ---
 .../raft/server/impl/RaftServerConstants.java   |   46 -
 .../apache/raft/server/impl/RaftServerImpl.java |  813 ---------
 .../raft/server/impl/ServerProtoUtils.java      |  179 --
 .../apache/raft/server/impl/ServerState.java    |  345 ----
 .../apache/raft/server/impl/ServerUtils.java    |   81 -
 .../raft/server/impl/StateMachineUpdater.java   |  213 ---
 .../server/protocol/RaftServerProtocol.java     |   31 -
 .../apache/raft/server/protocol/TermIndex.java  |   36 -
 .../server/storage/BufferedChannelBase.java     |   52 -
 .../server/storage/BufferedWriteChannel.java    |  159 --
 .../apache/raft/server/storage/FileInfo.java    |   59 -
 .../raft/server/storage/LogInputStream.java     |  258 ---
 .../raft/server/storage/LogOutputStream.java    |  176 --
 .../apache/raft/server/storage/LogReader.java   |  292 ---
 .../apache/raft/server/storage/LogSegment.java  |  232 ---
 .../raft/server/storage/MemoryRaftLog.java      |  182 --
 .../apache/raft/server/storage/MetaFile.java    |  130 --
 .../org/apache/raft/server/storage/RaftLog.java |  292 ---
 .../raft/server/storage/RaftLogCache.java       |  323 ----
 .../raft/server/storage/RaftLogWorker.java      |  371 ----
 .../apache/raft/server/storage/RaftStorage.java |  144 --
 .../server/storage/RaftStorageDirectory.java    |  360 ----
 .../raft/server/storage/SegmentedRaftLog.java   |  327 ----
 .../raft/server/storage/SnapshotManager.java    |  133 --
 .../raft/statemachine/BaseStateMachine.java     |  150 --
 .../raft/statemachine/FileListSnapshotInfo.java |   64 -
 .../statemachine/SimpleStateMachineStorage.java |  134 --
 .../statemachine/SingleFileSnapshotInfo.java    |   38 -
 .../apache/raft/statemachine/SnapshotInfo.java  |   58 -
 .../apache/raft/statemachine/StateMachine.java  |  168 --
 .../raft/statemachine/StateMachineStorage.java  |   39 -
 .../raft/statemachine/TransactionContext.java   |  210 ---
 .../java/org/apache/raft/MiniRaftCluster.java   |  444 -----
 .../java/org/apache/raft/RaftBasicTests.java    |  199 ---
 .../raft/RaftNotLeaderExceptionBaseTest.java    |  162 --
 .../test/java/org/apache/raft/RaftTestUtil.java |  305 ----
 .../impl/BlockRequestHandlingInjection.java     |   84 -
 .../impl/DelayLocalExecutionInjection.java      |   67 -
 .../impl/RaftReconfigurationBaseTest.java       |  577 ------
 .../raft/server/impl/RaftServerTestUtil.java    |   73 -
 .../MiniRaftClusterWithSimulatedRpc.java        |  136 --
 .../raft/server/simulation/RaftServerReply.java |   99 --
 .../server/simulation/RaftServerRequest.java    |   98 --
 .../raft/server/simulation/RequestHandler.java  |  134 --
 .../simulation/SimulatedClientRequestReply.java |   45 -
 .../simulation/SimulatedRequestReply.java       |  201 ---
 .../server/simulation/SimulatedServerRpc.java   |  167 --
 .../TestNotLeaderExceptionWithSimulation.java   |   32 -
 ...TestRaftReconfigurationWithSimulatedRpc.java |   31 -
 .../TestRaftSnapshotWithSimulatedRpc.java       |   32 -
 .../simulation/TestRaftWithSimulatedRpc.java    |   51 -
 .../raft/server/storage/TestRaftLogCache.java   |  255 ---
 .../server/storage/TestRaftLogReadWrite.java    |  266 ---
 .../raft/server/storage/TestRaftLogSegment.java |  303 ----
 .../raft/server/storage/TestRaftStorage.java    |  212 ---
 .../server/storage/TestSegmentedRaftLog.java    |  329 ----
 .../raft/statemachine/RaftSnapshotBaseTest.java |  215 ---
 .../SimpleStateMachine4Testing.java             |  245 ---
 .../raft/statemachine/TermIndexTracker.java     |   66 -
 .../raft/statemachine/TestStateMachine.java     |  186 --
 raft-server/src/test/resources/log4j.properties |   18 -
 ratis-client/pom.xml                            |   54 +
 .../org/apache/ratis/client/RaftClient.java     |   49 +
 .../ratis/client/RaftClientConfigKeys.java      |   23 +
 .../ratis/client/RaftClientRequestSender.java   |   34 +
 .../ratis/client/impl/ClientProtoUtils.java     |  131 ++
 .../ratis/client/impl/RaftClientImpl.java       |  172 ++
 ratis-common/RatisCommon.cmake                  |  208 +++
 ratis-common/RatisJNI.cmake                     |   97 +
 ratis-common/pom.xml                            |  143 ++
 ratis-common/src/CMakeLists.txt                 |  108 ++
 ratis-common/src/config.h.cmake                 |   24 +
 ratis-common/src/main/conf/log4j.properties     |   74 +
 .../org/apache/ratis/conf/RaftProperties.java   | 1658 ++++++++++++++++++
 .../main/java/org/apache/ratis/io/MD5Hash.java  |  205 +++
 .../org/apache/ratis/io/nativeio/Errno.java     |   64 +
 .../org/apache/ratis/io/nativeio/NativeIO.java  |  805 +++++++++
 .../ratis/io/nativeio/NativeIOException.java    |   70 +
 .../ratis/protocol/AlreadyExistsException.java  |   36 +
 .../ratis/protocol/ChecksumException.java       |   35 +
 .../java/org/apache/ratis/protocol/Message.java |   30 +
 .../ratis/protocol/NotLeaderException.java      |   40 +
 .../RaftClientAsynchronousProtocol.java         |   30 +
 .../ratis/protocol/RaftClientProtocol.java      |   26 +
 .../apache/ratis/protocol/RaftClientReply.java  |   91 +
 .../ratis/protocol/RaftClientRequest.java       |   73 +
 .../apache/ratis/protocol/RaftException.java    |   36 +
 .../org/apache/ratis/protocol/RaftPeer.java     |   79 +
 .../apache/ratis/protocol/RaftRpcMessage.java   |   33 +
 .../ReconfigurationInProgressException.java     |   24 +
 .../ReconfigurationTimeoutException.java        |   24 +
 .../ratis/protocol/SetConfigurationRequest.java |   39 +
 .../ratis/protocol/StateMachineException.java   |   28 +
 .../ratis/util/AtomicFileOutputStream.java      |  112 ++
 .../apache/ratis/util/AutoCloseableLock.java    |   57 +
 .../org/apache/ratis/util/CheckedFunction.java  |   30 +
 .../org/apache/ratis/util/CheckedRunnable.java  |   28 +
 .../ratis/util/CodeInjectionForTesting.java     |   67 +
 .../main/java/org/apache/ratis/util/Daemon.java |   35 +
 .../java/org/apache/ratis/util/ExitUtils.java   |   98 ++
 .../java/org/apache/ratis/util/FileUtils.java   |  207 +++
 .../java/org/apache/ratis/util/LifeCycle.java   |  210 +++
 .../java/org/apache/ratis/util/MD5FileUtil.java |  177 ++
 .../org/apache/ratis/util/NativeCodeLoader.java |   70 +
 .../java/org/apache/ratis/util/NativeCrc32.java |  143 ++
 .../apache/ratis/util/NativeLibraryChecker.java |   64 +
 .../java/org/apache/ratis/util/NetUtils.java    |  151 ++
 .../org/apache/ratis/util/PeerProxyMap.java     |  128 ++
 .../java/org/apache/ratis/util/ProtoUtils.java  |  147 ++
 .../org/apache/ratis/util/PureJavaCrc32.java    |  619 +++++++
 .../org/apache/ratis/util/PureJavaCrc32C.java   |  632 +++++++
 .../java/org/apache/ratis/util/RaftUtils.java   |  240 +++
 .../java/org/apache/ratis/util/StringUtils.java |  108 ++
 .../java/org/apache/ratis/util/Timestamp.java   |   84 +
 ratis-common/src/main/native/src/exception.c    |  124 ++
 ratis-common/src/main/native/src/exception.h    |  104 ++
 .../src/org/apache/ratis/io/nativeio/NativeIO.c | 1061 +++++++++++
 .../org/apache/ratis/io/nativeio/errno_enum.c   |  123 ++
 .../org/apache/ratis/io/nativeio/errno_enum.h   |   27 +
 .../apache/ratis/io/nativeio/file_descriptor.c  |  115 ++
 .../apache/ratis/io/nativeio/file_descriptor.h  |   36 +
 .../org/apache/ratis/util/NativeCodeLoader.c    |   56 +
 .../src/org/apache/ratis/util/NativeCrc32.c     |  276 +++
 .../src/org/apache/ratis/util/bulk_crc32.c      |  244 +++
 .../src/org/apache/ratis/util/bulk_crc32.h      |   73 +
 .../org/apache/ratis/util/bulk_crc32_aarch64.c  |  362 ++++
 .../src/org/apache/ratis/util/bulk_crc32_x86.c  |  345 ++++
 .../ratis/util/crc32_zlib_polynomial_tables.h   |  552 ++++++
 .../src/org/apache/ratis/util/crc32c_tables.h   |  550 ++++++
 .../org/apache/ratis/util/gcc_optimizations.h   |   30 +
 .../src/main/native/src/org_apache_ratis.h      |  189 ++
 .../org/apache/ratis/util/test_bulk_crc32.c     |  113 ++
 .../org/apache/ratis/util/TestLifeCycle.java    |   54 +
 ratis-examples/pom.xml                          |  130 ++
 .../arithmetic/ArithmeticStateMachine.java      |  181 ++
 .../examples/arithmetic/AssignmentMessage.java  |   83 +
 .../ratis/examples/arithmetic/Evaluable.java    |   24 +
 .../arithmetic/expression/BinaryExpression.java |  103 ++
 .../arithmetic/expression/DoubleValue.java      |   61 +
 .../arithmetic/expression/Expression.java       |  112 ++
 .../arithmetic/expression/NullValue.java        |   55 +
 .../arithmetic/expression/UnaryExpression.java  |   95 +
 .../arithmetic/expression/Variable.java         |  126 ++
 .../java/org/apache/ratis/TestBatchAppend.java  |  172 ++
 .../org/apache/ratis/TestRestartRaftPeer.java   |  116 ++
 .../ratis/examples/RaftExamplesTestUtil.java    |   81 +
 .../examples/arithmetic/TestArithmetic.java     |  106 ++
 .../arithmetic/expression/TestExpression.java   |  107 ++
 .../TestRaftStateMachineException.java          |   88 +
 .../src/test/resources/log4j.properties         |   18 +
 ratis-grpc/pom.xml                              |   93 +
 .../org/apache/ratis/grpc/RaftGRpcService.java  |  152 ++
 .../apache/ratis/grpc/RaftGrpcConfigKeys.java   |   47 +
 .../org/apache/ratis/grpc/RaftGrpcUtil.java     |   87 +
 .../ratis/grpc/client/AppendStreamer.java       |  396 +++++
 .../grpc/client/RaftClientProtocolClient.java   |   73 +
 .../grpc/client/RaftClientProtocolProxy.java    |  104 ++
 .../grpc/client/RaftClientProtocolService.java  |  195 ++
 .../grpc/client/RaftClientSenderWithGrpc.java   |  119 ++
 .../ratis/grpc/client/RaftOutputStream.java     |  112 ++
 .../ratis/grpc/server/GRpcLogAppender.java      |  416 +++++
 .../server/PipelinedLogAppenderFactory.java     |   32 +
 .../grpc/server/RaftServerProtocolClient.java   |   63 +
 .../grpc/server/RaftServerProtocolService.java  |  116 ++
 .../ratis/grpc/MiniRaftClusterWithGRpc.java     |  138 ++
 .../grpc/TestNotLeaderExceptionWithGrpc.java    |   39 +
 .../grpc/TestRaftReconfigurationWithGRpc.java   |   47 +
 .../ratis/grpc/TestRaftSnapshotWithGrpc.java    |   32 +
 .../org/apache/ratis/grpc/TestRaftStream.java   |  319 ++++
 .../org/apache/ratis/grpc/TestRaftWithGrpc.java |   73 +
 ratis-grpc/src/test/resources/log4j.properties  |   18 +
 ratis-hadoop/pom.xml                            |   99 ++
 .../hadoop/ipc/ProtobufRpcEngineShaded.java     |  623 +++++++
 .../apache/ratis/hadooprpc/HadoopConstants.java |   29 +
 .../java/org/apache/ratis/hadooprpc/Proxy.java  |   55 +
 .../client/HadoopClientRequestSender.java       |   68 +
 ...aftClientProtocolClientSideTranslatorPB.java |   70 +
 .../hadooprpc/client/RaftClientProtocolPB.java  |   37 +
 ...aftClientProtocolServerSideTranslatorPB.java |   69 +
 .../hadooprpc/server/HadoopRpcService.java      |  184 ++
 .../hadooprpc/server/RaftServerProtocolPB.java  |   37 +
 ...aftServerProtocolServerSideTranslatorPB.java |   73 +
 .../hadooprpc/MiniRaftClusterWithHadoopRpc.java |  117 ++
 .../TestNotLeaderExceptionWithHadoopRpc.java    |   37 +
 .../TestRaftReconfigurationWithHadoopRpc.java   |   40 +
 .../TestRaftSnapshotWithHadoopRpc.java          |   32 +
 .../ratis/hadooprpc/TestRaftWithHadoopRpc.java  |   69 +
 .../src/test/resources/log4j.properties         |   18 +
 ratis-netty/pom.xml                             |   99 ++
 .../org/apache/ratis/netty/NettyClient.java     |   73 +
 .../org/apache/ratis/netty/NettyRpcProxy.java   |  186 ++
 .../netty/client/NettyClientRequestSender.java  |   71 +
 .../ratis/netty/server/NettyRpcService.java     |  254 +++
 .../ratis/netty/MiniRaftClusterWithNetty.java   |  117 ++
 .../netty/TestNotLeaderExceptionWithNetty.java  |   33 +
 .../netty/TestRaftReconfigurationWithNetty.java |   31 +
 .../ratis/netty/TestRaftSnapshotWithNetty.java  |   32 +
 .../apache/ratis/netty/TestRaftWithNetty.java   |   62 +
 ratis-netty/src/test/resources/log4j.properties |   18 +
 ratis-project-dist/pom.xml                      |  169 ++
 ratis-project/pom.xml                           |  409 +++++
 ratis-proto-shaded/.gitignore                   |    2 +
 ratis-proto-shaded/README.md                    |   23 +
 ratis-proto-shaded/pom.xml                      |  426 +++++
 ratis-proto-shaded/src/main/proto/GRpc.proto    |   45 +
 ratis-proto-shaded/src/main/proto/Hadoop.proto  |   44 +
 ratis-proto-shaded/src/main/proto/Netty.proto   |   49 +
 ratis-proto-shaded/src/main/proto/Raft.proto    |  165 ++
 ....ratis.shaded.io.grpc.ManagedChannelProvider |   16 +
 ...he.ratis.shaded.io.grpc.NameResolverProvider |   16 +
 ...g.apache.ratis.shaded.io.grpc.ServerProvider |   16 +
 ratis-server/pom.xml                            |   80 +
 .../org/apache/ratis/server/RaftServer.java     |   44 +
 .../ratis/server/RaftServerConfigKeys.java      |  150 ++
 .../org/apache/ratis/server/RaftServerRpc.java  |   39 +
 .../ratis/server/impl/ConfigurationManager.java |   91 +
 .../apache/ratis/server/impl/FollowerInfo.java  |  103 ++
 .../apache/ratis/server/impl/FollowerState.java |   91 +
 .../ratis/server/impl/LeaderElection.java       |  247 +++
 .../apache/ratis/server/impl/LeaderState.java   |  601 +++++++
 .../apache/ratis/server/impl/LogAppender.java   |  494 ++++++
 .../ratis/server/impl/LogAppenderFactory.java   |   31 +
 .../ratis/server/impl/PeerConfiguration.java    |   91 +
 .../ratis/server/impl/PendingRequest.java       |   88 +
 .../ratis/server/impl/PendingRequests.java      |  130 ++
 .../ratis/server/impl/RaftConfiguration.java    |  262 +++
 .../ratis/server/impl/RaftServerConstants.java  |   46 +
 .../ratis/server/impl/RaftServerImpl.java       |  832 +++++++++
 .../ratis/server/impl/ServerProtoUtils.java     |  191 ++
 .../apache/ratis/server/impl/ServerState.java   |  350 ++++
 .../apache/ratis/server/impl/ServerUtils.java   |   81 +
 .../ratis/server/impl/StateMachineUpdater.java  |  214 +++
 .../server/protocol/RaftServerProtocol.java     |   36 +
 .../apache/ratis/server/protocol/TermIndex.java |   36 +
 .../server/storage/BufferedChannelBase.java     |   52 +
 .../server/storage/BufferedWriteChannel.java    |  159 ++
 .../apache/ratis/server/storage/FileInfo.java   |   59 +
 .../ratis/server/storage/LogInputStream.java    |  259 +++
 .../ratis/server/storage/LogOutputStream.java   |  181 ++
 .../apache/ratis/server/storage/LogReader.java  |  302 ++++
 .../apache/ratis/server/storage/LogSegment.java |  233 +++
 .../ratis/server/storage/MemoryRaftLog.java     |  183 ++
 .../apache/ratis/server/storage/MetaFile.java   |  131 ++
 .../apache/ratis/server/storage/RaftLog.java    |  293 ++++
 .../ratis/server/storage/RaftLogCache.java      |  328 ++++
 .../ratis/server/storage/RaftLogWorker.java     |  372 ++++
 .../ratis/server/storage/RaftStorage.java       |  145 ++
 .../server/storage/RaftStorageDirectory.java    |  361 ++++
 .../ratis/server/storage/SegmentedRaftLog.java  |  328 ++++
 .../ratis/server/storage/SnapshotManager.java   |  134 ++
 .../ratis/statemachine/BaseStateMachine.java    |  151 ++
 .../statemachine/FileListSnapshotInfo.java      |   64 +
 .../statemachine/SimpleStateMachineStorage.java |  135 ++
 .../statemachine/SingleFileSnapshotInfo.java    |   38 +
 .../apache/ratis/statemachine/SnapshotInfo.java |   58 +
 .../apache/ratis/statemachine/StateMachine.java |  168 ++
 .../ratis/statemachine/StateMachineStorage.java |   40 +
 .../ratis/statemachine/TransactionContext.java  |  210 +++
 .../java/org/apache/ratis/MiniRaftCluster.java  |  445 +++++
 .../java/org/apache/ratis/RaftBasicTests.java   |  199 +++
 .../ratis/RaftNotLeaderExceptionBaseTest.java   |  162 ++
 .../java/org/apache/ratis/RaftTestUtil.java     |  306 ++++
 .../impl/BlockRequestHandlingInjection.java     |   85 +
 .../impl/DelayLocalExecutionInjection.java      |   67 +
 .../impl/RaftReconfigurationBaseTest.java       |  581 ++++++
 .../ratis/server/impl/RaftServerTestUtil.java   |   75 +
 .../MiniRaftClusterWithSimulatedRpc.java        |  136 ++
 .../server/simulation/RaftServerReply.java      |  100 ++
 .../server/simulation/RaftServerRequest.java    |   98 ++
 .../ratis/server/simulation/RequestHandler.java |  134 ++
 .../simulation/SimulatedClientRequestReply.java |   45 +
 .../simulation/SimulatedRequestReply.java       |  202 +++
 .../server/simulation/SimulatedServerRpc.java   |  175 ++
 .../TestNotLeaderExceptionWithSimulation.java   |   32 +
 ...TestRaftReconfigurationWithSimulatedRpc.java |   31 +
 .../TestRaftSnapshotWithSimulatedRpc.java       |   32 +
 .../simulation/TestRaftWithSimulatedRpc.java    |   51 +
 .../ratis/server/storage/TestRaftLogCache.java  |  255 +++
 .../server/storage/TestRaftLogReadWrite.java    |  269 +++
 .../server/storage/TestRaftLogSegment.java      |  305 ++++
 .../ratis/server/storage/TestRaftStorage.java   |  215 +++
 .../server/storage/TestSegmentedRaftLog.java    |  329 ++++
 .../statemachine/RaftSnapshotBaseTest.java      |  214 +++
 .../SimpleStateMachine4Testing.java             |  246 +++
 .../ratis/statemachine/TermIndexTracker.java    |   67 +
 .../ratis/statemachine/TestStateMachine.java    |  192 ++
 .../src/test/resources/log4j.properties         |   18 +
 454 files changed, 34514 insertions(+), 34302 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-ratis/blob/7e71a2e0/README.md
----------------------------------------------------------------------
diff --git a/README.md b/README.md
index 3c2320b..1ec276d 100644
--- a/README.md
+++ b/README.md
@@ -1,8 +1,8 @@
-# Concur
-Concur is a java library that implements the RAFT protocol[1]. The Raft paper can be accessed at [this link] (https://raft.github.io/raft.pdf). The paper introduces Raft and states its motivations in following words:
+# Ratis
+Ratis is a java library that implements the RAFT protocol[1]. The Raft paper can be accessed at [this link] (https://raft.github.io/raft.pdf). The paper introduces Raft and states its motivations in following words:
   > Raft is a consensus algorithm for managing a replicated log. It produces a result equivalent to (multi-)Paxos, and it is as efficient as Paxos, but its structure is different from Paxos; this makes Raft more understandable than Paxos and also provides a better foundation for building practical systems.
 
-  Concur aims to make raft available as a java library that can be used by any system that needs to use a replicated log. Concur provides pluggability for state machine implementations to manage replicated states. It also provides pluggability for Raft log, and rpc implementations to make it easy for integration with other projects. Another important goal is to support high throughput data ingest so that it can be used for more general data replication use cases.
+  Ratis aims to make raft available as a java library that can be used by any system that needs to use a replicated log. It provides pluggability for state machine implementations to manage replicated states. It also provides pluggability for Raft log, and rpc implementations to make it easy for integration with other projects. Another important goal is to support high throughput data ingest so that it can be used for more general data replication use cases.
 
 
 # Reference

http://git-wip-us.apache.org/repos/asf/incubator-ratis/blob/7e71a2e0/pom.xml
----------------------------------------------------------------------
diff --git a/pom.xml b/pom.xml
index e40e132..9ad746d 100644
--- a/pom.xml
+++ b/pom.xml
@@ -17,10 +17,10 @@
          xsi:schemaLocation="http://maven.apache.org/POM/4.0.0 http://maven.apache.org/xsd/maven-4.0.0.xsd">
   <modelVersion>4.0.0</modelVersion>
 
-  <artifactId>raft-main</artifactId>
-  <groupId>com.hortonworks.raft</groupId>
+  <artifactId>ratis-main</artifactId>
+  <groupId>org.apache.ratis</groupId>
   <version>1.0-SNAPSHOT</version>
-  <name>Raft Main</name>
+  <name>Ratis Main</name>
   <packaging>pom</packaging>
 
   <licenses>
@@ -63,8 +63,8 @@
   </properties>
 
   <modules>
-    <module>raft-proto-shaded</module>
-    <module>raft-project</module>
+    <module>ratis-proto-shaded</module>
+    <module>ratis-project</module>
   </modules>
 
   <build>
@@ -142,8 +142,8 @@
             <forkedProcessTimeoutInSeconds>600</forkedProcessTimeoutInSeconds>
             <argLine>-Xmx2048m -XX:+HeapDumpOnOutOfMemoryError</argLine>
             <systemPropertyVariables>
-              <raft.log.dir>${project.build.directory}/log</raft.log.dir>
-              <raft.tmp.dir>${project.build.directory}/tmp</raft.tmp.dir>
+              <ratis.log.dir>${project.build.directory}/log</ratis.log.dir>
+              <ratis.tmp.dir>${project.build.directory}/tmp</ratis.tmp.dir>
 
               <java.net.preferIPv4Stack>true</java.net.preferIPv4Stack>
             </systemPropertyVariables>

http://git-wip-us.apache.org/repos/asf/incubator-ratis/blob/7e71a2e0/raft-client/pom.xml
----------------------------------------------------------------------
diff --git a/raft-client/pom.xml b/raft-client/pom.xml
deleted file mode 100644
index 7d63931..0000000
--- a/raft-client/pom.xml
+++ /dev/null
@@ -1,54 +0,0 @@
-<?xml version="1.0" encoding="UTF-8"?>
-<!--
-  Licensed 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. See accompanying LICENSE file.
--->
-<project xmlns="http://maven.apache.org/POM/4.0.0"
-         xmlns:xsi="http://www.w3.org/2001/XMLSchema-instance"
-         xsi:schemaLocation="http://maven.apache.org/POM/4.0.0 http://maven.apache.org/xsd/maven-4.0.0.xsd">
-  <modelVersion>4.0.0</modelVersion>
-  <parent>
-    <artifactId>raft-project-dist</artifactId>
-    <groupId>com.hortonworks.raft</groupId>
-    <version>1.0-SNAPSHOT</version>
-    <relativePath>../raft-project-dist</relativePath>
-  </parent>
-
-  <artifactId>raft-client</artifactId>
-  <name>Raft Client</name>
-
-  <dependencies>
-    <dependency>
-      <artifactId>raft-proto-shaded</artifactId>
-      <groupId>com.hortonworks.raft</groupId>
-      <scope>provided</scope>
-    </dependency>
-
-    <dependency>
-      <artifactId>raft-common</artifactId>
-      <groupId>com.hortonworks.raft</groupId>
-      <scope>provided</scope>
-    </dependency>
-    
-    <dependency>
-      <groupId>org.slf4j</groupId>
-      <artifactId>slf4j-api</artifactId>
-      <scope>compile</scope>
-    </dependency>
-
-    <dependency>
-      <groupId>com.google.guava</groupId>
-      <artifactId>guava</artifactId>
-      <scope>compile</scope>
-    </dependency>
-  </dependencies>
-</project>

http://git-wip-us.apache.org/repos/asf/incubator-ratis/blob/7e71a2e0/raft-client/src/main/java/org/apache/raft/client/RaftClient.java
----------------------------------------------------------------------
diff --git a/raft-client/src/main/java/org/apache/raft/client/RaftClient.java b/raft-client/src/main/java/org/apache/raft/client/RaftClient.java
deleted file mode 100644
index d67eded..0000000
--- a/raft-client/src/main/java/org/apache/raft/client/RaftClient.java
+++ /dev/null
@@ -1,49 +0,0 @@
-/**
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- *     http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-package org.apache.raft.client;
-
-import org.apache.raft.protocol.Message;
-import org.apache.raft.protocol.RaftClientReply;
-import org.apache.raft.protocol.RaftPeer;
-import org.slf4j.Logger;
-import org.slf4j.LoggerFactory;
-
-import java.io.Closeable;
-import java.io.IOException;
-
-/** A client who sends requests to a raft service. */
-public interface RaftClient extends Closeable {
-  Logger LOG = LoggerFactory.getLogger(RaftClient.class);
-  long DEFAULT_SEQNUM = 0;
-
-  /** @return the id of this client. */
-  String getId();
-
-  /**
-   * Send the given message to the raft service.
-   * The message may change the state of the service.
-   * For readonly messages, use {@link #sendReadOnly(Message)} instead.
-   */
-  RaftClientReply send(Message message) throws IOException;
-
-  /** Send the given readonly message to the raft service. */
-  RaftClientReply sendReadOnly(Message message) throws IOException;
-
-  /** Send set configuration request to the raft service. */
-  RaftClientReply setConfiguration(RaftPeer[] peersInNewConf) throws IOException;
-}

http://git-wip-us.apache.org/repos/asf/incubator-ratis/blob/7e71a2e0/raft-client/src/main/java/org/apache/raft/client/RaftClientConfigKeys.java
----------------------------------------------------------------------
diff --git a/raft-client/src/main/java/org/apache/raft/client/RaftClientConfigKeys.java b/raft-client/src/main/java/org/apache/raft/client/RaftClientConfigKeys.java
deleted file mode 100644
index 7e93d08..0000000
--- a/raft-client/src/main/java/org/apache/raft/client/RaftClientConfigKeys.java
+++ /dev/null
@@ -1,23 +0,0 @@
-/**
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- *     http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-package org.apache.raft.client;
-
-public interface RaftClientConfigKeys {
-  String RAFT_RPC_TIMEOUT_MS_KEY = "raft.rpc.timeout.ms";
-  int RAFT_RPC_TIMEOUT_MS_DEFAULT = 300;
-}

http://git-wip-us.apache.org/repos/asf/incubator-ratis/blob/7e71a2e0/raft-client/src/main/java/org/apache/raft/client/RaftClientRequestSender.java
----------------------------------------------------------------------
diff --git a/raft-client/src/main/java/org/apache/raft/client/RaftClientRequestSender.java b/raft-client/src/main/java/org/apache/raft/client/RaftClientRequestSender.java
deleted file mode 100644
index aca0840..0000000
--- a/raft-client/src/main/java/org/apache/raft/client/RaftClientRequestSender.java
+++ /dev/null
@@ -1,34 +0,0 @@
-/**
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- *     http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-package org.apache.raft.client;
-
-import org.apache.raft.protocol.RaftClientReply;
-import org.apache.raft.protocol.RaftClientRequest;
-import org.apache.raft.protocol.RaftPeer;
-
-import java.io.Closeable;
-import java.io.IOException;
-
-/** Send requests to a raft service. */
-public interface RaftClientRequestSender extends Closeable {
-  /** Send a request. */
-  RaftClientReply sendRequest(RaftClientRequest request) throws IOException;
-
-  /** Add the information of the given raft servers */
-  void addServers(Iterable<RaftPeer> servers);
-}

http://git-wip-us.apache.org/repos/asf/incubator-ratis/blob/7e71a2e0/raft-client/src/main/java/org/apache/raft/client/impl/ClientProtoUtils.java
----------------------------------------------------------------------
diff --git a/raft-client/src/main/java/org/apache/raft/client/impl/ClientProtoUtils.java b/raft-client/src/main/java/org/apache/raft/client/impl/ClientProtoUtils.java
deleted file mode 100644
index f909418..0000000
--- a/raft-client/src/main/java/org/apache/raft/client/impl/ClientProtoUtils.java
+++ /dev/null
@@ -1,131 +0,0 @@
-/**
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- *     http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-package org.apache.raft.client.impl;
-
-import org.apache.raft.protocol.*;
-import org.apache.raft.shaded.com.google.protobuf.ByteString;
-import org.apache.raft.shaded.proto.RaftProtos.*;
-import org.apache.raft.util.ProtoUtils;
-
-import java.util.Arrays;
-
-public class ClientProtoUtils {
-  public static RaftRpcReplyProto.Builder toRaftRpcReplyProtoBuilder(
-      String requestorId, String replyId, long seqNum, boolean success) {
-    return RaftRpcReplyProto.newBuilder()
-        .setRequestorId(requestorId).setReplyId(replyId).setSeqNum(seqNum)
-        .setSuccess(success);
-  }
-
-  public static RaftRpcRequestProto.Builder toRaftRpcRequestProtoBuilder(
-      String requesterId, String replyId, long seqNum) {
-    return RaftRpcRequestProto.newBuilder()
-        .setRequestorId(requesterId).setReplyId(replyId).setSeqNum(seqNum);
-  }
-
-  public static RaftClientRequest toRaftClientRequest(RaftClientRequestProto p) {
-    return new RaftClientRequest(p.getRpcRequest().getRequestorId(),
-        p.getRpcRequest().getReplyId(), p.getRpcRequest().getSeqNum(),
-        toMessage(p.getMessage()), p.getReadOnly());
-  }
-
-  public static RaftClientRequestProto toRaftClientRequestProto(
-      RaftClientRequest request) {
-    return RaftClientRequestProto.newBuilder()
-        .setRpcRequest(toRaftRpcRequestProtoBuilder(request.getRequestorId(),
-            request.getReplierId(), request.getSeqNum()))
-        .setMessage(toClientMessageEntryProto(request.getMessage()))
-        .setReadOnly(request.isReadOnly())
-        .build();
-  }
-
-  public static RaftClientRequestProto genRaftClientRequestProto(
-      String requestorId, String replierId, long seqNum, ByteString content,
-      boolean readOnly) {
-    return RaftClientRequestProto.newBuilder()
-        .setRpcRequest(toRaftRpcRequestProtoBuilder(requestorId, replierId, seqNum))
-        .setMessage(ClientMessageEntryProto.newBuilder().setContent(content))
-        .setReadOnly(readOnly)
-        .build();
-  }
-
-  public static RaftClientReplyProto toRaftClientReplyProto(
-      RaftClientReply reply) {
-    final RaftClientReplyProto.Builder b = RaftClientReplyProto.newBuilder();
-    if (reply != null) {
-      b.setRpcReply(toRaftRpcReplyProtoBuilder(reply.getRequestorId(),
-          reply.getReplierId(), reply.getSeqNum(), reply.isSuccess()));
-      if (reply.getMessage() != null) {
-        b.setMessage(toClientMessageEntryProto(reply.getMessage()));
-      }
-      if (reply.isNotLeader()) {
-        b.setIsNotLeader(true);
-        final RaftPeer suggestedLeader = reply.getNotLeaderException()
-            .getSuggestedLeader();
-        if (suggestedLeader != null) {
-          b.setSuggestedLeader(ProtoUtils.toRaftPeerProto(suggestedLeader));
-        }
-        b.addAllPeersInConf(ProtoUtils.toRaftPeerProtos(
-            Arrays.asList(reply.getNotLeaderException().getPeers())));
-      }
-    }
-    return b.build();
-  }
-
-  public static RaftClientReply toRaftClientReply(
-      RaftClientReplyProto replyProto) {
-    final RaftRpcReplyProto rp = replyProto.getRpcReply();
-    NotLeaderException e = null;
-    if (replyProto.getIsNotLeader()) {
-      final RaftPeer suggestedLeader = replyProto.hasSuggestedLeader() ?
-          ProtoUtils.toRaftPeer(replyProto.getSuggestedLeader()) : null;
-      final RaftPeer[] peers = ProtoUtils.toRaftPeerArray(
-          replyProto.getPeersInConfList());
-      e = new NotLeaderException(rp.getReplyId(), suggestedLeader, peers);
-    }
-    return new RaftClientReply(rp.getRequestorId(), rp.getReplyId(),
-        rp.getSeqNum(), rp.getSuccess(), toMessage(replyProto.getMessage()), e);
-  }
-
-  public static Message toMessage(final ClientMessageEntryProto p) {
-    return p::getContent;
-  }
-
-  public static ClientMessageEntryProto toClientMessageEntryProto(Message message) {
-    return ClientMessageEntryProto.newBuilder()
-        .setContent(message.getContent()).build();
-  }
-
-  public static SetConfigurationRequest toSetConfigurationRequest(
-      SetConfigurationRequestProto p) {
-    final RaftRpcRequestProto m = p.getRpcRequest();
-    final RaftPeer[] peers = ProtoUtils.toRaftPeerArray(p.getPeersList());
-    return new SetConfigurationRequest(m.getRequestorId(), m.getReplyId(),
-        p.getRpcRequest().getSeqNum(), peers);
-  }
-
-  public static SetConfigurationRequestProto toSetConfigurationRequestProto(
-      SetConfigurationRequest request) {
-    return SetConfigurationRequestProto.newBuilder()
-        .setRpcRequest(toRaftRpcRequestProtoBuilder(request.getRequestorId(),
-            request.getReplierId(), request.getSeqNum()))
-        .addAllPeers(ProtoUtils.toRaftPeerProtos(
-            Arrays.asList(request.getPeersInNewConf())))
-        .build();
-  }
-}

http://git-wip-us.apache.org/repos/asf/incubator-ratis/blob/7e71a2e0/raft-client/src/main/java/org/apache/raft/client/impl/RaftClientImpl.java
----------------------------------------------------------------------
diff --git a/raft-client/src/main/java/org/apache/raft/client/impl/RaftClientImpl.java b/raft-client/src/main/java/org/apache/raft/client/impl/RaftClientImpl.java
deleted file mode 100644
index 132d092..0000000
--- a/raft-client/src/main/java/org/apache/raft/client/impl/RaftClientImpl.java
+++ /dev/null
@@ -1,171 +0,0 @@
-/**
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- *     http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-package org.apache.raft.client.impl;
-
-import com.google.common.annotations.VisibleForTesting;
-import org.apache.raft.client.RaftClient;
-import org.apache.raft.client.RaftClientConfigKeys;
-import org.apache.raft.client.RaftClientRequestSender;
-import org.apache.raft.conf.RaftProperties;
-import org.apache.raft.protocol.*;
-import org.apache.raft.util.RaftUtils;
-import org.apache.raft.util.StringUtils;
-
-import java.io.IOException;
-import java.io.InterruptedIOException;
-import java.util.Arrays;
-import java.util.Collection;
-import java.util.Map;
-import java.util.function.Function;
-import java.util.function.Supplier;
-import java.util.stream.Collectors;
-
-/** A client who sends requests to a raft service. */
-public final class RaftClientImpl implements RaftClient {
-  public static final long DEFAULT_SEQNUM = 0;
-
-  private final String clientId;
-  private final RaftClientRequestSender requestSender;
-  private final Map<String, RaftPeer> peers;
-  private final int retryInterval;
-
-  private volatile String leaderId;
-
-  public RaftClientImpl(
-      String clientId, Collection<RaftPeer> peers,
-      RaftClientRequestSender requestSender, String leaderId,
-      RaftProperties properties) {
-    this.clientId = clientId;
-    this.requestSender = requestSender;
-    this.peers = peers.stream().collect(
-        Collectors.toMap(RaftPeer::getId, Function.identity()));
-    this.leaderId = leaderId != null? leaderId : peers.iterator().next().getId();
-    this.retryInterval = properties.getInt(
-        RaftClientConfigKeys.RAFT_RPC_TIMEOUT_MS_KEY,
-        RaftClientConfigKeys.RAFT_RPC_TIMEOUT_MS_DEFAULT);
-  }
-
-  @Override
-  public String getId() {
-    return clientId;
-  }
-
-  @Override
-  public RaftClientReply send(Message message) throws IOException {
-    return send(message, false);
-  }
-
-  @Override
-  public RaftClientReply sendReadOnly(Message message) throws IOException {
-    return send(message, true);
-  }
-
-  private RaftClientReply send(Message message, boolean readOnly) throws IOException {
-    return sendRequestWithRetry(() -> new RaftClientRequest(
-        clientId, leaderId, DEFAULT_SEQNUM, message, readOnly));
-  }
-
-  @Override
-  public RaftClientReply setConfiguration(RaftPeer[] peersInNewConf)
-      throws IOException {
-    return sendRequestWithRetry(() -> new SetConfigurationRequest(
-        clientId, leaderId, DEFAULT_SEQNUM, peersInNewConf));
-  }
-
-  private RaftClientReply sendRequestWithRetry(
-      Supplier<RaftClientRequest> supplier)
-      throws InterruptedIOException, StateMachineException {
-    for(;;) {
-      final RaftClientRequest request = supplier.get();
-      LOG.debug("{}: {}", clientId, request);
-      final RaftClientReply reply = sendRequest(request);
-      if (reply != null) {
-        LOG.debug("{}: {}", clientId, reply);
-        return reply;
-      }
-
-      // sleep and then retry
-      try {
-        Thread.sleep(retryInterval);
-      } catch (InterruptedException ie) {
-        Thread.currentThread().interrupt();
-        throw RaftUtils.toInterruptedIOException(
-            "Interrupted when sending " + request, ie);
-      }
-    }
-  }
-
-  private RaftClientReply sendRequest(RaftClientRequest request)
-      throws StateMachineException {
-    try {
-      RaftClientReply reply = requestSender.sendRequest(request);
-      if (reply.isNotLeader()) {
-        handleNotLeaderException(request, reply.getNotLeaderException());
-        return null;
-      } else {
-        return reply;
-      }
-    } catch (StateMachineException e) {
-      throw e;
-    } catch (IOException ioe) {
-      // TODO different retry policies for different exceptions
-      handleIOException(request, ioe, null);
-    }
-    return null;
-  }
-
-  private void handleNotLeaderException(RaftClientRequest request, NotLeaderException nle) {
-    refreshPeers(nle.getPeers());
-    final String newLeader = nle.getSuggestedLeader() == null? null
-        : nle.getSuggestedLeader().getId();
-    handleIOException(request, nle, newLeader);
-  }
-
-  private void refreshPeers(RaftPeer[] newPeers) {
-    if (newPeers != null && newPeers.length > 0) {
-      peers.clear();
-      for (RaftPeer p : newPeers) {
-        peers.put(p.getId(), p);
-      }
-      // also refresh the rpc proxies for these peers
-      requestSender.addServers(Arrays.asList(newPeers));
-    }
-  }
-
-  private void handleIOException(RaftClientRequest request, IOException ioe, String newLeader) {
-    LOG.debug("{}: Failed with {}", clientId, ioe);
-    final String oldLeader = request.getReplierId();
-    if (newLeader == null && oldLeader.equals(leaderId)) {
-      newLeader = StringUtils.next(oldLeader, peers.keySet());
-    }
-    if (newLeader != null && oldLeader.equals(leaderId)) {
-      LOG.debug("{}: change Leader from {} to {}", clientId, oldLeader, newLeader);
-      this.leaderId = newLeader;
-    }
-  }
-
-  @VisibleForTesting
-  public RaftClientRequestSender getRequestSender() {
-    return requestSender;
-  }
-
-  @Override
-  public void close() throws IOException {
-    requestSender.close();
-  }
-}

http://git-wip-us.apache.org/repos/asf/incubator-ratis/blob/7e71a2e0/raft-common/RaftCommon.cmake
----------------------------------------------------------------------
diff --git a/raft-common/RaftCommon.cmake b/raft-common/RaftCommon.cmake
deleted file mode 100644
index 61d2d5a..0000000
--- a/raft-common/RaftCommon.cmake
+++ /dev/null
@@ -1,208 +0,0 @@
-#
-# Licensed to the Apache Software Foundation (ASF) under one
-# or more contributor license agreements.  See the NOTICE file
-# distributed with this work for additional information
-# regarding copyright ownership.  The ASF licenses this file
-# to you under the Apache License, Version 2.0 (the
-# "License"); you may not use this file except in compliance
-# with the License.  You may obtain a copy of the License at
-#
-#     http://www.apache.org/licenses/LICENSE-2.0
-#
-# Unless required by applicable law or agreed to in writing, software
-# distributed under the License is distributed on an "AS IS" BASIS,
-# WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
-# See the License for the specific language governing permissions and
-# limitations under the License.
-#
-
-#
-# Common CMake utilities and configuration, shared by all Native components.
-#
-
-#
-# Platform-specific prerequisite checks.
-#
-
-if(CMAKE_SYSTEM_NAME STREQUAL "SunOS")
-    # Only 64-bit Java is supported.
-    if(NOT JVM_ARCH_DATA_MODEL EQUAL 64)
-        message(FATAL_ERROR "Unrecognised JVM_ARCH_DATA_MODEL '${JVM_ARCH_DATA_MODEL}'. "
-          "A 64-bit JVM must be used on Solaris, make sure that one is installed and, "
-          "if necessary, the MAVEN_OPTS environment variable includes '-d64'")
-    endif()
-
-    # Only gcc is suported for now.
-    if(NOT(CMAKE_COMPILER_IS_GNUCC AND CMAKE_COMPILER_IS_GNUCXX))
-        message(FATAL_ERROR "Only gcc is supported on Solaris")
-    endif()
-endif()
-
-#
-# Helper functions and macros.
-#
-
-# Add flags to all the CMake compiler variables
-macro(raft_add_compiler_flags FLAGS)
-    set(CMAKE_C_FLAGS "${CMAKE_C_FLAGS} ${FLAGS}")
-    set(CMAKE_CXX_FLAGS "${CMAKE_CXX_FLAGS} ${FLAGS}")
-endmacro()
-
-# Add flags to all the CMake linker variables.
-macro(raft_add_linker_flags FLAGS)
-    set(CMAKE_EXE_LINKER_FLAGS "${CMAKE_EXE_LINKER_FLAGS} ${FLAGS}")
-    set(CMAKE_SHARED_LINKER_FLAGS "${CMAKE_SHARED_LINKER_FLAGS} ${FLAGS}")
-endmacro()
-
-# Compile a library with both shared and static variants.
-function(raft_add_dual_library LIBNAME)
-    add_library(${LIBNAME} SHARED ${ARGN})
-    add_library(${LIBNAME}_static STATIC ${ARGN})
-    set_target_properties(${LIBNAME}_static PROPERTIES OUTPUT_NAME ${LIBNAME})
-endfunction()
-
-# Link both a static and a dynamic target against some libraries.
-function(raft_target_link_dual_libraries LIBNAME)
-    target_link_libraries(${LIBNAME} ${ARGN})
-    target_link_libraries(${LIBNAME}_static ${ARGN})
-endfunction()
-
-# Set all the output directories to the same place.
-function(raft_output_directory TGT DIR)
-    set_target_properties(${TGT} PROPERTIES RUNTIME_OUTPUT_DIRECTORY "${CMAKE_BINARY_DIR}/${DIR}")
-    set_target_properties(${TGT} PROPERTIES ARCHIVE_OUTPUT_DIRECTORY "${CMAKE_BINARY_DIR}/${DIR}")
-    set_target_properties(${TGT} PROPERTIES LIBRARY_OUTPUT_DIRECTORY "${CMAKE_BINARY_DIR}/${DIR}")
-endfunction()
-
-# Set the target directories for dynamic and static builds.
-function(raft_dual_output_directory TGT DIR)
-    raft_output_directory(${TGT} "${DIR}")
-    raft_output_directory(${TGT}_static "${DIR}")
-endfunction()
-
-# Alter the behavior of find_package and find_library so that we find only
-# shared libraries with a given version suffix.  You should save
-# CMAKE_FIND_LIBRARY_SUFFIXES before calling this function and restore it
-# afterwards.  On Windows this function is a no-op.  Windows does not encode
-# version number information information into library path names.
-macro(raft_set_find_shared_library_version LVERS)
-    if(${CMAKE_SYSTEM_NAME} MATCHES "Darwin")
-        # Mac OS uses .dylib
-        set(CMAKE_FIND_LIBRARY_SUFFIXES ".${LVERS}.dylib")
-    elseif(${CMAKE_SYSTEM_NAME} MATCHES "FreeBSD")
-        # FreeBSD has always .so installed.
-        set(CMAKE_FIND_LIBRARY_SUFFIXES ".so")
-    elseif(${CMAKE_SYSTEM_NAME} MATCHES "Windows")
-        # Windows doesn't support finding shared libraries by version.
-    else()
-        # Most UNIX variants use .so
-        set(CMAKE_FIND_LIBRARY_SUFFIXES ".so.${LVERS}")
-    endif()
-endmacro()
-
-# Alter the behavior of find_package and find_library so that we find only
-# shared libraries without any version suffix.  You should save
-# CMAKE_FIND_LIBRARY_SUFFIXES before calling this function and restore it
-# afterwards. On Windows this function is a no-op.  Windows does not encode
-# version number information information into library path names.
-macro(raft_set_find_shared_library_without_version)
-    if(${CMAKE_SYSTEM_NAME} MATCHES "Darwin")
-        # Mac OS uses .dylib
-        set(CMAKE_FIND_LIBRARY_SUFFIXES ".dylib")
-    elseif(${CMAKE_SYSTEM_NAME} MATCHES "Windows")
-        # No effect
-    else()
-        # Most UNIX variants use .so
-        set(CMAKE_FIND_LIBRARY_SUFFIXES ".so")
-    endif()
-endmacro()
-
-#
-# Configuration.
-#
-
-# Initialise the shared gcc/g++ flags if they aren't already defined.
-if(NOT DEFINED GCC_SHARED_FLAGS)
-    set(GCC_SHARED_FLAGS "-g -O2 -Wall -pthread -D_FILE_OFFSET_BITS=64")
-endif()
-
-# Add in support other compilers here, if necessary,
-# the assumption is that GCC or a GCC-compatible compiler is being used.
-
-# Set the shared GCC-compatible compiler and linker flags.
-raft_add_compiler_flags("${GCC_SHARED_FLAGS}")
-raft_add_linker_flags("${LINKER_SHARED_FLAGS}")
-
-#
-# Linux-specific configuration.
-#
-if(CMAKE_SYSTEM_NAME STREQUAL "Linux")
-    # Make GNU extensions available.
-    raft_add_compiler_flags("-D_GNU_SOURCE")
-
-    # If JVM_ARCH_DATA_MODEL is 32, compile all binaries as 32-bit.
-    if(JVM_ARCH_DATA_MODEL EQUAL 32)
-        # Force 32-bit code generation on amd64/x86_64, ppc64, sparc64
-        if(CMAKE_COMPILER_IS_GNUCC AND CMAKE_SYSTEM_PROCESSOR MATCHES ".*64")
-            raft_add_compiler_flags("-m32")
-            raft_add_linker_flags("-m32")
-        endif()
-        # Set CMAKE_SYSTEM_PROCESSOR to ensure that find_package(JNI) will use 32-bit libraries
-        if(CMAKE_SYSTEM_PROCESSOR STREQUAL "x86_64" OR CMAKE_SYSTEM_PROCESSOR STREQUAL "amd64")
-            set(CMAKE_SYSTEM_PROCESSOR "i686")
-        endif()
-    endif()
-
-    # Determine float ABI of JVM on ARM.
-    if(CMAKE_SYSTEM_PROCESSOR MATCHES "^arm")
-        find_program(READELF readelf)
-        if(READELF MATCHES "NOTFOUND")
-            message(WARNING "readelf not found; JVM float ABI detection disabled")
-        else(READELF MATCHES "NOTFOUND")
-            execute_process(
-                COMMAND ${READELF} -A ${JAVA_JVM_LIBRARY}
-                OUTPUT_VARIABLE JVM_ELF_ARCH
-                ERROR_QUIET)
-            if(NOT JVM_ELF_ARCH MATCHES "Tag_ABI_VFP_args: VFP registers")
-                # Test compilation with -mfloat-abi=softfp using an arbitrary libc function
-                # (typically fails with "fatal error: bits/predefs.h: No such file or directory"
-                # if soft-float dev libraries are not installed)
-                message("Soft-float JVM detected")
-                include(CMakePushCheckState)
-                cmake_push_check_state()
-                set(CMAKE_REQUIRED_FLAGS "${CMAKE_REQUIRED_FLAGS} -mfloat-abi=softfp")
-                include(CheckSymbolExists)
-                check_symbol_exists(exit stdlib.h SOFTFP_AVAILABLE)
-                if(NOT SOFTFP_AVAILABLE)
-                    message(FATAL_ERROR "Soft-float dev libraries required (e.g. 'apt-get install libc6-dev-armel' on Debian/Ubuntu)")
-                endif()
-                cmake_pop_check_state()
-                raft_add_compiler_flags("-mfloat-abi=softfp")
-            endif()
-        endif()
-    endif()
-
-#
-# Solaris-specific configuration.
-#
-elseif(CMAKE_SYSTEM_NAME STREQUAL "SunOS")
-    # Solaris flags. 64-bit compilation is mandatory, and is checked earlier.
-    raft_add_compiler_flags("-m64 -D_POSIX_C_SOURCE=200112L -D__EXTENSIONS__ -D_POSIX_PTHREAD_SEMANTICS")
-    set(CMAKE_C_FLAGS "-std=gnu99 ${CMAKE_C_FLAGS}")
-    set(CMAKE_CXX_FLAGS "-std=gnu++98 ${CMAKE_CXX_FLAGS}")
-    raft_add_linker_flags("-m64")
-
-    # CMAKE_SYSTEM_PROCESSOR is set to the output of 'uname -p', which on Solaris is
-    # the 'lowest' ISA supported, i.e. 'i386' or 'sparc'. However in order for the
-    # standard CMake modules to look in the right places it needs to reflect the required
-    # compilation mode, i.e. 64 bit. We therefore force it to either 'amd64' or 'sparcv9'.
-    if(CMAKE_SYSTEM_PROCESSOR STREQUAL "i386")
-        set(CMAKE_SYSTEM_PROCESSOR "amd64")
-        set(CMAKE_LIBRARY_ARCHITECTURE "amd64")
-    elseif(CMAKE_SYSTEM_PROCESSOR STREQUAL "sparc")
-        set(CMAKE_SYSTEM_PROCESSOR "sparcv9")
-        set(CMAKE_LIBRARY_ARCHITECTURE "sparcv9")
-    else()
-        message(FATAL_ERROR "Unrecognised CMAKE_SYSTEM_PROCESSOR ${CMAKE_SYSTEM_PROCESSOR}")
-    endif()
-endif()

http://git-wip-us.apache.org/repos/asf/incubator-ratis/blob/7e71a2e0/raft-common/RaftJNI.cmake
----------------------------------------------------------------------
diff --git a/raft-common/RaftJNI.cmake b/raft-common/RaftJNI.cmake
deleted file mode 100644
index 78d7ffd..0000000
--- a/raft-common/RaftJNI.cmake
+++ /dev/null
@@ -1,97 +0,0 @@
-#
-# Licensed to the Apache Software Foundation (ASF) under one
-# or more contributor license agreements.  See the NOTICE file
-# distributed with this work for additional information
-# regarding copyright ownership.  The ASF licenses this file
-# to you under the Apache License, Version 2.0 (the
-# "License"); you may not use this file except in compliance
-# with the License.  You may obtain a copy of the License at
-#
-#     http://www.apache.org/licenses/LICENSE-2.0
-#
-# Unless required by applicable law or agreed to in writing, software
-# distributed under the License is distributed on an "AS IS" BASIS,
-# WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
-# See the License for the specific language governing permissions and
-# limitations under the License.
-#
-
-#
-# Common JNI detection for CMake, shared by all Native components.
-#
-
-# Check the JVM_ARCH_DATA_MODEL variable as been set to 32 or 64 by maven.
-if(NOT DEFINED JVM_ARCH_DATA_MODEL)
-    message(FATAL_ERROR "JVM_ARCH_DATA_MODEL is not defined")
-elseif(NOT (JVM_ARCH_DATA_MODEL EQUAL 32 OR JVM_ARCH_DATA_MODEL EQUAL 64))
-    message(FATAL_ERROR "JVM_ARCH_DATA_MODEL is not 32 or 64")
-endif()
-
-#
-# Linux-specific JNI configuration.
-#
-if(CMAKE_SYSTEM_NAME STREQUAL "Linux")
-    # Locate JNI_INCLUDE_DIRS and JNI_LIBRARIES.
-    # Since we were invoked from Maven, we know that the JAVA_HOME environment
-    # variable is valid.  So we ignore system paths here and just use JAVA_HOME.
-    file(TO_CMAKE_PATH "$ENV{JAVA_HOME}" _java_home)
-    if(CMAKE_SYSTEM_PROCESSOR MATCHES "^i.86$")
-        set(_java_libarch "i386")
-    elseif(CMAKE_SYSTEM_PROCESSOR STREQUAL "x86_64" OR CMAKE_SYSTEM_PROCESSOR STREQUAL "amd64")
-        set(_java_libarch "amd64")
-    elseif(CMAKE_SYSTEM_PROCESSOR MATCHES "^arm")
-        set(_java_libarch "arm")
-    elseif(CMAKE_SYSTEM_PROCESSOR MATCHES "^(powerpc|ppc)64le")
-        if(EXISTS "${_java_home}/jre/lib/ppc64le")
-            set(_java_libarch "ppc64le")
-        else()
-            set(_java_libarch "ppc64")
-        endif()
-    else()
-        set(_java_libarch ${CMAKE_SYSTEM_PROCESSOR})
-    endif()
-    set(_JDK_DIRS "${_java_home}/jre/lib/${_java_libarch}/*"
-                  "${_java_home}/jre/lib/${_java_libarch}"
-                  "${_java_home}/jre/lib/*"
-                  "${_java_home}/jre/lib"
-                  "${_java_home}/lib/*"
-                  "${_java_home}/lib"
-                  "${_java_home}/include/*"
-                  "${_java_home}/include"
-                  "${_java_home}"
-    )
-    find_path(JAVA_INCLUDE_PATH
-        NAMES jni.h
-        PATHS ${_JDK_DIRS}
-        NO_DEFAULT_PATH)
-    #In IBM java, it's jniport.h instead of jni_md.h
-    find_path(JAVA_INCLUDE_PATH2
-        NAMES jni_md.h jniport.h
-        PATHS ${_JDK_DIRS}
-        NO_DEFAULT_PATH)
-    set(JNI_INCLUDE_DIRS ${JAVA_INCLUDE_PATH} ${JAVA_INCLUDE_PATH2})
-    find_library(JAVA_JVM_LIBRARY
-        NAMES jvm JavaVM
-        PATHS ${_JDK_DIRS}
-        NO_DEFAULT_PATH)
-    set(JNI_LIBRARIES ${JAVA_JVM_LIBRARY})
-    unset(_java_libarch)
-    unset(_java_home)
-
-    message("JAVA_HOME=${JAVA_HOME}, JAVA_JVM_LIBRARY=${JAVA_JVM_LIBRARY}")
-    message("JAVA_INCLUDE_PATH=${JAVA_INCLUDE_PATH}, JAVA_INCLUDE_PATH2=${JAVA_INCLUDE_PATH2}")
-    if(JAVA_JVM_LIBRARY AND JAVA_INCLUDE_PATH AND JAVA_INCLUDE_PATH2)
-        message("Located all JNI components successfully.")
-    else()
-        message(FATAL_ERROR "Failed to find a viable JVM installation under JAVA_HOME.")
-    endif()
-
-    # Use the standard FindJNI module to locate the JNI components.
-    find_package(JNI REQUIRED)
-
-#
-# Otherwise, use the standard FindJNI module to locate the JNI components.
-#
-else()
-    find_package(JNI REQUIRED)
-endif()

http://git-wip-us.apache.org/repos/asf/incubator-ratis/blob/7e71a2e0/raft-common/pom.xml
----------------------------------------------------------------------
diff --git a/raft-common/pom.xml b/raft-common/pom.xml
deleted file mode 100644
index d9c3d12..0000000
--- a/raft-common/pom.xml
+++ /dev/null
@@ -1,143 +0,0 @@
-<?xml version="1.0" encoding="UTF-8"?>
-<!--
-  Licensed 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. See accompanying LICENSE file.
--->
-<project xmlns="http://maven.apache.org/POM/4.0.0"
-         xmlns:xsi="http://www.w3.org/2001/XMLSchema-instance"
-         xsi:schemaLocation="http://maven.apache.org/POM/4.0.0 http://maven.apache.org/xsd/maven-4.0.0.xsd">
-  <modelVersion>4.0.0</modelVersion>
-  <parent>
-    <artifactId>raft-project-dist</artifactId>
-    <groupId>com.hortonworks.raft</groupId>
-    <version>1.0-SNAPSHOT</version>
-    <relativePath>../raft-project-dist</relativePath>
-  </parent>
-
-  <artifactId>raft-common</artifactId>
-  <name>Raft Common</name>
-
-  <dependencies>
-    <dependency>
-      <artifactId>raft-proto-shaded</artifactId>
-      <groupId>com.hortonworks.raft</groupId>
-      <scope>provided</scope>
-    </dependency>
-
-    <dependency>
-      <groupId>org.slf4j</groupId>
-      <artifactId>slf4j-api</artifactId>
-    </dependency>
-
-    <dependency>
-      <groupId>com.google.guava</groupId>
-      <artifactId>guava</artifactId>
-    </dependency>
-
-    <dependency>
-      <groupId>junit</groupId>
-      <artifactId>junit</artifactId>
-      <scope>test</scope>
-    </dependency>
-    <dependency>
-      <groupId>org.mockito</groupId>
-      <artifactId>mockito-all</artifactId>
-      <scope>test</scope>
-    </dependency>
-  </dependencies>
-
-  <profiles>
-    <profile>
-      <id>native</id>
-      <activation>
-        <activeByDefault>false</activeByDefault>
-      </activation>
-      <properties>
-        <runningWithNative>true</runningWithNative>
-      </properties>
-      <build>
-        <plugins>
-          <plugin>
-            <groupId>org.apache.maven.plugins</groupId>
-            <artifactId>maven-enforcer-plugin</artifactId>
-            <executions>
-              <execution>
-                <id>enforce-os</id>
-                <goals>
-                  <goal>enforce</goal>
-                </goals>
-                <configuration>
-                  <rules>
-                    <requireOS>
-                      <family>mac</family>
-                      <family>unix</family>
-                      <message>native build only supported on Mac or Unix</message>
-                    </requireOS>
-                  </rules>
-                  <fail>true</fail>
-                </configuration>
-              </execution>
-            </executions>
-          </plugin>
-          <plugin>
-            <groupId>org.codehaus.mojo</groupId>
-            <artifactId>native-maven-plugin</artifactId>
-            <executions>
-              <execution>
-                <phase>compile</phase>
-                <goals>
-                  <goal>javah</goal>
-                </goals>
-                <configuration>
-                  <javahPath>${env.JAVA_HOME}/bin/javah</javahPath>
-                  <javahClassNames>
-                    <javahClassName>org.apache.raft.io.nativeio.NativeIO</javahClassName>
-                    <javahClassName>org.apache.raft.util.NativeCrc32</javahClassName>
-                  </javahClassNames>
-                  <javahOutputDirectory>${project.build.directory}/native/javah</javahOutputDirectory>
-                </configuration>
-              </execution>
-            </executions>
-          </plugin>
-          <plugin>
-            <groupId>org.apache.hadoop</groupId>
-            <artifactId>hadoop-maven-plugins</artifactId>
-            <executions>
-              <execution>
-                <id>cmake-compile</id>
-                <phase>compile</phase>
-                <goals><goal>cmake-compile</goal></goals>
-                <configuration>
-                  <source>${basedir}/src</source>
-                  <vars>
-                    <GENERATED_JAVAH>${project.build.directory}/native/javah</GENERATED_JAVAH>
-                    <JVM_ARCH_DATA_MODEL>${sun.arch.data.model}</JVM_ARCH_DATA_MODEL>
-                  </vars>
-                </configuration>
-              </execution>
-              <execution>
-                <id>test_bulk_crc32</id>
-                <goals><goal>cmake-test</goal></goals>
-                <phase>test</phase>
-                <configuration>
-                  <binary>${project.build.directory}/native/test_bulk_crc32</binary>
-                  <timeout>1200</timeout>
-                  <results>${project.build.directory}/native-results</results>
-                </configuration>
-              </execution>
-            </executions>
-          </plugin>
-        </plugins>
-      </build>
-    </profile>
-  </profiles>
-</project>

http://git-wip-us.apache.org/repos/asf/incubator-ratis/blob/7e71a2e0/raft-common/src/CMakeLists.txt
----------------------------------------------------------------------
diff --git a/raft-common/src/CMakeLists.txt b/raft-common/src/CMakeLists.txt
deleted file mode 100644
index 9e5c766..0000000
--- a/raft-common/src/CMakeLists.txt
+++ /dev/null
@@ -1,108 +0,0 @@
-#
-# Licensed to the Apache Software Foundation (ASF) under one
-# or more contributor license agreements.  See the NOTICE file
-# distributed with this work for additional information
-# regarding copyright ownership.  The ASF licenses this file
-# to you under the Apache License, Version 2.0 (the
-# "License"); you may not use this file except in compliance
-# with the License.  You may obtain a copy of the License at
-#
-#     http://www.apache.org/licenses/LICENSE-2.0
-#
-# Unless required by applicable law or agreed to in writing, software
-# distributed under the License is distributed on an "AS IS" BASIS,
-# WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
-# See the License for the specific language governing permissions and
-# limitations under the License.
-#
-
-#
-# CMake configuration.
-#
-
-cmake_minimum_required(VERSION 2.6 FATAL_ERROR)
-
-list(APPEND CMAKE_MODULE_PATH ${CMAKE_SOURCE_DIR}/..)
-include(RaftCommon)
-
-# Source and test locations.
-set(SRC main/native/src/org/apache/raft)
-set(TST main/native/src/test/org/apache/raft)
-
-#
-# Main configuration.
-#
-
-# The caller must specify where the generated headers have been placed.
-if(NOT GENERATED_JAVAH)
-    message(FATAL_ERROR "You must set the CMake variable GENERATED_JAVAH")
-endif()
-
-# Configure JNI.
-include(RaftJNI)
-
-# Build hardware CRC32 acceleration, if supported on the platform.
-if(CMAKE_SYSTEM_PROCESSOR MATCHES "^i.86$" OR CMAKE_SYSTEM_PROCESSOR STREQUAL "x86_64" OR CMAKE_SYSTEM_PROCESSOR STREQUAL "amd64")
-  set(BULK_CRC_ARCH_SOURCE_FIlE "${SRC}/util/bulk_crc32_x86.c")
-elseif(CMAKE_SYSTEM_PROCESSOR STREQUAL "aarch64")
-  set(BULK_CRC_ARCH_SOURCE_FIlE "${SRC}/util/bulk_crc32_aarch64.c")
-else()
-  message("No HW CRC acceleration for ${CMAKE_SYSTEM_PROCESSOR}, falling back to SW")
-endif()
-
-# Check for platform-specific functions and libraries.
-include(CheckFunctionExists)
-include(CheckLibraryExists)
-check_function_exists(sync_file_range HAVE_SYNC_FILE_RANGE)
-check_function_exists(posix_fadvise HAVE_POSIX_FADVISE)
-check_library_exists(dl dlopen "" NEED_LINK_DL)
-
-# Configure the build.
-include_directories(
-    ${GENERATED_JAVAH}
-    main/native/src
-    ${CMAKE_CURRENT_SOURCE_DIR}
-    ${CMAKE_CURRENT_SOURCE_DIR}/src
-    ${CMAKE_BINARY_DIR}
-    ${JNI_INCLUDE_DIRS}
-    ${SRC}/util
-)
-configure_file(${CMAKE_SOURCE_DIR}/config.h.cmake ${CMAKE_BINARY_DIR}/config.h)
-
-set(CMAKE_BUILD_WITH_INSTALL_RPATH TRUE)
-raft_add_dual_library(raft
-    main/native/src/exception.c
-    ${SRC}/io/nativeio/NativeIO.c
-    ${SRC}/io/nativeio/errno_enum.c
-    ${SRC}/io/nativeio/file_descriptor.c
-    ${SRC}/util/NativeCodeLoader.c
-    ${SRC}/util/NativeCrc32.c
-    ${SRC}/util/bulk_crc32.c
-    ${BULK_CRC_ARCH_SOURCE_FIlE}
-)
-if(NEED_LINK_DL)
-   set(LIB_DL dl)
-endif()
-
-raft_target_link_dual_libraries(raft ${LIB_DL} ${JAVA_JVM_LIBRARY})
-set(LIBRAFT_VERSION "1.0.0")
-set_target_properties(raft PROPERTIES SOVERSION ${LIBRAFT_VERSION})
-raft_dual_output_directory(raft target/usr/local/lib)
-
-# By embedding '$ORIGIN' into the RPATH of libraft.so, dlopen will look in
-# the directory containing libraft.so. However, $ORIGIN is not supported by
-# all operating systems.
-if(${CMAKE_SYSTEM_NAME} MATCHES "Linux|SunOS")
-    set(RPATH "\$ORIGIN/")
-    if(EXTRA_LIBRAFT_RPATH)
-        set(RPATH "${RPATH}:${EXTRA_LIBRAFT_RPATH}/")
-    endif()
-    set_target_properties(raft PROPERTIES INSTALL_RPATH "${RPATH}")
-endif()
-
-# Build the CRC32 test executable.
-add_executable(test_bulk_crc32
-    ${SRC}/util/bulk_crc32.c
-    ${BULK_CRC_ARCH_SOURCE_FIlE}
-    ${TST}/util/test_bulk_crc32.c
-)

http://git-wip-us.apache.org/repos/asf/incubator-ratis/blob/7e71a2e0/raft-common/src/config.h.cmake
----------------------------------------------------------------------
diff --git a/raft-common/src/config.h.cmake b/raft-common/src/config.h.cmake
deleted file mode 100644
index 709fc75..0000000
--- a/raft-common/src/config.h.cmake
+++ /dev/null
@@ -1,24 +0,0 @@
-/**
-* Licensed to the Apache Software Foundation (ASF) under one
-* or more contributor license agreements.  See the NOTICE file
-* distributed with this work for additional information
-* regarding copyright ownership.  The ASF licenses this file
-* to you under the Apache License, Version 2.0 (the
-* "License"); you may not use this file except in compliance
-* with the License.  You may obtain a copy of the License at
-*
-*     http://www.apache.org/licenses/LICENSE-2.0
-*
-* Unless required by applicable law or agreed to in writing, software
-* distributed under the License is distributed on an "AS IS" BASIS,
-* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
-* See the License for the specific language governing permissions and
-* limitations under the License.
-*/
-#ifndef CONFIG_H
-#define CONFIG_H
-
-#cmakedefine HAVE_SYNC_FILE_RANGE
-#cmakedefine HAVE_POSIX_FADVISE
-
-#endif

http://git-wip-us.apache.org/repos/asf/incubator-ratis/blob/7e71a2e0/raft-common/src/main/conf/log4j.properties
----------------------------------------------------------------------
diff --git a/raft-common/src/main/conf/log4j.properties b/raft-common/src/main/conf/log4j.properties
deleted file mode 100644
index 5d16991..0000000
--- a/raft-common/src/main/conf/log4j.properties
+++ /dev/null
@@ -1,74 +0,0 @@
-# Licensed to the Apache Software Foundation (ASF) under one
-# or more contributor license agreements.  See the NOTICE file
-# distributed with this work for additional information
-# regarding copyright ownership.  The ASF licenses this file
-# to you under the Apache License, Version 2.0 (the
-# "License"); you may not use this file except in compliance
-# with the License.  You may obtain a copy of the License at
-#
-#     http://www.apache.org/licenses/LICENSE-2.0
-#
-# Unless required by applicable law or agreed to in writing, software
-# distributed under the License is distributed on an "AS IS" BASIS,
-# WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
-# See the License for the specific language governing permissions and
-# limitations under the License.
-
-# Define some default values that can be overridden by system properties
-raft.root.logger=INFO,console
-raft.log.dir=.
-raft.log.file=raft.log
-
-# Define the root logger to the system property "raft.root.logger".
-log4j.rootLogger=${raft.root.logger}
-
-# Logging Threshold
-log4j.threshold=ALL
-
-# Null Appender
-log4j.appender.NullAppender=org.apache.log4j.varia.NullAppender
-
-#
-# Rolling File Appender - cap space usage at 5gb.
-#
-raft.log.maxfilesize=256MB
-raft.log.maxbackupindex=20
-log4j.appender.RFA=org.apache.log4j.RollingFileAppender
-log4j.appender.RFA.File=${raft.log.dir}/${raft.log.file}
-
-log4j.appender.RFA.MaxFileSize=${raft.log.maxfilesize}
-log4j.appender.RFA.MaxBackupIndex=${raft.log.maxbackupindex}
-
-log4j.appender.RFA.layout=org.apache.log4j.PatternLayout
-
-# Pattern format: Date LogLevel LoggerName LogMessage
-log4j.appender.RFA.layout.ConversionPattern=%d{ISO8601} %p %c: %m%n
-# Debugging Pattern format
-#log4j.appender.RFA.layout.ConversionPattern=%d{ISO8601} %-5p %c{2} (%F:%M(%L)) - %m%n
-
-
-#
-# Daily Rolling File Appender
-#
-log4j.appender.DRFA=org.apache.log4j.DailyRollingFileAppender
-log4j.appender.DRFA.File=${raft.log.dir}/${raft.log.file}
-
-# Rollover at midnight
-log4j.appender.DRFA.DatePattern=.yyyy-MM-dd
-
-log4j.appender.DRFA.layout=org.apache.log4j.PatternLayout
-
-# Pattern format: Date LogLevel LoggerName LogMessage
-log4j.appender.DRFA.layout.ConversionPattern=%d{ISO8601} %p %c: %m%n
-# Debugging Pattern format
-#log4j.appender.DRFA.layout.ConversionPattern=%d{ISO8601} %-5p %c{2} (%F:%M(%L)) - %m%n
-
-
-#
-# console
-# Add "console" to rootlogger above if you want to use this
-#
-log4j.appender.console=org.apache.log4j.ConsoleAppender
-log4j.appender.console.target=System.err
-log4j.appender.console.layout=org.apache.log4j.PatternLayout
-log4j.appender.console.layout.ConversionPattern=%d{yy/MM/dd HH:mm:ss} %p %c{2}: %m%n


[09/54] [abbrv] incubator-ratis git commit: Change RaftServerRpc to extend RaftServerProtocol and Closeable.

Posted by ji...@apache.org.
Change RaftServerRpc to extend RaftServerProtocol and Closeable.


Project: http://git-wip-us.apache.org/repos/asf/incubator-ratis/repo
Commit: http://git-wip-us.apache.org/repos/asf/incubator-ratis/commit/673a2827
Tree: http://git-wip-us.apache.org/repos/asf/incubator-ratis/tree/673a2827
Diff: http://git-wip-us.apache.org/repos/asf/incubator-ratis/diff/673a2827

Branch: refs/heads/master
Commit: 673a28278346fb641f7c1e81c20bc37168be33f0
Parents: 56e9b71
Author: Tsz-Wo Nicholas Sze <sz...@hortonworks.com>
Authored: Mon Jan 2 23:59:14 2017 +0800
Committer: Tsz-Wo Nicholas Sze <sz...@hortonworks.com>
Committed: Mon Jan 2 23:59:14 2017 +0800

----------------------------------------------------------------------
 .../org/apache/raft/grpc/RaftGRpcService.java   | 10 ++++-----
 .../raft/hadooprpc/server/HadoopRpcService.java |  8 +++----
 .../raft/netty/server/NettyRpcService.java      |  8 +++----
 .../org/apache/raft/server/RaftServerRpc.java   | 23 ++++++++------------
 .../apache/raft/server/impl/LeaderElection.java |  2 +-
 .../apache/raft/server/impl/LogAppender.java    |  4 ++--
 .../apache/raft/server/impl/RaftServerImpl.java | 11 ++++------
 .../apache/raft/server/impl/ServerState.java    |  2 +-
 .../server/simulation/SimulatedServerRpc.java   |  8 +++----
 9 files changed, 34 insertions(+), 42 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-ratis/blob/673a2827/raft-grpc/src/main/java/org/apache/raft/grpc/RaftGRpcService.java
----------------------------------------------------------------------
diff --git a/raft-grpc/src/main/java/org/apache/raft/grpc/RaftGRpcService.java b/raft-grpc/src/main/java/org/apache/raft/grpc/RaftGRpcService.java
index f3d894a..d0c98c3 100644
--- a/raft-grpc/src/main/java/org/apache/raft/grpc/RaftGRpcService.java
+++ b/raft-grpc/src/main/java/org/apache/raft/grpc/RaftGRpcService.java
@@ -90,14 +90,14 @@ public class RaftGRpcService implements RaftServerRpc {
       @Override
       public void run() {
         System.err.println("*** shutting down gRPC server since JVM is shutting down");
-        RaftGRpcService.this.shutdown();
+        RaftGRpcService.this.close();
         System.err.println("*** server shut down");
       }
     });
   }
 
   @Override
-  public void shutdown() {
+  public void close() {
     if (server != null) {
       server.shutdown();
     }
@@ -110,21 +110,21 @@ public class RaftGRpcService implements RaftServerRpc {
   }
 
   @Override
-  public AppendEntriesReplyProto sendAppendEntries(
+  public AppendEntriesReplyProto appendEntries(
       AppendEntriesRequestProto request) throws IOException {
     throw new UnsupportedOperationException(
         "Blocking AppendEntries call is not supported");
   }
 
   @Override
-  public InstallSnapshotReplyProto sendInstallSnapshot(
+  public InstallSnapshotReplyProto installSnapshot(
       InstallSnapshotRequestProto request) throws IOException {
     throw new UnsupportedOperationException(
         "Blocking InstallSnapshot call is not supported");
   }
 
   @Override
-  public RequestVoteReplyProto sendRequestVote(RequestVoteRequestProto request)
+  public RequestVoteReplyProto requestVote(RequestVoteRequestProto request)
       throws IOException {
     CodeInjectionForTesting.execute(GRPC_SEND_SERVER_REQUEST, selfId,
         null, request);

http://git-wip-us.apache.org/repos/asf/incubator-ratis/blob/673a2827/raft-hadoop/src/main/java/org/apache/raft/hadooprpc/server/HadoopRpcService.java
----------------------------------------------------------------------
diff --git a/raft-hadoop/src/main/java/org/apache/raft/hadooprpc/server/HadoopRpcService.java b/raft-hadoop/src/main/java/org/apache/raft/hadooprpc/server/HadoopRpcService.java
index 3330d78..b73deca 100644
--- a/raft-hadoop/src/main/java/org/apache/raft/hadooprpc/server/HadoopRpcService.java
+++ b/raft-hadoop/src/main/java/org/apache/raft/hadooprpc/server/HadoopRpcService.java
@@ -122,12 +122,12 @@ public class HadoopRpcService implements RaftServerRpc {
   }
 
   @Override
-  public void shutdown() {
+  public void close() {
     ipcServer.stop();
   }
 
   @Override
-  public AppendEntriesReplyProto sendAppendEntries(
+  public AppendEntriesReplyProto appendEntries(
       AppendEntriesRequestProto request) throws IOException {
     Preconditions.checkArgument(id.equals(request.getServerRequest().getRequestorId()));
     CodeInjectionForTesting.execute(SEND_SERVER_REQUEST, id, null, request);
@@ -142,7 +142,7 @@ public class HadoopRpcService implements RaftServerRpc {
   }
 
   @Override
-  public InstallSnapshotReplyProto sendInstallSnapshot(
+  public InstallSnapshotReplyProto installSnapshot(
       InstallSnapshotRequestProto request) throws IOException {
     Preconditions.checkArgument(id.equals(request.getServerRequest().getRequestorId()));
     CodeInjectionForTesting.execute(SEND_SERVER_REQUEST, id, null, request);
@@ -157,7 +157,7 @@ public class HadoopRpcService implements RaftServerRpc {
   }
 
   @Override
-  public RequestVoteReplyProto sendRequestVote(
+  public RequestVoteReplyProto requestVote(
       RequestVoteRequestProto request) throws IOException {
     Preconditions.checkArgument(id.equals(request.getServerRequest().getRequestorId()));
     CodeInjectionForTesting.execute(SEND_SERVER_REQUEST, id, null, request);

http://git-wip-us.apache.org/repos/asf/incubator-ratis/blob/673a2827/raft-netty/src/main/java/org/apache/raft/netty/server/NettyRpcService.java
----------------------------------------------------------------------
diff --git a/raft-netty/src/main/java/org/apache/raft/netty/server/NettyRpcService.java b/raft-netty/src/main/java/org/apache/raft/netty/server/NettyRpcService.java
index 19f5979..08e379a 100644
--- a/raft-netty/src/main/java/org/apache/raft/netty/server/NettyRpcService.java
+++ b/raft-netty/src/main/java/org/apache/raft/netty/server/NettyRpcService.java
@@ -112,7 +112,7 @@ public final class NettyRpcService implements RaftServerRpc {
   }
 
   @Override
-  public void shutdown() {
+  public void close() {
     lifeCycle.checkStateAndClose(() -> {
       bossGroup.shutdownGracefully();
       workerGroup.shutdownGracefully();
@@ -199,7 +199,7 @@ public final class NettyRpcService implements RaftServerRpc {
   }
 
   @Override
-  public RequestVoteReplyProto sendRequestVote(RequestVoteRequestProto request) throws IOException {
+  public RequestVoteReplyProto requestVote(RequestVoteRequestProto request) throws IOException {
     Preconditions.checkArgument(id.equals(request.getServerRequest().getRequestorId()));
     CodeInjectionForTesting.execute(SEND_SERVER_REQUEST, id, null, request);
 
@@ -211,7 +211,7 @@ public final class NettyRpcService implements RaftServerRpc {
   }
 
   @Override
-  public AppendEntriesReplyProto sendAppendEntries(AppendEntriesRequestProto request) throws IOException {
+  public AppendEntriesReplyProto appendEntries(AppendEntriesRequestProto request) throws IOException {
     Preconditions.checkArgument(id.equals(request.getServerRequest().getRequestorId()));
     CodeInjectionForTesting.execute(SEND_SERVER_REQUEST, id, null, request);
 
@@ -223,7 +223,7 @@ public final class NettyRpcService implements RaftServerRpc {
   }
 
   @Override
-  public InstallSnapshotReplyProto sendInstallSnapshot(InstallSnapshotRequestProto request) throws IOException {
+  public InstallSnapshotReplyProto installSnapshot(InstallSnapshotRequestProto request) throws IOException {
     Preconditions.checkArgument(id.equals(request.getServerRequest().getRequestorId()));
     CodeInjectionForTesting.execute(SEND_SERVER_REQUEST, id, null, request);
 

http://git-wip-us.apache.org/repos/asf/incubator-ratis/blob/673a2827/raft-server/src/main/java/org/apache/raft/server/RaftServerRpc.java
----------------------------------------------------------------------
diff --git a/raft-server/src/main/java/org/apache/raft/server/RaftServerRpc.java b/raft-server/src/main/java/org/apache/raft/server/RaftServerRpc.java
index de81ec2..6526bea 100644
--- a/raft-server/src/main/java/org/apache/raft/server/RaftServerRpc.java
+++ b/raft-server/src/main/java/org/apache/raft/server/RaftServerRpc.java
@@ -18,27 +18,22 @@
 package org.apache.raft.server;
 
 import org.apache.raft.protocol.RaftPeer;
-import org.apache.raft.shaded.proto.RaftProtos.*;
+import org.apache.raft.server.protocol.RaftServerProtocol;
 
-import java.io.IOException;
+import java.io.Closeable;
 import java.net.InetSocketAddress;
 
-public interface RaftServerRpc {
+/**
+ * An server-side interface for supporting different RPC implementations
+ * such as Netty, gRPC and Hadoop.
+ */
+public interface RaftServerRpc extends RaftServerProtocol, Closeable {
+  /** Start the RPC service. */
   void start();
 
-  void shutdown();
-
+  /** @return the address where this RPC server is listening to. */
   InetSocketAddress getInetSocketAddress();
 
-  AppendEntriesReplyProto sendAppendEntries(
-      AppendEntriesRequestProto request) throws IOException;
-
-  InstallSnapshotReplyProto sendInstallSnapshot(
-      InstallSnapshotRequestProto request) throws IOException;
-
-  RequestVoteReplyProto sendRequestVote(RequestVoteRequestProto request)
-      throws IOException;
-
   /** add information of the given peers */
   void addPeers(Iterable<RaftPeer> peers);
 }

http://git-wip-us.apache.org/repos/asf/incubator-ratis/blob/673a2827/raft-server/src/main/java/org/apache/raft/server/impl/LeaderElection.java
----------------------------------------------------------------------
diff --git a/raft-server/src/main/java/org/apache/raft/server/impl/LeaderElection.java b/raft-server/src/main/java/org/apache/raft/server/impl/LeaderElection.java
index 8552029..39bdb13 100644
--- a/raft-server/src/main/java/org/apache/raft/server/impl/LeaderElection.java
+++ b/raft-server/src/main/java/org/apache/raft/server/impl/LeaderElection.java
@@ -187,7 +187,7 @@ class LeaderElection extends Daemon {
       final RequestVoteRequestProto r = server.createRequestVoteRequest(
           peer.getId(), electionTerm, lastEntry);
       service.submit(
-          () -> server.getServerRpc().sendRequestVote(r));
+          () -> server.getServerRpc().requestVote(r));
       submitted++;
     }
     return submitted;

http://git-wip-us.apache.org/repos/asf/incubator-ratis/blob/673a2827/raft-server/src/main/java/org/apache/raft/server/impl/LogAppender.java
----------------------------------------------------------------------
diff --git a/raft-server/src/main/java/org/apache/raft/server/impl/LogAppender.java b/raft-server/src/main/java/org/apache/raft/server/impl/LogAppender.java
index 3b18f13..cf613ca 100644
--- a/raft-server/src/main/java/org/apache/raft/server/impl/LogAppender.java
+++ b/raft-server/src/main/java/org/apache/raft/server/impl/LogAppender.java
@@ -198,7 +198,7 @@ public class LogAppender extends Daemon {
 
         follower.updateLastRpcSendTime();
         final AppendEntriesReplyProto r = server.getServerRpc()
-            .sendAppendEntries(request);
+            .appendEntries(request);
         follower.updateLastRpcResponseTime();
 
         return r;
@@ -328,7 +328,7 @@ public class LogAppender extends Daemon {
       for (InstallSnapshotRequestProto request :
           new SnapshotRequestIter(snapshot, requestId)) {
         follower.updateLastRpcSendTime();
-        reply = server.getServerRpc().sendInstallSnapshot(request);
+        reply = server.getServerRpc().installSnapshot(request);
         follower.updateLastRpcResponseTime();
 
         if (!reply.getServerReply().getSuccess()) {

http://git-wip-us.apache.org/repos/asf/incubator-ratis/blob/673a2827/raft-server/src/main/java/org/apache/raft/server/impl/RaftServerImpl.java
----------------------------------------------------------------------
diff --git a/raft-server/src/main/java/org/apache/raft/server/impl/RaftServerImpl.java b/raft-server/src/main/java/org/apache/raft/server/impl/RaftServerImpl.java
index 131e002..3026afa 100644
--- a/raft-server/src/main/java/org/apache/raft/server/impl/RaftServerImpl.java
+++ b/raft-server/src/main/java/org/apache/raft/server/impl/RaftServerImpl.java
@@ -132,12 +132,13 @@ public class RaftServerImpl implements RaftServer {
     this.state.setInitialConf(conf);
   }
 
+  @Override
   public void setServerRpc(RaftServerRpc serverRpc) {
     this.serverRpc = serverRpc;
     // add peers into rpc service
     RaftConfiguration conf = getRaftConf();
     if (conf != null) {
-      addPeersToRPC(conf.getPeers());
+      serverRpc.addPeers(conf.getPeers());
     }
   }
 
@@ -201,7 +202,7 @@ public class RaftServerImpl implements RaftServer {
         shutdownElectionDaemon();
         shutdownLeaderState();
 
-        serverRpc.shutdown();
+        serverRpc.close();
         state.close();
       } catch (Exception ignored) {
         LOG.warn("Failed to kill " + state.getSelfId(), ignored);
@@ -418,7 +419,7 @@ public class RaftServerImpl implements RaftServer {
       }
 
       // add new peers into the rpc service
-      addPeersToRPC(Arrays.asList(peersInNewConf));
+      getServerRpc().addPeers(Arrays.asList(peersInNewConf));
       // add staging state into the leaderState
       pending = leaderState.startSetConfiguration(request);
     }
@@ -726,10 +727,6 @@ public class RaftServerImpl implements RaftServer {
     }
   }
 
-  public void addPeersToRPC(Iterable<RaftPeer> peers) {
-    serverRpc.addPeers(peers);
-  }
-
   synchronized void replyPendingRequest(long logIndex,
       CompletableFuture<Message> message) {
     if (isLeader() && leaderState != null) { // is leader and is running

http://git-wip-us.apache.org/repos/asf/incubator-ratis/blob/673a2827/raft-server/src/main/java/org/apache/raft/server/impl/ServerState.java
----------------------------------------------------------------------
diff --git a/raft-server/src/main/java/org/apache/raft/server/impl/ServerState.java b/raft-server/src/main/java/org/apache/raft/server/impl/ServerState.java
index c91968c..8611101 100644
--- a/raft-server/src/main/java/org/apache/raft/server/impl/ServerState.java
+++ b/raft-server/src/main/java/org/apache/raft/server/impl/ServerState.java
@@ -285,7 +285,7 @@ public class ServerState implements Closeable {
           final RaftConfiguration conf = ServerProtoUtils.toRaftConfiguration(
               entry.getIndex(), entry.getConfigurationEntry());
           configurationManager.addConfiguration(entry.getIndex(), conf);
-          server.addPeersToRPC(conf.getPeers());
+          server.getServerRpc().addPeers(conf.getPeers());
         }
       }
     }

http://git-wip-us.apache.org/repos/asf/incubator-ratis/blob/673a2827/raft-server/src/test/java/org/apache/raft/server/simulation/SimulatedServerRpc.java
----------------------------------------------------------------------
diff --git a/raft-server/src/test/java/org/apache/raft/server/simulation/SimulatedServerRpc.java b/raft-server/src/test/java/org/apache/raft/server/simulation/SimulatedServerRpc.java
index cc3fb35..8a7e752 100644
--- a/raft-server/src/test/java/org/apache/raft/server/simulation/SimulatedServerRpc.java
+++ b/raft-server/src/test/java/org/apache/raft/server/simulation/SimulatedServerRpc.java
@@ -69,7 +69,7 @@ class SimulatedServerRpc implements RaftServerRpc {
   }
 
   @Override
-  public void shutdown() {
+  public void close() {
     try {
       interruptAndJoin();
       executor.shutdown();
@@ -86,7 +86,7 @@ class SimulatedServerRpc implements RaftServerRpc {
   }
 
   @Override
-  public AppendEntriesReplyProto sendAppendEntries(AppendEntriesRequestProto request)
+  public AppendEntriesReplyProto appendEntries(AppendEntriesRequestProto request)
       throws IOException {
     RaftServerReply reply = serverHandler.getRpc()
         .sendRequest(new RaftServerRequest(request));
@@ -94,7 +94,7 @@ class SimulatedServerRpc implements RaftServerRpc {
   }
 
   @Override
-  public InstallSnapshotReplyProto sendInstallSnapshot(InstallSnapshotRequestProto request)
+  public InstallSnapshotReplyProto installSnapshot(InstallSnapshotRequestProto request)
       throws IOException {
     RaftServerReply reply = serverHandler.getRpc()
         .sendRequest(new RaftServerRequest(request));
@@ -102,7 +102,7 @@ class SimulatedServerRpc implements RaftServerRpc {
   }
 
   @Override
-  public RequestVoteReplyProto sendRequestVote(RequestVoteRequestProto request)
+  public RequestVoteReplyProto requestVote(RequestVoteRequestProto request)
       throws IOException {
     RaftServerReply reply = serverHandler.getRpc()
         .sendRequest(new RaftServerRequest(request));


[03/54] [abbrv] incubator-ratis git commit: Move o.a.r.s.* to o.a.r.s.impl.

Posted by ji...@apache.org.
http://git-wip-us.apache.org/repos/asf/incubator-ratis/blob/c36810ed/raft-server/src/main/java/org/apache/raft/server/ServerState.java
----------------------------------------------------------------------
diff --git a/raft-server/src/main/java/org/apache/raft/server/ServerState.java b/raft-server/src/main/java/org/apache/raft/server/ServerState.java
deleted file mode 100644
index 8975eb3..0000000
--- a/raft-server/src/main/java/org/apache/raft/server/ServerState.java
+++ /dev/null
@@ -1,346 +0,0 @@
-/**
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- *     http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-package org.apache.raft.server;
-
-import com.google.common.annotations.VisibleForTesting;
-import com.google.common.base.Preconditions;
-import org.apache.raft.conf.RaftProperties;
-import org.apache.raft.server.impl.ServerProtoUtils;
-import org.apache.raft.server.protocol.TermIndex;
-import org.apache.raft.server.storage.*;
-import org.apache.raft.shaded.proto.RaftProtos.InstallSnapshotRequestProto;
-import org.apache.raft.shaded.proto.RaftProtos.LogEntryProto;
-import org.apache.raft.statemachine.SnapshotInfo;
-import org.apache.raft.statemachine.StateMachine;
-import org.apache.raft.statemachine.TransactionContext;
-import org.apache.raft.util.ProtoUtils;
-
-import java.io.Closeable;
-import java.io.IOException;
-
-import static org.apache.raft.server.RaftServerConfigKeys.RAFT_SERVER_USE_MEMORY_LOG_DEFAULT;
-import static org.apache.raft.server.RaftServerConfigKeys.RAFT_SERVER_USE_MEMORY_LOG_KEY;
-
-/**
- * Common states of a raft peer. Protected by RaftServer's lock.
- */
-public class ServerState implements Closeable {
-  private final String selfId;
-  private final RaftServer server;
-  /** Raft log */
-  private final RaftLog log;
-  /** Raft configuration */
-  private final ConfigurationManager configurationManager;
-  /** The thread that applies committed log entries to the state machine */
-  private final StateMachineUpdater stateMachineUpdater;
-  /** local storage for log and snapshot */
-  private final RaftStorage storage;
-  private final SnapshotManager snapshotManager;
-
-  /**
-   * Latest term server has seen. initialized to 0 on first boot, increases
-   * monotonically.
-   */
-  private long currentTerm;
-  /**
-   * The server ID of the leader for this term. Null means either there is
-   * no leader for this term yet or this server does not know who it is yet.
-   */
-  private String leaderId;
-  /**
-   * Candidate that this peer granted vote for in current term (or null if none)
-   */
-  private String votedFor;
-
-  /**
-   * Latest installed snapshot for this server. This maybe different than StateMachine's latest
-   * snapshot. Once we successfully install a snapshot, the SM may not pick it up immediately.
-   * Further, this will not get updated when SM does snapshots itself.
-   */
-  private TermIndex latestInstalledSnapshot;
-
-  ServerState(String id, RaftConfiguration conf, RaftProperties prop,
-      RaftServer server, StateMachine stateMachine) throws IOException {
-    this.selfId = id;
-    this.server = server;
-    configurationManager = new ConfigurationManager(conf);
-    storage = new RaftStorage(prop, RaftServerConstants.StartupOption.REGULAR);
-    snapshotManager = new SnapshotManager(storage, id);
-
-    long lastApplied = initStatemachine(stateMachine, prop);
-
-    leaderId = null;
-    log = initLog(id, prop, server, lastApplied);
-    RaftLog.Metadata metadata = log.loadMetadata();
-    currentTerm = metadata.getTerm();
-    votedFor = metadata.getVotedFor();
-
-    stateMachineUpdater = new StateMachineUpdater(stateMachine, server, log,
-         lastApplied, prop);
-  }
-
-  /**
-   * Used by tests to set initial raft configuration with correct port bindings.
-   */
-  @VisibleForTesting
-  public void setInitialConf(RaftConfiguration initialConf) {
-    configurationManager.setInitialConf(initialConf);
-  }
-
-  private long initStatemachine(StateMachine sm, RaftProperties properties)
-      throws IOException {
-    sm.initialize(selfId, properties, storage);
-    storage.setStateMachineStorage(sm.getStateMachineStorage());
-    SnapshotInfo snapshot = sm.getLatestSnapshot();
-
-    if (snapshot == null || snapshot.getTermIndex().getIndex() < 0) {
-      return RaftServerConstants.INVALID_LOG_INDEX;
-    }
-
-    // get the raft configuration from the snapshot
-    RaftConfiguration raftConf = sm.getRaftConfiguration();
-    if (raftConf != null) {
-      configurationManager.addConfiguration(raftConf.getLogEntryIndex(),
-          raftConf);
-    }
-    return snapshot.getIndex();
-  }
-
-  void start() {
-    stateMachineUpdater.start();
-  }
-
-  /**
-   * note we do not apply log entries to the state machine here since we do not
-   * know whether they have been committed.
-   */
-  private RaftLog initLog(String id, RaftProperties prop, RaftServer server,
-      long lastIndexInSnapshot) throws IOException {
-    final RaftLog log;
-    if (prop.getBoolean(RAFT_SERVER_USE_MEMORY_LOG_KEY,
-        RAFT_SERVER_USE_MEMORY_LOG_DEFAULT)) {
-      log = new MemoryRaftLog(id);
-    } else {
-      log = new SegmentedRaftLog(id, server, this.storage,
-          lastIndexInSnapshot, prop);
-    }
-    log.open(configurationManager, lastIndexInSnapshot);
-    return log;
-  }
-
-  public RaftConfiguration getRaftConf() {
-    return configurationManager.getCurrent();
-  }
-
-  @VisibleForTesting
-
-  public String getSelfId() {
-    return this.selfId;
-  }
-
-  public long getCurrentTerm() {
-    return currentTerm;
-  }
-
-  void setCurrentTerm(long term) {
-    currentTerm = term;
-  }
-
-  String getLeaderId() {
-    return leaderId;
-  }
-
-  boolean hasLeader() {
-    return leaderId != null;
-  }
-
-  /**
-   * Become a candidate and start leader election
-   */
-  long initElection() {
-    votedFor = selfId;
-    leaderId = null;
-    return ++currentTerm;
-  }
-
-  void persistMetadata() throws IOException {
-    this.log.writeMetadata(currentTerm, votedFor);
-  }
-
-  void resetLeaderAndVotedFor() {
-    votedFor = null;
-    leaderId = null;
-  }
-
-  /**
-   * Vote for a candidate and update the local state.
-   */
-  void grantVote(String candidateId) {
-    votedFor = candidateId;
-    leaderId = null;
-  }
-
-  void setLeader(String leaderId) {
-    this.leaderId = leaderId;
-  }
-
-  void becomeLeader() {
-    leaderId = selfId;
-  }
-
-  public RaftLog getLog() {
-    return log;
-  }
-
-  long applyLog(TransactionContext operation) throws IOException {
-    return log.append(currentTerm, operation);
-  }
-
-  /**
-   * Check if accept the leader selfId and term from the incoming AppendEntries rpc.
-   * If accept, update the current state.
-   * @return true if the check passes
-   */
-  boolean recognizeLeader(String leaderId, long leaderTerm) {
-    if (leaderTerm < currentTerm) {
-      return false;
-    } else if (leaderTerm > currentTerm || this.leaderId == null) {
-      // If the request indicates a term that is greater than the current term
-      // or no leader has been set for the current term, make sure to update
-      // leader and term later
-      return true;
-    }
-    Preconditions.checkArgument(this.leaderId.equals(leaderId),
-        "selfId:%s, this.leaderId:%s, received leaderId:%s",
-        selfId, this.leaderId, leaderId);
-    return true;
-  }
-
-  /**
-   * Check if the candidate's term is acceptable
-   */
-  boolean recognizeCandidate(String candidateId,
-      long candidateTerm) {
-    if (candidateTerm > currentTerm) {
-      return true;
-    } else if (candidateTerm == currentTerm) {
-      // has not voted yet or this is a retry
-      return votedFor == null || votedFor.equals(candidateId);
-    }
-    return false;
-  }
-
-  boolean isLogUpToDate(TermIndex candidateLastEntry) {
-    LogEntryProto lastEntry = log.getLastEntry();
-    // need to take into account snapshot
-    SnapshotInfo snapshot = server.getStateMachine().getLatestSnapshot();
-     if (lastEntry == null && snapshot == null) {
-      return true;
-    } else if (candidateLastEntry == null) {
-      return false;
-    }
-    TermIndex local = ServerProtoUtils.toTermIndex(lastEntry);
-    if (local == null || (snapshot != null && snapshot.getIndex() > lastEntry.getIndex())) {
-      local = snapshot.getTermIndex();
-    }
-    return local.compareTo(candidateLastEntry) <= 0;
-  }
-
-  @Override
-  public String toString() {
-    return selfId + ":t" + currentTerm + ", leader=" + leaderId
-        + ", voted=" + votedFor + ", raftlog=" + log + ", conf=" + getRaftConf();
-  }
-
-  boolean isConfCommitted() {
-    return getLog().getLastCommittedIndex() >=
-        getRaftConf().getLogEntryIndex();
-  }
-
-  public void setRaftConf(long logIndex, RaftConfiguration conf) {
-    configurationManager.addConfiguration(logIndex, conf);
-    RaftServer.LOG.info("{}: successfully update the configuration {}",
-        getSelfId(), conf);
-  }
-
-  void updateConfiguration(LogEntryProto[] entries) {
-    if (entries != null && entries.length > 0) {
-      configurationManager.removeConfigurations(entries[0].getIndex());
-      for (LogEntryProto entry : entries) {
-        if (ProtoUtils.isConfigurationLogEntry(entry)) {
-          final RaftConfiguration conf = ServerProtoUtils.toRaftConfiguration(
-              entry.getIndex(), entry.getConfigurationEntry());
-          configurationManager.addConfiguration(entry.getIndex(), conf);
-          server.addPeersToRPC(conf.getPeers());
-        }
-      }
-    }
-  }
-
-  void updateStatemachine(long majorityIndex, long currentTerm) {
-    log.updateLastCommitted(majorityIndex, currentTerm);
-    stateMachineUpdater.notifyUpdater();
-  }
-
-  void reloadStateMachine(long lastIndexInSnapshot, long currentTerm)
-      throws IOException {
-    log.updateLastCommitted(lastIndexInSnapshot, currentTerm);
-
-    stateMachineUpdater.reloadStateMachine();
-  }
-
-  @Override
-  public void close() throws IOException {
-    stateMachineUpdater.stop();
-    RaftServer.LOG.info("{} closes. The last applied log index is {}",
-        getSelfId(), getLastAppliedIndex());
-    storage.close();
-  }
-
-  @VisibleForTesting
-  public RaftStorage getStorage() {
-    return storage;
-  }
-
-  void installSnapshot(InstallSnapshotRequestProto request) throws IOException {
-    // TODO: verify that we need to install the snapshot
-    StateMachine sm = server.getStateMachine();
-    sm.pause(); // pause the SM to prepare for install snapshot
-    snapshotManager.installSnapshot(sm, request);
-    log.syncWithSnapshot(request.getTermIndex().getIndex());
-    this.latestInstalledSnapshot = ServerProtoUtils.toTermIndex(
-        request.getTermIndex());
-  }
-
-  SnapshotInfo getLatestSnapshot() {
-    return server.getStateMachine().getStateMachineStorage().getLatestSnapshot();
-  }
-
-  public TermIndex getLatestInstalledSnapshot() {
-    return latestInstalledSnapshot;
-  }
-
-  @VisibleForTesting
-  public long getLastAppliedIndex() {
-    return stateMachineUpdater.getLastAppliedIndex();
-  }
-
-  boolean isCurrentConfCommitted() {
-    return getRaftConf().getLogEntryIndex() <= getLog().getLastCommittedIndex();
-  }
-}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/incubator-ratis/blob/c36810ed/raft-server/src/main/java/org/apache/raft/server/StateMachineUpdater.java
----------------------------------------------------------------------
diff --git a/raft-server/src/main/java/org/apache/raft/server/StateMachineUpdater.java b/raft-server/src/main/java/org/apache/raft/server/StateMachineUpdater.java
deleted file mode 100644
index b6f88be..0000000
--- a/raft-server/src/main/java/org/apache/raft/server/StateMachineUpdater.java
+++ /dev/null
@@ -1,213 +0,0 @@
-/**
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- *     http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-package org.apache.raft.server;
-
-import com.google.common.base.Preconditions;
-import org.apache.raft.conf.RaftProperties;
-import org.apache.raft.protocol.Message;
-import org.apache.raft.server.impl.ServerProtoUtils;
-import org.apache.raft.server.storage.RaftLog;
-import org.apache.raft.server.storage.RaftStorage;
-import org.apache.raft.shaded.proto.RaftProtos.LogEntryProto;
-import org.apache.raft.statemachine.SnapshotInfo;
-import org.apache.raft.statemachine.StateMachine;
-import org.apache.raft.statemachine.TransactionContext;
-import org.apache.raft.util.Daemon;
-import org.apache.raft.util.ExitUtils;
-import org.apache.raft.util.LifeCycle;
-import org.slf4j.Logger;
-import org.slf4j.LoggerFactory;
-
-import java.io.IOException;
-import java.util.concurrent.CompletableFuture;
-
-import static org.apache.raft.shaded.proto.RaftProtos.LogEntryProto.LogEntryBodyCase.CONFIGURATIONENTRY;
-import static org.apache.raft.shaded.proto.RaftProtos.LogEntryProto.LogEntryBodyCase.SMLOGENTRY;
-
-/**
- * This class tracks the log entries that have been committed in a quorum and
- * applies them to the state machine. We let a separate thread do this work
- * asynchronously so that this will not block normal raft protocol.
- *
- * If the auto log compaction is enabled, the state machine updater thread will
- * trigger a snapshot of the state machine by calling
- * {@link StateMachine#takeSnapshot} when the log size exceeds a limit.
- */
-class StateMachineUpdater implements Runnable {
-  static final Logger LOG = LoggerFactory.getLogger(StateMachineUpdater.class);
-
-  enum State {
-    RUNNING, STOP, RELOAD
-  }
-
-  private final RaftProperties properties;
-  private final StateMachine stateMachine;
-  private final RaftServer server;
-  private final RaftLog raftLog;
-
-  private volatile long lastAppliedIndex;
-
-  private final boolean autoSnapshotEnabled;
-  private final long snapshotThreshold;
-  private long lastSnapshotIndex;
-
-  private final Thread updater;
-  private volatile State state = State.RUNNING;
-
-  StateMachineUpdater(StateMachine stateMachine, RaftServer server,
-      RaftLog raftLog, long lastAppliedIndex, RaftProperties properties) {
-    this.properties = properties;
-    this.stateMachine = stateMachine;
-    this.server = server;
-    this.raftLog = raftLog;
-
-    this.lastAppliedIndex = lastAppliedIndex;
-    lastSnapshotIndex = lastAppliedIndex;
-
-    autoSnapshotEnabled = properties.getBoolean(
-        RaftServerConfigKeys.RAFT_SERVER_AUTO_SNAPSHOT_ENABLED_KEY,
-        RaftServerConfigKeys.RAFT_SERVER_AUTO_SNAPSHOT_ENABLED_DEFAULT);
-    snapshotThreshold = properties.getLong(
-        RaftServerConfigKeys.RAFT_SERVER_SNAPSHOT_TRIGGER_THRESHOLD_KEY,
-        RaftServerConfigKeys.RAFT_SERVER_SNAPSHOT_TRIGGER_THRESHOLD_DEFAULT);
-    updater = new Daemon(this);
-  }
-
-  void start() {
-    updater.start();
-  }
-
-  void stop() {
-    state = State.STOP;
-    updater.interrupt();
-    try {
-      stateMachine.close();
-    } catch (IOException ignored) {
-    }
-  }
-
-  void reloadStateMachine() {
-    state = State.RELOAD;
-    notifyUpdater();
-  }
-
-  synchronized void notifyUpdater() {
-    notifyAll();
-  }
-
-  @Override
-  public String toString() {
-    return this.getClass().getSimpleName() + "-" + raftLog.getSelfId();
-  }
-
-  @Override
-  public void run() {
-    final RaftStorage storage = server.getState().getStorage();
-    while (isRunning()) {
-      try {
-        synchronized (this) {
-          // when the peers just start, the committedIndex is initialized as 0
-          // and will be updated only after the leader contacts other peers.
-          // Thus initially lastAppliedIndex can be greater than lastCommitted.
-          while (lastAppliedIndex >= raftLog.getLastCommittedIndex()) {
-            wait();
-          }
-        }
-
-        final long committedIndex = raftLog.getLastCommittedIndex();
-        Preconditions.checkState(lastAppliedIndex < committedIndex);
-
-        if (state == State.RELOAD) {
-          Preconditions.checkState(stateMachine.getLifeCycleState() == LifeCycle.State.PAUSED);
-
-          stateMachine.reinitialize(server.getId(), properties, storage);
-
-          SnapshotInfo snapshot = stateMachine.getLatestSnapshot();
-          Preconditions.checkState(snapshot != null && snapshot.getIndex() > lastAppliedIndex,
-              "Snapshot: %s, lastAppliedIndex: %s", snapshot, lastAppliedIndex);
-
-          lastAppliedIndex = snapshot.getIndex();
-          lastSnapshotIndex = snapshot.getIndex();
-          state = State.RUNNING;
-        }
-
-        while (lastAppliedIndex < committedIndex) {
-          final LogEntryProto next = raftLog.get(lastAppliedIndex + 1);
-          if (next != null) {
-            if (next.getLogEntryBodyCase() == CONFIGURATIONENTRY) {
-              // the reply should have already been set. only need to record
-              // the new conf in the state machine.
-              stateMachine.setRaftConfiguration(
-                  ServerProtoUtils.toRaftConfiguration(next.getIndex(),
-                      next.getConfigurationEntry()));
-            } else if (next.getLogEntryBodyCase() == SMLOGENTRY) {
-              // check whether there is a TransactionContext because we are the leader.
-              TransactionContext trx = server.getTransactionContext(next.getIndex());
-              if (trx == null) {
-                trx = new TransactionContext(stateMachine, next);
-              }
-
-              // Let the StateMachine inject logic for committed transactions in sequential order.
-              trx = stateMachine.applyTransactionSerial(trx);
-
-              // TODO: This step can be parallelized
-              CompletableFuture<Message> messageFuture =
-                  stateMachine.applyTransaction(trx);
-              server.replyPendingRequest(next.getIndex(), messageFuture);
-            }
-            lastAppliedIndex++;
-          } else {
-            LOG.debug("{}: logEntry {} is null. There may be snapshot to load. state:{}",
-                this, lastAppliedIndex + 1, state);
-            break;
-          }
-        }
-
-        // check if need to trigger a snapshot
-        if (shouldTakeSnapshot(lastAppliedIndex)) {
-          stateMachine.takeSnapshot();
-          // TODO purge logs, including log cache. but should keep log for leader's RPCSenders
-          lastSnapshotIndex = lastAppliedIndex;
-        }
-      } catch (InterruptedException e) {
-        if (!isRunning()) {
-          LOG.info("{}: the StateMachineUpdater is interrupted and will exit.", this);
-        } else {
-          final String s = this + ": the StateMachineUpdater is wrongly interrupted";
-          ExitUtils.terminate(1, s, e, LOG);
-        }
-      } catch (Throwable t) {
-        final String s = this + ": the StateMachineUpdater hits Throwable";
-        ExitUtils.terminate(2, s, t, LOG);
-      }
-    }
-  }
-
-  private boolean isRunning() {
-    return state != State.STOP;
-  }
-
-  private boolean shouldTakeSnapshot(long currentAppliedIndex) {
-    return autoSnapshotEnabled && (state != State.RELOAD) &&
-        (currentAppliedIndex - lastSnapshotIndex >= snapshotThreshold);
-  }
-
-  long getLastAppliedIndex() {
-    return lastAppliedIndex;
-  }
-}

http://git-wip-us.apache.org/repos/asf/incubator-ratis/blob/c36810ed/raft-server/src/main/java/org/apache/raft/server/impl/ConfigurationManager.java
----------------------------------------------------------------------
diff --git a/raft-server/src/main/java/org/apache/raft/server/impl/ConfigurationManager.java b/raft-server/src/main/java/org/apache/raft/server/impl/ConfigurationManager.java
new file mode 100644
index 0000000..b2f077d
--- /dev/null
+++ b/raft-server/src/main/java/org/apache/raft/server/impl/ConfigurationManager.java
@@ -0,0 +1,91 @@
+/**
+ * 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.raft.server.impl;
+
+import com.google.common.annotations.VisibleForTesting;
+import com.google.common.base.Preconditions;
+
+import java.util.*;
+
+/**
+ * Maintain the mappings between log index and corresponding raft configuration.
+ * Initialized when starting the raft peer. The mappings are loaded from the
+ * raft log, and updated while appending/truncating configuration related log
+ * entries.
+ */
+public class ConfigurationManager {
+  private RaftConfiguration initialConf;
+  private final NavigableMap<Long, RaftConfiguration> configurations =
+      new TreeMap<>();
+  /**
+   * The current raft configuration. If configurations is not empty, should be
+   * the last entry of the map. Otherwise is initialConf.
+   */
+  private RaftConfiguration currentConf;
+
+  public ConfigurationManager(RaftConfiguration initialConf) {
+    setInitialConf(initialConf);
+  }
+
+  @VisibleForTesting
+  public synchronized void setInitialConf(RaftConfiguration initialConf) {
+    /**
+     * initialConf should actually be defined as "final". But for tests we want
+     * to change the initial configuration to reflect the correct port binding.
+     */
+    this.initialConf = initialConf;
+    this.currentConf = initialConf;
+  }
+
+  public synchronized void addConfiguration(long logIndex,
+      RaftConfiguration conf) {
+    Preconditions.checkArgument(configurations.isEmpty() ||
+        configurations.lastEntry().getKey() < logIndex);
+    configurations.put(logIndex, conf);
+    this.currentConf = conf;
+  }
+
+  synchronized RaftConfiguration getCurrent() {
+    return currentConf;
+  }
+
+  /**
+   * Remove all the configurations whose log index is >= the given index.
+   * @param index The given index. All the configurations whose log index is >=
+   *              this value will be removed.
+   * @return The configuration with largest log index < the given index.
+   */
+  synchronized RaftConfiguration removeConfigurations(long index) {
+    SortedMap<Long, RaftConfiguration> toRemove = configurations.tailMap(index);
+    for (Iterator<Map.Entry<Long, RaftConfiguration>> iter =
+         toRemove.entrySet().iterator(); iter.hasNext();) {
+      iter.next();
+      iter.remove();
+    }
+    currentConf = configurations.isEmpty() ? initialConf :
+        configurations.lastEntry().getValue();
+    return currentConf;
+  }
+
+  @VisibleForTesting
+  synchronized int numOfConf() {
+    return 1 + configurations.size();
+  }
+
+  // TODO: remove Configuration entries after they are committed
+}

http://git-wip-us.apache.org/repos/asf/incubator-ratis/blob/c36810ed/raft-server/src/main/java/org/apache/raft/server/impl/FollowerInfo.java
----------------------------------------------------------------------
diff --git a/raft-server/src/main/java/org/apache/raft/server/impl/FollowerInfo.java b/raft-server/src/main/java/org/apache/raft/server/impl/FollowerInfo.java
new file mode 100644
index 0000000..683599e
--- /dev/null
+++ b/raft-server/src/main/java/org/apache/raft/server/impl/FollowerInfo.java
@@ -0,0 +1,103 @@
+/**
+ * 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.raft.server.impl;
+
+import org.apache.raft.protocol.RaftPeer;
+import org.apache.raft.util.Timestamp;
+
+import java.util.concurrent.atomic.AtomicLong;
+import java.util.concurrent.atomic.AtomicReference;
+
+public class FollowerInfo {
+  private final RaftPeer peer;
+  private final AtomicReference<Timestamp> lastRpcResponseTime;
+  private final AtomicReference<Timestamp> lastRpcSendTime;
+  private long nextIndex;
+  private final AtomicLong matchIndex;
+  private volatile boolean attendVote;
+
+  FollowerInfo(RaftPeer peer, Timestamp lastRpcTime, long nextIndex,
+      boolean attendVote) {
+    this.peer = peer;
+    this.lastRpcResponseTime = new AtomicReference<>(lastRpcTime);
+    this.lastRpcSendTime = new AtomicReference<>(lastRpcTime);
+    this.nextIndex = nextIndex;
+    this.matchIndex = new AtomicLong(0);
+    this.attendVote = attendVote;
+  }
+
+  public void updateMatchIndex(final long matchIndex) {
+    this.matchIndex.set(matchIndex);
+  }
+
+  long getMatchIndex() {
+    return matchIndex.get();
+  }
+
+  public synchronized long getNextIndex() {
+    return nextIndex;
+  }
+
+  public synchronized void updateNextIndex(long i) {
+    nextIndex = i;
+  }
+
+  public synchronized void decreaseNextIndex(long targetIndex) {
+    if (nextIndex > 0) {
+      nextIndex = Math.min(nextIndex - 1, targetIndex);
+    }
+  }
+
+  @Override
+  public String toString() {
+    return peer.getId() + "(next=" + nextIndex + ", match=" + matchIndex + "," +
+        " attendVote=" + attendVote +
+        ", lastRpcSendTime=" + lastRpcSendTime +
+        ", lastRpcResponseTime=" + lastRpcResponseTime + ")";
+  }
+
+  void startAttendVote() {
+    attendVote = true;
+  }
+
+  public boolean isAttendingVote() {
+    return attendVote;
+  }
+
+  public RaftPeer getPeer() {
+    return peer;
+  }
+
+  /** Update lastRpcResponseTime to the current time. */
+  public void updateLastRpcResponseTime() {
+    lastRpcResponseTime.set(new Timestamp());
+  }
+
+  public Timestamp getLastRpcResponseTime() {
+    return lastRpcResponseTime.get();
+  }
+
+  /** Update lastRpcSendTime to the current time. */
+  public void updateLastRpcSendTime() {
+    lastRpcSendTime.set(new Timestamp());
+  }
+
+  public Timestamp getLastRpcTime() {
+    return Timestamp.latest(lastRpcResponseTime.get(), lastRpcSendTime.get());
+  }
+}

http://git-wip-us.apache.org/repos/asf/incubator-ratis/blob/c36810ed/raft-server/src/main/java/org/apache/raft/server/impl/FollowerState.java
----------------------------------------------------------------------
diff --git a/raft-server/src/main/java/org/apache/raft/server/impl/FollowerState.java b/raft-server/src/main/java/org/apache/raft/server/impl/FollowerState.java
new file mode 100644
index 0000000..61b3c92
--- /dev/null
+++ b/raft-server/src/main/java/org/apache/raft/server/impl/FollowerState.java
@@ -0,0 +1,91 @@
+/**
+ * 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.raft.server.impl;
+
+import org.apache.raft.util.Daemon;
+import org.apache.raft.util.Timestamp;
+import org.slf4j.Logger;
+
+/**
+ * Used when the peer is a follower. Used to track the election timeout.
+ */
+class FollowerState extends Daemon {
+  static final Logger LOG = RaftServer.LOG;
+
+  private final RaftServer server;
+
+  private volatile Timestamp lastRpcTime = new Timestamp();
+  private volatile boolean monitorRunning = true;
+  private volatile boolean inLogSync = false;
+
+  FollowerState(RaftServer server) {
+    this.server = server;
+  }
+
+  void updateLastRpcTime(boolean inLogSync) {
+    lastRpcTime = new Timestamp();
+    LOG.trace("{} update last rpc time to {}", server.getId(), lastRpcTime);
+    this.inLogSync = inLogSync;
+  }
+
+  Timestamp getLastRpcTime() {
+    return lastRpcTime;
+  }
+
+  boolean shouldWithholdVotes() {
+    return lastRpcTime.elapsedTimeMs() < server.getMinTimeoutMs();
+  }
+
+  void stopRunning() {
+    this.monitorRunning = false;
+  }
+
+  @Override
+  public  void run() {
+    while (monitorRunning && server.isFollower()) {
+      final long electionTimeout = server.getRandomTimeoutMs();
+      try {
+        Thread.sleep(electionTimeout);
+        if (!monitorRunning || !server.isFollower()) {
+          LOG.info("{} heartbeat monitor quit", server.getId());
+          break;
+        }
+        synchronized (server) {
+          if (!inLogSync && lastRpcTime.elapsedTimeMs() >= electionTimeout) {
+            LOG.info("{} changes to {}, lastRpcTime:{}, electionTimeout:{}",
+                server.getId(), Role.CANDIDATE, lastRpcTime, electionTimeout);
+            // election timeout, should become a candidate
+            server.changeToCandidate();
+            break;
+          }
+        }
+      } catch (InterruptedException e) {
+        LOG.info(this + " was interrupted: " + e);
+        LOG.trace("TRACE", e);
+        return;
+      } catch (Exception e) {
+        LOG.warn(this + " caught an exception", e);
+      }
+    }
+  }
+
+  @Override
+  public String toString() {
+    return server.getId() + ": " + getClass().getSimpleName();
+  }
+}

http://git-wip-us.apache.org/repos/asf/incubator-ratis/blob/c36810ed/raft-server/src/main/java/org/apache/raft/server/impl/LeaderElection.java
----------------------------------------------------------------------
diff --git a/raft-server/src/main/java/org/apache/raft/server/impl/LeaderElection.java b/raft-server/src/main/java/org/apache/raft/server/impl/LeaderElection.java
new file mode 100644
index 0000000..10e901f
--- /dev/null
+++ b/raft-server/src/main/java/org/apache/raft/server/impl/LeaderElection.java
@@ -0,0 +1,241 @@
+/**
+ * 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.raft.server.impl;
+
+import com.google.common.base.Preconditions;
+import com.google.common.util.concurrent.ThreadFactoryBuilder;
+import org.apache.raft.protocol.RaftPeer;
+import org.apache.raft.server.protocol.TermIndex;
+import org.apache.raft.shaded.proto.RaftProtos.RequestVoteReplyProto;
+import org.apache.raft.shaded.proto.RaftProtos.RequestVoteRequestProto;
+import org.apache.raft.statemachine.SnapshotInfo;
+import org.apache.raft.util.Daemon;
+import org.apache.raft.util.ProtoUtils;
+import org.apache.raft.util.Timestamp;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.io.IOException;
+import java.util.ArrayList;
+import java.util.Collection;
+import java.util.List;
+import java.util.concurrent.*;
+import java.util.stream.Collectors;
+
+class LeaderElection extends Daemon {
+  public static final Logger LOG = LoggerFactory.getLogger(LeaderElection.class);
+
+  private ResultAndTerm logAndReturn(Result result,
+      List<RequestVoteReplyProto> responses,
+      List<Exception> exceptions, long newTerm) {
+    LOG.info(server.getId() + ": Election " + result + "; received "
+        + responses.size() + " response(s) "
+        + responses.stream().map(r -> ProtoUtils.toString(r)).collect(Collectors.toList())
+        + " and " + exceptions.size() + " exception(s); " + server.getState());
+    int i = 0;
+    for(Exception e : exceptions) {
+      LOG.info("  " + i++ + ": " + e);
+      LOG.trace("TRACE", e);
+    }
+    return new ResultAndTerm(result, newTerm);
+  }
+
+  enum Result {PASSED, REJECTED, TIMEOUT, DISCOVERED_A_NEW_TERM, SHUTDOWN}
+
+  private static class ResultAndTerm {
+    final Result result;
+    final long term;
+
+    ResultAndTerm(Result result, long term) {
+      this.result = result;
+      this.term = term;
+    }
+  }
+
+  private final RaftServer server;
+  private ExecutorCompletionService<RequestVoteReplyProto> service;
+  private ExecutorService executor;
+  private volatile boolean running;
+  /**
+   * The Raft configuration should not change while the peer is in candidate
+   * state. If the configuration changes, another peer should be acting as a
+   * leader and this LeaderElection session should end.
+   */
+  private final RaftConfiguration conf;
+  private final Collection<RaftPeer> others;
+
+  LeaderElection(RaftServer server) {
+    this.server = server;
+    conf = server.getRaftConf();
+    others = conf.getOtherPeers(server.getId());
+    this.running = true;
+  }
+
+  void stopRunning() {
+    this.running = false;
+  }
+
+  private void initExecutor() {
+    Preconditions.checkState(!others.isEmpty());
+    executor = Executors.newFixedThreadPool(others.size(),
+        new ThreadFactoryBuilder().setDaemon(true).build());
+    service = new ExecutorCompletionService<>(executor);
+  }
+
+  @Override
+  public void run() {
+    try {
+      askForVotes();
+    } catch (InterruptedException e) {
+      // the leader election thread is interrupted. The peer may already step
+      // down to a follower. The leader election should skip.
+      LOG.info("The leader election thread of peer {} is interrupted. " +
+          "Currently role: {}.", server.getId(), server.getRole());
+    } catch (IOException e) {
+      LOG.warn("Failed to persist votedFor/term. Exit the leader election.", e);
+      stopRunning();
+    }
+  }
+
+  /**
+   * After a peer changes its role to candidate, it invokes this method to
+   * send out requestVote rpc to all other peers.
+   */
+  private void askForVotes() throws InterruptedException, IOException {
+    final ServerState state = server.getState();
+    while (running && server.isCandidate()) {
+      // one round of requestVotes
+      final long electionTerm;
+      synchronized (server) {
+        electionTerm = state.initElection();
+        server.getState().persistMetadata();
+      }
+      LOG.info(state.getSelfId() + ": begin an election in Term "
+          + electionTerm);
+
+      TermIndex lastEntry = ServerProtoUtils.toTermIndex(
+          state.getLog().getLastEntry());
+      if (lastEntry == null) {
+        // lastEntry may need to be derived from snapshot
+        SnapshotInfo snapshot = state.getLatestSnapshot();
+        if (snapshot != null) {
+          lastEntry = snapshot.getTermIndex();
+        }
+      }
+
+      final ResultAndTerm r;
+      if (others.isEmpty()) {
+        r = new ResultAndTerm(Result.PASSED, electionTerm);
+      } else {
+        try {
+          initExecutor();
+          int submitted = submitRequests(electionTerm, lastEntry);
+          r = waitForResults(electionTerm, submitted);
+        } finally {
+          if (executor != null) {
+            executor.shutdown();
+          }
+        }
+      }
+
+      synchronized (server) {
+        if (electionTerm != state.getCurrentTerm() || !running ||
+            !server.isCandidate()) {
+          return; // term already passed or no longer a candidate.
+        }
+
+        switch (r.result) {
+          case PASSED:
+            server.changeToLeader();
+            return;
+          case SHUTDOWN:
+            LOG.info("{} received shutdown response when requesting votes.",
+                server.getId());
+            server.close();
+            return;
+          case REJECTED:
+          case DISCOVERED_A_NEW_TERM:
+            final long term = r.term > server.getState().getCurrentTerm() ?
+                r.term : server.getState().getCurrentTerm();
+            server.changeToFollower(term, true);
+            return;
+          case TIMEOUT:
+            // should start another election
+        }
+      }
+    }
+  }
+
+  private int submitRequests(final long electionTerm, final TermIndex lastEntry) {
+    int submitted = 0;
+    for (final RaftPeer peer : others) {
+      final RequestVoteRequestProto r = server.createRequestVoteRequest(
+          peer.getId(), electionTerm, lastEntry);
+      service.submit(
+          () -> server.getServerRpc().sendRequestVote(r));
+      submitted++;
+    }
+    return submitted;
+  }
+
+  private ResultAndTerm waitForResults(final long electionTerm,
+      final int submitted) throws InterruptedException {
+    final Timestamp timeout = new Timestamp().addTimeMs(server.getRandomTimeoutMs());
+    final List<RequestVoteReplyProto> responses = new ArrayList<>();
+    final List<Exception> exceptions = new ArrayList<>();
+    int waitForNum = submitted;
+    Collection<String> votedPeers = new ArrayList<>();
+    while (waitForNum > 0 && running && server.isCandidate()) {
+      final long waitTime = -timeout.elapsedTimeMs();
+      if (waitTime <= 0) {
+        return logAndReturn(Result.TIMEOUT, responses, exceptions, -1);
+      }
+
+      try {
+        final Future<RequestVoteReplyProto> future = service.poll(
+            waitTime, TimeUnit.MILLISECONDS);
+        if (future == null) {
+          continue; // poll timeout, continue to return Result.TIMEOUT
+        }
+
+        final RequestVoteReplyProto r = future.get();
+        responses.add(r);
+        if (r.getShouldShutdown()) {
+          return logAndReturn(Result.SHUTDOWN, responses, exceptions, -1);
+        }
+        if (r.getTerm() > electionTerm) {
+          return logAndReturn(Result.DISCOVERED_A_NEW_TERM, responses,
+              exceptions, r.getTerm());
+        }
+        if (r.getServerReply().getSuccess()) {
+          votedPeers.add(r.getServerReply().getReplyId());
+          if (conf.hasMajority(votedPeers, server.getId())) {
+            return logAndReturn(Result.PASSED, responses, exceptions, -1);
+          }
+        }
+      } catch(ExecutionException e) {
+        LOG.info("Got exception when requesting votes: " + e);
+        LOG.trace("TRACE", e);
+        exceptions.add(e);
+      }
+      waitForNum--;
+    }
+    // received all the responses
+    return logAndReturn(Result.REJECTED, responses, exceptions, -1);
+  }
+}

http://git-wip-us.apache.org/repos/asf/incubator-ratis/blob/c36810ed/raft-server/src/main/java/org/apache/raft/server/impl/LeaderState.java
----------------------------------------------------------------------
diff --git a/raft-server/src/main/java/org/apache/raft/server/impl/LeaderState.java b/raft-server/src/main/java/org/apache/raft/server/impl/LeaderState.java
new file mode 100644
index 0000000..79d0d9e
--- /dev/null
+++ b/raft-server/src/main/java/org/apache/raft/server/impl/LeaderState.java
@@ -0,0 +1,581 @@
+/**
+ * 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.raft.server.impl;
+
+import com.google.common.base.Preconditions;
+import org.apache.raft.conf.RaftProperties;
+import org.apache.raft.protocol.*;
+import org.apache.raft.server.storage.RaftLog;
+import org.apache.raft.shaded.proto.RaftProtos.LeaderNoOp;
+import org.apache.raft.shaded.proto.RaftProtos.LogEntryProto;
+import org.apache.raft.statemachine.TransactionContext;
+import org.apache.raft.util.Daemon;
+import org.apache.raft.util.ProtoUtils;
+import org.apache.raft.util.Timestamp;
+import org.slf4j.Logger;
+
+import java.io.IOException;
+import java.util.*;
+import java.util.concurrent.ArrayBlockingQueue;
+import java.util.concurrent.BlockingQueue;
+import java.util.concurrent.CompletableFuture;
+import java.util.concurrent.TimeUnit;
+import java.util.stream.Collectors;
+
+import static org.apache.raft.server.RaftServerConfigKeys.*;
+import static org.apache.raft.server.impl.LeaderState.StateUpdateEventType.*;
+
+/**
+ * States for leader only. It contains three different types of processors:
+ * 1. RPC senders: each thread is appending log to a follower
+ * 2. EventProcessor: a single thread updating the raft server's state based on
+ *                    status of log appending response
+ * 3. PendingRequestHandler: a handler sending back responses to clients when
+ *                           corresponding log entries are committed
+ */
+public class LeaderState {
+  private static final Logger LOG = RaftServer.LOG;
+
+  enum StateUpdateEventType {
+    STEPDOWN, UPDATECOMMIT, STAGINGPROGRESS
+  }
+
+  enum BootStrapProgress {
+    NOPROGRESS, PROGRESSING, CAUGHTUP
+  }
+
+  static class StateUpdateEvent {
+    final StateUpdateEventType type;
+    final long newTerm;
+
+    StateUpdateEvent(StateUpdateEventType type, long newTerm) {
+      this.type = type;
+      this.newTerm = newTerm;
+    }
+  }
+
+  static final StateUpdateEvent UPDATE_COMMIT_EVENT =
+      new StateUpdateEvent(StateUpdateEventType.UPDATECOMMIT, -1);
+  static final StateUpdateEvent STAGING_PROGRESS_EVENT =
+      new StateUpdateEvent(StateUpdateEventType.STAGINGPROGRESS, -1);
+
+  private final RaftServer server;
+  private final RaftLog raftLog;
+  private final long currentTerm;
+  private volatile ConfigurationStagingState stagingState;
+  private List<List<FollowerInfo>> voterLists;
+
+  /**
+   * The list of threads appending entries to followers.
+   * The list is protected by the RaftServer's lock.
+   */
+  private final List<LogAppender> senders;
+  private final BlockingQueue<StateUpdateEvent> eventQ;
+  private final EventProcessor processor;
+  private final PendingRequests pendingRequests;
+  private volatile boolean running = true;
+
+  private final int stagingCatchupGap;
+  private final int snapshotChunkMaxSize;
+  private final int syncInterval;
+
+  LeaderState(RaftServer server, RaftProperties properties) {
+    this.server = server;
+
+    stagingCatchupGap = properties.getInt(
+        RAFT_SERVER_STAGING_CATCHUP_GAP_KEY,
+        RAFT_SERVER_STAGING_CATCHUP_GAP_DEFAULT);
+    snapshotChunkMaxSize = properties.getInt(
+        RAFT_SNAPSHOT_CHUNK_MAX_SIZE_KEY,
+        RAFT_SNAPSHOT_CHUNK_MAX_SIZE_DEFAULT);
+    syncInterval = properties.getInt(
+        RAFT_SERVER_RPC_SLEEP_TIME_MS_KEY,
+        RAFT_SERVER_RPC_SLEEP_TIME_MS_DEFAULT);
+
+    final ServerState state = server.getState();
+    this.raftLog = state.getLog();
+    this.currentTerm = state.getCurrentTerm();
+    eventQ = new ArrayBlockingQueue<>(4096);
+    processor = new EventProcessor();
+    pendingRequests = new PendingRequests(server);
+
+    final RaftConfiguration conf = server.getRaftConf();
+    Collection<RaftPeer> others = conf.getOtherPeers(state.getSelfId());
+    final Timestamp t = new Timestamp().addTimeMs(-server.getMaxTimeoutMs());
+    final long nextIndex = raftLog.getNextIndex();
+    senders = new ArrayList<>(others.size());
+    for (RaftPeer p : others) {
+      FollowerInfo f = new FollowerInfo(p, t, nextIndex, true);
+      senders.add(server.getLogAppenderFactory().getLogAppender(server, this, f));
+    }
+    voterLists = divideFollowers(conf);
+  }
+
+  void start() {
+    // In the beginning of the new term, replicate an empty entry in order
+    // to finally commit entries in the previous term.
+    // Also this message can help identify the last committed index when
+    // the leader peer is just started.
+    final LogEntryProto placeHolder = LogEntryProto.newBuilder()
+        .setTerm(server.getState().getCurrentTerm())
+        .setIndex(raftLog.getNextIndex())
+        .setNoOp(LeaderNoOp.newBuilder()).build();
+    raftLog.append(placeHolder);
+
+    processor.start();
+    startSenders();
+  }
+
+  private void startSenders() {
+    senders.forEach(Thread::start);
+  }
+
+  void stop() {
+    this.running = false;
+    // do not interrupt event processor since it may be in the middle of logSync
+    for (LogAppender sender : senders) {
+      sender.stopSender();
+      sender.interrupt();
+    }
+    try {
+      pendingRequests.sendNotLeaderResponses();
+    } catch (IOException e) {
+      LOG.warn("Caught exception in sendNotLeaderResponses", e);
+    }
+  }
+
+  void notifySenders() {
+    senders.forEach(LogAppender::notifyAppend);
+  }
+
+  boolean inStagingState() {
+    return stagingState != null;
+  }
+
+  ConfigurationStagingState getStagingState() {
+    return stagingState;
+  }
+
+  long getCurrentTerm() {
+    return currentTerm;
+  }
+
+  int getSnapshotChunkMaxSize() {
+    return snapshotChunkMaxSize;
+  }
+
+  int getSyncInterval() {
+    return syncInterval;
+  }
+
+  /**
+   * Start bootstrapping new peers
+   */
+  PendingRequest startSetConfiguration(SetConfigurationRequest request) {
+    Preconditions.checkState(running && !inStagingState());
+
+    RaftPeer[] peersInNewConf = request.getPeersInNewConf();
+    Collection<RaftPeer> peersToBootStrap = RaftConfiguration
+        .computeNewPeers(peersInNewConf, server.getRaftConf());
+
+    // add the request to the pending queue
+    final PendingRequest pending = pendingRequests.addConfRequest(request);
+
+    ConfigurationStagingState stagingState = new ConfigurationStagingState(
+        peersToBootStrap, new PeerConfiguration(Arrays.asList(peersInNewConf)));
+    Collection<RaftPeer> newPeers = stagingState.getNewPeers();
+    // set the staging state
+    this.stagingState = stagingState;
+
+    if (newPeers.isEmpty()) {
+      applyOldNewConf();
+    } else {
+      // update the LeaderState's sender list
+      addSenders(newPeers);
+    }
+    return pending;
+  }
+
+  PendingRequest addPendingRequest(long index, RaftClientRequest request,
+      TransactionContext entry) {
+    return pendingRequests.addPendingRequest(index, request, entry);
+  }
+
+  private void applyOldNewConf() {
+    final ServerState state = server.getState();
+    final RaftConfiguration current = server.getRaftConf();
+    final RaftConfiguration oldNewConf= stagingState.generateOldNewConf(current,
+        state.getLog().getNextIndex());
+    // apply the (old, new) configuration to log, and use it as the current conf
+    long index = state.getLog().append(state.getCurrentTerm(), oldNewConf);
+    updateConfiguration(index, oldNewConf);
+
+    this.stagingState = null;
+    notifySenders();
+  }
+
+  private void updateConfiguration(long logIndex, RaftConfiguration newConf) {
+    voterLists = divideFollowers(newConf);
+    server.getState().setRaftConf(logIndex, newConf);
+  }
+
+  /**
+   * After receiving a setConfiguration request, the leader should update its
+   * RpcSender list.
+   */
+  void addSenders(Collection<RaftPeer> newMembers) {
+    final Timestamp t = new Timestamp().addTimeMs(-server.getMaxTimeoutMs());
+    final long nextIndex = raftLog.getNextIndex();
+    for (RaftPeer peer : newMembers) {
+      FollowerInfo f = new FollowerInfo(peer, t, nextIndex, false);
+      LogAppender sender = server.getLogAppenderFactory()
+          .getLogAppender(server, this, f);
+      senders.add(sender);
+      sender.start();
+    }
+  }
+
+  /**
+   * Update the RpcSender list based on the current configuration
+   */
+  private void updateSenders(RaftConfiguration conf) {
+    Preconditions.checkState(conf.isStable() && !inStagingState());
+    Iterator<LogAppender> iterator = senders.iterator();
+    while (iterator.hasNext()) {
+      LogAppender sender = iterator.next();
+      if (!conf.containsInConf(sender.getFollower().getPeer().getId())) {
+        iterator.remove();
+        sender.stopSender();
+        sender.interrupt();
+      }
+    }
+  }
+
+  void submitUpdateStateEvent(StateUpdateEvent event) {
+    try {
+      eventQ.put(event);
+    } catch (InterruptedException e) {
+      LOG.info("Interrupted when adding event {} into the queue", event);
+    }
+  }
+
+  private void prepare() {
+    synchronized (server) {
+      if (running) {
+        final RaftConfiguration conf = server.getRaftConf();
+        if (conf.isTransitional() && server.getState().isConfCommitted()) {
+          // the configuration is in transitional state, and has been committed
+          // so it is time to generate and replicate (new) conf.
+          replicateNewConf();
+        }
+      }
+    }
+  }
+
+  /**
+   * The processor thread takes the responsibility to update the raft server's
+   * state, such as changing to follower, or updating the committed index.
+   */
+  private class EventProcessor extends Daemon {
+    @Override
+    public void run() {
+      // apply an empty message; check if necessary to replicate (new) conf
+      prepare();
+
+      while (running) {
+        try {
+          StateUpdateEvent event = eventQ.poll(server.getMaxTimeoutMs(),
+              TimeUnit.MILLISECONDS);
+          synchronized (server) {
+            if (running) {
+              handleEvent(event);
+            }
+          }
+          // the updated configuration does not need to be sync'ed here
+        } catch (InterruptedException e) {
+          if (!running) {
+            LOG.info("The LeaderState gets is stopped");
+          } else {
+            LOG.warn("The leader election thread of peer {} is interrupted. "
+                + "Currently role: {}.", server.getId(), server.getRole());
+            throw new RuntimeException(e);
+          }
+        } catch (IOException e) {
+          LOG.warn("Failed to persist new votedFor/term.", e);
+          // the failure should happen while changing the state to follower
+          // thus the in-memory state should have been updated
+          Preconditions.checkState(!running);
+        }
+      }
+    }
+  }
+
+  private void handleEvent(StateUpdateEvent e) throws IOException {
+    if (e == null) {
+      if (inStagingState()) {
+        checkNewPeers();
+      }
+    } else {
+      if (e.type == STEPDOWN) {
+        server.changeToFollower(e.newTerm, true);
+      } else if (e.type == UPDATECOMMIT) {
+        updateLastCommitted();
+      } else if (e.type == STAGINGPROGRESS) {
+        checkNewPeers();
+      }
+    }
+  }
+
+  /**
+   * So far we use a simple implementation for catchup checking:
+   * 1. If the latest rpc time of the remote peer is before 3 * max_timeout,
+   *    the peer made no progress for that long. We should fail the whole
+   *    setConfiguration request.
+   * 2. If the peer's matching index is just behind for a small gap, and the
+   *    peer was updated recently (within max_timeout), declare the peer as
+   *    caught-up.
+   * 3. Otherwise the peer is making progressing. Keep waiting.
+   */
+  private BootStrapProgress checkProgress(FollowerInfo follower,
+      long committed) {
+    Preconditions.checkArgument(!follower.isAttendingVote());
+    final Timestamp progressTime = new Timestamp().addTimeMs(-server.getMaxTimeoutMs());
+    final Timestamp timeoutTime = new Timestamp().addTimeMs(-3*server.getMaxTimeoutMs());
+    if (follower.getLastRpcResponseTime().compareTo(timeoutTime) < 0) {
+      LOG.debug("{} detects a follower {} timeout for bootstrapping," +
+              " timeoutTime: {}", server.getId(), follower, timeoutTime);
+      return BootStrapProgress.NOPROGRESS;
+    } else if (follower.getMatchIndex() + stagingCatchupGap > committed
+        && follower.getLastRpcResponseTime().compareTo(progressTime) > 0) {
+      return BootStrapProgress.CAUGHTUP;
+    } else {
+      return BootStrapProgress.PROGRESSING;
+    }
+  }
+
+  private Collection<BootStrapProgress> checkAllProgress(long committed) {
+    Preconditions.checkState(inStagingState());
+    return senders.stream()
+        .filter(sender -> !sender.getFollower().isAttendingVote())
+        .map(sender -> checkProgress(sender.getFollower(), committed))
+        .collect(Collectors.toCollection(ArrayList::new));
+  }
+
+  private void checkNewPeers() {
+    if (!inStagingState()) {
+      // it is possible that the bootstrapping is done and we still have
+      // remaining STAGINGPROGRESS event to handle.
+      updateLastCommitted();
+    } else {
+      final long committedIndex = server.getState().getLog()
+          .getLastCommittedIndex();
+      Collection<BootStrapProgress> reports = checkAllProgress(committedIndex);
+      if (reports.contains(BootStrapProgress.NOPROGRESS)) {
+        LOG.debug("{} fails the setConfiguration request", server.getId());
+        stagingState.fail();
+      } else if (!reports.contains(BootStrapProgress.PROGRESSING)) {
+        // all caught up!
+        applyOldNewConf();
+        for (LogAppender sender : senders) {
+          sender.getFollower().startAttendVote();
+        }
+      }
+    }
+  }
+
+  boolean isBootStrappingPeer(String peerId) {
+    return inStagingState() && getStagingState().contains(peerId);
+  }
+
+  private void updateLastCommitted() {
+    final String selfId = server.getId();
+    final RaftConfiguration conf = server.getRaftConf();
+    long majorityInNewConf = computeLastCommitted(voterLists.get(0),
+        conf.containsInConf(selfId));
+    final long oldLastCommitted = raftLog.getLastCommittedIndex();
+    final LogEntryProto[] entriesToCommit;
+    if (!conf.isTransitional()) {
+      // copy the entries that may get committed out of the raftlog, to prevent
+      // the possible race that the log gets purged after the statemachine does
+      // a snapshot
+      entriesToCommit = raftLog.getEntries(oldLastCommitted + 1,
+          Math.max(majorityInNewConf, oldLastCommitted) + 1);
+      server.getState().updateStatemachine(majorityInNewConf, currentTerm);
+    } else { // configuration is in transitional state
+      long majorityInOldConf = computeLastCommitted(voterLists.get(1),
+          conf.containsInOldConf(selfId));
+      final long majority = Math.min(majorityInNewConf, majorityInOldConf);
+      entriesToCommit = raftLog.getEntries(oldLastCommitted + 1,
+          Math.max(majority, oldLastCommitted) + 1);
+      server.getState().updateStatemachine(majority, currentTerm);
+    }
+    checkAndUpdateConfiguration(entriesToCommit);
+  }
+
+  private boolean committedConf(LogEntryProto[] entries) {
+    final long currentCommitted = raftLog.getLastCommittedIndex();
+    for (LogEntryProto entry : entries) {
+      if (entry.getIndex() <= currentCommitted &&
+          ProtoUtils.isConfigurationLogEntry(entry)) {
+        return true;
+      }
+    }
+    return false;
+  }
+
+  private void checkAndUpdateConfiguration(LogEntryProto[] entriesToCheck) {
+    final RaftConfiguration conf = server.getRaftConf();
+    if (committedConf(entriesToCheck)) {
+      if (conf.isTransitional()) {
+        replicateNewConf();
+      } else { // the (new) log entry has been committed
+        LOG.debug("{} sends success to setConfiguration request", server.getId());
+        pendingRequests.replySetConfiguration();
+        // if the leader is not included in the current configuration, step down
+        if (!conf.containsInConf(server.getId())) {
+          LOG.info("{} is not included in the new configuration {}. Step down.",
+              server.getId(), conf);
+          try {
+            // leave some time for all RPC senders to send out new conf entry
+            Thread.sleep(server.getMinTimeoutMs());
+          } catch (InterruptedException ignored) {
+          }
+          // the pending request handler will send NotLeaderException for
+          // pending client requests when it stops
+          server.close();
+        }
+      }
+    }
+  }
+
+  /**
+   * when the (old, new) log entry has been committed, should replicate (new):
+   * 1) append (new) to log
+   * 2) update conf to (new)
+   * 3) update RpcSenders list
+   * 4) start replicating the log entry
+   */
+  private void replicateNewConf() {
+    final RaftConfiguration conf = server.getRaftConf();
+    final RaftConfiguration newConf = RaftConfiguration.newBuilder()
+        .setConf(conf)
+        .setLogEntryIndex(raftLog.getNextIndex())
+        .build();
+    // stop the LogAppender if the corresponding follower is no longer in the conf
+    updateSenders(newConf);
+    long index = raftLog.append(server.getState().getCurrentTerm(), newConf);
+    updateConfiguration(index, newConf);
+    notifySenders();
+  }
+
+  private long computeLastCommitted(List<FollowerInfo> followers,
+      boolean includeSelf) {
+    final int length = includeSelf ? followers.size() + 1 : followers.size();
+    final long[] indices = new long[length];
+    for (int i = 0; i < followers.size(); i++) {
+      indices[i] = followers.get(i).getMatchIndex();
+    }
+    if (includeSelf) {
+      // note that we also need to wait for the local disk I/O
+      indices[length - 1] = raftLog.getLatestFlushedIndex();
+    }
+
+    Arrays.sort(indices);
+    return indices[(indices.length - 1) / 2];
+  }
+
+  private List<List<FollowerInfo>> divideFollowers(RaftConfiguration conf) {
+    List<List<FollowerInfo>> lists = new ArrayList<>(2);
+    List<FollowerInfo> listForNew = senders.stream()
+        .filter(sender -> conf.containsInConf(sender.getFollower().getPeer().getId()))
+        .map(LogAppender::getFollower)
+        .collect(Collectors.toList());
+    lists.add(listForNew);
+    if (conf.isTransitional()) {
+      List<FollowerInfo> listForOld = senders.stream()
+          .filter(sender -> conf.containsInOldConf(sender.getFollower().getPeer().getId()))
+          .map(LogAppender::getFollower)
+          .collect(Collectors.toList());
+      lists.add(listForOld);
+    }
+    return lists;
+  }
+
+  PendingRequest returnNoConfChange(SetConfigurationRequest r) {
+    PendingRequest pending = new PendingRequest(r);
+    pending.setSuccessReply(null);
+    return pending;
+  }
+
+  void replyPendingRequest(long logIndex, CompletableFuture<Message> message) {
+    pendingRequests.replyPendingRequest(logIndex, message);
+  }
+
+  TransactionContext getTransactionContext(long index) {
+    return pendingRequests.getTransactionContext(index);
+  }
+
+  private class ConfigurationStagingState {
+    private final Map<String, RaftPeer> newPeers;
+    private final PeerConfiguration newConf;
+
+    ConfigurationStagingState(Collection<RaftPeer> newPeers,
+        PeerConfiguration newConf) {
+      Map<String, RaftPeer> map = new HashMap<>();
+      for (RaftPeer peer : newPeers) {
+        map.put(peer.getId(), peer);
+      }
+      this.newPeers = Collections.unmodifiableMap(map);
+      this.newConf = newConf;
+    }
+
+    RaftConfiguration generateOldNewConf(RaftConfiguration current,
+        long logIndex) {
+      return RaftConfiguration.newBuilder()
+          .setConf(newConf)
+          .setOldConf(current)
+          .setLogEntryIndex(logIndex)
+          .build();
+    }
+
+    Collection<RaftPeer> getNewPeers() {
+      return newPeers.values();
+    }
+
+    boolean contains(String peerId) {
+      return newPeers.containsKey(peerId);
+    }
+
+    void fail() {
+      Iterator<LogAppender> iterator = senders.iterator();
+      while (iterator.hasNext()) {
+        LogAppender sender = iterator.next();
+        if (!sender.getFollower().isAttendingVote()) {
+          iterator.remove();
+          sender.stopSender();
+          sender.interrupt();
+        }
+      }
+      LeaderState.this.stagingState = null;
+      // send back failure response to client's request
+      pendingRequests.failSetConfiguration(
+          new ReconfigurationTimeoutException("Fail to set configuration "
+              + newConf + ". Timeout when bootstrapping new peers."));
+    }
+  }
+}

http://git-wip-us.apache.org/repos/asf/incubator-ratis/blob/c36810ed/raft-server/src/main/java/org/apache/raft/server/impl/LogAppender.java
----------------------------------------------------------------------
diff --git a/raft-server/src/main/java/org/apache/raft/server/impl/LogAppender.java b/raft-server/src/main/java/org/apache/raft/server/impl/LogAppender.java
new file mode 100644
index 0000000..143da28
--- /dev/null
+++ b/raft-server/src/main/java/org/apache/raft/server/impl/LogAppender.java
@@ -0,0 +1,480 @@
+/**
+ * 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.raft.server.impl;
+
+import com.google.common.base.Preconditions;
+import com.google.common.collect.Lists;
+import org.apache.commons.io.IOUtils;
+import org.apache.raft.server.impl.LeaderState.StateUpdateEventType;
+import org.apache.raft.server.protocol.TermIndex;
+import org.apache.raft.server.storage.FileInfo;
+import org.apache.raft.server.storage.RaftLog;
+import org.apache.raft.shaded.com.google.protobuf.ByteString;
+import org.apache.raft.shaded.proto.RaftProtos.*;
+import org.apache.raft.statemachine.SnapshotInfo;
+import org.apache.raft.util.Daemon;
+import org.apache.raft.util.ProtoUtils;
+import org.apache.raft.util.Timestamp;
+import org.slf4j.Logger;
+
+import java.io.File;
+import java.io.FileInputStream;
+import java.io.IOException;
+import java.io.InterruptedIOException;
+import java.nio.file.Path;
+import java.util.*;
+
+import static org.apache.raft.server.RaftServerConfigKeys.*;
+import static org.apache.raft.server.impl.RaftServerConstants.INVALID_LOG_INDEX;
+
+/**
+ * A daemon thread appending log entries to a follower peer.
+ */
+public class LogAppender extends Daemon {
+  public static final Logger LOG = RaftServer.LOG;
+
+  protected final RaftServer server;
+  private final LeaderState leaderState;
+  protected final RaftLog raftLog;
+  protected final FollowerInfo follower;
+  private final int maxBufferSize;
+  private final boolean batchSending;
+  private final LogEntryBuffer buffer;
+  private final long leaderTerm;
+
+  private volatile boolean sending = true;
+
+  public LogAppender(RaftServer server, LeaderState leaderState, FollowerInfo f) {
+    this.follower = f;
+    this.server = server;
+    this.leaderState = leaderState;
+    this.raftLog = server.getState().getLog();
+    this.maxBufferSize = server.getProperties().getInt(
+        RAFT_SERVER_LOG_APPENDER_BUFFER_CAPACITY_KEY,
+        RAFT_SERVER_LOG_APPENDER_BUFFER_CAPACITY_DEFAULT);
+    this.batchSending = server.getProperties().getBoolean(
+        RAFT_SERVER_LOG_APPENDER_BATCH_ENABLED_KEY,
+        RAFT_SERVER_LOG_APPENDER_BATCH_ENABLED_DEFAULT);
+    this.buffer = new LogEntryBuffer();
+    this.leaderTerm = server.getState().getCurrentTerm();
+  }
+
+  @Override
+  public String toString() {
+    return getClass().getSimpleName() + "(" + server.getId() + " -> " +
+        follower.getPeer().getId() + ")";
+  }
+
+  @Override
+  public void run() {
+    try {
+      checkAndSendAppendEntries();
+    } catch (InterruptedException | InterruptedIOException e) {
+      LOG.info(this + " was interrupted: " + e);
+    }
+  }
+
+  protected boolean isAppenderRunning() {
+    return sending;
+  }
+
+  public void stopSender() {
+    this.sending = false;
+  }
+
+  public FollowerInfo getFollower() {
+    return follower;
+  }
+
+  /**
+   * A buffer for log entries with size limitation.
+   */
+  private class LogEntryBuffer {
+    private final List<LogEntryProto> buf = new ArrayList<>();
+    private int totalSize = 0;
+
+    void addEntry(LogEntryProto entry) {
+      buf.add(entry);
+      totalSize += entry.getSerializedSize();
+    }
+
+    boolean isFull() {
+      return totalSize >= maxBufferSize;
+    }
+
+    boolean isEmpty() {
+      return buf.isEmpty();
+    }
+
+    AppendEntriesRequestProto getAppendRequest(TermIndex previous) {
+      final AppendEntriesRequestProto request = server
+          .createAppendEntriesRequest(leaderTerm, follower.getPeer().getId(),
+              previous, buf, !follower.isAttendingVote());
+      buf.clear();
+      totalSize = 0;
+      return request;
+    }
+
+    int getPendingEntryNum() {
+      return buf.size();
+    }
+  }
+
+  private TermIndex getPrevious() {
+    TermIndex previous = ServerProtoUtils.toTermIndex(
+        raftLog.get(follower.getNextIndex() - 1));
+    if (previous == null) {
+      // if previous is null, nextIndex must be equal to the log start
+      // index (otherwise we will install snapshot).
+      Preconditions.checkState(follower.getNextIndex() == raftLog.getStartIndex(),
+          "follower's next index %s, local log start index %s",
+          follower.getNextIndex(), raftLog.getStartIndex());
+      SnapshotInfo snapshot = server.getState().getLatestSnapshot();
+      previous = snapshot == null ? null : snapshot.getTermIndex();
+    }
+    return previous;
+  }
+
+  protected AppendEntriesRequestProto createRequest() {
+    final TermIndex previous = getPrevious();
+    final long leaderNext = raftLog.getNextIndex();
+    long next = follower.getNextIndex() + buffer.getPendingEntryNum();
+    boolean toSend = false;
+
+    if (leaderNext == next && !buffer.isEmpty()) {
+      // no new entries, then send out the entries in the buffer
+      toSend = true;
+    } else if (leaderNext > next) {
+      while (leaderNext > next && !buffer.isFull()) {
+        // stop adding entry once the buffer size is >= the max size
+        buffer.addEntry(raftLog.get(next++));
+      }
+      if (buffer.isFull() || !batchSending) {
+        // buffer is full or batch sending is disabled, send out a request
+        toSend = true;
+      }
+    }
+
+    if (toSend || shouldHeartbeat()) {
+      return buffer.getAppendRequest(previous);
+    }
+    return null;
+  }
+
+  /** Send an appendEntries RPC; retry indefinitely. */
+  private AppendEntriesReplyProto sendAppendEntriesWithRetries()
+      throws InterruptedException, InterruptedIOException {
+    int retry = 0;
+    AppendEntriesRequestProto request = null;
+    while (isAppenderRunning()) { // keep retrying for IOException
+      try {
+        if (request == null || request.getEntriesCount() == 0) {
+          request = createRequest();
+        }
+
+        if (request == null) {
+          LOG.trace("{} need not send AppendEntries now." +
+              " Wait for more entries.", server.getId());
+          return null;
+        } else if (!isAppenderRunning()) {
+          LOG.debug("LogAppender {} has been stopped. Skip the request.", this);
+          return null;
+        }
+
+        follower.updateLastRpcSendTime();
+        final AppendEntriesReplyProto r = server.getServerRpc()
+            .sendAppendEntries(request);
+        follower.updateLastRpcResponseTime();
+
+        return r;
+      } catch (InterruptedIOException iioe) {
+        throw iioe;
+      } catch (IOException ioe) {
+        LOG.debug(this + ": failed to send appendEntries; retry " + retry++, ioe);
+      }
+      if (isAppenderRunning()) {
+        Thread.sleep(leaderState.getSyncInterval());
+      }
+    }
+    return null;
+  }
+
+  protected class SnapshotRequestIter
+      implements Iterable<InstallSnapshotRequestProto> {
+    private final SnapshotInfo snapshot;
+    private final List<FileInfo> files;
+    private FileInputStream in;
+    private int fileIndex = 0;
+
+    private FileInfo currentFileInfo;
+    private byte[] currentBuf;
+    private long currentFileSize;
+    private long currentOffset = 0;
+    private int chunkIndex = 0;
+
+    private final String requestId;
+    private int requestIndex = 0;
+
+    public SnapshotRequestIter(SnapshotInfo snapshot, String requestId)
+        throws IOException {
+      this.snapshot = snapshot;
+      this.requestId = requestId;
+      this.files = snapshot.getFiles();
+      if (files.size() > 0) {
+        startReadFile();
+      }
+    }
+
+    private void startReadFile() throws IOException {
+      currentFileInfo = files.get(fileIndex);
+      File snapshotFile = currentFileInfo.getPath().toFile();
+      currentFileSize = snapshotFile.length();
+      final int bufLength =
+          (int) Math.min(leaderState.getSnapshotChunkMaxSize(), currentFileSize);
+      currentBuf = new byte[bufLength];
+      currentOffset = 0;
+      chunkIndex = 0;
+      in = new FileInputStream(snapshotFile);
+    }
+
+    @Override
+    public Iterator<InstallSnapshotRequestProto> iterator() {
+      return new Iterator<InstallSnapshotRequestProto>() {
+        @Override
+        public boolean hasNext() {
+          return fileIndex < files.size();
+        }
+
+        @Override
+        public InstallSnapshotRequestProto next() {
+          if (fileIndex >= files.size()) {
+            throw new NoSuchElementException();
+          }
+          int targetLength = (int) Math.min(currentFileSize - currentOffset,
+              leaderState.getSnapshotChunkMaxSize());
+          FileChunkProto chunk;
+          try {
+            chunk = readFileChunk(currentFileInfo, in, currentBuf,
+                targetLength, currentOffset, chunkIndex);
+            boolean done = (fileIndex == files.size() - 1) &&
+                chunk.getDone();
+            InstallSnapshotRequestProto request =
+                server.createInstallSnapshotRequest(follower.getPeer().getId(),
+                    requestId, requestIndex++, snapshot,
+                    Lists.newArrayList(chunk), done);
+            currentOffset += targetLength;
+            chunkIndex++;
+
+            if (currentOffset >= currentFileSize) {
+              in.close();
+              fileIndex++;
+              if (fileIndex < files.size()) {
+                startReadFile();
+              }
+            }
+
+            return request;
+          } catch (IOException e) {
+            if (in != null) {
+              try {
+                in.close();
+              } catch (IOException ignored) {
+              }
+            }
+            LOG.warn("Got exception when preparing InstallSnapshot request", e);
+            throw new RuntimeException(e);
+          }
+        }
+      };
+    }
+  }
+
+  private FileChunkProto readFileChunk(FileInfo fileInfo,
+      FileInputStream in, byte[] buf, int length, long offset, int chunkIndex)
+      throws IOException {
+    FileChunkProto.Builder builder = FileChunkProto.newBuilder()
+        .setOffset(offset).setChunkIndex(chunkIndex);
+    IOUtils.readFully(in, buf, 0, length);
+    Path relativePath = server.getState().getStorage().getStorageDir()
+        .relativizeToRoot(fileInfo.getPath());
+    builder.setFilename(relativePath.toString());
+    builder.setDone(offset + length == fileInfo.getFileSize());
+    builder.setFileDigest(
+        ByteString.copyFrom(fileInfo.getFileDigest().getDigest()));
+    builder.setData(ByteString.copyFrom(buf, 0, length));
+    return builder.build();
+  }
+
+  private InstallSnapshotReplyProto installSnapshot(SnapshotInfo snapshot)
+      throws InterruptedException, InterruptedIOException {
+    String requestId = UUID.randomUUID().toString();
+    InstallSnapshotReplyProto reply = null;
+    try {
+      for (InstallSnapshotRequestProto request :
+          new SnapshotRequestIter(snapshot, requestId)) {
+        follower.updateLastRpcSendTime();
+        reply = server.getServerRpc().sendInstallSnapshot(request);
+        follower.updateLastRpcResponseTime();
+
+        if (!reply.getServerReply().getSuccess()) {
+          return reply;
+        }
+      }
+    } catch (InterruptedIOException iioe) {
+      throw iioe;
+    } catch (Exception ioe) {
+      LOG.warn(this + ": failed to install SnapshotInfo " + snapshot.getFiles(),
+          ioe);
+      return null;
+    }
+
+    if (reply != null) {
+      follower.updateMatchIndex(snapshot.getTermIndex().getIndex());
+      follower.updateNextIndex(snapshot.getTermIndex().getIndex() + 1);
+      LOG.info("{}: install snapshot-{} successfully on follower {}",
+          server.getId(), snapshot.getTermIndex().getIndex(), follower.getPeer());
+    }
+    return reply;
+  }
+
+  protected SnapshotInfo shouldInstallSnapshot() {
+    final long logStartIndex = raftLog.getStartIndex();
+    // we should install snapshot if the follower needs to catch up and:
+    // 1. there is no local log entry but there is snapshot
+    // 2. or the follower's next index is smaller than the log start index
+    if (follower.getNextIndex() < raftLog.getNextIndex()) {
+      SnapshotInfo snapshot = server.getState().getLatestSnapshot();
+      if (follower.getNextIndex() < logStartIndex ||
+          (logStartIndex == INVALID_LOG_INDEX && snapshot != null)) {
+        return snapshot;
+      }
+    }
+    return null;
+  }
+
+  /** Check and send appendEntries RPC */
+  private void checkAndSendAppendEntries()
+      throws InterruptedException, InterruptedIOException {
+    while (isAppenderRunning()) {
+      if (shouldSendRequest()) {
+        SnapshotInfo snapshot = shouldInstallSnapshot();
+        if (snapshot != null) {
+          LOG.info("{}: follower {}'s next index is {}," +
+              " log's start index is {}, need to install snapshot",
+              server.getId(), follower.getPeer(), follower.getNextIndex(),
+              raftLog.getStartIndex());
+
+          final InstallSnapshotReplyProto r = installSnapshot(snapshot);
+          if (r != null && r.getResult() == InstallSnapshotResult.NOT_LEADER) {
+            checkResponseTerm(r.getTerm());
+          } // otherwise if r is null, retry the snapshot installation
+        } else {
+          final AppendEntriesReplyProto r = sendAppendEntriesWithRetries();
+          if (r != null) {
+            handleReply(r);
+          }
+        }
+      }
+      if (isAppenderRunning() && !shouldAppendEntries(
+          follower.getNextIndex() + buffer.getPendingEntryNum())) {
+        final long waitTime = getHeartbeatRemainingTime(
+            follower.getLastRpcTime());
+        if (waitTime > 0) {
+          synchronized (this) {
+            wait(waitTime);
+          }
+        }
+      }
+    }
+  }
+
+  private void handleReply(AppendEntriesReplyProto reply) {
+    if (reply != null) {
+      switch (reply.getResult()) {
+        case SUCCESS:
+          final long oldNextIndex = follower.getNextIndex();
+          final long nextIndex = reply.getNextIndex();
+          if (nextIndex < oldNextIndex) {
+            throw new IllegalStateException("nextIndex=" + nextIndex
+                + " < oldNextIndex=" + oldNextIndex
+                + ", reply=" + ProtoUtils.toString(reply));
+          }
+
+          if (nextIndex > oldNextIndex) {
+            follower.updateMatchIndex(nextIndex - 1);
+            follower.updateNextIndex(nextIndex);
+            submitEventOnSuccessAppend();
+          }
+          break;
+        case NOT_LEADER:
+          // check if should step down
+          checkResponseTerm(reply.getTerm());
+          break;
+        case INCONSISTENCY:
+          follower.decreaseNextIndex(reply.getNextIndex());
+          break;
+        case UNRECOGNIZED:
+          LOG.warn("{} received UNRECOGNIZED AppendResult from {}",
+              server.getId(), follower.getPeer().getId());
+          break;
+      }
+    }
+  }
+
+  protected void submitEventOnSuccessAppend() {
+    LeaderState.StateUpdateEvent e = follower.isAttendingVote() ?
+        LeaderState.UPDATE_COMMIT_EVENT :
+        LeaderState.STAGING_PROGRESS_EVENT;
+    leaderState.submitUpdateStateEvent(e);
+  }
+
+  public synchronized void notifyAppend() {
+    this.notify();
+  }
+
+  /** Should the leader send appendEntries RPC to this follower? */
+  protected boolean shouldSendRequest() {
+    return shouldAppendEntries(follower.getNextIndex()) || shouldHeartbeat();
+  }
+
+  private boolean shouldAppendEntries(long followerIndex) {
+    return followerIndex < raftLog.getNextIndex();
+  }
+
+  private boolean shouldHeartbeat() {
+    return getHeartbeatRemainingTime(follower.getLastRpcTime()) <= 0;
+  }
+
+  /**
+   * @return the time in milliseconds that the leader should send a heartbeat.
+   */
+  protected long getHeartbeatRemainingTime(Timestamp lastTime) {
+    return server.getMinTimeoutMs() / 2 - lastTime.elapsedTimeMs();
+  }
+
+  protected void checkResponseTerm(long responseTerm) {
+    synchronized (server) {
+      if (isAppenderRunning() && follower.isAttendingVote()
+          && responseTerm > leaderState.getCurrentTerm()) {
+        leaderState.submitUpdateStateEvent(
+            new LeaderState.StateUpdateEvent(StateUpdateEventType.STEPDOWN,
+                responseTerm));
+      }
+    }
+  }
+}

http://git-wip-us.apache.org/repos/asf/incubator-ratis/blob/c36810ed/raft-server/src/main/java/org/apache/raft/server/impl/LogAppenderFactory.java
----------------------------------------------------------------------
diff --git a/raft-server/src/main/java/org/apache/raft/server/impl/LogAppenderFactory.java b/raft-server/src/main/java/org/apache/raft/server/impl/LogAppenderFactory.java
new file mode 100644
index 0000000..b5ed775
--- /dev/null
+++ b/raft-server/src/main/java/org/apache/raft/server/impl/LogAppenderFactory.java
@@ -0,0 +1,31 @@
+/**
+ * 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.raft.server.impl;
+
+public interface LogAppenderFactory {
+  LogAppender getLogAppender(RaftServer server, LeaderState state,
+      FollowerInfo f);
+
+  class SynchronousLogAppenderFactory implements LogAppenderFactory {
+    @Override
+    public LogAppender getLogAppender(RaftServer server, LeaderState state,
+        FollowerInfo f) {
+      return new LogAppender(server, state, f);
+    }
+  }
+}

http://git-wip-us.apache.org/repos/asf/incubator-ratis/blob/c36810ed/raft-server/src/main/java/org/apache/raft/server/impl/PeerConfiguration.java
----------------------------------------------------------------------
diff --git a/raft-server/src/main/java/org/apache/raft/server/impl/PeerConfiguration.java b/raft-server/src/main/java/org/apache/raft/server/impl/PeerConfiguration.java
new file mode 100644
index 0000000..774a0c5
--- /dev/null
+++ b/raft-server/src/main/java/org/apache/raft/server/impl/PeerConfiguration.java
@@ -0,0 +1,90 @@
+/**
+ * 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.raft.server.impl;
+
+import com.google.common.base.Preconditions;
+import org.apache.raft.protocol.RaftPeer;
+
+import java.util.*;
+
+/**
+ * The peer configuration of a raft cluster.
+ *
+ * The objects of this class are immutable.
+ */
+class PeerConfiguration {
+  private final Map<String, RaftPeer> peers;
+
+  PeerConfiguration(Iterable<RaftPeer> peers) {
+    Preconditions.checkNotNull(peers);
+    Map<String, RaftPeer> map = new HashMap<>();
+    for(RaftPeer p : peers) {
+      map.put(p.getId(), p);
+    }
+    this.peers = Collections.unmodifiableMap(map);
+    Preconditions.checkState(!this.peers.isEmpty());
+  }
+
+  Collection<RaftPeer> getPeers() {
+    return Collections.unmodifiableCollection(peers.values());
+  }
+
+  int size() {
+    return peers.size();
+  }
+
+  @Override
+  public String toString() {
+    return peers.values().toString();
+  }
+
+  RaftPeer getPeer(String id) {
+    return peers.get(id);
+  }
+
+  boolean contains(String id) {
+    return peers.containsKey(id);
+  }
+
+  List<RaftPeer> getOtherPeers(String selfId) {
+    List<RaftPeer> others = new ArrayList<>();
+    for (Map.Entry<String, RaftPeer> entry : peers.entrySet()) {
+      if (!selfId.equals(entry.getValue().getId())) {
+        others.add(entry.getValue());
+      }
+    }
+    return others;
+  }
+
+  boolean hasMajority(Collection<String> others, String selfId) {
+    Preconditions.checkArgument(!others.contains(selfId));
+    int num = 0;
+    if (contains(selfId)) {
+      num++;
+    }
+    for (String other : others) {
+      if (contains(other)) {
+        num++;
+      }
+      if (num > size() / 2) {
+        return true;
+      }
+    }
+    return false;
+  }
+}

http://git-wip-us.apache.org/repos/asf/incubator-ratis/blob/c36810ed/raft-server/src/main/java/org/apache/raft/server/impl/PendingRequest.java
----------------------------------------------------------------------
diff --git a/raft-server/src/main/java/org/apache/raft/server/impl/PendingRequest.java b/raft-server/src/main/java/org/apache/raft/server/impl/PendingRequest.java
new file mode 100644
index 0000000..689566a
--- /dev/null
+++ b/raft-server/src/main/java/org/apache/raft/server/impl/PendingRequest.java
@@ -0,0 +1,87 @@
+/**
+ * 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
+ * <p>
+ * http://www.apache.org/licenses/LICENSE-2.0
+ * <p>
+ * 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.raft.server.impl;
+
+import com.google.common.base.Preconditions;
+import org.apache.raft.protocol.Message;
+import org.apache.raft.protocol.RaftClientReply;
+import org.apache.raft.protocol.RaftClientRequest;
+import org.apache.raft.protocol.SetConfigurationRequest;
+import org.apache.raft.statemachine.TransactionContext;
+
+import java.util.concurrent.CompletableFuture;
+
+public class PendingRequest implements Comparable<PendingRequest> {
+  private final Long index;
+  private final RaftClientRequest request;
+  private final TransactionContext entry;
+  private final CompletableFuture<RaftClientReply> future;
+
+  PendingRequest(long index, RaftClientRequest request,
+                 TransactionContext entry) {
+    this.index = index;
+    this.request = request;
+    this.entry = entry;
+    this.future = new CompletableFuture<>();
+  }
+
+  PendingRequest(SetConfigurationRequest request) {
+    this(RaftServerConstants.INVALID_LOG_INDEX, request, null);
+  }
+
+  long getIndex() {
+    return index;
+  }
+
+  RaftClientRequest getRequest() {
+    return request;
+  }
+
+  public CompletableFuture<RaftClientReply> getFuture() {
+    return future;
+  }
+
+  TransactionContext getEntry() {
+    return entry;
+  }
+
+  synchronized void setException(Throwable e) {
+    Preconditions.checkArgument(e != null);
+    future.completeExceptionally(e);
+  }
+
+  synchronized void setReply(RaftClientReply r) {
+    Preconditions.checkArgument(r != null);
+    future.complete(r);
+  }
+
+  void setSuccessReply(Message message) {
+    setReply(new RaftClientReply(getRequest(), message));
+  }
+
+  @Override
+  public int compareTo(PendingRequest that) {
+    return Long.compare(this.index, that.index);
+  }
+
+  @Override
+  public String toString() {
+    return getClass().getSimpleName() + "(index=" + index
+        + ", request=" + request;
+  }
+}



[10/54] [abbrv] incubator-ratis git commit: Move RequestDispatcher code to RaftServerImpl.

Posted by ji...@apache.org.
Move RequestDispatcher code to RaftServerImpl.


Project: http://git-wip-us.apache.org/repos/asf/incubator-ratis/repo
Commit: http://git-wip-us.apache.org/repos/asf/incubator-ratis/commit/a38e2f71
Tree: http://git-wip-us.apache.org/repos/asf/incubator-ratis/tree/a38e2f71
Diff: http://git-wip-us.apache.org/repos/asf/incubator-ratis/diff/a38e2f71

Branch: refs/heads/master
Commit: a38e2f71ef0e84cb0af9a394544274e0dd56bcd9
Parents: 673a282
Author: Tsz-Wo Nicholas Sze <sz...@hortonworks.com>
Authored: Tue Jan 3 02:05:43 2017 +0800
Committer: Tsz-Wo Nicholas Sze <sz...@hortonworks.com>
Committed: Tue Jan 3 02:05:43 2017 +0800

----------------------------------------------------------------------
 .../RaftClientAsynchronousProtocol.java         |  30 ++++
 .../org/apache/raft/grpc/RaftGRpcService.java   |   6 +-
 .../grpc/client/RaftClientProtocolService.java  |  14 +-
 .../grpc/server/RaftServerProtocolService.java  |  15 +-
 .../raft/hadooprpc/server/HadoopRpcService.java |  18 +--
 .../raft/netty/server/NettyRpcService.java      |  15 +-
 .../java/org/apache/raft/server/RaftServer.java |   5 +-
 .../apache/raft/server/impl/RaftServerImpl.java |  72 +++++++++-
 .../raft/server/impl/RequestDispatcher.java     | 140 -------------------
 .../server/simulation/SimulatedServerRpc.java   |  23 ++-
 10 files changed, 142 insertions(+), 196 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-ratis/blob/a38e2f71/raft-common/src/main/java/org/apache/raft/protocol/RaftClientAsynchronousProtocol.java
----------------------------------------------------------------------
diff --git a/raft-common/src/main/java/org/apache/raft/protocol/RaftClientAsynchronousProtocol.java b/raft-common/src/main/java/org/apache/raft/protocol/RaftClientAsynchronousProtocol.java
new file mode 100644
index 0000000..3572b7e
--- /dev/null
+++ b/raft-common/src/main/java/org/apache/raft/protocol/RaftClientAsynchronousProtocol.java
@@ -0,0 +1,30 @@
+/**
+ * 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.raft.protocol;
+
+import java.io.IOException;
+import java.util.concurrent.CompletableFuture;
+
+/** Asynchronous version of {@link RaftClientProtocol}. */
+public interface RaftClientAsynchronousProtocol {
+  CompletableFuture<RaftClientReply> submitClientRequestAsync(
+      RaftClientRequest request) throws IOException;
+
+  CompletableFuture<RaftClientReply> setConfigurationAsync(
+      SetConfigurationRequest request) throws IOException;
+}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/incubator-ratis/blob/a38e2f71/raft-grpc/src/main/java/org/apache/raft/grpc/RaftGRpcService.java
----------------------------------------------------------------------
diff --git a/raft-grpc/src/main/java/org/apache/raft/grpc/RaftGRpcService.java b/raft-grpc/src/main/java/org/apache/raft/grpc/RaftGRpcService.java
index d0c98c3..1184e2e 100644
--- a/raft-grpc/src/main/java/org/apache/raft/grpc/RaftGRpcService.java
+++ b/raft-grpc/src/main/java/org/apache/raft/grpc/RaftGRpcService.java
@@ -25,7 +25,6 @@ import org.apache.raft.grpc.server.RaftServerProtocolService;
 import org.apache.raft.protocol.RaftPeer;
 import org.apache.raft.server.RaftServer;
 import org.apache.raft.server.RaftServerRpc;
-import org.apache.raft.server.impl.RequestDispatcher;
 import org.apache.raft.shaded.io.grpc.Server;
 import org.apache.raft.shaded.io.grpc.ServerBuilder;
 import org.apache.raft.shaded.io.grpc.netty.NettyServerBuilder;
@@ -61,11 +60,10 @@ public class RaftGRpcService implements RaftServerRpc {
         RaftGrpcConfigKeys.RAFT_GRPC_MESSAGE_MAXSIZE_KEY,
         RaftGrpcConfigKeys.RAFT_GRPC_MESSAGE_MAXSIZE_DEFAULT);
     ServerBuilder serverBuilder = ServerBuilder.forPort(port);
-    final RequestDispatcher dispatcher = new RequestDispatcher(raftServer);
     selfId = raftServer.getId();
     server = ((NettyServerBuilder) serverBuilder).maxMessageSize(maxMessageSize)
-        .addService(new RaftServerProtocolService(selfId, dispatcher))
-        .addService(new RaftClientProtocolService(selfId, dispatcher))
+        .addService(new RaftServerProtocolService(selfId, raftServer))
+        .addService(new RaftClientProtocolService(selfId, raftServer))
         .build();
 
     // start service to determine the port (in case port is configured as 0)

http://git-wip-us.apache.org/repos/asf/incubator-ratis/blob/a38e2f71/raft-grpc/src/main/java/org/apache/raft/grpc/client/RaftClientProtocolService.java
----------------------------------------------------------------------
diff --git a/raft-grpc/src/main/java/org/apache/raft/grpc/client/RaftClientProtocolService.java b/raft-grpc/src/main/java/org/apache/raft/grpc/client/RaftClientProtocolService.java
index 32dbac7..8f41bdc 100644
--- a/raft-grpc/src/main/java/org/apache/raft/grpc/client/RaftClientProtocolService.java
+++ b/raft-grpc/src/main/java/org/apache/raft/grpc/client/RaftClientProtocolService.java
@@ -20,8 +20,8 @@ package org.apache.raft.grpc.client;
 import com.google.common.base.Preconditions;
 import org.apache.raft.client.impl.ClientProtoUtils;
 import org.apache.raft.grpc.RaftGrpcUtil;
+import org.apache.raft.protocol.RaftClientAsynchronousProtocol;
 import org.apache.raft.protocol.RaftClientReply;
-import org.apache.raft.server.impl.RequestDispatcher;
 import org.apache.raft.shaded.io.grpc.stub.StreamObserver;
 import org.apache.raft.shaded.proto.RaftProtos.RaftClientReplyProto;
 import org.apache.raft.shaded.proto.RaftProtos.RaftClientRequestProto;
@@ -65,18 +65,18 @@ public class RaftClientProtocolService extends RaftClientProtocolServiceImplBase
   private static final PendingAppend COMPLETED = new PendingAppend(Long.MAX_VALUE);
 
   private final String id;
-  private final RequestDispatcher dispatcher;
+  private final RaftClientAsynchronousProtocol client;
 
-  public RaftClientProtocolService(String id, RequestDispatcher dispatcher) {
+  public RaftClientProtocolService(String id, RaftClientAsynchronousProtocol client) {
     this.id = id;
-    this.dispatcher = dispatcher;
+    this.client = client;
   }
 
   @Override
   public void setConfiguration(SetConfigurationRequestProto request,
       StreamObserver<RaftClientReplyProto> responseObserver) {
     try {
-      CompletableFuture<RaftClientReply> future = dispatcher.setConfigurationAsync(
+      CompletableFuture<RaftClientReply> future = client.setConfigurationAsync(
           ClientProtoUtils.toSetConfigurationRequest(request));
       future.whenCompleteAsync((reply, exception) -> {
         if (exception != null) {
@@ -114,8 +114,8 @@ public class RaftClientProtocolService extends RaftClientProtocolServiceImplBase
           pendingList.add(p);
         }
 
-        CompletableFuture<RaftClientReply> future = dispatcher
-            .handleClientRequest(ClientProtoUtils.toRaftClientRequest(request));
+        CompletableFuture<RaftClientReply> future = client.submitClientRequestAsync(
+            ClientProtoUtils.toRaftClientRequest(request));
         future.whenCompleteAsync((reply, exception) -> {
           if (exception != null) {
             // TODO: the exception may be from either raft or state machine.

http://git-wip-us.apache.org/repos/asf/incubator-ratis/blob/a38e2f71/raft-grpc/src/main/java/org/apache/raft/grpc/server/RaftServerProtocolService.java
----------------------------------------------------------------------
diff --git a/raft-grpc/src/main/java/org/apache/raft/grpc/server/RaftServerProtocolService.java b/raft-grpc/src/main/java/org/apache/raft/grpc/server/RaftServerProtocolService.java
index 2f06c59..53dbb6a 100644
--- a/raft-grpc/src/main/java/org/apache/raft/grpc/server/RaftServerProtocolService.java
+++ b/raft-grpc/src/main/java/org/apache/raft/grpc/server/RaftServerProtocolService.java
@@ -18,7 +18,7 @@
 package org.apache.raft.grpc.server;
 
 import org.apache.raft.grpc.RaftGrpcUtil;
-import org.apache.raft.server.impl.RequestDispatcher;
+import org.apache.raft.server.protocol.RaftServerProtocol;
 import org.apache.raft.shaded.io.grpc.stub.StreamObserver;
 import org.apache.raft.shaded.proto.RaftProtos.*;
 import org.apache.raft.shaded.proto.grpc.RaftServerProtocolServiceGrpc.RaftServerProtocolServiceImplBase;
@@ -29,18 +29,18 @@ public class RaftServerProtocolService extends RaftServerProtocolServiceImplBase
   public static final Logger LOG = LoggerFactory.getLogger(RaftServerProtocolService.class);
 
   private final String id;
-  private final RequestDispatcher dispatcher;
+  private final RaftServerProtocol server;
 
-  public RaftServerProtocolService(String id, RequestDispatcher dispatcher) {
+  public RaftServerProtocolService(String id, RaftServerProtocol server) {
     this.id = id;
-    this.dispatcher = dispatcher;
+    this.server = server;
   }
 
   @Override
   public void requestVote(RequestVoteRequestProto request,
       StreamObserver<RequestVoteReplyProto> responseObserver) {
     try {
-      final RequestVoteReplyProto reply = dispatcher.requestVote(request);
+      final RequestVoteReplyProto reply = server.requestVote(request);
       responseObserver.onNext(reply);
       responseObserver.onCompleted();
     } catch (Throwable e) {
@@ -57,7 +57,7 @@ public class RaftServerProtocolService extends RaftServerProtocolServiceImplBase
       @Override
       public void onNext(AppendEntriesRequestProto request) {
         try {
-          final AppendEntriesReplyProto reply = dispatcher.appendEntries(request);
+          final AppendEntriesReplyProto reply = server.appendEntries(request);
           responseObserver.onNext(reply);
         } catch (Throwable e) {
           LOG.info("{} got exception when handling appendEntries {}: {}",
@@ -87,8 +87,7 @@ public class RaftServerProtocolService extends RaftServerProtocolServiceImplBase
       @Override
       public void onNext(InstallSnapshotRequestProto request) {
         try {
-          final InstallSnapshotReplyProto reply =
-              dispatcher.installSnapshot(request);
+          final InstallSnapshotReplyProto reply = server.installSnapshot(request);
           responseObserver.onNext(reply);
         } catch (Throwable e) {
           LOG.info("{} got exception when handling installSnapshot {}: {}",

http://git-wip-us.apache.org/repos/asf/incubator-ratis/blob/a38e2f71/raft-hadoop/src/main/java/org/apache/raft/hadooprpc/server/HadoopRpcService.java
----------------------------------------------------------------------
diff --git a/raft-hadoop/src/main/java/org/apache/raft/hadooprpc/server/HadoopRpcService.java b/raft-hadoop/src/main/java/org/apache/raft/hadooprpc/server/HadoopRpcService.java
index b73deca..24e1d2c 100644
--- a/raft-hadoop/src/main/java/org/apache/raft/hadooprpc/server/HadoopRpcService.java
+++ b/raft-hadoop/src/main/java/org/apache/raft/hadooprpc/server/HadoopRpcService.java
@@ -24,11 +24,12 @@ import org.apache.hadoop.ipc.RPC;
 import org.apache.raft.hadooprpc.Proxy;
 import org.apache.raft.hadooprpc.client.RaftClientProtocolPB;
 import org.apache.raft.hadooprpc.client.RaftClientProtocolServerSideTranslatorPB;
+import org.apache.raft.protocol.RaftClientProtocol;
 import org.apache.raft.protocol.RaftPeer;
 import org.apache.raft.server.RaftServer;
 import org.apache.raft.server.RaftServerConfigKeys;
 import org.apache.raft.server.RaftServerRpc;
-import org.apache.raft.server.impl.RequestDispatcher;
+import org.apache.raft.server.protocol.RaftServerProtocol;
 import org.apache.raft.shaded.com.google.protobuf.BlockingService;
 import org.apache.raft.shaded.com.google.protobuf.ServiceException;
 import org.apache.raft.shaded.proto.RaftProtos.*;
@@ -49,7 +50,6 @@ public class HadoopRpcService implements RaftServerRpc {
   static final String CLASS_NAME = HadoopRpcService.class.getSimpleName();
   public static final String SEND_SERVER_REQUEST = CLASS_NAME + ".sendServerRequest";
 
-  private final RequestDispatcher raftService;
   private final String id;
   private final RPC.Server ipcServer;
   private final InetSocketAddress ipcServerAddress;
@@ -60,12 +60,11 @@ public class HadoopRpcService implements RaftServerRpc {
       throws IOException {
     this.proxies = new PeerProxyMap<>(
         p -> new Proxy(RaftServerProtocolPB.class, p.getAddress(), conf));
-    this.raftService = new RequestDispatcher(server);
     this.id = server.getId();
-    this.ipcServer = newRpcServer(conf);
+    this.ipcServer = newRpcServer(server, conf);
     this.ipcServerAddress = ipcServer.getListenerAddress();
 
-    addRaftClientProtocol(conf);
+    addRaftClientProtocol(server, conf);
 
     LOG.info(getClass().getSimpleName() + " created RPC.Server at "
         + ipcServerAddress);
@@ -76,7 +75,8 @@ public class HadoopRpcService implements RaftServerRpc {
     return ipcServerAddress;
   }
 
-  private RPC.Server newRpcServer(final Configuration conf) throws IOException {
+  private RPC.Server newRpcServer(RaftServerProtocol serverProtocol, final Configuration conf)
+      throws IOException {
     final RaftServerConfigKeys.Get get = new RaftServerConfigKeys.Get() {
       @Override
       protected int getInt(String key, int defaultValue) {
@@ -94,7 +94,7 @@ public class HadoopRpcService implements RaftServerRpc {
 
     final BlockingService service
         = RaftServerProtocolService.newReflectiveBlockingService(
-            new RaftServerProtocolServerSideTranslatorPB(raftService));
+            new RaftServerProtocolServerSideTranslatorPB(serverProtocol));
     RPC.setProtocolEngine(conf, RaftServerProtocolPB.class, ProtobufRpcEngineShaded.class);
     return new RPC.Builder(conf)
         .setProtocol(RaftServerProtocolPB.class)
@@ -106,13 +106,13 @@ public class HadoopRpcService implements RaftServerRpc {
         .build();
   }
 
-  private void addRaftClientProtocol(Configuration conf) {
+  private void addRaftClientProtocol(RaftClientProtocol clientProtocol, Configuration conf) {
     final Class<?> protocol = RaftClientProtocolPB.class;
     RPC.setProtocolEngine(conf,protocol, ProtobufRpcEngineShaded.class);
 
     final BlockingService service
         = RaftClientProtocolService.newReflectiveBlockingService(
-        new RaftClientProtocolServerSideTranslatorPB(raftService));
+        new RaftClientProtocolServerSideTranslatorPB(clientProtocol));
     ipcServer.addProtocol(RPC.RpcKind.RPC_PROTOCOL_BUFFER, protocol, service);
   }
 

http://git-wip-us.apache.org/repos/asf/incubator-ratis/blob/a38e2f71/raft-netty/src/main/java/org/apache/raft/netty/server/NettyRpcService.java
----------------------------------------------------------------------
diff --git a/raft-netty/src/main/java/org/apache/raft/netty/server/NettyRpcService.java b/raft-netty/src/main/java/org/apache/raft/netty/server/NettyRpcService.java
index 08e379a..50833fb 100644
--- a/raft-netty/src/main/java/org/apache/raft/netty/server/NettyRpcService.java
+++ b/raft-netty/src/main/java/org/apache/raft/netty/server/NettyRpcService.java
@@ -31,7 +31,6 @@ import org.apache.raft.protocol.RaftClientReply;
 import org.apache.raft.protocol.RaftPeer;
 import org.apache.raft.server.RaftServer;
 import org.apache.raft.server.RaftServerRpc;
-import org.apache.raft.server.impl.RequestDispatcher;
 import org.apache.raft.shaded.io.netty.handler.codec.protobuf.ProtobufDecoder;
 import org.apache.raft.shaded.io.netty.handler.codec.protobuf.ProtobufEncoder;
 import org.apache.raft.shaded.io.netty.handler.codec.protobuf.ProtobufVarint32FrameDecoder;
@@ -56,7 +55,7 @@ public final class NettyRpcService implements RaftServerRpc {
   public static final String SEND_SERVER_REQUEST = CLASS_NAME + ".sendServerRequest";
 
   private final LifeCycle lifeCycle = new LifeCycle(getClass().getSimpleName());
-  private final RequestDispatcher raftService;
+  private final RaftServer server;
   private final String id;
 
   private final EventLoopGroup bossGroup = new NioEventLoopGroup();
@@ -76,7 +75,7 @@ public final class NettyRpcService implements RaftServerRpc {
 
   /** Constructs a netty server with the given port. */
   public NettyRpcService(int port, RaftServer server) {
-    this.raftService = new RequestDispatcher(server);
+    this.server = server;
     this.id = server.getId();
 
     final ChannelInitializer<SocketChannel> initializer
@@ -134,7 +133,7 @@ public final class NettyRpcService implements RaftServerRpc {
         case REQUESTVOTEREQUEST: {
           final RequestVoteRequestProto request = proto.getRequestVoteRequest();
           rpcRequest = request.getServerRequest();
-          final RequestVoteReplyProto reply = raftService.requestVote(request);
+          final RequestVoteReplyProto reply = server.requestVote(request);
           return RaftNettyServerReplyProto.newBuilder()
               .setRequestVoteReply(reply)
               .build();
@@ -142,7 +141,7 @@ public final class NettyRpcService implements RaftServerRpc {
         case APPENDENTRIESREQUEST: {
           final AppendEntriesRequestProto request = proto.getAppendEntriesRequest();
           rpcRequest = request.getServerRequest();
-          final AppendEntriesReplyProto reply = raftService.appendEntries(request);
+          final AppendEntriesReplyProto reply = server.appendEntries(request);
           return RaftNettyServerReplyProto.newBuilder()
               .setAppendEntriesReply(reply)
               .build();
@@ -150,7 +149,7 @@ public final class NettyRpcService implements RaftServerRpc {
         case INSTALLSNAPSHOTREQUEST: {
           final InstallSnapshotRequestProto request = proto.getInstallSnapshotRequest();
           rpcRequest = request.getServerRequest();
-          final InstallSnapshotReplyProto reply = raftService.installSnapshot(request);
+          final InstallSnapshotReplyProto reply = server.installSnapshot(request);
           return RaftNettyServerReplyProto.newBuilder()
               .setInstallSnapshotReply(reply)
               .build();
@@ -158,7 +157,7 @@ public final class NettyRpcService implements RaftServerRpc {
         case RAFTCLIENTREQUEST: {
           final RaftClientRequestProto request = proto.getRaftClientRequest();
           rpcRequest = request.getRpcRequest();
-          final RaftClientReply reply = raftService.submitClientRequest(
+          final RaftClientReply reply = server.submitClientRequest(
               ClientProtoUtils.toRaftClientRequest(request));
           return RaftNettyServerReplyProto.newBuilder()
               .setRaftClientReply(ClientProtoUtils.toRaftClientReplyProto(reply))
@@ -167,7 +166,7 @@ public final class NettyRpcService implements RaftServerRpc {
         case SETCONFIGURATIONREQUEST: {
           final SetConfigurationRequestProto request = proto.getSetConfigurationRequest();
           rpcRequest = request.getRpcRequest();
-          final RaftClientReply reply = raftService.setConfiguration(
+          final RaftClientReply reply = server.setConfiguration(
               ClientProtoUtils.toSetConfigurationRequest(request));
           return RaftNettyServerReplyProto.newBuilder()
               .setRaftClientReply(ClientProtoUtils.toRaftClientReplyProto(reply))

http://git-wip-us.apache.org/repos/asf/incubator-ratis/blob/a38e2f71/raft-server/src/main/java/org/apache/raft/server/RaftServer.java
----------------------------------------------------------------------
diff --git a/raft-server/src/main/java/org/apache/raft/server/RaftServer.java b/raft-server/src/main/java/org/apache/raft/server/RaftServer.java
index 7141eca..aa4dfbf 100644
--- a/raft-server/src/main/java/org/apache/raft/server/RaftServer.java
+++ b/raft-server/src/main/java/org/apache/raft/server/RaftServer.java
@@ -17,12 +17,15 @@
  */
 package org.apache.raft.server;
 
+import org.apache.raft.protocol.RaftClientAsynchronousProtocol;
+import org.apache.raft.protocol.RaftClientProtocol;
 import org.apache.raft.server.protocol.RaftServerProtocol;
 
 import java.io.Closeable;
 
 /** Raft server interface */
-public interface RaftServer extends RaftServerProtocol, Closeable {
+public interface RaftServer extends Closeable, RaftServerProtocol,
+    RaftClientProtocol, RaftClientAsynchronousProtocol {
   /** @return the server ID. */
   String getId();
 

http://git-wip-us.apache.org/repos/asf/incubator-ratis/blob/a38e2f71/raft-server/src/main/java/org/apache/raft/server/impl/RaftServerImpl.java
----------------------------------------------------------------------
diff --git a/raft-server/src/main/java/org/apache/raft/server/impl/RaftServerImpl.java b/raft-server/src/main/java/org/apache/raft/server/impl/RaftServerImpl.java
index 3026afa..1ea40f6 100644
--- a/raft-server/src/main/java/org/apache/raft/server/impl/RaftServerImpl.java
+++ b/raft-server/src/main/java/org/apache/raft/server/impl/RaftServerImpl.java
@@ -44,6 +44,7 @@ import java.util.Collection;
 import java.util.List;
 import java.util.OptionalLong;
 import java.util.concurrent.CompletableFuture;
+import java.util.concurrent.ExecutionException;
 
 import static org.apache.raft.shaded.proto.RaftProtos.AppendEntriesReplyProto.AppendResult.*;
 import static org.apache.raft.util.LifeCycle.State.*;
@@ -146,6 +147,7 @@ public class RaftServerImpl implements RaftServer {
     return serverRpc;
   }
 
+  @Override
   public void start() {
     lifeCycle.transition(STARTING);
     state.start();
@@ -186,11 +188,12 @@ public class RaftServerImpl implements RaftServer {
     return this.state;
   }
 
+  @Override
   public String getId() {
     return getState().getSelfId();
   }
 
-  public RaftConfiguration getRaftConf() {
+  RaftConfiguration getRaftConf() {
     return getState().getRaftConf();
   }
 
@@ -323,7 +326,7 @@ public class RaftServerImpl implements RaftServer {
   /**
    * @return null if the server is in leader state.
    */
-  CompletableFuture<RaftClientReply> checkLeaderState(
+  private CompletableFuture<RaftClientReply> checkLeaderState(
       RaftClientRequest request) {
     if (!isLeader()) {
       NotLeaderException exception = generateNotLeaderException();
@@ -355,7 +358,7 @@ public class RaftServerImpl implements RaftServer {
   /**
    * Handle a normal update request from client.
    */
-  public CompletableFuture<RaftClientReply> appendTransaction(
+  private CompletableFuture<RaftClientReply> appendTransaction(
       RaftClientRequest request, TransactionContext entry)
       throws RaftException {
     LOG.debug("{}: receive client request({})", getId(), request);
@@ -384,10 +387,71 @@ public class RaftServerImpl implements RaftServer {
     return pending.getFuture();
   }
 
+  @Override
+  public CompletableFuture<RaftClientReply> submitClientRequestAsync(
+      RaftClientRequest request) throws IOException {
+    // first check the server's leader state
+    CompletableFuture<RaftClientReply> reply = checkLeaderState(request);
+    if (reply != null) {
+      return reply;
+    }
+
+    // let the state machine handle read-only request from client
+    if (request.isReadOnly()) {
+      // TODO: We might not be the leader anymore by the time this completes. See the RAFT paper,
+      // section 8 (last part)
+      return stateMachine.query(request);
+    }
+
+    // TODO: this client request will not be added to pending requests
+    // until later which means that any failure in between will leave partial state in the
+    // state machine. We should call cancelTransaction() for failed requests
+    TransactionContext entry = stateMachine.startTransaction(request);
+    if (entry.getException().isPresent()) {
+      throw RaftUtils.asIOException(entry.getException().get());
+    }
+
+    return appendTransaction(request, entry);
+  }
+
+  @Override
+  public RaftClientReply submitClientRequest(RaftClientRequest request)
+      throws IOException {
+    return waitForReply(getId(), request, submitClientRequestAsync(request));
+  }
+
+  private static RaftClientReply waitForReply(String id, RaftClientRequest request,
+      CompletableFuture<RaftClientReply> future) throws IOException {
+    try {
+      return future.get();
+    } catch (InterruptedException e) {
+      final String s = id + ": Interrupted when waiting for reply, request=" + request;
+      LOG.info(s, e);
+      throw RaftUtils.toInterruptedIOException(s, e);
+    } catch (ExecutionException e) {
+      final Throwable cause = e.getCause();
+      if (cause == null) {
+        throw new IOException(e);
+      }
+      if (cause instanceof NotLeaderException) {
+        return new RaftClientReply(request, (NotLeaderException)cause);
+      } else {
+        throw RaftUtils.asIOException(cause);
+      }
+    }
+  }
+
+  @Override
+  public RaftClientReply setConfiguration(SetConfigurationRequest request)
+      throws IOException {
+    return waitForReply(getId(), request, setConfigurationAsync(request));
+  }
+
   /**
    * Handle a raft configuration change request from client.
    */
-  public CompletableFuture<RaftClientReply> setConfiguration(
+  @Override
+  public CompletableFuture<RaftClientReply> setConfigurationAsync(
       SetConfigurationRequest request) throws IOException {
     LOG.debug("{}: receive setConfiguration({})", getId(), request);
     lifeCycle.assertCurrentState(RUNNING);

http://git-wip-us.apache.org/repos/asf/incubator-ratis/blob/a38e2f71/raft-server/src/main/java/org/apache/raft/server/impl/RequestDispatcher.java
----------------------------------------------------------------------
diff --git a/raft-server/src/main/java/org/apache/raft/server/impl/RequestDispatcher.java b/raft-server/src/main/java/org/apache/raft/server/impl/RequestDispatcher.java
deleted file mode 100644
index 39a4ac8..0000000
--- a/raft-server/src/main/java/org/apache/raft/server/impl/RequestDispatcher.java
+++ /dev/null
@@ -1,140 +0,0 @@
-/**
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- *     http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-package org.apache.raft.server.impl;
-
-import com.google.common.base.Preconditions;
-import org.apache.raft.protocol.*;
-import org.apache.raft.server.RaftServer;
-import org.apache.raft.server.protocol.RaftServerProtocol;
-import org.apache.raft.shaded.proto.RaftProtos.*;
-import org.apache.raft.statemachine.StateMachine;
-import org.apache.raft.statemachine.TransactionContext;
-import org.apache.raft.util.RaftUtils;
-import org.slf4j.Logger;
-import org.slf4j.LoggerFactory;
-
-import java.io.IOException;
-import java.util.concurrent.CompletableFuture;
-import java.util.concurrent.ExecutionException;
-
-/**
- * Each RPC request is first handled by the RequestDispatcher:
- * 1. A request from another RaftPeer is to be handled by RaftServer.
- *
- * If the raft peer is the leader, then:
- *
- * 2. A read-only request from client is to be handled by the state machine.
- * 3. A write request from client is first validated by the state machine. The
- * state machine returns the content of the raft log entry, which is then passed
- * to the RaftServer for replication.
- */
-public class RequestDispatcher implements RaftClientProtocol, RaftServerProtocol {
-  static final Logger LOG = LoggerFactory.getLogger(RequestDispatcher.class);
-
-  private final RaftServerImpl server;
-  private final StateMachine stateMachine;
-
-  public RequestDispatcher(RaftServer server) {
-    Preconditions.checkArgument(server instanceof RaftServerImpl);
-    this.server = (RaftServerImpl)server;
-    this.stateMachine = this.server.getStateMachine();
-  }
-
-  public CompletableFuture<RaftClientReply> handleClientRequest(
-      RaftClientRequest request) throws IOException {
-    // first check the server's leader state
-    CompletableFuture<RaftClientReply> reply = server.checkLeaderState(request);
-    if (reply != null) {
-      return reply;
-    }
-
-    // let the state machine handle read-only request from client
-    if (request.isReadOnly()) {
-      // TODO: We might not be the leader anymore by the time this completes. See the RAFT paper,
-      // section 8 (last part)
-      return stateMachine.query(request);
-    }
-
-    // TODO: this client request will not be added to pending requests
-    // until later which means that any failure in between will leave partial state in the
-    // state machine. We should call cancelTransaction() for failed requests
-    TransactionContext entry = stateMachine.startTransaction(request);
-    if (entry.getException().isPresent()) {
-      throw RaftUtils.asIOException(entry.getException().get());
-    }
-
-    return server.appendTransaction(request, entry);
-  }
-
-  @Override
-  public RaftClientReply submitClientRequest(RaftClientRequest request)
-      throws IOException {
-    return waitForReply(server.getId(), request, handleClientRequest(request));
-  }
-
-  public CompletableFuture<RaftClientReply> setConfigurationAsync(
-      SetConfigurationRequest request) throws IOException {
-    return server.setConfiguration(request);
-  }
-
-  @Override
-  public RaftClientReply setConfiguration(SetConfigurationRequest request)
-      throws IOException {
-    return waitForReply(server.getId(), request, setConfigurationAsync(request));
-  }
-
-  private static RaftClientReply waitForReply(String serverId,
-      RaftClientRequest request, CompletableFuture<RaftClientReply> future)
-      throws IOException {
-    try {
-      return future.get();
-    } catch (InterruptedException e) {
-      final String s = serverId + ": Interrupted when waiting for reply, request=" + request;
-      LOG.info(s, e);
-      throw RaftUtils.toInterruptedIOException(s, e);
-    } catch (ExecutionException e) {
-      final Throwable cause = e.getCause();
-      if (cause == null) {
-        throw new IOException(e);
-      }
-      if (cause instanceof NotLeaderException) {
-        return new RaftClientReply(request, (NotLeaderException)cause);
-      } else {
-        throw RaftUtils.asIOException(cause);
-      }
-    }
-  }
-
-  @Override
-  public RequestVoteReplyProto requestVote(RequestVoteRequestProto request)
-      throws IOException {
-    return server.requestVote(request);
-  }
-
-  @Override
-  public AppendEntriesReplyProto appendEntries(AppendEntriesRequestProto request)
-      throws IOException {
-    return server.appendEntries(request);
-  }
-
-  @Override
-  public InstallSnapshotReplyProto installSnapshot(
-      InstallSnapshotRequestProto request) throws IOException {
-    return server.installSnapshot(request);
-  }
-}

http://git-wip-us.apache.org/repos/asf/incubator-ratis/blob/a38e2f71/raft-server/src/test/java/org/apache/raft/server/simulation/SimulatedServerRpc.java
----------------------------------------------------------------------
diff --git a/raft-server/src/test/java/org/apache/raft/server/simulation/SimulatedServerRpc.java b/raft-server/src/test/java/org/apache/raft/server/simulation/SimulatedServerRpc.java
index 8a7e752..799ee65 100644
--- a/raft-server/src/test/java/org/apache/raft/server/simulation/SimulatedServerRpc.java
+++ b/raft-server/src/test/java/org/apache/raft/server/simulation/SimulatedServerRpc.java
@@ -18,13 +18,10 @@
 package org.apache.raft.server.simulation;
 
 import com.google.common.util.concurrent.ThreadFactoryBuilder;
-import org.apache.raft.protocol.RaftClientReply;
-import org.apache.raft.protocol.RaftClientRequest;
-import org.apache.raft.protocol.RaftPeer;
-import org.apache.raft.protocol.SetConfigurationRequest;
-import org.apache.raft.server.impl.RaftServerImpl;
+import org.apache.raft.protocol.*;
 import org.apache.raft.server.RaftServerRpc;
-import org.apache.raft.server.impl.RequestDispatcher;
+import org.apache.raft.server.impl.RaftServerImpl;
+import org.apache.raft.server.protocol.RaftServerProtocol;
 import org.apache.raft.shaded.proto.RaftProtos.*;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
@@ -40,7 +37,6 @@ class SimulatedServerRpc implements RaftServerRpc {
   static final Logger LOG = LoggerFactory.getLogger(SimulatedServerRpc.class);
 
   private final RaftServerImpl server;
-  private final RequestDispatcher dispatcher;
   private final RequestHandler<RaftServerRequest, RaftServerReply> serverHandler;
   private final RequestHandler<RaftClientRequest, RaftClientReply> clientHandler;
   private final ExecutorService executor = Executors.newFixedThreadPool(3,
@@ -50,7 +46,6 @@ class SimulatedServerRpc implements RaftServerRpc {
       SimulatedRequestReply<RaftServerRequest, RaftServerReply> serverRequestReply,
       SimulatedRequestReply<RaftClientRequest, RaftClientReply> clientRequestReply) {
     this.server = server;
-    this.dispatcher = new RequestDispatcher(server);
     this.serverHandler = new RequestHandler<>(server.getId(),
         "serverHandler", serverRequestReply, serverHandlerImpl, 3);
     this.clientHandler = new RequestHandler<>(server.getId(),
@@ -125,13 +120,11 @@ class SimulatedServerRpc implements RaftServerRpc {
     public RaftServerReply handleRequest(RaftServerRequest r)
         throws IOException {
       if (r.isAppendEntries()) {
-        return new RaftServerReply(
-            dispatcher.appendEntries(r.getAppendEntries()));
+        return new RaftServerReply(server.appendEntries(r.getAppendEntries()));
       } else if (r.isRequestVote()) {
-        return new RaftServerReply(dispatcher.requestVote(r.getRequestVote()));
+        return new RaftServerReply(server.requestVote(r.getRequestVote()));
       } else if (r.isInstallSnapshot()) {
-        return new RaftServerReply(
-            dispatcher.installSnapshot(r.getInstallSnapshot()));
+        return new RaftServerReply(server.installSnapshot(r.getInstallSnapshot()));
       } else {
         throw new IllegalStateException("unexpected state");
       }
@@ -150,9 +143,9 @@ class SimulatedServerRpc implements RaftServerRpc {
         throws IOException {
       final CompletableFuture<RaftClientReply> future;
       if (request instanceof SetConfigurationRequest) {
-        future = dispatcher.setConfigurationAsync((SetConfigurationRequest) request);
+        future = server.setConfigurationAsync((SetConfigurationRequest) request);
       } else {
-        future = dispatcher.handleClientRequest(request);
+        future = server.submitClientRequestAsync(request);
       }
 
       future.whenCompleteAsync((reply, exception) -> {


[46/54] [abbrv] incubator-ratis git commit: Renamed the packages from raft to ratis in preperation for Apache Incubation - Moved all java packages from org.apache.raft to org.apache.ratis. - Moved native package to org_apache_ratis, and native lib to l

Posted by ji...@apache.org.
http://git-wip-us.apache.org/repos/asf/incubator-ratis/blob/7e71a2e0/raft-common/src/main/java/org/apache/raft/util/LifeCycle.java
----------------------------------------------------------------------
diff --git a/raft-common/src/main/java/org/apache/raft/util/LifeCycle.java b/raft-common/src/main/java/org/apache/raft/util/LifeCycle.java
deleted file mode 100644
index 7165662..0000000
--- a/raft-common/src/main/java/org/apache/raft/util/LifeCycle.java
+++ /dev/null
@@ -1,210 +0,0 @@
-/**
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- *     http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-package org.apache.raft.util;
-
-import com.google.common.base.Preconditions;
-import org.slf4j.Logger;
-import org.slf4j.LoggerFactory;
-
-import java.util.*;
-import java.util.concurrent.atomic.AtomicReference;
-
-/**
- * The life cycle of a machine.
- * <pre>
- *   -------------------------------------------------
- *  |        --------------------------------         |
- *  |       |     ------------------------   |        |
- *  |       |    |                        |  |        |
- *  |       |  PAUSED <---- PAUSING----   |  |        |
- *  |       |    |          ^     |    |  |  |        |
- *  |       |    V          |     |    V  V  V        V
- * NEW --> STARTING --> RUNNING --|--> CLOSING --> [CLOSED]
- *  ^       |    |          |     |       ^
- *  |       |    |          V     V       |
- *   -------      -------> EXCEPTION -----
- * </pre>
- * Note that there is no transition from PAUSING to CLOSING.
- */
-public class LifeCycle {
-  public static final Logger LOG = LoggerFactory.getLogger(LifeCycle.class);
-
-  /** The states in the life cycle. */
-  public enum State {
-    /** The machine is newly created and holds zero resource. */
-    NEW,
-    /** The machine is starting and does not yet provide any service. */
-    STARTING,
-    /** The machine is running and providing service. */
-    RUNNING,
-    /** The machine is pausing and stopping providing service. */
-    PAUSING,
-    /** The machine is paused and does not provide any service. */
-    PAUSED,
-    /** The machine catches an internal exception so that it must be closed. */
-    EXCEPTION,
-    /** The machine is closing, stopping providing service and releasing resources. */
-    CLOSING,
-    /** The machine is closed, a final state. */
-    CLOSED;
-
-    private static final Map<State, List<State>> PREDECESSORS;
-
-    /** Does this object equal to one of the given states? */
-    public boolean isOneOf(State... states) {
-      for(State e : states) {
-        if (e == this) {
-          return true;
-        }
-      }
-      return false;
-    }
-
-    static void put(State key, Map<State, List<State>> map, State... values) {
-      map.put(key, Collections.unmodifiableList(Arrays.asList(values)));
-    }
-
-    static {
-      final Map<State, List<State>> predecessors = new EnumMap<>(State.class);
-      put(NEW,       predecessors, STARTING);
-      put(STARTING,  predecessors, NEW, PAUSED);
-      put(RUNNING,   predecessors, STARTING);
-      put(PAUSING,   predecessors, RUNNING);
-      put(PAUSED,    predecessors, PAUSING);
-      put(EXCEPTION, predecessors, STARTING, PAUSING, RUNNING);
-      put(CLOSING,   predecessors, STARTING, RUNNING, PAUSING, PAUSED, EXCEPTION);
-      put(CLOSED,    predecessors, NEW, CLOSING);
-
-      PREDECESSORS = Collections.unmodifiableMap(predecessors);
-    }
-
-    /** Is the given transition valid? */
-    static boolean isValid(State from, State to) {
-      return PREDECESSORS.get(to).contains(from);
-    }
-
-    /** Validate the given transition. */
-    static void validate(Object name, State from, State to) {
-      LOG.debug("{}: {} -> {}", name, from, to);
-      if (LOG.isTraceEnabled()) {
-        LOG.trace("TRACE", new Throwable());
-      }
-
-      Preconditions.checkState(isValid(from, to),
-          "ILLEGAL TRANSITION: In %s, %s -> %s", name, from, to);
-    }
-  }
-
-  private volatile String name;
-  private final AtomicReference<State> current = new AtomicReference<>(State.NEW);
-
-  public LifeCycle(Object name) {
-    this.name = name.toString();
-    LOG.debug("{}: {}", name, current);
-  }
-
-  public void setName(String name) {
-    this.name = name;
-  }
-
-  /** Transition from the current state to the given state. */
-  public void transition(final State to) {
-    final State from = current.getAndSet(to);
-    State.validate(name, from, to);
-  }
-
-  /**
-   * If the current state is equal to the specified from state,
-   * then transition to the give to state; otherwise, make no change.
-   *
-   * @return true iff the current state is equal to the specified from state.
-   */
-  public boolean compareAndTransition(final State from, final State to) {
-    if (current.compareAndSet(from, to)) {
-      State.validate(name, from, to);
-      return true;
-    }
-    return false;
-  }
-
-  /** @return the current state. */
-  public State getCurrentState() {
-    return current.get();
-  }
-
-  /** Assert if the current state equals to one of the expected states. */
-  public void assertCurrentState(State... expected) {
-    final State c = getCurrentState();
-    if (!c.isOneOf(expected)) {
-      throw new IllegalStateException("STATE MISMATCHED: In " + name
-          + ", current state " + c + " is not one of the expected states "
-          + Arrays.toString(expected));
-    }
-  }
-
-  @Override
-  public String toString() {
-    return name + ":" + getCurrentState();
-  }
-
-  /** Run the given start method and transition the current state accordingly. */
-  public <T extends Throwable> void startAndTransition(
-      CheckedRunnable<T> startImpl, Class<? extends Throwable>... exceptionClasses)
-      throws T {
-    transition(State.STARTING);
-    try {
-      startImpl.run();
-      transition(State.RUNNING);
-    } catch (Throwable t) {
-      transition(RaftUtils.isInstance(t, exceptionClasses)?
-          State.NEW: State.EXCEPTION);
-      throw t;
-    }
-  }
-
-
-  /**
-   * Check the current state and, if applicable, run the given close method.
-   * This method can be called multiple times
-   * while the given close method will only be executed at most once.
-   */
-  public <T extends Throwable> void checkStateAndClose(
-      CheckedRunnable<T> closeImpl) throws T {
-    if (compareAndTransition(State.NEW, State.CLOSED)) {
-      return;
-    }
-
-    for(;;) {
-      final State c = getCurrentState();
-      if (c.isOneOf(State.CLOSING, State.CLOSED)) {
-        return; //already closing or closed.
-      }
-
-      if (compareAndTransition(c, State.CLOSING)) {
-        try {
-          closeImpl.run();
-        } finally {
-          transition(State.CLOSED);
-        }
-        return;
-      }
-
-      // lifecycle state is changed, retry.
-    }
-  }
-}

http://git-wip-us.apache.org/repos/asf/incubator-ratis/blob/7e71a2e0/raft-common/src/main/java/org/apache/raft/util/MD5FileUtil.java
----------------------------------------------------------------------
diff --git a/raft-common/src/main/java/org/apache/raft/util/MD5FileUtil.java b/raft-common/src/main/java/org/apache/raft/util/MD5FileUtil.java
deleted file mode 100644
index ddb81b7..0000000
--- a/raft-common/src/main/java/org/apache/raft/util/MD5FileUtil.java
+++ /dev/null
@@ -1,176 +0,0 @@
-/**
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- *     http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-package org.apache.raft.util;
-
-import com.google.common.base.Charsets;
-import org.apache.raft.io.MD5Hash;
-import org.slf4j.Logger;
-import org.slf4j.LoggerFactory;
-
-import java.io.*;
-import java.security.DigestInputStream;
-import java.security.MessageDigest;
-import java.util.regex.Matcher;
-import java.util.regex.Pattern;
-
-public abstract class MD5FileUtil {
-  public static final Logger LOG = LoggerFactory.getLogger(MD5FileUtil.class);
-
-  // TODO: we should provide something like Hadoop's checksum fs for the local filesystem
-  // so that individual state machines do not have to deal with checksumming/corruption prevention.
-  // Keep the checksum and data in the same block format instead of individual files.
-
-  public static final String MD5_SUFFIX = ".md5";
-  private static final Pattern LINE_REGEX =
-      Pattern.compile("([0-9a-f]{32}) [ *](.+)");
-
-  /**
-   * Verify that the previously saved md5 for the given file matches
-   * expectedMd5.
-   */
-  public static void verifySavedMD5(File dataFile, MD5Hash expectedMD5)
-      throws IOException {
-    MD5Hash storedHash = readStoredMd5ForFile(dataFile);
-    // Check the hash itself
-    if (!expectedMD5.equals(storedHash)) {
-      throw new IOException(
-          "File " + dataFile + " did not match stored MD5 checksum " +
-              " (stored: " + storedHash + ", computed: " + expectedMD5);
-    }
-  }
-
-  /**
-   * Read the md5 file stored alongside the given data file and match the md5
-   * file content.
-   * @param md5File the file containing data
-   * @return a matcher with two matched groups where group(1) is the md5 string
-   *         and group(2) is the data file path.
-   */
-  private static Matcher readStoredMd5(File md5File) throws IOException {
-    BufferedReader reader =
-        new BufferedReader(new InputStreamReader(new FileInputStream(
-            md5File), Charsets.UTF_8));
-    String md5Line;
-    try {
-      md5Line = reader.readLine();
-      if (md5Line == null) { md5Line = ""; }
-      md5Line = md5Line.trim();
-    } catch (IOException ioe) {
-      throw new IOException("Error reading md5 file at " + md5File, ioe);
-    } finally {
-      RaftUtils.cleanup(LOG, reader);
-    }
-
-    Matcher matcher = LINE_REGEX.matcher(md5Line);
-    if (!matcher.matches()) {
-      throw new IOException("Invalid MD5 file " + md5File + ": the content \""
-          + md5Line + "\" does not match the expected pattern.");
-    }
-    return matcher;
-  }
-
-  /**
-   * Read the md5 checksum stored alongside the given data file.
-   * @param dataFile the file containing data
-   * @return the checksum stored in dataFile.md5
-   */
-  public static MD5Hash readStoredMd5ForFile(File dataFile) throws IOException {
-    final File md5File = getDigestFileForFile(dataFile);
-    if (!md5File.exists()) {
-      return null;
-    }
-
-    final Matcher matcher = readStoredMd5(md5File);
-    String storedHash = matcher.group(1);
-    File referencedFile = new File(matcher.group(2));
-
-    // Sanity check: Make sure that the file referenced in the .md5 file at
-    // least has the same name as the file we expect
-    if (!referencedFile.getName().equals(dataFile.getName())) {
-      throw new IOException(
-          "MD5 file at " + md5File + " references file named " +
-              referencedFile.getName() + " but we expected it to reference " +
-              dataFile);
-    }
-    return new MD5Hash(storedHash);
-  }
-
-  /**
-   * Read dataFile and compute its MD5 checksum.
-   */
-  public static MD5Hash computeMd5ForFile(File dataFile) throws IOException {
-    InputStream in = new FileInputStream(dataFile);
-    try {
-      MessageDigest digester = MD5Hash.getDigester();
-      DigestInputStream dis = new DigestInputStream(in, digester);
-      RaftUtils.readFully(dis, 128*1024);
-
-      return new MD5Hash(digester.digest());
-    } finally {
-      RaftUtils.cleanup(LOG, in);
-    }
-  }
-
-  /**
-   * Save the ".md5" file that lists the md5sum of another file.
-   * @param dataFile the original file whose md5 was computed
-   * @param digest the computed digest
-   */
-  public static void saveMD5File(File dataFile, MD5Hash digest)
-      throws IOException {
-    final String digestString = StringUtils.bytes2HexString(digest.getDigest());
-    saveMD5File(dataFile, digestString);
-  }
-
-  private static void saveMD5File(File dataFile, String digestString)
-      throws IOException {
-    File md5File = getDigestFileForFile(dataFile);
-    String md5Line = digestString + " *" + dataFile.getName() + "\n";
-
-    AtomicFileOutputStream afos = new AtomicFileOutputStream(md5File);
-    afos.write(md5Line.getBytes(Charsets.UTF_8));
-    afos.close();
-
-    if (LOG.isDebugEnabled()) {
-      LOG.debug("Saved MD5 " + digestString + " to " + md5File);
-    }
-  }
-
-  public static void renameMD5File(File oldDataFile, File newDataFile)
-      throws IOException {
-    final File fromFile = getDigestFileForFile(oldDataFile);
-    if (!fromFile.exists()) {
-      throw new FileNotFoundException(fromFile + " does not exist.");
-    }
-
-    final String digestString = readStoredMd5(fromFile).group(1);
-    saveMD5File(newDataFile, digestString);
-
-    if (!fromFile.delete()) {
-      LOG.warn("deleting  " + fromFile.getAbsolutePath() + " FAILED");
-    }
-  }
-
-  /**
-   * @return a reference to the file with .md5 suffix that will
-   * contain the md5 checksum for the given data file.
-   */
-  public static File getDigestFileForFile(File file) {
-    return new File(file.getParentFile(), file.getName() + MD5_SUFFIX);
-  }
-}

http://git-wip-us.apache.org/repos/asf/incubator-ratis/blob/7e71a2e0/raft-common/src/main/java/org/apache/raft/util/NativeCodeLoader.java
----------------------------------------------------------------------
diff --git a/raft-common/src/main/java/org/apache/raft/util/NativeCodeLoader.java b/raft-common/src/main/java/org/apache/raft/util/NativeCodeLoader.java
deleted file mode 100644
index c741c55..0000000
--- a/raft-common/src/main/java/org/apache/raft/util/NativeCodeLoader.java
+++ /dev/null
@@ -1,70 +0,0 @@
-/**
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- *     http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-
-package org.apache.raft.util;
-
-import org.slf4j.Logger;
-import org.slf4j.LoggerFactory;
-
-/**
- * A helper to load the native raft code i.e. libraft.so.
- * This handles the fallback to either the bundled libraft-Linux-i386-32.so
- * or the default java implementations where appropriate.
- */
-public final class NativeCodeLoader {
-
-  private static final Logger LOG = LoggerFactory.getLogger(NativeCodeLoader.class);
-  
-  private static boolean nativeCodeLoaded = false;
-  
-  static {
-    // Try to load native raft library and set fallback flag appropriately
-    LOG.debug("Trying to load the custom-built native-raft library...");
-    try {
-      System.loadLibrary("raft");
-      LOG.debug("Loaded the native-raft library");
-      nativeCodeLoaded = true;
-    } catch (Throwable t) {
-      // Ignore failure to load
-      if(LOG.isDebugEnabled()) {
-        LOG.debug("Failed to load native-raft with error: " + t);
-        LOG.debug("java.library.path=" +
-            System.getProperty("java.library.path"));
-      }
-    }
-    
-    if (!nativeCodeLoaded) {
-      LOG.warn("Unable to load native-raft library for your platform... " +
-               "using builtin-java classes where applicable");
-    }
-  }
-
-  private NativeCodeLoader() {}
-
-  /**
-   * Check if native-raft code is loaded for this platform.
-   * 
-   * @return <code>true</code> if native-raft is loaded,
-   *         else <code>false</code>
-   */
-  public static boolean isNativeCodeLoaded() {
-    return nativeCodeLoaded;
-  }
-
-  public static native String getLibraryName();
-}

http://git-wip-us.apache.org/repos/asf/incubator-ratis/blob/7e71a2e0/raft-common/src/main/java/org/apache/raft/util/NativeCrc32.java
----------------------------------------------------------------------
diff --git a/raft-common/src/main/java/org/apache/raft/util/NativeCrc32.java b/raft-common/src/main/java/org/apache/raft/util/NativeCrc32.java
deleted file mode 100644
index 57083ce..0000000
--- a/raft-common/src/main/java/org/apache/raft/util/NativeCrc32.java
+++ /dev/null
@@ -1,142 +0,0 @@
-/**
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- *     http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-package org.apache.raft.util;
-
-import com.google.common.annotations.VisibleForTesting;
-import org.apache.raft.protocol.ChecksumException;
-
-import java.nio.ByteBuffer;
-
-/**
- * Wrapper around JNI support code to do checksum computation
- * natively.
- */
-class NativeCrc32 {
-  
-  /**
-   * Return true if the JNI-based native CRC extensions are available.
-   */
-  public static boolean isAvailable() {
-    if (System.getProperty("os.arch").toLowerCase().startsWith("sparc")) {
-      return false;
-    } else {
-      return NativeCodeLoader.isNativeCodeLoaded();
-    }
-  }
-
-  /**
-   * Verify the given buffers of data and checksums, and throw an exception
-   * if any checksum is invalid. The buffers given to this function should
-   * have their position initially at the start of the data, and their limit
-   * set at the end of the data. The position, limit, and mark are not
-   * modified.
-   * 
-   * @param bytesPerSum the chunk size (eg 512 bytes)
-   * @param checksumType the DataChecksum type constant (NULL is not supported)
-   * @param sums the DirectByteBuffer pointing at the beginning of the
-   *             stored checksums
-   * @param data the DirectByteBuffer pointing at the beginning of the
-   *             data to check
-   * @param basePos the position in the file where the data buffer starts 
-   * @param fileName the name of the file being verified
-   * @throws ChecksumException if there is an invalid checksum
-   */
-  public static void verifyChunkedSums(int bytesPerSum, int checksumType,
-      ByteBuffer sums, ByteBuffer data, String fileName, long basePos)
-      throws ChecksumException {
-    nativeComputeChunkedSums(bytesPerSum, checksumType,
-        sums, sums.position(),
-        data, data.position(), data.remaining(),
-        fileName, basePos, true);
-  }
-
-  public static void verifyChunkedSumsByteArray(int bytesPerSum,
-      int checksumType, byte[] sums, int sumsOffset, byte[] data,
-      int dataOffset, int dataLength, String fileName, long basePos)
-      throws ChecksumException {
-    nativeComputeChunkedSumsByteArray(bytesPerSum, checksumType,
-        sums, sumsOffset,
-        data, dataOffset, dataLength,
-        fileName, basePos, true);
-  }
-
-  public static void calculateChunkedSums(int bytesPerSum, int checksumType,
-      ByteBuffer sums, ByteBuffer data) {
-    nativeComputeChunkedSums(bytesPerSum, checksumType,
-        sums, sums.position(),
-        data, data.position(), data.remaining(),
-        "", 0, false);
-  }
-
-  public static void calculateChunkedSumsByteArray(int bytesPerSum,
-      int checksumType, byte[] sums, int sumsOffset, byte[] data,
-      int dataOffset, int dataLength) {
-    nativeComputeChunkedSumsByteArray(bytesPerSum, checksumType,
-        sums, sumsOffset,
-        data, dataOffset, dataLength,
-        "", 0, false);
-  }
-
-  /**
-   * Verify the given buffers of data and checksums, and throw an exception
-   * if any checksum is invalid. The buffers given to this function should
-   * have their position initially at the start of the data, and their limit
-   * set at the end of the data. The position, limit, and mark are not
-   * modified.  This method is retained only for backwards-compatibility with
-   * prior jar versions that need the corresponding JNI function.
-   *
-   * @param bytesPerSum the chunk size (eg 512 bytes)
-   * @param checksumType the DataChecksum type constant
-   * @param sums the DirectByteBuffer pointing at the beginning of the
-   *             stored checksums
-   * @param sumsOffset start offset in sums buffer
-   * @param data the DirectByteBuffer pointing at the beginning of the
-   *             data to check
-   * @param dataOffset start offset in data buffer
-   * @param dataLength length of data buffer
-   * @param fileName the name of the file being verified
-   * @param basePos the position in the file where the data buffer starts 
-   * @throws ChecksumException if there is an invalid checksum
-   * @deprecated use {@link #nativeComputeChunkedSums(int, int, ByteBuffer, int,
-   *   ByteBuffer, int, int, String, long, boolean)} instead
-   */
-  @Deprecated
-  @VisibleForTesting
-  static native void nativeVerifyChunkedSums(
-      int bytesPerSum, int checksumType,
-      ByteBuffer sums, int sumsOffset,
-      ByteBuffer data, int dataOffset, int dataLength,
-      String fileName, long basePos) throws ChecksumException;
-  
-  private static native void nativeComputeChunkedSums(
-      int bytesPerSum, int checksumType,
-      ByteBuffer sums, int sumsOffset,
-      ByteBuffer data, int dataOffset, int dataLength,
-      String fileName, long basePos, boolean verify);
-
-  private static native void nativeComputeChunkedSumsByteArray(
-      int bytesPerSum, int checksumType,
-      byte[] sums, int sumsOffset,
-      byte[] data, int dataOffset, int dataLength,
-      String fileName, long basePos, boolean verify);
-
-  // Copy the constants over from DataChecksum so that javah will pick them up
-  // and make them available in the native code header.
-  public static final int CHECKSUM_CRC32 = 1;  //DataChecksum.CHECKSUM_CRC32
-  public static final int CHECKSUM_CRC32C = 2; //DataChecksum.CHECKSUM_CRC32C
-}

http://git-wip-us.apache.org/repos/asf/incubator-ratis/blob/7e71a2e0/raft-common/src/main/java/org/apache/raft/util/NativeLibraryChecker.java
----------------------------------------------------------------------
diff --git a/raft-common/src/main/java/org/apache/raft/util/NativeLibraryChecker.java b/raft-common/src/main/java/org/apache/raft/util/NativeLibraryChecker.java
deleted file mode 100644
index dc18bbd..0000000
--- a/raft-common/src/main/java/org/apache/raft/util/NativeLibraryChecker.java
+++ /dev/null
@@ -1,64 +0,0 @@
-/**
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- *     http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-
-package org.apache.raft.util;
-
-import org.slf4j.Logger;
-import org.slf4j.LoggerFactory;
-
-public class NativeLibraryChecker {
-  public static final Logger LOG = LoggerFactory.getLogger(NativeLibraryChecker.class);
-
-  /**
-   * A tool to test native library availability, 
-   */
-  public static void main(String[] args) {
-    String usage = "NativeLibraryChecker [-a|-h]\n"
-        + "  -a  use -a to check all libraries are available\n"
-        + "      by default just check raft library (and\n"
-        + "      winutils.exe on Windows OS) is available\n"
-        + "      exit with error code 1 if check failed\n"
-        + "  -h  print this message\n";
-    if (args.length > 1 ||
-        (args.length == 1 &&
-            !(args[0].equals("-a") || args[0].equals("-h")))) {
-      System.err.println(usage);
-      ExitUtils.terminate(1, "Illegal arguments.", LOG);
-    }
-    if (args.length == 1) {
-      if (args[0].equals("-h")) {
-        System.out.println(usage);
-        return;
-      }
-    }
-    boolean nativeRaftLoaded = NativeCodeLoader.isNativeCodeLoaded();
-    String raftLibraryName = "";
-
-    if (nativeRaftLoaded) {
-      raftLibraryName = NativeCodeLoader.getLibraryName();
-    }
-
-    System.out.println("Native library checking:");
-    System.out.printf("raft:  %b %s%n", nativeRaftLoaded, raftLibraryName);
-
-    if (!nativeRaftLoaded) {
-      // return 1 to indicated check failed
-      ExitUtils.terminate(1, "Failed to load native library.", LOG);
-    }
-  }
-}

http://git-wip-us.apache.org/repos/asf/incubator-ratis/blob/7e71a2e0/raft-common/src/main/java/org/apache/raft/util/NetUtils.java
----------------------------------------------------------------------
diff --git a/raft-common/src/main/java/org/apache/raft/util/NetUtils.java b/raft-common/src/main/java/org/apache/raft/util/NetUtils.java
deleted file mode 100644
index 0c72d46..0000000
--- a/raft-common/src/main/java/org/apache/raft/util/NetUtils.java
+++ /dev/null
@@ -1,151 +0,0 @@
-/**
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- *     http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-package org.apache.raft.util;
-
-import com.google.common.base.Preconditions;
-import org.slf4j.Logger;
-import org.slf4j.LoggerFactory;
-
-import java.net.InetAddress;
-import java.net.InetSocketAddress;
-import java.net.URI;
-import java.net.UnknownHostException;
-import java.util.Map;
-import java.util.concurrent.ConcurrentHashMap;
-
-public abstract class NetUtils {
-  public static final Logger LOG = LoggerFactory.getLogger(NetUtils.class);
-
-  public static abstract class StaticResolution {
-    /** Host -> resolved name */
-    private static final Map<String, String> hostToResolved = new ConcurrentHashMap<>();
-
-    /** Adds a static resolution for host. */
-    public static void put(String host, String resolvedName) {
-      hostToResolved.put(host, resolvedName);
-    }
-
-    /** @return the resolved name, or null if the host is not found. */
-    public static String get(String host) {
-      return hostToResolved.get(host);
-    }
-  }
-
-  public static InetSocketAddress newInetSocketAddress(String address) {
-    if (address.charAt(0) == '/') {
-      address = address.substring(1);
-    }
-    try {
-      return createSocketAddr(address);
-    } catch (Exception e) {
-      LOG.trace("", e);
-      return null;
-    }
-  }
-
-  /**
-   * Util method to build socket addr from either:
-   *   <host>:<port>
-   *   <fs>://<host>:<port>/<path>
-   */
-  public static InetSocketAddress createSocketAddr(String target) {
-    return createSocketAddr(target, -1);
-  }
-
-  /**
-   * Util method to build socket addr from either:
-   *   <host>
-   *   <host>:<port>
-   *   <fs>://<host>:<port>/<path>
-   */
-  public static InetSocketAddress createSocketAddr(String target, int defaultPort) {
-    return createSocketAddr(target, defaultPort, null);
-  }
-
-  /**
-   * Create an InetSocketAddress from the given target string and
-   * default port. If the string cannot be parsed correctly, the
-   * <code>configName</code> parameter is used as part of the
-   * exception message, allowing the user to better diagnose
-   * the misconfiguration.
-   *
-   * @param target a string of either "host" or "host:port"
-   * @param defaultPort the default port if <code>target</code> does not
-   *                    include a port number
-   * @param propertyName the name of the configuration from which
-   *                   <code>target</code> was loaded. This is used in the
-   *                   exception message in the case that parsing fails.
-   */
-  public static InetSocketAddress createSocketAddr(
-      String target, int defaultPort, String propertyName) {
-    final String helpText = propertyName == null? ""
-        : " (property '" + propertyName + "')";
-    Preconditions.checkNotNull(target, "Target address cannot be null.%s", helpText);
-
-    target = target.trim();
-    boolean hasScheme = target.contains("://");
-    final URI uri;
-    try {
-      uri = hasScheme ? URI.create(target) : URI.create("dummyscheme://"+target);
-    } catch (IllegalArgumentException e) {
-      throw new IllegalArgumentException(
-          "Invalid host:port authority: " + target + helpText, e);
-    }
-
-    final String host = uri.getHost();
-    int port = uri.getPort();
-    if (port == -1) {
-      port = defaultPort;
-    }
-    final String path = uri.getPath();
-
-    if (host == null || port < 0
-        || (!hasScheme && path != null && !path.isEmpty())) {
-      throw new IllegalArgumentException(
-          "Invalid host:port authority: " + target + helpText);
-    }
-    return createSocketAddrForHost(host, port);
-  }
-
-  /**
-   * Create a socket address with the given host and port.  The hostname
-   * might be replaced with another host that was set via
-   * {@link StaticResolution#put(String, String)}.
-   * @param host the hostname or IP use to instantiate the object
-   * @param port the port number
-   * @return InetSocketAddress
-   */
-  public static InetSocketAddress createSocketAddrForHost(String host, int port) {
-    String staticHost = StaticResolution.get(host);
-    String resolveHost = (staticHost != null) ? staticHost : host;
-
-    InetSocketAddress addr;
-    try {
-      InetAddress iaddr = InetAddress.getByName(resolveHost);
-      // if there is a static entry for the host, make the returned
-      // address look like the original given host
-      if (staticHost != null) {
-        iaddr = InetAddress.getByAddress(host, iaddr.getAddress());
-      }
-      addr = new InetSocketAddress(iaddr, port);
-    } catch (UnknownHostException e) {
-      addr = InetSocketAddress.createUnresolved(host, port);
-    }
-    return addr;
-  }
-}

http://git-wip-us.apache.org/repos/asf/incubator-ratis/blob/7e71a2e0/raft-common/src/main/java/org/apache/raft/util/PeerProxyMap.java
----------------------------------------------------------------------
diff --git a/raft-common/src/main/java/org/apache/raft/util/PeerProxyMap.java b/raft-common/src/main/java/org/apache/raft/util/PeerProxyMap.java
deleted file mode 100644
index 3a01593..0000000
--- a/raft-common/src/main/java/org/apache/raft/util/PeerProxyMap.java
+++ /dev/null
@@ -1,127 +0,0 @@
-/**
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- *     http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-package org.apache.raft.util;
-
-import com.google.common.base.Preconditions;
-import org.apache.raft.protocol.RaftPeer;
-import org.slf4j.Logger;
-import org.slf4j.LoggerFactory;
-
-import java.io.Closeable;
-import java.io.IOException;
-import java.util.Map;
-import java.util.concurrent.ConcurrentHashMap;
-
-/** A map from peer id to peer and its proxy. */
-public class PeerProxyMap<PROXY extends Closeable> implements Closeable {
-  public static final Logger LOG = LoggerFactory.getLogger(PeerProxyMap.class);
-
-  /** Peer and its proxy. */
-  private class PeerAndProxy implements Closeable {
-    private final RaftPeer peer;
-    private volatile PROXY proxy = null;
-    private final LifeCycle lifeCycle;
-
-    PeerAndProxy(RaftPeer peer) {
-      this.peer = peer;
-      this.lifeCycle = new LifeCycle(peer);
-    }
-
-    RaftPeer getPeer() {
-      return peer;
-    }
-
-    PROXY getProxy() throws IOException {
-      if (proxy == null) {
-        synchronized (this) {
-          if (proxy == null) {
-            lifeCycle.startAndTransition(
-                () -> proxy = createProxy.apply(peer), IOException.class);
-          }
-        }
-      }
-      return proxy;
-    }
-
-    @Override
-    public synchronized void close() {
-      lifeCycle.checkStateAndClose(() -> {
-        if (proxy != null) {
-          try {
-            proxy.close();
-          } catch (IOException e) {
-            LOG.warn("Failed to close proxy for peer {}, proxy class: ",
-                peer, proxy.getClass());
-          }
-        }
-      });
-    }
-  }
-
-  private final Map<String, PeerAndProxy> peers = new ConcurrentHashMap<>();
-  private final Object resetLock = new Object();
-
-  private final CheckedFunction<RaftPeer, PROXY, IOException> createProxy;
-
-  public PeerProxyMap(CheckedFunction<RaftPeer, PROXY, IOException> createProxy) {
-    this.createProxy = createProxy;
-  }
-  public PeerProxyMap() {
-    this.createProxy = this::createProxyImpl;
-  }
-
-  public PROXY getProxy(String id) throws IOException {
-    PeerAndProxy p = peers.get(id);
-    if (p == null) {
-      synchronized (resetLock) {
-        p = peers.get(id);
-      }
-    }
-    Preconditions.checkNotNull(p, "Server %s not found; peers=%s",
-        id, peers.keySet());
-    return p.getProxy();
-  }
-
-  public void addPeers(Iterable<RaftPeer> newPeers) {
-    for(RaftPeer p : newPeers) {
-      peers.put(p.getId(), new PeerAndProxy(p));
-    }
-  }
-
-  public void putIfAbsent(RaftPeer p) {
-    peers.putIfAbsent(p.getId(), new PeerAndProxy(p));
-  }
-
-  public void resetProxy(String id) {
-    synchronized (resetLock) {
-      final PeerAndProxy pp = peers.remove(id);
-      final RaftPeer peer = pp.getPeer();
-      pp.close();
-      peers.put(id, new PeerAndProxy(peer));
-    }
-  }
-
-  public PROXY createProxyImpl(RaftPeer peer) throws IOException {
-    throw new UnsupportedOperationException();
-  }
-
-  @Override
-  public void close() {
-    peers.values().forEach(PeerAndProxy::close);
-  }
-}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/incubator-ratis/blob/7e71a2e0/raft-common/src/main/java/org/apache/raft/util/ProtoUtils.java
----------------------------------------------------------------------
diff --git a/raft-common/src/main/java/org/apache/raft/util/ProtoUtils.java b/raft-common/src/main/java/org/apache/raft/util/ProtoUtils.java
deleted file mode 100644
index 4027e6c..0000000
--- a/raft-common/src/main/java/org/apache/raft/util/ProtoUtils.java
+++ /dev/null
@@ -1,141 +0,0 @@
-/**
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- *     http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-package org.apache.raft.util;
-
-import org.apache.raft.protocol.RaftPeer;
-import org.apache.raft.shaded.com.google.protobuf.ByteString;
-import org.apache.raft.shaded.com.google.protobuf.ServiceException;
-import org.apache.raft.shaded.proto.RaftProtos.*;
-
-import java.io.IOException;
-import java.io.ObjectInputStream;
-import java.io.ObjectOutputStream;
-import java.util.Collection;
-import java.util.Iterator;
-import java.util.List;
-
-public class ProtoUtils {
-  public static ByteString toByteString(Object obj) {
-    final ByteString.Output byteOut = ByteString.newOutput();
-    try(final ObjectOutputStream objOut = new ObjectOutputStream(byteOut)) {
-      objOut.writeObject(obj);
-    } catch (IOException e) {
-      throw new IllegalStateException(
-          "Unexpected IOException when writing an object to a ByteString.", e);
-    }
-    return byteOut.toByteString();
-  }
-
-  public static Object toObject(ByteString bytes) {
-    try(final ObjectInputStream in = new ObjectInputStream(bytes.newInput())) {
-      return in.readObject();
-    } catch (IOException e) {
-      throw new IllegalStateException(
-          "Unexpected IOException when reading an object from a ByteString.", e);
-    } catch (ClassNotFoundException e) {
-      throw new IllegalStateException(e);
-    }
-  }
-
-  public static ByteString toByteString(byte[] bytes) {
-    return toByteString(bytes, 0, bytes.length);
-  }
-
-  public static ByteString toByteString(byte[] bytes, int offset, int size) {
-    // return singleton to reduce object allocation
-    return bytes.length == 0 ?
-        ByteString.EMPTY : ByteString.copyFrom(bytes, offset, size);
-  }
-
-  public static RaftPeerProto toRaftPeerProto(RaftPeer peer) {
-    RaftPeerProto.Builder builder = RaftPeerProto.newBuilder()
-        .setId(peer.getId());
-    if (peer.getAddress() != null) {
-      builder.setAddress(peer.getAddress());
-    }
-    return builder.build();
-  }
-
-  public static RaftPeer toRaftPeer(RaftPeerProto p) {
-    return new RaftPeer(p.getId(), p.getAddress());
-  }
-
-  public static RaftPeer[] toRaftPeerArray(List<RaftPeerProto> protos) {
-    final RaftPeer[] peers = new RaftPeer[protos.size()];
-    for (int i = 0; i < peers.length; i++) {
-      peers[i] = toRaftPeer(protos.get(i));
-    }
-    return peers;
-  }
-
-  public static Iterable<RaftPeerProto> toRaftPeerProtos(
-      final Collection<RaftPeer> peers) {
-    return () -> new Iterator<RaftPeerProto>() {
-      final Iterator<RaftPeer> i = peers.iterator();
-
-      @Override
-      public boolean hasNext() {
-        return i.hasNext();
-      }
-
-      @Override
-      public RaftPeerProto next() {
-        return toRaftPeerProto(i.next());
-      }
-    };
-  }
-
-  public static boolean isConfigurationLogEntry(LogEntryProto entry) {
-    return entry.getLogEntryBodyCase() ==
-        LogEntryProto.LogEntryBodyCase.CONFIGURATIONENTRY;
-  }
-
-  public static LogEntryProto toLogEntryProto(
-      SMLogEntryProto operation, long term, long index) {
-    return LogEntryProto.newBuilder().setTerm(term).setIndex(index)
-        .setSmLogEntry(operation)
-        .build();
-  }
-
-  public static IOException toIOException(ServiceException se) {
-    final Throwable t = se.getCause();
-    if (t == null) {
-      return new IOException(se);
-    }
-    return t instanceof IOException? (IOException)t : new IOException(se);
-  }
-
-  public static String toString(RaftRpcRequestProto proto) {
-    return proto.getRequestorId() + "->" + proto.getReplyId()
-        + "#" + proto.getSeqNum();
-  }
-
-  public static String toString(RaftRpcReplyProto proto) {
-    return proto.getRequestorId() + "<-" + proto.getReplyId()
-        + "#" + proto.getSeqNum() + ":"
-        + (proto.getSuccess()? "OK": "FAIL");
-  }
-  public static String toString(RequestVoteReplyProto proto) {
-    return toString(proto.getServerReply()) + "-t" + proto.getTerm();
-  }
-  public static String toString(AppendEntriesReplyProto proto) {
-    return toString(proto.getServerReply()) + "-t" + proto.getTerm()
-        + ", nextIndex=" + proto.getNextIndex()
-        + ", result: " + proto.getResult();
-  }
-}

http://git-wip-us.apache.org/repos/asf/incubator-ratis/blob/7e71a2e0/raft-common/src/main/java/org/apache/raft/util/PureJavaCrc32.java
----------------------------------------------------------------------
diff --git a/raft-common/src/main/java/org/apache/raft/util/PureJavaCrc32.java b/raft-common/src/main/java/org/apache/raft/util/PureJavaCrc32.java
deleted file mode 100644
index 3427eb1..0000000
--- a/raft-common/src/main/java/org/apache/raft/util/PureJavaCrc32.java
+++ /dev/null
@@ -1,619 +0,0 @@
-/**
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- *     http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-package org.apache.raft.util;
-
-import java.util.zip.Checksum;
-
-/**
- * A pure-java implementation of the CRC32 checksum that uses
- * the same polynomial as the built-in native CRC32.
- *
- * This is to avoid the JNI overhead for certain uses of Checksumming
- * where many small pieces of data are checksummed in succession.
- *
- * The current version is ~10x to 1.8x as fast as Sun's native
- * java.util.zip.CRC32 in Java 1.6
- *
- * @see java.util.zip.CRC32
- */
-public class PureJavaCrc32 implements Checksum {
-
-  /** the current CRC value, bit-flipped */
-  private int crc;
-
-  /** Create a new PureJavaCrc32 object. */
-  public PureJavaCrc32() {
-    reset();
-  }
-
-  @Override
-  public long getValue() {
-    return (~crc) & 0xffffffffL;
-  }
-
-  @Override
-  public void reset() {
-    crc = 0xffffffff;
-  }
-
-  @Override
-  public void update(final byte[] b, final int offset, final int len) {
-    int localCrc = crc;
-
-    final int remainder = len & 0x7;
-    int i = offset;
-    for(final int end = offset + len - remainder; i < end; i += 8) {
-      final int x = localCrc
-          ^ ((((b[i  ] << 24) >>> 24) + ((b[i+1] << 24) >>> 16))
-           + (((b[i+2] << 24) >>> 8 ) +  (b[i+3] << 24)));
-
-      localCrc = ((T[((x << 24) >>> 24) + 0x700] ^ T[((x << 16) >>> 24) + 0x600])
-                ^ (T[((x <<  8) >>> 24) + 0x500] ^ T[ (x        >>> 24) + 0x400]))
-               ^ ((T[((b[i+4] << 24) >>> 24) + 0x300] ^ T[((b[i+5] << 24) >>> 24) + 0x200])
-                ^ (T[((b[i+6] << 24) >>> 24) + 0x100] ^ T[((b[i+7] << 24) >>> 24)]));
-    }
-
-    /* loop unroll - duff's device style */
-    switch(remainder) {
-      case 7: localCrc = (localCrc >>> 8) ^ T[((localCrc ^ b[i++]) << 24) >>> 24];
-      case 6: localCrc = (localCrc >>> 8) ^ T[((localCrc ^ b[i++]) << 24) >>> 24];
-      case 5: localCrc = (localCrc >>> 8) ^ T[((localCrc ^ b[i++]) << 24) >>> 24];
-      case 4: localCrc = (localCrc >>> 8) ^ T[((localCrc ^ b[i++]) << 24) >>> 24];
-      case 3: localCrc = (localCrc >>> 8) ^ T[((localCrc ^ b[i++]) << 24) >>> 24];
-      case 2: localCrc = (localCrc >>> 8) ^ T[((localCrc ^ b[i++]) << 24) >>> 24];
-      case 1: localCrc = (localCrc >>> 8) ^ T[((localCrc ^ b[i++]) << 24) >>> 24];
-      default:
-        /* nothing */
-    }
-    
-    // Publish crc out to object
-    crc = localCrc;
-  }
-
-  @Override
-  final public void update(int b) {
-    crc = (crc >>> 8) ^ T[(((crc ^ b) << 24) >>> 24)];
-  }
-
-  /*
-   * CRC-32 lookup tables generated by the polynomial 0xEDB88320.
-   * See also TestPureJavaCrc32.Table.
-   */
-  private static final int[] T = new int[] {
-    /* T8_0 */
-    0x00000000, 0x77073096, 0xEE0E612C, 0x990951BA, 
-    0x076DC419, 0x706AF48F, 0xE963A535, 0x9E6495A3, 
-    0x0EDB8832, 0x79DCB8A4, 0xE0D5E91E, 0x97D2D988, 
-    0x09B64C2B, 0x7EB17CBD, 0xE7B82D07, 0x90BF1D91, 
-    0x1DB71064, 0x6AB020F2, 0xF3B97148, 0x84BE41DE, 
-    0x1ADAD47D, 0x6DDDE4EB, 0xF4D4B551, 0x83D385C7, 
-    0x136C9856, 0x646BA8C0, 0xFD62F97A, 0x8A65C9EC, 
-    0x14015C4F, 0x63066CD9, 0xFA0F3D63, 0x8D080DF5, 
-    0x3B6E20C8, 0x4C69105E, 0xD56041E4, 0xA2677172, 
-    0x3C03E4D1, 0x4B04D447, 0xD20D85FD, 0xA50AB56B, 
-    0x35B5A8FA, 0x42B2986C, 0xDBBBC9D6, 0xACBCF940, 
-    0x32D86CE3, 0x45DF5C75, 0xDCD60DCF, 0xABD13D59, 
-    0x26D930AC, 0x51DE003A, 0xC8D75180, 0xBFD06116, 
-    0x21B4F4B5, 0x56B3C423, 0xCFBA9599, 0xB8BDA50F, 
-    0x2802B89E, 0x5F058808, 0xC60CD9B2, 0xB10BE924, 
-    0x2F6F7C87, 0x58684C11, 0xC1611DAB, 0xB6662D3D, 
-    0x76DC4190, 0x01DB7106, 0x98D220BC, 0xEFD5102A, 
-    0x71B18589, 0x06B6B51F, 0x9FBFE4A5, 0xE8B8D433, 
-    0x7807C9A2, 0x0F00F934, 0x9609A88E, 0xE10E9818, 
-    0x7F6A0DBB, 0x086D3D2D, 0x91646C97, 0xE6635C01, 
-    0x6B6B51F4, 0x1C6C6162, 0x856530D8, 0xF262004E, 
-    0x6C0695ED, 0x1B01A57B, 0x8208F4C1, 0xF50FC457, 
-    0x65B0D9C6, 0x12B7E950, 0x8BBEB8EA, 0xFCB9887C, 
-    0x62DD1DDF, 0x15DA2D49, 0x8CD37CF3, 0xFBD44C65, 
-    0x4DB26158, 0x3AB551CE, 0xA3BC0074, 0xD4BB30E2, 
-    0x4ADFA541, 0x3DD895D7, 0xA4D1C46D, 0xD3D6F4FB, 
-    0x4369E96A, 0x346ED9FC, 0xAD678846, 0xDA60B8D0, 
-    0x44042D73, 0x33031DE5, 0xAA0A4C5F, 0xDD0D7CC9, 
-    0x5005713C, 0x270241AA, 0xBE0B1010, 0xC90C2086, 
-    0x5768B525, 0x206F85B3, 0xB966D409, 0xCE61E49F, 
-    0x5EDEF90E, 0x29D9C998, 0xB0D09822, 0xC7D7A8B4, 
-    0x59B33D17, 0x2EB40D81, 0xB7BD5C3B, 0xC0BA6CAD, 
-    0xEDB88320, 0x9ABFB3B6, 0x03B6E20C, 0x74B1D29A, 
-    0xEAD54739, 0x9DD277AF, 0x04DB2615, 0x73DC1683, 
-    0xE3630B12, 0x94643B84, 0x0D6D6A3E, 0x7A6A5AA8, 
-    0xE40ECF0B, 0x9309FF9D, 0x0A00AE27, 0x7D079EB1, 
-    0xF00F9344, 0x8708A3D2, 0x1E01F268, 0x6906C2FE, 
-    0xF762575D, 0x806567CB, 0x196C3671, 0x6E6B06E7, 
-    0xFED41B76, 0x89D32BE0, 0x10DA7A5A, 0x67DD4ACC, 
-    0xF9B9DF6F, 0x8EBEEFF9, 0x17B7BE43, 0x60B08ED5, 
-    0xD6D6A3E8, 0xA1D1937E, 0x38D8C2C4, 0x4FDFF252, 
-    0xD1BB67F1, 0xA6BC5767, 0x3FB506DD, 0x48B2364B, 
-    0xD80D2BDA, 0xAF0A1B4C, 0x36034AF6, 0x41047A60, 
-    0xDF60EFC3, 0xA867DF55, 0x316E8EEF, 0x4669BE79, 
-    0xCB61B38C, 0xBC66831A, 0x256FD2A0, 0x5268E236, 
-    0xCC0C7795, 0xBB0B4703, 0x220216B9, 0x5505262F, 
-    0xC5BA3BBE, 0xB2BD0B28, 0x2BB45A92, 0x5CB36A04, 
-    0xC2D7FFA7, 0xB5D0CF31, 0x2CD99E8B, 0x5BDEAE1D, 
-    0x9B64C2B0, 0xEC63F226, 0x756AA39C, 0x026D930A, 
-    0x9C0906A9, 0xEB0E363F, 0x72076785, 0x05005713, 
-    0x95BF4A82, 0xE2B87A14, 0x7BB12BAE, 0x0CB61B38, 
-    0x92D28E9B, 0xE5D5BE0D, 0x7CDCEFB7, 0x0BDBDF21, 
-    0x86D3D2D4, 0xF1D4E242, 0x68DDB3F8, 0x1FDA836E, 
-    0x81BE16CD, 0xF6B9265B, 0x6FB077E1, 0x18B74777, 
-    0x88085AE6, 0xFF0F6A70, 0x66063BCA, 0x11010B5C, 
-    0x8F659EFF, 0xF862AE69, 0x616BFFD3, 0x166CCF45, 
-    0xA00AE278, 0xD70DD2EE, 0x4E048354, 0x3903B3C2, 
-    0xA7672661, 0xD06016F7, 0x4969474D, 0x3E6E77DB, 
-    0xAED16A4A, 0xD9D65ADC, 0x40DF0B66, 0x37D83BF0, 
-    0xA9BCAE53, 0xDEBB9EC5, 0x47B2CF7F, 0x30B5FFE9, 
-    0xBDBDF21C, 0xCABAC28A, 0x53B39330, 0x24B4A3A6, 
-    0xBAD03605, 0xCDD70693, 0x54DE5729, 0x23D967BF, 
-    0xB3667A2E, 0xC4614AB8, 0x5D681B02, 0x2A6F2B94, 
-    0xB40BBE37, 0xC30C8EA1, 0x5A05DF1B, 0x2D02EF8D,
-    /* T8_1 */
-    0x00000000, 0x191B3141, 0x32366282, 0x2B2D53C3, 
-    0x646CC504, 0x7D77F445, 0x565AA786, 0x4F4196C7, 
-    0xC8D98A08, 0xD1C2BB49, 0xFAEFE88A, 0xE3F4D9CB, 
-    0xACB54F0C, 0xB5AE7E4D, 0x9E832D8E, 0x87981CCF, 
-    0x4AC21251, 0x53D92310, 0x78F470D3, 0x61EF4192, 
-    0x2EAED755, 0x37B5E614, 0x1C98B5D7, 0x05838496, 
-    0x821B9859, 0x9B00A918, 0xB02DFADB, 0xA936CB9A, 
-    0xE6775D5D, 0xFF6C6C1C, 0xD4413FDF, 0xCD5A0E9E, 
-    0x958424A2, 0x8C9F15E3, 0xA7B24620, 0xBEA97761, 
-    0xF1E8E1A6, 0xE8F3D0E7, 0xC3DE8324, 0xDAC5B265, 
-    0x5D5DAEAA, 0x44469FEB, 0x6F6BCC28, 0x7670FD69, 
-    0x39316BAE, 0x202A5AEF, 0x0B07092C, 0x121C386D, 
-    0xDF4636F3, 0xC65D07B2, 0xED705471, 0xF46B6530, 
-    0xBB2AF3F7, 0xA231C2B6, 0x891C9175, 0x9007A034, 
-    0x179FBCFB, 0x0E848DBA, 0x25A9DE79, 0x3CB2EF38, 
-    0x73F379FF, 0x6AE848BE, 0x41C51B7D, 0x58DE2A3C, 
-    0xF0794F05, 0xE9627E44, 0xC24F2D87, 0xDB541CC6, 
-    0x94158A01, 0x8D0EBB40, 0xA623E883, 0xBF38D9C2, 
-    0x38A0C50D, 0x21BBF44C, 0x0A96A78F, 0x138D96CE, 
-    0x5CCC0009, 0x45D73148, 0x6EFA628B, 0x77E153CA, 
-    0xBABB5D54, 0xA3A06C15, 0x888D3FD6, 0x91960E97, 
-    0xDED79850, 0xC7CCA911, 0xECE1FAD2, 0xF5FACB93, 
-    0x7262D75C, 0x6B79E61D, 0x4054B5DE, 0x594F849F, 
-    0x160E1258, 0x0F152319, 0x243870DA, 0x3D23419B, 
-    0x65FD6BA7, 0x7CE65AE6, 0x57CB0925, 0x4ED03864, 
-    0x0191AEA3, 0x188A9FE2, 0x33A7CC21, 0x2ABCFD60, 
-    0xAD24E1AF, 0xB43FD0EE, 0x9F12832D, 0x8609B26C, 
-    0xC94824AB, 0xD05315EA, 0xFB7E4629, 0xE2657768, 
-    0x2F3F79F6, 0x362448B7, 0x1D091B74, 0x04122A35, 
-    0x4B53BCF2, 0x52488DB3, 0x7965DE70, 0x607EEF31, 
-    0xE7E6F3FE, 0xFEFDC2BF, 0xD5D0917C, 0xCCCBA03D, 
-    0x838A36FA, 0x9A9107BB, 0xB1BC5478, 0xA8A76539, 
-    0x3B83984B, 0x2298A90A, 0x09B5FAC9, 0x10AECB88, 
-    0x5FEF5D4F, 0x46F46C0E, 0x6DD93FCD, 0x74C20E8C, 
-    0xF35A1243, 0xEA412302, 0xC16C70C1, 0xD8774180, 
-    0x9736D747, 0x8E2DE606, 0xA500B5C5, 0xBC1B8484, 
-    0x71418A1A, 0x685ABB5B, 0x4377E898, 0x5A6CD9D9, 
-    0x152D4F1E, 0x0C367E5F, 0x271B2D9C, 0x3E001CDD, 
-    0xB9980012, 0xA0833153, 0x8BAE6290, 0x92B553D1, 
-    0xDDF4C516, 0xC4EFF457, 0xEFC2A794, 0xF6D996D5, 
-    0xAE07BCE9, 0xB71C8DA8, 0x9C31DE6B, 0x852AEF2A, 
-    0xCA6B79ED, 0xD37048AC, 0xF85D1B6F, 0xE1462A2E, 
-    0x66DE36E1, 0x7FC507A0, 0x54E85463, 0x4DF36522, 
-    0x02B2F3E5, 0x1BA9C2A4, 0x30849167, 0x299FA026, 
-    0xE4C5AEB8, 0xFDDE9FF9, 0xD6F3CC3A, 0xCFE8FD7B, 
-    0x80A96BBC, 0x99B25AFD, 0xB29F093E, 0xAB84387F, 
-    0x2C1C24B0, 0x350715F1, 0x1E2A4632, 0x07317773, 
-    0x4870E1B4, 0x516BD0F5, 0x7A468336, 0x635DB277, 
-    0xCBFAD74E, 0xD2E1E60F, 0xF9CCB5CC, 0xE0D7848D, 
-    0xAF96124A, 0xB68D230B, 0x9DA070C8, 0x84BB4189, 
-    0x03235D46, 0x1A386C07, 0x31153FC4, 0x280E0E85, 
-    0x674F9842, 0x7E54A903, 0x5579FAC0, 0x4C62CB81, 
-    0x8138C51F, 0x9823F45E, 0xB30EA79D, 0xAA1596DC, 
-    0xE554001B, 0xFC4F315A, 0xD7626299, 0xCE7953D8, 
-    0x49E14F17, 0x50FA7E56, 0x7BD72D95, 0x62CC1CD4, 
-    0x2D8D8A13, 0x3496BB52, 0x1FBBE891, 0x06A0D9D0, 
-    0x5E7EF3EC, 0x4765C2AD, 0x6C48916E, 0x7553A02F, 
-    0x3A1236E8, 0x230907A9, 0x0824546A, 0x113F652B, 
-    0x96A779E4, 0x8FBC48A5, 0xA4911B66, 0xBD8A2A27, 
-    0xF2CBBCE0, 0xEBD08DA1, 0xC0FDDE62, 0xD9E6EF23, 
-    0x14BCE1BD, 0x0DA7D0FC, 0x268A833F, 0x3F91B27E, 
-    0x70D024B9, 0x69CB15F8, 0x42E6463B, 0x5BFD777A, 
-    0xDC656BB5, 0xC57E5AF4, 0xEE530937, 0xF7483876, 
-    0xB809AEB1, 0xA1129FF0, 0x8A3FCC33, 0x9324FD72,
-    /* T8_2 */
-    0x00000000, 0x01C26A37, 0x0384D46E, 0x0246BE59, 
-    0x0709A8DC, 0x06CBC2EB, 0x048D7CB2, 0x054F1685, 
-    0x0E1351B8, 0x0FD13B8F, 0x0D9785D6, 0x0C55EFE1, 
-    0x091AF964, 0x08D89353, 0x0A9E2D0A, 0x0B5C473D, 
-    0x1C26A370, 0x1DE4C947, 0x1FA2771E, 0x1E601D29, 
-    0x1B2F0BAC, 0x1AED619B, 0x18ABDFC2, 0x1969B5F5, 
-    0x1235F2C8, 0x13F798FF, 0x11B126A6, 0x10734C91, 
-    0x153C5A14, 0x14FE3023, 0x16B88E7A, 0x177AE44D, 
-    0x384D46E0, 0x398F2CD7, 0x3BC9928E, 0x3A0BF8B9, 
-    0x3F44EE3C, 0x3E86840B, 0x3CC03A52, 0x3D025065, 
-    0x365E1758, 0x379C7D6F, 0x35DAC336, 0x3418A901, 
-    0x3157BF84, 0x3095D5B3, 0x32D36BEA, 0x331101DD, 
-    0x246BE590, 0x25A98FA7, 0x27EF31FE, 0x262D5BC9, 
-    0x23624D4C, 0x22A0277B, 0x20E69922, 0x2124F315, 
-    0x2A78B428, 0x2BBADE1F, 0x29FC6046, 0x283E0A71, 
-    0x2D711CF4, 0x2CB376C3, 0x2EF5C89A, 0x2F37A2AD, 
-    0x709A8DC0, 0x7158E7F7, 0x731E59AE, 0x72DC3399, 
-    0x7793251C, 0x76514F2B, 0x7417F172, 0x75D59B45, 
-    0x7E89DC78, 0x7F4BB64F, 0x7D0D0816, 0x7CCF6221, 
-    0x798074A4, 0x78421E93, 0x7A04A0CA, 0x7BC6CAFD, 
-    0x6CBC2EB0, 0x6D7E4487, 0x6F38FADE, 0x6EFA90E9, 
-    0x6BB5866C, 0x6A77EC5B, 0x68315202, 0x69F33835, 
-    0x62AF7F08, 0x636D153F, 0x612BAB66, 0x60E9C151, 
-    0x65A6D7D4, 0x6464BDE3, 0x662203BA, 0x67E0698D, 
-    0x48D7CB20, 0x4915A117, 0x4B531F4E, 0x4A917579, 
-    0x4FDE63FC, 0x4E1C09CB, 0x4C5AB792, 0x4D98DDA5, 
-    0x46C49A98, 0x4706F0AF, 0x45404EF6, 0x448224C1, 
-    0x41CD3244, 0x400F5873, 0x4249E62A, 0x438B8C1D, 
-    0x54F16850, 0x55330267, 0x5775BC3E, 0x56B7D609, 
-    0x53F8C08C, 0x523AAABB, 0x507C14E2, 0x51BE7ED5, 
-    0x5AE239E8, 0x5B2053DF, 0x5966ED86, 0x58A487B1, 
-    0x5DEB9134, 0x5C29FB03, 0x5E6F455A, 0x5FAD2F6D, 
-    0xE1351B80, 0xE0F771B7, 0xE2B1CFEE, 0xE373A5D9, 
-    0xE63CB35C, 0xE7FED96B, 0xE5B86732, 0xE47A0D05, 
-    0xEF264A38, 0xEEE4200F, 0xECA29E56, 0xED60F461, 
-    0xE82FE2E4, 0xE9ED88D3, 0xEBAB368A, 0xEA695CBD, 
-    0xFD13B8F0, 0xFCD1D2C7, 0xFE976C9E, 0xFF5506A9, 
-    0xFA1A102C, 0xFBD87A1B, 0xF99EC442, 0xF85CAE75, 
-    0xF300E948, 0xF2C2837F, 0xF0843D26, 0xF1465711, 
-    0xF4094194, 0xF5CB2BA3, 0xF78D95FA, 0xF64FFFCD, 
-    0xD9785D60, 0xD8BA3757, 0xDAFC890E, 0xDB3EE339, 
-    0xDE71F5BC, 0xDFB39F8B, 0xDDF521D2, 0xDC374BE5, 
-    0xD76B0CD8, 0xD6A966EF, 0xD4EFD8B6, 0xD52DB281, 
-    0xD062A404, 0xD1A0CE33, 0xD3E6706A, 0xD2241A5D, 
-    0xC55EFE10, 0xC49C9427, 0xC6DA2A7E, 0xC7184049, 
-    0xC25756CC, 0xC3953CFB, 0xC1D382A2, 0xC011E895, 
-    0xCB4DAFA8, 0xCA8FC59F, 0xC8C97BC6, 0xC90B11F1, 
-    0xCC440774, 0xCD866D43, 0xCFC0D31A, 0xCE02B92D, 
-    0x91AF9640, 0x906DFC77, 0x922B422E, 0x93E92819, 
-    0x96A63E9C, 0x976454AB, 0x9522EAF2, 0x94E080C5, 
-    0x9FBCC7F8, 0x9E7EADCF, 0x9C381396, 0x9DFA79A1, 
-    0x98B56F24, 0x99770513, 0x9B31BB4A, 0x9AF3D17D, 
-    0x8D893530, 0x8C4B5F07, 0x8E0DE15E, 0x8FCF8B69, 
-    0x8A809DEC, 0x8B42F7DB, 0x89044982, 0x88C623B5, 
-    0x839A6488, 0x82580EBF, 0x801EB0E6, 0x81DCDAD1, 
-    0x8493CC54, 0x8551A663, 0x8717183A, 0x86D5720D, 
-    0xA9E2D0A0, 0xA820BA97, 0xAA6604CE, 0xABA46EF9, 
-    0xAEEB787C, 0xAF29124B, 0xAD6FAC12, 0xACADC625, 
-    0xA7F18118, 0xA633EB2F, 0xA4755576, 0xA5B73F41, 
-    0xA0F829C4, 0xA13A43F3, 0xA37CFDAA, 0xA2BE979D, 
-    0xB5C473D0, 0xB40619E7, 0xB640A7BE, 0xB782CD89, 
-    0xB2CDDB0C, 0xB30FB13B, 0xB1490F62, 0xB08B6555, 
-    0xBBD72268, 0xBA15485F, 0xB853F606, 0xB9919C31, 
-    0xBCDE8AB4, 0xBD1CE083, 0xBF5A5EDA, 0xBE9834ED,
-    /* T8_3 */
-    0x00000000, 0xB8BC6765, 0xAA09C88B, 0x12B5AFEE, 
-    0x8F629757, 0x37DEF032, 0x256B5FDC, 0x9DD738B9, 
-    0xC5B428EF, 0x7D084F8A, 0x6FBDE064, 0xD7018701, 
-    0x4AD6BFB8, 0xF26AD8DD, 0xE0DF7733, 0x58631056, 
-    0x5019579F, 0xE8A530FA, 0xFA109F14, 0x42ACF871, 
-    0xDF7BC0C8, 0x67C7A7AD, 0x75720843, 0xCDCE6F26, 
-    0x95AD7F70, 0x2D111815, 0x3FA4B7FB, 0x8718D09E, 
-    0x1ACFE827, 0xA2738F42, 0xB0C620AC, 0x087A47C9, 
-    0xA032AF3E, 0x188EC85B, 0x0A3B67B5, 0xB28700D0, 
-    0x2F503869, 0x97EC5F0C, 0x8559F0E2, 0x3DE59787, 
-    0x658687D1, 0xDD3AE0B4, 0xCF8F4F5A, 0x7733283F, 
-    0xEAE41086, 0x525877E3, 0x40EDD80D, 0xF851BF68, 
-    0xF02BF8A1, 0x48979FC4, 0x5A22302A, 0xE29E574F, 
-    0x7F496FF6, 0xC7F50893, 0xD540A77D, 0x6DFCC018, 
-    0x359FD04E, 0x8D23B72B, 0x9F9618C5, 0x272A7FA0, 
-    0xBAFD4719, 0x0241207C, 0x10F48F92, 0xA848E8F7, 
-    0x9B14583D, 0x23A83F58, 0x311D90B6, 0x89A1F7D3, 
-    0x1476CF6A, 0xACCAA80F, 0xBE7F07E1, 0x06C36084, 
-    0x5EA070D2, 0xE61C17B7, 0xF4A9B859, 0x4C15DF3C, 
-    0xD1C2E785, 0x697E80E0, 0x7BCB2F0E, 0xC377486B, 
-    0xCB0D0FA2, 0x73B168C7, 0x6104C729, 0xD9B8A04C, 
-    0x446F98F5, 0xFCD3FF90, 0xEE66507E, 0x56DA371B, 
-    0x0EB9274D, 0xB6054028, 0xA4B0EFC6, 0x1C0C88A3, 
-    0x81DBB01A, 0x3967D77F, 0x2BD27891, 0x936E1FF4, 
-    0x3B26F703, 0x839A9066, 0x912F3F88, 0x299358ED, 
-    0xB4446054, 0x0CF80731, 0x1E4DA8DF, 0xA6F1CFBA, 
-    0xFE92DFEC, 0x462EB889, 0x549B1767, 0xEC277002, 
-    0x71F048BB, 0xC94C2FDE, 0xDBF98030, 0x6345E755, 
-    0x6B3FA09C, 0xD383C7F9, 0xC1366817, 0x798A0F72, 
-    0xE45D37CB, 0x5CE150AE, 0x4E54FF40, 0xF6E89825, 
-    0xAE8B8873, 0x1637EF16, 0x048240F8, 0xBC3E279D, 
-    0x21E91F24, 0x99557841, 0x8BE0D7AF, 0x335CB0CA, 
-    0xED59B63B, 0x55E5D15E, 0x47507EB0, 0xFFEC19D5, 
-    0x623B216C, 0xDA874609, 0xC832E9E7, 0x708E8E82, 
-    0x28ED9ED4, 0x9051F9B1, 0x82E4565F, 0x3A58313A, 
-    0xA78F0983, 0x1F336EE6, 0x0D86C108, 0xB53AA66D, 
-    0xBD40E1A4, 0x05FC86C1, 0x1749292F, 0xAFF54E4A, 
-    0x322276F3, 0x8A9E1196, 0x982BBE78, 0x2097D91D, 
-    0x78F4C94B, 0xC048AE2E, 0xD2FD01C0, 0x6A4166A5, 
-    0xF7965E1C, 0x4F2A3979, 0x5D9F9697, 0xE523F1F2, 
-    0x4D6B1905, 0xF5D77E60, 0xE762D18E, 0x5FDEB6EB, 
-    0xC2098E52, 0x7AB5E937, 0x680046D9, 0xD0BC21BC, 
-    0x88DF31EA, 0x3063568F, 0x22D6F961, 0x9A6A9E04, 
-    0x07BDA6BD, 0xBF01C1D8, 0xADB46E36, 0x15080953, 
-    0x1D724E9A, 0xA5CE29FF, 0xB77B8611, 0x0FC7E174, 
-    0x9210D9CD, 0x2AACBEA8, 0x38191146, 0x80A57623, 
-    0xD8C66675, 0x607A0110, 0x72CFAEFE, 0xCA73C99B, 
-    0x57A4F122, 0xEF189647, 0xFDAD39A9, 0x45115ECC, 
-    0x764DEE06, 0xCEF18963, 0xDC44268D, 0x64F841E8, 
-    0xF92F7951, 0x41931E34, 0x5326B1DA, 0xEB9AD6BF, 
-    0xB3F9C6E9, 0x0B45A18C, 0x19F00E62, 0xA14C6907, 
-    0x3C9B51BE, 0x842736DB, 0x96929935, 0x2E2EFE50, 
-    0x2654B999, 0x9EE8DEFC, 0x8C5D7112, 0x34E11677, 
-    0xA9362ECE, 0x118A49AB, 0x033FE645, 0xBB838120, 
-    0xE3E09176, 0x5B5CF613, 0x49E959FD, 0xF1553E98, 
-    0x6C820621, 0xD43E6144, 0xC68BCEAA, 0x7E37A9CF, 
-    0xD67F4138, 0x6EC3265D, 0x7C7689B3, 0xC4CAEED6, 
-    0x591DD66F, 0xE1A1B10A, 0xF3141EE4, 0x4BA87981, 
-    0x13CB69D7, 0xAB770EB2, 0xB9C2A15C, 0x017EC639, 
-    0x9CA9FE80, 0x241599E5, 0x36A0360B, 0x8E1C516E, 
-    0x866616A7, 0x3EDA71C2, 0x2C6FDE2C, 0x94D3B949, 
-    0x090481F0, 0xB1B8E695, 0xA30D497B, 0x1BB12E1E, 
-    0x43D23E48, 0xFB6E592D, 0xE9DBF6C3, 0x516791A6, 
-    0xCCB0A91F, 0x740CCE7A, 0x66B96194, 0xDE0506F1,
-    /* T8_4 */
-    0x00000000, 0x3D6029B0, 0x7AC05360, 0x47A07AD0, 
-    0xF580A6C0, 0xC8E08F70, 0x8F40F5A0, 0xB220DC10, 
-    0x30704BC1, 0x0D106271, 0x4AB018A1, 0x77D03111, 
-    0xC5F0ED01, 0xF890C4B1, 0xBF30BE61, 0x825097D1, 
-    0x60E09782, 0x5D80BE32, 0x1A20C4E2, 0x2740ED52, 
-    0x95603142, 0xA80018F2, 0xEFA06222, 0xD2C04B92, 
-    0x5090DC43, 0x6DF0F5F3, 0x2A508F23, 0x1730A693, 
-    0xA5107A83, 0x98705333, 0xDFD029E3, 0xE2B00053, 
-    0xC1C12F04, 0xFCA106B4, 0xBB017C64, 0x866155D4, 
-    0x344189C4, 0x0921A074, 0x4E81DAA4, 0x73E1F314, 
-    0xF1B164C5, 0xCCD14D75, 0x8B7137A5, 0xB6111E15, 
-    0x0431C205, 0x3951EBB5, 0x7EF19165, 0x4391B8D5, 
-    0xA121B886, 0x9C419136, 0xDBE1EBE6, 0xE681C256, 
-    0x54A11E46, 0x69C137F6, 0x2E614D26, 0x13016496, 
-    0x9151F347, 0xAC31DAF7, 0xEB91A027, 0xD6F18997, 
-    0x64D15587, 0x59B17C37, 0x1E1106E7, 0x23712F57, 
-    0x58F35849, 0x659371F9, 0x22330B29, 0x1F532299, 
-    0xAD73FE89, 0x9013D739, 0xD7B3ADE9, 0xEAD38459, 
-    0x68831388, 0x55E33A38, 0x124340E8, 0x2F236958, 
-    0x9D03B548, 0xA0639CF8, 0xE7C3E628, 0xDAA3CF98, 
-    0x3813CFCB, 0x0573E67B, 0x42D39CAB, 0x7FB3B51B, 
-    0xCD93690B, 0xF0F340BB, 0xB7533A6B, 0x8A3313DB, 
-    0x0863840A, 0x3503ADBA, 0x72A3D76A, 0x4FC3FEDA, 
-    0xFDE322CA, 0xC0830B7A, 0x872371AA, 0xBA43581A, 
-    0x9932774D, 0xA4525EFD, 0xE3F2242D, 0xDE920D9D, 
-    0x6CB2D18D, 0x51D2F83D, 0x167282ED, 0x2B12AB5D, 
-    0xA9423C8C, 0x9422153C, 0xD3826FEC, 0xEEE2465C, 
-    0x5CC29A4C, 0x61A2B3FC, 0x2602C92C, 0x1B62E09C, 
-    0xF9D2E0CF, 0xC4B2C97F, 0x8312B3AF, 0xBE729A1F, 
-    0x0C52460F, 0x31326FBF, 0x7692156F, 0x4BF23CDF, 
-    0xC9A2AB0E, 0xF4C282BE, 0xB362F86E, 0x8E02D1DE, 
-    0x3C220DCE, 0x0142247E, 0x46E25EAE, 0x7B82771E, 
-    0xB1E6B092, 0x8C869922, 0xCB26E3F2, 0xF646CA42, 
-    0x44661652, 0x79063FE2, 0x3EA64532, 0x03C66C82, 
-    0x8196FB53, 0xBCF6D2E3, 0xFB56A833, 0xC6368183, 
-    0x74165D93, 0x49767423, 0x0ED60EF3, 0x33B62743, 
-    0xD1062710, 0xEC660EA0, 0xABC67470, 0x96A65DC0, 
-    0x248681D0, 0x19E6A860, 0x5E46D2B0, 0x6326FB00, 
-    0xE1766CD1, 0xDC164561, 0x9BB63FB1, 0xA6D61601, 
-    0x14F6CA11, 0x2996E3A1, 0x6E369971, 0x5356B0C1, 
-    0x70279F96, 0x4D47B626, 0x0AE7CCF6, 0x3787E546, 
-    0x85A73956, 0xB8C710E6, 0xFF676A36, 0xC2074386, 
-    0x4057D457, 0x7D37FDE7, 0x3A978737, 0x07F7AE87, 
-    0xB5D77297, 0x88B75B27, 0xCF1721F7, 0xF2770847, 
-    0x10C70814, 0x2DA721A4, 0x6A075B74, 0x576772C4, 
-    0xE547AED4, 0xD8278764, 0x9F87FDB4, 0xA2E7D404, 
-    0x20B743D5, 0x1DD76A65, 0x5A7710B5, 0x67173905, 
-    0xD537E515, 0xE857CCA5, 0xAFF7B675, 0x92979FC5, 
-    0xE915E8DB, 0xD475C16B, 0x93D5BBBB, 0xAEB5920B, 
-    0x1C954E1B, 0x21F567AB, 0x66551D7B, 0x5B3534CB, 
-    0xD965A31A, 0xE4058AAA, 0xA3A5F07A, 0x9EC5D9CA, 
-    0x2CE505DA, 0x11852C6A, 0x562556BA, 0x6B457F0A, 
-    0x89F57F59, 0xB49556E9, 0xF3352C39, 0xCE550589, 
-    0x7C75D999, 0x4115F029, 0x06B58AF9, 0x3BD5A349, 
-    0xB9853498, 0x84E51D28, 0xC34567F8, 0xFE254E48, 
-    0x4C059258, 0x7165BBE8, 0x36C5C138, 0x0BA5E888, 
-    0x28D4C7DF, 0x15B4EE6F, 0x521494BF, 0x6F74BD0F, 
-    0xDD54611F, 0xE03448AF, 0xA794327F, 0x9AF41BCF, 
-    0x18A48C1E, 0x25C4A5AE, 0x6264DF7E, 0x5F04F6CE, 
-    0xED242ADE, 0xD044036E, 0x97E479BE, 0xAA84500E, 
-    0x4834505D, 0x755479ED, 0x32F4033D, 0x0F942A8D, 
-    0xBDB4F69D, 0x80D4DF2D, 0xC774A5FD, 0xFA148C4D, 
-    0x78441B9C, 0x4524322C, 0x028448FC, 0x3FE4614C, 
-    0x8DC4BD5C, 0xB0A494EC, 0xF704EE3C, 0xCA64C78C,
-    /* T8_5 */
-    0x00000000, 0xCB5CD3A5, 0x4DC8A10B, 0x869472AE, 
-    0x9B914216, 0x50CD91B3, 0xD659E31D, 0x1D0530B8, 
-    0xEC53826D, 0x270F51C8, 0xA19B2366, 0x6AC7F0C3, 
-    0x77C2C07B, 0xBC9E13DE, 0x3A0A6170, 0xF156B2D5, 
-    0x03D6029B, 0xC88AD13E, 0x4E1EA390, 0x85427035, 
-    0x9847408D, 0x531B9328, 0xD58FE186, 0x1ED33223, 
-    0xEF8580F6, 0x24D95353, 0xA24D21FD, 0x6911F258, 
-    0x7414C2E0, 0xBF481145, 0x39DC63EB, 0xF280B04E, 
-    0x07AC0536, 0xCCF0D693, 0x4A64A43D, 0x81387798, 
-    0x9C3D4720, 0x57619485, 0xD1F5E62B, 0x1AA9358E, 
-    0xEBFF875B, 0x20A354FE, 0xA6372650, 0x6D6BF5F5, 
-    0x706EC54D, 0xBB3216E8, 0x3DA66446, 0xF6FAB7E3, 
-    0x047A07AD, 0xCF26D408, 0x49B2A6A6, 0x82EE7503, 
-    0x9FEB45BB, 0x54B7961E, 0xD223E4B0, 0x197F3715, 
-    0xE82985C0, 0x23755665, 0xA5E124CB, 0x6EBDF76E, 
-    0x73B8C7D6, 0xB8E41473, 0x3E7066DD, 0xF52CB578, 
-    0x0F580A6C, 0xC404D9C9, 0x4290AB67, 0x89CC78C2, 
-    0x94C9487A, 0x5F959BDF, 0xD901E971, 0x125D3AD4, 
-    0xE30B8801, 0x28575BA4, 0xAEC3290A, 0x659FFAAF, 
-    0x789ACA17, 0xB3C619B2, 0x35526B1C, 0xFE0EB8B9, 
-    0x0C8E08F7, 0xC7D2DB52, 0x4146A9FC, 0x8A1A7A59, 
-    0x971F4AE1, 0x5C439944, 0xDAD7EBEA, 0x118B384F, 
-    0xE0DD8A9A, 0x2B81593F, 0xAD152B91, 0x6649F834, 
-    0x7B4CC88C, 0xB0101B29, 0x36846987, 0xFDD8BA22, 
-    0x08F40F5A, 0xC3A8DCFF, 0x453CAE51, 0x8E607DF4, 
-    0x93654D4C, 0x58399EE9, 0xDEADEC47, 0x15F13FE2, 
-    0xE4A78D37, 0x2FFB5E92, 0xA96F2C3C, 0x6233FF99, 
-    0x7F36CF21, 0xB46A1C84, 0x32FE6E2A, 0xF9A2BD8F, 
-    0x0B220DC1, 0xC07EDE64, 0x46EAACCA, 0x8DB67F6F, 
-    0x90B34FD7, 0x5BEF9C72, 0xDD7BEEDC, 0x16273D79, 
-    0xE7718FAC, 0x2C2D5C09, 0xAAB92EA7, 0x61E5FD02, 
-    0x7CE0CDBA, 0xB7BC1E1F, 0x31286CB1, 0xFA74BF14, 
-    0x1EB014D8, 0xD5ECC77D, 0x5378B5D3, 0x98246676, 
-    0x852156CE, 0x4E7D856B, 0xC8E9F7C5, 0x03B52460, 
-    0xF2E396B5, 0x39BF4510, 0xBF2B37BE, 0x7477E41B, 
-    0x6972D4A3, 0xA22E0706, 0x24BA75A8, 0xEFE6A60D, 
-    0x1D661643, 0xD63AC5E6, 0x50AEB748, 0x9BF264ED, 
-    0x86F75455, 0x4DAB87F0, 0xCB3FF55E, 0x006326FB, 
-    0xF135942E, 0x3A69478B, 0xBCFD3525, 0x77A1E680, 
-    0x6AA4D638, 0xA1F8059D, 0x276C7733, 0xEC30A496, 
-    0x191C11EE, 0xD240C24B, 0x54D4B0E5, 0x9F886340, 
-    0x828D53F8, 0x49D1805D, 0xCF45F2F3, 0x04192156, 
-    0xF54F9383, 0x3E134026, 0xB8873288, 0x73DBE12D, 
-    0x6EDED195, 0xA5820230, 0x2316709E, 0xE84AA33B, 
-    0x1ACA1375, 0xD196C0D0, 0x5702B27E, 0x9C5E61DB, 
-    0x815B5163, 0x4A0782C6, 0xCC93F068, 0x07CF23CD, 
-    0xF6999118, 0x3DC542BD, 0xBB513013, 0x700DE3B6, 
-    0x6D08D30E, 0xA65400AB, 0x20C07205, 0xEB9CA1A0, 
-    0x11E81EB4, 0xDAB4CD11, 0x5C20BFBF, 0x977C6C1A, 
-    0x8A795CA2, 0x41258F07, 0xC7B1FDA9, 0x0CED2E0C, 
-    0xFDBB9CD9, 0x36E74F7C, 0xB0733DD2, 0x7B2FEE77, 
-    0x662ADECF, 0xAD760D6A, 0x2BE27FC4, 0xE0BEAC61, 
-    0x123E1C2F, 0xD962CF8A, 0x5FF6BD24, 0x94AA6E81, 
-    0x89AF5E39, 0x42F38D9C, 0xC467FF32, 0x0F3B2C97, 
-    0xFE6D9E42, 0x35314DE7, 0xB3A53F49, 0x78F9ECEC, 
-    0x65FCDC54, 0xAEA00FF1, 0x28347D5F, 0xE368AEFA, 
-    0x16441B82, 0xDD18C827, 0x5B8CBA89, 0x90D0692C, 
-    0x8DD55994, 0x46898A31, 0xC01DF89F, 0x0B412B3A, 
-    0xFA1799EF, 0x314B4A4A, 0xB7DF38E4, 0x7C83EB41, 
-    0x6186DBF9, 0xAADA085C, 0x2C4E7AF2, 0xE712A957, 
-    0x15921919, 0xDECECABC, 0x585AB812, 0x93066BB7, 
-    0x8E035B0F, 0x455F88AA, 0xC3CBFA04, 0x089729A1, 
-    0xF9C19B74, 0x329D48D1, 0xB4093A7F, 0x7F55E9DA, 
-    0x6250D962, 0xA90C0AC7, 0x2F987869, 0xE4C4ABCC,
-    /* T8_6 */
-    0x00000000, 0xA6770BB4, 0x979F1129, 0x31E81A9D, 
-    0xF44F2413, 0x52382FA7, 0x63D0353A, 0xC5A73E8E, 
-    0x33EF4E67, 0x959845D3, 0xA4705F4E, 0x020754FA, 
-    0xC7A06A74, 0x61D761C0, 0x503F7B5D, 0xF64870E9, 
-    0x67DE9CCE, 0xC1A9977A, 0xF0418DE7, 0x56368653, 
-    0x9391B8DD, 0x35E6B369, 0x040EA9F4, 0xA279A240, 
-    0x5431D2A9, 0xF246D91D, 0xC3AEC380, 0x65D9C834, 
-    0xA07EF6BA, 0x0609FD0E, 0x37E1E793, 0x9196EC27, 
-    0xCFBD399C, 0x69CA3228, 0x582228B5, 0xFE552301, 
-    0x3BF21D8F, 0x9D85163B, 0xAC6D0CA6, 0x0A1A0712, 
-    0xFC5277FB, 0x5A257C4F, 0x6BCD66D2, 0xCDBA6D66, 
-    0x081D53E8, 0xAE6A585C, 0x9F8242C1, 0x39F54975, 
-    0xA863A552, 0x0E14AEE6, 0x3FFCB47B, 0x998BBFCF, 
-    0x5C2C8141, 0xFA5B8AF5, 0xCBB39068, 0x6DC49BDC, 
-    0x9B8CEB35, 0x3DFBE081, 0x0C13FA1C, 0xAA64F1A8, 
-    0x6FC3CF26, 0xC9B4C492, 0xF85CDE0F, 0x5E2BD5BB, 
-    0x440B7579, 0xE27C7ECD, 0xD3946450, 0x75E36FE4, 
-    0xB044516A, 0x16335ADE, 0x27DB4043, 0x81AC4BF7, 
-    0x77E43B1E, 0xD19330AA, 0xE07B2A37, 0x460C2183, 
-    0x83AB1F0D, 0x25DC14B9, 0x14340E24, 0xB2430590, 
-    0x23D5E9B7, 0x85A2E203, 0xB44AF89E, 0x123DF32A, 
-    0xD79ACDA4, 0x71EDC610, 0x4005DC8D, 0xE672D739, 
-    0x103AA7D0, 0xB64DAC64, 0x87A5B6F9, 0x21D2BD4D, 
-    0xE47583C3, 0x42028877, 0x73EA92EA, 0xD59D995E, 
-    0x8BB64CE5, 0x2DC14751, 0x1C295DCC, 0xBA5E5678, 
-    0x7FF968F6, 0xD98E6342, 0xE86679DF, 0x4E11726B, 
-    0xB8590282, 0x1E2E0936, 0x2FC613AB, 0x89B1181F, 
-    0x4C162691, 0xEA612D25, 0xDB8937B8, 0x7DFE3C0C, 
-    0xEC68D02B, 0x4A1FDB9F, 0x7BF7C102, 0xDD80CAB6, 
-    0x1827F438, 0xBE50FF8C, 0x8FB8E511, 0x29CFEEA5, 
-    0xDF879E4C, 0x79F095F8, 0x48188F65, 0xEE6F84D1, 
-    0x2BC8BA5F, 0x8DBFB1EB, 0xBC57AB76, 0x1A20A0C2, 
-    0x8816EAF2, 0x2E61E146, 0x1F89FBDB, 0xB9FEF06F, 
-    0x7C59CEE1, 0xDA2EC555, 0xEBC6DFC8, 0x4DB1D47C, 
-    0xBBF9A495, 0x1D8EAF21, 0x2C66B5BC, 0x8A11BE08, 
-    0x4FB68086, 0xE9C18B32, 0xD82991AF, 0x7E5E9A1B, 
-    0xEFC8763C, 0x49BF7D88, 0x78576715, 0xDE206CA1, 
-    0x1B87522F, 0xBDF0599B, 0x8C184306, 0x2A6F48B2, 
-    0xDC27385B, 0x7A5033EF, 0x4BB82972, 0xEDCF22C6, 
-    0x28681C48, 0x8E1F17FC, 0xBFF70D61, 0x198006D5, 
-    0x47ABD36E, 0xE1DCD8DA, 0xD034C247, 0x7643C9F3, 
-    0xB3E4F77D, 0x1593FCC9, 0x247BE654, 0x820CEDE0, 
-    0x74449D09, 0xD23396BD, 0xE3DB8C20, 0x45AC8794, 
-    0x800BB91A, 0x267CB2AE, 0x1794A833, 0xB1E3A387, 
-    0x20754FA0, 0x86024414, 0xB7EA5E89, 0x119D553D, 
-    0xD43A6BB3, 0x724D6007, 0x43A57A9A, 0xE5D2712E, 
-    0x139A01C7, 0xB5ED0A73, 0x840510EE, 0x22721B5A, 
-    0xE7D525D4, 0x41A22E60, 0x704A34FD, 0xD63D3F49, 
-    0xCC1D9F8B, 0x6A6A943F, 0x5B828EA2, 0xFDF58516, 
-    0x3852BB98, 0x9E25B02C, 0xAFCDAAB1, 0x09BAA105, 
-    0xFFF2D1EC, 0x5985DA58, 0x686DC0C5, 0xCE1ACB71, 
-    0x0BBDF5FF, 0xADCAFE4B, 0x9C22E4D6, 0x3A55EF62, 
-    0xABC30345, 0x0DB408F1, 0x3C5C126C, 0x9A2B19D8, 
-    0x5F8C2756, 0xF9FB2CE2, 0xC813367F, 0x6E643DCB, 
-    0x982C4D22, 0x3E5B4696, 0x0FB35C0B, 0xA9C457BF, 
-    0x6C636931, 0xCA146285, 0xFBFC7818, 0x5D8B73AC, 
-    0x03A0A617, 0xA5D7ADA3, 0x943FB73E, 0x3248BC8A, 
-    0xF7EF8204, 0x519889B0, 0x6070932D, 0xC6079899, 
-    0x304FE870, 0x9638E3C4, 0xA7D0F959, 0x01A7F2ED, 
-    0xC400CC63, 0x6277C7D7, 0x539FDD4A, 0xF5E8D6FE, 
-    0x647E3AD9, 0xC209316D, 0xF3E12BF0, 0x55962044, 
-    0x90311ECA, 0x3646157E, 0x07AE0FE3, 0xA1D90457, 
-    0x579174BE, 0xF1E67F0A, 0xC00E6597, 0x66796E23, 
-    0xA3DE50AD, 0x05A95B19, 0x34414184, 0x92364A30,
-    /* T8_7 */
-    0x00000000, 0xCCAA009E, 0x4225077D, 0x8E8F07E3, 
-    0x844A0EFA, 0x48E00E64, 0xC66F0987, 0x0AC50919, 
-    0xD3E51BB5, 0x1F4F1B2B, 0x91C01CC8, 0x5D6A1C56, 
-    0x57AF154F, 0x9B0515D1, 0x158A1232, 0xD92012AC, 
-    0x7CBB312B, 0xB01131B5, 0x3E9E3656, 0xF23436C8, 
-    0xF8F13FD1, 0x345B3F4F, 0xBAD438AC, 0x767E3832, 
-    0xAF5E2A9E, 0x63F42A00, 0xED7B2DE3, 0x21D12D7D, 
-    0x2B142464, 0xE7BE24FA, 0x69312319, 0xA59B2387, 
-    0xF9766256, 0x35DC62C8, 0xBB53652B, 0x77F965B5, 
-    0x7D3C6CAC, 0xB1966C32, 0x3F196BD1, 0xF3B36B4F, 
-    0x2A9379E3, 0xE639797D, 0x68B67E9E, 0xA41C7E00, 
-    0xAED97719, 0x62737787, 0xECFC7064, 0x205670FA, 
-    0x85CD537D, 0x496753E3, 0xC7E85400, 0x0B42549E, 
-    0x01875D87, 0xCD2D5D19, 0x43A25AFA, 0x8F085A64, 
-    0x562848C8, 0x9A824856, 0x140D4FB5, 0xD8A74F2B, 
-    0xD2624632, 0x1EC846AC, 0x9047414F, 0x5CED41D1, 
-    0x299DC2ED, 0xE537C273, 0x6BB8C590, 0xA712C50E, 
-    0xADD7CC17, 0x617DCC89, 0xEFF2CB6A, 0x2358CBF4, 
-    0xFA78D958, 0x36D2D9C6, 0xB85DDE25, 0x74F7DEBB, 
-    0x7E32D7A2, 0xB298D73C, 0x3C17D0DF, 0xF0BDD041, 
-    0x5526F3C6, 0x998CF358, 0x1703F4BB, 0xDBA9F425, 
-    0xD16CFD3C, 0x1DC6FDA2, 0x9349FA41, 0x5FE3FADF, 
-    0x86C3E873, 0x4A69E8ED, 0xC4E6EF0E, 0x084CEF90, 
-    0x0289E689, 0xCE23E617, 0x40ACE1F4, 0x8C06E16A, 
-    0xD0EBA0BB, 0x1C41A025, 0x92CEA7C6, 0x5E64A758, 
-    0x54A1AE41, 0x980BAEDF, 0x1684A93C, 0xDA2EA9A2, 
-    0x030EBB0E, 0xCFA4BB90, 0x412BBC73, 0x8D81BCED, 
-    0x8744B5F4, 0x4BEEB56A, 0xC561B289, 0x09CBB217, 
-    0xAC509190, 0x60FA910E, 0xEE7596ED, 0x22DF9673, 
-    0x281A9F6A, 0xE4B09FF4, 0x6A3F9817, 0xA6959889, 
-    0x7FB58A25, 0xB31F8ABB, 0x3D908D58, 0xF13A8DC6, 
-    0xFBFF84DF, 0x37558441, 0xB9DA83A2, 0x7570833C, 
-    0x533B85DA, 0x9F918544, 0x111E82A7, 0xDDB48239, 
-    0xD7718B20, 0x1BDB8BBE, 0x95548C5D, 0x59FE8CC3, 
-    0x80DE9E6F, 0x4C749EF1, 0xC2FB9912, 0x0E51998C, 
-    0x04949095, 0xC83E900B, 0x46B197E8, 0x8A1B9776, 
-    0x2F80B4F1, 0xE32AB46F, 0x6DA5B38C, 0xA10FB312, 
-    0xABCABA0B, 0x6760BA95, 0xE9EFBD76, 0x2545BDE8, 
-    0xFC65AF44, 0x30CFAFDA, 0xBE40A839, 0x72EAA8A7, 
-    0x782FA1BE, 0xB485A120, 0x3A0AA6C3, 0xF6A0A65D, 
-    0xAA4DE78C, 0x66E7E712, 0xE868E0F1, 0x24C2E06F, 
-    0x2E07E976, 0xE2ADE9E8, 0x6C22EE0B, 0xA088EE95, 
-    0x79A8FC39, 0xB502FCA7, 0x3B8DFB44, 0xF727FBDA, 
-    0xFDE2F2C3, 0x3148F25D, 0xBFC7F5BE, 0x736DF520, 
-    0xD6F6D6A7, 0x1A5CD639, 0x94D3D1DA, 0x5879D144, 
-    0x52BCD85D, 0x9E16D8C3, 0x1099DF20, 0xDC33DFBE, 
-    0x0513CD12, 0xC9B9CD8C, 0x4736CA6F, 0x8B9CCAF1, 
-    0x8159C3E8, 0x4DF3C376, 0xC37CC495, 0x0FD6C40B, 
-    0x7AA64737, 0xB60C47A9, 0x3883404A, 0xF42940D4, 
-    0xFEEC49CD, 0x32464953, 0xBCC94EB0, 0x70634E2E, 
-    0xA9435C82, 0x65E95C1C, 0xEB665BFF, 0x27CC5B61, 
-    0x2D095278, 0xE1A352E6, 0x6F2C5505, 0xA386559B, 
-    0x061D761C, 0xCAB77682, 0x44387161, 0x889271FF, 
-    0x825778E6, 0x4EFD7878, 0xC0727F9B, 0x0CD87F05, 
-    0xD5F86DA9, 0x19526D37, 0x97DD6AD4, 0x5B776A4A, 
-    0x51B26353, 0x9D1863CD, 0x1397642E, 0xDF3D64B0, 
-    0x83D02561, 0x4F7A25FF, 0xC1F5221C, 0x0D5F2282, 
-    0x079A2B9B, 0xCB302B05, 0x45BF2CE6, 0x89152C78, 
-    0x50353ED4, 0x9C9F3E4A, 0x121039A9, 0xDEBA3937, 
-    0xD47F302E, 0x18D530B0, 0x965A3753, 0x5AF037CD, 
-    0xFF6B144A, 0x33C114D4, 0xBD4E1337, 0x71E413A9, 
-    0x7B211AB0, 0xB78B1A2E, 0x39041DCD, 0xF5AE1D53, 
-    0x2C8E0FFF, 0xE0240F61, 0x6EAB0882, 0xA201081C, 
-    0xA8C40105, 0x646E019B, 0xEAE10678, 0x264B06E6
-  };
-}


[44/54] [abbrv] incubator-ratis git commit: Renamed the packages from raft to ratis in preperation for Apache Incubation - Moved all java packages from org.apache.raft to org.apache.ratis. - Moved native package to org_apache_ratis, and native lib to l

Posted by ji...@apache.org.
http://git-wip-us.apache.org/repos/asf/incubator-ratis/blob/7e71a2e0/raft-common/src/main/native/src/org/apache/raft/io/nativeio/errno_enum.c
----------------------------------------------------------------------
diff --git a/raft-common/src/main/native/src/org/apache/raft/io/nativeio/errno_enum.c b/raft-common/src/main/native/src/org/apache/raft/io/nativeio/errno_enum.c
deleted file mode 100644
index a9f06dc..0000000
--- a/raft-common/src/main/native/src/org/apache/raft/io/nativeio/errno_enum.c
+++ /dev/null
@@ -1,123 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- *     http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
- #include <assert.h>
- #include <errno.h>
- #include <jni.h>
-
-#include "org_apache_raft.h"
-
-typedef struct errno_mapping {
-  int errno_val;
-  char *errno_str;
-} errno_mapping_t;
-
-// Macro to define structs like {FOO, "FOO"} for each errno value
-#define MAPPING(x) {x, #x}
-static errno_mapping_t ERRNO_MAPPINGS[] = {
-  MAPPING(EPERM),
-  MAPPING(ENOENT),
-  MAPPING(ESRCH),
-  MAPPING(EINTR),
-  MAPPING(EIO),
-  MAPPING(ENXIO),
-  MAPPING(E2BIG),
-  MAPPING(ENOEXEC),
-  MAPPING(EBADF),
-  MAPPING(ECHILD),
-  MAPPING(EAGAIN),
-  MAPPING(ENOMEM),
-  MAPPING(EACCES),
-  MAPPING(EFAULT),
-  MAPPING(ENOTBLK),
-  MAPPING(EBUSY),
-  MAPPING(EEXIST),
-  MAPPING(EXDEV),
-  MAPPING(ENODEV),
-  MAPPING(ENOTDIR),
-  MAPPING(EISDIR),
-  MAPPING(EINVAL),
-  MAPPING(ENFILE),
-  MAPPING(EMFILE),
-  MAPPING(ENOTTY),
-  MAPPING(ETXTBSY),
-  MAPPING(EFBIG),
-  MAPPING(ENOSPC),
-  MAPPING(ESPIPE),
-  MAPPING(EROFS),
-  MAPPING(EMLINK),
-  MAPPING(EPIPE),
-  MAPPING(EDOM),
-  MAPPING(ERANGE),
-  MAPPING(ELOOP),
-  MAPPING(ENAMETOOLONG),
-  MAPPING(ENOTEMPTY),
-  MAPPING(EOVERFLOW),
-  {-1, NULL}
-};
-
-static jclass enum_class;
-static jmethodID enum_valueOf;
-static jclass errno_class;
-
-void errno_enum_init(JNIEnv *env) {
-  if (enum_class != NULL) return;
-
-  enum_class = (*env)->FindClass(env, "java/lang/Enum");
-  PASS_EXCEPTIONS(env);
-  enum_class = (*env)->NewGlobalRef(env, enum_class);
-  enum_valueOf = (*env)->GetStaticMethodID(env, enum_class,
-    "valueOf", "(Ljava/lang/Class;Ljava/lang/String;)Ljava/lang/Enum;");
-  PASS_EXCEPTIONS(env);
-
-  errno_class = (*env)->FindClass(env, "org/apache/raft/io/nativeio/Errno");
-  PASS_EXCEPTIONS(env);
-  errno_class = (*env)->NewGlobalRef(env, errno_class);
-}
-
-void errno_enum_deinit(JNIEnv *env) {
-  if (enum_class != NULL) {
-    (*env)->DeleteGlobalRef(env, enum_class);
-    enum_class = NULL;
-  }
-  if (errno_class != NULL) {
-    (*env)->DeleteGlobalRef(env, errno_class);
-    errno_class = NULL;
-  }
-  enum_valueOf = NULL;
-}
-
-
-static char *errno_to_string(int errnum) {
-  int i;
-  for (i = 0; ERRNO_MAPPINGS[i].errno_str != NULL; i++) {
-    if (ERRNO_MAPPINGS[i].errno_val == errnum)
-      return ERRNO_MAPPINGS[i].errno_str;
-  }
-  return "UNKNOWN";
-}
-
-jobject errno_to_enum(JNIEnv *env, int errnum) {
-  char *str = errno_to_string(errnum);
-  assert(str != NULL);
-
-  jstring jstr = (*env)->NewStringUTF(env, str);
-  PASS_EXCEPTIONS_RET(env, NULL);
-
-  return (*env)->CallStaticObjectMethod(
-    env, enum_class, enum_valueOf, errno_class, jstr);
-}

http://git-wip-us.apache.org/repos/asf/incubator-ratis/blob/7e71a2e0/raft-common/src/main/native/src/org/apache/raft/io/nativeio/errno_enum.h
----------------------------------------------------------------------
diff --git a/raft-common/src/main/native/src/org/apache/raft/io/nativeio/errno_enum.h b/raft-common/src/main/native/src/org/apache/raft/io/nativeio/errno_enum.h
deleted file mode 100644
index 1eee11a..0000000
--- a/raft-common/src/main/native/src/org/apache/raft/io/nativeio/errno_enum.h
+++ /dev/null
@@ -1,27 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- *     http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-#ifndef ERRNO_ENUM_H
-#define ERRNO_ENUM_H
-
-#include <jni.h>
-
-void errno_enum_init(JNIEnv *env);
-void errno_enum_deinit(JNIEnv *env);
-jobject errno_to_enum(JNIEnv *env, int errnum);
-
-#endif

http://git-wip-us.apache.org/repos/asf/incubator-ratis/blob/7e71a2e0/raft-common/src/main/native/src/org/apache/raft/io/nativeio/file_descriptor.c
----------------------------------------------------------------------
diff --git a/raft-common/src/main/native/src/org/apache/raft/io/nativeio/file_descriptor.c b/raft-common/src/main/native/src/org/apache/raft/io/nativeio/file_descriptor.c
deleted file mode 100644
index 92109fc..0000000
--- a/raft-common/src/main/native/src/org/apache/raft/io/nativeio/file_descriptor.c
+++ /dev/null
@@ -1,115 +0,0 @@
-/*
- *  Licensed to the Apache Software Foundation (ASF) under one or more
- *  contributor license agreements.  See the NOTICE file distributed with
- *  this work for additional information regarding copyright ownership.
- *  The ASF licenses this file to You under the Apache License, Version 2.0
- *  (the "License"); you may not use this file except in compliance with
- *  the License.  You may obtain a copy of the License at
- *
- *     http://www.apache.org/licenses/LICENSE-2.0
- *
- *  Unless required by applicable law or agreed to in writing, software
- *  distributed under the License is distributed on an "AS IS" BASIS,
- *  WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- *  See the License for the specific language governing permissions and
- *  limitations under the License.
- */
-
-#include <jni.h>
-#include "file_descriptor.h"
-#include "org_apache_raft.h"
-
-// class of java.io.FileDescriptor
-static jclass fd_class;
-// the internal field for the integer fd
-static jfieldID fd_descriptor;
-// the no-argument constructor
-static jmethodID fd_constructor;
-
-#ifdef WINDOWS
-// the internal field for the long handle
-static jfieldID fd_handle;
-#endif
-
-void fd_init(JNIEnv* env)
-{
-  if (fd_class != NULL) return; // already initted
-
-  fd_class = (*env)->FindClass(env, "java/io/FileDescriptor");
-  PASS_EXCEPTIONS(env);
-  fd_class = (*env)->NewGlobalRef(env, fd_class);
-
-  fd_descriptor = (*env)->GetFieldID(env, fd_class, "fd", "I");
-  PASS_EXCEPTIONS(env);
-
-#ifdef WINDOWS
-  fd_handle = (*env)->GetFieldID(env, fd_class, "handle", "J");
-  PASS_EXCEPTIONS(env);
-#endif
-
-  fd_constructor = (*env)->GetMethodID(env, fd_class, "<init>", "()V");
-}
-
-void fd_deinit(JNIEnv *env) {
-  if (fd_class != NULL) {
-    (*env)->DeleteGlobalRef(env, fd_class);
-    fd_class = NULL;
-  }
-  fd_descriptor = NULL;
-#ifdef WINDOWS
-  fd_handle = NULL;
-#endif
-  fd_constructor = NULL;
-}
-
-#ifdef UNIX
-/*
- * Given an instance 'obj' of java.io.FileDescriptor, return the
- * underlying fd, or throw if unavailable
- */
-int fd_get(JNIEnv* env, jobject obj) {
-  if (obj == NULL) {
-    THROW(env, "java/lang/NullPointerException",
-          "FileDescriptor object is null");
-    return -1;
-  }
-  return (*env)->GetIntField(env, obj, fd_descriptor);
-}
-
-/*
- * Create a FileDescriptor object corresponding to the given int fd
- */
-jobject fd_create(JNIEnv *env, int fd) {
-  jobject obj = (*env)->NewObject(env, fd_class, fd_constructor);
-  PASS_EXCEPTIONS_RET(env, NULL);
-
-  (*env)->SetIntField(env, obj, fd_descriptor, fd);
-  return obj;
-}
-#endif
-
-#ifdef WINDOWS
-/*
- * Given an instance 'obj' of java.io.FileDescriptor, return the
- * underlying fd, or throw if unavailable
- */
-long fd_get(JNIEnv* env, jobject obj) {
-  if (obj == NULL) {
-    THROW(env, "java/lang/NullPointerException",
-          "FileDescriptor object is null");
-    return -1;
-  }
-  return (long) (*env)->GetLongField(env, obj, fd_handle);
-}
-
-/*
- * Create a FileDescriptor object corresponding to the given int fd
- */
-jobject fd_create(JNIEnv *env, long fd) {
-  jobject obj = (*env)->NewObject(env, fd_class, fd_constructor);
-  PASS_EXCEPTIONS_RET(env, (jobject) NULL);
-
-  (*env)->SetLongField(env, obj, fd_handle, fd);
-  return obj;
-}
-#endif

http://git-wip-us.apache.org/repos/asf/incubator-ratis/blob/7e71a2e0/raft-common/src/main/native/src/org/apache/raft/io/nativeio/file_descriptor.h
----------------------------------------------------------------------
diff --git a/raft-common/src/main/native/src/org/apache/raft/io/nativeio/file_descriptor.h b/raft-common/src/main/native/src/org/apache/raft/io/nativeio/file_descriptor.h
deleted file mode 100644
index 89b9c6e..0000000
--- a/raft-common/src/main/native/src/org/apache/raft/io/nativeio/file_descriptor.h
+++ /dev/null
@@ -1,36 +0,0 @@
-/*
- *  Licensed to the Apache Software Foundation (ASF) under one or more
- *  contributor license agreements.  See the NOTICE file distributed with
- *  this work for additional information regarding copyright ownership.
- *  The ASF licenses this file to You under the Apache License, Version 2.0
- *  (the "License"); you may not use this file except in compliance with
- *  the License.  You may obtain a copy of the License at
- *
- *     http://www.apache.org/licenses/LICENSE-2.0
- *
- *  Unless required by applicable law or agreed to in writing, software
- *  distributed under the License is distributed on an "AS IS" BASIS,
- *  WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- *  See the License for the specific language governing permissions and
- *  limitations under the License.
- */
-#ifndef FILE_DESCRIPTOR_H
-#define FILE_DESCRIPTOR_H
-
-#include <jni.h>
-#include "org_apache_raft.h"
-
-void fd_init(JNIEnv *env);
-void fd_deinit(JNIEnv *env);
-
-#ifdef UNIX
-int fd_get(JNIEnv* env, jobject obj);
-jobject fd_create(JNIEnv *env, int fd);
-#endif
-
-#ifdef WINDOWS
-long fd_get(JNIEnv* env, jobject obj);
-jobject fd_create(JNIEnv *env, long fd);
-#endif
-
-#endif

http://git-wip-us.apache.org/repos/asf/incubator-ratis/blob/7e71a2e0/raft-common/src/main/native/src/org/apache/raft/util/NativeCodeLoader.c
----------------------------------------------------------------------
diff --git a/raft-common/src/main/native/src/org/apache/raft/util/NativeCodeLoader.c b/raft-common/src/main/native/src/org/apache/raft/util/NativeCodeLoader.c
deleted file mode 100644
index 38898fd..0000000
--- a/raft-common/src/main/native/src/org/apache/raft/util/NativeCodeLoader.c
+++ /dev/null
@@ -1,56 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- *     http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-
-#include "org_apache_raft.h"
-
-#ifdef UNIX
-#include <dlfcn.h>
-#include "config.h"
-#endif // UNIX
-
-#ifdef WINDOWS
-#include "winutils.h"
-#endif
-
-#include <jni.h>
-
-JNIEXPORT jstring JNICALL Java_org_apache_raft_util_NativeCodeLoader_getLibraryName
-  (JNIEnv *env, jclass clazz)
-{
-#ifdef UNIX
-  Dl_info dl_info;
-  int ret = dladdr(
-      Java_org_apache_raft_util_NativeCodeLoader_getLibraryName,
-      &dl_info);
-  return (*env)->NewStringUTF(env, ret==0 ? "Unavailable" : dl_info.dli_fname);
-#endif
-
-#ifdef WINDOWS
-  LPWSTR filename = NULL;
-  GetLibraryName(Java_org_apache_raft_util_NativeCodeLoader_getLibraryName,
-    &filename);
-  if (filename != NULL)
-  {
-    return (*env)->NewString(env, filename, (jsize) wcslen(filename));
-  }
-  else
-  {
-    return (*env)->NewStringUTF(env, "Unavailable");
-  }
-#endif
-}

http://git-wip-us.apache.org/repos/asf/incubator-ratis/blob/7e71a2e0/raft-common/src/main/native/src/org/apache/raft/util/NativeCrc32.c
----------------------------------------------------------------------
diff --git a/raft-common/src/main/native/src/org/apache/raft/util/NativeCrc32.c b/raft-common/src/main/native/src/org/apache/raft/util/NativeCrc32.c
deleted file mode 100644
index 6fbc0d0..0000000
--- a/raft-common/src/main/native/src/org/apache/raft/util/NativeCrc32.c
+++ /dev/null
@@ -1,276 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- *     http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-
-#include "org_apache_raft.h"
-#include "org_apache_raft_util_NativeCrc32.h"
-
-#include <assert.h>
-#include <stdlib.h>
-#include <stdint.h>
-#include <string.h>
-
-#ifdef UNIX
-#include <inttypes.h>
-#include <arpa/inet.h>
-#include <unistd.h>
-#include "config.h"
-#include "gcc_optimizations.h"
-#endif // UNIX
-
-#include "bulk_crc32.h"
-
-#define MBYTE 1048576
-#define MIN(X,Y) ((X) < (Y) ? (X) : (Y))
-#define MAX(X,Y) ((X) > (Y) ? (X) : (Y))
-
-static void throw_checksum_exception(JNIEnv *env,
-    uint32_t got_crc, uint32_t expected_crc,
-    jstring j_filename, jlong pos) {
-  char message[1024];
-  jstring jstr_message;
-  char *filename;
-  jclass checksum_exception_clazz;
-  jmethodID checksum_exception_ctor;
-  jthrowable obj;
-
-  // Get filename as C string, or "null" if not provided
-  if (j_filename == NULL) {
-    filename = strdup("null");
-  } else {
-    const char *c_filename = (*env)->GetStringUTFChars(env, j_filename, NULL);
-    if (c_filename == NULL) {
-      return; // OOME already thrown
-    }
-    filename = strdup(c_filename);
-    (*env)->ReleaseStringUTFChars(env, j_filename, c_filename);
-  }
-
-  // Format error message
-#ifdef WINDOWS
-  _snprintf_s(
-	message,
-	sizeof(message),
-	_TRUNCATE,
-    "Checksum error: %s at %I64d exp: %d got: %d",
-    filename, pos, expected_crc, got_crc);
-#else
-  snprintf(message, sizeof(message),
-    "Checksum error: %s at %"PRId64" exp: %"PRId32" got: %"PRId32,
-    filename, pos, expected_crc, got_crc);
-#endif // WINDOWS
-
-  if ((jstr_message = (*env)->NewStringUTF(env, message)) == NULL) {
-    goto cleanup;
-  }
- 
-  // Throw exception
-  checksum_exception_clazz = (*env)->FindClass(
-    env, "org/apache/raft/protocol/ChecksumException");
-  if (checksum_exception_clazz == NULL) {
-    goto cleanup;
-  }
-
-  checksum_exception_ctor = (*env)->GetMethodID(env,
-    checksum_exception_clazz, "<init>",
-    "(Ljava/lang/String;J)V");
-  if (checksum_exception_ctor == NULL) {
-    goto cleanup;
-  }
-
-  obj = (jthrowable)(*env)->NewObject(env, checksum_exception_clazz,
-    checksum_exception_ctor, jstr_message, pos);
-  if (obj == NULL) goto cleanup;
-
-  (*env)->Throw(env, obj);
-
-cleanup:
-  if (filename != NULL) {
-    free(filename);
-  }
-}
-
-static int convert_java_crc_type(JNIEnv *env, jint crc_type) {
-  switch (crc_type) {
-    case org_apache_raft_util_NativeCrc32_CHECKSUM_CRC32:
-      return CRC32_ZLIB_POLYNOMIAL;
-    case org_apache_raft_util_NativeCrc32_CHECKSUM_CRC32C:
-      return CRC32C_POLYNOMIAL;
-    default:
-      THROW(env, "java/lang/IllegalArgumentException",
-        "Invalid checksum type");
-      return -1;
-  }
-}
-
-JNIEXPORT void JNICALL Java_org_apache_raft_util_NativeCrc32_nativeComputeChunkedSums
-  (JNIEnv *env, jclass clazz,
-    jint bytes_per_checksum, jint j_crc_type,
-    jobject j_sums, jint sums_offset,
-    jobject j_data, jint data_offset, jint data_len,
-    jstring j_filename, jlong base_pos, jboolean verify)
-{
-  uint8_t *sums_addr;
-  uint8_t *data_addr;
-  uint32_t *sums;
-  uint8_t *data;
-  int crc_type;
-  crc32_error_t error_data;
-  int ret;
-
-  if (unlikely(!j_sums || !j_data)) {
-    THROW(env, "java/lang/NullPointerException",
-      "input ByteBuffers must not be null");
-    return;
-  }
-
-  // Convert direct byte buffers to C pointers
-  sums_addr = (*env)->GetDirectBufferAddress(env, j_sums);
-  data_addr = (*env)->GetDirectBufferAddress(env, j_data);
-
-  if (unlikely(!sums_addr || !data_addr)) {
-    THROW(env, "java/lang/IllegalArgumentException",
-      "input ByteBuffers must be direct buffers");
-    return;
-  }
-  if (unlikely(sums_offset < 0 || data_offset < 0 || data_len < 0)) {
-    THROW(env, "java/lang/IllegalArgumentException",
-      "bad offsets or lengths");
-    return;
-  }
-  if (unlikely(bytes_per_checksum) <= 0) {
-    THROW(env, "java/lang/IllegalArgumentException",
-      "invalid bytes_per_checksum");
-    return;
-  }
-
-  sums = (uint32_t *)(sums_addr + sums_offset);
-  data = data_addr + data_offset;
-
-  // Convert to correct internal C constant for CRC type
-  crc_type = convert_java_crc_type(env, j_crc_type);
-  if (crc_type == -1) return; // exception already thrown
-
-  // Setup complete. Actually verify checksums.
-  ret = bulk_crc(data, data_len, sums, crc_type,
-                            bytes_per_checksum, verify ? &error_data : NULL);
-  if (likely((verify && ret == CHECKSUMS_VALID) || (!verify && ret == 0))) {
-    return;
-  } else if (unlikely(verify && ret == INVALID_CHECKSUM_DETECTED)) {
-    long pos = base_pos + (error_data.bad_data - data);
-    throw_checksum_exception(
-      env, error_data.got_crc, error_data.expected_crc,
-      j_filename, pos);
-  } else {
-    THROW(env, "java/lang/AssertionError",
-      "Bad response code from native bulk_crc");
-  }
-}
-
-JNIEXPORT void JNICALL Java_org_apache_raft_util_NativeCrc32_nativeVerifyChunkedSums
-  (JNIEnv *env, jclass clazz,
-    jint bytes_per_checksum, jint j_crc_type,
-    jobject j_sums, jint sums_offset,
-    jobject j_data, jint data_offset, jint data_len,
-    jstring j_filename, jlong base_pos)
-{
-  Java_org_apache_raft_util_NativeCrc32_nativeComputeChunkedSums(env, clazz,
-    bytes_per_checksum, j_crc_type, j_sums, sums_offset, j_data, data_offset,
-    data_len, j_filename, base_pos, JNI_TRUE);
-}
-
-JNIEXPORT void JNICALL Java_org_apache_raft_util_NativeCrc32_nativeComputeChunkedSumsByteArray
-  (JNIEnv *env, jclass clazz,
-    jint bytes_per_checksum, jint j_crc_type,
-    jarray j_sums, jint sums_offset,
-    jarray j_data, jint data_offset, jint data_len,
-    jstring j_filename, jlong base_pos, jboolean verify)
-{
-  uint8_t *sums_addr;
-  uint8_t *data_addr;
-  uint32_t *sums;
-  uint8_t *data;
-  int crc_type;
-  crc32_error_t error_data;
-  int ret;
-  int numChecksumsPerIter;
-  int checksumNum;
-
-  if (unlikely(!j_sums || !j_data)) {
-    THROW(env, "java/lang/NullPointerException",
-      "input byte arrays must not be null");
-    return;
-  }
-  if (unlikely(sums_offset < 0 || data_offset < 0 || data_len < 0)) {
-    THROW(env, "java/lang/IllegalArgumentException",
-      "bad offsets or lengths");
-    return;
-  }
-  if (unlikely(bytes_per_checksum) <= 0) {
-    THROW(env, "java/lang/IllegalArgumentException",
-      "invalid bytes_per_checksum");
-    return;
-  }
-
-  // Convert to correct internal C constant for CRC type
-  crc_type = convert_java_crc_type(env, j_crc_type);
-  if (crc_type == -1) return; // exception already thrown
-
-  numChecksumsPerIter = MAX(1, MBYTE / bytes_per_checksum);
-  checksumNum = 0;
-  while (checksumNum * bytes_per_checksum < data_len) {
-    // Convert byte arrays to C pointers
-    sums_addr = (*env)->GetPrimitiveArrayCritical(env, j_sums, NULL);
-    data_addr = (*env)->GetPrimitiveArrayCritical(env, j_data, NULL);
-
-    if (unlikely(!sums_addr || !data_addr)) {
-      if (data_addr) (*env)->ReleasePrimitiveArrayCritical(env, j_data, data_addr, 0);
-      if (sums_addr) (*env)->ReleasePrimitiveArrayCritical(env, j_sums, sums_addr, 0);
-      THROW(env, "java/lang/OutOfMemoryError",
-        "not enough memory for byte arrays in JNI code");
-      return;
-    }
-
-    sums = (uint32_t *)(sums_addr + sums_offset) + checksumNum;
-    data = data_addr + data_offset + checksumNum * bytes_per_checksum;
-
-    // Setup complete. Actually verify checksums.
-    ret = bulk_crc(data, MIN(numChecksumsPerIter * bytes_per_checksum,
-                             data_len - checksumNum * bytes_per_checksum),
-                   sums, crc_type, bytes_per_checksum, verify ? &error_data : NULL);
-    (*env)->ReleasePrimitiveArrayCritical(env, j_data, data_addr, 0);
-    (*env)->ReleasePrimitiveArrayCritical(env, j_sums, sums_addr, 0);
-    if (unlikely(verify && ret == INVALID_CHECKSUM_DETECTED)) {
-      long pos = base_pos + (error_data.bad_data - data) + checksumNum *
-        bytes_per_checksum;
-      throw_checksum_exception(
-        env, error_data.got_crc, error_data.expected_crc,
-        j_filename, pos);
-      return;
-    } else if (unlikely((verify && ret != CHECKSUMS_VALID) || (!verify && ret != 0))) {
-      THROW(env, "java/lang/AssertionError",
-        "Bad response code from native bulk_crc");
-      return;
-    }
-    checksumNum += numChecksumsPerIter;
-  }
-
-}
-
-/**
- * vim: sw=2: ts=2: et:
- */

http://git-wip-us.apache.org/repos/asf/incubator-ratis/blob/7e71a2e0/raft-common/src/main/native/src/org/apache/raft/util/bulk_crc32.c
----------------------------------------------------------------------
diff --git a/raft-common/src/main/native/src/org/apache/raft/util/bulk_crc32.c b/raft-common/src/main/native/src/org/apache/raft/util/bulk_crc32.c
deleted file mode 100644
index 30b03c0..0000000
--- a/raft-common/src/main/native/src/org/apache/raft/util/bulk_crc32.c
+++ /dev/null
@@ -1,244 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- *     http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- *
- * Portions of this file are from http://www.evanjones.ca/crc32c.html under
- * the BSD license:
- *   Copyright 2008,2009,2010 Massachusetts Institute of Technology.
- *   All rights reserved. Use of this source code is governed by a
- *   BSD-style license that can be found in the LICENSE file.
- */
-
-#include "org_apache_raft.h"
-
-#include <assert.h>
-#include <errno.h>
-#include <stdint.h>
-
-#ifdef UNIX
-#include <arpa/inet.h>
-#include <unistd.h>
-#endif // UNIX
-
-#include "crc32_zlib_polynomial_tables.h"
-#include "crc32c_tables.h"
-#include "bulk_crc32.h"
-#include "gcc_optimizations.h"
-
-#define CRC_INITIAL_VAL 0xffffffff
-
-static uint32_t crc_val(uint32_t crc);
-
-typedef void (*crc_pipelined_func_t)(uint32_t *, uint32_t *, uint32_t *, const uint8_t *, size_t, int);
-
-// The software versions of pipelined crc
-static void pipelined_crc32c_sb8(uint32_t *crc1, uint32_t *crc2, uint32_t *crc3,
-                                 const uint8_t *p_buf, size_t block_size, int num_blocks);
-static void pipelined_crc32_zlib_sb8(uint32_t *crc1, uint32_t *crc2, uint32_t *crc3,
-                                 const uint8_t *p_buf, size_t block_size, int num_blocks);
-
-// Satically initialise the function pointers to the software versions
-// If a HW implementation is available they will subsequently be initialised in the dynamic
-// initialisers to point to the HW routines.
-crc_pipelined_func_t pipelined_crc32c_func = pipelined_crc32c_sb8;
-crc_pipelined_func_t pipelined_crc32_zlib_func = pipelined_crc32_zlib_sb8;
-
-static inline int store_or_verify(uint32_t *sums, uint32_t crc,
-                                   int is_verify) {
-  if (!is_verify) {
-    *sums = crc;
-    return 1;
-  } else {
-    return crc == *sums;
-  }
-}
-
-int bulk_crc(const uint8_t *data, size_t data_len,
-                    uint32_t *sums, int checksum_type,
-                    int bytes_per_checksum,
-                    crc32_error_t *error_info) {
-
-  int is_verify = error_info != NULL;
-
-  uint32_t crc1, crc2, crc3;
-  int n_blocks = data_len / bytes_per_checksum;
-  int remainder = data_len % bytes_per_checksum;
-  uint32_t crc;
-  crc_pipelined_func_t crc_pipelined_func;
-  switch (checksum_type) {
-    case CRC32_ZLIB_POLYNOMIAL:
-      crc_pipelined_func = pipelined_crc32_zlib_func;
-      break;
-    case CRC32C_POLYNOMIAL:
-      crc_pipelined_func = pipelined_crc32c_func;
-      break;
-    default:
-      return is_verify ? INVALID_CHECKSUM_TYPE : -EINVAL;
-  }
-
-  /* Process three blocks at a time */
-  while (likely(n_blocks >= 3)) {
-    crc1 = crc2 = crc3 = CRC_INITIAL_VAL;
-    crc_pipelined_func(&crc1, &crc2, &crc3, data, bytes_per_checksum, 3);
-
-    if (unlikely(!store_or_verify(sums, (crc = ntohl(crc_val(crc1))), is_verify)))
-      goto return_crc_error;
-    sums++;
-    data += bytes_per_checksum;
-    if (unlikely(!store_or_verify(sums, (crc = ntohl(crc_val(crc2))), is_verify)))
-      goto return_crc_error;
-    sums++;
-    data += bytes_per_checksum;
-    if (unlikely(!store_or_verify(sums, (crc = ntohl(crc_val(crc3))), is_verify)))
-      goto return_crc_error;
-    sums++;
-    data += bytes_per_checksum;
-    n_blocks -= 3;
-  }
-
-  /* One or two blocks */
-  if (n_blocks) {
-    crc1 = crc2 = crc3 = CRC_INITIAL_VAL;
-    crc_pipelined_func(&crc1, &crc2, &crc3, data, bytes_per_checksum, n_blocks);
-
-    if (unlikely(!store_or_verify(sums, (crc = ntohl(crc_val(crc1))), is_verify)))
-      goto return_crc_error;
-    data += bytes_per_checksum;
-    sums++;
-    if (n_blocks == 2) {
-      if (unlikely(!store_or_verify(sums, (crc = ntohl(crc_val(crc2))), is_verify)))
-        goto return_crc_error;
-      sums++;
-      data += bytes_per_checksum;
-    }
-  }
-
-  /* For something smaller than a block */
-  if (remainder) {
-    crc1 = crc2 = crc3 = CRC_INITIAL_VAL;
-    crc_pipelined_func(&crc1, &crc2, &crc3, data, remainder, 1);
-
-    if (unlikely(!store_or_verify(sums, (crc = ntohl(crc_val(crc1))), is_verify)))
-      goto return_crc_error;
-  }
-  return is_verify ? CHECKSUMS_VALID : 0;
-
-return_crc_error:
-  if (error_info != NULL) {
-    error_info->got_crc = crc;
-    error_info->expected_crc = *sums;
-    error_info->bad_data = data;
-  }
-  return INVALID_CHECKSUM_DETECTED;
-}
-
-/**
- * Extract the final result of a CRC
- */
-static uint32_t crc_val(uint32_t crc) {
-  return ~crc;
-}
-
-/**
- * Computes the CRC32c checksum for the specified buffer using the slicing by 8 
- * algorithm over 64 bit quantities.
- */
-static uint32_t crc32c_sb8(uint32_t crc, const uint8_t *buf, size_t length) {
-  uint32_t running_length = ((length)/8)*8;
-  uint32_t end_bytes = length - running_length; 
-  int li;
-  for (li=0; li < running_length/8; li++) {
-	uint32_t term1;
-	uint32_t term2;
-    crc ^= *(uint32_t *)buf;
-    buf += 4;
-    term1 = CRC32C_T8_7[crc & 0x000000FF] ^
-        CRC32C_T8_6[(crc >> 8) & 0x000000FF];
-    term2 = crc >> 16;
-    crc = term1 ^
-        CRC32C_T8_5[term2 & 0x000000FF] ^ 
-        CRC32C_T8_4[(term2 >> 8) & 0x000000FF];
-    term1 = CRC32C_T8_3[(*(uint32_t *)buf) & 0x000000FF] ^
-        CRC32C_T8_2[((*(uint32_t *)buf) >> 8) & 0x000000FF];
-    
-    term2 = (*(uint32_t *)buf) >> 16;
-    crc =  crc ^ 
-        term1 ^    
-        CRC32C_T8_1[term2  & 0x000000FF] ^  
-        CRC32C_T8_0[(term2 >> 8) & 0x000000FF];  
-    buf += 4;
-  }
-  for (li=0; li < end_bytes; li++) {
-    crc = CRC32C_T8_0[(crc ^ *buf++) & 0x000000FF] ^ (crc >> 8);
-  }
-  return crc;    
-}
-
-static void pipelined_crc32c_sb8(uint32_t *crc1, uint32_t *crc2, uint32_t *crc3,
-                                 const uint8_t *p_buf, size_t block_size, int num_blocks) {
-  assert(num_blocks >= 1 && num_blocks <=3 && "invalid num_blocks");
-  *crc1 = crc32c_sb8(*crc1, p_buf, block_size);
-  if (num_blocks >= 2)
-    *crc2 = crc32c_sb8(*crc2, p_buf+block_size, block_size);
-  if (num_blocks >= 3)
-    *crc3 = crc32c_sb8(*crc3, p_buf+2*block_size, block_size);
-}
-
-/**
- * Update a CRC using the "zlib" polynomial -- what Raft calls CHECKSUM_CRC32
- * using slicing-by-8
- */
-static uint32_t crc32_zlib_sb8(
-    uint32_t crc, const uint8_t *buf, size_t length) {
-  uint32_t running_length = ((length)/8)*8;
-  uint32_t end_bytes = length - running_length; 
-  int li;
-  for (li=0; li < running_length/8; li++) {
-	uint32_t term1;
-	uint32_t term2;
-    crc ^= *(uint32_t *)buf;
-    buf += 4;
-    term1 = CRC32_T8_7[crc & 0x000000FF] ^
-        CRC32_T8_6[(crc >> 8) & 0x000000FF];
-    term2 = crc >> 16;
-    crc = term1 ^
-        CRC32_T8_5[term2 & 0x000000FF] ^ 
-        CRC32_T8_4[(term2 >> 8) & 0x000000FF];
-    term1 = CRC32_T8_3[(*(uint32_t *)buf) & 0x000000FF] ^
-        CRC32_T8_2[((*(uint32_t *)buf) >> 8) & 0x000000FF];
-    
-    term2 = (*(uint32_t *)buf) >> 16;
-    crc =  crc ^ 
-        term1 ^    
-        CRC32_T8_1[term2  & 0x000000FF] ^  
-        CRC32_T8_0[(term2 >> 8) & 0x000000FF];  
-    buf += 4;
-  }
-  for (li=0; li < end_bytes; li++) {
-    crc = CRC32_T8_0[(crc ^ *buf++) & 0x000000FF] ^ (crc >> 8);
-  }
-  return crc;    
-}
-
-static void pipelined_crc32_zlib_sb8(uint32_t *crc1, uint32_t *crc2, uint32_t *crc3,
-                                     const uint8_t *p_buf, size_t block_size, int num_blocks) {
-  assert(num_blocks >= 1 && num_blocks <=3 && "invalid num_blocks");
-  *crc1 = crc32_zlib_sb8(*crc1, p_buf, block_size);
-  if (num_blocks >= 2)
-    *crc2 = crc32_zlib_sb8(*crc2, p_buf+block_size, block_size);
-  if (num_blocks >= 3)
-    *crc3 = crc32_zlib_sb8(*crc3, p_buf+2*block_size, block_size);
-}

http://git-wip-us.apache.org/repos/asf/incubator-ratis/blob/7e71a2e0/raft-common/src/main/native/src/org/apache/raft/util/bulk_crc32.h
----------------------------------------------------------------------
diff --git a/raft-common/src/main/native/src/org/apache/raft/util/bulk_crc32.h b/raft-common/src/main/native/src/org/apache/raft/util/bulk_crc32.h
deleted file mode 100644
index b38a65a..0000000
--- a/raft-common/src/main/native/src/org/apache/raft/util/bulk_crc32.h
+++ /dev/null
@@ -1,73 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- *     http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-#ifndef BULK_CRC32_H_INCLUDED
-#define BULK_CRC32_H_INCLUDED
-
-#include <stdint.h>
-
-#ifdef UNIX
-#include <unistd.h> /* for size_t */
-#endif // UNIX
-
-// Constants for different CRC algorithms
-#define CRC32C_POLYNOMIAL 1
-#define CRC32_ZLIB_POLYNOMIAL 2
-
-// Return codes for bulk_verify_crc
-#define CHECKSUMS_VALID 0
-#define INVALID_CHECKSUM_DETECTED -1
-#define INVALID_CHECKSUM_TYPE -2
-
-// Return type for bulk verification when verification fails
-typedef struct crc32_error {
-  uint32_t got_crc;
-  uint32_t expected_crc;
-  const uint8_t *bad_data; // pointer to start of data chunk with error
-} crc32_error_t;
-
-
-/**
- * Either calculates checksums for or verifies a buffer of data.
- * Checksums performed in chunks of bytes_per_checksum bytes. The checksums
- * are each 32 bits and are stored in sequential indexes of the 'sums' array.
- * Verification is done (sums is assumed to already contain the checksums)
- * if error_info is non-null; otherwise calculation is done and checksums
- * are stored into sums.
- *
- * @param data                  The data to checksum
- * @param dataLen               Length of the data buffer
- * @param sums                  (out param) buffer to write checksums into or
- *                              where checksums are already stored.
- *                              It must contain at least
- *                              ((dataLen - 1) / bytes_per_checksum + 1) * 4 bytes.
- * @param checksum_type         One of the CRC32 algorithm constants defined 
- *                              above
- * @param bytes_per_checksum    How many bytes of data to process per checksum.
- * @param error_info            If non-NULL, verification will be performed and
- *                              it will be filled in if an error
- *                              is detected. Otherwise calculation is performed.
- *
- * @return                      0 for success, non-zero for an error, result codes
- *                              for verification are defined above
- */
-extern int bulk_crc(const uint8_t *data, size_t data_len,
-    uint32_t *sums, int checksum_type,
-    int bytes_per_checksum,
-    crc32_error_t *error_info);
-
-#endif

http://git-wip-us.apache.org/repos/asf/incubator-ratis/blob/7e71a2e0/raft-common/src/main/native/src/org/apache/raft/util/bulk_crc32_aarch64.c
----------------------------------------------------------------------
diff --git a/raft-common/src/main/native/src/org/apache/raft/util/bulk_crc32_aarch64.c b/raft-common/src/main/native/src/org/apache/raft/util/bulk_crc32_aarch64.c
deleted file mode 100644
index ab4690b..0000000
--- a/raft-common/src/main/native/src/org/apache/raft/util/bulk_crc32_aarch64.c
+++ /dev/null
@@ -1,362 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- *     http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-
-#include <assert.h>
-#include <stddef.h>    // for size_t
-
-#include  "bulk_crc32.h"
-#include "gcc_optimizations.h"
-
-/**
- * Hardware-accelerated CRC32 calculation using the 64-bit instructions.
- * 2 variants:-
- *   pipelined_crc32c uses the Castagnoli polynomial 0x1EDC6F41
- *   pipelined_crc32_zlib uses the Zlib polynomial 0x04C11DB7
- */
-
-// gcc doesn't know how to vectorize a 128 bit load, so use the following to tell it
-#define LDP(x,y,p) asm("ldp %x[a], %x[b], [%x[c]], #16" : [a]"=r"(x),[b]"=r"(y),[c]"+r"(p))
-
-#define CRC32CX(crc,value) asm("crc32cx %w[c], %w[c], %x[v]" : [c]"+r"(*&crc) : [v]"r"(+value))
-#define CRC32CW(crc,value) asm("crc32cw %w[c], %w[c], %w[v]" : [c]"+r"(*&crc) : [v]"r"(+value))
-#define CRC32CH(crc,value) asm("crc32ch %w[c], %w[c], %w[v]" : [c]"+r"(*&crc) : [v]"r"(+value))
-#define CRC32CB(crc,value) asm("crc32cb %w[c], %w[c], %w[v]" : [c]"+r"(*&crc) : [v]"r"(+value))
-
-#define CRC32ZX(crc,value) asm("crc32x %w[c], %w[c], %x[v]" : [c]"+r"(crc) : [v]"r"(value))
-#define CRC32ZW(crc,value) asm("crc32w %w[c], %w[c], %w[v]" : [c]"+r"(crc) : [v]"r"(value))
-#define CRC32ZH(crc,value) asm("crc32h %w[c], %w[c], %w[v]" : [c]"+r"(crc) : [v]"r"(value))
-#define CRC32ZB(crc,value) asm("crc32b %w[c], %w[c], %w[v]" : [c]"+r"(crc) : [v]"r"(value))
-
-/**
- * Pipelined version of hardware-accelerated CRC32 calculation using
- * the 64 bit crc32 instructions. 
- * One crc32 instruction takes three cycles, but two more with no data
- * dependency can be in the pipeline to achieve something close to single 
- * instruction/cycle. Here we feed three blocks in RR.
- *
- * 2 variants:-
- *   pipelined_crc32c uses the Castagnoli polynomial 0x1EDC6F41
- *   pipelined_crc32_zlib uses the Zlib polynomial 0x04C11DB7
- *
- *   crc1, crc2, crc3 : Store initial checksum for each block before
- *           calling. When it returns, updated checksums are stored.
- *   p_buf : The base address of the data buffer. The buffer should be
- *           at least as big as block_size * num_blocks.
- *   block_size : The size of each block in bytes.
- *   num_blocks : The number of blocks to work on. Min = 1, Max = 3
- */
-static void pipelined_crc32c(uint32_t *crc1, uint32_t *crc2, uint32_t *crc3, const uint8_t *p_buf1, size_t block_size, int num_blocks) {
-  uint64_t c1 = *crc1;
-  uint64_t c2 = *crc2;
-  uint64_t c3 = *crc3;
-  const uint8_t *p_buf2 = p_buf1 + block_size;
-  const uint8_t *p_buf3 = p_buf1 + block_size * 2;
-  uint64_t x1, y1, x2, y2, x3, y3;
-  long len = block_size;
-
-  /* We do switch here because the loop has to be tight in order
-   * to fill the pipeline. Any other statement inside the loop
-   * or inbetween crc32 instruction can slow things down.
-   *
-   * Do verify that this code generates the expected assembler
-   * by disassembling test_bulk_crc32
-   */
-
-  asm(".cpu generic+crc");	// Allow crc instructions in asm
-  switch (num_blocks) {
-    case 3:
-      /* Do three blocks */
-      while ((len -= 2*sizeof(uint64_t)) >= 0) {
-        LDP(x1,y1,p_buf1);
-        LDP(x2,y2,p_buf2);
-        LDP(x3,y3,p_buf3);
-        CRC32CX(c1, x1);
-        CRC32CX(c2, x2);
-        CRC32CX(c3, x3);
-        CRC32CX(c1, y1);
-        CRC32CX(c2, y2);
-        CRC32CX(c3, y3);
-      }
-
-      if (unlikely(len & sizeof(uint64_t))) {
-        x1 = *(uint64_t*)p_buf1; p_buf1 += sizeof(uint64_t);
-        x2 = *(uint64_t*)p_buf2; p_buf2 += sizeof(uint64_t);
-        x3 = *(uint64_t*)p_buf3; p_buf3 += sizeof(uint64_t);
-        CRC32CX(c1, x1);
-        CRC32CX(c2, x2);
-        CRC32CX(c3, x3);
-      }
-      if (unlikely(len & sizeof(uint32_t))) {
-        x1 = *(uint32_t*)p_buf1; p_buf1 += sizeof(uint32_t);
-        x2 = *(uint32_t*)p_buf2; p_buf2 += sizeof(uint32_t);
-        x3 = *(uint32_t*)p_buf3; p_buf3 += sizeof(uint32_t);
-        CRC32CW(c1, x1);
-        CRC32CW(c2, x2);
-        CRC32CW(c3, x3);
-      }
-      if (unlikely(len & sizeof(uint16_t))) {
-        x1 = *(uint16_t*)p_buf1; p_buf1 += sizeof(uint16_t);
-        x2 = *(uint16_t*)p_buf2; p_buf2 += sizeof(uint16_t);
-        x3 = *(uint16_t*)p_buf3; p_buf3 += sizeof(uint16_t);
-        CRC32CH(c1, x1);
-        CRC32CH(c2, x2);
-        CRC32CH(c3, x3);
-      }
-      if (unlikely(len & sizeof(uint8_t))) {
-        x1 = *p_buf1;
-        x2 = *p_buf2;
-        x3 = *p_buf3;
-        CRC32CB(c1, x1);
-        CRC32CB(c2, x2);
-        CRC32CB(c3, x3);
-      }
-      break;
-    case 2:
-      /* Do two blocks */
-      while ((len -= 2*sizeof(uint64_t)) >= 0) {
-        LDP(x1,y1,p_buf1);
-        LDP(x2,y2,p_buf2);
-        CRC32CX(c1, x1);
-        CRC32CX(c2, x2);
-        CRC32CX(c1, y1);
-        CRC32CX(c2, y2);
-      }
-
-      if (unlikely(len & sizeof(uint64_t))) {
-        x1 = *(uint64_t*)p_buf1; p_buf1 += sizeof(uint64_t);
-        x2 = *(uint64_t*)p_buf2; p_buf2 += sizeof(uint64_t);
-        CRC32CX(c1, x1);
-        CRC32CX(c2, x2);
-      }
-      if (unlikely(len & sizeof(uint32_t))) {
-        x1 = *(uint32_t*)p_buf1; p_buf1 += sizeof(uint32_t);
-        x2 = *(uint32_t*)p_buf2; p_buf2 += sizeof(uint32_t);
-        CRC32CW(c1, x1);
-        CRC32CW(c2, x2);
-      }
-      if (unlikely(len & sizeof(uint16_t))) {
-        x1 = *(uint16_t*)p_buf1; p_buf1 += sizeof(uint16_t);
-        x2 = *(uint16_t*)p_buf2; p_buf2 += sizeof(uint16_t);
-        CRC32CH(c1, x1);
-        CRC32CH(c2, x2);
-      }
-      if (unlikely(len & sizeof(uint8_t))) {
-        x1 = *p_buf1;
-        x2 = *p_buf2;
-        CRC32CB(c1, x1);
-        CRC32CB(c2, x2);
-      }
-      break;
-    case 1:
-      /* single block */
-      while ((len -= 2*sizeof(uint64_t)) >= 0) {
-        LDP(x1,y1,p_buf1);
-        CRC32CX(c1, x1);
-        CRC32CX(c1, y1);
-      }
-
-      if (unlikely(len & sizeof(uint64_t))) {
-        x1 = *(uint64_t*)p_buf1; p_buf1 += sizeof(uint64_t);
-        CRC32CX(c1, x1);
-      }
-      if (unlikely(len & sizeof(uint32_t))) {
-        x1 = *(uint32_t*)p_buf1; p_buf1 += sizeof(uint32_t);
-        CRC32CW(c1, x1);
-      }
-      if (unlikely(len & sizeof(uint16_t))) {
-        x1 = *(uint16_t*)p_buf1; p_buf1 += sizeof(uint16_t);
-        CRC32CH(c1, x1);
-      }
-      if (unlikely(len & sizeof(uint8_t))) {
-        x1 = *p_buf1;
-        CRC32CB(c1, x1);
-      }
-      break;
-    case 0:
-      return;
-    default:
-      assert(0 && "BUG: Invalid number of checksum blocks");
-  }
-
-  *crc1 = c1;
-  *crc2 = c2;
-  *crc3 = c3;
-  return;
-}
-
-static void pipelined_crc32_zlib(uint32_t *crc1, uint32_t *crc2, uint32_t *crc3, const uint8_t *p_buf1, size_t block_size, int num_blocks) {
-  uint64_t c1 = *crc1;
-  uint64_t c2 = *crc2;
-  uint64_t c3 = *crc3;
-  const uint8_t *p_buf2 = p_buf1 + block_size;
-  const uint8_t *p_buf3 = p_buf1 + block_size * 2;
-  uint64_t x1, y1, x2, y2, x3, y3;
-  long len = block_size;
-
-  /* We do switch here because the loop has to be tight in order
-   * to fill the pipeline. Any other statement inside the loop
-   * or inbetween crc32 instruction can slow things down.
-   *
-   * Do verify that this code generates the expected assembler
-   * by disassembling test_bulk_crc32
-   */
-
-  asm(".cpu generic+crc");	// Allow crc instructions in asm
-  switch (num_blocks) {
-    case 3:
-      /* Do three blocks */
-      while ((len -= 2*sizeof(uint64_t)) >= 0) {
-        LDP(x1,y1,p_buf1);
-        LDP(x2,y2,p_buf2);
-        LDP(x3,y3,p_buf3);
-        CRC32ZX(c1, x1);
-        CRC32ZX(c2, x2);
-        CRC32ZX(c3, x3);
-        CRC32ZX(c1, y1);
-        CRC32ZX(c2, y2);
-        CRC32ZX(c3, y3);
-      }
-
-      if (unlikely(len & sizeof(uint64_t))) {
-        x1 = *(uint64_t*)p_buf1; p_buf1 += sizeof(uint64_t);
-        x2 = *(uint64_t*)p_buf2; p_buf2 += sizeof(uint64_t);
-        x3 = *(uint64_t*)p_buf3; p_buf3 += sizeof(uint64_t);
-        CRC32ZX(c1, x1);
-        CRC32ZX(c2, x2);
-        CRC32ZX(c3, x3);
-      }
-      if (unlikely(len & sizeof(uint32_t))) {
-        x1 = *(uint32_t*)p_buf1; p_buf1 += sizeof(uint32_t);
-        x2 = *(uint32_t*)p_buf2; p_buf2 += sizeof(uint32_t);
-        x3 = *(uint32_t*)p_buf3; p_buf3 += sizeof(uint32_t);
-        CRC32ZW(c1, x1);
-        CRC32ZW(c2, x2);
-        CRC32ZW(c3, x3);
-      }
-      if (unlikely(len & sizeof(uint16_t))) {
-        x1 = *(uint16_t*)p_buf1; p_buf1 += sizeof(uint16_t);
-        x2 = *(uint16_t*)p_buf2; p_buf2 += sizeof(uint16_t);
-        x3 = *(uint16_t*)p_buf3; p_buf3 += sizeof(uint16_t);
-        CRC32ZH(c1, x1);
-        CRC32ZH(c2, x2);
-        CRC32ZH(c3, x3);
-      }
-      if (unlikely(len & sizeof(uint8_t))) {
-        x1 = *p_buf1;
-        x2 = *p_buf2;
-        x3 = *p_buf3;
-        CRC32ZB(c1, x1);
-        CRC32ZB(c2, x2);
-        CRC32ZB(c3, x3);
-      }
-      break;
-    case 2:
-      /* Do two blocks */
-      while ((len -= 2*sizeof(uint64_t)) >= 0) {
-        LDP(x1,y1,p_buf1);
-        LDP(x2,y2,p_buf2);
-        CRC32ZX(c1, x1);
-        CRC32ZX(c2, x2);
-        CRC32ZX(c1, y1);
-        CRC32ZX(c2, y2);
-      }
-
-      if (unlikely(len & sizeof(uint64_t))) {
-        x1 = *(uint64_t*)p_buf1; p_buf1 += sizeof(uint64_t);
-        x2 = *(uint64_t*)p_buf2; p_buf2 += sizeof(uint64_t);
-        CRC32ZX(c1, x1);
-        CRC32ZX(c2, x2);
-      }
-      if (unlikely(len & sizeof(uint32_t))) {
-        x1 = *(uint32_t*)p_buf1; p_buf1 += sizeof(uint32_t);
-        x2 = *(uint32_t*)p_buf2; p_buf2 += sizeof(uint32_t);
-        CRC32ZW(c1, x1);
-        CRC32ZW(c2, x2);
-      }
-      if (unlikely(len & sizeof(uint16_t))) {
-        x1 = *(uint16_t*)p_buf1; p_buf1 += sizeof(uint16_t);
-        x2 = *(uint16_t*)p_buf2; p_buf2 += sizeof(uint16_t);
-        CRC32ZH(c1, x1);
-        CRC32ZH(c2, x2);
-      }
-      if (unlikely(len & sizeof(uint8_t))) {
-        x1 = *p_buf1;
-        x2 = *p_buf2;
-        CRC32ZB(c1, x1);
-        CRC32ZB(c2, x2);
-      }
-      break;
-    case 1:
-      /* single block */
-      while ((len -= 2*sizeof(uint64_t)) >= 0) {
-        LDP(x1,y1,p_buf1);
-        CRC32ZX(c1, x1);
-        CRC32ZX(c1, y1);
-      }
-
-      if (unlikely(len & sizeof(uint64_t))) {
-        x1 = *(uint64_t*)p_buf1; p_buf1 += sizeof(uint64_t);
-        CRC32ZX(c1, x1);
-      }
-      if (unlikely(len & sizeof(uint32_t))) {
-        x1 = *(uint32_t*)p_buf1; p_buf1 += sizeof(uint32_t);
-        CRC32ZW(c1, x1);
-      }
-      if (unlikely(len & sizeof(uint16_t))) {
-        x1 = *(uint16_t*)p_buf1; p_buf1 += sizeof(uint16_t);
-        CRC32ZH(c1, x1);
-      }
-      if (unlikely(len & sizeof(uint8_t))) {
-        x1 = *p_buf1;
-        CRC32ZB(c1, x1);
-      }
-      break;
-    case 0:
-      return;
-    default:
-      assert(0 && "BUG: Invalid number of checksum blocks");
-  }
-
-  *crc1 = c1;
-  *crc2 = c2;
-  *crc3 = c3;
-  return;
-}
-
-typedef void (*crc_pipelined_func_t)(uint32_t *, uint32_t *, uint32_t *, const uint8_t *, size_t, int);
-extern crc_pipelined_func_t pipelined_crc32c_func;
-extern crc_pipelined_func_t pipelined_crc32_zlib_func;
-
-#include <sys/auxv.h>
-#include <asm/hwcap.h>
-
-#ifndef HWCAP_CRC32
-#define HWCAP_CRC32 (1 << 7)
-#endif
-
-/**
- * On library load, determine what sort of crc we are going to do
- * and set crc function pointers appropriately.
- */
-void __attribute__ ((constructor)) init_cpu_support_flag(void) {
-  unsigned long auxv = getauxval(AT_HWCAP);
-  if (auxv & HWCAP_CRC32) {
-    pipelined_crc32c_func = pipelined_crc32c;
-    pipelined_crc32_zlib_func = pipelined_crc32_zlib;
-  }
-}

http://git-wip-us.apache.org/repos/asf/incubator-ratis/blob/7e71a2e0/raft-common/src/main/native/src/org/apache/raft/util/bulk_crc32_x86.c
----------------------------------------------------------------------
diff --git a/raft-common/src/main/native/src/org/apache/raft/util/bulk_crc32_x86.c b/raft-common/src/main/native/src/org/apache/raft/util/bulk_crc32_x86.c
deleted file mode 100644
index 290b8a6..0000000
--- a/raft-common/src/main/native/src/org/apache/raft/util/bulk_crc32_x86.c
+++ /dev/null
@@ -1,345 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- *     http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- *
- * Portions of this file are from http://www.evanjones.ca/crc32c.html under
- * the BSD license:
- *   Copyright 2008,2009,2010 Massachusetts Institute of Technology.
- *   All rights reserved. Use of this source code is governed by a
- *   BSD-style license that can be found in the LICENSE file.
- */
-
-#include <assert.h>
-#include <stddef.h>    // for size_t
-
-#include  "bulk_crc32.h"
-#include "gcc_optimizations.h"
-#include "gcc_optimizations.h"
-
-///////////////////////////////////////////////////////////////////////////
-// Begin code for SSE4.2 specific hardware support of CRC32C
-///////////////////////////////////////////////////////////////////////////
-
-#  define SSE42_FEATURE_BIT (1 << 20)
-#  define CPUID_FEATURES 1
-/**
- * Call the cpuid instruction to determine CPU feature flags.
- */
-static uint32_t cpuid(uint32_t eax_in) {
-  uint32_t eax, ebx, ecx, edx;
-#  if defined(__PIC__) && !defined(__LP64__)
-// 32-bit PIC code uses the ebx register for the base offset --
-// have to save and restore it on the stack
-  asm("pushl %%ebx\n\t"
-      "cpuid\n\t"
-      "movl %%ebx, %[ebx]\n\t"
-      "popl %%ebx" : "=a" (eax), [ebx] "=r"(ebx),  "=c"(ecx), "=d"(edx) : "a" (eax_in)
-      : "cc");
-#  else
-  asm("cpuid" : "=a" (eax), "=b"(ebx), "=c"(ecx), "=d"(edx) : "a"(eax_in)
-      : "cc");
-#  endif
-
-  return ecx;
-}
-
-//
-// Definitions of the SSE4.2 crc32 operations. Using these instead of
-// the GCC __builtin_* intrinsics allows this code to compile without
-// -msse4.2, since we do dynamic CPU detection at runtime.
-//
-
-#  ifdef __LP64__
-inline uint64_t _mm_crc32_u64(uint64_t crc, uint64_t value) {
-  asm("crc32q %[value], %[crc]\n" : [crc] "+r" (crc) : [value] "rm" (value));
-  return crc;
-}
-#  endif
-
-inline uint32_t _mm_crc32_u32(uint32_t crc, uint32_t value) {
-  asm("crc32l %[value], %[crc]\n" : [crc] "+r" (crc) : [value] "rm" (value));
-  return crc;
-}
-
-inline uint32_t _mm_crc32_u16(uint32_t crc, uint16_t value) {
-  asm("crc32w %[value], %[crc]\n" : [crc] "+r" (crc) : [value] "rm" (value));
-  return crc;
-}
-
-inline uint32_t _mm_crc32_u8(uint32_t crc, uint8_t value) {
-  asm("crc32b %[value], %[crc]\n" : [crc] "+r" (crc) : [value] "rm" (value));
-  return crc;
-}
-
-#  ifdef __LP64__
-/**
- * Pipelined version of hardware-accelerated CRC32C calculation using
- * the 64 bit crc32q instruction. 
- * One crc32c instruction takes three cycles, but two more with no data
- * dependency can be in the pipeline to achieve something close to single 
- * instruction/cycle. Here we feed three blocks in RR.
- *
- *   crc1, crc2, crc3 : Store initial checksum for each block before
- *           calling. When it returns, updated checksums are stored.
- *   p_buf : The base address of the data buffer. The buffer should be
- *           at least as big as block_size * num_blocks.
- *   block_size : The size of each block in bytes.
- *   num_blocks : The number of blocks to work on. Min = 1, Max = 3
- */
-static void pipelined_crc32c(uint32_t *crc1, uint32_t *crc2, uint32_t *crc3, const uint8_t *p_buf, size_t block_size, int num_blocks) {
-  uint64_t c1 = *crc1;
-  uint64_t c2 = *crc2;
-  uint64_t c3 = *crc3;
-  uint64_t *data = (uint64_t*)p_buf;
-  int counter = block_size / sizeof(uint64_t);
-  int remainder = block_size % sizeof(uint64_t);
-  uint8_t *bdata;
-
-  /* We do switch here because the loop has to be tight in order
-   * to fill the pipeline. Any other statement inside the loop
-   * or inbetween crc32 instruction can slow things down. Calling
-   * individual crc32 instructions three times from C also causes
-   * gcc to insert other instructions inbetween.
-   *
-   * Do not rearrange the following code unless you have verified
-   * the generated machine code is as efficient as before.
-   */
-  switch (num_blocks) {
-    case 3:
-      /* Do three blocks */
-      while (likely(counter)) {
-        __asm__ __volatile__(
-        "crc32q (%7), %0;\n\t"
-        "crc32q (%7,%6,1), %1;\n\t"
-        "crc32q (%7,%6,2), %2;\n\t"
-         : "=r"(c1), "=r"(c2), "=r"(c3)
-         : "0"(c1), "1"(c2), "2"(c3), "r"(block_size), "r"(data)
-        );
-        data++;
-        counter--;
-      }
-
-      /* Take care of the remainder. They are only up to seven bytes,
-       * so performing byte-level crc32 won't take much time.
-       */
-      bdata = (uint8_t*)data;
-      while (likely(remainder)) {
-        __asm__ __volatile__(
-        "crc32b (%7), %0;\n\t"
-        "crc32b (%7,%6,1), %1;\n\t"
-        "crc32b (%7,%6,2), %2;\n\t"
-         : "=r"(c1), "=r"(c2), "=r"(c3)
-         : "0"(c1), "1"(c2), "2"(c3), "r"(block_size), "r"(bdata)
-        );
-        bdata++;
-        remainder--;
-      }
-      break;
-    case 2:
-      /* Do two blocks */
-      while (likely(counter)) {
-        __asm__ __volatile__(
-        "crc32q (%5), %0;\n\t"
-        "crc32q (%5,%4,1), %1;\n\t"
-         : "=r"(c1), "=r"(c2) 
-         : "0"(c1), "1"(c2), "r"(block_size), "r"(data)
-        );
-        data++;
-        counter--;
-      }
-
-      bdata = (uint8_t*)data;
-      while (likely(remainder)) {
-        __asm__ __volatile__(
-        "crc32b (%5), %0;\n\t"
-        "crc32b (%5,%4,1), %1;\n\t"
-         : "=r"(c1), "=r"(c2) 
-         : "0"(c1), "1"(c2), "r"(block_size), "r"(bdata)
-        );
-        bdata++;
-        remainder--;
-      }
-      break;
-    case 1:
-      /* single block */
-      while (likely(counter)) {
-        __asm__ __volatile__(
-        "crc32q (%2), %0;\n\t"
-         : "=r"(c1) 
-         : "0"(c1), "r"(data)
-        );
-        data++;
-        counter--;
-      }
-      bdata = (uint8_t*)data;
-      while (likely(remainder)) {
-        __asm__ __volatile__(
-        "crc32b (%2), %0;\n\t"
-         : "=r"(c1) 
-         : "0"(c1), "r"(bdata)
-        );
-        bdata++;
-        remainder--;
-      }
-      break;
-    case 0:
-      return;
-    default:
-      assert(0 && "BUG: Invalid number of checksum blocks");
-  }
-
-  *crc1 = c1;
-  *crc2 = c2;
-  *crc3 = c3;
-  return;
-}
-
-# else  // 32-bit
-
-/**
- * Pipelined version of hardware-accelerated CRC32C calculation using
- * the 32 bit crc32l instruction. 
- * One crc32c instruction takes three cycles, but two more with no data
- * dependency can be in the pipeline to achieve something close to single 
- * instruction/cycle. Here we feed three blocks in RR.
- *
- *   crc1, crc2, crc3 : Store initial checksum for each block before
- *                calling. When it returns, updated checksums are stored.
- *   data       : The base address of the data buffer. The buffer should be
- *                at least as big as block_size * num_blocks.
- *   block_size : The size of each block in bytes. 
- *   num_blocks : The number of blocks to work on. Min = 1, Max = 3
- */
-static void pipelined_crc32c(uint32_t *crc1, uint32_t *crc2, uint32_t *crc3, const uint8_t *p_buf, size_t block_size, int num_blocks) {
-  uint32_t c1 = *crc1;
-  uint32_t c2 = *crc2;
-  uint32_t c3 = *crc3;
-  int counter = block_size / sizeof(uint32_t);
-  int remainder = block_size % sizeof(uint32_t);
-  uint32_t *data = (uint32_t*)p_buf;
-  uint8_t *bdata;
-
-  /* We do switch here because the loop has to be tight in order
-   * to fill the pipeline. Any other statement inside the loop
-   * or inbetween crc32 instruction can slow things down. Calling
-   * individual crc32 instructions three times from C also causes
-   * gcc to insert other instructions inbetween.
-   *
-   * Do not rearrange the following code unless you have verified
-   * the generated machine code is as efficient as before.
-   */
-  switch (num_blocks) {
-    case 3:
-      /* Do three blocks */
-      while (likely(counter)) {
-        __asm__ __volatile__(
-        "crc32l (%7), %0;\n\t"
-        "crc32l (%7,%6,1), %1;\n\t"
-        "crc32l (%7,%6,2), %2;\n\t"
-         : "=r"(c1), "=r"(c2), "=r"(c3)
-         : "r"(c1), "r"(c2), "r"(c3), "r"(block_size), "r"(data)
-        );
-        data++;
-        counter--;
-      }
-      /* Take care of the remainder. They are only up to three bytes,
-       * so performing byte-level crc32 won't take much time.
-       */
-      bdata = (uint8_t*)data;
-      while (likely(remainder)) {
-        __asm__ __volatile__(
-        "crc32b (%7), %0;\n\t"
-        "crc32b (%7,%6,1), %1;\n\t"
-        "crc32b (%7,%6,2), %2;\n\t"
-         : "=r"(c1), "=r"(c2), "=r"(c3)
-         : "r"(c1), "r"(c2), "r"(c3), "r"(block_size), "r"(bdata)
-        );
-        bdata++;
-        remainder--;
-      }
-      break;
-    case 2:
-      /* Do two blocks */
-      while (likely(counter)) {
-        __asm__ __volatile__(
-        "crc32l (%5), %0;\n\t"
-        "crc32l (%5,%4,1), %1;\n\t"
-         : "=r"(c1), "=r"(c2) 
-         : "r"(c1), "r"(c2), "r"(block_size), "r"(data)
-        );
-        data++;
-        counter--;
-      }
-
-      bdata = (uint8_t*)data;
-      while (likely(remainder)) {
-        __asm__ __volatile__(
-        "crc32b (%5), %0;\n\t"
-        "crc32b (%5,%4,1), %1;\n\t"
-         : "=r"(c1), "=r"(c2) 
-         : "r"(c1), "r"(c2), "r"(block_size), "r"(bdata)
-        );
-        bdata++;
-        remainder--;
-      }
-      break;
-    case 1:
-      /* single block */
-      while (likely(counter)) {
-        __asm__ __volatile__(
-        "crc32l (%2), %0;\n\t"
-         : "=r"(c1) 
-         : "r"(c1), "r"(data)
-        );
-        data++;
-        counter--;
-      }
-      bdata = (uint8_t*)data;
-      while (likely(remainder)) {
-        __asm__ __volatile__(
-        "crc32b (%2), %0;\n\t"
-         : "=r"(c1) 
-         : "r"(c1), "r"(bdata)
-        );
-        bdata++;
-        remainder--;
-      }
-      break;
-    case 0:
-       return;
-    default:
-      assert(0 && "BUG: Invalid number of checksum blocks");
-  }
-
-  *crc1 = c1;
-  *crc2 = c2;
-  *crc3 = c3;
-  return;
-}
-
-# endif // 64-bit vs 32-bit
-
-/**
- * On library load, initiailize the cached function pointer
- * if cpu supports SSE4.2's crc32 instruction.
- */
-typedef void (*crc_pipelined_func_t)(uint32_t *, uint32_t *, uint32_t *, const uint8_t *, size_t, int);
-extern crc_pipelined_func_t pipelined_crc32c_func;
-
-void __attribute__ ((constructor)) init_cpu_support_flag(void) {
-  uint32_t ecx = cpuid(CPUID_FEATURES);
-  if (ecx & SSE42_FEATURE_BIT) pipelined_crc32c_func = pipelined_crc32c;
-}

http://git-wip-us.apache.org/repos/asf/incubator-ratis/blob/7e71a2e0/raft-common/src/main/native/src/org/apache/raft/util/crc32_zlib_polynomial_tables.h
----------------------------------------------------------------------
diff --git a/raft-common/src/main/native/src/org/apache/raft/util/crc32_zlib_polynomial_tables.h b/raft-common/src/main/native/src/org/apache/raft/util/crc32_zlib_polynomial_tables.h
deleted file mode 100644
index 59d8f4d..0000000
--- a/raft-common/src/main/native/src/org/apache/raft/util/crc32_zlib_polynomial_tables.h
+++ /dev/null
@@ -1,552 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- *     http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-
-/*
- * CRC-32 lookup tables generated by the polynomial 0xEDB88320.
- * See also TestPureJavaCrc32.Table.
- */
-const uint32_t CRC32_T8_0[] = {
-  0x00000000, 0x77073096, 0xEE0E612C, 0x990951BA, 
-  0x076DC419, 0x706AF48F, 0xE963A535, 0x9E6495A3, 
-  0x0EDB8832, 0x79DCB8A4, 0xE0D5E91E, 0x97D2D988, 
-  0x09B64C2B, 0x7EB17CBD, 0xE7B82D07, 0x90BF1D91, 
-  0x1DB71064, 0x6AB020F2, 0xF3B97148, 0x84BE41DE, 
-  0x1ADAD47D, 0x6DDDE4EB, 0xF4D4B551, 0x83D385C7, 
-  0x136C9856, 0x646BA8C0, 0xFD62F97A, 0x8A65C9EC, 
-  0x14015C4F, 0x63066CD9, 0xFA0F3D63, 0x8D080DF5, 
-  0x3B6E20C8, 0x4C69105E, 0xD56041E4, 0xA2677172, 
-  0x3C03E4D1, 0x4B04D447, 0xD20D85FD, 0xA50AB56B, 
-  0x35B5A8FA, 0x42B2986C, 0xDBBBC9D6, 0xACBCF940, 
-  0x32D86CE3, 0x45DF5C75, 0xDCD60DCF, 0xABD13D59, 
-  0x26D930AC, 0x51DE003A, 0xC8D75180, 0xBFD06116, 
-  0x21B4F4B5, 0x56B3C423, 0xCFBA9599, 0xB8BDA50F, 
-  0x2802B89E, 0x5F058808, 0xC60CD9B2, 0xB10BE924, 
-  0x2F6F7C87, 0x58684C11, 0xC1611DAB, 0xB6662D3D, 
-  0x76DC4190, 0x01DB7106, 0x98D220BC, 0xEFD5102A, 
-  0x71B18589, 0x06B6B51F, 0x9FBFE4A5, 0xE8B8D433, 
-  0x7807C9A2, 0x0F00F934, 0x9609A88E, 0xE10E9818, 
-  0x7F6A0DBB, 0x086D3D2D, 0x91646C97, 0xE6635C01, 
-  0x6B6B51F4, 0x1C6C6162, 0x856530D8, 0xF262004E, 
-  0x6C0695ED, 0x1B01A57B, 0x8208F4C1, 0xF50FC457, 
-  0x65B0D9C6, 0x12B7E950, 0x8BBEB8EA, 0xFCB9887C, 
-  0x62DD1DDF, 0x15DA2D49, 0x8CD37CF3, 0xFBD44C65, 
-  0x4DB26158, 0x3AB551CE, 0xA3BC0074, 0xD4BB30E2, 
-  0x4ADFA541, 0x3DD895D7, 0xA4D1C46D, 0xD3D6F4FB, 
-  0x4369E96A, 0x346ED9FC, 0xAD678846, 0xDA60B8D0, 
-  0x44042D73, 0x33031DE5, 0xAA0A4C5F, 0xDD0D7CC9, 
-  0x5005713C, 0x270241AA, 0xBE0B1010, 0xC90C2086, 
-  0x5768B525, 0x206F85B3, 0xB966D409, 0xCE61E49F, 
-  0x5EDEF90E, 0x29D9C998, 0xB0D09822, 0xC7D7A8B4, 
-  0x59B33D17, 0x2EB40D81, 0xB7BD5C3B, 0xC0BA6CAD, 
-  0xEDB88320, 0x9ABFB3B6, 0x03B6E20C, 0x74B1D29A, 
-  0xEAD54739, 0x9DD277AF, 0x04DB2615, 0x73DC1683, 
-  0xE3630B12, 0x94643B84, 0x0D6D6A3E, 0x7A6A5AA8, 
-  0xE40ECF0B, 0x9309FF9D, 0x0A00AE27, 0x7D079EB1, 
-  0xF00F9344, 0x8708A3D2, 0x1E01F268, 0x6906C2FE, 
-  0xF762575D, 0x806567CB, 0x196C3671, 0x6E6B06E7, 
-  0xFED41B76, 0x89D32BE0, 0x10DA7A5A, 0x67DD4ACC, 
-  0xF9B9DF6F, 0x8EBEEFF9, 0x17B7BE43, 0x60B08ED5, 
-  0xD6D6A3E8, 0xA1D1937E, 0x38D8C2C4, 0x4FDFF252, 
-  0xD1BB67F1, 0xA6BC5767, 0x3FB506DD, 0x48B2364B, 
-  0xD80D2BDA, 0xAF0A1B4C, 0x36034AF6, 0x41047A60, 
-  0xDF60EFC3, 0xA867DF55, 0x316E8EEF, 0x4669BE79, 
-  0xCB61B38C, 0xBC66831A, 0x256FD2A0, 0x5268E236, 
-  0xCC0C7795, 0xBB0B4703, 0x220216B9, 0x5505262F, 
-  0xC5BA3BBE, 0xB2BD0B28, 0x2BB45A92, 0x5CB36A04, 
-  0xC2D7FFA7, 0xB5D0CF31, 0x2CD99E8B, 0x5BDEAE1D, 
-  0x9B64C2B0, 0xEC63F226, 0x756AA39C, 0x026D930A, 
-  0x9C0906A9, 0xEB0E363F, 0x72076785, 0x05005713, 
-  0x95BF4A82, 0xE2B87A14, 0x7BB12BAE, 0x0CB61B38, 
-  0x92D28E9B, 0xE5D5BE0D, 0x7CDCEFB7, 0x0BDBDF21, 
-  0x86D3D2D4, 0xF1D4E242, 0x68DDB3F8, 0x1FDA836E, 
-  0x81BE16CD, 0xF6B9265B, 0x6FB077E1, 0x18B74777, 
-  0x88085AE6, 0xFF0F6A70, 0x66063BCA, 0x11010B5C, 
-  0x8F659EFF, 0xF862AE69, 0x616BFFD3, 0x166CCF45, 
-  0xA00AE278, 0xD70DD2EE, 0x4E048354, 0x3903B3C2, 
-  0xA7672661, 0xD06016F7, 0x4969474D, 0x3E6E77DB, 
-  0xAED16A4A, 0xD9D65ADC, 0x40DF0B66, 0x37D83BF0, 
-  0xA9BCAE53, 0xDEBB9EC5, 0x47B2CF7F, 0x30B5FFE9, 
-  0xBDBDF21C, 0xCABAC28A, 0x53B39330, 0x24B4A3A6, 
-  0xBAD03605, 0xCDD70693, 0x54DE5729, 0x23D967BF, 
-  0xB3667A2E, 0xC4614AB8, 0x5D681B02, 0x2A6F2B94, 
-  0xB40BBE37, 0xC30C8EA1, 0x5A05DF1B, 0x2D02EF8D
-};
-const uint32_t CRC32_T8_1[] = {
-  0x00000000, 0x191B3141, 0x32366282, 0x2B2D53C3, 
-  0x646CC504, 0x7D77F445, 0x565AA786, 0x4F4196C7, 
-  0xC8D98A08, 0xD1C2BB49, 0xFAEFE88A, 0xE3F4D9CB, 
-  0xACB54F0C, 0xB5AE7E4D, 0x9E832D8E, 0x87981CCF, 
-  0x4AC21251, 0x53D92310, 0x78F470D3, 0x61EF4192, 
-  0x2EAED755, 0x37B5E614, 0x1C98B5D7, 0x05838496, 
-  0x821B9859, 0x9B00A918, 0xB02DFADB, 0xA936CB9A, 
-  0xE6775D5D, 0xFF6C6C1C, 0xD4413FDF, 0xCD5A0E9E, 
-  0x958424A2, 0x8C9F15E3, 0xA7B24620, 0xBEA97761, 
-  0xF1E8E1A6, 0xE8F3D0E7, 0xC3DE8324, 0xDAC5B265, 
-  0x5D5DAEAA, 0x44469FEB, 0x6F6BCC28, 0x7670FD69, 
-  0x39316BAE, 0x202A5AEF, 0x0B07092C, 0x121C386D, 
-  0xDF4636F3, 0xC65D07B2, 0xED705471, 0xF46B6530, 
-  0xBB2AF3F7, 0xA231C2B6, 0x891C9175, 0x9007A034, 
-  0x179FBCFB, 0x0E848DBA, 0x25A9DE79, 0x3CB2EF38, 
-  0x73F379FF, 0x6AE848BE, 0x41C51B7D, 0x58DE2A3C, 
-  0xF0794F05, 0xE9627E44, 0xC24F2D87, 0xDB541CC6, 
-  0x94158A01, 0x8D0EBB40, 0xA623E883, 0xBF38D9C2, 
-  0x38A0C50D, 0x21BBF44C, 0x0A96A78F, 0x138D96CE, 
-  0x5CCC0009, 0x45D73148, 0x6EFA628B, 0x77E153CA, 
-  0xBABB5D54, 0xA3A06C15, 0x888D3FD6, 0x91960E97, 
-  0xDED79850, 0xC7CCA911, 0xECE1FAD2, 0xF5FACB93, 
-  0x7262D75C, 0x6B79E61D, 0x4054B5DE, 0x594F849F, 
-  0x160E1258, 0x0F152319, 0x243870DA, 0x3D23419B, 
-  0x65FD6BA7, 0x7CE65AE6, 0x57CB0925, 0x4ED03864, 
-  0x0191AEA3, 0x188A9FE2, 0x33A7CC21, 0x2ABCFD60, 
-  0xAD24E1AF, 0xB43FD0EE, 0x9F12832D, 0x8609B26C, 
-  0xC94824AB, 0xD05315EA, 0xFB7E4629, 0xE2657768, 
-  0x2F3F79F6, 0x362448B7, 0x1D091B74, 0x04122A35, 
-  0x4B53BCF2, 0x52488DB3, 0x7965DE70, 0x607EEF31, 
-  0xE7E6F3FE, 0xFEFDC2BF, 0xD5D0917C, 0xCCCBA03D, 
-  0x838A36FA, 0x9A9107BB, 0xB1BC5478, 0xA8A76539, 
-  0x3B83984B, 0x2298A90A, 0x09B5FAC9, 0x10AECB88, 
-  0x5FEF5D4F, 0x46F46C0E, 0x6DD93FCD, 0x74C20E8C, 
-  0xF35A1243, 0xEA412302, 0xC16C70C1, 0xD8774180, 
-  0x9736D747, 0x8E2DE606, 0xA500B5C5, 0xBC1B8484, 
-  0x71418A1A, 0x685ABB5B, 0x4377E898, 0x5A6CD9D9, 
-  0x152D4F1E, 0x0C367E5F, 0x271B2D9C, 0x3E001CDD, 
-  0xB9980012, 0xA0833153, 0x8BAE6290, 0x92B553D1, 
-  0xDDF4C516, 0xC4EFF457, 0xEFC2A794, 0xF6D996D5, 
-  0xAE07BCE9, 0xB71C8DA8, 0x9C31DE6B, 0x852AEF2A, 
-  0xCA6B79ED, 0xD37048AC, 0xF85D1B6F, 0xE1462A2E, 
-  0x66DE36E1, 0x7FC507A0, 0x54E85463, 0x4DF36522, 
-  0x02B2F3E5, 0x1BA9C2A4, 0x30849167, 0x299FA026, 
-  0xE4C5AEB8, 0xFDDE9FF9, 0xD6F3CC3A, 0xCFE8FD7B, 
-  0x80A96BBC, 0x99B25AFD, 0xB29F093E, 0xAB84387F, 
-  0x2C1C24B0, 0x350715F1, 0x1E2A4632, 0x07317773, 
-  0x4870E1B4, 0x516BD0F5, 0x7A468336, 0x635DB277, 
-  0xCBFAD74E, 0xD2E1E60F, 0xF9CCB5CC, 0xE0D7848D, 
-  0xAF96124A, 0xB68D230B, 0x9DA070C8, 0x84BB4189, 
-  0x03235D46, 0x1A386C07, 0x31153FC4, 0x280E0E85, 
-  0x674F9842, 0x7E54A903, 0x5579FAC0, 0x4C62CB81, 
-  0x8138C51F, 0x9823F45E, 0xB30EA79D, 0xAA1596DC, 
-  0xE554001B, 0xFC4F315A, 0xD7626299, 0xCE7953D8, 
-  0x49E14F17, 0x50FA7E56, 0x7BD72D95, 0x62CC1CD4, 
-  0x2D8D8A13, 0x3496BB52, 0x1FBBE891, 0x06A0D9D0, 
-  0x5E7EF3EC, 0x4765C2AD, 0x6C48916E, 0x7553A02F, 
-  0x3A1236E8, 0x230907A9, 0x0824546A, 0x113F652B, 
-  0x96A779E4, 0x8FBC48A5, 0xA4911B66, 0xBD8A2A27, 
-  0xF2CBBCE0, 0xEBD08DA1, 0xC0FDDE62, 0xD9E6EF23, 
-  0x14BCE1BD, 0x0DA7D0FC, 0x268A833F, 0x3F91B27E, 
-  0x70D024B9, 0x69CB15F8, 0x42E6463B, 0x5BFD777A, 
-  0xDC656BB5, 0xC57E5AF4, 0xEE530937, 0xF7483876, 
-  0xB809AEB1, 0xA1129FF0, 0x8A3FCC33, 0x9324FD72
-};
-const uint32_t CRC32_T8_2[] = {
-  0x00000000, 0x01C26A37, 0x0384D46E, 0x0246BE59, 
-  0x0709A8DC, 0x06CBC2EB, 0x048D7CB2, 0x054F1685, 
-  0x0E1351B8, 0x0FD13B8F, 0x0D9785D6, 0x0C55EFE1, 
-  0x091AF964, 0x08D89353, 0x0A9E2D0A, 0x0B5C473D, 
-  0x1C26A370, 0x1DE4C947, 0x1FA2771E, 0x1E601D29, 
-  0x1B2F0BAC, 0x1AED619B, 0x18ABDFC2, 0x1969B5F5, 
-  0x1235F2C8, 0x13F798FF, 0x11B126A6, 0x10734C91, 
-  0x153C5A14, 0x14FE3023, 0x16B88E7A, 0x177AE44D, 
-  0x384D46E0, 0x398F2CD7, 0x3BC9928E, 0x3A0BF8B9, 
-  0x3F44EE3C, 0x3E86840B, 0x3CC03A52, 0x3D025065, 
-  0x365E1758, 0x379C7D6F, 0x35DAC336, 0x3418A901, 
-  0x3157BF84, 0x3095D5B3, 0x32D36BEA, 0x331101DD, 
-  0x246BE590, 0x25A98FA7, 0x27EF31FE, 0x262D5BC9, 
-  0x23624D4C, 0x22A0277B, 0x20E69922, 0x2124F315, 
-  0x2A78B428, 0x2BBADE1F, 0x29FC6046, 0x283E0A71, 
-  0x2D711CF4, 0x2CB376C3, 0x2EF5C89A, 0x2F37A2AD, 
-  0x709A8DC0, 0x7158E7F7, 0x731E59AE, 0x72DC3399, 
-  0x7793251C, 0x76514F2B, 0x7417F172, 0x75D59B45, 
-  0x7E89DC78, 0x7F4BB64F, 0x7D0D0816, 0x7CCF6221, 
-  0x798074A4, 0x78421E93, 0x7A04A0CA, 0x7BC6CAFD, 
-  0x6CBC2EB0, 0x6D7E4487, 0x6F38FADE, 0x6EFA90E9, 
-  0x6BB5866C, 0x6A77EC5B, 0x68315202, 0x69F33835, 
-  0x62AF7F08, 0x636D153F, 0x612BAB66, 0x60E9C151, 
-  0x65A6D7D4, 0x6464BDE3, 0x662203BA, 0x67E0698D, 
-  0x48D7CB20, 0x4915A117, 0x4B531F4E, 0x4A917579, 
-  0x4FDE63FC, 0x4E1C09CB, 0x4C5AB792, 0x4D98DDA5, 
-  0x46C49A98, 0x4706F0AF, 0x45404EF6, 0x448224C1, 
-  0x41CD3244, 0x400F5873, 0x4249E62A, 0x438B8C1D, 
-  0x54F16850, 0x55330267, 0x5775BC3E, 0x56B7D609, 
-  0x53F8C08C, 0x523AAABB, 0x507C14E2, 0x51BE7ED5, 
-  0x5AE239E8, 0x5B2053DF, 0x5966ED86, 0x58A487B1, 
-  0x5DEB9134, 0x5C29FB03, 0x5E6F455A, 0x5FAD2F6D, 
-  0xE1351B80, 0xE0F771B7, 0xE2B1CFEE, 0xE373A5D9, 
-  0xE63CB35C, 0xE7FED96B, 0xE5B86732, 0xE47A0D05, 
-  0xEF264A38, 0xEEE4200F, 0xECA29E56, 0xED60F461, 
-  0xE82FE2E4, 0xE9ED88D3, 0xEBAB368A, 0xEA695CBD, 
-  0xFD13B8F0, 0xFCD1D2C7, 0xFE976C9E, 0xFF5506A9, 
-  0xFA1A102C, 0xFBD87A1B, 0xF99EC442, 0xF85CAE75, 
-  0xF300E948, 0xF2C2837F, 0xF0843D26, 0xF1465711, 
-  0xF4094194, 0xF5CB2BA3, 0xF78D95FA, 0xF64FFFCD, 
-  0xD9785D60, 0xD8BA3757, 0xDAFC890E, 0xDB3EE339, 
-  0xDE71F5BC, 0xDFB39F8B, 0xDDF521D2, 0xDC374BE5, 
-  0xD76B0CD8, 0xD6A966EF, 0xD4EFD8B6, 0xD52DB281, 
-  0xD062A404, 0xD1A0CE33, 0xD3E6706A, 0xD2241A5D, 
-  0xC55EFE10, 0xC49C9427, 0xC6DA2A7E, 0xC7184049, 
-  0xC25756CC, 0xC3953CFB, 0xC1D382A2, 0xC011E895, 
-  0xCB4DAFA8, 0xCA8FC59F, 0xC8C97BC6, 0xC90B11F1, 
-  0xCC440774, 0xCD866D43, 0xCFC0D31A, 0xCE02B92D, 
-  0x91AF9640, 0x906DFC77, 0x922B422E, 0x93E92819, 
-  0x96A63E9C, 0x976454AB, 0x9522EAF2, 0x94E080C5, 
-  0x9FBCC7F8, 0x9E7EADCF, 0x9C381396, 0x9DFA79A1, 
-  0x98B56F24, 0x99770513, 0x9B31BB4A, 0x9AF3D17D, 
-  0x8D893530, 0x8C4B5F07, 0x8E0DE15E, 0x8FCF8B69, 
-  0x8A809DEC, 0x8B42F7DB, 0x89044982, 0x88C623B5, 
-  0x839A6488, 0x82580EBF, 0x801EB0E6, 0x81DCDAD1, 
-  0x8493CC54, 0x8551A663, 0x8717183A, 0x86D5720D, 
-  0xA9E2D0A0, 0xA820BA97, 0xAA6604CE, 0xABA46EF9, 
-  0xAEEB787C, 0xAF29124B, 0xAD6FAC12, 0xACADC625, 
-  0xA7F18118, 0xA633EB2F, 0xA4755576, 0xA5B73F41, 
-  0xA0F829C4, 0xA13A43F3, 0xA37CFDAA, 0xA2BE979D, 
-  0xB5C473D0, 0xB40619E7, 0xB640A7BE, 0xB782CD89, 
-  0xB2CDDB0C, 0xB30FB13B, 0xB1490F62, 0xB08B6555, 
-  0xBBD72268, 0xBA15485F, 0xB853F606, 0xB9919C31, 
-  0xBCDE8AB4, 0xBD1CE083, 0xBF5A5EDA, 0xBE9834ED
-};
-const uint32_t CRC32_T8_3[] = {
-  0x00000000, 0xB8BC6765, 0xAA09C88B, 0x12B5AFEE, 
-  0x8F629757, 0x37DEF032, 0x256B5FDC, 0x9DD738B9, 
-  0xC5B428EF, 0x7D084F8A, 0x6FBDE064, 0xD7018701, 
-  0x4AD6BFB8, 0xF26AD8DD, 0xE0DF7733, 0x58631056, 
-  0x5019579F, 0xE8A530FA, 0xFA109F14, 0x42ACF871, 
-  0xDF7BC0C8, 0x67C7A7AD, 0x75720843, 0xCDCE6F26, 
-  0x95AD7F70, 0x2D111815, 0x3FA4B7FB, 0x8718D09E, 
-  0x1ACFE827, 0xA2738F42, 0xB0C620AC, 0x087A47C9, 
-  0xA032AF3E, 0x188EC85B, 0x0A3B67B5, 0xB28700D0, 
-  0x2F503869, 0x97EC5F0C, 0x8559F0E2, 0x3DE59787, 
-  0x658687D1, 0xDD3AE0B4, 0xCF8F4F5A, 0x7733283F, 
-  0xEAE41086, 0x525877E3, 0x40EDD80D, 0xF851BF68, 
-  0xF02BF8A1, 0x48979FC4, 0x5A22302A, 0xE29E574F, 
-  0x7F496FF6, 0xC7F50893, 0xD540A77D, 0x6DFCC018, 
-  0x359FD04E, 0x8D23B72B, 0x9F9618C5, 0x272A7FA0, 
-  0xBAFD4719, 0x0241207C, 0x10F48F92, 0xA848E8F7, 
-  0x9B14583D, 0x23A83F58, 0x311D90B6, 0x89A1F7D3, 
-  0x1476CF6A, 0xACCAA80F, 0xBE7F07E1, 0x06C36084, 
-  0x5EA070D2, 0xE61C17B7, 0xF4A9B859, 0x4C15DF3C, 
-  0xD1C2E785, 0x697E80E0, 0x7BCB2F0E, 0xC377486B, 
-  0xCB0D0FA2, 0x73B168C7, 0x6104C729, 0xD9B8A04C, 
-  0x446F98F5, 0xFCD3FF90, 0xEE66507E, 0x56DA371B, 
-  0x0EB9274D, 0xB6054028, 0xA4B0EFC6, 0x1C0C88A3, 
-  0x81DBB01A, 0x3967D77F, 0x2BD27891, 0x936E1FF4, 
-  0x3B26F703, 0x839A9066, 0x912F3F88, 0x299358ED, 
-  0xB4446054, 0x0CF80731, 0x1E4DA8DF, 0xA6F1CFBA, 
-  0xFE92DFEC, 0x462EB889, 0x549B1767, 0xEC277002, 
-  0x71F048BB, 0xC94C2FDE, 0xDBF98030, 0x6345E755, 
-  0x6B3FA09C, 0xD383C7F9, 0xC1366817, 0x798A0F72, 
-  0xE45D37CB, 0x5CE150AE, 0x4E54FF40, 0xF6E89825, 
-  0xAE8B8873, 0x1637EF16, 0x048240F8, 0xBC3E279D, 
-  0x21E91F24, 0x99557841, 0x8BE0D7AF, 0x335CB0CA, 
-  0xED59B63B, 0x55E5D15E, 0x47507EB0, 0xFFEC19D5, 
-  0x623B216C, 0xDA874609, 0xC832E9E7, 0x708E8E82, 
-  0x28ED9ED4, 0x9051F9B1, 0x82E4565F, 0x3A58313A, 
-  0xA78F0983, 0x1F336EE6, 0x0D86C108, 0xB53AA66D, 
-  0xBD40E1A4, 0x05FC86C1, 0x1749292F, 0xAFF54E4A, 
-  0x322276F3, 0x8A9E1196, 0x982BBE78, 0x2097D91D, 
-  0x78F4C94B, 0xC048AE2E, 0xD2FD01C0, 0x6A4166A5, 
-  0xF7965E1C, 0x4F2A3979, 0x5D9F9697, 0xE523F1F2, 
-  0x4D6B1905, 0xF5D77E60, 0xE762D18E, 0x5FDEB6EB, 
-  0xC2098E52, 0x7AB5E937, 0x680046D9, 0xD0BC21BC, 
-  0x88DF31EA, 0x3063568F, 0x22D6F961, 0x9A6A9E04, 
-  0x07BDA6BD, 0xBF01C1D8, 0xADB46E36, 0x15080953, 
-  0x1D724E9A, 0xA5CE29FF, 0xB77B8611, 0x0FC7E174, 
-  0x9210D9CD, 0x2AACBEA8, 0x38191146, 0x80A57623, 
-  0xD8C66675, 0x607A0110, 0x72CFAEFE, 0xCA73C99B, 
-  0x57A4F122, 0xEF189647, 0xFDAD39A9, 0x45115ECC, 
-  0x764DEE06, 0xCEF18963, 0xDC44268D, 0x64F841E8, 
-  0xF92F7951, 0x41931E34, 0x5326B1DA, 0xEB9AD6BF, 
-  0xB3F9C6E9, 0x0B45A18C, 0x19F00E62, 0xA14C6907, 
-  0x3C9B51BE, 0x842736DB, 0x96929935, 0x2E2EFE50, 
-  0x2654B999, 0x9EE8DEFC, 0x8C5D7112, 0x34E11677, 
-  0xA9362ECE, 0x118A49AB, 0x033FE645, 0xBB838120, 
-  0xE3E09176, 0x5B5CF613, 0x49E959FD, 0xF1553E98, 
-  0x6C820621, 0xD43E6144, 0xC68BCEAA, 0x7E37A9CF, 
-  0xD67F4138, 0x6EC3265D, 0x7C7689B3, 0xC4CAEED6, 
-  0x591DD66F, 0xE1A1B10A, 0xF3141EE4, 0x4BA87981, 
-  0x13CB69D7, 0xAB770EB2, 0xB9C2A15C, 0x017EC639, 
-  0x9CA9FE80, 0x241599E5, 0x36A0360B, 0x8E1C516E, 
-  0x866616A7, 0x3EDA71C2, 0x2C6FDE2C, 0x94D3B949, 
-  0x090481F0, 0xB1B8E695, 0xA30D497B, 0x1BB12E1E, 
-  0x43D23E48, 0xFB6E592D, 0xE9DBF6C3, 0x516791A6, 
-  0xCCB0A91F, 0x740CCE7A, 0x66B96194, 0xDE0506F1
-};
-const uint32_t CRC32_T8_4[] = {
-  0x00000000, 0x3D6029B0, 0x7AC05360, 0x47A07AD0, 
-  0xF580A6C0, 0xC8E08F70, 0x8F40F5A0, 0xB220DC10, 
-  0x30704BC1, 0x0D106271, 0x4AB018A1, 0x77D03111, 
-  0xC5F0ED01, 0xF890C4B1, 0xBF30BE61, 0x825097D1, 
-  0x60E09782, 0x5D80BE32, 0x1A20C4E2, 0x2740ED52, 
-  0x95603142, 0xA80018F2, 0xEFA06222, 0xD2C04B92, 
-  0x5090DC43, 0x6DF0F5F3, 0x2A508F23, 0x1730A693, 
-  0xA5107A83, 0x98705333, 0xDFD029E3, 0xE2B00053, 
-  0xC1C12F04, 0xFCA106B4, 0xBB017C64, 0x866155D4, 
-  0x344189C4, 0x0921A074, 0x4E81DAA4, 0x73E1F314, 
-  0xF1B164C5, 0xCCD14D75, 0x8B7137A5, 0xB6111E15, 
-  0x0431C205, 0x3951EBB5, 0x7EF19165, 0x4391B8D5, 
-  0xA121B886, 0x9C419136, 0xDBE1EBE6, 0xE681C256, 
-  0x54A11E46, 0x69C137F6, 0x2E614D26, 0x13016496, 
-  0x9151F347, 0xAC31DAF7, 0xEB91A027, 0xD6F18997, 
-  0x64D15587, 0x59B17C37, 0x1E1106E7, 0x23712F57, 
-  0x58F35849, 0x659371F9, 0x22330B29, 0x1F532299, 
-  0xAD73FE89, 0x9013D739, 0xD7B3ADE9, 0xEAD38459, 
-  0x68831388, 0x55E33A38, 0x124340E8, 0x2F236958, 
-  0x9D03B548, 0xA0639CF8, 0xE7C3E628, 0xDAA3CF98, 
-  0x3813CFCB, 0x0573E67B, 0x42D39CAB, 0x7FB3B51B, 
-  0xCD93690B, 0xF0F340BB, 0xB7533A6B, 0x8A3313DB, 
-  0x0863840A, 0x3503ADBA, 0x72A3D76A, 0x4FC3FEDA, 
-  0xFDE322CA, 0xC0830B7A, 0x872371AA, 0xBA43581A, 
-  0x9932774D, 0xA4525EFD, 0xE3F2242D, 0xDE920D9D, 
-  0x6CB2D18D, 0x51D2F83D, 0x167282ED, 0x2B12AB5D, 
-  0xA9423C8C, 0x9422153C, 0xD3826FEC, 0xEEE2465C, 
-  0x5CC29A4C, 0x61A2B3FC, 0x2602C92C, 0x1B62E09C, 
-  0xF9D2E0CF, 0xC4B2C97F, 0x8312B3AF, 0xBE729A1F, 
-  0x0C52460F, 0x31326FBF, 0x7692156F, 0x4BF23CDF, 
-  0xC9A2AB0E, 0xF4C282BE, 0xB362F86E, 0x8E02D1DE, 
-  0x3C220DCE, 0x0142247E, 0x46E25EAE, 0x7B82771E, 
-  0xB1E6B092, 0x8C869922, 0xCB26E3F2, 0xF646CA42, 
-  0x44661652, 0x79063FE2, 0x3EA64532, 0x03C66C82, 
-  0x8196FB53, 0xBCF6D2E3, 0xFB56A833, 0xC6368183, 
-  0x74165D93, 0x49767423, 0x0ED60EF3, 0x33B62743, 
-  0xD1062710, 0xEC660EA0, 0xABC67470, 0x96A65DC0, 
-  0x248681D0, 0x19E6A860, 0x5E46D2B0, 0x6326FB00, 
-  0xE1766CD1, 0xDC164561, 0x9BB63FB1, 0xA6D61601, 
-  0x14F6CA11, 0x2996E3A1, 0x6E369971, 0x5356B0C1, 
-  0x70279F96, 0x4D47B626, 0x0AE7CCF6, 0x3787E546, 
-  0x85A73956, 0xB8C710E6, 0xFF676A36, 0xC2074386, 
-  0x4057D457, 0x7D37FDE7, 0x3A978737, 0x07F7AE87, 
-  0xB5D77297, 0x88B75B27, 0xCF1721F7, 0xF2770847, 
-  0x10C70814, 0x2DA721A4, 0x6A075B74, 0x576772C4, 
-  0xE547AED4, 0xD8278764, 0x9F87FDB4, 0xA2E7D404, 
-  0x20B743D5, 0x1DD76A65, 0x5A7710B5, 0x67173905, 
-  0xD537E515, 0xE857CCA5, 0xAFF7B675, 0x92979FC5, 
-  0xE915E8DB, 0xD475C16B, 0x93D5BBBB, 0xAEB5920B, 
-  0x1C954E1B, 0x21F567AB, 0x66551D7B, 0x5B3534CB, 
-  0xD965A31A, 0xE4058AAA, 0xA3A5F07A, 0x9EC5D9CA, 
-  0x2CE505DA, 0x11852C6A, 0x562556BA, 0x6B457F0A, 
-  0x89F57F59, 0xB49556E9, 0xF3352C39, 0xCE550589, 
-  0x7C75D999, 0x4115F029, 0x06B58AF9, 0x3BD5A349, 
-  0xB9853498, 0x84E51D28, 0xC34567F8, 0xFE254E48, 
-  0x4C059258, 0x7165BBE8, 0x36C5C138, 0x0BA5E888, 
-  0x28D4C7DF, 0x15B4EE6F, 0x521494BF, 0x6F74BD0F, 
-  0xDD54611F, 0xE03448AF, 0xA794327F, 0x9AF41BCF, 
-  0x18A48C1E, 0x25C4A5AE, 0x6264DF7E, 0x5F04F6CE, 
-  0xED242ADE, 0xD044036E, 0x97E479BE, 0xAA84500E, 
-  0x4834505D, 0x755479ED, 0x32F4033D, 0x0F942A8D, 
-  0xBDB4F69D, 0x80D4DF2D, 0xC774A5FD, 0xFA148C4D, 
-  0x78441B9C, 0x4524322C, 0x028448FC, 0x3FE4614C, 
-  0x8DC4BD5C, 0xB0A494EC, 0xF704EE3C, 0xCA64C78C
-};
-const uint32_t CRC32_T8_5[] = {
-  0x00000000, 0xCB5CD3A5, 0x4DC8A10B, 0x869472AE, 
-  0x9B914216, 0x50CD91B3, 0xD659E31D, 0x1D0530B8, 
-  0xEC53826D, 0x270F51C8, 0xA19B2366, 0x6AC7F0C3, 
-  0x77C2C07B, 0xBC9E13DE, 0x3A0A6170, 0xF156B2D5, 
-  0x03D6029B, 0xC88AD13E, 0x4E1EA390, 0x85427035, 
-  0x9847408D, 0x531B9328, 0xD58FE186, 0x1ED33223, 
-  0xEF8580F6, 0x24D95353, 0xA24D21FD, 0x6911F258, 
-  0x7414C2E0, 0xBF481145, 0x39DC63EB, 0xF280B04E, 
-  0x07AC0536, 0xCCF0D693, 0x4A64A43D, 0x81387798, 
-  0x9C3D4720, 0x57619485, 0xD1F5E62B, 0x1AA9358E, 
-  0xEBFF875B, 0x20A354FE, 0xA6372650, 0x6D6BF5F5, 
-  0x706EC54D, 0xBB3216E8, 0x3DA66446, 0xF6FAB7E3, 
-  0x047A07AD, 0xCF26D408, 0x49B2A6A6, 0x82EE7503, 
-  0x9FEB45BB, 0x54B7961E, 0xD223E4B0, 0x197F3715, 
-  0xE82985C0, 0x23755665, 0xA5E124CB, 0x6EBDF76E, 
-  0x73B8C7D6, 0xB8E41473, 0x3E7066DD, 0xF52CB578, 
-  0x0F580A6C, 0xC404D9C9, 0x4290AB67, 0x89CC78C2, 
-  0x94C9487A, 0x5F959BDF, 0xD901E971, 0x125D3AD4, 
-  0xE30B8801, 0x28575BA4, 0xAEC3290A, 0x659FFAAF, 
-  0x789ACA17, 0xB3C619B2, 0x35526B1C, 0xFE0EB8B9, 
-  0x0C8E08F7, 0xC7D2DB52, 0x4146A9FC, 0x8A1A7A59, 
-  0x971F4AE1, 0x5C439944, 0xDAD7EBEA, 0x118B384F, 
-  0xE0DD8A9A, 0x2B81593F, 0xAD152B91, 0x6649F834, 
-  0x7B4CC88C, 0xB0101B29, 0x36846987, 0xFDD8BA22, 
-  0x08F40F5A, 0xC3A8DCFF, 0x453CAE51, 0x8E607DF4, 
-  0x93654D4C, 0x58399EE9, 0xDEADEC47, 0x15F13FE2, 
-  0xE4A78D37, 0x2FFB5E92, 0xA96F2C3C, 0x6233FF99, 
-  0x7F36CF21, 0xB46A1C84, 0x32FE6E2A, 0xF9A2BD8F, 
-  0x0B220DC1, 0xC07EDE64, 0x46EAACCA, 0x8DB67F6F, 
-  0x90B34FD7, 0x5BEF9C72, 0xDD7BEEDC, 0x16273D79, 
-  0xE7718FAC, 0x2C2D5C09, 0xAAB92EA7, 0x61E5FD02, 
-  0x7CE0CDBA, 0xB7BC1E1F, 0x31286CB1, 0xFA74BF14, 
-  0x1EB014D8, 0xD5ECC77D, 0x5378B5D3, 0x98246676, 
-  0x852156CE, 0x4E7D856B, 0xC8E9F7C5, 0x03B52460, 
-  0xF2E396B5, 0x39BF4510, 0xBF2B37BE, 0x7477E41B, 
-  0x6972D4A3, 0xA22E0706, 0x24BA75A8, 0xEFE6A60D, 
-  0x1D661643, 0xD63AC5E6, 0x50AEB748, 0x9BF264ED, 
-  0x86F75455, 0x4DAB87F0, 0xCB3FF55E, 0x006326FB, 
-  0xF135942E, 0x3A69478B, 0xBCFD3525, 0x77A1E680, 
-  0x6AA4D638, 0xA1F8059D, 0x276C7733, 0xEC30A496, 
-  0x191C11EE, 0xD240C24B, 0x54D4B0E5, 0x9F886340, 
-  0x828D53F8, 0x49D1805D, 0xCF45F2F3, 0x04192156, 
-  0xF54F9383, 0x3E134026, 0xB8873288, 0x73DBE12D, 
-  0x6EDED195, 0xA5820230, 0x2316709E, 0xE84AA33B, 
-  0x1ACA1375, 0xD196C0D0, 0x5702B27E, 0x9C5E61DB, 
-  0x815B5163, 0x4A0782C6, 0xCC93F068, 0x07CF23CD, 
-  0xF6999118, 0x3DC542BD, 0xBB513013, 0x700DE3B6, 
-  0x6D08D30E, 0xA65400AB, 0x20C07205, 0xEB9CA1A0, 
-  0x11E81EB4, 0xDAB4CD11, 0x5C20BFBF, 0x977C6C1A, 
-  0x8A795CA2, 0x41258F07, 0xC7B1FDA9, 0x0CED2E0C, 
-  0xFDBB9CD9, 0x36E74F7C, 0xB0733DD2, 0x7B2FEE77, 
-  0x662ADECF, 0xAD760D6A, 0x2BE27FC4, 0xE0BEAC61, 
-  0x123E1C2F, 0xD962CF8A, 0x5FF6BD24, 0x94AA6E81, 
-  0x89AF5E39, 0x42F38D9C, 0xC467FF32, 0x0F3B2C97, 
-  0xFE6D9E42, 0x35314DE7, 0xB3A53F49, 0x78F9ECEC, 
-  0x65FCDC54, 0xAEA00FF1, 0x28347D5F, 0xE368AEFA, 
-  0x16441B82, 0xDD18C827, 0x5B8CBA89, 0x90D0692C, 
-  0x8DD55994, 0x46898A31, 0xC01DF89F, 0x0B412B3A, 
-  0xFA1799EF, 0x314B4A4A, 0xB7DF38E4, 0x7C83EB41, 
-  0x6186DBF9, 0xAADA085C, 0x2C4E7AF2, 0xE712A957, 
-  0x15921919, 0xDECECABC, 0x585AB812, 0x93066BB7, 
-  0x8E035B0F, 0x455F88AA, 0xC3CBFA04, 0x089729A1, 
-  0xF9C19B74, 0x329D48D1, 0xB4093A7F, 0x7F55E9DA, 
-  0x6250D962, 0xA90C0AC7, 0x2F987869, 0xE4C4ABCC
-};
-const uint32_t CRC32_T8_6[] = {
-  0x00000000, 0xA6770BB4, 0x979F1129, 0x31E81A9D, 
-  0xF44F2413, 0x52382FA7, 0x63D0353A, 0xC5A73E8E, 
-  0x33EF4E67, 0x959845D3, 0xA4705F4E, 0x020754FA, 
-  0xC7A06A74, 0x61D761C0, 0x503F7B5D, 0xF64870E9, 
-  0x67DE9CCE, 0xC1A9977A, 0xF0418DE7, 0x56368653, 
-  0x9391B8DD, 0x35E6B369, 0x040EA9F4, 0xA279A240, 
-  0x5431D2A9, 0xF246D91D, 0xC3AEC380, 0x65D9C834, 
-  0xA07EF6BA, 0x0609FD0E, 0x37E1E793, 0x9196EC27, 
-  0xCFBD399C, 0x69CA3228, 0x582228B5, 0xFE552301, 
-  0x3BF21D8F, 0x9D85163B, 0xAC6D0CA6, 0x0A1A0712, 
-  0xFC5277FB, 0x5A257C4F, 0x6BCD66D2, 0xCDBA6D66, 
-  0x081D53E8, 0xAE6A585C, 0x9F8242C1, 0x39F54975, 
-  0xA863A552, 0x0E14AEE6, 0x3FFCB47B, 0x998BBFCF, 
-  0x5C2C8141, 0xFA5B8AF5, 0xCBB39068, 0x6DC49BDC, 
-  0x9B8CEB35, 0x3DFBE081, 0x0C13FA1C, 0xAA64F1A8, 
-  0x6FC3CF26, 0xC9B4C492, 0xF85CDE0F, 0x5E2BD5BB, 
-  0x440B7579, 0xE27C7ECD, 0xD3946450, 0x75E36FE4, 
-  0xB044516A, 0x16335ADE, 0x27DB4043, 0x81AC4BF7, 
-  0x77E43B1E, 0xD19330AA, 0xE07B2A37, 0x460C2183, 
-  0x83AB1F0D, 0x25DC14B9, 0x14340E24, 0xB2430590, 
-  0x23D5E9B7, 0x85A2E203, 0xB44AF89E, 0x123DF32A, 
-  0xD79ACDA4, 0x71EDC610, 0x4005DC8D, 0xE672D739, 
-  0x103AA7D0, 0xB64DAC64, 0x87A5B6F9, 0x21D2BD4D, 
-  0xE47583C3, 0x42028877, 0x73EA92EA, 0xD59D995E, 
-  0x8BB64CE5, 0x2DC14751, 0x1C295DCC, 0xBA5E5678, 
-  0x7FF968F6, 0xD98E6342, 0xE86679DF, 0x4E11726B, 
-  0xB8590282, 0x1E2E0936, 0x2FC613AB, 0x89B1181F, 
-  0x4C162691, 0xEA612D25, 0xDB8937B8, 0x7DFE3C0C, 
-  0xEC68D02B, 0x4A1FDB9F, 0x7BF7C102, 0xDD80CAB6, 
-  0x1827F438, 0xBE50FF8C, 0x8FB8E511, 0x29CFEEA5, 
-  0xDF879E4C, 0x79F095F8, 0x48188F65, 0xEE6F84D1, 
-  0x2BC8BA5F, 0x8DBFB1EB, 0xBC57AB76, 0x1A20A0C2, 
-  0x8816EAF2, 0x2E61E146, 0x1F89FBDB, 0xB9FEF06F, 
-  0x7C59CEE1, 0xDA2EC555, 0xEBC6DFC8, 0x4DB1D47C, 
-  0xBBF9A495, 0x1D8EAF21, 0x2C66B5BC, 0x8A11BE08, 
-  0x4FB68086, 0xE9C18B32, 0xD82991AF, 0x7E5E9A1B, 
-  0xEFC8763C, 0x49BF7D88, 0x78576715, 0xDE206CA1, 
-  0x1B87522F, 0xBDF0599B, 0x8C184306, 0x2A6F48B2, 
-  0xDC27385B, 0x7A5033EF, 0x4BB82972, 0xEDCF22C6, 
-  0x28681C48, 0x8E1F17FC, 0xBFF70D61, 0x198006D5, 
-  0x47ABD36E, 0xE1DCD8DA, 0xD034C247, 0x7643C9F3, 
-  0xB3E4F77D, 0x1593FCC9, 0x247BE654, 0x820CEDE0, 
-  0x74449D09, 0xD23396BD, 0xE3DB8C20, 0x45AC8794, 
-  0x800BB91A, 0x267CB2AE, 0x1794A833, 0xB1E3A387, 
-  0x20754FA0, 0x86024414, 0xB7EA5E89, 0x119D553D, 
-  0xD43A6BB3, 0x724D6007, 0x43A57A9A, 0xE5D2712E, 
-  0x139A01C7, 0xB5ED0A73, 0x840510EE, 0x22721B5A, 
-  0xE7D525D4, 0x41A22E60, 0x704A34FD, 0xD63D3F49, 
-  0xCC1D9F8B, 0x6A6A943F, 0x5B828EA2, 0xFDF58516, 
-  0x3852BB98, 0x9E25B02C, 0xAFCDAAB1, 0x09BAA105, 
-  0xFFF2D1EC, 0x5985DA58, 0x686DC0C5, 0xCE1ACB71, 
-  0x0BBDF5FF, 0xADCAFE4B, 0x9C22E4D6, 0x3A55EF62, 
-  0xABC30345, 0x0DB408F1, 0x3C5C126C, 0x9A2B19D8, 
-  0x5F8C2756, 0xF9FB2CE2, 0xC813367F, 0x6E643DCB, 
-  0x982C4D22, 0x3E5B4696, 0x0FB35C0B, 0xA9C457BF, 
-  0x6C636931, 0xCA146285, 0xFBFC7818, 0x5D8B73AC, 
-  0x03A0A617, 0xA5D7ADA3, 0x943FB73E, 0x3248BC8A, 
-  0xF7EF8204, 0x519889B0, 0x6070932D, 0xC6079899, 
-  0x304FE870, 0x9638E3C4, 0xA7D0F959, 0x01A7F2ED, 
-  0xC400CC63, 0x6277C7D7, 0x539FDD4A, 0xF5E8D6FE, 
-  0x647E3AD9, 0xC209316D, 0xF3E12BF0, 0x55962044, 
-  0x90311ECA, 0x3646157E, 0x07AE0FE3, 0xA1D90457, 
-  0x579174BE, 0xF1E67F0A, 0xC00E6597, 0x66796E23, 
-  0xA3DE50AD, 0x05A95B19, 0x34414184, 0x92364A30
-};
-const uint32_t CRC32_T8_7[] = {
-  0x00000000, 0xCCAA009E, 0x4225077D, 0x8E8F07E3, 
-  0x844A0EFA, 0x48E00E64, 0xC66F0987, 0x0AC50919, 
-  0xD3E51BB5, 0x1F4F1B2B, 0x91C01CC8, 0x5D6A1C56, 
-  0x57AF154F, 0x9B0515D1, 0x158A1232, 0xD92012AC, 
-  0x7CBB312B, 0xB01131B5, 0x3E9E3656, 0xF23436C8, 
-  0xF8F13FD1, 0x345B3F4F, 0xBAD438AC, 0x767E3832, 
-  0xAF5E2A9E, 0x63F42A00, 0xED7B2DE3, 0x21D12D7D, 
-  0x2B142464, 0xE7BE24FA, 0x69312319, 0xA59B2387, 
-  0xF9766256, 0x35DC62C8, 0xBB53652B, 0x77F965B5, 
-  0x7D3C6CAC, 0xB1966C32, 0x3F196BD1, 0xF3B36B4F, 
-  0x2A9379E3, 0xE639797D, 0x68B67E9E, 0xA41C7E00, 
-  0xAED97719, 0x62737787, 0xECFC7064, 0x205670FA, 
-  0x85CD537D, 0x496753E3, 0xC7E85400, 0x0B42549E, 
-  0x01875D87, 0xCD2D5D19, 0x43A25AFA, 0x8F085A64, 
-  0x562848C8, 0x9A824856, 0x140D4FB5, 0xD8A74F2B, 
-  0xD2624632, 0x1EC846AC, 0x9047414F, 0x5CED41D1, 
-  0x299DC2ED, 0xE537C273, 0x6BB8C590, 0xA712C50E, 
-  0xADD7CC17, 0x617DCC89, 0xEFF2CB6A, 0x2358CBF4, 
-  0xFA78D958, 0x36D2D9C6, 0xB85DDE25, 0x74F7DEBB, 
-  0x7E32D7A2, 0xB298D73C, 0x3C17D0DF, 0xF0BDD041, 
-  0x5526F3C6, 0x998CF358, 0x1703F4BB, 0xDBA9F425, 
-  0xD16CFD3C, 0x1DC6FDA2, 0x9349FA41, 0x5FE3FADF, 
-  0x86C3E873, 0x4A69E8ED, 0xC4E6EF0E, 0x084CEF90, 
-  0x0289E689, 0xCE23E617, 0x40ACE1F4, 0x8C06E16A, 
-  0xD0EBA0BB, 0x1C41A025, 0x92CEA7C6, 0x5E64A758, 
-  0x54A1AE41, 0x980BAEDF, 0x1684A93C, 0xDA2EA9A2, 
-  0x030EBB0E, 0xCFA4BB90, 0x412BBC73, 0x8D81BCED, 
-  0x8744B5F4, 0x4BEEB56A, 0xC561B289, 0x09CBB217, 
-  0xAC509190, 0x60FA910E, 0xEE7596ED, 0x22DF9673, 
-  0x281A9F6A, 0xE4B09FF4, 0x6A3F9817, 0xA6959889, 
-  0x7FB58A25, 0xB31F8ABB, 0x3D908D58, 0xF13A8DC6, 
-  0xFBFF84DF, 0x37558441, 0xB9DA83A2, 0x7570833C, 
-  0x533B85DA, 0x9F918544, 0x111E82A7, 0xDDB48239, 
-  0xD7718B20, 0x1BDB8BBE, 0x95548C5D, 0x59FE8CC3, 
-  0x80DE9E6F, 0x4C749EF1, 0xC2FB9912, 0x0E51998C, 
-  0x04949095, 0xC83E900B, 0x46B197E8, 0x8A1B9776, 
-  0x2F80B4F1, 0xE32AB46F, 0x6DA5B38C, 0xA10FB312, 
-  0xABCABA0B, 0x6760BA95, 0xE9EFBD76, 0x2545BDE8, 
-  0xFC65AF44, 0x30CFAFDA, 0xBE40A839, 0x72EAA8A7, 
-  0x782FA1BE, 0xB485A120, 0x3A0AA6C3, 0xF6A0A65D, 
-  0xAA4DE78C, 0x66E7E712, 0xE868E0F1, 0x24C2E06F, 
-  0x2E07E976, 0xE2ADE9E8, 0x6C22EE0B, 0xA088EE95, 
-  0x79A8FC39, 0xB502FCA7, 0x3B8DFB44, 0xF727FBDA, 
-  0xFDE2F2C3, 0x3148F25D, 0xBFC7F5BE, 0x736DF520, 
-  0xD6F6D6A7, 0x1A5CD639, 0x94D3D1DA, 0x5879D144, 
-  0x52BCD85D, 0x9E16D8C3, 0x1099DF20, 0xDC33DFBE, 
-  0x0513CD12, 0xC9B9CD8C, 0x4736CA6F, 0x8B9CCAF1, 
-  0x8159C3E8, 0x4DF3C376, 0xC37CC495, 0x0FD6C40B, 
-  0x7AA64737, 0xB60C47A9, 0x3883404A, 0xF42940D4, 
-  0xFEEC49CD, 0x32464953, 0xBCC94EB0, 0x70634E2E, 
-  0xA9435C82, 0x65E95C1C, 0xEB665BFF, 0x27CC5B61, 
-  0x2D095278, 0xE1A352E6, 0x6F2C5505, 0xA386559B, 
-  0x061D761C, 0xCAB77682, 0x44387161, 0x889271FF, 
-  0x825778E6, 0x4EFD7878, 0xC0727F9B, 0x0CD87F05, 
-  0xD5F86DA9, 0x19526D37, 0x97DD6AD4, 0x5B776A4A, 
-  0x51B26353, 0x9D1863CD, 0x1397642E, 0xDF3D64B0, 
-  0x83D02561, 0x4F7A25FF, 0xC1F5221C, 0x0D5F2282, 
-  0x079A2B9B, 0xCB302B05, 0x45BF2CE6, 0x89152C78, 
-  0x50353ED4, 0x9C9F3E4A, 0x121039A9, 0xDEBA3937, 
-  0xD47F302E, 0x18D530B0, 0x965A3753, 0x5AF037CD, 
-  0xFF6B144A, 0x33C114D4, 0xBD4E1337, 0x71E413A9, 
-  0x7B211AB0, 0xB78B1A2E, 0x39041DCD, 0xF5AE1D53, 
-  0x2C8E0FFF, 0xE0240F61, 0x6EAB0882, 0xA201081C, 
-  0xA8C40105, 0x646E019B, 0xEAE10678, 0x264B06E6
-};
-
-



[50/54] [abbrv] incubator-ratis git commit: Add a Builder for RaftClient.

Posted by ji...@apache.org.
Add a Builder for RaftClient.


Project: http://git-wip-us.apache.org/repos/asf/incubator-ratis/repo
Commit: http://git-wip-us.apache.org/repos/asf/incubator-ratis/commit/336874fa
Tree: http://git-wip-us.apache.org/repos/asf/incubator-ratis/tree/336874fa
Diff: http://git-wip-us.apache.org/repos/asf/incubator-ratis/diff/336874fa

Branch: refs/heads/master
Commit: 336874fa36cd040df2d698bfc1030983ab3d0331
Parents: 7e71a2e
Author: Tsz-Wo Nicholas Sze <sz...@hortonworks.com>
Authored: Fri Jan 13 09:48:27 2017 +0800
Committer: Tsz-Wo Nicholas Sze <sz...@hortonworks.com>
Committed: Fri Jan 13 09:48:27 2017 +0800

----------------------------------------------------------------------
 .../org/apache/ratis/client/RaftClient.java     | 76 +++++++++++++++++++-
 .../ratis/client/impl/ClientImplUtils.java      | 33 +++++++++
 .../ratis/client/impl/RaftClientImpl.java       | 21 ++----
 .../java/org/apache/ratis/MiniRaftCluster.java  | 15 ++--
 .../ratis/RaftNotLeaderExceptionBaseTest.java   |  9 ++-
 .../impl/RaftReconfigurationBaseTest.java       | 41 +++++------
 6 files changed, 145 insertions(+), 50 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-ratis/blob/336874fa/ratis-client/src/main/java/org/apache/ratis/client/RaftClient.java
----------------------------------------------------------------------
diff --git a/ratis-client/src/main/java/org/apache/ratis/client/RaftClient.java b/ratis-client/src/main/java/org/apache/ratis/client/RaftClient.java
index e4e0b84..72dc1ca 100644
--- a/ratis-client/src/main/java/org/apache/ratis/client/RaftClient.java
+++ b/ratis-client/src/main/java/org/apache/ratis/client/RaftClient.java
@@ -17,6 +17,9 @@
  */
 package org.apache.ratis.client;
 
+import com.google.common.base.Preconditions;
+import org.apache.ratis.client.impl.ClientImplUtils;
+import org.apache.ratis.conf.RaftProperties;
 import org.apache.ratis.protocol.Message;
 import org.apache.ratis.protocol.RaftClientReply;
 import org.apache.ratis.protocol.RaftPeer;
@@ -25,6 +28,8 @@ import org.slf4j.LoggerFactory;
 
 import java.io.Closeable;
 import java.io.IOException;
+import java.util.Collection;
+import java.util.concurrent.atomic.AtomicInteger;
 
 /** A client who sends requests to a raft service. */
 public interface RaftClient extends Closeable {
@@ -34,6 +39,9 @@ public interface RaftClient extends Closeable {
   /** @return the id of this client. */
   String getId();
 
+  /** @return the request sender of this client. */
+  RaftClientRequestSender getRequestSender();
+
   /**
    * Send the given message to the raft service.
    * The message may change the state of the service.
@@ -45,5 +53,71 @@ public interface RaftClient extends Closeable {
   RaftClientReply sendReadOnly(Message message) throws IOException;
 
   /** Send set configuration request to the raft service. */
-  RaftClientReply setConfiguration(RaftPeer[] peersInNewConf) throws IOException;
+  RaftClientReply setConfiguration(RaftPeer[] serversInNewConf) throws IOException;
+
+  /** @return a {@link Builder}. */
+  static Builder newBuilder() {
+    return new Builder();
+  }
+
+  /** To build {@link RaftClient} objects. */
+  class Builder {
+    private static final AtomicInteger COUNT = new AtomicInteger();
+
+    private String clientId = RaftClient.class.getSimpleName() + COUNT.incrementAndGet();
+    private RaftClientRequestSender requestSender;
+    private Collection<RaftPeer> servers;
+    private String leaderId;
+    private RaftProperties properties;
+    private int retryInterval = RaftClientConfigKeys.RAFT_RPC_TIMEOUT_MS_DEFAULT;
+
+    private Builder() {}
+
+    /** @return a {@link RaftClient} object. */
+    public RaftClient build() {
+      Preconditions.checkNotNull(requestSender);
+      Preconditions.checkNotNull(servers);
+
+      if (leaderId == null) {
+        leaderId = servers.iterator().next().getId(); //use the first peer
+      }
+      if (properties != null) {
+        retryInterval = properties.getInt(
+            RaftClientConfigKeys.RAFT_RPC_TIMEOUT_MS_KEY,
+            RaftClientConfigKeys.RAFT_RPC_TIMEOUT_MS_DEFAULT);
+      }
+      return ClientImplUtils.newRaftClient(clientId, servers, leaderId,
+          requestSender, retryInterval);
+    }
+
+    /** Set {@link RaftClient} ID. */
+    public Builder setClientId(String clientId) {
+      this.clientId = clientId;
+      return this;
+    }
+
+    /** Set servers. */
+    public Builder setServers(Collection<RaftPeer> servers) {
+      this.servers = servers;
+      return this;
+    }
+
+    /** Set leader ID. */
+    public Builder setLeaderId(String leaderId) {
+      this.leaderId = leaderId;
+      return this;
+    }
+
+    /** Set {@link RaftClientRequestSender}. */
+    public Builder setRequestSender(RaftClientRequestSender requestSender) {
+      this.requestSender = requestSender;
+      return this;
+    }
+
+    /** Set {@link RaftProperties}. */
+    public Builder setProperties(RaftProperties properties) {
+      this.properties = properties;
+      return this;
+    }
+  }
 }

http://git-wip-us.apache.org/repos/asf/incubator-ratis/blob/336874fa/ratis-client/src/main/java/org/apache/ratis/client/impl/ClientImplUtils.java
----------------------------------------------------------------------
diff --git a/ratis-client/src/main/java/org/apache/ratis/client/impl/ClientImplUtils.java b/ratis-client/src/main/java/org/apache/ratis/client/impl/ClientImplUtils.java
new file mode 100644
index 0000000..472c8b4
--- /dev/null
+++ b/ratis-client/src/main/java/org/apache/ratis/client/impl/ClientImplUtils.java
@@ -0,0 +1,33 @@
+/**
+ * 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.ratis.client.impl;
+
+import org.apache.ratis.client.RaftClient;
+import org.apache.ratis.client.RaftClientRequestSender;
+import org.apache.ratis.protocol.RaftPeer;
+
+import java.util.Collection;
+
+/** Utilities for the client implementation. */
+public class ClientImplUtils {
+  public static RaftClient newRaftClient(
+      String clientId, Collection< RaftPeer > peers, String leaderId,
+      RaftClientRequestSender requestSender, int retryInterval) {
+    return new RaftClientImpl(clientId, peers, leaderId, requestSender, retryInterval);
+  }
+}

http://git-wip-us.apache.org/repos/asf/incubator-ratis/blob/336874fa/ratis-client/src/main/java/org/apache/ratis/client/impl/RaftClientImpl.java
----------------------------------------------------------------------
diff --git a/ratis-client/src/main/java/org/apache/ratis/client/impl/RaftClientImpl.java b/ratis-client/src/main/java/org/apache/ratis/client/impl/RaftClientImpl.java
index e101554..2f5d450 100644
--- a/ratis-client/src/main/java/org/apache/ratis/client/impl/RaftClientImpl.java
+++ b/ratis-client/src/main/java/org/apache/ratis/client/impl/RaftClientImpl.java
@@ -17,12 +17,8 @@
  */
 package org.apache.ratis.client.impl;
 
-import com.google.common.annotations.VisibleForTesting;
-
 import org.apache.ratis.client.RaftClient;
-import org.apache.ratis.client.RaftClientConfigKeys;
 import org.apache.ratis.client.RaftClientRequestSender;
-import org.apache.ratis.conf.RaftProperties;
 import org.apache.ratis.protocol.*;
 import org.apache.ratis.util.RaftUtils;
 import org.apache.ratis.util.StringUtils;
@@ -37,9 +33,7 @@ import java.util.function.Supplier;
 import java.util.stream.Collectors;
 
 /** A client who sends requests to a raft service. */
-public final class RaftClientImpl implements RaftClient {
-  public static final long DEFAULT_SEQNUM = 0;
-
+final class RaftClientImpl implements RaftClient {
   private final String clientId;
   private final RaftClientRequestSender requestSender;
   private final Map<String, RaftPeer> peers;
@@ -47,18 +41,15 @@ public final class RaftClientImpl implements RaftClient {
 
   private volatile String leaderId;
 
-  public RaftClientImpl(
-      String clientId, Collection<RaftPeer> peers,
-      RaftClientRequestSender requestSender, String leaderId,
-      RaftProperties properties) {
+  RaftClientImpl(
+      String clientId, Collection<RaftPeer> peers, String leaderId,
+      RaftClientRequestSender requestSender, int retryInterval) {
     this.clientId = clientId;
     this.requestSender = requestSender;
     this.peers = peers.stream().collect(
         Collectors.toMap(RaftPeer::getId, Function.identity()));
     this.leaderId = leaderId != null? leaderId : peers.iterator().next().getId();
-    this.retryInterval = properties.getInt(
-        RaftClientConfigKeys.RAFT_RPC_TIMEOUT_MS_KEY,
-        RaftClientConfigKeys.RAFT_RPC_TIMEOUT_MS_DEFAULT);
+    this.retryInterval = retryInterval;
   }
 
   @Override
@@ -160,7 +151,7 @@ public final class RaftClientImpl implements RaftClient {
     }
   }
 
-  @VisibleForTesting
+  @Override
   public RaftClientRequestSender getRequestSender() {
     return requestSender;
   }

http://git-wip-us.apache.org/repos/asf/incubator-ratis/blob/336874fa/ratis-server/src/test/java/org/apache/ratis/MiniRaftCluster.java
----------------------------------------------------------------------
diff --git a/ratis-server/src/test/java/org/apache/ratis/MiniRaftCluster.java b/ratis-server/src/test/java/org/apache/ratis/MiniRaftCluster.java
index 60cbb9c..7eb6ddb 100644
--- a/ratis-server/src/test/java/org/apache/ratis/MiniRaftCluster.java
+++ b/ratis-server/src/test/java/org/apache/ratis/MiniRaftCluster.java
@@ -18,10 +18,8 @@
 package org.apache.ratis;
 
 import com.google.common.base.Preconditions;
-
 import org.apache.ratis.client.RaftClient;
 import org.apache.ratis.client.RaftClientRequestSender;
-import org.apache.ratis.client.impl.RaftClientImpl;
 import org.apache.ratis.conf.RaftProperties;
 import org.apache.ratis.protocol.RaftPeer;
 import org.apache.ratis.server.RaftServerConfigKeys;
@@ -40,13 +38,13 @@ import org.junit.Assert;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
-import static org.apache.ratis.server.RaftServerConfigKeys.RAFT_SERVER_RPC_TIMEOUT_MIN_MS_DEFAULT;
-
 import java.io.File;
 import java.io.IOException;
 import java.util.*;
 import java.util.stream.Collectors;
 
+import static org.apache.ratis.server.RaftServerConfigKeys.RAFT_SERVER_RPC_TIMEOUT_MIN_MS_DEFAULT;
+
 public abstract class MiniRaftCluster {
   public static final Logger LOG = LoggerFactory.getLogger(MiniRaftCluster.class);
   public static final DelayLocalExecutionInjection logSyncDelay =
@@ -393,8 +391,13 @@ public abstract class MiniRaftCluster {
   }
 
   public RaftClient createClient(String clientId, String leaderId) {
-    return new RaftClientImpl(clientId, conf.getPeers(),
-        getRaftClientRequestSender(), leaderId, properties);
+    return RaftClient.newBuilder()
+        .setClientId(clientId)
+        .setServers(conf.getPeers())
+        .setLeaderId(leaderId)
+        .setRequestSender(getRaftClientRequestSender())
+        .setProperties(properties)
+        .build();
   }
 
   public void shutdown() {

http://git-wip-us.apache.org/repos/asf/incubator-ratis/blob/336874fa/ratis-server/src/test/java/org/apache/ratis/RaftNotLeaderExceptionBaseTest.java
----------------------------------------------------------------------
diff --git a/ratis-server/src/test/java/org/apache/ratis/RaftNotLeaderExceptionBaseTest.java b/ratis-server/src/test/java/org/apache/ratis/RaftNotLeaderExceptionBaseTest.java
index 6d25835..3ac3bf4 100644
--- a/ratis-server/src/test/java/org/apache/ratis/RaftNotLeaderExceptionBaseTest.java
+++ b/ratis-server/src/test/java/org/apache/ratis/RaftNotLeaderExceptionBaseTest.java
@@ -21,7 +21,6 @@ import org.apache.log4j.Level;
 import org.apache.ratis.RaftTestUtil.SimpleMessage;
 import org.apache.ratis.client.RaftClient;
 import org.apache.ratis.client.RaftClientRequestSender;
-import org.apache.ratis.client.impl.RaftClientImpl;
 import org.apache.ratis.protocol.RaftClientReply;
 import org.apache.ratis.protocol.RaftClientRequest;
 import org.apache.ratis.protocol.RaftPeer;
@@ -34,12 +33,12 @@ import org.junit.rules.Timeout;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
-import static org.apache.ratis.server.impl.RaftServerConstants.DEFAULT_SEQNUM;
-
 import java.io.IOException;
 import java.util.Arrays;
 import java.util.Collection;
 
+import static org.apache.ratis.server.impl.RaftServerConstants.DEFAULT_SEQNUM;
+
 public abstract class RaftNotLeaderExceptionBaseTest {
   static {
     RaftUtils.setLogLevel(RaftServerImpl.LOG, Level.DEBUG);
@@ -85,7 +84,7 @@ public abstract class RaftNotLeaderExceptionBaseTest {
     String newLeader = RaftTestUtil.changeLeader(cluster, leaderId);
     Assert.assertNotEquals(leaderId, newLeader);
 
-    RaftClientRequestSender rpc = ((RaftClientImpl)client).getRequestSender();
+    RaftClientRequestSender rpc = client.getRequestSender();
     reply= null;
     for (int i = 0; reply == null && i < 10; i++) {
       try {
@@ -131,7 +130,7 @@ public abstract class RaftNotLeaderExceptionBaseTest {
     }
     LOG.info(cluster.printServers());
 
-    RaftClientRequestSender rpc = ((RaftClientImpl)client).getRequestSender();
+    RaftClientRequestSender rpc = client.getRequestSender();
     RaftClientReply reply = null;
     // it is possible that the remote peer's rpc server is not ready. need retry
     for (int i = 0; reply == null && i < 10; i++) {

http://git-wip-us.apache.org/repos/asf/incubator-ratis/blob/336874fa/ratis-server/src/test/java/org/apache/ratis/server/impl/RaftReconfigurationBaseTest.java
----------------------------------------------------------------------
diff --git a/ratis-server/src/test/java/org/apache/ratis/server/impl/RaftReconfigurationBaseTest.java b/ratis-server/src/test/java/org/apache/ratis/server/impl/RaftReconfigurationBaseTest.java
index 30b334f..847678a 100644
--- a/ratis-server/src/test/java/org/apache/ratis/server/impl/RaftReconfigurationBaseTest.java
+++ b/ratis-server/src/test/java/org/apache/ratis/server/impl/RaftReconfigurationBaseTest.java
@@ -17,20 +17,6 @@
  */
 package org.apache.ratis.server.impl;
 
-import static java.util.Arrays.asList;
-import static org.apache.ratis.MiniRaftCluster.logSyncDelay;
-import static org.apache.ratis.server.impl.RaftServerConstants.DEFAULT_SEQNUM;
-import static org.apache.ratis.server.impl.RaftServerTestUtil.waitAndCheckNewConf;
-import static org.apache.ratis.shaded.proto.RaftProtos.LogEntryProto.LogEntryBodyCase.CONFIGURATIONENTRY;
-
-import java.io.IOException;
-import java.util.ArrayList;
-import java.util.Arrays;
-import java.util.Collections;
-import java.util.concurrent.CountDownLatch;
-import java.util.concurrent.atomic.AtomicBoolean;
-import java.util.concurrent.atomic.AtomicReference;
-
 import org.apache.log4j.Level;
 import org.apache.ratis.MiniRaftCluster;
 import org.apache.ratis.MiniRaftCluster.PeerChanges;
@@ -38,13 +24,8 @@ import org.apache.ratis.RaftTestUtil;
 import org.apache.ratis.RaftTestUtil.SimpleMessage;
 import org.apache.ratis.client.RaftClient;
 import org.apache.ratis.client.RaftClientRequestSender;
-import org.apache.ratis.client.impl.RaftClientImpl;
 import org.apache.ratis.conf.RaftProperties;
-import org.apache.ratis.protocol.RaftClientReply;
-import org.apache.ratis.protocol.RaftPeer;
-import org.apache.ratis.protocol.ReconfigurationInProgressException;
-import org.apache.ratis.protocol.ReconfigurationTimeoutException;
-import org.apache.ratis.protocol.SetConfigurationRequest;
+import org.apache.ratis.protocol.*;
 import org.apache.ratis.server.RaftServerConfigKeys;
 import org.apache.ratis.server.simulation.RequestHandler;
 import org.apache.ratis.server.storage.RaftLog;
@@ -56,6 +37,20 @@ import org.mockito.internal.util.reflection.Whitebox;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
+import java.io.IOException;
+import java.util.ArrayList;
+import java.util.Arrays;
+import java.util.Collections;
+import java.util.concurrent.CountDownLatch;
+import java.util.concurrent.atomic.AtomicBoolean;
+import java.util.concurrent.atomic.AtomicReference;
+
+import static java.util.Arrays.asList;
+import static org.apache.ratis.MiniRaftCluster.logSyncDelay;
+import static org.apache.ratis.server.impl.RaftServerConstants.DEFAULT_SEQNUM;
+import static org.apache.ratis.server.impl.RaftServerTestUtil.waitAndCheckNewConf;
+import static org.apache.ratis.shaded.proto.RaftProtos.LogEntryProto.LogEntryBodyCase.CONFIGURATIONENTRY;
+
 public abstract class RaftReconfigurationBaseTest {
   static {
     RaftUtils.setLogLevel(RaftServerImpl.LOG, Level.DEBUG);
@@ -255,7 +250,7 @@ public abstract class RaftReconfigurationBaseTest {
           asList(c1.allPeersInNewConf));
       Assert.assertFalse(cluster.getLeader().getRaftConf().isTransitional());
 
-      final RaftClientRequestSender sender = ((RaftClientImpl)client).getRequestSender();
+      final RaftClientRequestSender sender = client.getRequestSender();
       final SetConfigurationRequest request = new SetConfigurationRequest(
           "client", leaderId, DEFAULT_SEQNUM, c1.allPeersInNewConf);
       try {
@@ -472,7 +467,7 @@ public abstract class RaftReconfigurationBaseTest {
         try(final RaftClient client2 = cluster.createClient("client2", leaderId)) {
           latch.await();
           LOG.info("client2 starts to change conf");
-          final RaftClientRequestSender sender2 = ((RaftClientImpl)client2).getRequestSender();
+          final RaftClientRequestSender sender2 = client2.getRequestSender();
           sender2.sendRequest(new SetConfigurationRequest(
               "client2", leaderId, DEFAULT_SEQNUM, peersInRequest2));
         } catch (ReconfigurationInProgressException e) {
@@ -537,7 +532,7 @@ public abstract class RaftReconfigurationBaseTest {
       new Thread(() -> {
         try(final RaftClient client = cluster.createClient("client1", leaderId)) {
           LOG.info("client starts to change conf");
-          final RaftClientRequestSender sender = ((RaftClientImpl)client).getRequestSender();
+          final RaftClientRequestSender sender = client.getRequestSender();
           RaftClientReply reply = sender.sendRequest(new SetConfigurationRequest(
               "client", leaderId, DEFAULT_SEQNUM, change.allPeersInNewConf));
           if (reply.isNotLeader()) {


[06/54] [abbrv] incubator-ratis git commit: Rename RaftServer to RaftServerImpl (will add an interface.)

Posted by ji...@apache.org.
http://git-wip-us.apache.org/repos/asf/incubator-ratis/blob/22f3ee0b/raft-server/src/main/java/org/apache/raft/server/impl/RaftServerImpl.java
----------------------------------------------------------------------
diff --git a/raft-server/src/main/java/org/apache/raft/server/impl/RaftServerImpl.java b/raft-server/src/main/java/org/apache/raft/server/impl/RaftServerImpl.java
new file mode 100644
index 0000000..6778683
--- /dev/null
+++ b/raft-server/src/main/java/org/apache/raft/server/impl/RaftServerImpl.java
@@ -0,0 +1,749 @@
+/**
+ * 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.raft.server.impl;
+
+import com.google.common.annotations.VisibleForTesting;
+import com.google.common.base.Preconditions;
+import org.apache.raft.conf.RaftProperties;
+import org.apache.raft.protocol.*;
+import org.apache.raft.server.RaftServerConfigKeys;
+import org.apache.raft.server.protocol.RaftServerProtocol;
+import org.apache.raft.server.protocol.TermIndex;
+import org.apache.raft.server.storage.FileInfo;
+import org.apache.raft.shaded.proto.RaftProtos.*;
+import org.apache.raft.statemachine.SnapshotInfo;
+import org.apache.raft.statemachine.StateMachine;
+import org.apache.raft.statemachine.TransactionContext;
+import org.apache.raft.util.CodeInjectionForTesting;
+import org.apache.raft.util.LifeCycle;
+import org.apache.raft.util.ProtoUtils;
+import org.apache.raft.util.RaftUtils;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.io.Closeable;
+import java.io.IOException;
+import java.io.InterruptedIOException;
+import java.util.Arrays;
+import java.util.Collection;
+import java.util.List;
+import java.util.OptionalLong;
+import java.util.concurrent.CompletableFuture;
+
+import static org.apache.raft.shaded.proto.RaftProtos.AppendEntriesReplyProto.AppendResult.*;
+import static org.apache.raft.util.LifeCycle.State.*;
+
+public class RaftServerImpl implements RaftServerProtocol, Closeable {
+  public static final Logger LOG = LoggerFactory.getLogger(RaftServerImpl.class);
+
+  private static final String CLASS_NAME = RaftServerImpl.class.getSimpleName();
+  static final String REQUEST_VOTE = CLASS_NAME + ".requestVote";
+  static final String APPEND_ENTRIES = CLASS_NAME + ".appendEntries";
+  static final String INSTALL_SNAPSHOT = CLASS_NAME + ".installSnapshot";
+
+
+  private final int minTimeoutMs;
+  private final int maxTimeoutMs;
+
+  private final LifeCycle lifeCycle;
+  private final ServerState state;
+  private final StateMachine stateMachine;
+  private final RaftProperties properties;
+  private volatile Role role;
+
+  /** used when the peer is follower, to monitor election timeout */
+  private volatile FollowerState heartbeatMonitor;
+
+  /** used when the peer is candidate, to request votes from other peers */
+  private volatile LeaderElection electionDaemon;
+
+  /** used when the peer is leader */
+  private volatile LeaderState leaderState;
+
+  private RaftServerRpc serverRpc;
+
+  private final LogAppenderFactory appenderFactory;
+
+  public RaftServerImpl(String id, RaftConfiguration raftConf,
+                        RaftProperties properties, StateMachine stateMachine) throws IOException {
+    this.lifeCycle = new LifeCycle(id);
+    minTimeoutMs = properties.getInt(
+        RaftServerConfigKeys.RAFT_SERVER_RPC_TIMEOUT_MIN_MS_KEY,
+        RaftServerConfigKeys.RAFT_SERVER_RPC_TIMEOUT_MIN_MS_DEFAULT);
+    maxTimeoutMs = properties.getInt(
+        RaftServerConfigKeys.RAFT_SERVER_RPC_TIMEOUT_MAX_MS_KEY,
+        RaftServerConfigKeys.RAFT_SERVER_RPC_TIMEOUT_MAX_MS_DEFAULT);
+    Preconditions.checkArgument(maxTimeoutMs > minTimeoutMs,
+        "max timeout: %s, min timeout: %s", maxTimeoutMs, minTimeoutMs);
+    this.properties = properties;
+    this.stateMachine = stateMachine;
+    this.state = new ServerState(id, raftConf, properties, this, stateMachine);
+    appenderFactory = initAppenderFactory();
+  }
+
+  public int getMinTimeoutMs() {
+    return minTimeoutMs;
+  }
+
+  public int getMaxTimeoutMs() {
+    return maxTimeoutMs;
+  }
+
+  public int getRandomTimeoutMs() {
+    return RaftUtils.getRandomBetween(minTimeoutMs, maxTimeoutMs);
+  }
+
+  public StateMachine getStateMachine() {
+    return this.stateMachine;
+  }
+
+  public LogAppenderFactory getLogAppenderFactory() {
+    return appenderFactory;
+  }
+
+  private LogAppenderFactory initAppenderFactory() {
+    Class<? extends LogAppenderFactory> factoryClass = properties.getClass(
+        RaftServerConfigKeys.RAFT_SERVER_LOG_APPENDER_FACTORY_CLASS_KEY,
+        RaftServerConfigKeys.RAFT_SERVER_LOG_APPENDER_FACTORY_CLASS_DEFAULT,
+        LogAppenderFactory.class);
+    return RaftUtils.newInstance(factoryClass);
+  }
+
+  /**
+   * Used by tests to set initial raft configuration with correct port bindings.
+   */
+  @VisibleForTesting
+  public void setInitialConf(RaftConfiguration conf) {
+    this.state.setInitialConf(conf);
+  }
+
+  public void setServerRpc(RaftServerRpc serverRpc) {
+    this.serverRpc = serverRpc;
+    // add peers into rpc service
+    RaftConfiguration conf = getRaftConf();
+    if (conf != null) {
+      addPeersToRPC(conf.getPeers());
+    }
+  }
+
+  public RaftServerRpc getServerRpc() {
+    return serverRpc;
+  }
+
+  public void start() {
+    lifeCycle.transition(STARTING);
+    state.start();
+    RaftConfiguration conf = getRaftConf();
+    if (conf != null && conf.contains(getId())) {
+      LOG.debug("{} starts as a follower", getId());
+      startAsFollower();
+    } else {
+      LOG.debug("{} starts with initializing state", getId());
+      startInitializing();
+    }
+  }
+
+  /**
+   * The peer belongs to the current configuration, should start as a follower
+   */
+  private void startAsFollower() {
+    role = Role.FOLLOWER;
+    heartbeatMonitor = new FollowerState(this);
+    heartbeatMonitor.start();
+
+    serverRpc.start();
+    lifeCycle.transition(RUNNING);
+  }
+
+  /**
+   * The peer does not have any configuration (maybe it will later be included
+   * in some configuration). Start still as a follower but will not vote or
+   * start election.
+   */
+  private void startInitializing() {
+    role = Role.FOLLOWER;
+    // do not start heartbeatMonitoring
+    serverRpc.start();
+  }
+
+  public ServerState getState() {
+    return this.state;
+  }
+
+  public String getId() {
+    return getState().getSelfId();
+  }
+
+  public RaftConfiguration getRaftConf() {
+    return getState().getRaftConf();
+  }
+
+  @Override
+  public void close() {
+    lifeCycle.checkStateAndClose(() -> {
+      try {
+        shutdownHeartbeatMonitor();
+        shutdownElectionDaemon();
+        shutdownLeaderState();
+
+        serverRpc.shutdown();
+        state.close();
+      } catch (Exception ignored) {
+        LOG.warn("Failed to kill " + state.getSelfId(), ignored);
+      }
+    });
+  }
+
+  public boolean isAlive() {
+    return !lifeCycle.getCurrentState().isOneOf(CLOSING, CLOSED);
+  }
+
+  public boolean isFollower() {
+    return role == Role.FOLLOWER;
+  }
+
+  public boolean isCandidate() {
+    return role == Role.CANDIDATE;
+  }
+
+  public boolean isLeader() {
+    return role == Role.LEADER;
+  }
+
+  Role getRole() {
+    return role;
+  }
+
+  /**
+   * Change the server state to Follower if necessary
+   * @param newTerm The new term.
+   * @param sync We will call {@link ServerState#persistMetadata()} if this is
+   *             set to true and term/votedFor get updated.
+   * @return if the term/votedFor should be updated to the new term
+   * @throws IOException if term/votedFor persistence failed.
+   */
+  synchronized boolean changeToFollower(long newTerm, boolean sync)
+      throws IOException {
+    final Role old = role;
+    role = Role.FOLLOWER;
+
+    boolean metadataUpdated = false;
+    if (newTerm > state.getCurrentTerm()) {
+      state.setCurrentTerm(newTerm);
+      state.resetLeaderAndVotedFor();
+      metadataUpdated = true;
+    }
+
+    if (old == Role.LEADER) {
+      assert leaderState != null;
+      shutdownLeaderState();
+    } else if (old == Role.CANDIDATE) {
+      shutdownElectionDaemon();
+    }
+
+    if (old != Role.FOLLOWER) {
+      heartbeatMonitor = new FollowerState(this);
+      heartbeatMonitor.start();
+    }
+
+    if (metadataUpdated && sync) {
+      state.persistMetadata();
+    }
+    return metadataUpdated;
+  }
+
+  private synchronized void shutdownLeaderState() {
+    final LeaderState leader = leaderState;
+    if (leader != null) {
+      leader.stop();
+    }
+    leaderState = null;
+    // TODO: make sure that StateMachineUpdater has applied all transactions that have context
+  }
+
+  private void shutdownElectionDaemon() {
+    final LeaderElection election = electionDaemon;
+    if (election != null) {
+      election.stopRunning();
+      // no need to interrupt the election thread
+    }
+    electionDaemon = null;
+  }
+
+  synchronized void changeToLeader() {
+    Preconditions.checkState(isCandidate());
+    shutdownElectionDaemon();
+    role = Role.LEADER;
+    state.becomeLeader();
+    // start sending AppendEntries RPC to followers
+    leaderState = new LeaderState(this, properties);
+    leaderState.start();
+  }
+
+  private void shutdownHeartbeatMonitor() {
+    final FollowerState hm = heartbeatMonitor;
+    if (hm != null) {
+      hm.stopRunning();
+      hm.interrupt();
+    }
+    heartbeatMonitor = null;
+  }
+
+  synchronized void changeToCandidate() {
+    Preconditions.checkState(isFollower());
+    shutdownHeartbeatMonitor();
+    role = Role.CANDIDATE;
+    // start election
+    electionDaemon = new LeaderElection(this);
+    electionDaemon.start();
+  }
+
+  @Override
+  public String toString() {
+    return role + " " + state + " " + lifeCycle.getCurrentState();
+  }
+
+  /**
+   * @return null if the server is in leader state.
+   */
+  CompletableFuture<RaftClientReply> checkLeaderState(
+      RaftClientRequest request) {
+    if (!isLeader()) {
+      NotLeaderException exception = generateNotLeaderException();
+      CompletableFuture<RaftClientReply> future = new CompletableFuture<>();
+      future.complete(new RaftClientReply(request, exception));
+      return future;
+    }
+    return null;
+  }
+
+  NotLeaderException generateNotLeaderException() {
+    if (lifeCycle.getCurrentState() != RUNNING) {
+      return new NotLeaderException(getId(), null, null);
+    }
+    String leaderId = state.getLeaderId();
+    if (leaderId == null || leaderId.equals(state.getSelfId())) {
+      // No idea about who is the current leader. Or the peer is the current
+      // leader, but it is about to step down
+      RaftPeer suggestedLeader = state.getRaftConf()
+          .getRandomPeer(state.getSelfId());
+      leaderId = suggestedLeader == null ? null : suggestedLeader.getId();
+    }
+    RaftConfiguration conf = getRaftConf();
+    Collection<RaftPeer> peers = conf.getPeers();
+    return new NotLeaderException(getId(), conf.getPeer(leaderId),
+        peers.toArray(new RaftPeer[peers.size()]));
+  }
+
+  /**
+   * Handle a normal update request from client.
+   */
+  public CompletableFuture<RaftClientReply> appendTransaction(
+      RaftClientRequest request, TransactionContext entry)
+      throws RaftException {
+    LOG.debug("{}: receive client request({})", getId(), request);
+    lifeCycle.assertCurrentState(RUNNING);
+    CompletableFuture<RaftClientReply> reply;
+
+    final PendingRequest pending;
+    synchronized (this) {
+      reply = checkLeaderState(request);
+      if (reply != null) {
+        return reply;
+      }
+
+      // append the message to its local log
+      final long entryIndex;
+      try {
+        entryIndex = state.applyLog(entry);
+      } catch (IOException e) {
+        throw new RaftException(e);
+      }
+
+      // put the request into the pending queue
+      pending = leaderState.addPendingRequest(entryIndex, request, entry);
+      leaderState.notifySenders();
+    }
+    return pending.getFuture();
+  }
+
+  /**
+   * Handle a raft configuration change request from client.
+   */
+  public CompletableFuture<RaftClientReply> setConfiguration(
+      SetConfigurationRequest request) throws IOException {
+    LOG.debug("{}: receive setConfiguration({})", getId(), request);
+    lifeCycle.assertCurrentState(RUNNING);
+    CompletableFuture<RaftClientReply> reply = checkLeaderState(request);
+    if (reply != null) {
+      return reply;
+    }
+
+    final RaftPeer[] peersInNewConf = request.getPeersInNewConf();
+    final PendingRequest pending;
+    synchronized (this) {
+      reply = checkLeaderState(request);
+      if (reply != null) {
+        return reply;
+      }
+
+      final RaftConfiguration current = getRaftConf();
+      // make sure there is no other raft reconfiguration in progress
+      if (!current.isStable() || leaderState.inStagingState() ||
+          !state.isCurrentConfCommitted()) {
+        throw new ReconfigurationInProgressException(
+            "Reconfiguration is already in progress: " + current);
+      }
+
+      // return true if the new configuration is the same with the current one
+      if (current.hasNoChange(peersInNewConf)) {
+        pending = leaderState.returnNoConfChange(request);
+        return pending.getFuture();
+      }
+
+      // add new peers into the rpc service
+      addPeersToRPC(Arrays.asList(peersInNewConf));
+      // add staging state into the leaderState
+      pending = leaderState.startSetConfiguration(request);
+    }
+    return pending.getFuture();
+  }
+
+  private boolean shouldWithholdVotes() {
+    return isLeader() || (isFollower() && state.hasLeader()
+        && heartbeatMonitor.shouldWithholdVotes());
+  }
+
+  /**
+   * check if the remote peer is not included in the current conf
+   * and should shutdown. should shutdown if all the following stands:
+   * 1. this is a leader
+   * 2. current conf is stable and has been committed
+   * 3. candidate id is not included in conf
+   * 4. candidate's last entry's index < conf's index
+   */
+  private boolean shouldSendShutdown(String candidateId,
+      TermIndex candidateLastEntry) {
+    return isLeader()
+        && getRaftConf().isStable()
+        && getState().isConfCommitted()
+        && !getRaftConf().containsInConf(candidateId)
+        && candidateLastEntry.getIndex() < getRaftConf().getLogEntryIndex()
+        && !leaderState.isBootStrappingPeer(candidateId);
+  }
+
+  @Override
+  public RequestVoteReplyProto requestVote(RequestVoteRequestProto r)
+      throws IOException {
+    final String candidateId = r.getServerRequest().getRequestorId();
+    return requestVote(candidateId, r.getCandidateTerm(),
+        ServerProtoUtils.toTermIndex(r.getCandidateLastEntry()));
+  }
+
+  private RequestVoteReplyProto requestVote(String candidateId,
+      long candidateTerm, TermIndex candidateLastEntry) throws IOException {
+    CodeInjectionForTesting.execute(REQUEST_VOTE, getId(),
+        candidateId, candidateTerm, candidateLastEntry);
+    LOG.debug("{}: receive requestVote({}, {}, {})",
+        getId(), candidateId, candidateTerm, candidateLastEntry);
+    lifeCycle.assertCurrentState(RUNNING);
+
+    boolean voteGranted = false;
+    boolean shouldShutdown = false;
+    final RequestVoteReplyProto reply;
+    synchronized (this) {
+      if (shouldWithholdVotes()) {
+        LOG.info("{} Withhold vote from server {} with term {}. " +
+            "This server:{}, last rpc time from leader {} is {}", getId(),
+            candidateId, candidateTerm, this, this.getState().getLeaderId(),
+            (isFollower() ? heartbeatMonitor.getLastRpcTime() : -1));
+      } else if (state.recognizeCandidate(candidateId, candidateTerm)) {
+        boolean termUpdated = changeToFollower(candidateTerm, false);
+        // see Section 5.4.1 Election restriction
+        if (state.isLogUpToDate(candidateLastEntry)) {
+          heartbeatMonitor.updateLastRpcTime(false);
+          state.grantVote(candidateId);
+          voteGranted = true;
+        }
+        if (termUpdated || voteGranted) {
+          state.persistMetadata(); // sync metafile
+        }
+      }
+      if (!voteGranted && shouldSendShutdown(candidateId, candidateLastEntry)) {
+        shouldShutdown = true;
+      }
+      reply = ServerProtoUtils.toRequestVoteReplyProto(candidateId, getId(),
+          voteGranted, state.getCurrentTerm(), shouldShutdown);
+      if (LOG.isDebugEnabled()) {
+        LOG.debug("{} replies to vote request: {}. Peer's state: {}",
+            getId(), ProtoUtils.toString(reply), state);
+      }
+    }
+    return reply;
+  }
+
+  private void validateEntries(long expectedTerm, TermIndex previous,
+      LogEntryProto... entries) {
+    if (entries != null && entries.length > 0) {
+      final long index0 = entries[0].getIndex();
+
+      if (previous == null || previous.getTerm() == 0) {
+        Preconditions.checkArgument(index0 == 0,
+            "Unexpected Index: previous is null but entries[%s].getIndex()=%s",
+            0, index0);
+      } else {
+        Preconditions.checkArgument(previous.getIndex() == index0 - 1,
+            "Unexpected Index: previous is %s but entries[%s].getIndex()=%s",
+            previous, 0, index0);
+      }
+
+      for (int i = 0; i < entries.length; i++) {
+        final long t = entries[i].getTerm();
+        Preconditions.checkArgument(expectedTerm >= t,
+            "Unexpected Term: entries[%s].getTerm()=%s but expectedTerm=%s",
+            i, t, expectedTerm);
+
+        final long indexi = entries[i].getIndex();
+        Preconditions.checkArgument(indexi == index0 + i,
+            "Unexpected Index: entries[%s].getIndex()=%s but entries[0].getIndex()=%s",
+            i, indexi, index0);
+      }
+    }
+  }
+
+  @Override
+  public AppendEntriesReplyProto appendEntries(AppendEntriesRequestProto r)
+      throws IOException {
+    // TODO avoid converting list to array
+    final LogEntryProto[] entries = r.getEntriesList()
+        .toArray(new LogEntryProto[r.getEntriesCount()]);
+    final TermIndex previous = r.hasPreviousLog() ?
+        ServerProtoUtils.toTermIndex(r.getPreviousLog()) : null;
+    return appendEntries(r.getServerRequest().getRequestorId(),
+        r.getLeaderTerm(), previous, r.getLeaderCommit(), r.getInitializing(),
+        entries);
+  }
+
+  private AppendEntriesReplyProto appendEntries(String leaderId, long leaderTerm,
+      TermIndex previous, long leaderCommit, boolean initializing,
+      LogEntryProto... entries) throws IOException {
+    CodeInjectionForTesting.execute(APPEND_ENTRIES, getId(),
+        leaderId, leaderTerm, previous, leaderCommit, initializing, entries);
+    if (LOG.isDebugEnabled()) {
+      LOG.debug("{}: receive appendEntries({}, {}, {}, {}, {}, {})", getId(),
+          leaderId, leaderTerm, previous, leaderCommit, initializing,
+          ServerProtoUtils.toString(entries));
+    }
+    lifeCycle.assertCurrentState(STARTING, RUNNING);
+
+    try {
+      validateEntries(leaderTerm, previous, entries);
+    } catch (IllegalArgumentException e) {
+      throw new IOException(e);
+    }
+
+    final long currentTerm;
+    long nextIndex = state.getLog().getNextIndex();
+    synchronized (this) {
+      final boolean recognized = state.recognizeLeader(leaderId, leaderTerm);
+      currentTerm = state.getCurrentTerm();
+      if (!recognized) {
+        final AppendEntriesReplyProto reply = ServerProtoUtils.toAppendEntriesReplyProto(
+            leaderId, getId(), currentTerm, nextIndex, NOT_LEADER);
+        if (LOG.isDebugEnabled()) {
+          LOG.debug("{}: do not recognize leader. Reply: {}",
+              getId(), ProtoUtils.toString(reply));
+        }
+        return reply;
+      }
+      changeToFollower(leaderTerm, true);
+      state.setLeader(leaderId);
+
+      if (!initializing && lifeCycle.compareAndTransition(STARTING, RUNNING)) {
+        heartbeatMonitor = new FollowerState(this);
+        heartbeatMonitor.start();
+      }
+      if (lifeCycle.getCurrentState() == RUNNING) {
+        heartbeatMonitor.updateLastRpcTime(true);
+      }
+
+      // We need to check if "previous" is in the local peer. Note that it is
+      // possible that "previous" is covered by the latest snapshot: e.g.,
+      // it's possible there's no log entries outside of the latest snapshot.
+      // However, it is not possible that "previous" index is smaller than the
+      // last index included in snapshot. This is because indices <= snapshot's
+      // last index should have been committed.
+      if (previous != null && !containPrevious(previous)) {
+        final AppendEntriesReplyProto reply =
+            ServerProtoUtils.toAppendEntriesReplyProto(leaderId, getId(),
+                currentTerm, Math.min(nextIndex, previous.getIndex()), INCONSISTENCY);
+        LOG.debug("{}: inconsistency entries. Leader previous:{}, Reply:{}",
+            getId(), previous, ServerProtoUtils.toString(reply));
+        return reply;
+      }
+
+      state.getLog().append(entries);
+      state.updateConfiguration(entries);
+      state.updateStatemachine(leaderCommit, currentTerm);
+    }
+    if (entries != null && entries.length > 0) {
+      try {
+        state.getLog().logSync();
+      } catch (InterruptedException e) {
+        throw new InterruptedIOException("logSync got interrupted");
+      }
+      nextIndex = entries[entries.length - 1].getIndex() + 1;
+    }
+    synchronized (this) {
+      if (lifeCycle.getCurrentState() == RUNNING && isFollower()
+          && getState().getCurrentTerm() == currentTerm) {
+        // reset election timer to avoid punishing the leader for our own
+        // long disk writes
+        heartbeatMonitor.updateLastRpcTime(false);
+      }
+    }
+    final AppendEntriesReplyProto reply = ServerProtoUtils.toAppendEntriesReplyProto(
+        leaderId, getId(), currentTerm, nextIndex, SUCCESS);
+    LOG.debug("{}: succeeded to handle AppendEntries. Reply: {}", getId(),
+        ServerProtoUtils.toString(reply));
+    return reply;
+  }
+
+  private boolean containPrevious(TermIndex previous) {
+    LOG.debug("{}: prev:{}, latestSnapshot:{}, getLatestInstalledSnapshot:{}",
+        getId(), previous, state.getLatestSnapshot(), state.getLatestInstalledSnapshot());
+    return state.getLog().contains(previous)
+        ||  (state.getLatestSnapshot() != null
+             && state.getLatestSnapshot().getTermIndex().equals(previous))
+        || (state.getLatestInstalledSnapshot() != null)
+             && state.getLatestInstalledSnapshot().equals(previous);
+  }
+
+  @Override
+  public InstallSnapshotReplyProto installSnapshot(
+      InstallSnapshotRequestProto request) throws IOException {
+    final String leaderId = request.getServerRequest().getRequestorId();
+    CodeInjectionForTesting.execute(INSTALL_SNAPSHOT, getId(), leaderId, request);
+    LOG.debug("{}: receive installSnapshot({})", getId(), request);
+
+    lifeCycle.assertCurrentState(STARTING, RUNNING);
+
+    final long currentTerm;
+    final long leaderTerm = request.getLeaderTerm();
+    final TermIndex lastTermIndex = ServerProtoUtils.toTermIndex(
+        request.getTermIndex());
+    final long lastIncludedIndex = lastTermIndex.getIndex();
+    synchronized (this) {
+      final boolean recognized = state.recognizeLeader(leaderId, leaderTerm);
+      currentTerm = state.getCurrentTerm();
+      if (!recognized) {
+        final InstallSnapshotReplyProto reply = ServerProtoUtils
+            .toInstallSnapshotReplyProto(leaderId, getId(), currentTerm,
+                request.getRequestIndex(), InstallSnapshotResult.NOT_LEADER);
+        LOG.debug("{}: do not recognize leader for installing snapshot." +
+            " Reply: {}", getId(), reply);
+        return reply;
+      }
+      changeToFollower(leaderTerm, true);
+      state.setLeader(leaderId);
+
+      if (lifeCycle.getCurrentState() == RUNNING) {
+        heartbeatMonitor.updateLastRpcTime(true);
+      }
+
+      // Check and append the snapshot chunk. We simply put this in lock
+      // considering a follower peer requiring a snapshot installation does not
+      // have a lot of requests
+      Preconditions.checkState(
+          state.getLog().getNextIndex() <= lastIncludedIndex,
+          "%s log's next id is %s, last included index in snapshot is %s",
+          getId(),  state.getLog().getNextIndex(), lastIncludedIndex);
+
+      //TODO: We should only update State with installed snapshot once the request is done.
+      state.installSnapshot(request);
+
+      // update the committed index
+      // re-load the state machine if this is the last chunk
+      if (request.getDone()) {
+        state.reloadStateMachine(lastIncludedIndex, leaderTerm);
+      }
+      if (lifeCycle.getCurrentState() == RUNNING) {
+        heartbeatMonitor.updateLastRpcTime(false);
+      }
+    }
+    if (request.getDone()) {
+      LOG.info("{}: successfully install the whole snapshot-{}", getId(),
+          lastIncludedIndex);
+    }
+    return ServerProtoUtils.toInstallSnapshotReplyProto(leaderId, getId(),
+        currentTerm, request.getRequestIndex(), InstallSnapshotResult.SUCCESS);
+  }
+
+  AppendEntriesRequestProto createAppendEntriesRequest(long leaderTerm,
+      String targetId, TermIndex previous, List<LogEntryProto> entries,
+      boolean initializing) {
+    return ServerProtoUtils.toAppendEntriesRequestProto(getId(), targetId,
+        leaderTerm, entries, state.getLog().getLastCommittedIndex(),
+        initializing, previous);
+  }
+
+  synchronized InstallSnapshotRequestProto createInstallSnapshotRequest(
+      String targetId, String requestId, int requestIndex, SnapshotInfo snapshot,
+      List<FileChunkProto> chunks, boolean done) {
+    OptionalLong totalSize = snapshot.getFiles().stream()
+        .mapToLong(FileInfo::getFileSize).reduce(Long::sum);
+    assert totalSize.isPresent();
+    return ServerProtoUtils.toInstallSnapshotRequestProto(getId(), targetId,
+        requestId, requestIndex, state.getCurrentTerm(), snapshot.getTermIndex(),
+        chunks, totalSize.getAsLong(), done);
+  }
+
+  synchronized RequestVoteRequestProto createRequestVoteRequest(String targetId,
+      long term, TermIndex lastEntry) {
+    return ServerProtoUtils.toRequestVoteRequestProto(getId(), targetId, term,
+        lastEntry);
+  }
+
+  public synchronized void submitLocalSyncEvent() {
+    if (isLeader() && leaderState != null) {
+      leaderState.submitUpdateStateEvent(LeaderState.UPDATE_COMMIT_EVENT);
+    }
+  }
+
+  public void addPeersToRPC(Iterable<RaftPeer> peers) {
+    serverRpc.addPeers(peers);
+  }
+
+  synchronized void replyPendingRequest(long logIndex,
+      CompletableFuture<Message> message) {
+    if (isLeader() && leaderState != null) { // is leader and is running
+      leaderState.replyPendingRequest(logIndex, message);
+    }
+  }
+
+  TransactionContext getTransactionContext(long index) {
+    if (leaderState != null) { // is leader and is running
+      return leaderState.getTransactionContext(index);
+    }
+    return null;
+  }
+
+  public RaftProperties getProperties() {
+    return this.properties;
+  }
+}

http://git-wip-us.apache.org/repos/asf/incubator-ratis/blob/22f3ee0b/raft-server/src/main/java/org/apache/raft/server/impl/RequestDispatcher.java
----------------------------------------------------------------------
diff --git a/raft-server/src/main/java/org/apache/raft/server/impl/RequestDispatcher.java b/raft-server/src/main/java/org/apache/raft/server/impl/RequestDispatcher.java
index cc8651d..b897afd 100644
--- a/raft-server/src/main/java/org/apache/raft/server/impl/RequestDispatcher.java
+++ b/raft-server/src/main/java/org/apache/raft/server/impl/RequestDispatcher.java
@@ -44,10 +44,10 @@ import java.util.concurrent.ExecutionException;
 public class RequestDispatcher implements RaftClientProtocol, RaftServerProtocol {
   static final Logger LOG = LoggerFactory.getLogger(RequestDispatcher.class);
 
-  private final RaftServer server;
+  private final RaftServerImpl server;
   private final StateMachine stateMachine;
 
-  public RequestDispatcher(RaftServer server) {
+  public RequestDispatcher(RaftServerImpl server) {
     this.server = server;
     this.stateMachine = server.getStateMachine();
   }

http://git-wip-us.apache.org/repos/asf/incubator-ratis/blob/22f3ee0b/raft-server/src/main/java/org/apache/raft/server/impl/ServerState.java
----------------------------------------------------------------------
diff --git a/raft-server/src/main/java/org/apache/raft/server/impl/ServerState.java b/raft-server/src/main/java/org/apache/raft/server/impl/ServerState.java
index 6680175..c91968c 100644
--- a/raft-server/src/main/java/org/apache/raft/server/impl/ServerState.java
+++ b/raft-server/src/main/java/org/apache/raft/server/impl/ServerState.java
@@ -40,7 +40,7 @@ import static org.apache.raft.server.RaftServerConfigKeys.RAFT_SERVER_USE_MEMORY
  */
 public class ServerState implements Closeable {
   private final String selfId;
-  private final RaftServer server;
+  private final RaftServerImpl server;
   /** Raft log */
   private final RaftLog log;
   /** Raft configuration */
@@ -74,7 +74,7 @@ public class ServerState implements Closeable {
   private TermIndex latestInstalledSnapshot;
 
   ServerState(String id, RaftConfiguration conf, RaftProperties prop,
-      RaftServer server, StateMachine stateMachine) throws IOException {
+              RaftServerImpl server, StateMachine stateMachine) throws IOException {
     this.selfId = id;
     this.server = server;
     configurationManager = new ConfigurationManager(conf);
@@ -128,7 +128,7 @@ public class ServerState implements Closeable {
    * note we do not apply log entries to the state machine here since we do not
    * know whether they have been committed.
    */
-  private RaftLog initLog(String id, RaftProperties prop, RaftServer server,
+  private RaftLog initLog(String id, RaftProperties prop, RaftServerImpl server,
       long lastIndexInSnapshot) throws IOException {
     final RaftLog log;
     if (prop.getBoolean(RAFT_SERVER_USE_MEMORY_LOG_KEY,
@@ -273,7 +273,7 @@ public class ServerState implements Closeable {
 
   public void setRaftConf(long logIndex, RaftConfiguration conf) {
     configurationManager.addConfiguration(logIndex, conf);
-    RaftServer.LOG.info("{}: successfully update the configuration {}",
+    RaftServerImpl.LOG.info("{}: successfully update the configuration {}",
         getSelfId(), conf);
   }
 
@@ -306,7 +306,7 @@ public class ServerState implements Closeable {
   @Override
   public void close() throws IOException {
     stateMachineUpdater.stop();
-    RaftServer.LOG.info("{} closes. The last applied log index is {}",
+    RaftServerImpl.LOG.info("{} closes. The last applied log index is {}",
         getSelfId(), getLastAppliedIndex());
     storage.close();
   }

http://git-wip-us.apache.org/repos/asf/incubator-ratis/blob/22f3ee0b/raft-server/src/main/java/org/apache/raft/server/impl/StateMachineUpdater.java
----------------------------------------------------------------------
diff --git a/raft-server/src/main/java/org/apache/raft/server/impl/StateMachineUpdater.java b/raft-server/src/main/java/org/apache/raft/server/impl/StateMachineUpdater.java
index 06fa221..f85639b 100644
--- a/raft-server/src/main/java/org/apache/raft/server/impl/StateMachineUpdater.java
+++ b/raft-server/src/main/java/org/apache/raft/server/impl/StateMachineUpdater.java
@@ -57,7 +57,7 @@ class StateMachineUpdater implements Runnable {
 
   private final RaftProperties properties;
   private final StateMachine stateMachine;
-  private final RaftServer server;
+  private final RaftServerImpl server;
   private final RaftLog raftLog;
 
   private volatile long lastAppliedIndex;
@@ -69,7 +69,7 @@ class StateMachineUpdater implements Runnable {
   private final Thread updater;
   private volatile State state = State.RUNNING;
 
-  StateMachineUpdater(StateMachine stateMachine, RaftServer server,
+  StateMachineUpdater(StateMachine stateMachine, RaftServerImpl server,
       RaftLog raftLog, long lastAppliedIndex, RaftProperties properties) {
     this.properties = properties;
     this.stateMachine = stateMachine;

http://git-wip-us.apache.org/repos/asf/incubator-ratis/blob/22f3ee0b/raft-server/src/main/java/org/apache/raft/server/storage/RaftLogWorker.java
----------------------------------------------------------------------
diff --git a/raft-server/src/main/java/org/apache/raft/server/storage/RaftLogWorker.java b/raft-server/src/main/java/org/apache/raft/server/storage/RaftLogWorker.java
index 1837e94..6cef212 100644
--- a/raft-server/src/main/java/org/apache/raft/server/storage/RaftLogWorker.java
+++ b/raft-server/src/main/java/org/apache/raft/server/storage/RaftLogWorker.java
@@ -20,7 +20,7 @@ package org.apache.raft.server.storage;
 import com.google.common.base.Preconditions;
 import org.apache.raft.conf.RaftProperties;
 import org.apache.raft.io.nativeio.NativeIO;
-import org.apache.raft.server.impl.RaftServer;
+import org.apache.raft.server.impl.RaftServerImpl;
 import org.apache.raft.server.impl.RaftServerConstants;
 import org.apache.raft.server.storage.LogSegment.SegmentFileInfo;
 import org.apache.raft.server.storage.RaftLogCache.TruncationSegments;
@@ -56,7 +56,7 @@ class RaftLogWorker implements Runnable {
 
   private final RaftStorage storage;
   private LogOutputStream out;
-  private final RaftServer raftServer;
+  private final RaftServerImpl raftServer;
 
   /**
    * The number of entries that have been written into the LogOutputStream but
@@ -72,8 +72,8 @@ class RaftLogWorker implements Runnable {
 
   private final  RaftProperties properties;
 
-  RaftLogWorker(RaftServer raftServer, RaftStorage storage,
-      RaftProperties properties) {
+  RaftLogWorker(RaftServerImpl raftServer, RaftStorage storage,
+                RaftProperties properties) {
     this.raftServer = raftServer;
     this.storage = storage;
     this.properties = properties;

http://git-wip-us.apache.org/repos/asf/incubator-ratis/blob/22f3ee0b/raft-server/src/main/java/org/apache/raft/server/storage/SegmentedRaftLog.java
----------------------------------------------------------------------
diff --git a/raft-server/src/main/java/org/apache/raft/server/storage/SegmentedRaftLog.java b/raft-server/src/main/java/org/apache/raft/server/storage/SegmentedRaftLog.java
index 9c55491..293e1a4 100644
--- a/raft-server/src/main/java/org/apache/raft/server/storage/SegmentedRaftLog.java
+++ b/raft-server/src/main/java/org/apache/raft/server/storage/SegmentedRaftLog.java
@@ -22,7 +22,7 @@ import com.google.common.base.Preconditions;
 import org.apache.commons.io.Charsets;
 import org.apache.raft.conf.RaftProperties;
 import org.apache.raft.server.impl.ConfigurationManager;
-import org.apache.raft.server.impl.RaftServer;
+import org.apache.raft.server.impl.RaftServerImpl;
 import org.apache.raft.server.impl.RaftServerConstants;
 import org.apache.raft.server.storage.RaftStorageDirectory.LogPathAndIndex;
 import org.apache.raft.shaded.proto.RaftProtos.LogEntryProto;
@@ -100,8 +100,8 @@ public class SegmentedRaftLog extends RaftLog {
   private final RaftLogWorker fileLogWorker;
   private final long segmentMaxSize;
 
-  public SegmentedRaftLog(String selfId, RaftServer server, RaftStorage storage,
-      long lastIndexInSnapshot, RaftProperties properties) throws IOException {
+  public SegmentedRaftLog(String selfId, RaftServerImpl server, RaftStorage storage,
+                          long lastIndexInSnapshot, RaftProperties properties) throws IOException {
     super(selfId);
     this.storage = storage;
     this.segmentMaxSize = properties.getLong(RAFT_LOG_SEGMENT_MAX_SIZE_KEY,

http://git-wip-us.apache.org/repos/asf/incubator-ratis/blob/22f3ee0b/raft-server/src/test/java/org/apache/raft/MiniRaftCluster.java
----------------------------------------------------------------------
diff --git a/raft-server/src/test/java/org/apache/raft/MiniRaftCluster.java b/raft-server/src/test/java/org/apache/raft/MiniRaftCluster.java
index 4f0871f..45cec15 100644
--- a/raft-server/src/test/java/org/apache/raft/MiniRaftCluster.java
+++ b/raft-server/src/test/java/org/apache/raft/MiniRaftCluster.java
@@ -26,7 +26,7 @@ import org.apache.raft.protocol.RaftPeer;
 import org.apache.raft.server.RaftServerConfigKeys;
 import org.apache.raft.server.impl.DelayLocalExecutionInjection;
 import org.apache.raft.server.impl.RaftConfiguration;
-import org.apache.raft.server.impl.RaftServer;
+import org.apache.raft.server.impl.RaftServerImpl;
 import org.apache.raft.server.impl.RaftServerRpc;
 import org.apache.raft.server.storage.MemoryRaftLog;
 import org.apache.raft.server.storage.RaftLog;
@@ -72,7 +72,7 @@ public abstract class MiniRaftCluster {
       super(ids, properties, formatted);
     }
 
-    protected abstract RaftServer setPeerRpc(RaftPeer peer) throws IOException;
+    protected abstract RaftServerImpl setPeerRpc(RaftPeer peer) throws IOException;
 
     @Override
     protected void setPeerRpc() throws IOException {
@@ -133,7 +133,7 @@ public abstract class MiniRaftCluster {
   protected RaftConfiguration conf;
   protected final RaftProperties properties;
   private final String testBaseDir;
-  protected final Map<String, RaftServer> servers =
+  protected final Map<String, RaftServerImpl> servers =
       Collections.synchronizedMap(new LinkedHashMap<>());
 
   public MiniRaftCluster(String[] ids, RaftProperties properties,
@@ -152,7 +152,7 @@ public abstract class MiniRaftCluster {
     LOG.info("peers = " + peers.keySet());
     conf = RaftConfiguration.newBuilder().setConf(peers.keySet()).build();
     for (Map.Entry<RaftPeer, RPC> entry : peers.entrySet()) {
-      final RaftServer server = servers.get(entry.getKey().getId());
+      final RaftServerImpl server = servers.get(entry.getKey().getId());
       server.setInitialConf(conf);
       server.setServerRpc(entry.getValue());
     }
@@ -160,7 +160,7 @@ public abstract class MiniRaftCluster {
 
   public void start() {
     LOG.info("Starting " + getClass().getSimpleName());
-    servers.values().forEach(RaftServer::start);
+    servers.values().forEach(RaftServerImpl::start);
   }
 
   /**
@@ -173,8 +173,8 @@ public abstract class MiniRaftCluster {
   }
 
   public final void restart(boolean format) throws IOException {
-    servers.values().stream().filter(RaftServer::isAlive)
-        .forEach(RaftServer::close);
+    servers.values().stream().filter(RaftServerImpl::isAlive)
+        .forEach(RaftServerImpl::close);
     List<String> idList = new ArrayList<>(servers.keySet());
     for (String id : idList) {
       servers.remove(id);
@@ -197,16 +197,16 @@ public abstract class MiniRaftCluster {
     return conf;
   }
 
-  private RaftServer newRaftServer(String id, RaftConfiguration conf,
-      boolean format) {
-    final RaftServer s;
+  private RaftServerImpl newRaftServer(String id, RaftConfiguration conf,
+                                       boolean format) {
+    final RaftServerImpl s;
     try {
       final String dirStr = testBaseDir + id;
       if (format) {
         formatDir(dirStr);
       }
       properties.set(RaftServerConfigKeys.RAFT_SERVER_STORAGE_DIR_KEY, dirStr);
-      s = new RaftServer(id, conf, properties, getStateMachine4Test(properties));
+      s = new RaftServerImpl(id, conf, properties, getStateMachine4Test(properties));
     } catch (IOException e) {
       throw new RuntimeException(e);
     }
@@ -224,20 +224,20 @@ public abstract class MiniRaftCluster {
   public abstract RaftClientRequestSender getRaftClientRequestSender();
 
   protected <RPC extends RaftServerRpc> Collection<RaftPeer> addNewPeers(
-      Map<RaftPeer, RPC> newPeers, Collection<RaftServer> newServers,
+      Map<RaftPeer, RPC> newPeers, Collection<RaftServerImpl> newServers,
       boolean startService) throws IOException {
     for (Map.Entry<RaftPeer, RPC> entry : newPeers.entrySet()) {
-      RaftServer server = servers.get(entry.getKey().getId());
+      RaftServerImpl server = servers.get(entry.getKey().getId());
       server.setServerRpc(entry.getValue());
     }
     if (startService) {
-      newServers.forEach(RaftServer::start);
+      newServers.forEach(RaftServerImpl::start);
     }
     return new ArrayList<>(newPeers.keySet());
   }
 
   protected abstract Collection<RaftPeer> addNewPeers(
-      Collection<RaftPeer> newPeers, Collection<RaftServer> newServers,
+      Collection<RaftPeer> newPeers, Collection<RaftServerImpl> newServers,
       boolean startService) throws IOException;
 
   public PeerChanges addNewPeers(int number, boolean startNewPeer)
@@ -254,9 +254,9 @@ public abstract class MiniRaftCluster {
     }
 
     // create and add new RaftServers
-    final List<RaftServer> newServers = new ArrayList<>(ids.length);
+    final List<RaftServerImpl> newServers = new ArrayList<>(ids.length);
     for (RaftPeer p : newPeers) {
-      RaftServer newServer = newRaftServer(p.getId(), conf, true);
+      RaftServerImpl newServer = newRaftServer(p.getId(), conf, true);
       Preconditions.checkArgument(!servers.containsKey(p.getId()));
       servers.put(p.getId(), newServer);
       newServers.add(newServer);
@@ -273,12 +273,12 @@ public abstract class MiniRaftCluster {
   }
 
   public void startServer(String id) {
-    RaftServer server = servers.get(id);
+    RaftServerImpl server = servers.get(id);
     assert server != null;
     server.start();
   }
 
-  private RaftPeer getPeer(RaftServer s) {
+  private RaftPeer getPeer(RaftServerImpl s) {
     return new RaftPeer(s.getId(), s.getServerRpc().getInetSocketAddress());
   }
 
@@ -295,7 +295,7 @@ public abstract class MiniRaftCluster {
       peers.remove(leader);
       removedPeers.add(leader);
     }
-    List<RaftServer> followers = getFollowers();
+    List<RaftServerImpl> followers = getFollowers();
     for (int i = 0, removed = 0; i < followers.size() &&
         removed < (removeLeader ? number - 1 : number); i++) {
       RaftPeer toRemove = getPeer(followers.get(i));
@@ -317,7 +317,7 @@ public abstract class MiniRaftCluster {
 
   public String printServers() {
     StringBuilder b = new StringBuilder("\n#servers = " + servers.size() + "\n");
-    for (RaftServer s : servers.values()) {
+    for (RaftServerImpl s : servers.values()) {
       b.append("  ");
       b.append(s).append("\n");
     }
@@ -326,7 +326,7 @@ public abstract class MiniRaftCluster {
 
   public String printAllLogs() {
     StringBuilder b = new StringBuilder("\n#servers = " + servers.size() + "\n");
-    for (RaftServer s : servers.values()) {
+    for (RaftServerImpl s : servers.values()) {
       b.append("  ");
       b.append(s).append("\n");
 
@@ -339,8 +339,8 @@ public abstract class MiniRaftCluster {
     return b.toString();
   }
 
-  public RaftServer getLeader() {
-    final List<RaftServer> leaders = new ArrayList<>();
+  public RaftServerImpl getLeader() {
+    final List<RaftServerImpl> leaders = new ArrayList<>();
     servers.values().stream()
         .filter(s -> s.isAlive() && s.isLeader())
         .forEach(s -> {
@@ -367,21 +367,21 @@ public abstract class MiniRaftCluster {
   }
 
   public boolean isLeader(String leaderId) throws InterruptedException {
-    final RaftServer leader = getLeader();
+    final RaftServerImpl leader = getLeader();
     return leader != null && leader.getId().equals(leaderId);
   }
 
-  public List<RaftServer> getFollowers() {
+  public List<RaftServerImpl> getFollowers() {
     return servers.values().stream()
         .filter(s -> s.isAlive() && s.isFollower())
         .collect(Collectors.toList());
   }
 
-  public Collection<RaftServer> getServers() {
+  public Collection<RaftServerImpl> getServers() {
     return servers.values();
   }
 
-  public RaftServer getServer(String id) {
+  public RaftServerImpl getServer(String id) {
     return servers.get(id);
   }
 
@@ -398,8 +398,8 @@ public abstract class MiniRaftCluster {
 
   public void shutdown() {
     LOG.info("Stopping " + getClass().getSimpleName());
-    servers.values().stream().filter(RaftServer::isAlive)
-        .forEach(RaftServer::close);
+    servers.values().stream().filter(RaftServerImpl::isAlive)
+        .forEach(RaftServerImpl::close);
 
     if (ExitUtils.isTerminated()) {
       LOG.error("Test resulted in an unexpected exit",

http://git-wip-us.apache.org/repos/asf/incubator-ratis/blob/22f3ee0b/raft-server/src/test/java/org/apache/raft/RaftBasicTests.java
----------------------------------------------------------------------
diff --git a/raft-server/src/test/java/org/apache/raft/RaftBasicTests.java b/raft-server/src/test/java/org/apache/raft/RaftBasicTests.java
index 921e063..ed40bde 100644
--- a/raft-server/src/test/java/org/apache/raft/RaftBasicTests.java
+++ b/raft-server/src/test/java/org/apache/raft/RaftBasicTests.java
@@ -20,7 +20,7 @@ package org.apache.raft;
 import org.apache.raft.RaftTestUtil.SimpleMessage;
 import org.apache.raft.client.RaftClient;
 import org.apache.raft.conf.RaftProperties;
-import org.apache.raft.server.impl.RaftServer;
+import org.apache.raft.server.impl.RaftServerImpl;
 import org.junit.*;
 import org.junit.rules.Timeout;
 import org.slf4j.Logger;
@@ -80,7 +80,7 @@ public abstract class RaftBasicTests {
   public void testBasicAppendEntries() throws Exception {
     LOG.info("Running testBasicAppendEntries");
     final MiniRaftCluster cluster = getCluster();
-    RaftServer leader = waitForLeader(cluster);
+    RaftServerImpl leader = waitForLeader(cluster);
     final long term = leader.getState().getCurrentTerm();
     final String killed = cluster.getFollowers().get(3).getId();
     cluster.killServer(killed);
@@ -96,7 +96,7 @@ public abstract class RaftBasicTests {
     Thread.sleep(cluster.getMaxTimeout() + 100);
     LOG.info(cluster.printAllLogs());
 
-    cluster.getServers().stream().filter(RaftServer::isAlive)
+    cluster.getServers().stream().filter(RaftServerImpl::isAlive)
         .map(s -> s.getState().getLog().getEntries(1, Long.MAX_VALUE))
         .forEach(e -> RaftTestUtil.assertLogEntries(e, 1, term, messages));
   }
@@ -174,7 +174,7 @@ public abstract class RaftBasicTests {
       lastStep = n;
       count++;
 
-      RaftServer leader = cluster.getLeader();
+      RaftServerImpl leader = cluster.getLeader();
       if (leader != null) {
         final String oldLeader = leader.getId();
         LOG.info("Block all requests sent by leader " + oldLeader);

http://git-wip-us.apache.org/repos/asf/incubator-ratis/blob/22f3ee0b/raft-server/src/test/java/org/apache/raft/RaftNotLeaderExceptionBaseTest.java
----------------------------------------------------------------------
diff --git a/raft-server/src/test/java/org/apache/raft/RaftNotLeaderExceptionBaseTest.java b/raft-server/src/test/java/org/apache/raft/RaftNotLeaderExceptionBaseTest.java
index 8a249e9..195cbec 100644
--- a/raft-server/src/test/java/org/apache/raft/RaftNotLeaderExceptionBaseTest.java
+++ b/raft-server/src/test/java/org/apache/raft/RaftNotLeaderExceptionBaseTest.java
@@ -25,7 +25,7 @@ import org.apache.raft.client.impl.RaftClientImpl;
 import org.apache.raft.protocol.RaftClientReply;
 import org.apache.raft.protocol.RaftClientRequest;
 import org.apache.raft.protocol.RaftPeer;
-import org.apache.raft.server.impl.RaftServer;
+import org.apache.raft.server.impl.RaftServerImpl;
 import org.apache.raft.server.simulation.RequestHandler;
 import org.apache.raft.server.storage.RaftLog;
 import org.apache.raft.util.RaftUtils;
@@ -42,7 +42,7 @@ import static org.apache.raft.server.impl.RaftServerConstants.DEFAULT_SEQNUM;
 
 public abstract class RaftNotLeaderExceptionBaseTest {
   static {
-    RaftUtils.setLogLevel(RaftServer.LOG, Level.DEBUG);
+    RaftUtils.setLogLevel(RaftServerImpl.LOG, Level.DEBUG);
     RaftUtils.setLogLevel(RaftLog.LOG, Level.DEBUG);
     RaftUtils.setLogLevel(RequestHandler.LOG, Level.DEBUG);
     RaftUtils.setLogLevel(RaftClient.LOG, Level.DEBUG);

http://git-wip-us.apache.org/repos/asf/incubator-ratis/blob/22f3ee0b/raft-server/src/test/java/org/apache/raft/RaftTestUtil.java
----------------------------------------------------------------------
diff --git a/raft-server/src/test/java/org/apache/raft/RaftTestUtil.java b/raft-server/src/test/java/org/apache/raft/RaftTestUtil.java
index 92bf5c4..461dd15 100644
--- a/raft-server/src/test/java/org/apache/raft/RaftTestUtil.java
+++ b/raft-server/src/test/java/org/apache/raft/RaftTestUtil.java
@@ -23,7 +23,7 @@ import org.apache.raft.protocol.Message;
 import org.apache.raft.server.RaftServerConfigKeys;
 import org.apache.raft.server.impl.BlockRequestHandlingInjection;
 import org.apache.raft.server.impl.DelayLocalExecutionInjection;
-import org.apache.raft.server.impl.RaftServer;
+import org.apache.raft.server.impl.RaftServerImpl;
 import org.apache.raft.shaded.com.google.protobuf.ByteString;
 import org.apache.raft.shaded.proto.RaftProtos.LogEntryProto;
 import org.apache.raft.shaded.proto.RaftProtos.SMLogEntryProto;
@@ -46,11 +46,11 @@ import static org.apache.raft.util.ProtoUtils.toByteString;
 public class RaftTestUtil {
   static final Logger LOG = LoggerFactory.getLogger(RaftTestUtil.class);
 
-  public static RaftServer waitForLeader(MiniRaftCluster cluster)
+  public static RaftServerImpl waitForLeader(MiniRaftCluster cluster)
       throws InterruptedException {
     final long sleepTime = (cluster.getMaxTimeout() * 3) >> 1;
     LOG.info(cluster.printServers());
-    RaftServer leader = null;
+    RaftServerImpl leader = null;
     for(int i = 0; leader == null && i < 10; i++) {
       Thread.sleep(sleepTime);
       leader = cluster.getLeader();
@@ -59,11 +59,11 @@ public class RaftTestUtil {
     return leader;
   }
 
-  public static RaftServer waitForLeader(MiniRaftCluster cluster,
-      final String leaderId) throws InterruptedException {
+  public static RaftServerImpl waitForLeader(MiniRaftCluster cluster,
+                                             final String leaderId) throws InterruptedException {
     LOG.info(cluster.printServers());
     for(int i = 0; !cluster.tryEnforceLeader(leaderId) && i < 10; i++) {
-      RaftServer currLeader = cluster.getLeader();
+      RaftServerImpl currLeader = cluster.getLeader();
       if (LOG.isDebugEnabled()) {
         LOG.debug("try enforcing leader to " + leaderId + " but "
             + (currLeader == null? "no leader for this round"
@@ -72,14 +72,14 @@ public class RaftTestUtil {
     }
     LOG.info(cluster.printServers());
 
-    final RaftServer leader = cluster.getLeader();
+    final RaftServerImpl leader = cluster.getLeader();
     Assert.assertEquals(leaderId, leader.getId());
     return leader;
   }
 
   public static String waitAndKillLeader(MiniRaftCluster cluster,
       boolean expectLeader) throws InterruptedException {
-    final RaftServer leader = waitForLeader(cluster);
+    final RaftServerImpl leader = waitForLeader(cluster);
     if (!expectLeader) {
       Assert.assertNull(leader);
     } else {
@@ -105,11 +105,11 @@ public class RaftTestUtil {
     return idxExpected == expectedMessages.length;
   }
 
-  public static void assertLogEntries(Collection<RaftServer> servers,
+  public static void assertLogEntries(Collection<RaftServerImpl> servers,
                                       SimpleMessage... expectedMessages) {
     final int size = servers.size();
     final long count = servers.stream()
-        .filter(RaftServer::isAlive)
+        .filter(RaftServerImpl::isAlive)
         .map(s -> s.getState().getLog().getEntries(0, Long.MAX_VALUE))
         .filter(e -> logEntriesContains(e, expectedMessages))
         .count();
@@ -269,7 +269,7 @@ public class RaftTestUtil {
     return newLeader;
   }
 
-  public static void blockQueueAndSetDelay(Collection<RaftServer> servers,
+  public static void blockQueueAndSetDelay(Collection<RaftServerImpl> servers,
       DelayLocalExecutionInjection injection, String leaderId, int delayMs,
       long maxTimeout) throws InterruptedException {
     // block reqeusts sent to leader if delayMs > 0

http://git-wip-us.apache.org/repos/asf/incubator-ratis/blob/22f3ee0b/raft-server/src/test/java/org/apache/raft/server/impl/BlockRequestHandlingInjection.java
----------------------------------------------------------------------
diff --git a/raft-server/src/test/java/org/apache/raft/server/impl/BlockRequestHandlingInjection.java b/raft-server/src/test/java/org/apache/raft/server/impl/BlockRequestHandlingInjection.java
index 0980e93..7f7de9a 100644
--- a/raft-server/src/test/java/org/apache/raft/server/impl/BlockRequestHandlingInjection.java
+++ b/raft-server/src/test/java/org/apache/raft/server/impl/BlockRequestHandlingInjection.java
@@ -29,9 +29,9 @@ public class BlockRequestHandlingInjection implements CodeInjectionForTesting.Co
       new BlockRequestHandlingInjection();
 
   static {
-    CodeInjectionForTesting.put(RaftServer.REQUEST_VOTE, INSTANCE);
-    CodeInjectionForTesting.put(RaftServer.APPEND_ENTRIES, INSTANCE);
-    CodeInjectionForTesting.put(RaftServer.INSTALL_SNAPSHOT, INSTANCE);
+    CodeInjectionForTesting.put(RaftServerImpl.REQUEST_VOTE, INSTANCE);
+    CodeInjectionForTesting.put(RaftServerImpl.APPEND_ENTRIES, INSTANCE);
+    CodeInjectionForTesting.put(RaftServerImpl.INSTALL_SNAPSHOT, INSTANCE);
   }
 
   public static BlockRequestHandlingInjection getInstance() {

http://git-wip-us.apache.org/repos/asf/incubator-ratis/blob/22f3ee0b/raft-server/src/test/java/org/apache/raft/server/impl/RaftReconfigurationBaseTest.java
----------------------------------------------------------------------
diff --git a/raft-server/src/test/java/org/apache/raft/server/impl/RaftReconfigurationBaseTest.java b/raft-server/src/test/java/org/apache/raft/server/impl/RaftReconfigurationBaseTest.java
index 30f1e15..8a5af69 100644
--- a/raft-server/src/test/java/org/apache/raft/server/impl/RaftReconfigurationBaseTest.java
+++ b/raft-server/src/test/java/org/apache/raft/server/impl/RaftReconfigurationBaseTest.java
@@ -54,7 +54,7 @@ import static org.apache.raft.shaded.proto.RaftProtos.LogEntryProto.LogEntryBody
 
 public abstract class RaftReconfigurationBaseTest {
   static {
-    RaftUtils.setLogLevel(RaftServer.LOG, Level.DEBUG);
+    RaftUtils.setLogLevel(RaftServerImpl.LOG, Level.DEBUG);
     RaftUtils.setLogLevel(RequestHandler.LOG, Level.DEBUG);
     RaftUtils.setLogLevel(RaftClient.LOG, Level.DEBUG);
   }
@@ -221,7 +221,7 @@ public abstract class RaftReconfigurationBaseTest {
 
       // check configuration manager's internal state
       // each reconf will generate two configurations: (old, new) and (new)
-      cluster.getServers().stream().filter(RaftServer::isAlive)
+      cluster.getServers().stream().filter(RaftServerImpl::isAlive)
           .forEach(server -> {
         ConfigurationManager confManager =
             (ConfigurationManager) Whitebox.getInternalState(server.getState(),

http://git-wip-us.apache.org/repos/asf/incubator-ratis/blob/22f3ee0b/raft-server/src/test/java/org/apache/raft/server/impl/RaftServerTestUtil.java
----------------------------------------------------------------------
diff --git a/raft-server/src/test/java/org/apache/raft/server/impl/RaftServerTestUtil.java b/raft-server/src/test/java/org/apache/raft/server/impl/RaftServerTestUtil.java
index b30ddc9..5103fca 100644
--- a/raft-server/src/test/java/org/apache/raft/server/impl/RaftServerTestUtil.java
+++ b/raft-server/src/test/java/org/apache/raft/server/impl/RaftServerTestUtil.java
@@ -46,7 +46,7 @@ public class RaftServerTestUtil {
     int deadIncluded = 0;
     final RaftConfiguration current = RaftConfiguration.newBuilder()
         .setConf(peers).setLogEntryIndex(0).build();
-    for (RaftServer server : cluster.getServers()) {
+    for (RaftServerImpl server : cluster.getServers()) {
       if (deadPeers != null && deadPeers.contains(server.getId())) {
         if (current.containsInConf(server.getId())) {
           deadIncluded++;

http://git-wip-us.apache.org/repos/asf/incubator-ratis/blob/22f3ee0b/raft-server/src/test/java/org/apache/raft/server/simulation/MiniRaftClusterWithSimulatedRpc.java
----------------------------------------------------------------------
diff --git a/raft-server/src/test/java/org/apache/raft/server/simulation/MiniRaftClusterWithSimulatedRpc.java b/raft-server/src/test/java/org/apache/raft/server/simulation/MiniRaftClusterWithSimulatedRpc.java
index 360fe1e..7414872 100644
--- a/raft-server/src/test/java/org/apache/raft/server/simulation/MiniRaftClusterWithSimulatedRpc.java
+++ b/raft-server/src/test/java/org/apache/raft/server/simulation/MiniRaftClusterWithSimulatedRpc.java
@@ -21,7 +21,7 @@ import org.apache.raft.MiniRaftCluster;
 import org.apache.raft.client.RaftClientRequestSender;
 import org.apache.raft.conf.RaftProperties;
 import org.apache.raft.protocol.RaftPeer;
-import org.apache.raft.server.impl.RaftServer;
+import org.apache.raft.server.impl.RaftServerImpl;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
@@ -70,7 +70,7 @@ public class MiniRaftClusterWithSimulatedRpc extends MiniRaftCluster {
     setRpcServers(getServers());
   }
 
-  private void setRpcServers(Collection<RaftServer> newServers) {
+  private void setRpcServers(Collection<RaftServerImpl> newServers) {
     newServers.forEach(s -> s.setServerRpc(
         new SimulatedServerRpc(s, serverRequestReply, client2serverRequestReply)));
   }
@@ -88,7 +88,7 @@ public class MiniRaftClusterWithSimulatedRpc extends MiniRaftCluster {
   @Override
   public void restartServer(String id, boolean format) throws IOException {
     super.restartServer(id, format);
-    RaftServer s = getServer(id);
+    RaftServerImpl s = getServer(id);
     addPeersToRpc(Collections.singletonList(conf.getPeer(id)));
     s.setServerRpc(new SimulatedServerRpc(s, serverRequestReply,
         client2serverRequestReply));
@@ -97,11 +97,11 @@ public class MiniRaftClusterWithSimulatedRpc extends MiniRaftCluster {
 
   @Override
   public Collection<RaftPeer> addNewPeers(Collection<RaftPeer> newPeers,
-      Collection<RaftServer> newServers, boolean startService) {
+                                          Collection<RaftServerImpl> newServers, boolean startService) {
     addPeersToRpc(newPeers);
     setRpcServers(newServers);
     if (startService) {
-      newServers.forEach(RaftServer::start);
+      newServers.forEach(RaftServerImpl::start);
     }
     return newPeers;
   }

http://git-wip-us.apache.org/repos/asf/incubator-ratis/blob/22f3ee0b/raft-server/src/test/java/org/apache/raft/server/simulation/SimulatedServerRpc.java
----------------------------------------------------------------------
diff --git a/raft-server/src/test/java/org/apache/raft/server/simulation/SimulatedServerRpc.java b/raft-server/src/test/java/org/apache/raft/server/simulation/SimulatedServerRpc.java
index ed522d4..93e3f5c 100644
--- a/raft-server/src/test/java/org/apache/raft/server/simulation/SimulatedServerRpc.java
+++ b/raft-server/src/test/java/org/apache/raft/server/simulation/SimulatedServerRpc.java
@@ -22,7 +22,7 @@ import org.apache.raft.protocol.RaftClientReply;
 import org.apache.raft.protocol.RaftClientRequest;
 import org.apache.raft.protocol.RaftPeer;
 import org.apache.raft.protocol.SetConfigurationRequest;
-import org.apache.raft.server.impl.RaftServer;
+import org.apache.raft.server.impl.RaftServerImpl;
 import org.apache.raft.server.impl.RaftServerRpc;
 import org.apache.raft.server.impl.RequestDispatcher;
 import org.apache.raft.shaded.proto.RaftProtos.*;
@@ -39,14 +39,14 @@ import java.util.concurrent.TimeUnit;
 public class SimulatedServerRpc implements RaftServerRpc {
   static final Logger LOG = LoggerFactory.getLogger(SimulatedServerRpc.class);
 
-  private final RaftServer server;
+  private final RaftServerImpl server;
   private final RequestDispatcher dispatcher;
   private final RequestHandler<RaftServerRequest, RaftServerReply> serverHandler;
   private final RequestHandler<RaftClientRequest, RaftClientReply> clientHandler;
   private final ExecutorService executor = Executors.newFixedThreadPool(3,
       new ThreadFactoryBuilder().setDaemon(true).build());
 
-  public SimulatedServerRpc(RaftServer server,
+  public SimulatedServerRpc(RaftServerImpl server,
       SimulatedRequestReply<RaftServerRequest, RaftServerReply> serverRequestReply,
       SimulatedRequestReply<RaftClientRequest, RaftClientReply> clientRequestReply) {
     this.server = server;

http://git-wip-us.apache.org/repos/asf/incubator-ratis/blob/22f3ee0b/raft-server/src/test/java/org/apache/raft/server/simulation/TestRaftWithSimulatedRpc.java
----------------------------------------------------------------------
diff --git a/raft-server/src/test/java/org/apache/raft/server/simulation/TestRaftWithSimulatedRpc.java b/raft-server/src/test/java/org/apache/raft/server/simulation/TestRaftWithSimulatedRpc.java
index 669226a..faa9dd8 100644
--- a/raft-server/src/test/java/org/apache/raft/server/simulation/TestRaftWithSimulatedRpc.java
+++ b/raft-server/src/test/java/org/apache/raft/server/simulation/TestRaftWithSimulatedRpc.java
@@ -21,7 +21,7 @@ import org.apache.log4j.Level;
 import org.apache.raft.RaftBasicTests;
 import org.apache.raft.client.RaftClient;
 import org.apache.raft.conf.RaftProperties;
-import org.apache.raft.server.impl.RaftServer;
+import org.apache.raft.server.impl.RaftServerImpl;
 import org.apache.raft.util.RaftUtils;
 
 import java.io.IOException;
@@ -29,7 +29,7 @@ import java.util.concurrent.ThreadLocalRandom;
 
 public class TestRaftWithSimulatedRpc extends RaftBasicTests {
   static {
-    RaftUtils.setLogLevel(RaftServer.LOG, Level.DEBUG);
+    RaftUtils.setLogLevel(RaftServerImpl.LOG, Level.DEBUG);
     RaftUtils.setLogLevel(RaftClient.LOG, Level.DEBUG);
   }
 

http://git-wip-us.apache.org/repos/asf/incubator-ratis/blob/22f3ee0b/raft-server/src/test/java/org/apache/raft/statemachine/RaftSnapshotBaseTest.java
----------------------------------------------------------------------
diff --git a/raft-server/src/test/java/org/apache/raft/statemachine/RaftSnapshotBaseTest.java b/raft-server/src/test/java/org/apache/raft/statemachine/RaftSnapshotBaseTest.java
index 41ae9af..721d12c 100644
--- a/raft-server/src/test/java/org/apache/raft/statemachine/RaftSnapshotBaseTest.java
+++ b/raft-server/src/test/java/org/apache/raft/statemachine/RaftSnapshotBaseTest.java
@@ -25,7 +25,7 @@ import org.apache.raft.client.RaftClient;
 import org.apache.raft.conf.RaftProperties;
 import org.apache.raft.protocol.RaftClientReply;
 import org.apache.raft.protocol.SetConfigurationRequest;
-import org.apache.raft.server.impl.RaftServer;
+import org.apache.raft.server.impl.RaftServerImpl;
 import org.apache.raft.server.impl.RaftServerTestUtil;
 import org.apache.raft.server.simulation.RequestHandler;
 import org.apache.raft.server.storage.RaftLog;
@@ -52,7 +52,7 @@ import static org.apache.raft.server.impl.RaftServerConstants.DEFAULT_SEQNUM;
 
 public abstract class RaftSnapshotBaseTest {
   static {
-    RaftUtils.setLogLevel(RaftServer.LOG, Level.DEBUG);
+    RaftUtils.setLogLevel(RaftServerImpl.LOG, Level.DEBUG);
     RaftUtils.setLogLevel(RaftLog.LOG, Level.DEBUG);
     RaftUtils.setLogLevel(RequestHandler.LOG, Level.DEBUG);
     RaftUtils.setLogLevel(RaftClient.LOG, Level.DEBUG);

http://git-wip-us.apache.org/repos/asf/incubator-ratis/blob/22f3ee0b/raft-server/src/test/java/org/apache/raft/statemachine/TestStateMachine.java
----------------------------------------------------------------------
diff --git a/raft-server/src/test/java/org/apache/raft/statemachine/TestStateMachine.java b/raft-server/src/test/java/org/apache/raft/statemachine/TestStateMachine.java
index c9dd99c..5892c65 100644
--- a/raft-server/src/test/java/org/apache/raft/statemachine/TestStateMachine.java
+++ b/raft-server/src/test/java/org/apache/raft/statemachine/TestStateMachine.java
@@ -25,7 +25,7 @@ import org.apache.raft.conf.RaftProperties;
 import org.apache.raft.protocol.Message;
 import org.apache.raft.protocol.RaftClientRequest;
 import org.apache.raft.server.RaftServerConfigKeys;
-import org.apache.raft.server.impl.RaftServer;
+import org.apache.raft.server.impl.RaftServerImpl;
 import org.apache.raft.server.simulation.MiniRaftClusterWithSimulatedRpc;
 import org.apache.raft.shaded.proto.RaftProtos.SMLogEntryProto;
 import org.apache.raft.util.RaftUtils;
@@ -50,7 +50,7 @@ import static org.junit.Assert.*;
  */
 public class TestStateMachine {
   static {
-    RaftUtils.setLogLevel(RaftServer.LOG, Level.DEBUG);
+    RaftUtils.setLogLevel(RaftServerImpl.LOG, Level.DEBUG);
     RaftUtils.setLogLevel(RaftClient.LOG, Level.DEBUG);
   }
 
@@ -161,14 +161,14 @@ public class TestStateMachine {
     // TODO: there eshould be a better way to ensure all data is replicated and applied
     Thread.sleep(cluster.getMaxTimeout() + 100);
 
-    for (RaftServer raftServer : cluster.getServers()) {
+    for (RaftServerImpl raftServer : cluster.getServers()) {
       SMTransactionContext sm = ((SMTransactionContext)raftServer.getStateMachine());
       sm.rethrowIfException();
       assertEquals(numTrx, sm.numApplied.get());
     }
 
     // check leader
-    RaftServer raftServer = cluster.getLeader();
+    RaftServerImpl raftServer = cluster.getLeader();
     // assert every transaction has obtained context in leader
     SMTransactionContext sm = ((SMTransactionContext)raftServer.getStateMachine());
     List<Long> ll = sm.applied.stream().collect(Collectors.toList());