You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@ratis.apache.org by sz...@apache.org on 2018/11/13 23:39:22 UTC

[1/6] incubator-ratis git commit: RATIS-399. Move all tests to a new module.

Repository: incubator-ratis
Updated Branches:
  refs/heads/master d28b6493f -> 5c37675fa


http://git-wip-us.apache.org/repos/asf/incubator-ratis/blob/5c37675f/ratis-test/src/test/java/org/apache/ratis/statemachine/TestStateMachine.java
----------------------------------------------------------------------
diff --git a/ratis-test/src/test/java/org/apache/ratis/statemachine/TestStateMachine.java b/ratis-test/src/test/java/org/apache/ratis/statemachine/TestStateMachine.java
new file mode 100644
index 0000000..a4dc88a
--- /dev/null
+++ b/ratis-test/src/test/java/org/apache/ratis/statemachine/TestStateMachine.java
@@ -0,0 +1,201 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS 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.BaseTest;
+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.protocol.RaftGroup;
+import org.apache.ratis.protocol.RaftGroupId;
+import org.apache.ratis.protocol.RaftPeer;
+import org.apache.ratis.protocol.RaftPeerId;
+import org.apache.ratis.server.RaftServerConfigKeys;
+import org.apache.ratis.server.impl.RaftServerImpl;
+import org.apache.ratis.server.impl.RaftServerProxy;
+import org.apache.ratis.server.impl.RaftServerTestUtil;
+import org.apache.ratis.server.simulation.MiniRaftClusterWithSimulatedRpc;
+import org.apache.ratis.util.LogUtils;
+import org.junit.*;
+
+import java.io.IOException;
+import java.util.Collections;
+import java.util.List;
+import java.util.Map;
+import java.util.concurrent.CompletableFuture;
+import java.util.concurrent.ConcurrentHashMap;
+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 extends BaseTest implements MiniRaftClusterWithSimulatedRpc.FactoryGet {
+  static {
+    LogUtils.setLogLevel(RaftServerImpl.LOG, Level.DEBUG);
+    LogUtils.setLogLevel(RaftClient.LOG, Level.DEBUG);
+  }
+
+  public static final int NUM_SERVERS = 3;
+
+  static class SMTransactionContext extends SimpleStateMachine4Testing {
+    public static SMTransactionContext get(RaftServerImpl s) {
+      return (SMTransactionContext)s.getStateMachine();
+    }
+
+    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) {
+      // only leader will get this call
+      isLeader.set(true);
+      // send the next transaction id as the "context" from SM
+      return TransactionContext.newBuilder()
+          .setStateMachine(this)
+          .setClientRequest(request)
+          .setStateMachineContext(transactions.incrementAndGet())
+          .build();
+    }
+
+    @Override
+    public CompletableFuture<Message> applyTransaction(TransactionContext trx) {
+      try {
+        assertNotNull(trx.getLogEntry());
+        assertNotNull(trx.getStateMachineLogEntry());
+        Object context = trx.getStateMachineContext();
+        if (isLeader.get()) {
+          assertNotNull(trx.getClientRequest());
+          assertNotNull(context);
+          assertTrue(context instanceof Long);
+          Long val = (Long)context;
+          assertTrue(val <= transactions.get());
+          applied.add(val);
+        } else {
+          assertNull(trx.getClientRequest());
+          assertNull(context);
+        }
+        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 {
+    runTestTransactionContextIsPassedBack(false);
+  }
+
+  @Test
+  public void testTransactionContextIsPassedBackUseMemory() throws Throwable {
+    runTestTransactionContextIsPassedBack(true);
+  }
+
+  void runTestTransactionContextIsPassedBack(boolean useMemory) throws Throwable {
+    final RaftProperties properties = new RaftProperties();
+    properties.setClass(MiniRaftCluster.STATEMACHINE_CLASS_KEY, SMTransactionContext.class, StateMachine.class);
+    RaftServerConfigKeys.Log.setUseMemory(properties, useMemory);
+
+    try(MiniRaftClusterWithSimulatedRpc cluster = getFactory().newCluster(NUM_SERVERS, properties)) {
+      cluster.start();
+      runTestTransactionContextIsPassedBack(cluster);
+    }
+  }
+
+  static void runTestTransactionContextIsPassedBack(MiniRaftCluster cluster) throws Throwable {
+    // tests that the TrxContext set by the StateMachine in Leader is passed back to the SM
+    int numTrx = 100;
+    final RaftTestUtil.SimpleMessage[] messages = RaftTestUtil.SimpleMessage.create(numTrx);
+    try(final RaftClient client = cluster.createClient()) {
+      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.iterateServerImpls()) {
+      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));
+    }
+  }
+
+  @Test
+  public void testStateMachineRegistry() throws Throwable {
+    final Map<RaftGroupId, StateMachine> registry = new ConcurrentHashMap<>();
+    registry.put(RaftGroupId.randomId(), new SimpleStateMachine4Testing());
+    registry.put(RaftGroupId.randomId(), new SMTransactionContext());
+
+    try(MiniRaftClusterWithSimulatedRpc cluster = newCluster(0)) {
+      cluster.setStateMachineRegistry(registry::get);
+
+      final RaftPeerId id = RaftPeerId.valueOf("s0");
+      cluster.putNewServer(id, null, true);
+      cluster.start();
+
+      for(RaftGroupId gid : registry.keySet()) {
+        final RaftGroup newGroup = RaftGroup.valueOf(gid, cluster.getPeers());
+        LOG.info("add new group: " + newGroup);
+        final RaftClient client = cluster.createClient(newGroup);
+        for(RaftPeer p : newGroup.getPeers()) {
+          client.groupAdd(newGroup, p.getId());
+        }
+      }
+
+      final RaftServerProxy proxy = cluster.getServer(id);
+      for(Map.Entry<RaftGroupId, StateMachine> e: registry.entrySet()) {
+        final RaftServerImpl impl = RaftServerTestUtil.getRaftServerImpl(proxy, e.getKey());
+        Assert.assertSame(e.getValue(), impl.getStateMachine());
+      }
+    }
+  }
+}

http://git-wip-us.apache.org/repos/asf/incubator-ratis/blob/5c37675f/ratis-test/src/test/java/org/apache/ratis/util/TestLifeCycle.java
----------------------------------------------------------------------
diff --git a/ratis-test/src/test/java/org/apache/ratis/util/TestLifeCycle.java b/ratis-test/src/test/java/org/apache/ratis/util/TestLifeCycle.java
new file mode 100644
index 0000000..9782792
--- /dev/null
+++ b/ratis-test/src/test/java/org/apache/ratis/util/TestLifeCycle.java
@@ -0,0 +1,53 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS 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.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(timeout = 1000)
+  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/5c37675f/ratis-test/src/test/java/org/apache/ratis/util/TestMinMax.java
----------------------------------------------------------------------
diff --git a/ratis-test/src/test/java/org/apache/ratis/util/TestMinMax.java b/ratis-test/src/test/java/org/apache/ratis/util/TestMinMax.java
new file mode 100644
index 0000000..8d315b7
--- /dev/null
+++ b/ratis-test/src/test/java/org/apache/ratis/util/TestMinMax.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 org.junit.Assert;
+import org.junit.Test;
+
+import java.util.ArrayList;
+import java.util.List;
+import java.util.OptionalLong;
+import java.util.concurrent.ThreadLocalRandom;
+import java.util.stream.LongStream;
+
+public class TestMinMax {
+  @Test(timeout = 1000)
+  public void testMinMax() {
+    runTestMinMax(LongStream.empty());
+    runTestMinMax(LongStream.iterate(0, n -> n).limit(10));
+    for(int count = 1; count < 10; count++) {
+      runTestMinMax(LongStream.iterate(1, n -> n + 1).limit(count));
+    }
+    for(int count = 1; count < 10; count++) {
+      runTestMinMax(LongStream.iterate(0, _dummy -> ThreadLocalRandom.current().nextLong()).limit(count));
+    }
+  }
+
+  static void runTestMinMax(LongStream stream) {
+    final List<Long> list = stream.collect(ArrayList::new, List::add, List::addAll);
+    final LongMinMax longMinMax = toLongStream(list).collect(LongMinMax::new, LongMinMax::accumulate, LongMinMax::combine);
+    if (longMinMax.isInitialized()) {
+      Assert.assertEquals(toLongStream(list).min().getAsLong(), longMinMax.getMin());
+      Assert.assertEquals(toLongStream(list).max().getAsLong(), longMinMax.getMax());
+    } else {
+      Assert.assertEquals(OptionalLong.empty(), toLongStream(list).min());
+      Assert.assertEquals(OptionalLong.empty(), toLongStream(list).max());
+    }
+  }
+
+  static LongStream toLongStream(List<Long> list) {
+    return list.stream().mapToLong(Long::longValue);
+  }
+}

http://git-wip-us.apache.org/repos/asf/incubator-ratis/blob/5c37675f/ratis-test/src/test/java/org/apache/ratis/util/TestTimeDuration.java
----------------------------------------------------------------------
diff --git a/ratis-test/src/test/java/org/apache/ratis/util/TestTimeDuration.java b/ratis-test/src/test/java/org/apache/ratis/util/TestTimeDuration.java
new file mode 100644
index 0000000..06d9301
--- /dev/null
+++ b/ratis-test/src/test/java/org/apache/ratis/util/TestTimeDuration.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;
+
+import org.junit.Test;
+
+import java.util.Arrays;
+import java.util.List;
+import java.util.concurrent.TimeUnit;
+import java.util.stream.Collectors;
+
+import static org.apache.ratis.util.TimeDuration.Abbreviation;
+import static org.apache.ratis.util.TimeDuration.parse;
+import static org.junit.Assert.assertEquals;
+import static org.junit.Assert.assertNotNull;
+
+public class TestTimeDuration {
+  @Test(timeout = 1000)
+  public void testTimeDuration() throws Exception {
+    Arrays.asList(TimeUnit.values())
+        .forEach(a -> assertNotNull(Abbreviation.valueOf(a.name())));
+    assertEquals(TimeUnit.values().length, Abbreviation.values().length);
+
+    final List<String> allSymbols = Arrays.asList(Abbreviation.values()).stream()
+        .map(Abbreviation::getSymbols)
+        .flatMap(List::stream)
+        .collect(Collectors.toList());
+    Arrays.asList(TimeUnit.values()).forEach(unit ->
+        allSymbols.stream()
+            .map(s -> "0" + s)
+            .forEach(s -> assertEquals(s, 0L, parse(s, unit))));
+
+    assertEquals(1L, parse("1000000 ns", TimeUnit.MILLISECONDS));
+    assertEquals(10L, parse("10000000 nanos", TimeUnit.MILLISECONDS));
+    assertEquals(100L, parse("100000000 nanosecond", TimeUnit.MILLISECONDS));
+    assertEquals(1000L, parse("1000000000 nanoseconds", TimeUnit.MILLISECONDS));
+
+    assertEquals(1L, parse("1000 us", TimeUnit.MILLISECONDS));
+    assertEquals(10L, parse("10000 μs", TimeUnit.MILLISECONDS));
+    assertEquals(100L, parse("100000 micros", TimeUnit.MILLISECONDS));
+    assertEquals(1000L, parse("1000000 microsecond", TimeUnit.MILLISECONDS));
+    assertEquals(10000L, parse("10000000 microseconds", TimeUnit.MILLISECONDS));
+
+    assertEquals(1L, parse("1 ms", TimeUnit.MILLISECONDS));
+    assertEquals(10L, parse("10 msec", TimeUnit.MILLISECONDS));
+    assertEquals(100L, parse("100 millis", TimeUnit.MILLISECONDS));
+    assertEquals(1000L, parse("1000 millisecond", TimeUnit.MILLISECONDS));
+    assertEquals(10000L, parse("10000 milliseconds", TimeUnit.MILLISECONDS));
+
+    assertEquals(1000L, parse("1 s", TimeUnit.MILLISECONDS));
+    assertEquals(10000L, parse("10 sec", TimeUnit.MILLISECONDS));
+    assertEquals(100000L, parse("100 second", TimeUnit.MILLISECONDS));
+    assertEquals(1000000L, parse("1000 seconds", TimeUnit.MILLISECONDS));
+
+    assertEquals(60, parse("1 m", TimeUnit.SECONDS));
+    assertEquals(600, parse("10 min", TimeUnit.SECONDS));
+    assertEquals(6000, parse("100 minutes", TimeUnit.SECONDS));
+    assertEquals(60000, parse("1000 minutes", TimeUnit.SECONDS));
+
+    assertEquals(60, parse("1 h", TimeUnit.MINUTES));
+    assertEquals(600, parse("10 hr", TimeUnit.MINUTES));
+    assertEquals(6000, parse("100 hour", TimeUnit.MINUTES));
+    assertEquals(60000, parse("1000 hours", TimeUnit.MINUTES));
+
+    assertEquals(24, parse("1 d", TimeUnit.HOURS));
+    assertEquals(240, parse("10 day", TimeUnit.HOURS));
+    assertEquals(2400, parse("100 days", TimeUnit.HOURS));
+  }
+}

http://git-wip-us.apache.org/repos/asf/incubator-ratis/blob/5c37675f/ratis-test/src/test/java/org/apache/ratis/util/TestTimeoutScheduler.java
----------------------------------------------------------------------
diff --git a/ratis-test/src/test/java/org/apache/ratis/util/TestTimeoutScheduler.java b/ratis-test/src/test/java/org/apache/ratis/util/TestTimeoutScheduler.java
new file mode 100644
index 0000000..6a63569
--- /dev/null
+++ b/ratis-test/src/test/java/org/apache/ratis/util/TestTimeoutScheduler.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 org.apache.log4j.Level;
+import org.junit.Assert;
+import org.junit.Test;
+
+import java.util.concurrent.TimeUnit;
+import java.util.concurrent.atomic.AtomicBoolean;
+import java.util.function.Consumer;
+
+public class TestTimeoutScheduler {
+  {
+    LogUtils.setLogLevel(TimeoutScheduler.LOG, Level.ALL);
+  }
+
+  static class ErrorHandler implements Consumer<RuntimeException> {
+    private final AtomicBoolean hasError = new AtomicBoolean(false);
+
+    @Override
+    public void accept(RuntimeException e) {
+      hasError.set(true);
+      TimeoutScheduler.LOG.error("Failed", e);
+    }
+
+    void assertNoError() {
+      Assert.assertFalse(hasError.get());
+    }
+  }
+
+  @Test(timeout = 1000)
+  public void testSingleTask() throws Exception {
+    final TimeoutScheduler scheduler = TimeoutScheduler.newInstance(1);
+    final TimeDuration grace = TimeDuration.valueOf(100, TimeUnit.MILLISECONDS);
+    scheduler.setGracePeriod(grace);
+    Assert.assertFalse(scheduler.hasScheduler());
+
+    final ErrorHandler errorHandler = new ErrorHandler();
+
+    final AtomicBoolean fired = new AtomicBoolean(false);
+    scheduler.onTimeout(TimeDuration.valueOf(250, TimeUnit.MILLISECONDS), () -> {
+      Assert.assertFalse(fired.get());
+      fired.set(true);
+    }, errorHandler);
+    Assert.assertTrue(scheduler.hasScheduler());
+
+    Thread.sleep(100);
+    Assert.assertFalse(fired.get());
+    Assert.assertTrue(scheduler.hasScheduler());
+
+    Thread.sleep(100);
+    Assert.assertFalse(fired.get());
+    Assert.assertTrue(scheduler.hasScheduler());
+
+    Thread.sleep(100);
+    Assert.assertTrue(fired.get());
+    Assert.assertTrue(scheduler.hasScheduler());
+
+    Thread.sleep(100);
+    Assert.assertTrue(fired.get());
+    Assert.assertFalse(scheduler.hasScheduler());
+
+    errorHandler.assertNoError();
+  }
+
+  @Test(timeout = 1000)
+  public void testMultipleTasks() throws Exception {
+    final TimeoutScheduler scheduler = TimeoutScheduler.newInstance(1);
+    final TimeDuration grace = TimeDuration.valueOf(100, TimeUnit.MILLISECONDS);
+    scheduler.setGracePeriod(grace);
+    Assert.assertFalse(scheduler.hasScheduler());
+
+    final ErrorHandler errorHandler = new ErrorHandler();
+
+    final AtomicBoolean[] fired = new AtomicBoolean[3];
+    for(int i = 0; i < fired.length; i++) {
+      final AtomicBoolean f = fired[i] = new AtomicBoolean(false);
+      scheduler.onTimeout(TimeDuration.valueOf(100*i + 50, TimeUnit.MILLISECONDS), () -> {
+        Assert.assertFalse(f.get());
+        f.set(true);
+      }, errorHandler);
+      Assert.assertTrue(scheduler.hasScheduler());
+    }
+
+    Thread.sleep(100);
+    Assert.assertTrue(fired[0].get());
+    Assert.assertFalse(fired[1].get());
+    Assert.assertFalse(fired[2].get());
+    Assert.assertTrue(scheduler.hasScheduler());
+
+    Thread.sleep(100);
+    Assert.assertTrue(fired[0].get());
+    Assert.assertTrue(fired[1].get());
+    Assert.assertFalse(fired[2].get());
+    Assert.assertTrue(scheduler.hasScheduler());
+
+    Thread.sleep(100);
+    Assert.assertTrue(fired[0].get());
+    Assert.assertTrue(fired[1].get());
+    Assert.assertTrue(fired[2].get());
+    Assert.assertTrue(scheduler.hasScheduler());
+
+    Thread.sleep(100);
+    Assert.assertTrue(fired[0].get());
+    Assert.assertTrue(fired[1].get());
+    Assert.assertTrue(fired[2].get());
+    Assert.assertFalse(scheduler.hasScheduler());
+
+    errorHandler.assertNoError();
+  }
+
+  @Test(timeout = 1000)
+  public void testExtendingGracePeriod() throws Exception {
+    final TimeoutScheduler scheduler = TimeoutScheduler.newInstance(1);
+    final TimeDuration grace = TimeDuration.valueOf(100, TimeUnit.MILLISECONDS);
+    scheduler.setGracePeriod(grace);
+    Assert.assertFalse(scheduler.hasScheduler());
+
+    final ErrorHandler errorHandler = new ErrorHandler();
+
+    {
+      final AtomicBoolean fired = new AtomicBoolean(false);
+      scheduler.onTimeout(TimeDuration.valueOf(150, TimeUnit.MILLISECONDS), () -> {
+        Assert.assertFalse(fired.get());
+        fired.set(true);
+      }, errorHandler);
+      Assert.assertTrue(scheduler.hasScheduler());
+
+      Thread.sleep(100);
+      Assert.assertFalse(fired.get());
+      Assert.assertTrue(scheduler.hasScheduler());
+
+      Thread.sleep(100);
+      Assert.assertTrue(fired.get());
+      Assert.assertTrue(scheduler.hasScheduler());
+    }
+
+    {
+      // submit another task during grace period
+      final AtomicBoolean fired2 = new AtomicBoolean(false);
+      scheduler.onTimeout(TimeDuration.valueOf(150, TimeUnit.MILLISECONDS), () -> {
+        Assert.assertFalse(fired2.get());
+        fired2.set(true);
+      }, errorHandler);
+
+      Thread.sleep(100);
+      Assert.assertFalse(fired2.get());
+      Assert.assertTrue(scheduler.hasScheduler());
+
+      Thread.sleep(100);
+      Assert.assertTrue(fired2.get());
+      Assert.assertTrue(scheduler.hasScheduler());
+
+      Thread.sleep(100);
+      Assert.assertTrue(fired2.get());
+      Assert.assertFalse(scheduler.hasScheduler());
+    }
+
+    errorHandler.assertNoError();
+  }
+
+  @Test(timeout = 1000)
+  public void testRestartingScheduler() throws Exception {
+    final TimeoutScheduler scheduler = TimeoutScheduler.newInstance(1);
+    final TimeDuration grace = TimeDuration.valueOf(100, TimeUnit.MILLISECONDS);
+    scheduler.setGracePeriod(grace);
+    Assert.assertFalse(scheduler.hasScheduler());
+
+    final ErrorHandler errorHandler = new ErrorHandler();
+
+    for(int i = 0; i < 2; i++) {
+      final AtomicBoolean fired = new AtomicBoolean(false);
+      scheduler.onTimeout(TimeDuration.valueOf(150, TimeUnit.MILLISECONDS), () -> {
+        Assert.assertFalse(fired.get());
+        fired.set(true);
+      }, errorHandler);
+      Assert.assertTrue(scheduler.hasScheduler());
+
+      Thread.sleep(100);
+      Assert.assertFalse(fired.get());
+      Assert.assertTrue(scheduler.hasScheduler());
+
+      Thread.sleep(100);
+      Assert.assertTrue(fired.get());
+      Assert.assertTrue(scheduler.hasScheduler());
+
+      Thread.sleep(100);
+      Assert.assertTrue(fired.get());
+      Assert.assertFalse(scheduler.hasScheduler());
+    }
+
+    errorHandler.assertNoError();
+  }
+}

http://git-wip-us.apache.org/repos/asf/incubator-ratis/blob/5c37675f/ratis-test/src/test/java/org/apache/ratis/util/TestTraditionalBinaryPrefix.java
----------------------------------------------------------------------
diff --git a/ratis-test/src/test/java/org/apache/ratis/util/TestTraditionalBinaryPrefix.java b/ratis-test/src/test/java/org/apache/ratis/util/TestTraditionalBinaryPrefix.java
new file mode 100644
index 0000000..26a62da
--- /dev/null
+++ b/ratis-test/src/test/java/org/apache/ratis/util/TestTraditionalBinaryPrefix.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.util;
+
+import org.junit.Test;
+
+import static org.apache.ratis.util.TraditionalBinaryPrefix.long2String;
+import static org.apache.ratis.util.TraditionalBinaryPrefix.string2long;
+import static org.junit.Assert.assertEquals;
+import static org.junit.Assert.fail;
+
+public class TestTraditionalBinaryPrefix {
+  @Test(timeout = 1000)
+  public void testTraditionalBinaryPrefix() throws Exception {
+    //test string2long(..)
+    String[] symbol = {"k", "m", "g", "t", "p", "e"};
+    long m = 1024;
+    for(String s : symbol) {
+      assertEquals(0, string2long(0 + s));
+      assertEquals(m, string2long(1 + s));
+      m *= 1024;
+    }
+
+    assertEquals(0L, string2long("0"));
+    assertEquals(1024L, string2long("1k"));
+    assertEquals(-1024L, string2long("-1k"));
+    assertEquals(1259520L, string2long("1230K"));
+    assertEquals(-1259520L, string2long("-1230K"));
+    assertEquals(104857600L, string2long("100m"));
+    assertEquals(-104857600L, string2long("-100M"));
+    assertEquals(956703965184L, string2long("891g"));
+    assertEquals(-956703965184L, string2long("-891G"));
+    assertEquals(501377302265856L, string2long("456t"));
+    assertEquals(-501377302265856L, string2long("-456T"));
+    assertEquals(11258999068426240L, string2long("10p"));
+    assertEquals(-11258999068426240L, string2long("-10P"));
+    assertEquals(1152921504606846976L, string2long("1e"));
+    assertEquals(-1152921504606846976L, string2long("-1E"));
+
+    String tooLargeNumStr = "10e";
+    try {
+      string2long(tooLargeNumStr);
+      fail("Test passed for a number " + tooLargeNumStr + " too large");
+    } catch (IllegalArgumentException e) {
+      assertEquals(tooLargeNumStr + " does not fit in a Long", e.getMessage());
+    }
+
+    String tooSmallNumStr = "-10e";
+    try {
+      string2long(tooSmallNumStr);
+      fail("Test passed for a number " + tooSmallNumStr + " too small");
+    } catch (IllegalArgumentException e) {
+      assertEquals(tooSmallNumStr + " does not fit in a Long", e.getMessage());
+    }
+
+    String invalidFormatNumStr = "10kb";
+    char invalidPrefix = 'b';
+    try {
+      string2long(invalidFormatNumStr);
+      fail("Test passed for a number " + invalidFormatNumStr
+          + " has invalid format");
+    } catch (IllegalArgumentException e) {
+      assertEquals("Invalid size prefix '" + invalidPrefix + "' in '"
+              + invalidFormatNumStr
+              + "'. Allowed prefixes are k, m, g, t, p, e (case insensitive)",
+          e.getMessage());
+    }
+
+    //test long2string(..)
+    assertEquals("0", long2String(0, null, 2));
+    for(int decimalPlace = 0; decimalPlace < 2; decimalPlace++) {
+      for(int n = 1; n < TraditionalBinaryPrefix.KILO.getValue(); n++) {
+        assertEquals(n + "", long2String(n, null, decimalPlace));
+        assertEquals(-n + "", long2String(-n, null, decimalPlace));
+      }
+      assertEquals("1 K", long2String(1L << 10, null, decimalPlace));
+      assertEquals("-1 K", long2String(-1L << 10, null, decimalPlace));
+    }
+
+    assertEquals("8.00 E", long2String(Long.MAX_VALUE, null, 2));
+    assertEquals("8.00 E", long2String(Long.MAX_VALUE - 1, null, 2));
+    assertEquals("-8 E", long2String(Long.MIN_VALUE, null, 2));
+    assertEquals("-8.00 E", long2String(Long.MIN_VALUE + 1, null, 2));
+
+    final String[] zeros = {" ", ".0 ", ".00 "};
+    for(int decimalPlace = 0; decimalPlace < zeros.length; decimalPlace++) {
+      final String trailingZeros = zeros[decimalPlace];
+
+      for(int e = 11; e < Long.SIZE - 1; e++) {
+        final TraditionalBinaryPrefix p
+            = TraditionalBinaryPrefix.values()[e/10 - 1];
+
+        { // n = 2^e
+          final long n = 1L << e;
+          final String expected = (n/p.getValue()) + " " + p.getSymbol();
+          assertEquals("n=" + n, expected, long2String(n, null, 2));
+        }
+
+        { // n = 2^e + 1
+          final long n = (1L << e) + 1;
+          final String expected = (n/p.getValue()) + trailingZeros + p.getSymbol();
+          assertEquals("n=" + n, expected, long2String(n, null, decimalPlace));
+        }
+
+        { // n = 2^e - 1
+          final long n = (1L << e) - 1;
+          final String expected = ((n+1)/p.getValue()) + trailingZeros + p.getSymbol();
+          assertEquals("n=" + n, expected, long2String(n, null, decimalPlace));
+        }
+      }
+    }
+
+    assertEquals("1.50 K", long2String(3L << 9, null, 2));
+    assertEquals("1.5 K", long2String(3L << 9, null, 1));
+    assertEquals("1.50 M", long2String(3L << 19, null, 2));
+    assertEquals("2 M", long2String(3L << 19, null, 0));
+    assertEquals("3 G", long2String(3L << 30, null, 2));
+
+    assertEquals("0 B", byteDescription(0));
+    assertEquals("-100 B", byteDescription(-100));
+    assertEquals("1 KB", byteDescription(1024));
+    assertEquals("1.50 KB", byteDescription(3L << 9));
+    assertEquals("1.50 MB", byteDescription(3L << 19));
+    assertEquals("3 GB", byteDescription(3L << 30));
+  }
+
+  private static String byteDescription(long len) {
+    return long2String(len, "B", 2);
+  }
+}

http://git-wip-us.apache.org/repos/asf/incubator-ratis/blob/5c37675f/ratis-test/src/test/resources/log4j.properties
----------------------------------------------------------------------
diff --git a/ratis-test/src/test/resources/log4j.properties b/ratis-test/src/test/resources/log4j.properties
new file mode 100644
index 0000000..ced0687
--- /dev/null
+++ b/ratis-test/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


[4/6] incubator-ratis git commit: RATIS-399. Move all tests to a new module.

Posted by sz...@apache.org.
http://git-wip-us.apache.org/repos/asf/incubator-ratis/blob/5c37675f/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
deleted file mode 100644
index 7d9fdf5..0000000
--- a/ratis-server/src/test/java/org/apache/ratis/server/storage/TestRaftLogReadWrite.java
+++ /dev/null
@@ -1,272 +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.storage;
-
-import org.apache.ratis.BaseTest;
-import org.apache.ratis.RaftTestUtil.SimpleOperation;
-import org.apache.ratis.conf.RaftProperties;
-import org.apache.ratis.protocol.ChecksumException;
-import org.apache.ratis.server.RaftServerConfigKeys;
-import org.apache.ratis.server.impl.RaftServerConstants;
-import org.apache.ratis.server.impl.RaftServerConstants.StartupOption;
-import org.apache.ratis.server.impl.ServerProtoUtils;
-import org.apache.ratis.thirdparty.com.google.protobuf.CodedOutputStream;
-import org.apache.ratis.proto.RaftProtos.LogEntryProto;
-import org.apache.ratis.util.FileUtils;
-import org.junit.After;
-import org.junit.Assert;
-import org.junit.Before;
-import org.junit.Test;
-
-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.Collections;
-import java.util.List;
-
-/**
- * Test basic functionality of LogReader, LogInputStream, and LogOutputStream.
- */
-public class TestRaftLogReadWrite extends BaseTest {
-  private File storageDir;
-  private long segmentMaxSize;
-  private long preallocatedSize;
-  private int bufferSize;
-
-  @Before
-  public void setup() throws Exception {
-    storageDir = getTestDir();
-    RaftProperties properties = new RaftProperties();
-    RaftServerConfigKeys.setStorageDirs(properties,  Collections.singletonList(storageDir));
-    this.segmentMaxSize =
-        RaftServerConfigKeys.Log.segmentSizeMax(properties).getSize();
-    this.preallocatedSize =
-        RaftServerConfigKeys.Log.preallocatedSize(properties).getSize();
-    this.bufferSize =
-        RaftServerConfigKeys.Log.writeBufferSize(properties).getSizeInt();
-  }
-
-  @After
-  public void tearDown() throws Exception {
-    if (storageDir != null) {
-      FileUtils.deleteFully(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] = ServerProtoUtils.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 {
-    final RaftStorage storage = new RaftStorage(storageDir, StartupOption.REGULAR);
-    File openSegment = storage.getStorageDir().getOpenLogFile(0);
-    long size = SegmentedRaftLogFormat.getHeaderLength();
-
-    final LogEntryProto[] entries = new LogEntryProto[100];
-    try (LogOutputStream out =
-             new LogOutputStream(openSegment, false, segmentMaxSize,
-                 preallocatedSize, bufferSize)) {
-      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 {
-    final RaftStorage storage = new RaftStorage(storageDir, StartupOption.REGULAR);
-    File openSegment = storage.getStorageDir().getOpenLogFile(0);
-    LogEntryProto[] entries = new LogEntryProto[200];
-    try (LogOutputStream out =
-             new LogOutputStream(openSegment, false, segmentMaxSize,
-                 preallocatedSize, bufferSize)) {
-      for (int i = 0; i < 100; i++) {
-        SimpleOperation m = new SimpleOperation("m" + i);
-        entries[i] = ServerProtoUtils.toLogEntryProto(m.getLogEntryContent(), 0, i);
-        out.write(entries[i]);
-      }
-    }
-
-    try (LogOutputStream out =
-             new LogOutputStream(openSegment, true, segmentMaxSize,
-                 preallocatedSize, bufferSize)) {
-      for (int i = 100; i < 200; i++) {
-        SimpleOperation m = new SimpleOperation("m" + i);
-        entries[i] = ServerProtoUtils.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 {
-    final RaftStorage storage = new RaftStorage(storageDir, StartupOption.REGULAR);
-    File openSegment = storage.getStorageDir().getOpenLogFile(0);
-    long size = SegmentedRaftLogFormat.getHeaderLength();
-
-    LogEntryProto[] entries = new LogEntryProto[100];
-    LogOutputStream out = new LogOutputStream(openSegment, false,
-        segmentMaxSize, preallocatedSize, bufferSize);
-    size += writeMessages(entries, out);
-    out.flush();
-
-    // make sure the file contains padding
-    Assert.assertEquals(
-        RaftServerConfigKeys.Log.PREALLOCATED_SIZE_DEFAULT.getSize(),
-        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 {
-    final RaftStorage storage = new RaftStorage(storageDir, StartupOption.REGULAR);
-    File openSegment = storage.getStorageDir().getOpenLogFile(0);
-
-    LogEntryProto[] entries = new LogEntryProto[10];
-    LogOutputStream out = new LogOutputStream(openSegment, false,
-        16 * 1024 * 1024, 4 * 1024 * 1024, bufferSize);
-    for (int i = 0; i < 10; i++) {
-      SimpleOperation m = new SimpleOperation("m" + i);
-      entries[i] = ServerProtoUtils.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(storageDir, StartupOption.REGULAR);
-    File openSegment = storage.getStorageDir().getOpenLogFile(0);
-    try (LogOutputStream out =
-             new LogOutputStream(openSegment, false, segmentMaxSize,
-                 preallocatedSize, bufferSize)) {
-      for (int i = 0; i < 100; i++) {
-        LogEntryProto entry = ServerProtoUtils.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/5c37675f/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
deleted file mode 100644
index 270e279..0000000
--- a/ratis-server/src/test/java/org/apache/ratis/server/storage/TestRaftLogSegment.java
+++ /dev/null
@@ -1,372 +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.storage;
-
-import org.apache.ratis.BaseTest;
-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.server.impl.ServerProtoUtils;
-import org.apache.ratis.server.storage.LogSegment.LogRecordWithEntry;
-import org.apache.ratis.proto.RaftProtos.LogEntryProto;
-import org.apache.ratis.proto.RaftProtos.StateMachineLogEntryProto;
-import org.apache.ratis.thirdparty.com.google.protobuf.CodedOutputStream;
-import org.apache.ratis.util.FileUtils;
-import org.apache.ratis.util.Preconditions;
-import org.apache.ratis.util.SizeInBytes;
-import org.apache.ratis.util.TraditionalBinaryPrefix;
-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.nio.file.Files;
-import java.nio.file.Path;
-import java.util.ArrayList;
-import java.util.Arrays;
-import java.util.Collections;
-import java.util.List;
-import java.util.concurrent.ThreadLocalRandom;
-
-import static org.apache.ratis.server.impl.RaftServerConstants.INVALID_LOG_INDEX;
-import static org.apache.ratis.server.storage.LogSegment.getEntrySize;
-
-/**
- * Test basic functionality of {@link LogSegment}
- */
-public class TestRaftLogSegment extends BaseTest {
-  private File storageDir;
-  private long segmentMaxSize;
-  private long preallocatedSize;
-  private int bufferSize;
-
-  @Before
-  public void setup() throws Exception {
-    RaftProperties properties = new RaftProperties();
-    storageDir = getTestDir();
-    RaftServerConfigKeys.setStorageDirs(properties,  Collections.singletonList(storageDir));
-    this.segmentMaxSize =
-        RaftServerConfigKeys.Log.segmentSizeMax(properties).getSize();
-    this.preallocatedSize =
-        RaftServerConfigKeys.Log.preallocatedSize(properties).getSize();
-    this.bufferSize =
-        RaftServerConfigKeys.Log.writeBufferSize(properties).getSizeInt();
-  }
-
-  @After
-  public void tearDown() throws Exception {
-    if (storageDir != null) {
-      FileUtils.deleteFully(storageDir.getParentFile());
-    }
-  }
-
-  File prepareLog(boolean isOpen, long startIndex, int numEntries, long term, boolean isLastEntryPartiallyWritten)
-      throws IOException {
-    if (!isOpen) {
-      Preconditions.assertTrue(!isLastEntryPartiallyWritten, "For closed log, the last entry cannot be partially written.");
-    }
-    RaftStorage storage = new RaftStorage(storageDir, StartupOption.REGULAR);
-    final File file = isOpen ?
-        storage.getStorageDir().getOpenLogFile(startIndex) :
-        storage.getStorageDir().getClosedLogFile(startIndex, startIndex + numEntries - 1);
-
-    final LogEntryProto[] entries = new LogEntryProto[numEntries];
-    try (LogOutputStream out = new LogOutputStream(file, false,
-        segmentMaxSize, preallocatedSize, bufferSize)) {
-      for (int i = 0; i < entries.length; i++) {
-        SimpleOperation op = new SimpleOperation("m" + i);
-        entries[i] = ServerProtoUtils.toLogEntryProto(op.getLogEntryContent(), term, i + startIndex);
-        out.write(entries[i]);
-      }
-    }
-
-    if (isLastEntryPartiallyWritten) {
-      final int entrySize = size(entries[entries.length - 1]);
-      final int truncatedEntrySize = ThreadLocalRandom.current().nextInt(entrySize - 1) + 1;
-      // 0 < truncatedEntrySize < entrySize
-      final long fileLength = file.length();
-      final long truncatedFileLength = fileLength - (entrySize - truncatedEntrySize);
-      LOG.info("truncate last entry: entry(size={}, truncated={}), file(length={}, truncated={})",
-          entrySize, truncatedEntrySize, fileLength, truncatedFileLength);
-      FileUtils.truncateFile(file, truncatedFileLength);
-    }
-
-    storage.close();
-    return file;
-  }
-
-  static int size(LogEntryProto entry) {
-    final int n = entry.getSerializedSize();
-    return CodedOutputStream.computeUInt32SizeNoTag(n) + n + 4;
-  }
-
-  static void checkLogSegment(LogSegment segment, long start, long end,
-      boolean isOpen, long totalSize, long term) throws Exception {
-    Assert.assertEquals(start, segment.getStartIndex());
-    Assert.assertEquals(end, segment.getEndIndex());
-    Assert.assertEquals(isOpen, segment.isOpen());
-    Assert.assertEquals(totalSize, segment.getTotalSize());
-
-    long offset = SegmentedRaftLogFormat.getHeaderLength();
-    for (long i = start; i <= end; i++) {
-      LogSegment.LogRecord record = segment.getLogRecord(i);
-      LogRecordWithEntry lre = segment.getEntryWithoutLoading(i);
-      Assert.assertEquals(i, lre.getRecord().getTermIndex().getIndex());
-      Assert.assertEquals(term, lre.getRecord().getTermIndex().getTerm());
-      Assert.assertEquals(offset, record.getOffset());
-
-      LogEntryProto entry = lre.hasEntry() ?
-          lre.getEntry() : segment.loadCache(lre.getRecord());
-      offset += getEntrySize(entry);
-    }
-  }
-
-  @Test
-  public void testLoadLogSegment() throws Exception {
-    testLoadSegment(true, false);
-  }
-
-  @Test
-  public void testLoadLogSegmentLastEntryPartiallyWritten() throws Exception {
-    testLoadSegment(true, true);
-  }
-
-  @Test
-  public void testLoadCache() throws Exception {
-    testLoadSegment(false, false);
-  }
-
-  @Test
-  public void testLoadCacheLastEntryPartiallyWritten() throws Exception {
-    testLoadSegment(false, true);
-  }
-
-  private void testLoadSegment(boolean loadInitial, boolean isLastEntryPartiallyWritten) throws Exception {
-    // load an open segment
-    final File openSegmentFile = prepareLog(true, 0, 100, 0, isLastEntryPartiallyWritten);
-    RaftStorage storage = new RaftStorage(storageDir, StartupOption.REGULAR);
-    LogSegment openSegment = LogSegment.loadSegment(storage, openSegmentFile, 0,
-        INVALID_LOG_INDEX, true, loadInitial, null);
-    final int delta = isLastEntryPartiallyWritten? 1: 0;
-    checkLogSegment(openSegment, 0, 99 - delta, true, openSegmentFile.length(), 0);
-    storage.close();
-    // for open segment we currently always keep log entries in the memory
-    Assert.assertEquals(0, openSegment.getLoadingTimes());
-
-    // load a closed segment (1000-1099)
-    final File closedSegmentFile = prepareLog(false, 1000, 100, 1, false);
-    LogSegment closedSegment = LogSegment.loadSegment(storage, closedSegmentFile,
-        1000, 1099, false, loadInitial, null);
-    checkLogSegment(closedSegment, 1000, 1099, false,
-        closedSegment.getTotalSize(), 1);
-    Assert.assertEquals(loadInitial ? 0 : 1, closedSegment.getLoadingTimes());
-  }
-
-  @Test
-  public void testAppendEntries() throws Exception {
-    final long start = 1000;
-    LogSegment segment = LogSegment.newOpenSegment(null, start);
-    long size = SegmentedRaftLogFormat.getHeaderLength();
-    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 = ServerProtoUtils.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(null, 1000);
-    SimpleOperation op = new SimpleOperation("m");
-    final StateMachineLogEntryProto m = op.getLogEntryContent();
-    try {
-      LogEntryProto entry = ServerProtoUtils.toLogEntryProto(m, 0, 1001);
-      segment.appendToOpenSegment(entry);
-      Assert.fail("should fail since the entry's index needs to be 1000");
-    } catch (IllegalStateException e) {
-      // the exception is expected.
-    }
-
-    LogEntryProto entry = ServerProtoUtils.toLogEntryProto(m, 0, 1000);
-    segment.appendToOpenSegment(entry);
-
-    try {
-      entry = ServerProtoUtils.toLogEntryProto(m, 0, 1002);
-      segment.appendToOpenSegment(entry);
-      Assert.fail("should fail since the entry's index needs to be 1001");
-    } catch (IllegalStateException e) {
-      // the exception is expected.
-    }
-
-    LogEntryProto[] entries = new LogEntryProto[2];
-    for (int i = 0; i < 2; i++) {
-      entries[i] = ServerProtoUtils.toLogEntryProto(m, 0, 1001 + i * 2);
-    }
-    try {
-      segment.appendToOpenSegment(entries);
-      Assert.fail("should fail since there is gap between entries");
-    } catch (IllegalStateException e) {
-      // the exception is expected.
-    }
-  }
-
-  @Test
-  public void testTruncate() throws Exception {
-    final long term = 1;
-    final long start = 1000;
-    LogSegment segment = LogSegment.newOpenSegment(null, start);
-    for (int i = 0; i < 100; i++) {
-      LogEntryProto entry = ServerProtoUtils.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).getOffset();
-    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).getOffset();
-    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,
-        SegmentedRaftLogFormat.getHeaderLength(), term);
-  }
-
-  @Test
-  public void testPreallocateSegment() throws Exception {
-    RaftStorage storage = new RaftStorage(storageDir, 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, max, a, bufferSize)) {
-          Assert.assertEquals("max=" + max + ", a=" + a, 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,
-        1024, 1024, bufferSize)) {
-      SimpleOperation op = new SimpleOperation(new String(content));
-      LogEntryProto entry = ServerProtoUtils.toLogEntryProto(op.getLogEntryContent(), 0, 0);
-      size = LogSegment.getEntrySize(entry);
-      out.write(entry);
-    }
-    Assert.assertEquals(file.length(),
-        size + SegmentedRaftLogFormat.getHeaderLength());
-    try (LogInputStream in = new LogInputStream(file, 0,
-        INVALID_LOG_INDEX, true)) {
-      LogEntryProto entry = in.nextEntry();
-      Assert.assertArrayEquals(content,
-          entry.getStateMachineLogEntry().getLogData().toByteArray());
-      Assert.assertNull(in.nextEntry());
-    }
-  }
-
-  /**
-   * Keep appending and check if pre-allocation is correct
-   */
-  @Test
-  public void testPreallocationAndAppend() throws Exception {
-    final SizeInBytes max = SizeInBytes.valueOf(2, TraditionalBinaryPrefix.MEGA);
-    RaftStorage storage = new RaftStorage(storageDir, 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 = ServerProtoUtils.toLogEntryProto(op.getLogEntryContent(), 0, 0);
-    final long entrySize = LogSegment.getEntrySize(entry);
-
-    long totalSize = SegmentedRaftLogFormat.getHeaderLength();
-    long preallocated = 16 * 1024;
-    try (LogOutputStream out = new LogOutputStream(file, false,
-        max.getSize(), 16 * 1024, 10 * 1024)) {
-      Assert.assertEquals(preallocated, file.length());
-      while (totalSize + entrySize < max.getSize()) {
-        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());
-  }
-
-  @Test
-  public void testZeroSizeInProgressFile() throws Exception {
-    final RaftStorage storage = new RaftStorage(storageDir, StartupOption.REGULAR);
-    final File file = storage.getStorageDir().getOpenLogFile(0);
-    storage.close();
-
-    // create zero size in-progress file
-    LOG.info("file: " + file);
-    Assert.assertTrue(file.createNewFile());
-    final Path path = file.toPath();
-    Assert.assertTrue(Files.exists(path));
-    Assert.assertEquals(0, Files.size(path));
-
-    // getLogSegmentFiles should remove it.
-    final List<RaftStorageDirectory.LogPathAndIndex> logs = storage.getStorageDir().getLogSegmentFiles();
-    Assert.assertEquals(0, logs.size());
-    Assert.assertFalse(Files.exists(path));
-  }
-}

http://git-wip-us.apache.org/repos/asf/incubator-ratis/blob/5c37675f/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
deleted file mode 100644
index 4a26f8c..0000000
--- a/ratis-server/src/test/java/org/apache/ratis/server/storage/TestRaftStorage.java
+++ /dev/null
@@ -1,206 +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.storage;
-
-import org.apache.ratis.BaseTest;
-import org.apache.ratis.server.impl.RaftServerConstants.StartupOption;
-import org.apache.ratis.server.protocol.TermIndex;
-import org.apache.ratis.server.storage.RaftStorageDirectory.StorageState;
-import org.apache.ratis.statemachine.impl.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 extends BaseTest {
-  private File storageDir;
-
-  @Before
-  public void setup() throws Exception {
-    storageDir = getTestDir();
-  }
-
-  @After
-  public void tearDown() throws Exception {
-    if (storageDir != null) {
-      FileUtils.deleteFully(storageDir.getParentFile());
-    }
-  }
-
-  @Test
-  public void testNotExistent() throws IOException {
-    FileUtils.deleteFully(storageDir);
-
-    // we will format the empty directory
-    RaftStorage storage = new RaftStorage(storageDir, StartupOption.REGULAR);
-    Assert.assertEquals(StorageState.NORMAL, storage.getState());
-
-    try {
-      new RaftStorage(storageDir, 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.deleteFully(storageDir);
-    Assert.assertTrue(storageDir.createNewFile());
-    try {
-      new RaftStorage(storageDir, 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(storageDir, 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(storageDir, 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(storageDir, 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(storageDir, StartupOption.REGULAR);
-    Assert.assertEquals(StorageState.NORMAL, storage.getState());
-    storage.close();
-
-    RaftStorageDirectory sd = new RaftStorageDirectory(storageDir);
-    File metaFile = sd.getMetaFile();
-    FileUtils.move(metaFile, sd.getMetaTmpFile());
-
-    Assert.assertEquals(StorageState.NOT_FORMATTED, sd.analyzeStorage(false));
-
-    try {
-      new RaftStorage(storageDir, 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(storageDir, StartupOption.FORMAT).close();
-    Assert.assertTrue(sd.getMetaFile().exists());
-    Assert.assertTrue(sd.getMetaTmpFile().createNewFile());
-    Assert.assertTrue(sd.getMetaTmpFile().exists());
-    try {
-      storage = new RaftStorage(storageDir, 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/5c37675f/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
deleted file mode 100644
index bcbfa73..0000000
--- a/ratis-server/src/test/java/org/apache/ratis/server/storage/TestSegmentedRaftLog.java
+++ /dev/null
@@ -1,524 +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.storage;
-
-import org.apache.log4j.Level;
-import org.apache.ratis.BaseTest;
-import org.apache.ratis.RaftTestUtil.SimpleOperation;
-import org.apache.ratis.conf.RaftProperties;
-import org.apache.ratis.protocol.RaftPeerId;
-import org.apache.ratis.protocol.TimeoutIOException;
-import org.apache.ratis.server.RaftServerConfigKeys;
-import org.apache.ratis.server.impl.RaftServerConstants;
-import org.apache.ratis.server.impl.RetryCacheTestUtil;
-import org.apache.ratis.server.impl.RetryCache;
-import org.apache.ratis.server.impl.RaftServerImpl;
-import org.apache.ratis.server.impl.ServerProtoUtils;
-import org.apache.ratis.server.protocol.TermIndex;
-import org.apache.ratis.proto.RaftProtos.LogEntryProto;
-import org.apache.ratis.statemachine.SimpleStateMachine4Testing;
-import org.apache.ratis.statemachine.StateMachine;
-import org.apache.ratis.statemachine.impl.BaseStateMachine;
-import org.apache.ratis.util.ExitUtils;
-import org.apache.ratis.util.FileUtils;
-import org.apache.ratis.util.JavaUtils;
-import org.apache.ratis.util.LogUtils;
-import org.apache.ratis.util.SizeInBytes;
-import org.apache.ratis.util.TimeDuration;
-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.nio.charset.StandardCharsets;
-import java.util.ArrayList;
-import java.util.Arrays;
-import java.util.Collections;
-import java.util.List;
-import java.util.Objects;
-import java.util.concurrent.CompletableFuture;
-import java.util.concurrent.TimeUnit;
-import java.util.function.Supplier;
-
-import static org.mockito.Matchers.any;
-import static org.mockito.Mockito.doCallRealMethod;
-import static org.mockito.Mockito.mock;
-import static org.mockito.Mockito.when;
-
-public class TestSegmentedRaftLog extends BaseTest {
-  static {
-    LogUtils.setLogLevel(RaftLogWorker.LOG, Level.DEBUG);
-  }
-
-  private static final RaftPeerId peerId = RaftPeerId.valueOf("s0");
-
-  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 long segmentMaxSize;
-  private long preallocatedSize;
-  private int bufferSize;
-
-  @Before
-  public void setup() throws Exception {
-    storageDir = getTestDir();
-    properties = new RaftProperties();
-    RaftServerConfigKeys.setStorageDirs(properties,  Collections.singletonList(storageDir));
-    storage = new RaftStorage(storageDir, RaftServerConstants.StartupOption.REGULAR);
-    this.segmentMaxSize =
-        RaftServerConfigKeys.Log.segmentSizeMax(properties).getSize();
-    this.preallocatedSize =
-        RaftServerConfigKeys.Log.preallocatedSize(properties).getSize();
-    this.bufferSize =
-        RaftServerConfigKeys.Log.writeBufferSize(properties).getSizeInt();
-  }
-
-  @After
-  public void tearDown() throws Exception {
-    if (storageDir != null) {
-      FileUtils.deleteFully(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,
-          segmentMaxSize, preallocatedSize, bufferSize)) {
-        for (int i = 0; i < size; i++) {
-          SimpleOperation m = new SimpleOperation("m" + (i + range.start));
-          entries[i] = ServerProtoUtils.toLogEntryProto(m.getLogEntryContent(), range.term, i + range.start);
-          out.write(entries[i]);
-        }
-      }
-      Collections.addAll(entryList, entries);
-    }
-    return entryList.toArray(new LogEntryProto[entryList.size()]);
-  }
-
-  static List<SegmentRange> prepareRanges(int startTerm, int endTerm, int segmentSize,
-      long startIndex) {
-    List<SegmentRange> list = new ArrayList<>(endTerm - startTerm);
-    for (int i = startTerm; i < endTerm; i++) {
-      list.add(new SegmentRange(startIndex, startIndex + segmentSize - 1, i,
-          i == endTerm - 1));
-      startIndex += segmentSize;
-    }
-    return list;
-  }
-
-  private LogEntryProto getLastEntry(SegmentedRaftLog raftLog)
-      throws IOException {
-    return raftLog.get(raftLog.getLastEntryTermIndex().getIndex());
-  }
-
-  @Test
-  public void testLoadLogSegments() throws Exception {
-    // first generate log files
-    List<SegmentRange> ranges = prepareRanges(0, 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(RaftServerConstants.INVALID_LOG_INDEX, null);
-      // check if log entries are loaded correctly
-      for (LogEntryProto e : entries) {
-        LogEntryProto entry = raftLog.get(e.getIndex());
-        Assert.assertEquals(e, entry);
-      }
-
-      TermIndex[] termIndices = raftLog.getEntries(0, 500);
-      LogEntryProto[] entriesFromLog = Arrays.stream(termIndices)
-          .map(ti -> {
-            try {
-              return raftLog.get(ti.getIndex());
-            } catch (IOException e) {
-              throw new RuntimeException(e);
-            }
-          })
-          .toArray(LogEntryProto[]::new);
-      Assert.assertArrayEquals(entries, entriesFromLog);
-      Assert.assertEquals(entries[entries.length - 1], getLastEntry(raftLog));
-    }
-  }
-
-  static List<LogEntryProto> prepareLogEntries(List<SegmentRange> slist,
-      Supplier<String> stringSupplier) {
-    List<LogEntryProto> eList = new ArrayList<>();
-    for (SegmentRange range : slist) {
-      prepareLogEntries(range, stringSupplier, false, eList);
-    }
-    return eList;
-  }
-
-  static List<LogEntryProto> prepareLogEntries(SegmentRange range,
-      Supplier<String> stringSupplier, boolean hasStataMachineData, List<LogEntryProto> eList) {
-    for(long index = range.start; index <= range.end; index++) {
-      eList.add(prepareLogEntry(range.term, index, stringSupplier, hasStataMachineData));
-    }
-    return eList;
-  }
-
-  static LogEntryProto prepareLogEntry(long term, long index, Supplier<String> stringSupplier, boolean hasStataMachineData) {
-    final SimpleOperation m = stringSupplier == null?
-        new SimpleOperation("m" + index, hasStataMachineData):
-        new SimpleOperation(stringSupplier.get(), hasStataMachineData);
-    return ServerProtoUtils.toLogEntryProto(m.getLogEntryContent(), term, index);
-  }
-
-  /**
-   * Append entry one by one and check if log state is correct.
-   */
-  @Test
-  public void testAppendEntry() throws Exception {
-    List<SegmentRange> ranges = prepareRanges(0, 5, 200, 0);
-    List<LogEntryProto> entries = prepareLogEntries(ranges, null);
-
-    try (SegmentedRaftLog raftLog =
-             new SegmentedRaftLog(peerId, null, storage, -1, properties)) {
-      raftLog.open(RaftServerConstants.INVALID_LOG_INDEX, null);
-      // append entries to the raftlog
-      entries.stream().map(raftLog::appendEntry).forEach(CompletableFuture::join);
-    }
-
-    try (SegmentedRaftLog raftLog =
-             new SegmentedRaftLog(peerId, null, storage, -1, properties)) {
-      raftLog.open(RaftServerConstants.INVALID_LOG_INDEX, null);
-      // check if the raft log is correct
-      checkEntries(raftLog, entries, 0, entries.size());
-    }
-
-    try (SegmentedRaftLog raftLog =
-        new SegmentedRaftLog(peerId, null, storage, -1, properties)) {
-      raftLog.open(RaftServerConstants.INVALID_LOG_INDEX, null);
-      TermIndex lastTermIndex  = raftLog.getLastEntryTermIndex();
-      IllegalStateException ex = null;
-      try {
-        // append entry fails if append entry term is lower than log's last entry term
-        raftLog.appendEntry(LogEntryProto.newBuilder(entries.get(0))
-            .setTerm(lastTermIndex.getTerm() - 1)
-            .setIndex(lastTermIndex.getIndex() + 1).build());
-      } catch (IllegalStateException e) {
-        ex = e;
-      }
-      Assert.assertTrue(ex.getMessage().contains("term less than RaftLog's last term"));
-      try {
-        // append entry fails if difference between append entry index and log's last entry index is greater than 1
-        raftLog.appendEntry(LogEntryProto.newBuilder(entries.get(0))
-            .setTerm(lastTermIndex.getTerm())
-            .setIndex(lastTermIndex.getIndex() + 2).build());
-      } catch (IllegalStateException e) {
-        ex = e;
-      }
-      Assert.assertTrue(ex.getMessage().contains("and RaftLog's last index " + lastTermIndex.getIndex() + " greater than 1"));
-    }
-  }
-
-  /**
-   * Keep appending entries, make sure the rolling is correct.
-   */
-  @Test
-  public void testAppendAndRoll() throws Exception {
-    RaftServerConfigKeys.Log.setPreallocatedSize(properties, SizeInBytes.valueOf("16KB"));
-    RaftServerConfigKeys.Log.setSegmentSizeMax(properties, SizeInBytes.valueOf("128KB"));
-
-    List<SegmentRange> ranges = prepareRanges(0, 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(RaftServerConstants.INVALID_LOG_INDEX, null);
-      // append entries to the raftlog
-      entries.stream().map(raftLog::appendEntry).forEach(CompletableFuture::join);
-    }
-
-    try (SegmentedRaftLog raftLog =
-             new SegmentedRaftLog(peerId, null, storage, -1, properties)) {
-      raftLog.open(RaftServerConstants.INVALID_LOG_INDEX, null);
-      // 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(0, 5, 200, 0);
-    List<LogEntryProto> entries = prepareLogEntries(ranges, null);
-
-    try (SegmentedRaftLog raftLog =
-             new SegmentedRaftLog(peerId, null, storage, -1, properties)) {
-      raftLog.open(RaftServerConstants.INVALID_LOG_INDEX, null);
-      // append entries to the raftlog
-      entries.stream().map(raftLog::appendEntry).forEach(CompletableFuture::join);
-    }
-
-    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(RaftServerConstants.INVALID_LOG_INDEX, null);
-      // truncate the log
-      raftLog.truncate(fromIndex).join();
-
-
-      checkEntries(raftLog, entries, 0, (int) fromIndex);
-    }
-
-    try (SegmentedRaftLog raftLog =
-             new SegmentedRaftLog(peerId, null, storage, -1, properties)) {
-      raftLog.open(RaftServerConstants.INVALID_LOG_INDEX, null);
-      // check if the raft log is correct
-      if (fromIndex > 0) {
-        Assert.assertEquals(entries.get((int) (fromIndex - 1)),
-            getLastEntry(raftLog));
-      } else {
-        Assert.assertNull(raftLog.getLastEntryTermIndex());
-      }
-      checkEntries(raftLog, entries, 0, (int) fromIndex);
-    }
-  }
-
-  private void checkEntries(RaftLog raftLog, List<LogEntryProto> expected,
-      int offset, int size) throws IOException {
-    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);
-      }
-      TermIndex[] termIndices = raftLog.getEntries(
-          expected.get(offset).getIndex(),
-          expected.get(offset + size - 1).getIndex() + 1);
-      LogEntryProto[] entriesFromLog = Arrays.stream(termIndices)
-          .map(ti -> {
-            try {
-              return raftLog.get(ti.getIndex());
-            } catch (IOException e) {
-              throw new RuntimeException(e);
-            }
-          })
-          .toArray(LogEntryProto[]::new);
-      LogEntryProto[] expectedArray = expected.subList(offset, offset + size)
-          .stream().toArray(LogEntryProto[]::new);
-      Assert.assertArrayEquals(expectedArray, entriesFromLog);
-    }
-  }
-
-  private void checkFailedEntries(List<LogEntryProto> entries, long fromIndex, RetryCache retryCache) {
-    for (int i = 0; i < entries.size(); i++) {
-      if (i < fromIndex) {
-        RetryCacheTestUtil.assertFailure(retryCache, entries.get(i), false);
-      } else {
-        RetryCacheTestUtil.assertFailure(retryCache, entries.get(i), true);
-      }
-    }
-  }
-
-  /**
-   * Test append with inconsistent entries
-   */
-  @Test
-  public void testAppendEntriesWithInconsistency() throws Exception {
-    // prepare the log for truncation
-    List<SegmentRange> ranges = prepareRanges(0, 5, 200, 0);
-    List<LogEntryProto> entries = prepareLogEntries(ranges, null);
-
-    RaftServerImpl server = mock(RaftServerImpl.class);
-    RetryCache retryCache = RetryCacheTestUtil.createRetryCache();
-    when(server.getRetryCache()).thenReturn(retryCache);
-    doCallRealMethod().when(server).failClientRequest(any(LogEntryProto.class));
-    try (SegmentedRaftLog raftLog =
-             new SegmentedRaftLog(peerId, server, storage, -1, properties)) {
-      raftLog.open(RaftServerConstants.INVALID_LOG_INDEX, null);
-      entries.stream().forEach(entry -> RetryCacheTestUtil.createEntry(retryCache, entry));
-      // append entries to the raftlog
-      entries.stream().map(raftLog::appendEntry).forEach(CompletableFuture::join);
-    }
-
-    // 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, server, storage, -1, properties)) {
-      raftLog.open(RaftServerConstants.INVALID_LOG_INDEX, null);
-      raftLog.append(newEntries.toArray(new LogEntryProto[newEntries.size()])).forEach(CompletableFuture::join);
-
-      checkFailedEntries(entries, 650, retryCache);
-      checkEntries(raftLog, entries, 0, 650);
-      checkEntries(raftLog, newEntries, 100, 100);
-      Assert.assertEquals(newEntries.get(newEntries.size() - 1),
-          getLastEntry(raftLog));
-      Assert.assertEquals(newEntries.get(newEntries.size() - 1).getIndex(),
-          raftLog.getLatestFlushedIndex());
-    }
-
-    // load the raftlog again and check
-    try (SegmentedRaftLog raftLog =
-             new SegmentedRaftLog(peerId, server, storage, -1, properties)) {
-      raftLog.open(RaftServerConstants.INVALID_LOG_INDEX, null);
-      checkEntries(raftLog, entries, 0, 650);
-      checkEntries(raftLog, newEntries, 100, 100);
-      Assert.assertEquals(newEntries.get(newEntries.size() - 1),
-          getLastEntry(raftLog));
-      Assert.assertEquals(newEntries.get(newEntries.size() - 1).getIndex(),
-          raftLog.getLatestFlushedIndex());
-
-      RaftLogCache cache = raftLog.getRaftLogCache();
-      Assert.assertEquals(5, cache.getNumOfSegments());
-    }
-  }
-
-  @Test
-  public void testSegmentedRaftLogStateMachineData() throws Exception {
-    final SegmentRange range = new SegmentRange(0, 10, 1, true);
-    final List<LogEntryProto> entries = prepareLogEntries(range, null, true, new ArrayList<>());
-
-    final SimpleStateMachine4Testing sm = new SimpleStateMachine4Testing();
-    try (SegmentedRaftLog raftLog = new SegmentedRaftLog(peerId, null, sm, null, storage, -1, properties)) {
-      raftLog.open(RaftServerConstants.INVALID_LOG_INDEX, null);
-
-      int next = 0;
-      long flush = -1;
-      assertIndices(raftLog, flush, next);
-      raftLog.appendEntry(entries.get(next++));
-      assertIndices(raftLog, flush, next);
-      raftLog.appendEntry(entries.get(next++));
-      assertIndices(raftLog, flush, next);
-      raftLog.appendEntry(entries.get(next++));
-      assertIndicesMultipleAttempts(raftLog, flush += 3, next);
-
-      sm.blockFlushStateMachineData();
-      raftLog.appendEntry(entries.get(next++));
-      {
-        sm.blockWriteStateMachineData();
-        final Thread t = startAppendEntryThread(raftLog, entries.get(next++));
-        TimeUnit.SECONDS.sleep(1);
-        Assert.assertTrue(t.isAlive());
-        sm.unblockWriteStateMachineData();
-        t.join();
-      }
-      assertIndices(raftLog, flush, next);
-      TimeUnit.SECONDS.sleep(1);
-      assertIndices(raftLog, flush, next);
-      sm.unblockFlushStateMachineData();
-      assertIndicesMultipleAttempts(raftLog, flush + 2, next);
-    }
-  }
-
-  @Test
-  public void testSegmentedRaftLogStateMachineDataTimeoutIOException() throws Exception {
-    RaftServerConfigKeys.Log.StateMachineData.setSync(properties, true);
-    final TimeDuration syncTimeout = TimeDuration.valueOf(100, TimeUnit.MILLISECONDS);
-    RaftServerConfigKeys.Log.StateMachineData.setSyncTimeout(properties, syncTimeout);
-    final int numRetries = 2;
-    RaftServerConfigKeys.Log.StateMachineData.setSyncTimeoutRetry(properties, numRetries);
-    ExitUtils.disableSystemExit();
-
-    final LogEntryProto entry = prepareLogEntry(0, 0, null, true);
-    final StateMachine sm = new BaseStateMachine() {
-      @Override
-      public CompletableFuture<?> writeStateMachineData(LogEntryProto entry) {
-        return new CompletableFuture<>(); // the future never completes
-      }
-    };
-
-    try (SegmentedRaftLog raftLog = new SegmentedRaftLog(peerId, null, sm, null, storage, -1, properties)) {
-      raftLog.open(RaftServerConstants.INVALID_LOG_INDEX, null);
-      raftLog.appendEntry(entry);  // RaftLogWorker should catch TimeoutIOException
-
-      JavaUtils.attempt(() -> {
-        final ExitUtils.ExitException exitException = ExitUtils.getFirstExitException();
-        Objects.requireNonNull(exitException, "exitException == null");
-        Assert.assertEquals(TimeoutIOException.class, exitException.getCause().getClass());
-      }, 3*numRetries, syncTimeout, "RaftLogWorker should catch TimeoutIOException and exit", LOG);
-    }
-  }
-
-  static Thread startAppendEntryThread(RaftLog raftLog, LogEntryProto entry) {
-    final Thread t = new Thread(() -> raftLog.appendEntry(entry));
-    t.start();
-    return t;
-  }
-
-  void assertIndices(RaftLog raftLog, long expectedFlushIndex, long expectedNextIndex) {
-    LOG.info("assert expectedFlushIndex={}", expectedFlushIndex);
-    Assert.assertEquals(expectedFlushIndex, raftLog.getLatestFlushedIndex());
-    LOG.info("assert expectedNextIndex={}", expectedNextIndex);
-    Assert.assertEquals(expectedNextIndex, raftLog.getNextIndex());
-  }
-
-  void assertIndicesMultipleAttempts(RaftLog raftLog, long expectedFlushIndex, long expectedNextIndex) throws Exception {
-    JavaUtils.attempt(() -> assertIndices(raftLog, expectedFlushIndex, expectedNextIndex),
-        10, 100, "assertIndices", LOG);
-  }
-
-  @Test
-  public void testSegmentedRaftLogFormatInternalHeader() throws Exception {
-    testFailureCase("testSegmentedRaftLogFormatInternalHeader",
-        () -> SegmentedRaftLogFormat.applyHeaderTo(header -> {
-          LOG.info("header  = " + new String(header, StandardCharsets.UTF_8));
-          header[0] += 1; // try changing the internal header
-          LOG.info("header' = " + new String(header, StandardCharsets.UTF_8));
-          return null;
-        }), IllegalStateException.class);
-
-    // reset the header
-    SegmentedRaftLogFormat.applyHeaderTo(header -> {
-      LOG.info("header'  = " + new String(header, StandardCharsets.UTF_8));
-      header[0] -= 1; // try changing the internal header
-      LOG.info("header'' = " + new String(header, StandardCharsets.UTF_8));
-      return null;
-    });
-  }
-}

http://git-wip-us.apache.org/repos/asf/incubator-ratis/blob/5c37675f/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
deleted file mode 100644
index a4dc88a..0000000
--- a/ratis-server/src/test/java/org/apache/ratis/statemachine/TestStateMachine.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.ratis.statemachine;
-
-import org.apache.log4j.Level;
-import org.apache.ratis.BaseTest;
-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.protocol.RaftGroup;
-import org.apache.ratis.protocol.RaftGroupId;
-import org.apache.ratis.protocol.RaftPeer;
-import org.apache.ratis.protocol.RaftPeerId;
-import org.apache.ratis.server.RaftServerConfigKeys;
-import org.apache.ratis.server.impl.RaftServerImpl;
-import org.apache.ratis.server.impl.RaftServerProxy;
-import org.apache.ratis.server.impl.RaftServerTestUtil;
-import org.apache.ratis.server.simulation.MiniRaftClusterWithSimulatedRpc;
-import org.apache.ratis.util.LogUtils;
-import org.junit.*;
-
-import java.io.IOException;
-import java.util.Collections;
-import java.util.List;
-import java.util.Map;
-import java.util.concurrent.CompletableFuture;
-import java.util.concurrent.ConcurrentHashMap;
-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 extends BaseTest implements MiniRaftClusterWithSimulatedRpc.FactoryGet {
-  static {
-    LogUtils.setLogLevel(RaftServerImpl.LOG, Level.DEBUG);
-    LogUtils.setLogLevel(RaftClient.LOG, Level.DEBUG);
-  }
-
-  public static final int NUM_SERVERS = 3;
-
-  static class SMTransactionContext extends SimpleStateMachine4Testing {
-    public static SMTransactionContext get(RaftServerImpl s) {
-      return (SMTransactionContext)s.getStateMachine();
-    }
-
-    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) {
-      // only leader will get this call
-      isLeader.set(true);
-      // send the next transaction id as the "context" from SM
-      return TransactionContext.newBuilder()
-          .setStateMachine(this)
-          .setClientRequest(request)
-          .setStateMachineContext(transactions.incrementAndGet())
-          .build();
-    }
-
-    @Override
-    public CompletableFuture<Message> applyTransaction(TransactionContext trx) {
-      try {
-        assertNotNull(trx.getLogEntry());
-        assertNotNull(trx.getStateMachineLogEntry());
-        Object context = trx.getStateMachineContext();
-        if (isLeader.get()) {
-          assertNotNull(trx.getClientRequest());
-          assertNotNull(context);
-          assertTrue(context instanceof Long);
-          Long val = (Long)context;
-          assertTrue(val <= transactions.get());
-          applied.add(val);
-        } else {
-          assertNull(trx.getClientRequest());
-          assertNull(context);
-        }
-        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 {
-    runTestTransactionContextIsPassedBack(false);
-  }
-
-  @Test
-  public void testTransactionContextIsPassedBackUseMemory() throws Throwable {
-    runTestTransactionContextIsPassedBack(true);
-  }
-
-  void runTestTransactionContextIsPassedBack(boolean useMemory) throws Throwable {
-    final RaftProperties properties = new RaftProperties();
-    properties.setClass(MiniRaftCluster.STATEMACHINE_CLASS_KEY, SMTransactionContext.class, StateMachine.class);
-    RaftServerConfigKeys.Log.setUseMemory(properties, useMemory);
-
-    try(MiniRaftClusterWithSimulatedRpc cluster = getFactory().newCluster(NUM_SERVERS, properties)) {
-      cluster.start();
-      runTestTransactionContextIsPassedBack(cluster);
-    }
-  }
-
-  static void runTestTransactionContextIsPassedBack(MiniRaftCluster cluster) throws Throwable {
-    // tests that the TrxContext set by the StateMachine in Leader is passed back to the SM
-    int numTrx = 100;
-    final RaftTestUtil.SimpleMessage[] messages = RaftTestUtil.SimpleMessage.create(numTrx);
-    try(final RaftClient client = cluster.createClient()) {
-      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.iterateServerImpls()) {
-      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));
-    }
-  }
-
-  @Test
-  public void testStateMachineRegistry() throws Throwable {
-    final Map<RaftGroupId, StateMachine> registry = new ConcurrentHashMap<>();
-    registry.put(RaftGroupId.randomId(), new SimpleStateMachine4Testing());
-    registry.put(RaftGroupId.randomId(), new SMTransactionContext());
-
-    try(MiniRaftClusterWithSimulatedRpc cluster = newCluster(0)) {
-      cluster.setStateMachineRegistry(registry::get);
-
-      final RaftPeerId id = RaftPeerId.valueOf("s0");
-      cluster.putNewServer(id, null, true);
-      cluster.start();
-
-      for(RaftGroupId gid : registry.keySet()) {
-        final RaftGroup newGroup = RaftGroup.valueOf(gid, cluster.getPeers());
-        LOG.info("add new group: " + newGroup);
-        final RaftClient client = cluster.createClient(newGroup);
-        for(RaftPeer p : newGroup.getPeers()) {
-          client.groupAdd(newGroup, p.getId());
-        }
-      }
-
-      final RaftServerProxy proxy = cluster.getServer(id);
-      for(Map.Entry<RaftGroupId, StateMachine> e: registry.entrySet()) {
-        final RaftServerImpl impl = RaftServerTestUtil.getRaftServerImpl(proxy, e.getKey());
-        Assert.assertSame(e.getValue(), impl.getStateMachine());
-      }
-    }
-  }
-}

http://git-wip-us.apache.org/repos/asf/incubator-ratis/blob/5c37675f/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
deleted file mode 100644
index ced0687..0000000
--- a/ratis-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/5c37675f/ratis-test/pom.xml
----------------------------------------------------------------------
diff --git a/ratis-test/pom.xml b/ratis-test/pom.xml
new file mode 100644
index 0000000..9a70ff9
--- /dev/null
+++ b/ratis-test/pom.xml
@@ -0,0 +1,92 @@
+<?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</artifactId>
+    <groupId>org.apache.ratis</groupId>
+    <version>0.3.0-SNAPSHOT</version>
+  </parent>
+
+  <artifactId>ratis-test</artifactId>
+  <name>Apache Ratis Test</name>
+
+  <dependencies>
+    <dependency>
+      <artifactId>ratis-common</artifactId>
+      <groupId>org.apache.ratis</groupId>
+    </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>
+    </dependency>
+
+    <dependency>
+      <artifactId>ratis-server</artifactId>
+      <groupId>org.apache.ratis</groupId>
+    </dependency>
+    <dependency>
+      <artifactId>ratis-server</artifactId>
+      <groupId>org.apache.ratis</groupId>
+      <scope>test</scope>
+      <type>test-jar</type>
+    </dependency>
+
+    <dependency>
+      <artifactId>ratis-netty</artifactId>
+      <groupId>org.apache.ratis</groupId>
+    </dependency>
+    <dependency>
+      <artifactId>ratis-netty</artifactId>
+      <groupId>org.apache.ratis</groupId>
+      <scope>test</scope>
+      <type>test-jar</type>
+    </dependency>
+
+    <dependency>
+      <artifactId>ratis-grpc</artifactId>
+      <groupId>org.apache.ratis</groupId>
+    </dependency>
+    <dependency>
+      <artifactId>ratis-grpc</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>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/5c37675f/ratis-test/src/test/java/org/apache/ratis/TestRaftServerLeaderElectionTimeout.java
----------------------------------------------------------------------
diff --git a/ratis-test/src/test/java/org/apache/ratis/TestRaftServerLeaderElectionTimeout.java b/ratis-test/src/test/java/org/apache/ratis/TestRaftServerLeaderElectionTimeout.java
new file mode 100644
index 0000000..55bcdfc
--- /dev/null
+++ b/ratis-test/src/test/java/org/apache/ratis/TestRaftServerLeaderElectionTimeout.java
@@ -0,0 +1,99 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS 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.client.RaftClient;
+import org.apache.ratis.conf.RaftProperties;
+import org.apache.ratis.server.RaftServerConfigKeys;
+import org.apache.ratis.server.impl.RaftServerImpl;
+import org.apache.ratis.server.simulation.MiniRaftClusterWithSimulatedRpc;
+import org.apache.ratis.proto.RaftProtos;
+import org.apache.ratis.statemachine.SimpleStateMachine4Testing;
+import org.apache.ratis.statemachine.StateMachine;
+import org.apache.ratis.util.LogUtils;
+import org.apache.ratis.util.TimeDuration;
+import org.junit.After;
+import org.junit.Assert;
+import org.junit.Before;
+import org.junit.Test;
+
+import java.io.IOException;
+import java.util.concurrent.TimeUnit;
+
+/**
+ * Test Raft Server Leader election timeout detection and notification to state machine.
+ */
+public class TestRaftServerLeaderElectionTimeout extends BaseTest {
+  static {
+    LogUtils.setLogLevel(RaftServerImpl.LOG, Level.DEBUG);
+    LogUtils.setLogLevel(RaftClient.LOG, Level.DEBUG);
+  }
+
+  public static final int NUM_SERVERS = 3;
+
+  protected static final RaftProperties properties = new RaftProperties();
+
+  private final MiniRaftClusterWithSimulatedRpc cluster = MiniRaftClusterWithSimulatedRpc
+      .FACTORY.newCluster(NUM_SERVERS, getProperties());
+
+  public RaftProperties getProperties() {
+    RaftServerConfigKeys
+        .setLeaderElectionTimeout(properties, TimeDuration.valueOf(1, TimeUnit.SECONDS));
+    properties.setClass(MiniRaftCluster.STATEMACHINE_CLASS_KEY,
+        SimpleStateMachine4Testing.class, StateMachine.class);
+    return properties;
+  }
+
+  @Before
+  public void setup() throws IOException {
+    Assert.assertNull(cluster.getLeader());
+    cluster.start();
+  }
+
+  @After
+  public void tearDown() {
+    if (cluster != null) {
+      cluster.shutdown();
+    }
+  }
+
+  @Test
+  public void testLeaderElectionDetection() throws Exception {
+    RaftTestUtil.waitForLeader(cluster);
+    long leaderElectionTimeout = RaftServerConfigKeys.
+        leaderElectionTimeout(cluster.getProperties()).toInt(TimeUnit.MILLISECONDS);
+
+    RaftServerImpl healthyFollower = cluster.getFollowers().get(1);
+    RaftServerImpl failedFollower = cluster.getFollowers().get(0);
+    // fail the leader and one of the followers to that quorum is not present
+    // for next leader election to succeed.
+    cluster.killServer(failedFollower.getId());
+    cluster.killServer(cluster.getLeader().getId());
+
+    // Wait to ensure that leader election is triggered and also state machine callback is triggered
+    Thread.sleep( leaderElectionTimeout * 2);
+
+    RaftProtos.RoleInfoProto roleInfoProto =
+        SimpleStateMachine4Testing.get(healthyFollower).getLeaderElectionTimeoutInfo();
+    Assert.assertNotNull(roleInfoProto);
+
+    Assert.assertEquals(roleInfoProto.getRole(), RaftProtos.RaftPeerRole.CANDIDATE);
+    Assert.assertTrue(roleInfoProto.getCandidateInfo().getLastLeaderElapsedTimeMs() > leaderElectionTimeout);
+  }
+}

http://git-wip-us.apache.org/repos/asf/incubator-ratis/blob/5c37675f/ratis-test/src/test/java/org/apache/ratis/TestRaftServerSlownessDetection.java
----------------------------------------------------------------------
diff --git a/ratis-test/src/test/java/org/apache/ratis/TestRaftServerSlownessDetection.java b/ratis-test/src/test/java/org/apache/ratis/TestRaftServerSlownessDetection.java
new file mode 100644
index 0000000..96a164e
--- /dev/null
+++ b/ratis-test/src/test/java/org/apache/ratis/TestRaftServerSlownessDetection.java
@@ -0,0 +1,109 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS 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.conf.RaftProperties;
+import org.apache.ratis.protocol.RaftPeerId;
+import org.apache.ratis.server.RaftServerConfigKeys;
+import org.apache.ratis.server.impl.RaftServerImpl;
+import org.apache.ratis.server.simulation.MiniRaftClusterWithSimulatedRpc;
+import org.apache.ratis.proto.RaftProtos;
+import org.apache.ratis.statemachine.SimpleStateMachine4Testing;
+import org.apache.ratis.statemachine.StateMachine;
+import org.apache.ratis.util.LogUtils;
+import org.apache.ratis.util.TimeDuration;
+import org.junit.After;
+import org.junit.Assert;
+import org.junit.Before;
+import org.junit.Ignore;
+import org.junit.Test;
+
+
+import java.io.IOException;
+import java.util.List;
+import java.util.concurrent.TimeUnit;
+
+/**
+ * Test Raft Server Slowness detection and notification to Leader's statemachine.
+ */
+//TODO: fix StateMachine.notifySlowness(..); see RATIS-370
+@Ignore
+public class TestRaftServerSlownessDetection extends BaseTest {
+  static {
+    LogUtils.setLogLevel(RaftServerImpl.LOG, Level.DEBUG);
+  }
+
+  public static final int NUM_SERVERS = 3;
+
+  protected static final RaftProperties properties = new RaftProperties();
+
+  private final MiniRaftClusterWithSimulatedRpc cluster = MiniRaftClusterWithSimulatedRpc
+      .FACTORY.newCluster(NUM_SERVERS, getProperties());
+
+  public RaftProperties getProperties() {
+    RaftServerConfigKeys.Rpc
+        .setSlownessTimeout(properties, TimeDuration.valueOf(1, TimeUnit.SECONDS));
+    properties.setClass(MiniRaftCluster.STATEMACHINE_CLASS_KEY,
+        SimpleStateMachine4Testing.class, StateMachine.class);
+    return properties;
+  }
+
+  @Before
+  public void setup() throws IOException {
+    Assert.assertNull(cluster.getLeader());
+    cluster.start();
+  }
+
+  @After
+  public void tearDown() {
+    if (cluster != null) {
+      cluster.shutdown();
+    }
+  }
+
+  @Test
+  public void testSlownessDetection() throws Exception {
+    RaftTestUtil.waitForLeader(cluster);
+    long slownessTimeout = RaftServerConfigKeys.Rpc
+        .slownessTimeout(cluster.getProperties()).toInt(TimeUnit.MILLISECONDS);
+    RaftServerImpl failedFollower = cluster.getFollowers().get(0);
+
+    // fail the node and wait for the callback to be triggered
+    cluster.killServer(failedFollower.getId());
+    Thread.sleep( slownessTimeout * 2);
+
+    // Followers should not get any failed not notification
+    for (RaftServerImpl followerServer : cluster.getFollowers()) {
+      Assert.assertNull(SimpleStateMachine4Testing.get(followerServer).getSlownessInfo());
+    }
+    // the leader should get notification that the follower has failed now
+    RaftProtos.RoleInfoProto roleInfoProto =
+        SimpleStateMachine4Testing.get(cluster.getLeader()).getSlownessInfo();
+    Assert.assertNotNull(roleInfoProto);
+
+    List<RaftProtos.ServerRpcProto> followers =
+        roleInfoProto.getLeaderInfo().getFollowerInfoList();
+    //Assert that the node shutdown is lagging behind
+    for (RaftProtos.ServerRpcProto serverProto : followers) {
+      if (RaftPeerId.valueOf(serverProto.getId().getId()).equals(failedFollower.getId())) {
+        Assert.assertTrue(serverProto.getLastRpcElapsedTimeMs() > slownessTimeout);
+      }
+    }
+  }
+}

http://git-wip-us.apache.org/repos/asf/incubator-ratis/blob/5c37675f/ratis-test/src/test/java/org/apache/ratis/TestRetryPolicy.java
----------------------------------------------------------------------
diff --git a/ratis-test/src/test/java/org/apache/ratis/TestRetryPolicy.java b/ratis-test/src/test/java/org/apache/ratis/TestRetryPolicy.java
new file mode 100644
index 0000000..d481003
--- /dev/null
+++ b/ratis-test/src/test/java/org/apache/ratis/TestRetryPolicy.java
@@ -0,0 +1,42 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS 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.retry.RetryPolicies;
+import org.apache.ratis.retry.RetryPolicy;
+import org.apache.ratis.util.TimeDuration;
+import org.junit.Assert;
+import org.junit.Test;
+
+import java.util.concurrent.TimeUnit;
+
+
+
+public class TestRetryPolicy {
+
+  @Test
+  public void testRetryMultipleTimesWithFixedSleep() {
+    RetryPolicy retryPolicy = RetryPolicies
+        .retryUpToMaximumCountWithFixedSleep(2,
+            TimeDuration.valueOf(1000L, TimeUnit.MILLISECONDS));
+     boolean shouldRetry = retryPolicy.shouldRetry(1);
+    Assert.assertTrue(shouldRetry);
+    Assert.assertTrue(1000 == retryPolicy.getSleepTime().getDuration());
+    Assert.assertFalse(retryPolicy.shouldRetry(2));
+  }
+}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/incubator-ratis/blob/5c37675f/ratis-test/src/test/java/org/apache/ratis/grpc/TestGroupManagementWithGrpc.java
----------------------------------------------------------------------
diff --git a/ratis-test/src/test/java/org/apache/ratis/grpc/TestGroupManagementWithGrpc.java b/ratis-test/src/test/java/org/apache/ratis/grpc/TestGroupManagementWithGrpc.java
new file mode 100644
index 0000000..657bfd1
--- /dev/null
+++ b/ratis-test/src/test/java/org/apache/ratis/grpc/TestGroupManagementWithGrpc.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.grpc;
+
+import org.apache.ratis.MiniRaftCluster;
+import org.apache.ratis.server.impl.GroupManagementBaseTest;
+
+public class TestGroupManagementWithGrpc extends GroupManagementBaseTest {
+  @Override
+  public MiniRaftCluster.Factory<? extends MiniRaftCluster> getClusterFactory() {
+    return MiniRaftClusterWithGrpc.FACTORY;
+  }
+}

http://git-wip-us.apache.org/repos/asf/incubator-ratis/blob/5c37675f/ratis-test/src/test/java/org/apache/ratis/grpc/TestLeaderElectionWithGrpc.java
----------------------------------------------------------------------
diff --git a/ratis-test/src/test/java/org/apache/ratis/grpc/TestLeaderElectionWithGrpc.java b/ratis-test/src/test/java/org/apache/ratis/grpc/TestLeaderElectionWithGrpc.java
new file mode 100644
index 0000000..eb08336
--- /dev/null
+++ b/ratis-test/src/test/java/org/apache/ratis/grpc/TestLeaderElectionWithGrpc.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.grpc;
+
+import org.apache.ratis.server.impl.BlockRequestHandlingInjection;
+import org.apache.ratis.server.impl.LeaderElectionTests;
+import org.junit.Test;
+
+public class TestLeaderElectionWithGrpc
+    extends LeaderElectionTests<MiniRaftClusterWithGrpc>
+    implements MiniRaftClusterWithGrpc.FactoryGet {
+
+  @Override
+  @Test
+  public void testEnforceLeader() throws Exception {
+    super.testEnforceLeader();
+
+    MiniRaftClusterWithGrpc.sendServerRequestInjection.clear();
+    BlockRequestHandlingInjection.getInstance().unblockAll();
+  }
+}

http://git-wip-us.apache.org/repos/asf/incubator-ratis/blob/5c37675f/ratis-test/src/test/java/org/apache/ratis/grpc/TestRaftAsyncWithGrpc.java
----------------------------------------------------------------------
diff --git a/ratis-test/src/test/java/org/apache/ratis/grpc/TestRaftAsyncWithGrpc.java b/ratis-test/src/test/java/org/apache/ratis/grpc/TestRaftAsyncWithGrpc.java
new file mode 100644
index 0000000..614787e
--- /dev/null
+++ b/ratis-test/src/test/java/org/apache/ratis/grpc/TestRaftAsyncWithGrpc.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.grpc;
+
+import org.apache.ratis.RaftAsyncTests;
+
+public class TestRaftAsyncWithGrpc extends RaftAsyncTests<MiniRaftClusterWithGrpc>
+    implements MiniRaftClusterWithGrpc.FactoryGet {
+}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/incubator-ratis/blob/5c37675f/ratis-test/src/test/java/org/apache/ratis/grpc/TestRaftExceptionWithGrpc.java
----------------------------------------------------------------------
diff --git a/ratis-test/src/test/java/org/apache/ratis/grpc/TestRaftExceptionWithGrpc.java b/ratis-test/src/test/java/org/apache/ratis/grpc/TestRaftExceptionWithGrpc.java
new file mode 100644
index 0000000..d2b71bc
--- /dev/null
+++ b/ratis-test/src/test/java/org/apache/ratis/grpc/TestRaftExceptionWithGrpc.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.ratis.grpc;
+
+import org.apache.ratis.RaftExceptionBaseTest;
+
+public class TestRaftExceptionWithGrpc
+    extends RaftExceptionBaseTest<MiniRaftClusterWithGrpc>
+    implements MiniRaftClusterWithGrpc.FactoryGet {
+}


[5/6] incubator-ratis git commit: RATIS-399. Move all tests to a new module.

Posted by sz...@apache.org.
http://git-wip-us.apache.org/repos/asf/incubator-ratis/blob/5c37675f/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
deleted file mode 100644
index da861d5..0000000
--- a/ratis-netty/src/test/java/org/apache/ratis/netty/TestRaftSnapshotWithNetty.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.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.Factory<?> getFactory() {
-    return MiniRaftClusterWithNetty.FACTORY;
-  }
-}

http://git-wip-us.apache.org/repos/asf/incubator-ratis/blob/5c37675f/ratis-netty/src/test/java/org/apache/ratis/netty/TestRaftStateMachineExceptionWithNetty.java
----------------------------------------------------------------------
diff --git a/ratis-netty/src/test/java/org/apache/ratis/netty/TestRaftStateMachineExceptionWithNetty.java b/ratis-netty/src/test/java/org/apache/ratis/netty/TestRaftStateMachineExceptionWithNetty.java
deleted file mode 100644
index 34fddc9..0000000
--- a/ratis-netty/src/test/java/org/apache/ratis/netty/TestRaftStateMachineExceptionWithNetty.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.ratis.netty;
-
-import org.apache.ratis.server.impl.RaftStateMachineExceptionTests;
-
-public class TestRaftStateMachineExceptionWithNetty
-    extends RaftStateMachineExceptionTests<MiniRaftClusterWithNetty>
-    implements MiniRaftClusterWithNetty.FactoryGet {
-}

http://git-wip-us.apache.org/repos/asf/incubator-ratis/blob/5c37675f/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
deleted file mode 100644
index 28815d7..0000000
--- a/ratis-netty/src/test/java/org/apache/ratis/netty/TestRaftWithNetty.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.ratis.netty;
-
-import org.apache.ratis.RaftBasicTests;
-import org.apache.ratis.server.impl.BlockRequestHandlingInjection;
-import org.junit.Test;
-
-public class TestRaftWithNetty
-    extends RaftBasicTests<MiniRaftClusterWithNetty>
-    implements MiniRaftClusterWithNetty.FactoryGet {
-
-  @Override
-  @Test
-  public void testWithLoad() throws Exception {
-    super.testWithLoad();
-    BlockRequestHandlingInjection.getInstance().unblockAll();
-  }
-}

http://git-wip-us.apache.org/repos/asf/incubator-ratis/blob/5c37675f/ratis-netty/src/test/java/org/apache/ratis/netty/TestRetryCacheWithNettyRpc.java
----------------------------------------------------------------------
diff --git a/ratis-netty/src/test/java/org/apache/ratis/netty/TestRetryCacheWithNettyRpc.java b/ratis-netty/src/test/java/org/apache/ratis/netty/TestRetryCacheWithNettyRpc.java
deleted file mode 100644
index 659e426..0000000
--- a/ratis-netty/src/test/java/org/apache/ratis/netty/TestRetryCacheWithNettyRpc.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.ratis.netty;
-
-import java.io.IOException;
-
-import org.apache.log4j.Level;
-import org.apache.ratis.RetryCacheTests;
-import org.apache.ratis.client.RaftClient;
-import org.apache.ratis.server.impl.RaftServerImpl;
-import org.apache.ratis.util.LogUtils;
-
-public class TestRetryCacheWithNettyRpc extends RetryCacheTests {
-  static {
-    LogUtils.setLogLevel(RaftServerImpl.LOG, Level.DEBUG);
-    LogUtils.setLogLevel(RaftClient.LOG, Level.DEBUG);
-  }
-
-  private final MiniRaftClusterWithNetty cluster;
-
-  public TestRetryCacheWithNettyRpc() throws IOException {
-    cluster = MiniRaftClusterWithNetty.FACTORY.newCluster(
-        NUM_SERVERS, getProperties());
-  }
-
-  @Override
-  public MiniRaftClusterWithNetty getCluster() {
-    return cluster;
-  }
-}

http://git-wip-us.apache.org/repos/asf/incubator-ratis/blob/5c37675f/ratis-netty/src/test/java/org/apache/ratis/netty/TestServerInformationWithNetty.java
----------------------------------------------------------------------
diff --git a/ratis-netty/src/test/java/org/apache/ratis/netty/TestServerInformationWithNetty.java b/ratis-netty/src/test/java/org/apache/ratis/netty/TestServerInformationWithNetty.java
deleted file mode 100644
index d1bcae4..0000000
--- a/ratis-netty/src/test/java/org/apache/ratis/netty/TestServerInformationWithNetty.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.ratis.netty;
-
-import org.apache.ratis.server.impl.ServerInformationBaseTest;
-
-public class TestServerInformationWithNetty
-    extends ServerInformationBaseTest<MiniRaftClusterWithNetty>
-    implements MiniRaftClusterWithNetty.FactoryGet {
-}

http://git-wip-us.apache.org/repos/asf/incubator-ratis/blob/5c37675f/ratis-netty/src/test/java/org/apache/ratis/netty/TestServerRestartWithNetty.java
----------------------------------------------------------------------
diff --git a/ratis-netty/src/test/java/org/apache/ratis/netty/TestServerRestartWithNetty.java b/ratis-netty/src/test/java/org/apache/ratis/netty/TestServerRestartWithNetty.java
deleted file mode 100644
index 15dc688..0000000
--- a/ratis-netty/src/test/java/org/apache/ratis/netty/TestServerRestartWithNetty.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.ratis.netty;
-
-import org.apache.ratis.server.ServerRestartTests;
-
-public class TestServerRestartWithNetty
-    extends ServerRestartTests<MiniRaftClusterWithNetty>
-    implements MiniRaftClusterWithNetty.FactoryGet {
-}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/incubator-ratis/blob/5c37675f/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
deleted file mode 100644
index ced0687..0000000
--- a/ratis-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/5c37675f/ratis-server/src/test/java/org/apache/ratis/TestRaftServerLeaderElectionTimeout.java
----------------------------------------------------------------------
diff --git a/ratis-server/src/test/java/org/apache/ratis/TestRaftServerLeaderElectionTimeout.java b/ratis-server/src/test/java/org/apache/ratis/TestRaftServerLeaderElectionTimeout.java
deleted file mode 100644
index 55bcdfc..0000000
--- a/ratis-server/src/test/java/org/apache/ratis/TestRaftServerLeaderElectionTimeout.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.ratis;
-
-import org.apache.log4j.Level;
-import org.apache.ratis.client.RaftClient;
-import org.apache.ratis.conf.RaftProperties;
-import org.apache.ratis.server.RaftServerConfigKeys;
-import org.apache.ratis.server.impl.RaftServerImpl;
-import org.apache.ratis.server.simulation.MiniRaftClusterWithSimulatedRpc;
-import org.apache.ratis.proto.RaftProtos;
-import org.apache.ratis.statemachine.SimpleStateMachine4Testing;
-import org.apache.ratis.statemachine.StateMachine;
-import org.apache.ratis.util.LogUtils;
-import org.apache.ratis.util.TimeDuration;
-import org.junit.After;
-import org.junit.Assert;
-import org.junit.Before;
-import org.junit.Test;
-
-import java.io.IOException;
-import java.util.concurrent.TimeUnit;
-
-/**
- * Test Raft Server Leader election timeout detection and notification to state machine.
- */
-public class TestRaftServerLeaderElectionTimeout extends BaseTest {
-  static {
-    LogUtils.setLogLevel(RaftServerImpl.LOG, Level.DEBUG);
-    LogUtils.setLogLevel(RaftClient.LOG, Level.DEBUG);
-  }
-
-  public static final int NUM_SERVERS = 3;
-
-  protected static final RaftProperties properties = new RaftProperties();
-
-  private final MiniRaftClusterWithSimulatedRpc cluster = MiniRaftClusterWithSimulatedRpc
-      .FACTORY.newCluster(NUM_SERVERS, getProperties());
-
-  public RaftProperties getProperties() {
-    RaftServerConfigKeys
-        .setLeaderElectionTimeout(properties, TimeDuration.valueOf(1, TimeUnit.SECONDS));
-    properties.setClass(MiniRaftCluster.STATEMACHINE_CLASS_KEY,
-        SimpleStateMachine4Testing.class, StateMachine.class);
-    return properties;
-  }
-
-  @Before
-  public void setup() throws IOException {
-    Assert.assertNull(cluster.getLeader());
-    cluster.start();
-  }
-
-  @After
-  public void tearDown() {
-    if (cluster != null) {
-      cluster.shutdown();
-    }
-  }
-
-  @Test
-  public void testLeaderElectionDetection() throws Exception {
-    RaftTestUtil.waitForLeader(cluster);
-    long leaderElectionTimeout = RaftServerConfigKeys.
-        leaderElectionTimeout(cluster.getProperties()).toInt(TimeUnit.MILLISECONDS);
-
-    RaftServerImpl healthyFollower = cluster.getFollowers().get(1);
-    RaftServerImpl failedFollower = cluster.getFollowers().get(0);
-    // fail the leader and one of the followers to that quorum is not present
-    // for next leader election to succeed.
-    cluster.killServer(failedFollower.getId());
-    cluster.killServer(cluster.getLeader().getId());
-
-    // Wait to ensure that leader election is triggered and also state machine callback is triggered
-    Thread.sleep( leaderElectionTimeout * 2);
-
-    RaftProtos.RoleInfoProto roleInfoProto =
-        SimpleStateMachine4Testing.get(healthyFollower).getLeaderElectionTimeoutInfo();
-    Assert.assertNotNull(roleInfoProto);
-
-    Assert.assertEquals(roleInfoProto.getRole(), RaftProtos.RaftPeerRole.CANDIDATE);
-    Assert.assertTrue(roleInfoProto.getCandidateInfo().getLastLeaderElapsedTimeMs() > leaderElectionTimeout);
-  }
-}

http://git-wip-us.apache.org/repos/asf/incubator-ratis/blob/5c37675f/ratis-server/src/test/java/org/apache/ratis/TestRaftServerSlownessDetection.java
----------------------------------------------------------------------
diff --git a/ratis-server/src/test/java/org/apache/ratis/TestRaftServerSlownessDetection.java b/ratis-server/src/test/java/org/apache/ratis/TestRaftServerSlownessDetection.java
deleted file mode 100644
index 96a164e..0000000
--- a/ratis-server/src/test/java/org/apache/ratis/TestRaftServerSlownessDetection.java
+++ /dev/null
@@ -1,109 +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;
-
-import org.apache.log4j.Level;
-import org.apache.ratis.conf.RaftProperties;
-import org.apache.ratis.protocol.RaftPeerId;
-import org.apache.ratis.server.RaftServerConfigKeys;
-import org.apache.ratis.server.impl.RaftServerImpl;
-import org.apache.ratis.server.simulation.MiniRaftClusterWithSimulatedRpc;
-import org.apache.ratis.proto.RaftProtos;
-import org.apache.ratis.statemachine.SimpleStateMachine4Testing;
-import org.apache.ratis.statemachine.StateMachine;
-import org.apache.ratis.util.LogUtils;
-import org.apache.ratis.util.TimeDuration;
-import org.junit.After;
-import org.junit.Assert;
-import org.junit.Before;
-import org.junit.Ignore;
-import org.junit.Test;
-
-
-import java.io.IOException;
-import java.util.List;
-import java.util.concurrent.TimeUnit;
-
-/**
- * Test Raft Server Slowness detection and notification to Leader's statemachine.
- */
-//TODO: fix StateMachine.notifySlowness(..); see RATIS-370
-@Ignore
-public class TestRaftServerSlownessDetection extends BaseTest {
-  static {
-    LogUtils.setLogLevel(RaftServerImpl.LOG, Level.DEBUG);
-  }
-
-  public static final int NUM_SERVERS = 3;
-
-  protected static final RaftProperties properties = new RaftProperties();
-
-  private final MiniRaftClusterWithSimulatedRpc cluster = MiniRaftClusterWithSimulatedRpc
-      .FACTORY.newCluster(NUM_SERVERS, getProperties());
-
-  public RaftProperties getProperties() {
-    RaftServerConfigKeys.Rpc
-        .setSlownessTimeout(properties, TimeDuration.valueOf(1, TimeUnit.SECONDS));
-    properties.setClass(MiniRaftCluster.STATEMACHINE_CLASS_KEY,
-        SimpleStateMachine4Testing.class, StateMachine.class);
-    return properties;
-  }
-
-  @Before
-  public void setup() throws IOException {
-    Assert.assertNull(cluster.getLeader());
-    cluster.start();
-  }
-
-  @After
-  public void tearDown() {
-    if (cluster != null) {
-      cluster.shutdown();
-    }
-  }
-
-  @Test
-  public void testSlownessDetection() throws Exception {
-    RaftTestUtil.waitForLeader(cluster);
-    long slownessTimeout = RaftServerConfigKeys.Rpc
-        .slownessTimeout(cluster.getProperties()).toInt(TimeUnit.MILLISECONDS);
-    RaftServerImpl failedFollower = cluster.getFollowers().get(0);
-
-    // fail the node and wait for the callback to be triggered
-    cluster.killServer(failedFollower.getId());
-    Thread.sleep( slownessTimeout * 2);
-
-    // Followers should not get any failed not notification
-    for (RaftServerImpl followerServer : cluster.getFollowers()) {
-      Assert.assertNull(SimpleStateMachine4Testing.get(followerServer).getSlownessInfo());
-    }
-    // the leader should get notification that the follower has failed now
-    RaftProtos.RoleInfoProto roleInfoProto =
-        SimpleStateMachine4Testing.get(cluster.getLeader()).getSlownessInfo();
-    Assert.assertNotNull(roleInfoProto);
-
-    List<RaftProtos.ServerRpcProto> followers =
-        roleInfoProto.getLeaderInfo().getFollowerInfoList();
-    //Assert that the node shutdown is lagging behind
-    for (RaftProtos.ServerRpcProto serverProto : followers) {
-      if (RaftPeerId.valueOf(serverProto.getId().getId()).equals(failedFollower.getId())) {
-        Assert.assertTrue(serverProto.getLastRpcElapsedTimeMs() > slownessTimeout);
-      }
-    }
-  }
-}

http://git-wip-us.apache.org/repos/asf/incubator-ratis/blob/5c37675f/ratis-server/src/test/java/org/apache/ratis/protocol/TestRaftId.java
----------------------------------------------------------------------
diff --git a/ratis-server/src/test/java/org/apache/ratis/protocol/TestRaftId.java b/ratis-server/src/test/java/org/apache/ratis/protocol/TestRaftId.java
deleted file mode 100644
index 30b7ed5..0000000
--- a/ratis-server/src/test/java/org/apache/ratis/protocol/TestRaftId.java
+++ /dev/null
@@ -1,54 +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.protocol;
-
-import org.apache.ratis.BaseTest;
-import org.apache.ratis.thirdparty.com.google.protobuf.ByteString;
-import org.junit.Assert;
-import org.junit.Test;
-
-public class TestRaftId extends BaseTest {
-  @Override
-  public int getGlobalTimeoutSeconds() {
-    return 1;
-  }
-
-  @Test
-  public void testClientId() {
-    final ClientId id = ClientId.randomId();
-    final ByteString bytes = id.toByteString();
-    Assert.assertEquals(bytes, id.toByteString());
-    Assert.assertEquals(id, ClientId.valueOf(bytes));
-  }
-
-  @Test
-  public void testRaftGroupId() {
-    final RaftGroupId id = RaftGroupId.randomId();
-    final ByteString bytes = id.toByteString();
-    Assert.assertEquals(bytes, id.toByteString());
-    Assert.assertEquals(id, RaftGroupId.valueOf(bytes));
-  }
-
-  @Test
-  public void testRaftPeerId() {
-    final RaftPeerId id = RaftPeerId.valueOf("abc");
-    final ByteString bytes = id.toByteString();
-    Assert.assertEquals(bytes, id.toByteString());
-    Assert.assertEquals(id, RaftPeerId.valueOf(bytes));
-  }
-}

http://git-wip-us.apache.org/repos/asf/incubator-ratis/blob/5c37675f/ratis-server/src/test/java/org/apache/ratis/server/TestRaftLogMetrics.java
----------------------------------------------------------------------
diff --git a/ratis-server/src/test/java/org/apache/ratis/server/TestRaftLogMetrics.java b/ratis-server/src/test/java/org/apache/ratis/server/TestRaftLogMetrics.java
deleted file mode 100644
index 58e319d..0000000
--- a/ratis-server/src/test/java/org/apache/ratis/server/TestRaftLogMetrics.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
- * <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;
-
-import com.codahale.metrics.Timer;
-import org.apache.log4j.Level;
-import org.apache.ratis.BaseTest;
-import org.apache.ratis.MiniRaftCluster;
-import org.apache.ratis.RaftTestUtil;
-import org.apache.ratis.client.RaftClient;
-import org.apache.ratis.metrics.RatisMetricsRegistry;
-import org.apache.ratis.server.impl.RaftServerImpl;
-import org.apache.ratis.server.simulation.MiniRaftClusterWithSimulatedRpc;
-import org.apache.ratis.server.storage.RaftStorageTestUtils;
-import org.apache.ratis.statemachine.StateMachine;
-import org.apache.ratis.statemachine.impl.BaseStateMachine;
-import org.apache.ratis.util.JavaUtils;
-import org.apache.ratis.util.LogUtils;
-import org.junit.Assert;
-import org.junit.Test;
-
-import javax.management.ObjectName;
-import java.lang.management.ManagementFactory;
-import java.util.concurrent.CompletableFuture;
-import java.util.concurrent.atomic.AtomicInteger;
-
-public class TestRaftLogMetrics extends BaseTest
-    implements MiniRaftClusterWithSimulatedRpc.FactoryGet {
-
-  {
-    LogUtils.setLogLevel(RaftServerImpl.LOG, Level.DEBUG);
-  }
-
-  public static final int NUM_SERVERS = 3;
-
-  {
-    getProperties().setClass(MiniRaftCluster.STATEMACHINE_CLASS_KEY,
-        MetricsStateMachine.class, StateMachine.class);
-  }
-
-  static class MetricsStateMachine extends BaseStateMachine {
-    static MetricsStateMachine get(RaftServerImpl s) {
-      return (MetricsStateMachine)s.getStateMachine();
-    }
-
-    private final AtomicInteger flushCount = new AtomicInteger();
-
-    int getFlushCount() {
-      return flushCount.get();
-    }
-
-    @Override
-    public CompletableFuture<Void> flushStateMachineData(long index) {
-      flushCount.incrementAndGet();
-      return super.flushStateMachineData(index);
-    }
-  }
-
-  @Test
-  public void testFlushMetric() throws Exception {
-    try(final MiniRaftCluster cluster = newCluster(NUM_SERVERS)) {
-      cluster.start();
-      runTestFlushMetric(cluster);
-    }
-  }
-
-  static void runTestFlushMetric(MiniRaftCluster cluster) throws Exception {
-    int numMsg = 2;
-    final RaftTestUtil.SimpleMessage[] messages = RaftTestUtil.SimpleMessage.create(numMsg);
-
-    try (final RaftClient client = cluster.createClient()) {
-      for (RaftTestUtil.SimpleMessage message : messages) {
-        client.send(message);
-      }
-    }
-
-    // For leader, flush must happen before client can get replies.
-    assertFlushCount(cluster.getLeader());
-
-    // For followers, flush can be lagged behind.  Attempt multiple times.
-    for(RaftServerImpl f : cluster.getFollowers()) {
-      JavaUtils.attempt(() -> assertFlushCount(f), 10, 100, f.getId() + "-assertFlushCount", null);
-    }
-  }
-
-  static void assertFlushCount(RaftServerImpl server) throws Exception {
-      final String flushTimeMetric = RaftStorageTestUtils.getLogFlushTimeMetric(server.getId());
-      Timer tm = RatisMetricsRegistry.getRegistry().getTimers().get(flushTimeMetric);
-      Assert.assertNotNull(tm);
-
-      final MetricsStateMachine stateMachine = MetricsStateMachine.get(server);
-      final int expectedFlush = stateMachine.getFlushCount();
-
-      Assert.assertEquals(expectedFlush, tm.getCount());
-      Assert.assertTrue(tm.getMeanRate() > 0);
-
-      // Test jmx
-      ObjectName oname = new ObjectName("metrics", "name", flushTimeMetric);
-      Assert.assertEquals(expectedFlush,
-          ((Long) ManagementFactory.getPlatformMBeanServer().getAttribute(oname, "Count"))
-              .intValue());
-  }
-}

http://git-wip-us.apache.org/repos/asf/incubator-ratis/blob/5c37675f/ratis-server/src/test/java/org/apache/ratis/server/TestRaftServerConfigKeys.java
----------------------------------------------------------------------
diff --git a/ratis-server/src/test/java/org/apache/ratis/server/TestRaftServerConfigKeys.java b/ratis-server/src/test/java/org/apache/ratis/server/TestRaftServerConfigKeys.java
deleted file mode 100644
index 53c8871..0000000
--- a/ratis-server/src/test/java/org/apache/ratis/server/TestRaftServerConfigKeys.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.ratis.server;
-
-import org.apache.ratis.BaseTest;
-import org.apache.ratis.conf.RaftProperties;
-import org.apache.ratis.util.FileUtils;
-import org.apache.ratis.util.JavaUtils;
-import org.junit.AfterClass;
-import org.junit.Assert;
-import org.junit.Test;
-
-import java.io.File;
-import java.io.IOException;
-import java.util.ArrayList;
-import java.util.List;
-import java.util.UUID;
-import java.util.concurrent.ThreadLocalRandom;
-import java.util.function.Supplier;
-import java.util.stream.Collectors;
-import java.util.stream.IntStream;
-
-/**
- * Test cases to verify RaftServerConfigKeys.
- */
-public class TestRaftServerConfigKeys {
-
-  private static final Supplier<File> rootTestDir = JavaUtils.memoize(
-      () -> new File(BaseTest.getRootTestDir(),
-          TestRaftServerConfigKeys.class.getSimpleName() +
-              Integer.toHexString(ThreadLocalRandom.current().nextInt())));
-
-  @AfterClass
-  public static void tearDown() throws IOException {
-    FileUtils.deleteFully(rootTestDir.get());
-  }
-
-  /**
-   * Sets the value to <code>raft.server.storage.dir</code> via
-   * RaftServerConfigKeys and verifies it by reading directly from property.
-   */
-  @Test
-  public void testStorageDirsProperty() {
-    final File testDir = new File(
-        rootTestDir.get(), UUID.randomUUID().toString());
-    final List<File> directories = new ArrayList<>();
-    final  RaftProperties properties = new RaftProperties();
-
-    IntStream.range(0, 10).mapToObj((i) -> new File(testDir,
-        Integer.toString(i))).forEach(directories::add);
-    RaftServerConfigKeys.setStorageDirs(properties, directories);
-
-    final String expected = directories.stream().map(File::getAbsolutePath)
-        .collect(Collectors.joining(","));
-    final String actual = properties.get(RaftServerConfigKeys.STORAGE_DIR_KEY);
-    Assert.assertEquals(expected, actual);
-  }
-
-  /**
-   * Sets the value to <code>raft.server.storage.dir</code> via
-   * RaftServerConfigKeys and also verifies the same via RaftServerConfigKeys.
-   */
-  @Test
-  public void testStorageDirs() {
-    final File testDir = new File(
-        rootTestDir.get(), UUID.randomUUID().toString());
-    final List<File> directories = new ArrayList<>();
-    IntStream.range(0, 10).mapToObj((i) -> new File(testDir,
-        Integer.toString(i))).forEach(directories::add);
-    RaftProperties properties = new RaftProperties();
-    RaftServerConfigKeys.setStorageDirs(properties, directories);
-
-    final List<File> storageDirs = RaftServerConfigKeys.storageDirs(properties);
-    final List<String> expectedDirs = directories.stream()
-        .map(File::getAbsolutePath).collect(Collectors.toList());
-    final List<String> actualDirs = storageDirs.stream()
-        .map(File::getAbsolutePath).collect(Collectors.toList());
-    actualDirs.removeAll(expectedDirs);
-    Assert.assertEquals(directories.size(), storageDirs.size());
-    Assert.assertEquals(0, actualDirs.size());
-  }
-}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/incubator-ratis/blob/5c37675f/ratis-server/src/test/java/org/apache/ratis/server/impl/TestRaftServerJmx.java
----------------------------------------------------------------------
diff --git a/ratis-server/src/test/java/org/apache/ratis/server/impl/TestRaftServerJmx.java b/ratis-server/src/test/java/org/apache/ratis/server/impl/TestRaftServerJmx.java
deleted file mode 100644
index f060645..0000000
--- a/ratis-server/src/test/java/org/apache/ratis/server/impl/TestRaftServerJmx.java
+++ /dev/null
@@ -1,109 +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.ratis.server.impl;
-
-import org.apache.ratis.BaseTest;
-import org.apache.ratis.conf.RaftProperties;
-import org.apache.ratis.protocol.RaftGroupId;
-import org.apache.ratis.protocol.RaftPeerId;
-import org.apache.ratis.server.RaftServerMXBean;
-import org.apache.ratis.server.simulation.MiniRaftClusterWithSimulatedRpc;
-import org.apache.ratis.util.JmxRegister;
-import org.junit.Assert;
-import org.junit.Test;
-
-import javax.management.JMException;
-import javax.management.MBeanServer;
-import javax.management.ObjectInstance;
-import javax.management.ObjectName;
-import java.lang.management.ManagementFactory;
-import java.util.List;
-import java.util.Set;
-
-import static org.apache.ratis.RaftTestUtil.waitForLeader;
-
-public class TestRaftServerJmx extends BaseTest {
-  @Test(timeout = 30000)
-  public void testJmxBeans() throws Exception {
-    final int NUM_SERVERS = 3;
-    final MiniRaftClusterWithSimulatedRpc cluster
-        = MiniRaftClusterWithSimulatedRpc.FACTORY.newCluster(3, new RaftProperties());
-    cluster.start();
-    waitForLeader(cluster);
-
-    MBeanServer platformMBeanServer = ManagementFactory.getPlatformMBeanServer();
-    Set<ObjectInstance> objectInstances = platformMBeanServer.queryMBeans(new ObjectName("Ratis:*"), null);
-    Assert.assertEquals(NUM_SERVERS, objectInstances.size());
-
-    for (ObjectInstance instance : objectInstances) {
-      Object groupId = platformMBeanServer.getAttribute(instance.getObjectName(), "GroupId");
-      Assert.assertEquals(cluster.getGroupId().toString(), groupId);
-    }
-    cluster.shutdown();
-  }
-
-  @Test(timeout = 30000)
-  public void testRegister() throws JMException {
-    {
-      final JmxRegister jmx = new JmxRegister();
-      runUnregister(false, jmx);
-
-      runRegister(true, "abc", jmx);
-      runRegister(false, "abc", jmx);
-      runUnregister(true, jmx);
-      runUnregister(false, jmx);
-
-      runRegister(true, "abc", jmx);
-      runUnregister(true, jmx);
-      runUnregister(false, jmx);
-    }
-
-    {
-      final JmxRegister jmx = new JmxRegister();
-      runRegister(true, "host:1234", jmx);
-      runUnregister(true, jmx);
-      runUnregister(false, jmx);
-    }
-  }
-
-  static void runRegister(boolean expectToSucceed, String name, JmxRegister jmx) {
-    final RaftServerMXBean mBean = new RaftServerMXBean() {
-      @Override
-      public String getId() { return null; }
-      @Override
-      public String getLeaderId() { return null; }
-      @Override
-      public long getCurrentTerm() { return 0; }
-      @Override
-      public String getGroupId() { return null; }
-      @Override
-      public String getRole() { return null; }
-      @Override
-      public List<String> getFollowers() { return null; }
-    };
-    final RaftPeerId id = RaftPeerId.valueOf(name);
-    final RaftGroupId groupId = RaftGroupId.randomId();
-    final boolean succeeded = RaftServerImpl.registerMBean(id, groupId, mBean, jmx);
-    Assert.assertEquals(expectToSucceed, succeeded);
-  }
-
-  static void runUnregister(boolean expectToSucceed, JmxRegister jmx) throws JMException {
-    final boolean succeeded = jmx.unregister();
-    Assert.assertEquals(expectToSucceed, succeeded);
-  }
-}

http://git-wip-us.apache.org/repos/asf/incubator-ratis/blob/5c37675f/ratis-server/src/test/java/org/apache/ratis/server/impl/TestServerState.java
----------------------------------------------------------------------
diff --git a/ratis-server/src/test/java/org/apache/ratis/server/impl/TestServerState.java b/ratis-server/src/test/java/org/apache/ratis/server/impl/TestServerState.java
deleted file mode 100644
index 5801d2b..0000000
--- a/ratis-server/src/test/java/org/apache/ratis/server/impl/TestServerState.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.ratis.server.impl;
-
-import org.apache.ratis.BaseTest;
-import org.apache.ratis.util.FileUtils;
-import org.apache.ratis.util.JavaUtils;
-import org.junit.AfterClass;
-import org.junit.Assert;
-import org.junit.Test;
-
-import java.io.File;
-import java.io.IOException;
-import java.util.ArrayList;
-import java.util.Collections;
-import java.util.List;
-import java.util.UUID;
-import java.util.concurrent.ThreadLocalRandom;
-import java.util.function.Supplier;
-import java.util.stream.IntStream;
-
-/**
- * Test cases to verify ServerState.
- */
-public class TestServerState {
-
-  private static final Supplier<File> rootTestDir = JavaUtils.memoize(
-      () -> new File(BaseTest.getRootTestDir(),
-          TestServerState.class.getSimpleName() +
-              Integer.toHexString(ThreadLocalRandom.current().nextInt())));
-
-  @AfterClass
-  public static void tearDown() throws IOException {
-    FileUtils.deleteFully(rootTestDir.get());
-  }
-
-  /**
-   * Tests choosing of storage directory when only one volume is configured.
-   *
-   * @throws IOException in case of exception.
-   */
-  @Test
-  public void testChooseStorageDirWithOneVolume() throws IOException {
-    File testDir = new File(rootTestDir.get(), UUID.randomUUID().toString());
-    List<File> directories = Collections.singletonList(testDir);
-    String subDirOne = UUID.randomUUID().toString();
-    String subDirTwo = UUID.randomUUID().toString();
-    File storageDirOne = ServerState.chooseStorageDir(directories, subDirOne);
-    File storageDirTwo = ServerState.chooseStorageDir(directories, subDirTwo);
-    File expectedOne = new File(testDir, subDirOne);
-    File expectedTwo = new File(testDir, subDirTwo);
-    Assert.assertEquals(expectedOne.getCanonicalPath(),
-        storageDirOne.getCanonicalPath());
-    Assert.assertEquals(expectedTwo.getCanonicalPath(),
-        storageDirTwo.getCanonicalPath());
-  }
-
-  /**
-   * Tests choosing of storage directory when multiple volumes are configured.
-   *
-   * @throws IOException in case of exception.
-   */
-  @Test
-  public void testChooseStorageDirWithMultipleVolumes() throws IOException {
-    File testDir = new File(rootTestDir.get(), UUID.randomUUID().toString());
-    List<File> directories = new ArrayList<>();
-    IntStream.range(0, 10).mapToObj((i) -> new File(testDir,
-        Integer.toString(i))).forEach((dir) -> {
-      try {
-        FileUtils.createDirectories(dir);
-        directories.add(dir);
-      } catch (IOException e) {
-        throw new RuntimeException(e);
-      }
-    });
-
-    directories.stream().filter((dir) -> Integer.parseInt(dir.getName()) != 6)
-        .forEach(
-            (dir) -> {
-              try {
-                FileUtils.createDirectories(
-                    new File(dir, UUID.randomUUID().toString()));
-              } catch (IOException e) {
-                throw new RuntimeException(e);
-              }
-            });
-    String subDir = UUID.randomUUID().toString();
-    File storageDirectory = ServerState.chooseStorageDir(directories, subDir);
-    File expected = new File(directories.get(6), subDir);
-    Assert.assertEquals(expected.getCanonicalPath(),
-        storageDirectory.getCanonicalPath());
-  }
-
-  /**
-   * Tests choosing of storage directory when only no volume is configured.
-   *
-   * @throws IOException in case of exception.
-   */
-  @Test
-  public void testChooseStorageDirWithNoVolume() {
-    try {
-      ServerState.chooseStorageDir(
-          Collections.emptyList(), UUID.randomUUID().toString());
-      Assert.fail();
-    } catch (IOException ex) {
-      String expectedErrMsg = "No storage directory found.";
-      Assert.assertEquals(expectedErrMsg, ex.getMessage());
-    }
-  }
-
-}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/incubator-ratis/blob/5c37675f/ratis-server/src/test/java/org/apache/ratis/server/simulation/TestGroupManagementWithSimulatedRpc.java
----------------------------------------------------------------------
diff --git a/ratis-server/src/test/java/org/apache/ratis/server/simulation/TestGroupManagementWithSimulatedRpc.java b/ratis-server/src/test/java/org/apache/ratis/server/simulation/TestGroupManagementWithSimulatedRpc.java
deleted file mode 100644
index 73fbae9..0000000
--- a/ratis-server/src/test/java/org/apache/ratis/server/simulation/TestGroupManagementWithSimulatedRpc.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.ratis.server.simulation;
-
-import org.apache.ratis.MiniRaftCluster;
-import org.apache.ratis.server.impl.GroupManagementBaseTest;
-
-public class TestGroupManagementWithSimulatedRpc extends GroupManagementBaseTest {
-  @Override
-  public MiniRaftCluster.Factory<? extends MiniRaftCluster> getClusterFactory() {
-    return MiniRaftClusterWithSimulatedRpc.FACTORY;
-  }
-}

http://git-wip-us.apache.org/repos/asf/incubator-ratis/blob/5c37675f/ratis-server/src/test/java/org/apache/ratis/server/simulation/TestLeaderElectionWithSimulatedRpc.java
----------------------------------------------------------------------
diff --git a/ratis-server/src/test/java/org/apache/ratis/server/simulation/TestLeaderElectionWithSimulatedRpc.java b/ratis-server/src/test/java/org/apache/ratis/server/simulation/TestLeaderElectionWithSimulatedRpc.java
deleted file mode 100644
index 7ea273b..0000000
--- a/ratis-server/src/test/java/org/apache/ratis/server/simulation/TestLeaderElectionWithSimulatedRpc.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.ratis.server.simulation;
-
-import org.apache.ratis.server.impl.LeaderElectionTests;
-
-public class TestLeaderElectionWithSimulatedRpc
-    extends LeaderElectionTests<MiniRaftClusterWithSimulatedRpc>
-    implements MiniRaftClusterWithSimulatedRpc.FactoryGet {
-}

http://git-wip-us.apache.org/repos/asf/incubator-ratis/blob/5c37675f/ratis-server/src/test/java/org/apache/ratis/server/simulation/TestRaftExceptionWithSimulation.java
----------------------------------------------------------------------
diff --git a/ratis-server/src/test/java/org/apache/ratis/server/simulation/TestRaftExceptionWithSimulation.java b/ratis-server/src/test/java/org/apache/ratis/server/simulation/TestRaftExceptionWithSimulation.java
deleted file mode 100644
index f328ea3..0000000
--- a/ratis-server/src/test/java/org/apache/ratis/server/simulation/TestRaftExceptionWithSimulation.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.ratis.server.simulation;
-
-import org.apache.ratis.RaftExceptionBaseTest;
-
-public class TestRaftExceptionWithSimulation
-    extends RaftExceptionBaseTest<MiniRaftClusterWithSimulatedRpc>
-    implements MiniRaftClusterWithSimulatedRpc.FactoryGet {
-}

http://git-wip-us.apache.org/repos/asf/incubator-ratis/blob/5c37675f/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
deleted file mode 100644
index b1bb26a..0000000
--- a/ratis-server/src/test/java/org/apache/ratis/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.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 MiniRaftClusterWithSimulatedRpc.FACTORY.newCluster(peerNum, prop);
-  }
-}

http://git-wip-us.apache.org/repos/asf/incubator-ratis/blob/5c37675f/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
deleted file mode 100644
index 8345872..0000000
--- a/ratis-server/src/test/java/org/apache/ratis/server/simulation/TestRaftSnapshotWithSimulatedRpc.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.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.Factory<?> getFactory() {
-    return MiniRaftClusterWithSimulatedRpc.FACTORY;
-  }
-}

http://git-wip-us.apache.org/repos/asf/incubator-ratis/blob/5c37675f/ratis-server/src/test/java/org/apache/ratis/server/simulation/TestRaftStateMachineExceptionWithSimulatedRpc.java
----------------------------------------------------------------------
diff --git a/ratis-server/src/test/java/org/apache/ratis/server/simulation/TestRaftStateMachineExceptionWithSimulatedRpc.java b/ratis-server/src/test/java/org/apache/ratis/server/simulation/TestRaftStateMachineExceptionWithSimulatedRpc.java
deleted file mode 100644
index d8ca86f..0000000
--- a/ratis-server/src/test/java/org/apache/ratis/server/simulation/TestRaftStateMachineExceptionWithSimulatedRpc.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.ratis.server.simulation;
-
-import org.apache.ratis.server.impl.RaftStateMachineExceptionTests;
-
-public class TestRaftStateMachineExceptionWithSimulatedRpc extends
-    RaftStateMachineExceptionTests<MiniRaftClusterWithSimulatedRpc>
-    implements MiniRaftClusterWithSimulatedRpc.FactoryGet {
-}

http://git-wip-us.apache.org/repos/asf/incubator-ratis/blob/5c37675f/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
deleted file mode 100644
index 391a6fa..0000000
--- a/ratis-server/src/test/java/org/apache/ratis/server/simulation/TestRaftWithSimulatedRpc.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.ratis.server.simulation;
-
-import org.apache.ratis.RaftBasicTests;
-
-public class TestRaftWithSimulatedRpc
-    extends RaftBasicTests<MiniRaftClusterWithSimulatedRpc>
-    implements MiniRaftClusterWithSimulatedRpc.FactoryGet {
-}

http://git-wip-us.apache.org/repos/asf/incubator-ratis/blob/5c37675f/ratis-server/src/test/java/org/apache/ratis/server/simulation/TestRetryCacheWithSimulatedRpc.java
----------------------------------------------------------------------
diff --git a/ratis-server/src/test/java/org/apache/ratis/server/simulation/TestRetryCacheWithSimulatedRpc.java b/ratis-server/src/test/java/org/apache/ratis/server/simulation/TestRetryCacheWithSimulatedRpc.java
deleted file mode 100644
index a088578..0000000
--- a/ratis-server/src/test/java/org/apache/ratis/server/simulation/TestRetryCacheWithSimulatedRpc.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.ratis.server.simulation;
-
-import java.io.IOException;
-
-import org.apache.log4j.Level;
-import org.apache.ratis.RetryCacheTests;
-import org.apache.ratis.client.RaftClient;
-import org.apache.ratis.server.impl.RaftServerImpl;
-import org.apache.ratis.util.LogUtils;
-
-public class TestRetryCacheWithSimulatedRpc extends RetryCacheTests {
-  static {
-    LogUtils.setLogLevel(RaftServerImpl.LOG, Level.DEBUG);
-    LogUtils.setLogLevel(RaftClient.LOG, Level.DEBUG);
-  }
-
-  private final MiniRaftClusterWithSimulatedRpc cluster;
-
-  public TestRetryCacheWithSimulatedRpc() throws IOException {
-    cluster = MiniRaftClusterWithSimulatedRpc.FACTORY.newCluster(
-        NUM_SERVERS, getProperties());
-  }
-
-  @Override
-  public MiniRaftClusterWithSimulatedRpc getCluster() {
-    return cluster;
-  }
-}

http://git-wip-us.apache.org/repos/asf/incubator-ratis/blob/5c37675f/ratis-server/src/test/java/org/apache/ratis/server/simulation/TestServerInformationWithSimulatedRpc.java
----------------------------------------------------------------------
diff --git a/ratis-server/src/test/java/org/apache/ratis/server/simulation/TestServerInformationWithSimulatedRpc.java b/ratis-server/src/test/java/org/apache/ratis/server/simulation/TestServerInformationWithSimulatedRpc.java
deleted file mode 100644
index f7025a5..0000000
--- a/ratis-server/src/test/java/org/apache/ratis/server/simulation/TestServerInformationWithSimulatedRpc.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.ratis.server.simulation;
-
-import org.apache.ratis.server.impl.ServerInformationBaseTest;
-
-public class TestServerInformationWithSimulatedRpc
-    extends ServerInformationBaseTest<MiniRaftClusterWithSimulatedRpc>
-    implements MiniRaftClusterWithSimulatedRpc.FactoryGet {
-}

http://git-wip-us.apache.org/repos/asf/incubator-ratis/blob/5c37675f/ratis-server/src/test/java/org/apache/ratis/server/simulation/TestServerRestartWithSimulatedRpc.java
----------------------------------------------------------------------
diff --git a/ratis-server/src/test/java/org/apache/ratis/server/simulation/TestServerRestartWithSimulatedRpc.java b/ratis-server/src/test/java/org/apache/ratis/server/simulation/TestServerRestartWithSimulatedRpc.java
deleted file mode 100644
index 306e5e7..0000000
--- a/ratis-server/src/test/java/org/apache/ratis/server/simulation/TestServerRestartWithSimulatedRpc.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.ratis.server.simulation;
-
-import org.apache.ratis.server.ServerRestartTests;
-
-public class TestServerRestartWithSimulatedRpc
-    extends ServerRestartTests<MiniRaftClusterWithSimulatedRpc>
-    implements MiniRaftClusterWithSimulatedRpc.FactoryGet {
-}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/incubator-ratis/blob/5c37675f/ratis-server/src/test/java/org/apache/ratis/server/storage/TestCacheEviction.java
----------------------------------------------------------------------
diff --git a/ratis-server/src/test/java/org/apache/ratis/server/storage/TestCacheEviction.java b/ratis-server/src/test/java/org/apache/ratis/server/storage/TestCacheEviction.java
deleted file mode 100644
index 1cd41a5..0000000
--- a/ratis-server/src/test/java/org/apache/ratis/server/storage/TestCacheEviction.java
+++ /dev/null
@@ -1,198 +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.storage;
-
-import org.apache.ratis.BaseTest;
-import org.apache.ratis.MiniRaftCluster;
-import org.apache.ratis.RaftTestUtil.SimpleOperation;
-import org.apache.ratis.conf.RaftProperties;
-import org.apache.ratis.protocol.RaftPeerId;
-import org.apache.ratis.server.RaftServerConfigKeys;
-import org.apache.ratis.server.impl.RaftServerConstants;
-import org.apache.ratis.server.impl.RaftServerImpl;
-import org.apache.ratis.server.impl.ServerProtoUtils;
-import org.apache.ratis.server.impl.ServerState;
-import org.apache.ratis.server.storage.CacheInvalidationPolicy.CacheInvalidationPolicyDefault;
-import org.apache.ratis.server.storage.TestSegmentedRaftLog.SegmentRange;
-import org.apache.ratis.proto.RaftProtos.LogEntryProto;
-import org.apache.ratis.statemachine.SimpleStateMachine4Testing;
-import org.apache.ratis.statemachine.StateMachine;
-import org.apache.ratis.util.SizeInBytes;
-import org.junit.Assert;
-import org.junit.Test;
-import org.mockito.Mockito;
-
-import java.io.File;
-import java.util.ArrayList;
-import java.util.Collections;
-import java.util.List;
-import java.util.concurrent.CompletableFuture;
-
-public class TestCacheEviction extends BaseTest {
-  private static final CacheInvalidationPolicy policy = new CacheInvalidationPolicyDefault();
-
-  private List<LogSegment> prepareSegments(int numSegments, boolean[] cached, long start, long size) {
-    Assert.assertEquals(numSegments, cached.length);
-    List<LogSegment> segments = new ArrayList<>(numSegments);
-    for (int i = 0; i < numSegments; i++) {
-      LogSegment s = LogSegment.newCloseSegment(null, start, start + size - 1);
-      if (cached[i]) {
-        s = Mockito.spy(s);
-        Mockito.when(s.hasCache()).thenReturn(true);
-      }
-      segments.add(s);
-      start += size;
-    }
-    return segments;
-  }
-
-  @Test
-  public void testBasicEviction() throws Exception {
-    final int maxCached = 5;
-    List<LogSegment> segments = prepareSegments(5,
-        new boolean[]{true, true, true, true, true}, 0, 10);
-
-    // case 1, make sure we do not evict cache for segments behind local flushed index
-    List<LogSegment> evicted = policy.evict(null, 5, 15, segments, maxCached);
-    Assert.assertEquals(0, evicted.size());
-
-    // case 2, suppose the local flushed index is in the 3rd segment, then we
-    // can evict the first two segment
-    evicted = policy.evict(null, 25, 30, segments, maxCached);
-    Assert.assertEquals(2, evicted.size());
-    Assert.assertSame(evicted.get(0), segments.get(0));
-    Assert.assertSame(evicted.get(1), segments.get(1));
-
-    // case 3, similar with case 2, but the local applied index is less than
-    // the local flushed index.
-    evicted = policy.evict(null, 25, 15, segments, maxCached);
-    Assert.assertEquals(1, evicted.size());
-    Assert.assertSame(evicted.get(0), segments.get(0));
-
-    // case 4, the local applied index is very small, then evict cache behind it
-    // first and let the state machine load the segments later
-    evicted = policy.evict(null, 35, 5, segments, maxCached);
-    Assert.assertEquals(1, evicted.size());
-    Assert.assertSame(evicted.get(0), segments.get(2));
-
-    Mockito.when(segments.get(2).hasCache()).thenReturn(false);
-    evicted = policy.evict(null, 35, 5, segments, maxCached);
-    Assert.assertEquals(1, evicted.size());
-    Assert.assertSame(evicted.get(0), segments.get(1));
-
-    Mockito.when(segments.get(1).hasCache()).thenReturn(false);
-    evicted = policy.evict(null, 35, 5, segments, maxCached);
-    Assert.assertEquals(0, evicted.size());
-  }
-
-  @Test
-  public void testEvictionWithFollowerIndices() throws Exception {
-    final int maxCached = 6;
-    List<LogSegment> segments = prepareSegments(6,
-        new boolean[]{true, true, true, true, true, true}, 0, 10);
-
-    // case 1, no matter where the followers are, we do not evict segments behind local
-    // flushed index
-    List<LogSegment> evicted = policy.evict(new long[]{20, 40, 40}, 5, 15, segments,
-        maxCached);
-    Assert.assertEquals(0, evicted.size());
-
-    // case 2, the follower indices are behind the local flushed index
-    evicted = policy.evict(new long[]{30, 40, 45}, 25, 30, segments, maxCached);
-    Assert.assertEquals(2, evicted.size());
-    Assert.assertSame(evicted.get(0), segments.get(0));
-    Assert.assertSame(evicted.get(1), segments.get(1));
-
-    // case 3, similar with case 3 in basic eviction test
-    evicted = policy.evict(new long[]{30, 40, 45}, 25, 15, segments, maxCached);
-    Assert.assertEquals(1, evicted.size());
-    Assert.assertSame(evicted.get(0), segments.get(0));
-
-    // case 4, the followers are slower than local flush
-    evicted = policy.evict(new long[]{15, 45, 45}, 55, 50, segments, maxCached);
-    Assert.assertEquals(1, evicted.size());
-    Assert.assertSame(evicted.get(0), segments.get(0));
-
-    Mockito.when(segments.get(0).hasCache()).thenReturn(false);
-    evicted = policy.evict(new long[]{15, 45, 45}, 55, 50, segments, maxCached);
-    Assert.assertEquals(1, evicted.size());
-    Assert.assertSame(evicted.get(0), segments.get(2));
-
-    Mockito.when(segments.get(2).hasCache()).thenReturn(false);
-    evicted = policy.evict(new long[]{15, 45, 45}, 55, 50, segments, maxCached);
-    Assert.assertEquals(1, evicted.size());
-    Assert.assertSame(evicted.get(0), segments.get(3));
-
-    Mockito.when(segments.get(3).hasCache()).thenReturn(false);
-    evicted = policy.evict(new long[]{15, 45, 45}, 55, 50, segments, maxCached);
-    Assert.assertEquals(0, evicted.size());
-  }
-
-  @Test
-  public void testEvictionInSegmentedLog() throws Exception {
-    final RaftProperties prop = new RaftProperties();
-    prop.setClass(MiniRaftCluster.STATEMACHINE_CLASS_KEY,
-        SimpleStateMachine4Testing.class, StateMachine.class);
-    RaftServerConfigKeys.Log.setSegmentSizeMax(prop, SizeInBytes.valueOf("8KB"));
-    RaftServerConfigKeys.Log.setPreallocatedSize(prop, SizeInBytes.valueOf("8KB"));
-    final RaftPeerId peerId = RaftPeerId.valueOf("s0");
-    final int maxCachedNum = RaftServerConfigKeys.Log.maxCachedSegmentNum(prop);
-
-    File storageDir = getTestDir();
-    RaftServerConfigKeys.setStorageDirs(prop,  Collections.singletonList(storageDir));
-    RaftStorage storage = new RaftStorage(storageDir, RaftServerConstants.StartupOption.REGULAR);
-
-    RaftServerImpl server = Mockito.mock(RaftServerImpl.class);
-    ServerState state = Mockito.mock(ServerState.class);
-    Mockito.when(server.getState()).thenReturn(state);
-    Mockito.when(server.getFollowerNextIndices()).thenReturn(new long[]{});
-    Mockito.when(state.getLastAppliedIndex()).thenReturn(0L);
-
-    SegmentedRaftLog raftLog = new SegmentedRaftLog(peerId, server, storage, -1, prop);
-    raftLog.open(RaftServerConstants.INVALID_LOG_INDEX, null);
-    List<SegmentRange> slist = TestSegmentedRaftLog.prepareRanges(0, maxCachedNum, 7, 0);
-    LogEntryProto[] entries = generateEntries(slist);
-    raftLog.append(entries).forEach(CompletableFuture::join);
-
-    // check the current cached segment number: the last segment is still open
-    Assert.assertEquals(maxCachedNum - 1,
-        raftLog.getRaftLogCache().getCachedSegmentNum());
-
-    Mockito.when(server.getFollowerNextIndices()).thenReturn(new long[]{21, 40, 40});
-    Mockito.when(state.getLastAppliedIndex()).thenReturn(35L);
-    slist = TestSegmentedRaftLog.prepareRanges(maxCachedNum, maxCachedNum + 2, 7, 7 * maxCachedNum);
-    entries = generateEntries(slist);
-    raftLog.append(entries).forEach(CompletableFuture::join);
-
-    // check the cached segment number again. since the slowest follower is on
-    // index 21, the eviction should happen and evict 3 segments
-    Assert.assertEquals(maxCachedNum + 1 - 3,
-        raftLog.getRaftLogCache().getCachedSegmentNum());
-  }
-
-  private LogEntryProto[] generateEntries(List<SegmentRange> slist) {
-    List<LogEntryProto> eList = new ArrayList<>();
-    for (SegmentRange range : slist) {
-      for (long index = range.start; index <= range.end; index++) {
-        SimpleOperation m = new SimpleOperation(new String(new byte[1024]));
-        eList.add(ServerProtoUtils.toLogEntryProto(m.getLogEntryContent(), range.term, index));
-      }
-    }
-    return eList.toArray(new LogEntryProto[eList.size()]);
-  }
-}

http://git-wip-us.apache.org/repos/asf/incubator-ratis/blob/5c37675f/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
deleted file mode 100644
index d3c216d..0000000
--- a/ratis-server/src/test/java/org/apache/ratis/server/storage/TestRaftLogCache.java
+++ /dev/null
@@ -1,257 +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.storage;
-
-import java.io.IOException;
-import java.util.Iterator;
-
-import org.apache.ratis.RaftTestUtil.SimpleOperation;
-import org.apache.ratis.conf.RaftProperties;
-import org.apache.ratis.server.impl.ServerProtoUtils;
-import org.apache.ratis.server.protocol.TermIndex;
-import org.apache.ratis.server.storage.RaftLogCache.TruncationSegments;
-import org.apache.ratis.proto.RaftProtos.LogEntryProto;
-import org.junit.Assert;
-import org.junit.Before;
-import org.junit.Test;
-
-public class TestRaftLogCache {
-  private static final RaftProperties prop = new RaftProperties();
-
-  private RaftLogCache cache;
-
-  @Before
-  public void setup() {
-    cache = new RaftLogCache(null, null, prop);
-  }
-
-  private LogSegment prepareLogSegment(long start, long end, boolean isOpen) {
-    LogSegment s = LogSegment.newOpenSegment(null, start);
-    for (long i = start; i <= end; i++) {
-      SimpleOperation m = new SimpleOperation("m" + i);
-      LogEntryProto entry = ServerProtoUtils.toLogEntryProto(m.getLogEntryContent(), 0, i);
-      s.appendToOpenSegment(entry);
-    }
-    if (!isOpen) {
-      s.close();
-    }
-    return s;
-  }
-
-  private void checkCache(long start, long end, int segmentSize) throws IOException {
-    Assert.assertEquals(start, cache.getStartIndex());
-    Assert.assertEquals(end, cache.getEndIndex());
-
-    for (long index = start; index <= end; index++) {
-      LogEntryProto entry = cache.getSegment(index).getEntryWithoutLoading(index).getEntry();
-      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) {
-    TermIndex[] entries = cache.getTermIndices(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 = ServerProtoUtils.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 = ServerProtoUtils.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) throws IOException {
-    Iterator<TermIndex> iterator = cache.iterator(startIndex);
-    TermIndex prev = null;
-    while (iterator.hasNext()) {
-      TermIndex termIndex = iterator.next();
-      Assert.assertEquals(cache.getLogRecord(termIndex.getIndex()).getTermIndex(), termIndex);
-      if (prev != null) {
-        Assert.assertEquals(prev.getIndex() + 1, termIndex.getIndex());
-      }
-      prev = termIndex;
-    }
-    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<TermIndex> iterator = cache.iterator(300);
-    Assert.assertFalse(iterator.hasNext());
-  }
-}


[6/6] incubator-ratis git commit: RATIS-399. Move all tests to a new module.

Posted by sz...@apache.org.
RATIS-399. Move all tests to a new module.


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

Branch: refs/heads/master
Commit: 5c37675fa38f54a7171b4581ba3148652f7f55c1
Parents: d28b649
Author: Tsz Wo Nicholas Sze <sz...@apache.org>
Authored: Tue Nov 13 15:38:23 2018 -0800
Committer: Tsz Wo Nicholas Sze <sz...@apache.org>
Committed: Tue Nov 13 15:39:04 2018 -0800

----------------------------------------------------------------------
 pom.xml                                         |   2 +
 .../java/org/apache/ratis/TestRetryPolicy.java  |  42 --
 .../org/apache/ratis/util/TestLifeCycle.java    |  53 --
 .../java/org/apache/ratis/util/TestMinMax.java  |  57 --
 .../org/apache/ratis/util/TestTimeDuration.java |  84 ---
 .../apache/ratis/util/TestTimeoutScheduler.java | 210 --------
 .../ratis/util/TestTraditionalBinaryPrefix.java | 145 -----
 ratis-examples/pom.xml                          |  15 +-
 ratis-grpc/pom.xml                              |  11 -
 .../ratis/grpc/TestGroupManagementWithGrpc.java |  28 -
 .../ratis/grpc/TestLeaderElectionWithGrpc.java  |  36 --
 .../ratis/grpc/TestRaftAsyncWithGrpc.java       |  24 -
 .../ratis/grpc/TestRaftExceptionWithGrpc.java   |  25 -
 .../grpc/TestRaftReconfigurationWithGrpc.java   |  36 --
 .../ratis/grpc/TestRaftServerWithGrpc.java      |  59 ---
 .../ratis/grpc/TestRaftSnapshotWithGrpc.java    |  28 -
 .../TestRaftStateMachineExceptionWithGrpc.java  |  26 -
 .../org/apache/ratis/grpc/TestRaftStream.java   | 311 -----------
 .../org/apache/ratis/grpc/TestRaftWithGrpc.java | 110 ----
 .../ratis/grpc/TestRetryCacheWithGrpc.java      | 115 ----
 .../grpc/TestServerInformationWithGrpc.java     |  25 -
 .../ratis/grpc/TestServerRestartWithGrpc.java   |  25 -
 .../grpc/TestStateMachineShutdownWithGrpc.java  |  26 -
 .../ratis/grpc/TestWatchRequestWithGrpc.java    |  25 -
 ratis-grpc/src/test/resources/log4j.properties  |  18 -
 ratis-netty/pom.xml                             |  11 -
 .../netty/TestGroupManagementWithNetty.java     |  28 -
 .../netty/TestLeaderElectionWithNetty.java      |  35 --
 .../ratis/netty/TestRaftExceptionWithNetty.java |  25 -
 .../netty/TestRaftReconfigurationWithNetty.java |  31 --
 .../ratis/netty/TestRaftSnapshotWithNetty.java  |  31 --
 .../TestRaftStateMachineExceptionWithNetty.java |  25 -
 .../apache/ratis/netty/TestRaftWithNetty.java   |  34 --
 .../ratis/netty/TestRetryCacheWithNettyRpc.java |  45 --
 .../netty/TestServerInformationWithNetty.java   |  25 -
 .../ratis/netty/TestServerRestartWithNetty.java |  25 -
 ratis-netty/src/test/resources/log4j.properties |  18 -
 .../TestRaftServerLeaderElectionTimeout.java    |  99 ----
 .../ratis/TestRaftServerSlownessDetection.java  | 109 ----
 .../org/apache/ratis/protocol/TestRaftId.java   |  54 --
 .../apache/ratis/server/TestRaftLogMetrics.java | 119 -----
 .../ratis/server/TestRaftServerConfigKeys.java  |  98 ----
 .../ratis/server/impl/TestRaftServerJmx.java    | 109 ----
 .../ratis/server/impl/TestServerState.java      | 127 -----
 .../TestGroupManagementWithSimulatedRpc.java    |  28 -
 .../TestLeaderElectionWithSimulatedRpc.java     |  25 -
 .../TestRaftExceptionWithSimulation.java        |  25 -
 ...TestRaftReconfigurationWithSimulatedRpc.java |  31 --
 .../TestRaftSnapshotWithSimulatedRpc.java       |  31 --
 ...ftStateMachineExceptionWithSimulatedRpc.java |  25 -
 .../simulation/TestRaftWithSimulatedRpc.java    |  25 -
 .../TestRetryCacheWithSimulatedRpc.java         |  45 --
 .../TestServerInformationWithSimulatedRpc.java  |  25 -
 .../TestServerRestartWithSimulatedRpc.java      |  25 -
 .../ratis/server/storage/TestCacheEviction.java | 198 -------
 .../ratis/server/storage/TestRaftLogCache.java  | 257 ---------
 .../server/storage/TestRaftLogReadWrite.java    | 272 ----------
 .../server/storage/TestRaftLogSegment.java      | 372 -------------
 .../ratis/server/storage/TestRaftStorage.java   | 206 --------
 .../server/storage/TestSegmentedRaftLog.java    | 524 -------------------
 .../ratis/statemachine/TestStateMachine.java    | 201 -------
 .../src/test/resources/log4j.properties         |  18 -
 ratis-test/pom.xml                              |  92 ++++
 .../TestRaftServerLeaderElectionTimeout.java    |  99 ++++
 .../ratis/TestRaftServerSlownessDetection.java  | 109 ++++
 .../java/org/apache/ratis/TestRetryPolicy.java  |  42 ++
 .../ratis/grpc/TestGroupManagementWithGrpc.java |  28 +
 .../ratis/grpc/TestLeaderElectionWithGrpc.java  |  36 ++
 .../ratis/grpc/TestRaftAsyncWithGrpc.java       |  24 +
 .../ratis/grpc/TestRaftExceptionWithGrpc.java   |  25 +
 .../grpc/TestRaftReconfigurationWithGrpc.java   |  36 ++
 .../ratis/grpc/TestRaftServerWithGrpc.java      |  59 +++
 .../ratis/grpc/TestRaftSnapshotWithGrpc.java    |  28 +
 .../TestRaftStateMachineExceptionWithGrpc.java  |  26 +
 .../org/apache/ratis/grpc/TestRaftStream.java   | 311 +++++++++++
 .../org/apache/ratis/grpc/TestRaftWithGrpc.java | 110 ++++
 .../ratis/grpc/TestRetryCacheWithGrpc.java      | 115 ++++
 .../grpc/TestServerInformationWithGrpc.java     |  25 +
 .../ratis/grpc/TestServerRestartWithGrpc.java   |  25 +
 .../grpc/TestStateMachineShutdownWithGrpc.java  |  26 +
 .../ratis/grpc/TestWatchRequestWithGrpc.java    |  25 +
 .../netty/TestGroupManagementWithNetty.java     |  28 +
 .../netty/TestLeaderElectionWithNetty.java      |  35 ++
 .../ratis/netty/TestRaftExceptionWithNetty.java |  25 +
 .../netty/TestRaftReconfigurationWithNetty.java |  31 ++
 .../ratis/netty/TestRaftSnapshotWithNetty.java  |  31 ++
 .../TestRaftStateMachineExceptionWithNetty.java |  25 +
 .../apache/ratis/netty/TestRaftWithNetty.java   |  34 ++
 .../ratis/netty/TestRetryCacheWithNettyRpc.java |  45 ++
 .../netty/TestServerInformationWithNetty.java   |  25 +
 .../ratis/netty/TestServerRestartWithNetty.java |  25 +
 .../org/apache/ratis/protocol/TestRaftId.java   |  54 ++
 .../apache/ratis/server/TestRaftLogMetrics.java | 119 +++++
 .../ratis/server/TestRaftServerConfigKeys.java  |  98 ++++
 .../ratis/server/impl/TestRaftServerJmx.java    | 109 ++++
 .../ratis/server/impl/TestServerState.java      | 127 +++++
 .../TestGroupManagementWithSimulatedRpc.java    |  28 +
 .../TestLeaderElectionWithSimulatedRpc.java     |  25 +
 .../TestRaftExceptionWithSimulation.java        |  25 +
 ...TestRaftReconfigurationWithSimulatedRpc.java |  31 ++
 .../TestRaftSnapshotWithSimulatedRpc.java       |  31 ++
 ...ftStateMachineExceptionWithSimulatedRpc.java |  25 +
 .../simulation/TestRaftWithSimulatedRpc.java    |  25 +
 .../TestRetryCacheWithSimulatedRpc.java         |  45 ++
 .../TestServerInformationWithSimulatedRpc.java  |  25 +
 .../TestServerRestartWithSimulatedRpc.java      |  25 +
 .../ratis/server/storage/TestCacheEviction.java | 198 +++++++
 .../ratis/server/storage/TestRaftLogCache.java  | 257 +++++++++
 .../server/storage/TestRaftLogReadWrite.java    | 272 ++++++++++
 .../server/storage/TestRaftLogSegment.java      | 372 +++++++++++++
 .../ratis/server/storage/TestRaftStorage.java   | 206 ++++++++
 .../server/storage/TestSegmentedRaftLog.java    | 524 +++++++++++++++++++
 .../ratis/statemachine/TestStateMachine.java    | 201 +++++++
 .../org/apache/ratis/util/TestLifeCycle.java    |  53 ++
 .../java/org/apache/ratis/util/TestMinMax.java  |  57 ++
 .../org/apache/ratis/util/TestTimeDuration.java |  84 +++
 .../apache/ratis/util/TestTimeoutScheduler.java | 210 ++++++++
 .../ratis/util/TestTraditionalBinaryPrefix.java | 145 +++++
 ratis-test/src/test/resources/log4j.properties  |  18 +
 119 files changed, 4941 insertions(+), 4910 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-ratis/blob/5c37675f/pom.xml
----------------------------------------------------------------------
diff --git a/pom.xml b/pom.xml
index 4558609..811ea18 100644
--- a/pom.xml
+++ b/pom.xml
@@ -50,6 +50,8 @@
     <module>ratis-netty</module>
     <module>ratis-hadoop</module>
 
+    <module>ratis-test</module>
+
     <module>ratis-assembly</module>
     <module>ratis-examples</module>
     <module>ratis-replicated-map</module>

http://git-wip-us.apache.org/repos/asf/incubator-ratis/blob/5c37675f/ratis-common/src/test/java/org/apache/ratis/TestRetryPolicy.java
----------------------------------------------------------------------
diff --git a/ratis-common/src/test/java/org/apache/ratis/TestRetryPolicy.java b/ratis-common/src/test/java/org/apache/ratis/TestRetryPolicy.java
deleted file mode 100644
index d481003..0000000
--- a/ratis-common/src/test/java/org/apache/ratis/TestRetryPolicy.java
+++ /dev/null
@@ -1,42 +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;
-
-import org.apache.ratis.retry.RetryPolicies;
-import org.apache.ratis.retry.RetryPolicy;
-import org.apache.ratis.util.TimeDuration;
-import org.junit.Assert;
-import org.junit.Test;
-
-import java.util.concurrent.TimeUnit;
-
-
-
-public class TestRetryPolicy {
-
-  @Test
-  public void testRetryMultipleTimesWithFixedSleep() {
-    RetryPolicy retryPolicy = RetryPolicies
-        .retryUpToMaximumCountWithFixedSleep(2,
-            TimeDuration.valueOf(1000L, TimeUnit.MILLISECONDS));
-     boolean shouldRetry = retryPolicy.shouldRetry(1);
-    Assert.assertTrue(shouldRetry);
-    Assert.assertTrue(1000 == retryPolicy.getSleepTime().getDuration());
-    Assert.assertFalse(retryPolicy.shouldRetry(2));
-  }
-}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/incubator-ratis/blob/5c37675f/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
deleted file mode 100644
index 9782792..0000000
--- a/ratis-common/src/test/java/org/apache/ratis/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.ratis.util;
-
-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(timeout = 1000)
-  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/5c37675f/ratis-common/src/test/java/org/apache/ratis/util/TestMinMax.java
----------------------------------------------------------------------
diff --git a/ratis-common/src/test/java/org/apache/ratis/util/TestMinMax.java b/ratis-common/src/test/java/org/apache/ratis/util/TestMinMax.java
deleted file mode 100644
index 8d315b7..0000000
--- a/ratis-common/src/test/java/org/apache/ratis/util/TestMinMax.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.ratis.util;
-
-import org.junit.Assert;
-import org.junit.Test;
-
-import java.util.ArrayList;
-import java.util.List;
-import java.util.OptionalLong;
-import java.util.concurrent.ThreadLocalRandom;
-import java.util.stream.LongStream;
-
-public class TestMinMax {
-  @Test(timeout = 1000)
-  public void testMinMax() {
-    runTestMinMax(LongStream.empty());
-    runTestMinMax(LongStream.iterate(0, n -> n).limit(10));
-    for(int count = 1; count < 10; count++) {
-      runTestMinMax(LongStream.iterate(1, n -> n + 1).limit(count));
-    }
-    for(int count = 1; count < 10; count++) {
-      runTestMinMax(LongStream.iterate(0, _dummy -> ThreadLocalRandom.current().nextLong()).limit(count));
-    }
-  }
-
-  static void runTestMinMax(LongStream stream) {
-    final List<Long> list = stream.collect(ArrayList::new, List::add, List::addAll);
-    final LongMinMax longMinMax = toLongStream(list).collect(LongMinMax::new, LongMinMax::accumulate, LongMinMax::combine);
-    if (longMinMax.isInitialized()) {
-      Assert.assertEquals(toLongStream(list).min().getAsLong(), longMinMax.getMin());
-      Assert.assertEquals(toLongStream(list).max().getAsLong(), longMinMax.getMax());
-    } else {
-      Assert.assertEquals(OptionalLong.empty(), toLongStream(list).min());
-      Assert.assertEquals(OptionalLong.empty(), toLongStream(list).max());
-    }
-  }
-
-  static LongStream toLongStream(List<Long> list) {
-    return list.stream().mapToLong(Long::longValue);
-  }
-}

http://git-wip-us.apache.org/repos/asf/incubator-ratis/blob/5c37675f/ratis-common/src/test/java/org/apache/ratis/util/TestTimeDuration.java
----------------------------------------------------------------------
diff --git a/ratis-common/src/test/java/org/apache/ratis/util/TestTimeDuration.java b/ratis-common/src/test/java/org/apache/ratis/util/TestTimeDuration.java
deleted file mode 100644
index 06d9301..0000000
--- a/ratis-common/src/test/java/org/apache/ratis/util/TestTimeDuration.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.ratis.util;
-
-import org.junit.Test;
-
-import java.util.Arrays;
-import java.util.List;
-import java.util.concurrent.TimeUnit;
-import java.util.stream.Collectors;
-
-import static org.apache.ratis.util.TimeDuration.Abbreviation;
-import static org.apache.ratis.util.TimeDuration.parse;
-import static org.junit.Assert.assertEquals;
-import static org.junit.Assert.assertNotNull;
-
-public class TestTimeDuration {
-  @Test(timeout = 1000)
-  public void testTimeDuration() throws Exception {
-    Arrays.asList(TimeUnit.values())
-        .forEach(a -> assertNotNull(Abbreviation.valueOf(a.name())));
-    assertEquals(TimeUnit.values().length, Abbreviation.values().length);
-
-    final List<String> allSymbols = Arrays.asList(Abbreviation.values()).stream()
-        .map(Abbreviation::getSymbols)
-        .flatMap(List::stream)
-        .collect(Collectors.toList());
-    Arrays.asList(TimeUnit.values()).forEach(unit ->
-        allSymbols.stream()
-            .map(s -> "0" + s)
-            .forEach(s -> assertEquals(s, 0L, parse(s, unit))));
-
-    assertEquals(1L, parse("1000000 ns", TimeUnit.MILLISECONDS));
-    assertEquals(10L, parse("10000000 nanos", TimeUnit.MILLISECONDS));
-    assertEquals(100L, parse("100000000 nanosecond", TimeUnit.MILLISECONDS));
-    assertEquals(1000L, parse("1000000000 nanoseconds", TimeUnit.MILLISECONDS));
-
-    assertEquals(1L, parse("1000 us", TimeUnit.MILLISECONDS));
-    assertEquals(10L, parse("10000 μs", TimeUnit.MILLISECONDS));
-    assertEquals(100L, parse("100000 micros", TimeUnit.MILLISECONDS));
-    assertEquals(1000L, parse("1000000 microsecond", TimeUnit.MILLISECONDS));
-    assertEquals(10000L, parse("10000000 microseconds", TimeUnit.MILLISECONDS));
-
-    assertEquals(1L, parse("1 ms", TimeUnit.MILLISECONDS));
-    assertEquals(10L, parse("10 msec", TimeUnit.MILLISECONDS));
-    assertEquals(100L, parse("100 millis", TimeUnit.MILLISECONDS));
-    assertEquals(1000L, parse("1000 millisecond", TimeUnit.MILLISECONDS));
-    assertEquals(10000L, parse("10000 milliseconds", TimeUnit.MILLISECONDS));
-
-    assertEquals(1000L, parse("1 s", TimeUnit.MILLISECONDS));
-    assertEquals(10000L, parse("10 sec", TimeUnit.MILLISECONDS));
-    assertEquals(100000L, parse("100 second", TimeUnit.MILLISECONDS));
-    assertEquals(1000000L, parse("1000 seconds", TimeUnit.MILLISECONDS));
-
-    assertEquals(60, parse("1 m", TimeUnit.SECONDS));
-    assertEquals(600, parse("10 min", TimeUnit.SECONDS));
-    assertEquals(6000, parse("100 minutes", TimeUnit.SECONDS));
-    assertEquals(60000, parse("1000 minutes", TimeUnit.SECONDS));
-
-    assertEquals(60, parse("1 h", TimeUnit.MINUTES));
-    assertEquals(600, parse("10 hr", TimeUnit.MINUTES));
-    assertEquals(6000, parse("100 hour", TimeUnit.MINUTES));
-    assertEquals(60000, parse("1000 hours", TimeUnit.MINUTES));
-
-    assertEquals(24, parse("1 d", TimeUnit.HOURS));
-    assertEquals(240, parse("10 day", TimeUnit.HOURS));
-    assertEquals(2400, parse("100 days", TimeUnit.HOURS));
-  }
-}

http://git-wip-us.apache.org/repos/asf/incubator-ratis/blob/5c37675f/ratis-common/src/test/java/org/apache/ratis/util/TestTimeoutScheduler.java
----------------------------------------------------------------------
diff --git a/ratis-common/src/test/java/org/apache/ratis/util/TestTimeoutScheduler.java b/ratis-common/src/test/java/org/apache/ratis/util/TestTimeoutScheduler.java
deleted file mode 100644
index 6a63569..0000000
--- a/ratis-common/src/test/java/org/apache/ratis/util/TestTimeoutScheduler.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.ratis.util;
-
-import org.apache.log4j.Level;
-import org.junit.Assert;
-import org.junit.Test;
-
-import java.util.concurrent.TimeUnit;
-import java.util.concurrent.atomic.AtomicBoolean;
-import java.util.function.Consumer;
-
-public class TestTimeoutScheduler {
-  {
-    LogUtils.setLogLevel(TimeoutScheduler.LOG, Level.ALL);
-  }
-
-  static class ErrorHandler implements Consumer<RuntimeException> {
-    private final AtomicBoolean hasError = new AtomicBoolean(false);
-
-    @Override
-    public void accept(RuntimeException e) {
-      hasError.set(true);
-      TimeoutScheduler.LOG.error("Failed", e);
-    }
-
-    void assertNoError() {
-      Assert.assertFalse(hasError.get());
-    }
-  }
-
-  @Test(timeout = 1000)
-  public void testSingleTask() throws Exception {
-    final TimeoutScheduler scheduler = TimeoutScheduler.newInstance(1);
-    final TimeDuration grace = TimeDuration.valueOf(100, TimeUnit.MILLISECONDS);
-    scheduler.setGracePeriod(grace);
-    Assert.assertFalse(scheduler.hasScheduler());
-
-    final ErrorHandler errorHandler = new ErrorHandler();
-
-    final AtomicBoolean fired = new AtomicBoolean(false);
-    scheduler.onTimeout(TimeDuration.valueOf(250, TimeUnit.MILLISECONDS), () -> {
-      Assert.assertFalse(fired.get());
-      fired.set(true);
-    }, errorHandler);
-    Assert.assertTrue(scheduler.hasScheduler());
-
-    Thread.sleep(100);
-    Assert.assertFalse(fired.get());
-    Assert.assertTrue(scheduler.hasScheduler());
-
-    Thread.sleep(100);
-    Assert.assertFalse(fired.get());
-    Assert.assertTrue(scheduler.hasScheduler());
-
-    Thread.sleep(100);
-    Assert.assertTrue(fired.get());
-    Assert.assertTrue(scheduler.hasScheduler());
-
-    Thread.sleep(100);
-    Assert.assertTrue(fired.get());
-    Assert.assertFalse(scheduler.hasScheduler());
-
-    errorHandler.assertNoError();
-  }
-
-  @Test(timeout = 1000)
-  public void testMultipleTasks() throws Exception {
-    final TimeoutScheduler scheduler = TimeoutScheduler.newInstance(1);
-    final TimeDuration grace = TimeDuration.valueOf(100, TimeUnit.MILLISECONDS);
-    scheduler.setGracePeriod(grace);
-    Assert.assertFalse(scheduler.hasScheduler());
-
-    final ErrorHandler errorHandler = new ErrorHandler();
-
-    final AtomicBoolean[] fired = new AtomicBoolean[3];
-    for(int i = 0; i < fired.length; i++) {
-      final AtomicBoolean f = fired[i] = new AtomicBoolean(false);
-      scheduler.onTimeout(TimeDuration.valueOf(100*i + 50, TimeUnit.MILLISECONDS), () -> {
-        Assert.assertFalse(f.get());
-        f.set(true);
-      }, errorHandler);
-      Assert.assertTrue(scheduler.hasScheduler());
-    }
-
-    Thread.sleep(100);
-    Assert.assertTrue(fired[0].get());
-    Assert.assertFalse(fired[1].get());
-    Assert.assertFalse(fired[2].get());
-    Assert.assertTrue(scheduler.hasScheduler());
-
-    Thread.sleep(100);
-    Assert.assertTrue(fired[0].get());
-    Assert.assertTrue(fired[1].get());
-    Assert.assertFalse(fired[2].get());
-    Assert.assertTrue(scheduler.hasScheduler());
-
-    Thread.sleep(100);
-    Assert.assertTrue(fired[0].get());
-    Assert.assertTrue(fired[1].get());
-    Assert.assertTrue(fired[2].get());
-    Assert.assertTrue(scheduler.hasScheduler());
-
-    Thread.sleep(100);
-    Assert.assertTrue(fired[0].get());
-    Assert.assertTrue(fired[1].get());
-    Assert.assertTrue(fired[2].get());
-    Assert.assertFalse(scheduler.hasScheduler());
-
-    errorHandler.assertNoError();
-  }
-
-  @Test(timeout = 1000)
-  public void testExtendingGracePeriod() throws Exception {
-    final TimeoutScheduler scheduler = TimeoutScheduler.newInstance(1);
-    final TimeDuration grace = TimeDuration.valueOf(100, TimeUnit.MILLISECONDS);
-    scheduler.setGracePeriod(grace);
-    Assert.assertFalse(scheduler.hasScheduler());
-
-    final ErrorHandler errorHandler = new ErrorHandler();
-
-    {
-      final AtomicBoolean fired = new AtomicBoolean(false);
-      scheduler.onTimeout(TimeDuration.valueOf(150, TimeUnit.MILLISECONDS), () -> {
-        Assert.assertFalse(fired.get());
-        fired.set(true);
-      }, errorHandler);
-      Assert.assertTrue(scheduler.hasScheduler());
-
-      Thread.sleep(100);
-      Assert.assertFalse(fired.get());
-      Assert.assertTrue(scheduler.hasScheduler());
-
-      Thread.sleep(100);
-      Assert.assertTrue(fired.get());
-      Assert.assertTrue(scheduler.hasScheduler());
-    }
-
-    {
-      // submit another task during grace period
-      final AtomicBoolean fired2 = new AtomicBoolean(false);
-      scheduler.onTimeout(TimeDuration.valueOf(150, TimeUnit.MILLISECONDS), () -> {
-        Assert.assertFalse(fired2.get());
-        fired2.set(true);
-      }, errorHandler);
-
-      Thread.sleep(100);
-      Assert.assertFalse(fired2.get());
-      Assert.assertTrue(scheduler.hasScheduler());
-
-      Thread.sleep(100);
-      Assert.assertTrue(fired2.get());
-      Assert.assertTrue(scheduler.hasScheduler());
-
-      Thread.sleep(100);
-      Assert.assertTrue(fired2.get());
-      Assert.assertFalse(scheduler.hasScheduler());
-    }
-
-    errorHandler.assertNoError();
-  }
-
-  @Test(timeout = 1000)
-  public void testRestartingScheduler() throws Exception {
-    final TimeoutScheduler scheduler = TimeoutScheduler.newInstance(1);
-    final TimeDuration grace = TimeDuration.valueOf(100, TimeUnit.MILLISECONDS);
-    scheduler.setGracePeriod(grace);
-    Assert.assertFalse(scheduler.hasScheduler());
-
-    final ErrorHandler errorHandler = new ErrorHandler();
-
-    for(int i = 0; i < 2; i++) {
-      final AtomicBoolean fired = new AtomicBoolean(false);
-      scheduler.onTimeout(TimeDuration.valueOf(150, TimeUnit.MILLISECONDS), () -> {
-        Assert.assertFalse(fired.get());
-        fired.set(true);
-      }, errorHandler);
-      Assert.assertTrue(scheduler.hasScheduler());
-
-      Thread.sleep(100);
-      Assert.assertFalse(fired.get());
-      Assert.assertTrue(scheduler.hasScheduler());
-
-      Thread.sleep(100);
-      Assert.assertTrue(fired.get());
-      Assert.assertTrue(scheduler.hasScheduler());
-
-      Thread.sleep(100);
-      Assert.assertTrue(fired.get());
-      Assert.assertFalse(scheduler.hasScheduler());
-    }
-
-    errorHandler.assertNoError();
-  }
-}

http://git-wip-us.apache.org/repos/asf/incubator-ratis/blob/5c37675f/ratis-common/src/test/java/org/apache/ratis/util/TestTraditionalBinaryPrefix.java
----------------------------------------------------------------------
diff --git a/ratis-common/src/test/java/org/apache/ratis/util/TestTraditionalBinaryPrefix.java b/ratis-common/src/test/java/org/apache/ratis/util/TestTraditionalBinaryPrefix.java
deleted file mode 100644
index 26a62da..0000000
--- a/ratis-common/src/test/java/org/apache/ratis/util/TestTraditionalBinaryPrefix.java
+++ /dev/null
@@ -1,145 +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.util;
-
-import org.junit.Test;
-
-import static org.apache.ratis.util.TraditionalBinaryPrefix.long2String;
-import static org.apache.ratis.util.TraditionalBinaryPrefix.string2long;
-import static org.junit.Assert.assertEquals;
-import static org.junit.Assert.fail;
-
-public class TestTraditionalBinaryPrefix {
-  @Test(timeout = 1000)
-  public void testTraditionalBinaryPrefix() throws Exception {
-    //test string2long(..)
-    String[] symbol = {"k", "m", "g", "t", "p", "e"};
-    long m = 1024;
-    for(String s : symbol) {
-      assertEquals(0, string2long(0 + s));
-      assertEquals(m, string2long(1 + s));
-      m *= 1024;
-    }
-
-    assertEquals(0L, string2long("0"));
-    assertEquals(1024L, string2long("1k"));
-    assertEquals(-1024L, string2long("-1k"));
-    assertEquals(1259520L, string2long("1230K"));
-    assertEquals(-1259520L, string2long("-1230K"));
-    assertEquals(104857600L, string2long("100m"));
-    assertEquals(-104857600L, string2long("-100M"));
-    assertEquals(956703965184L, string2long("891g"));
-    assertEquals(-956703965184L, string2long("-891G"));
-    assertEquals(501377302265856L, string2long("456t"));
-    assertEquals(-501377302265856L, string2long("-456T"));
-    assertEquals(11258999068426240L, string2long("10p"));
-    assertEquals(-11258999068426240L, string2long("-10P"));
-    assertEquals(1152921504606846976L, string2long("1e"));
-    assertEquals(-1152921504606846976L, string2long("-1E"));
-
-    String tooLargeNumStr = "10e";
-    try {
-      string2long(tooLargeNumStr);
-      fail("Test passed for a number " + tooLargeNumStr + " too large");
-    } catch (IllegalArgumentException e) {
-      assertEquals(tooLargeNumStr + " does not fit in a Long", e.getMessage());
-    }
-
-    String tooSmallNumStr = "-10e";
-    try {
-      string2long(tooSmallNumStr);
-      fail("Test passed for a number " + tooSmallNumStr + " too small");
-    } catch (IllegalArgumentException e) {
-      assertEquals(tooSmallNumStr + " does not fit in a Long", e.getMessage());
-    }
-
-    String invalidFormatNumStr = "10kb";
-    char invalidPrefix = 'b';
-    try {
-      string2long(invalidFormatNumStr);
-      fail("Test passed for a number " + invalidFormatNumStr
-          + " has invalid format");
-    } catch (IllegalArgumentException e) {
-      assertEquals("Invalid size prefix '" + invalidPrefix + "' in '"
-              + invalidFormatNumStr
-              + "'. Allowed prefixes are k, m, g, t, p, e (case insensitive)",
-          e.getMessage());
-    }
-
-    //test long2string(..)
-    assertEquals("0", long2String(0, null, 2));
-    for(int decimalPlace = 0; decimalPlace < 2; decimalPlace++) {
-      for(int n = 1; n < TraditionalBinaryPrefix.KILO.getValue(); n++) {
-        assertEquals(n + "", long2String(n, null, decimalPlace));
-        assertEquals(-n + "", long2String(-n, null, decimalPlace));
-      }
-      assertEquals("1 K", long2String(1L << 10, null, decimalPlace));
-      assertEquals("-1 K", long2String(-1L << 10, null, decimalPlace));
-    }
-
-    assertEquals("8.00 E", long2String(Long.MAX_VALUE, null, 2));
-    assertEquals("8.00 E", long2String(Long.MAX_VALUE - 1, null, 2));
-    assertEquals("-8 E", long2String(Long.MIN_VALUE, null, 2));
-    assertEquals("-8.00 E", long2String(Long.MIN_VALUE + 1, null, 2));
-
-    final String[] zeros = {" ", ".0 ", ".00 "};
-    for(int decimalPlace = 0; decimalPlace < zeros.length; decimalPlace++) {
-      final String trailingZeros = zeros[decimalPlace];
-
-      for(int e = 11; e < Long.SIZE - 1; e++) {
-        final TraditionalBinaryPrefix p
-            = TraditionalBinaryPrefix.values()[e/10 - 1];
-
-        { // n = 2^e
-          final long n = 1L << e;
-          final String expected = (n/p.getValue()) + " " + p.getSymbol();
-          assertEquals("n=" + n, expected, long2String(n, null, 2));
-        }
-
-        { // n = 2^e + 1
-          final long n = (1L << e) + 1;
-          final String expected = (n/p.getValue()) + trailingZeros + p.getSymbol();
-          assertEquals("n=" + n, expected, long2String(n, null, decimalPlace));
-        }
-
-        { // n = 2^e - 1
-          final long n = (1L << e) - 1;
-          final String expected = ((n+1)/p.getValue()) + trailingZeros + p.getSymbol();
-          assertEquals("n=" + n, expected, long2String(n, null, decimalPlace));
-        }
-      }
-    }
-
-    assertEquals("1.50 K", long2String(3L << 9, null, 2));
-    assertEquals("1.5 K", long2String(3L << 9, null, 1));
-    assertEquals("1.50 M", long2String(3L << 19, null, 2));
-    assertEquals("2 M", long2String(3L << 19, null, 0));
-    assertEquals("3 G", long2String(3L << 30, null, 2));
-
-    assertEquals("0 B", byteDescription(0));
-    assertEquals("-100 B", byteDescription(-100));
-    assertEquals("1 KB", byteDescription(1024));
-    assertEquals("1.50 KB", byteDescription(3L << 9));
-    assertEquals("1.50 MB", byteDescription(3L << 19));
-    assertEquals("3 GB", byteDescription(3L << 30));
-  }
-
-  private static String byteDescription(long len) {
-    return long2String(len, "B", 2);
-  }
-}

http://git-wip-us.apache.org/repos/asf/incubator-ratis/blob/5c37675f/ratis-examples/pom.xml
----------------------------------------------------------------------
diff --git a/ratis-examples/pom.xml b/ratis-examples/pom.xml
index e56a2dd..cf37b6b 100644
--- a/ratis-examples/pom.xml
+++ b/ratis-examples/pom.xml
@@ -48,22 +48,12 @@
       <artifactId>ratis-client</artifactId>
       <groupId>org.apache.ratis</groupId>
     </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>
     </dependency>
     <dependency>
-      <groupId>com.beust</groupId>
-      <artifactId>jcommander</artifactId>
-    </dependency>
-    <dependency>
       <artifactId>ratis-server</artifactId>
       <groupId>org.apache.ratis</groupId>
       <scope>test</scope>
@@ -110,6 +100,11 @@
     </dependency>
 
     <dependency>
+      <groupId>com.beust</groupId>
+      <artifactId>jcommander</artifactId>
+    </dependency>
+
+    <dependency>
       <groupId>org.slf4j</groupId>
       <artifactId>slf4j-api</artifactId>
     </dependency>

http://git-wip-us.apache.org/repos/asf/incubator-ratis/blob/5c37675f/ratis-grpc/pom.xml
----------------------------------------------------------------------
diff --git a/ratis-grpc/pom.xml b/ratis-grpc/pom.xml
index 4a3ab81..bd46c69 100644
--- a/ratis-grpc/pom.xml
+++ b/ratis-grpc/pom.xml
@@ -71,16 +71,5 @@
       <groupId>org.slf4j</groupId>
       <artifactId>slf4j-api</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/5c37675f/ratis-grpc/src/test/java/org/apache/ratis/grpc/TestGroupManagementWithGrpc.java
----------------------------------------------------------------------
diff --git a/ratis-grpc/src/test/java/org/apache/ratis/grpc/TestGroupManagementWithGrpc.java b/ratis-grpc/src/test/java/org/apache/ratis/grpc/TestGroupManagementWithGrpc.java
deleted file mode 100644
index 657bfd1..0000000
--- a/ratis-grpc/src/test/java/org/apache/ratis/grpc/TestGroupManagementWithGrpc.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.ratis.grpc;
-
-import org.apache.ratis.MiniRaftCluster;
-import org.apache.ratis.server.impl.GroupManagementBaseTest;
-
-public class TestGroupManagementWithGrpc extends GroupManagementBaseTest {
-  @Override
-  public MiniRaftCluster.Factory<? extends MiniRaftCluster> getClusterFactory() {
-    return MiniRaftClusterWithGrpc.FACTORY;
-  }
-}

http://git-wip-us.apache.org/repos/asf/incubator-ratis/blob/5c37675f/ratis-grpc/src/test/java/org/apache/ratis/grpc/TestLeaderElectionWithGrpc.java
----------------------------------------------------------------------
diff --git a/ratis-grpc/src/test/java/org/apache/ratis/grpc/TestLeaderElectionWithGrpc.java b/ratis-grpc/src/test/java/org/apache/ratis/grpc/TestLeaderElectionWithGrpc.java
deleted file mode 100644
index eb08336..0000000
--- a/ratis-grpc/src/test/java/org/apache/ratis/grpc/TestLeaderElectionWithGrpc.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.ratis.grpc;
-
-import org.apache.ratis.server.impl.BlockRequestHandlingInjection;
-import org.apache.ratis.server.impl.LeaderElectionTests;
-import org.junit.Test;
-
-public class TestLeaderElectionWithGrpc
-    extends LeaderElectionTests<MiniRaftClusterWithGrpc>
-    implements MiniRaftClusterWithGrpc.FactoryGet {
-
-  @Override
-  @Test
-  public void testEnforceLeader() throws Exception {
-    super.testEnforceLeader();
-
-    MiniRaftClusterWithGrpc.sendServerRequestInjection.clear();
-    BlockRequestHandlingInjection.getInstance().unblockAll();
-  }
-}

http://git-wip-us.apache.org/repos/asf/incubator-ratis/blob/5c37675f/ratis-grpc/src/test/java/org/apache/ratis/grpc/TestRaftAsyncWithGrpc.java
----------------------------------------------------------------------
diff --git a/ratis-grpc/src/test/java/org/apache/ratis/grpc/TestRaftAsyncWithGrpc.java b/ratis-grpc/src/test/java/org/apache/ratis/grpc/TestRaftAsyncWithGrpc.java
deleted file mode 100644
index 614787e..0000000
--- a/ratis-grpc/src/test/java/org/apache/ratis/grpc/TestRaftAsyncWithGrpc.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.ratis.grpc;
-
-import org.apache.ratis.RaftAsyncTests;
-
-public class TestRaftAsyncWithGrpc extends RaftAsyncTests<MiniRaftClusterWithGrpc>
-    implements MiniRaftClusterWithGrpc.FactoryGet {
-}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/incubator-ratis/blob/5c37675f/ratis-grpc/src/test/java/org/apache/ratis/grpc/TestRaftExceptionWithGrpc.java
----------------------------------------------------------------------
diff --git a/ratis-grpc/src/test/java/org/apache/ratis/grpc/TestRaftExceptionWithGrpc.java b/ratis-grpc/src/test/java/org/apache/ratis/grpc/TestRaftExceptionWithGrpc.java
deleted file mode 100644
index d2b71bc..0000000
--- a/ratis-grpc/src/test/java/org/apache/ratis/grpc/TestRaftExceptionWithGrpc.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.ratis.grpc;
-
-import org.apache.ratis.RaftExceptionBaseTest;
-
-public class TestRaftExceptionWithGrpc
-    extends RaftExceptionBaseTest<MiniRaftClusterWithGrpc>
-    implements MiniRaftClusterWithGrpc.FactoryGet {
-}

http://git-wip-us.apache.org/repos/asf/incubator-ratis/blob/5c37675f/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
deleted file mode 100644
index 29f8bea..0000000
--- a/ratis-grpc/src/test/java/org/apache/ratis/grpc/TestRaftReconfigurationWithGrpc.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.ratis.grpc;
-
-import org.apache.log4j.Level;
-import org.apache.ratis.grpc.server.GrpcServerProtocolService;
-import org.apache.ratis.server.impl.RaftReconfigurationBaseTest;
-import org.apache.ratis.util.LogUtils;
-
-import java.io.IOException;
-
-public class TestRaftReconfigurationWithGrpc extends RaftReconfigurationBaseTest {
-  static {
-    LogUtils.setLogLevel(GrpcServerProtocolService.LOG, Level.DEBUG);
-  }
-
-  @Override
-  public MiniRaftClusterWithGrpc getCluster(int peerNum) {
-    return MiniRaftClusterWithGrpc.FACTORY.newCluster(peerNum, prop);
-  }
-}

http://git-wip-us.apache.org/repos/asf/incubator-ratis/blob/5c37675f/ratis-grpc/src/test/java/org/apache/ratis/grpc/TestRaftServerWithGrpc.java
----------------------------------------------------------------------
diff --git a/ratis-grpc/src/test/java/org/apache/ratis/grpc/TestRaftServerWithGrpc.java b/ratis-grpc/src/test/java/org/apache/ratis/grpc/TestRaftServerWithGrpc.java
deleted file mode 100644
index 2ec7ae8..0000000
--- a/ratis-grpc/src/test/java/org/apache/ratis/grpc/TestRaftServerWithGrpc.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.ratis.grpc;
-
-import org.apache.ratis.BaseTest;
-import org.apache.ratis.RaftTestUtil;
-import org.apache.ratis.conf.RaftProperties;
-import org.apache.ratis.protocol.RaftPeerId;
-import org.apache.ratis.server.impl.ServerImplUtils;
-import org.apache.ratis.statemachine.StateMachine;
-import org.junit.Test;
-
-import java.io.IOException;
-import java.nio.channels.OverlappingFileLockException;
-
-public class TestRaftServerWithGrpc extends BaseTest {
-
-  @Test
-  public void testServerRestartOnException() throws Exception {
-    RaftProperties properties = new RaftProperties();
-    final MiniRaftClusterWithGrpc cluster
-        = MiniRaftClusterWithGrpc.FACTORY.newCluster(1, properties);
-    cluster.start();
-    RaftPeerId leaderId = RaftTestUtil.waitForLeader(cluster).getId();
-    GrpcConfigKeys.Server.setPort(properties, cluster.getLeader().getServerRpc().getInetSocketAddress().getPort());
-    // Create a raft server proxy with server rpc bound to a different address
-    // compared to leader. This helps in locking the raft storage directory to
-    // be used by next raft server proxy instance.
-    final StateMachine stateMachine = cluster.getLeader().getStateMachine();
-    ServerImplUtils.newRaftServer(leaderId, cluster.getGroup(), gid -> stateMachine, properties, null);
-    // Close the server rpc for leader so that new raft server can be bound to it.
-    cluster.getLeader().getServerRpc().close();
-
-    // Create a raft server proxy with server rpc bound to same address as
-    // the leader. This step would fail as the raft storage has been locked by
-    // the raft server proxy created earlier. Raft server proxy should close
-    // the rpc server on failure.
-    testFailureCase("start a new server with the same address",
-        () -> ServerImplUtils.newRaftServer(leaderId, cluster.getGroup(), gid -> stateMachine, properties, null).start(),
-        IOException.class, OverlappingFileLockException.class);
-    // Try to start a raft server rpc at the leader address.
-    cluster.getServer(leaderId).getFactory().newRaftServerRpc(cluster.getServer(leaderId));
-  }
-}

http://git-wip-us.apache.org/repos/asf/incubator-ratis/blob/5c37675f/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
deleted file mode 100644
index a960478..0000000
--- a/ratis-grpc/src/test/java/org/apache/ratis/grpc/TestRaftSnapshotWithGrpc.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.ratis.grpc;
-
-import org.apache.ratis.MiniRaftCluster;
-import org.apache.ratis.statemachine.RaftSnapshotBaseTest;
-
-public class TestRaftSnapshotWithGrpc extends RaftSnapshotBaseTest {
-  @Override
-  public MiniRaftCluster.Factory<?> getFactory() {
-    return MiniRaftClusterWithGrpc.FACTORY;
-  }
-}

http://git-wip-us.apache.org/repos/asf/incubator-ratis/blob/5c37675f/ratis-grpc/src/test/java/org/apache/ratis/grpc/TestRaftStateMachineExceptionWithGrpc.java
----------------------------------------------------------------------
diff --git a/ratis-grpc/src/test/java/org/apache/ratis/grpc/TestRaftStateMachineExceptionWithGrpc.java b/ratis-grpc/src/test/java/org/apache/ratis/grpc/TestRaftStateMachineExceptionWithGrpc.java
deleted file mode 100644
index c8789a7..0000000
--- a/ratis-grpc/src/test/java/org/apache/ratis/grpc/TestRaftStateMachineExceptionWithGrpc.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.ratis.grpc;
-
-import org.apache.ratis.server.impl.RaftStateMachineExceptionTests;
-
-public class TestRaftStateMachineExceptionWithGrpc
-    extends RaftStateMachineExceptionTests<MiniRaftClusterWithGrpc>
-    implements MiniRaftClusterWithGrpc.FactoryGet {
-
-}

http://git-wip-us.apache.org/repos/asf/incubator-ratis/blob/5c37675f/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
deleted file mode 100644
index ba31b2b..0000000
--- a/ratis-grpc/src/test/java/org/apache/ratis/grpc/TestRaftStream.java
+++ /dev/null
@@ -1,311 +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.grpc;
-
-import org.apache.log4j.Level;
-import org.apache.ratis.BaseTest;
-import org.apache.ratis.RaftTestUtil;
-import org.apache.ratis.conf.RaftProperties;
-import org.apache.ratis.grpc.client.GrpcClientStreamer;
-import org.apache.ratis.grpc.client.GrpcOutputStream;
-import org.apache.ratis.protocol.ClientId;
-import org.apache.ratis.server.impl.RaftServerImpl;
-import org.apache.ratis.server.protocol.TermIndex;
-import org.apache.ratis.server.storage.RaftLog;
-import org.apache.ratis.proto.RaftProtos;
-import org.apache.ratis.util.LogUtils;
-import org.apache.ratis.util.SizeInBytes;
-import org.apache.ratis.util.StringUtils;
-import org.junit.After;
-import org.junit.Assert;
-import org.junit.Ignore;
-import org.junit.Test;
-
-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.concurrent.atomic.AtomicReference;
-import java.util.function.Supplier;
-
-import static org.apache.ratis.RaftTestUtil.waitForLeader;
-import static org.junit.Assert.fail;
-
-@Ignore
-public class TestRaftStream extends BaseTest {
-  static {
-    LogUtils.setLogLevel(GrpcClientStreamer.LOG, Level.ALL);
-  }
-
-  private static final RaftProperties prop = new RaftProperties();
-  private static final int NUM_SERVERS = 3;
-  private static final byte[] BYTES = new byte[4];
-
-  private MiniRaftClusterWithGrpc cluster;
-
-  @After
-  public void tearDown() {
-    if (cluster != null) {
-      cluster.shutdown();
-    }
-  }
-
-  private byte[] toBytes(int i) {
-    return toBytes(i, BYTES);
-  }
-  private byte[] toBytes(int i, byte[] b) {
-    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 {
-    final int numRequests = 500;
-    LOG.info("Running testSimpleWrite, numRequests=" + numRequests);
-
-    // default 64K is too large for a test
-    GrpcConfigKeys.OutputStream.setBufferSize(prop, SizeInBytes.valueOf(4));
-    cluster = MiniRaftClusterWithGrpc.FACTORY.newCluster(NUM_SERVERS, prop);
-
-    cluster.start();
-    RaftServerImpl leader = waitForLeader(cluster);
-
-    try (GrpcOutputStream out = new GrpcOutputStream(prop, ClientId.randomId(),
-        cluster.getGroup(), leader.getId())) {
-      for (int i = 0; i < numRequests; i++) { // generate requests
-        out.write(toBytes(i));
-      }
-    }
-
-    // check the leader's raft log
-    final RaftLog raftLog = leader.getState().getLog();
-    final AtomicInteger i = new AtomicInteger();
-    checkLog(raftLog, numRequests, () -> toBytes(i.getAndIncrement()));
-  }
-
-  private void checkLog(RaftLog raftLog, long expectedCommittedIndex,
-      Supplier<byte[]> s) throws IOException {
-    long committedIndex = raftLog.getLastCommittedIndex();
-    Assert.assertEquals(expectedCommittedIndex, committedIndex);
-    // check the log content
-    TermIndex[] entries = raftLog.getEntries(1, expectedCommittedIndex + 1);
-    for (TermIndex entry : entries) {
-      RaftProtos.LogEntryProto log  = raftLog.get(entry.getIndex());
-      byte[] logData = log.getStateMachineLogEntry().getLogData().toByteArray();
-      byte[] expected = s.get();
-      LOG.info("log " + entry + " " + log.getLogEntryBodyCase() + " " + StringUtils.bytes2HexString(logData));
-      Assert.assertEquals(expected.length, logData.length);
-      Assert.assertArrayEquals(expected, logData);
-    }
-  }
-
-  @Test
-  public void testWriteAndFlush() throws Exception {
-    LOG.info("Running testWriteAndFlush");
-
-    GrpcConfigKeys.OutputStream.setBufferSize(prop, SizeInBytes.valueOf(ByteValue.BUFFERSIZE));
-    cluster = MiniRaftClusterWithGrpc.FACTORY.newCluster(NUM_SERVERS, prop);
-    cluster.start();
-
-    RaftServerImpl leader = waitForLeader(cluster);
-    GrpcOutputStream out = new GrpcOutputStream(prop, ClientId.randomId(),
-        cluster.getGroup(), 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");
-    GrpcConfigKeys.OutputStream.setBufferSize(prop, SizeInBytes.valueOf(ByteValue.BUFFERSIZE));
-
-    cluster = MiniRaftClusterWithGrpc.FACTORY.newCluster(NUM_SERVERS, prop);
-    cluster.start();
-    RaftServerImpl leader = waitForLeader(cluster);
-
-    GrpcOutputStream out = new GrpcOutputStream(prop, ClientId.randomId(),
-        cluster.getGroup(), 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());
-    TermIndex[] entries = log.getEntries(1, 9);
-    byte[] actual = new byte[ByteValue.BUFFERSIZE * 8];
-    totalSize = 0;
-    for (TermIndex e : entries) {
-      byte[] eValue = log.get(e.getIndex()).getStateMachineLogEntry().getLogData().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");
-
-    GrpcConfigKeys.OutputStream.setBufferSize(prop, SizeInBytes.valueOf(4));
-    cluster = MiniRaftClusterWithGrpc.FACTORY.newCluster(NUM_SERVERS, prop);
-    cluster.start();
-    final RaftServerImpl leader = waitForLeader(cluster);
-
-    final AtomicBoolean running  = new AtomicBoolean(true);
-    final AtomicReference<Boolean> success = new AtomicReference<>();
-    final AtomicInteger result = new AtomicInteger(0);
-    final CountDownLatch latch = new CountDownLatch(1);
-
-    new Thread(() -> {
-      LOG.info("Writer thread starts");
-      int count = 0;
-      try (GrpcOutputStream out = new GrpcOutputStream(prop, ClientId.randomId(),
-          cluster.getGroup(), 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
-    Thread.sleep(500);
-    RaftTestUtil.waitAndKillLeader(cluster);
-    final RaftServerImpl newLeader = waitForLeader(cluster);
-    Assert.assertNotEquals(leader.getId(), newLeader.getId());
-    Thread.sleep(500);
-
-    running.set(false);
-    latch.await(5, TimeUnit.SECONDS);
-    LOG.info("Writer success? " + success.get());
-    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/5c37675f/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
deleted file mode 100644
index d98be53..0000000
--- a/ratis-grpc/src/test/java/org/apache/ratis/grpc/TestRaftWithGrpc.java
+++ /dev/null
@@ -1,110 +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.grpc;
-
-import org.apache.ratis.MiniRaftCluster;
-import org.apache.ratis.RaftBasicTests;
-import org.apache.ratis.RaftTestUtil;
-import org.apache.ratis.client.RaftClient;
-import org.apache.ratis.protocol.RaftClientReply;
-import org.apache.ratis.server.impl.BlockRequestHandlingInjection;
-import org.apache.ratis.server.impl.RaftServerTestUtil;
-import org.apache.ratis.server.protocol.TermIndex;
-import org.apache.ratis.statemachine.SimpleStateMachine4Testing;
-import org.apache.ratis.statemachine.StateMachine;
-import org.junit.Assert;
-import org.junit.Test;
-
-import java.util.Arrays;
-import java.util.concurrent.CompletableFuture;
-
-import static org.apache.ratis.RaftTestUtil.waitForLeader;
-
-public class TestRaftWithGrpc
-    extends RaftBasicTests<MiniRaftClusterWithGrpc>
-    implements MiniRaftClusterWithGrpc.FactoryGet {
-
-  {
-    getProperties().setClass(MiniRaftCluster.STATEMACHINE_CLASS_KEY,
-        SimpleStateMachine4Testing.class, StateMachine.class);
-  }
-
-  @Override
-  @Test
-  public void testWithLoad() throws Exception {
-    super.testWithLoad();
-    BlockRequestHandlingInjection.getInstance().unblockAll();
-  }
-
-  @Test
-  public void testRequestTimeout() throws Exception {
-    try(MiniRaftClusterWithGrpc cluster = newCluster(NUM_SERVERS)) {
-      cluster.start();
-      testRequestTimeout(false, cluster, LOG);
-    }
-  }
-
-  @Test
-  public void testUpdateViaHeartbeat() throws Exception {
-    LOG.info("Running testUpdateViaHeartbeat");
-    final MiniRaftClusterWithGrpc cluster = newCluster(NUM_SERVERS);
-    cluster.start();
-    waitForLeader(cluster);
-    long waitTime = 5000;
-    try (final RaftClient client = cluster.createClient()) {
-      // block append requests
-      cluster.getServerAliveStream()
-          .filter(impl -> !impl.isLeader())
-          .map(SimpleStateMachine4Testing::get)
-          .forEach(SimpleStateMachine4Testing::blockWriteStateMachineData);
-
-      CompletableFuture<RaftClientReply>
-          replyFuture = client.sendAsync(new RaftTestUtil.SimpleMessage("abc"));
-      Thread.sleep(waitTime);
-      // replyFuture should not be completed until append request is unblocked.
-      Assert.assertTrue(!replyFuture.isDone());
-      // unblock append request.
-      cluster.getServerAliveStream()
-          .filter(impl -> !impl.isLeader())
-          .map(SimpleStateMachine4Testing::get)
-          .forEach(SimpleStateMachine4Testing::unblockWriteStateMachineData);
-
-      long index = cluster.getLeader().getState().getLog().getNextIndex();
-      TermIndex[] leaderEntries = cluster.getLeader().getState().getLog().getEntries(0, Integer.MAX_VALUE);
-      // The entries have been appended in the followers
-      // although the append entry timed out at the leader
-      cluster.getServerAliveStream().forEach(raftServer -> {
-        Assert.assertEquals(raftServer.getState().getLog().getNextIndex(), index);
-        if (!raftServer.isLeader()) {
-          TermIndex[] serverEntries = raftServer.getState().getLog().getEntries(0, Integer.MAX_VALUE);
-          Assert.assertArrayEquals(serverEntries, leaderEntries);
-        }
-      });
-
-      // Wait for heartbeats from leader to be received by followers
-      Thread.sleep(500);
-      RaftServerTestUtil.getLogAppenders(cluster.getLeader()).forEach(logAppender -> {
-        // FollowerInfo in the leader state should have updated next and match index.
-        final long followerMatchIndex = logAppender.getFollower().getMatchIndex();
-        Assert.assertTrue(followerMatchIndex >= index - 1);
-        Assert.assertEquals(followerMatchIndex + 1, logAppender.getFollower().getNextIndex());
-      });
-    }
-    cluster.shutdown();
-  }
-}

http://git-wip-us.apache.org/repos/asf/incubator-ratis/blob/5c37675f/ratis-grpc/src/test/java/org/apache/ratis/grpc/TestRetryCacheWithGrpc.java
----------------------------------------------------------------------
diff --git a/ratis-grpc/src/test/java/org/apache/ratis/grpc/TestRetryCacheWithGrpc.java b/ratis-grpc/src/test/java/org/apache/ratis/grpc/TestRetryCacheWithGrpc.java
deleted file mode 100644
index cd04b43..0000000
--- a/ratis-grpc/src/test/java/org/apache/ratis/grpc/TestRetryCacheWithGrpc.java
+++ /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.
- */
-package org.apache.ratis.grpc;
-
-import java.io.IOException;
-import java.util.ArrayList;
-import java.util.List;
-import java.util.concurrent.CompletableFuture;
-import java.util.concurrent.TimeUnit;
-import java.util.concurrent.TimeoutException;
-
-import org.apache.log4j.Level;
-import org.apache.ratis.MiniRaftCluster;
-import org.apache.ratis.RaftTestUtil;
-import org.apache.ratis.RetryCacheTests;
-import org.apache.ratis.client.RaftClient;
-import org.apache.ratis.client.RaftClientRpc;
-import org.apache.ratis.protocol.ClientId;
-import org.apache.ratis.protocol.RaftClientRequest;
-import org.apache.ratis.protocol.RaftPeerId;
-import org.apache.ratis.server.impl.RaftServerImpl;
-import org.apache.ratis.proto.RaftProtos;
-import org.apache.ratis.util.LogUtils;
-import org.junit.Assert;
-import org.junit.Test;
-
-public class TestRetryCacheWithGrpc extends RetryCacheTests {
-  static {
-    LogUtils.setLogLevel(RaftServerImpl.LOG, Level.DEBUG);
-  }
-
-  private final MiniRaftClusterWithGrpc cluster;
-
-  public TestRetryCacheWithGrpc() throws IOException {
-    cluster = MiniRaftClusterWithGrpc.FACTORY.newCluster(
-        NUM_SERVERS, properties);
-    Assert.assertNull(cluster.getLeader());
-  }
-
-  @Override
-  public MiniRaftClusterWithGrpc getCluster() {
-    return cluster;
-  }
-
-  @Test
-  public void testAsyncRetryWithReplicatedAll() throws Exception {
-    final MiniRaftCluster cluster = getCluster();
-    RaftTestUtil.waitForLeader(cluster);
-
-    final RaftPeerId leaderId = cluster.getLeaderAndSendFirstMessage().getId();
-    long oldLastApplied = cluster.getLeader().getState().getLastAppliedIndex();
-
-    // Kill a follower
-    final RaftPeerId killedFollower = cluster.getFollowers().get(0).getId();
-    cluster.killServer(killedFollower);
-
-    final long callId = 999;
-    final long seqNum = 111;
-    final ClientId clientId = ClientId.randomId();
-
-    // Retry with the same clientId and callId
-    final List<CompletableFuture<RaftClient>> futures = new ArrayList<>();
-    futures.addAll(sendRetry(clientId, leaderId, callId, seqNum, cluster));
-    futures.addAll(sendRetry(clientId, leaderId, callId, seqNum, cluster));
-
-    // restart the killed follower
-    cluster.restartServer(killedFollower, false);
-    for(CompletableFuture<RaftClient> f : futures) {
-      f.join().close();
-    }
-    assertServer(cluster, clientId, callId, oldLastApplied);
-  }
-
-  List<CompletableFuture<RaftClient>> sendRetry(
-      ClientId clientId, RaftPeerId leaderId, long callId, long seqNum, MiniRaftCluster cluster)
-      throws Exception {
-    List<CompletableFuture<RaftClient>> futures = new ArrayList<>();
-    final int numRequest = 3;
-    for (int i = 0; i < numRequest; i++) {
-      final RaftClient client = cluster.createClient(leaderId, cluster.getGroup(), clientId);
-      final RaftClientRpc rpc = client.getClientRpc();
-      final RaftClientRequest request = cluster.newRaftClientRequest(client.getId(), leaderId,
-          callId, seqNum, new RaftTestUtil.SimpleMessage("message"), RaftProtos.ReplicationLevel.ALL);
-
-      LOG.info("{} sendRequestAsync {}", i, request);
-      futures.add(rpc.sendRequestAsync(request)
-          .thenApply(reply -> assertReply(reply, client, callId)));
-    }
-
-    for(CompletableFuture<RaftClient> f : futures) {
-      try {
-        f.get(200, TimeUnit.MILLISECONDS);
-        Assert.fail("It should timeout for ReplicationLevel.ALL since a follower is down");
-      } catch(TimeoutException te) {
-        LOG.info("Expected " + te);
-      }
-    }
-    return futures;
-  }
-}

http://git-wip-us.apache.org/repos/asf/incubator-ratis/blob/5c37675f/ratis-grpc/src/test/java/org/apache/ratis/grpc/TestServerInformationWithGrpc.java
----------------------------------------------------------------------
diff --git a/ratis-grpc/src/test/java/org/apache/ratis/grpc/TestServerInformationWithGrpc.java b/ratis-grpc/src/test/java/org/apache/ratis/grpc/TestServerInformationWithGrpc.java
deleted file mode 100644
index 30be724..0000000
--- a/ratis-grpc/src/test/java/org/apache/ratis/grpc/TestServerInformationWithGrpc.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.ratis.grpc;
-
-import org.apache.ratis.server.impl.ServerInformationBaseTest;
-
-public class TestServerInformationWithGrpc
-    extends ServerInformationBaseTest<MiniRaftClusterWithGrpc>
-    implements MiniRaftClusterWithGrpc.FactoryGet {
-}

http://git-wip-us.apache.org/repos/asf/incubator-ratis/blob/5c37675f/ratis-grpc/src/test/java/org/apache/ratis/grpc/TestServerRestartWithGrpc.java
----------------------------------------------------------------------
diff --git a/ratis-grpc/src/test/java/org/apache/ratis/grpc/TestServerRestartWithGrpc.java b/ratis-grpc/src/test/java/org/apache/ratis/grpc/TestServerRestartWithGrpc.java
deleted file mode 100644
index 682b3ba..0000000
--- a/ratis-grpc/src/test/java/org/apache/ratis/grpc/TestServerRestartWithGrpc.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.ratis.grpc;
-
-import org.apache.ratis.server.ServerRestartTests;
-
-public class TestServerRestartWithGrpc
-    extends ServerRestartTests<MiniRaftClusterWithGrpc>
-    implements MiniRaftClusterWithGrpc.FactoryGet {
-}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/incubator-ratis/blob/5c37675f/ratis-grpc/src/test/java/org/apache/ratis/grpc/TestStateMachineShutdownWithGrpc.java
----------------------------------------------------------------------
diff --git a/ratis-grpc/src/test/java/org/apache/ratis/grpc/TestStateMachineShutdownWithGrpc.java b/ratis-grpc/src/test/java/org/apache/ratis/grpc/TestStateMachineShutdownWithGrpc.java
deleted file mode 100644
index d8d1d0d..0000000
--- a/ratis-grpc/src/test/java/org/apache/ratis/grpc/TestStateMachineShutdownWithGrpc.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.ratis.grpc;
-
-import org.apache.ratis.server.impl.StateMachineShutdownTests;
-
-public class TestStateMachineShutdownWithGrpc
-    extends StateMachineShutdownTests<MiniRaftClusterWithGrpc>
-    implements MiniRaftClusterWithGrpc.FactoryGet{
-
-}

http://git-wip-us.apache.org/repos/asf/incubator-ratis/blob/5c37675f/ratis-grpc/src/test/java/org/apache/ratis/grpc/TestWatchRequestWithGrpc.java
----------------------------------------------------------------------
diff --git a/ratis-grpc/src/test/java/org/apache/ratis/grpc/TestWatchRequestWithGrpc.java b/ratis-grpc/src/test/java/org/apache/ratis/grpc/TestWatchRequestWithGrpc.java
deleted file mode 100644
index 7b9061b..0000000
--- a/ratis-grpc/src/test/java/org/apache/ratis/grpc/TestWatchRequestWithGrpc.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.ratis.grpc;
-
-import org.apache.ratis.WatchRequestTests;
-
-public class TestWatchRequestWithGrpc
-    extends WatchRequestTests<MiniRaftClusterWithGrpc>
-    implements MiniRaftClusterWithGrpc.FactoryGet {
-}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/incubator-ratis/blob/5c37675f/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
deleted file mode 100644
index ced0687..0000000
--- a/ratis-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/5c37675f/ratis-netty/pom.xml
----------------------------------------------------------------------
diff --git a/ratis-netty/pom.xml b/ratis-netty/pom.xml
index b573420..42d8656 100644
--- a/ratis-netty/pom.xml
+++ b/ratis-netty/pom.xml
@@ -72,16 +72,5 @@
       <groupId>org.slf4j</groupId>
       <artifactId>slf4j-api</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/5c37675f/ratis-netty/src/test/java/org/apache/ratis/netty/TestGroupManagementWithNetty.java
----------------------------------------------------------------------
diff --git a/ratis-netty/src/test/java/org/apache/ratis/netty/TestGroupManagementWithNetty.java b/ratis-netty/src/test/java/org/apache/ratis/netty/TestGroupManagementWithNetty.java
deleted file mode 100644
index e049e32..0000000
--- a/ratis-netty/src/test/java/org/apache/ratis/netty/TestGroupManagementWithNetty.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.ratis.netty;
-
-import org.apache.ratis.MiniRaftCluster;
-import org.apache.ratis.server.impl.GroupManagementBaseTest;
-
-public class TestGroupManagementWithNetty extends GroupManagementBaseTest {
-  @Override
-  public MiniRaftCluster.Factory<? extends MiniRaftCluster> getClusterFactory() {
-    return MiniRaftClusterWithNetty.FACTORY;
-  }
-}

http://git-wip-us.apache.org/repos/asf/incubator-ratis/blob/5c37675f/ratis-netty/src/test/java/org/apache/ratis/netty/TestLeaderElectionWithNetty.java
----------------------------------------------------------------------
diff --git a/ratis-netty/src/test/java/org/apache/ratis/netty/TestLeaderElectionWithNetty.java b/ratis-netty/src/test/java/org/apache/ratis/netty/TestLeaderElectionWithNetty.java
deleted file mode 100644
index 6d40b60..0000000
--- a/ratis-netty/src/test/java/org/apache/ratis/netty/TestLeaderElectionWithNetty.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.ratis.netty;
-
-import org.apache.ratis.server.impl.BlockRequestHandlingInjection;
-import org.apache.ratis.server.impl.LeaderElectionTests;
-import org.junit.Test;
-
-public class TestLeaderElectionWithNetty
-    extends LeaderElectionTests<MiniRaftClusterWithNetty>
-    implements MiniRaftClusterWithNetty.FactoryGet {
-  @Override
-  @Test
-  public void testEnforceLeader() throws Exception {
-    super.testEnforceLeader();
-
-    MiniRaftClusterWithNetty.sendServerRequest.clear();
-    BlockRequestHandlingInjection.getInstance().unblockAll();
-  }
-}

http://git-wip-us.apache.org/repos/asf/incubator-ratis/blob/5c37675f/ratis-netty/src/test/java/org/apache/ratis/netty/TestRaftExceptionWithNetty.java
----------------------------------------------------------------------
diff --git a/ratis-netty/src/test/java/org/apache/ratis/netty/TestRaftExceptionWithNetty.java b/ratis-netty/src/test/java/org/apache/ratis/netty/TestRaftExceptionWithNetty.java
deleted file mode 100644
index 3414a65..0000000
--- a/ratis-netty/src/test/java/org/apache/ratis/netty/TestRaftExceptionWithNetty.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.ratis.netty;
-
-import org.apache.ratis.RaftExceptionBaseTest;
-
-public class TestRaftExceptionWithNetty
-    extends RaftExceptionBaseTest<MiniRaftClusterWithNetty>
-    implements MiniRaftClusterWithNetty.FactoryGet {
-}

http://git-wip-us.apache.org/repos/asf/incubator-ratis/blob/5c37675f/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
deleted file mode 100644
index 15fa8e6..0000000
--- a/ratis-netty/src/test/java/org/apache/ratis/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.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);
-  }
-}



[3/6] incubator-ratis git commit: RATIS-399. Move all tests to a new module.

Posted by sz...@apache.org.
http://git-wip-us.apache.org/repos/asf/incubator-ratis/blob/5c37675f/ratis-test/src/test/java/org/apache/ratis/grpc/TestRaftReconfigurationWithGrpc.java
----------------------------------------------------------------------
diff --git a/ratis-test/src/test/java/org/apache/ratis/grpc/TestRaftReconfigurationWithGrpc.java b/ratis-test/src/test/java/org/apache/ratis/grpc/TestRaftReconfigurationWithGrpc.java
new file mode 100644
index 0000000..29f8bea
--- /dev/null
+++ b/ratis-test/src/test/java/org/apache/ratis/grpc/TestRaftReconfigurationWithGrpc.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.grpc;
+
+import org.apache.log4j.Level;
+import org.apache.ratis.grpc.server.GrpcServerProtocolService;
+import org.apache.ratis.server.impl.RaftReconfigurationBaseTest;
+import org.apache.ratis.util.LogUtils;
+
+import java.io.IOException;
+
+public class TestRaftReconfigurationWithGrpc extends RaftReconfigurationBaseTest {
+  static {
+    LogUtils.setLogLevel(GrpcServerProtocolService.LOG, Level.DEBUG);
+  }
+
+  @Override
+  public MiniRaftClusterWithGrpc getCluster(int peerNum) {
+    return MiniRaftClusterWithGrpc.FACTORY.newCluster(peerNum, prop);
+  }
+}

http://git-wip-us.apache.org/repos/asf/incubator-ratis/blob/5c37675f/ratis-test/src/test/java/org/apache/ratis/grpc/TestRaftServerWithGrpc.java
----------------------------------------------------------------------
diff --git a/ratis-test/src/test/java/org/apache/ratis/grpc/TestRaftServerWithGrpc.java b/ratis-test/src/test/java/org/apache/ratis/grpc/TestRaftServerWithGrpc.java
new file mode 100644
index 0000000..2ec7ae8
--- /dev/null
+++ b/ratis-test/src/test/java/org/apache/ratis/grpc/TestRaftServerWithGrpc.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.grpc;
+
+import org.apache.ratis.BaseTest;
+import org.apache.ratis.RaftTestUtil;
+import org.apache.ratis.conf.RaftProperties;
+import org.apache.ratis.protocol.RaftPeerId;
+import org.apache.ratis.server.impl.ServerImplUtils;
+import org.apache.ratis.statemachine.StateMachine;
+import org.junit.Test;
+
+import java.io.IOException;
+import java.nio.channels.OverlappingFileLockException;
+
+public class TestRaftServerWithGrpc extends BaseTest {
+
+  @Test
+  public void testServerRestartOnException() throws Exception {
+    RaftProperties properties = new RaftProperties();
+    final MiniRaftClusterWithGrpc cluster
+        = MiniRaftClusterWithGrpc.FACTORY.newCluster(1, properties);
+    cluster.start();
+    RaftPeerId leaderId = RaftTestUtil.waitForLeader(cluster).getId();
+    GrpcConfigKeys.Server.setPort(properties, cluster.getLeader().getServerRpc().getInetSocketAddress().getPort());
+    // Create a raft server proxy with server rpc bound to a different address
+    // compared to leader. This helps in locking the raft storage directory to
+    // be used by next raft server proxy instance.
+    final StateMachine stateMachine = cluster.getLeader().getStateMachine();
+    ServerImplUtils.newRaftServer(leaderId, cluster.getGroup(), gid -> stateMachine, properties, null);
+    // Close the server rpc for leader so that new raft server can be bound to it.
+    cluster.getLeader().getServerRpc().close();
+
+    // Create a raft server proxy with server rpc bound to same address as
+    // the leader. This step would fail as the raft storage has been locked by
+    // the raft server proxy created earlier. Raft server proxy should close
+    // the rpc server on failure.
+    testFailureCase("start a new server with the same address",
+        () -> ServerImplUtils.newRaftServer(leaderId, cluster.getGroup(), gid -> stateMachine, properties, null).start(),
+        IOException.class, OverlappingFileLockException.class);
+    // Try to start a raft server rpc at the leader address.
+    cluster.getServer(leaderId).getFactory().newRaftServerRpc(cluster.getServer(leaderId));
+  }
+}

http://git-wip-us.apache.org/repos/asf/incubator-ratis/blob/5c37675f/ratis-test/src/test/java/org/apache/ratis/grpc/TestRaftSnapshotWithGrpc.java
----------------------------------------------------------------------
diff --git a/ratis-test/src/test/java/org/apache/ratis/grpc/TestRaftSnapshotWithGrpc.java b/ratis-test/src/test/java/org/apache/ratis/grpc/TestRaftSnapshotWithGrpc.java
new file mode 100644
index 0000000..a960478
--- /dev/null
+++ b/ratis-test/src/test/java/org/apache/ratis/grpc/TestRaftSnapshotWithGrpc.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.grpc;
+
+import org.apache.ratis.MiniRaftCluster;
+import org.apache.ratis.statemachine.RaftSnapshotBaseTest;
+
+public class TestRaftSnapshotWithGrpc extends RaftSnapshotBaseTest {
+  @Override
+  public MiniRaftCluster.Factory<?> getFactory() {
+    return MiniRaftClusterWithGrpc.FACTORY;
+  }
+}

http://git-wip-us.apache.org/repos/asf/incubator-ratis/blob/5c37675f/ratis-test/src/test/java/org/apache/ratis/grpc/TestRaftStateMachineExceptionWithGrpc.java
----------------------------------------------------------------------
diff --git a/ratis-test/src/test/java/org/apache/ratis/grpc/TestRaftStateMachineExceptionWithGrpc.java b/ratis-test/src/test/java/org/apache/ratis/grpc/TestRaftStateMachineExceptionWithGrpc.java
new file mode 100644
index 0000000..c8789a7
--- /dev/null
+++ b/ratis-test/src/test/java/org/apache/ratis/grpc/TestRaftStateMachineExceptionWithGrpc.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.grpc;
+
+import org.apache.ratis.server.impl.RaftStateMachineExceptionTests;
+
+public class TestRaftStateMachineExceptionWithGrpc
+    extends RaftStateMachineExceptionTests<MiniRaftClusterWithGrpc>
+    implements MiniRaftClusterWithGrpc.FactoryGet {
+
+}

http://git-wip-us.apache.org/repos/asf/incubator-ratis/blob/5c37675f/ratis-test/src/test/java/org/apache/ratis/grpc/TestRaftStream.java
----------------------------------------------------------------------
diff --git a/ratis-test/src/test/java/org/apache/ratis/grpc/TestRaftStream.java b/ratis-test/src/test/java/org/apache/ratis/grpc/TestRaftStream.java
new file mode 100644
index 0000000..ba31b2b
--- /dev/null
+++ b/ratis-test/src/test/java/org/apache/ratis/grpc/TestRaftStream.java
@@ -0,0 +1,311 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS 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.BaseTest;
+import org.apache.ratis.RaftTestUtil;
+import org.apache.ratis.conf.RaftProperties;
+import org.apache.ratis.grpc.client.GrpcClientStreamer;
+import org.apache.ratis.grpc.client.GrpcOutputStream;
+import org.apache.ratis.protocol.ClientId;
+import org.apache.ratis.server.impl.RaftServerImpl;
+import org.apache.ratis.server.protocol.TermIndex;
+import org.apache.ratis.server.storage.RaftLog;
+import org.apache.ratis.proto.RaftProtos;
+import org.apache.ratis.util.LogUtils;
+import org.apache.ratis.util.SizeInBytes;
+import org.apache.ratis.util.StringUtils;
+import org.junit.After;
+import org.junit.Assert;
+import org.junit.Ignore;
+import org.junit.Test;
+
+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.concurrent.atomic.AtomicReference;
+import java.util.function.Supplier;
+
+import static org.apache.ratis.RaftTestUtil.waitForLeader;
+import static org.junit.Assert.fail;
+
+@Ignore
+public class TestRaftStream extends BaseTest {
+  static {
+    LogUtils.setLogLevel(GrpcClientStreamer.LOG, Level.ALL);
+  }
+
+  private static final RaftProperties prop = new RaftProperties();
+  private static final int NUM_SERVERS = 3;
+  private static final byte[] BYTES = new byte[4];
+
+  private MiniRaftClusterWithGrpc cluster;
+
+  @After
+  public void tearDown() {
+    if (cluster != null) {
+      cluster.shutdown();
+    }
+  }
+
+  private byte[] toBytes(int i) {
+    return toBytes(i, BYTES);
+  }
+  private byte[] toBytes(int i, byte[] b) {
+    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 {
+    final int numRequests = 500;
+    LOG.info("Running testSimpleWrite, numRequests=" + numRequests);
+
+    // default 64K is too large for a test
+    GrpcConfigKeys.OutputStream.setBufferSize(prop, SizeInBytes.valueOf(4));
+    cluster = MiniRaftClusterWithGrpc.FACTORY.newCluster(NUM_SERVERS, prop);
+
+    cluster.start();
+    RaftServerImpl leader = waitForLeader(cluster);
+
+    try (GrpcOutputStream out = new GrpcOutputStream(prop, ClientId.randomId(),
+        cluster.getGroup(), leader.getId())) {
+      for (int i = 0; i < numRequests; i++) { // generate requests
+        out.write(toBytes(i));
+      }
+    }
+
+    // check the leader's raft log
+    final RaftLog raftLog = leader.getState().getLog();
+    final AtomicInteger i = new AtomicInteger();
+    checkLog(raftLog, numRequests, () -> toBytes(i.getAndIncrement()));
+  }
+
+  private void checkLog(RaftLog raftLog, long expectedCommittedIndex,
+      Supplier<byte[]> s) throws IOException {
+    long committedIndex = raftLog.getLastCommittedIndex();
+    Assert.assertEquals(expectedCommittedIndex, committedIndex);
+    // check the log content
+    TermIndex[] entries = raftLog.getEntries(1, expectedCommittedIndex + 1);
+    for (TermIndex entry : entries) {
+      RaftProtos.LogEntryProto log  = raftLog.get(entry.getIndex());
+      byte[] logData = log.getStateMachineLogEntry().getLogData().toByteArray();
+      byte[] expected = s.get();
+      LOG.info("log " + entry + " " + log.getLogEntryBodyCase() + " " + StringUtils.bytes2HexString(logData));
+      Assert.assertEquals(expected.length, logData.length);
+      Assert.assertArrayEquals(expected, logData);
+    }
+  }
+
+  @Test
+  public void testWriteAndFlush() throws Exception {
+    LOG.info("Running testWriteAndFlush");
+
+    GrpcConfigKeys.OutputStream.setBufferSize(prop, SizeInBytes.valueOf(ByteValue.BUFFERSIZE));
+    cluster = MiniRaftClusterWithGrpc.FACTORY.newCluster(NUM_SERVERS, prop);
+    cluster.start();
+
+    RaftServerImpl leader = waitForLeader(cluster);
+    GrpcOutputStream out = new GrpcOutputStream(prop, ClientId.randomId(),
+        cluster.getGroup(), 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");
+    GrpcConfigKeys.OutputStream.setBufferSize(prop, SizeInBytes.valueOf(ByteValue.BUFFERSIZE));
+
+    cluster = MiniRaftClusterWithGrpc.FACTORY.newCluster(NUM_SERVERS, prop);
+    cluster.start();
+    RaftServerImpl leader = waitForLeader(cluster);
+
+    GrpcOutputStream out = new GrpcOutputStream(prop, ClientId.randomId(),
+        cluster.getGroup(), 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());
+    TermIndex[] entries = log.getEntries(1, 9);
+    byte[] actual = new byte[ByteValue.BUFFERSIZE * 8];
+    totalSize = 0;
+    for (TermIndex e : entries) {
+      byte[] eValue = log.get(e.getIndex()).getStateMachineLogEntry().getLogData().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");
+
+    GrpcConfigKeys.OutputStream.setBufferSize(prop, SizeInBytes.valueOf(4));
+    cluster = MiniRaftClusterWithGrpc.FACTORY.newCluster(NUM_SERVERS, prop);
+    cluster.start();
+    final RaftServerImpl leader = waitForLeader(cluster);
+
+    final AtomicBoolean running  = new AtomicBoolean(true);
+    final AtomicReference<Boolean> success = new AtomicReference<>();
+    final AtomicInteger result = new AtomicInteger(0);
+    final CountDownLatch latch = new CountDownLatch(1);
+
+    new Thread(() -> {
+      LOG.info("Writer thread starts");
+      int count = 0;
+      try (GrpcOutputStream out = new GrpcOutputStream(prop, ClientId.randomId(),
+          cluster.getGroup(), 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
+    Thread.sleep(500);
+    RaftTestUtil.waitAndKillLeader(cluster);
+    final RaftServerImpl newLeader = waitForLeader(cluster);
+    Assert.assertNotEquals(leader.getId(), newLeader.getId());
+    Thread.sleep(500);
+
+    running.set(false);
+    latch.await(5, TimeUnit.SECONDS);
+    LOG.info("Writer success? " + success.get());
+    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/5c37675f/ratis-test/src/test/java/org/apache/ratis/grpc/TestRaftWithGrpc.java
----------------------------------------------------------------------
diff --git a/ratis-test/src/test/java/org/apache/ratis/grpc/TestRaftWithGrpc.java b/ratis-test/src/test/java/org/apache/ratis/grpc/TestRaftWithGrpc.java
new file mode 100644
index 0000000..d98be53
--- /dev/null
+++ b/ratis-test/src/test/java/org/apache/ratis/grpc/TestRaftWithGrpc.java
@@ -0,0 +1,110 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS 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.RaftBasicTests;
+import org.apache.ratis.RaftTestUtil;
+import org.apache.ratis.client.RaftClient;
+import org.apache.ratis.protocol.RaftClientReply;
+import org.apache.ratis.server.impl.BlockRequestHandlingInjection;
+import org.apache.ratis.server.impl.RaftServerTestUtil;
+import org.apache.ratis.server.protocol.TermIndex;
+import org.apache.ratis.statemachine.SimpleStateMachine4Testing;
+import org.apache.ratis.statemachine.StateMachine;
+import org.junit.Assert;
+import org.junit.Test;
+
+import java.util.Arrays;
+import java.util.concurrent.CompletableFuture;
+
+import static org.apache.ratis.RaftTestUtil.waitForLeader;
+
+public class TestRaftWithGrpc
+    extends RaftBasicTests<MiniRaftClusterWithGrpc>
+    implements MiniRaftClusterWithGrpc.FactoryGet {
+
+  {
+    getProperties().setClass(MiniRaftCluster.STATEMACHINE_CLASS_KEY,
+        SimpleStateMachine4Testing.class, StateMachine.class);
+  }
+
+  @Override
+  @Test
+  public void testWithLoad() throws Exception {
+    super.testWithLoad();
+    BlockRequestHandlingInjection.getInstance().unblockAll();
+  }
+
+  @Test
+  public void testRequestTimeout() throws Exception {
+    try(MiniRaftClusterWithGrpc cluster = newCluster(NUM_SERVERS)) {
+      cluster.start();
+      testRequestTimeout(false, cluster, LOG);
+    }
+  }
+
+  @Test
+  public void testUpdateViaHeartbeat() throws Exception {
+    LOG.info("Running testUpdateViaHeartbeat");
+    final MiniRaftClusterWithGrpc cluster = newCluster(NUM_SERVERS);
+    cluster.start();
+    waitForLeader(cluster);
+    long waitTime = 5000;
+    try (final RaftClient client = cluster.createClient()) {
+      // block append requests
+      cluster.getServerAliveStream()
+          .filter(impl -> !impl.isLeader())
+          .map(SimpleStateMachine4Testing::get)
+          .forEach(SimpleStateMachine4Testing::blockWriteStateMachineData);
+
+      CompletableFuture<RaftClientReply>
+          replyFuture = client.sendAsync(new RaftTestUtil.SimpleMessage("abc"));
+      Thread.sleep(waitTime);
+      // replyFuture should not be completed until append request is unblocked.
+      Assert.assertTrue(!replyFuture.isDone());
+      // unblock append request.
+      cluster.getServerAliveStream()
+          .filter(impl -> !impl.isLeader())
+          .map(SimpleStateMachine4Testing::get)
+          .forEach(SimpleStateMachine4Testing::unblockWriteStateMachineData);
+
+      long index = cluster.getLeader().getState().getLog().getNextIndex();
+      TermIndex[] leaderEntries = cluster.getLeader().getState().getLog().getEntries(0, Integer.MAX_VALUE);
+      // The entries have been appended in the followers
+      // although the append entry timed out at the leader
+      cluster.getServerAliveStream().forEach(raftServer -> {
+        Assert.assertEquals(raftServer.getState().getLog().getNextIndex(), index);
+        if (!raftServer.isLeader()) {
+          TermIndex[] serverEntries = raftServer.getState().getLog().getEntries(0, Integer.MAX_VALUE);
+          Assert.assertArrayEquals(serverEntries, leaderEntries);
+        }
+      });
+
+      // Wait for heartbeats from leader to be received by followers
+      Thread.sleep(500);
+      RaftServerTestUtil.getLogAppenders(cluster.getLeader()).forEach(logAppender -> {
+        // FollowerInfo in the leader state should have updated next and match index.
+        final long followerMatchIndex = logAppender.getFollower().getMatchIndex();
+        Assert.assertTrue(followerMatchIndex >= index - 1);
+        Assert.assertEquals(followerMatchIndex + 1, logAppender.getFollower().getNextIndex());
+      });
+    }
+    cluster.shutdown();
+  }
+}

http://git-wip-us.apache.org/repos/asf/incubator-ratis/blob/5c37675f/ratis-test/src/test/java/org/apache/ratis/grpc/TestRetryCacheWithGrpc.java
----------------------------------------------------------------------
diff --git a/ratis-test/src/test/java/org/apache/ratis/grpc/TestRetryCacheWithGrpc.java b/ratis-test/src/test/java/org/apache/ratis/grpc/TestRetryCacheWithGrpc.java
new file mode 100644
index 0000000..cd04b43
--- /dev/null
+++ b/ratis-test/src/test/java/org/apache/ratis/grpc/TestRetryCacheWithGrpc.java
@@ -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.
+ */
+package org.apache.ratis.grpc;
+
+import java.io.IOException;
+import java.util.ArrayList;
+import java.util.List;
+import java.util.concurrent.CompletableFuture;
+import java.util.concurrent.TimeUnit;
+import java.util.concurrent.TimeoutException;
+
+import org.apache.log4j.Level;
+import org.apache.ratis.MiniRaftCluster;
+import org.apache.ratis.RaftTestUtil;
+import org.apache.ratis.RetryCacheTests;
+import org.apache.ratis.client.RaftClient;
+import org.apache.ratis.client.RaftClientRpc;
+import org.apache.ratis.protocol.ClientId;
+import org.apache.ratis.protocol.RaftClientRequest;
+import org.apache.ratis.protocol.RaftPeerId;
+import org.apache.ratis.server.impl.RaftServerImpl;
+import org.apache.ratis.proto.RaftProtos;
+import org.apache.ratis.util.LogUtils;
+import org.junit.Assert;
+import org.junit.Test;
+
+public class TestRetryCacheWithGrpc extends RetryCacheTests {
+  static {
+    LogUtils.setLogLevel(RaftServerImpl.LOG, Level.DEBUG);
+  }
+
+  private final MiniRaftClusterWithGrpc cluster;
+
+  public TestRetryCacheWithGrpc() throws IOException {
+    cluster = MiniRaftClusterWithGrpc.FACTORY.newCluster(
+        NUM_SERVERS, properties);
+    Assert.assertNull(cluster.getLeader());
+  }
+
+  @Override
+  public MiniRaftClusterWithGrpc getCluster() {
+    return cluster;
+  }
+
+  @Test
+  public void testAsyncRetryWithReplicatedAll() throws Exception {
+    final MiniRaftCluster cluster = getCluster();
+    RaftTestUtil.waitForLeader(cluster);
+
+    final RaftPeerId leaderId = cluster.getLeaderAndSendFirstMessage().getId();
+    long oldLastApplied = cluster.getLeader().getState().getLastAppliedIndex();
+
+    // Kill a follower
+    final RaftPeerId killedFollower = cluster.getFollowers().get(0).getId();
+    cluster.killServer(killedFollower);
+
+    final long callId = 999;
+    final long seqNum = 111;
+    final ClientId clientId = ClientId.randomId();
+
+    // Retry with the same clientId and callId
+    final List<CompletableFuture<RaftClient>> futures = new ArrayList<>();
+    futures.addAll(sendRetry(clientId, leaderId, callId, seqNum, cluster));
+    futures.addAll(sendRetry(clientId, leaderId, callId, seqNum, cluster));
+
+    // restart the killed follower
+    cluster.restartServer(killedFollower, false);
+    for(CompletableFuture<RaftClient> f : futures) {
+      f.join().close();
+    }
+    assertServer(cluster, clientId, callId, oldLastApplied);
+  }
+
+  List<CompletableFuture<RaftClient>> sendRetry(
+      ClientId clientId, RaftPeerId leaderId, long callId, long seqNum, MiniRaftCluster cluster)
+      throws Exception {
+    List<CompletableFuture<RaftClient>> futures = new ArrayList<>();
+    final int numRequest = 3;
+    for (int i = 0; i < numRequest; i++) {
+      final RaftClient client = cluster.createClient(leaderId, cluster.getGroup(), clientId);
+      final RaftClientRpc rpc = client.getClientRpc();
+      final RaftClientRequest request = cluster.newRaftClientRequest(client.getId(), leaderId,
+          callId, seqNum, new RaftTestUtil.SimpleMessage("message"), RaftProtos.ReplicationLevel.ALL);
+
+      LOG.info("{} sendRequestAsync {}", i, request);
+      futures.add(rpc.sendRequestAsync(request)
+          .thenApply(reply -> assertReply(reply, client, callId)));
+    }
+
+    for(CompletableFuture<RaftClient> f : futures) {
+      try {
+        f.get(200, TimeUnit.MILLISECONDS);
+        Assert.fail("It should timeout for ReplicationLevel.ALL since a follower is down");
+      } catch(TimeoutException te) {
+        LOG.info("Expected " + te);
+      }
+    }
+    return futures;
+  }
+}

http://git-wip-us.apache.org/repos/asf/incubator-ratis/blob/5c37675f/ratis-test/src/test/java/org/apache/ratis/grpc/TestServerInformationWithGrpc.java
----------------------------------------------------------------------
diff --git a/ratis-test/src/test/java/org/apache/ratis/grpc/TestServerInformationWithGrpc.java b/ratis-test/src/test/java/org/apache/ratis/grpc/TestServerInformationWithGrpc.java
new file mode 100644
index 0000000..30be724
--- /dev/null
+++ b/ratis-test/src/test/java/org/apache/ratis/grpc/TestServerInformationWithGrpc.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.ratis.grpc;
+
+import org.apache.ratis.server.impl.ServerInformationBaseTest;
+
+public class TestServerInformationWithGrpc
+    extends ServerInformationBaseTest<MiniRaftClusterWithGrpc>
+    implements MiniRaftClusterWithGrpc.FactoryGet {
+}

http://git-wip-us.apache.org/repos/asf/incubator-ratis/blob/5c37675f/ratis-test/src/test/java/org/apache/ratis/grpc/TestServerRestartWithGrpc.java
----------------------------------------------------------------------
diff --git a/ratis-test/src/test/java/org/apache/ratis/grpc/TestServerRestartWithGrpc.java b/ratis-test/src/test/java/org/apache/ratis/grpc/TestServerRestartWithGrpc.java
new file mode 100644
index 0000000..682b3ba
--- /dev/null
+++ b/ratis-test/src/test/java/org/apache/ratis/grpc/TestServerRestartWithGrpc.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.ratis.grpc;
+
+import org.apache.ratis.server.ServerRestartTests;
+
+public class TestServerRestartWithGrpc
+    extends ServerRestartTests<MiniRaftClusterWithGrpc>
+    implements MiniRaftClusterWithGrpc.FactoryGet {
+}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/incubator-ratis/blob/5c37675f/ratis-test/src/test/java/org/apache/ratis/grpc/TestStateMachineShutdownWithGrpc.java
----------------------------------------------------------------------
diff --git a/ratis-test/src/test/java/org/apache/ratis/grpc/TestStateMachineShutdownWithGrpc.java b/ratis-test/src/test/java/org/apache/ratis/grpc/TestStateMachineShutdownWithGrpc.java
new file mode 100644
index 0000000..d8d1d0d
--- /dev/null
+++ b/ratis-test/src/test/java/org/apache/ratis/grpc/TestStateMachineShutdownWithGrpc.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.grpc;
+
+import org.apache.ratis.server.impl.StateMachineShutdownTests;
+
+public class TestStateMachineShutdownWithGrpc
+    extends StateMachineShutdownTests<MiniRaftClusterWithGrpc>
+    implements MiniRaftClusterWithGrpc.FactoryGet{
+
+}

http://git-wip-us.apache.org/repos/asf/incubator-ratis/blob/5c37675f/ratis-test/src/test/java/org/apache/ratis/grpc/TestWatchRequestWithGrpc.java
----------------------------------------------------------------------
diff --git a/ratis-test/src/test/java/org/apache/ratis/grpc/TestWatchRequestWithGrpc.java b/ratis-test/src/test/java/org/apache/ratis/grpc/TestWatchRequestWithGrpc.java
new file mode 100644
index 0000000..7b9061b
--- /dev/null
+++ b/ratis-test/src/test/java/org/apache/ratis/grpc/TestWatchRequestWithGrpc.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.ratis.grpc;
+
+import org.apache.ratis.WatchRequestTests;
+
+public class TestWatchRequestWithGrpc
+    extends WatchRequestTests<MiniRaftClusterWithGrpc>
+    implements MiniRaftClusterWithGrpc.FactoryGet {
+}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/incubator-ratis/blob/5c37675f/ratis-test/src/test/java/org/apache/ratis/netty/TestGroupManagementWithNetty.java
----------------------------------------------------------------------
diff --git a/ratis-test/src/test/java/org/apache/ratis/netty/TestGroupManagementWithNetty.java b/ratis-test/src/test/java/org/apache/ratis/netty/TestGroupManagementWithNetty.java
new file mode 100644
index 0000000..e049e32
--- /dev/null
+++ b/ratis-test/src/test/java/org/apache/ratis/netty/TestGroupManagementWithNetty.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.netty;
+
+import org.apache.ratis.MiniRaftCluster;
+import org.apache.ratis.server.impl.GroupManagementBaseTest;
+
+public class TestGroupManagementWithNetty extends GroupManagementBaseTest {
+  @Override
+  public MiniRaftCluster.Factory<? extends MiniRaftCluster> getClusterFactory() {
+    return MiniRaftClusterWithNetty.FACTORY;
+  }
+}

http://git-wip-us.apache.org/repos/asf/incubator-ratis/blob/5c37675f/ratis-test/src/test/java/org/apache/ratis/netty/TestLeaderElectionWithNetty.java
----------------------------------------------------------------------
diff --git a/ratis-test/src/test/java/org/apache/ratis/netty/TestLeaderElectionWithNetty.java b/ratis-test/src/test/java/org/apache/ratis/netty/TestLeaderElectionWithNetty.java
new file mode 100644
index 0000000..6d40b60
--- /dev/null
+++ b/ratis-test/src/test/java/org/apache/ratis/netty/TestLeaderElectionWithNetty.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.netty;
+
+import org.apache.ratis.server.impl.BlockRequestHandlingInjection;
+import org.apache.ratis.server.impl.LeaderElectionTests;
+import org.junit.Test;
+
+public class TestLeaderElectionWithNetty
+    extends LeaderElectionTests<MiniRaftClusterWithNetty>
+    implements MiniRaftClusterWithNetty.FactoryGet {
+  @Override
+  @Test
+  public void testEnforceLeader() throws Exception {
+    super.testEnforceLeader();
+
+    MiniRaftClusterWithNetty.sendServerRequest.clear();
+    BlockRequestHandlingInjection.getInstance().unblockAll();
+  }
+}

http://git-wip-us.apache.org/repos/asf/incubator-ratis/blob/5c37675f/ratis-test/src/test/java/org/apache/ratis/netty/TestRaftExceptionWithNetty.java
----------------------------------------------------------------------
diff --git a/ratis-test/src/test/java/org/apache/ratis/netty/TestRaftExceptionWithNetty.java b/ratis-test/src/test/java/org/apache/ratis/netty/TestRaftExceptionWithNetty.java
new file mode 100644
index 0000000..3414a65
--- /dev/null
+++ b/ratis-test/src/test/java/org/apache/ratis/netty/TestRaftExceptionWithNetty.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.ratis.netty;
+
+import org.apache.ratis.RaftExceptionBaseTest;
+
+public class TestRaftExceptionWithNetty
+    extends RaftExceptionBaseTest<MiniRaftClusterWithNetty>
+    implements MiniRaftClusterWithNetty.FactoryGet {
+}

http://git-wip-us.apache.org/repos/asf/incubator-ratis/blob/5c37675f/ratis-test/src/test/java/org/apache/ratis/netty/TestRaftReconfigurationWithNetty.java
----------------------------------------------------------------------
diff --git a/ratis-test/src/test/java/org/apache/ratis/netty/TestRaftReconfigurationWithNetty.java b/ratis-test/src/test/java/org/apache/ratis/netty/TestRaftReconfigurationWithNetty.java
new file mode 100644
index 0000000..15fa8e6
--- /dev/null
+++ b/ratis-test/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);
+  }
+}

http://git-wip-us.apache.org/repos/asf/incubator-ratis/blob/5c37675f/ratis-test/src/test/java/org/apache/ratis/netty/TestRaftSnapshotWithNetty.java
----------------------------------------------------------------------
diff --git a/ratis-test/src/test/java/org/apache/ratis/netty/TestRaftSnapshotWithNetty.java b/ratis-test/src/test/java/org/apache/ratis/netty/TestRaftSnapshotWithNetty.java
new file mode 100644
index 0000000..da861d5
--- /dev/null
+++ b/ratis-test/src/test/java/org/apache/ratis/netty/TestRaftSnapshotWithNetty.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 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.Factory<?> getFactory() {
+    return MiniRaftClusterWithNetty.FACTORY;
+  }
+}

http://git-wip-us.apache.org/repos/asf/incubator-ratis/blob/5c37675f/ratis-test/src/test/java/org/apache/ratis/netty/TestRaftStateMachineExceptionWithNetty.java
----------------------------------------------------------------------
diff --git a/ratis-test/src/test/java/org/apache/ratis/netty/TestRaftStateMachineExceptionWithNetty.java b/ratis-test/src/test/java/org/apache/ratis/netty/TestRaftStateMachineExceptionWithNetty.java
new file mode 100644
index 0000000..34fddc9
--- /dev/null
+++ b/ratis-test/src/test/java/org/apache/ratis/netty/TestRaftStateMachineExceptionWithNetty.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.ratis.netty;
+
+import org.apache.ratis.server.impl.RaftStateMachineExceptionTests;
+
+public class TestRaftStateMachineExceptionWithNetty
+    extends RaftStateMachineExceptionTests<MiniRaftClusterWithNetty>
+    implements MiniRaftClusterWithNetty.FactoryGet {
+}

http://git-wip-us.apache.org/repos/asf/incubator-ratis/blob/5c37675f/ratis-test/src/test/java/org/apache/ratis/netty/TestRaftWithNetty.java
----------------------------------------------------------------------
diff --git a/ratis-test/src/test/java/org/apache/ratis/netty/TestRaftWithNetty.java b/ratis-test/src/test/java/org/apache/ratis/netty/TestRaftWithNetty.java
new file mode 100644
index 0000000..28815d7
--- /dev/null
+++ b/ratis-test/src/test/java/org/apache/ratis/netty/TestRaftWithNetty.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.netty;
+
+import org.apache.ratis.RaftBasicTests;
+import org.apache.ratis.server.impl.BlockRequestHandlingInjection;
+import org.junit.Test;
+
+public class TestRaftWithNetty
+    extends RaftBasicTests<MiniRaftClusterWithNetty>
+    implements MiniRaftClusterWithNetty.FactoryGet {
+
+  @Override
+  @Test
+  public void testWithLoad() throws Exception {
+    super.testWithLoad();
+    BlockRequestHandlingInjection.getInstance().unblockAll();
+  }
+}

http://git-wip-us.apache.org/repos/asf/incubator-ratis/blob/5c37675f/ratis-test/src/test/java/org/apache/ratis/netty/TestRetryCacheWithNettyRpc.java
----------------------------------------------------------------------
diff --git a/ratis-test/src/test/java/org/apache/ratis/netty/TestRetryCacheWithNettyRpc.java b/ratis-test/src/test/java/org/apache/ratis/netty/TestRetryCacheWithNettyRpc.java
new file mode 100644
index 0000000..659e426
--- /dev/null
+++ b/ratis-test/src/test/java/org/apache/ratis/netty/TestRetryCacheWithNettyRpc.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.netty;
+
+import java.io.IOException;
+
+import org.apache.log4j.Level;
+import org.apache.ratis.RetryCacheTests;
+import org.apache.ratis.client.RaftClient;
+import org.apache.ratis.server.impl.RaftServerImpl;
+import org.apache.ratis.util.LogUtils;
+
+public class TestRetryCacheWithNettyRpc extends RetryCacheTests {
+  static {
+    LogUtils.setLogLevel(RaftServerImpl.LOG, Level.DEBUG);
+    LogUtils.setLogLevel(RaftClient.LOG, Level.DEBUG);
+  }
+
+  private final MiniRaftClusterWithNetty cluster;
+
+  public TestRetryCacheWithNettyRpc() throws IOException {
+    cluster = MiniRaftClusterWithNetty.FACTORY.newCluster(
+        NUM_SERVERS, getProperties());
+  }
+
+  @Override
+  public MiniRaftClusterWithNetty getCluster() {
+    return cluster;
+  }
+}

http://git-wip-us.apache.org/repos/asf/incubator-ratis/blob/5c37675f/ratis-test/src/test/java/org/apache/ratis/netty/TestServerInformationWithNetty.java
----------------------------------------------------------------------
diff --git a/ratis-test/src/test/java/org/apache/ratis/netty/TestServerInformationWithNetty.java b/ratis-test/src/test/java/org/apache/ratis/netty/TestServerInformationWithNetty.java
new file mode 100644
index 0000000..d1bcae4
--- /dev/null
+++ b/ratis-test/src/test/java/org/apache/ratis/netty/TestServerInformationWithNetty.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.ratis.netty;
+
+import org.apache.ratis.server.impl.ServerInformationBaseTest;
+
+public class TestServerInformationWithNetty
+    extends ServerInformationBaseTest<MiniRaftClusterWithNetty>
+    implements MiniRaftClusterWithNetty.FactoryGet {
+}

http://git-wip-us.apache.org/repos/asf/incubator-ratis/blob/5c37675f/ratis-test/src/test/java/org/apache/ratis/netty/TestServerRestartWithNetty.java
----------------------------------------------------------------------
diff --git a/ratis-test/src/test/java/org/apache/ratis/netty/TestServerRestartWithNetty.java b/ratis-test/src/test/java/org/apache/ratis/netty/TestServerRestartWithNetty.java
new file mode 100644
index 0000000..15dc688
--- /dev/null
+++ b/ratis-test/src/test/java/org/apache/ratis/netty/TestServerRestartWithNetty.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.ratis.netty;
+
+import org.apache.ratis.server.ServerRestartTests;
+
+public class TestServerRestartWithNetty
+    extends ServerRestartTests<MiniRaftClusterWithNetty>
+    implements MiniRaftClusterWithNetty.FactoryGet {
+}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/incubator-ratis/blob/5c37675f/ratis-test/src/test/java/org/apache/ratis/protocol/TestRaftId.java
----------------------------------------------------------------------
diff --git a/ratis-test/src/test/java/org/apache/ratis/protocol/TestRaftId.java b/ratis-test/src/test/java/org/apache/ratis/protocol/TestRaftId.java
new file mode 100644
index 0000000..30b7ed5
--- /dev/null
+++ b/ratis-test/src/test/java/org/apache/ratis/protocol/TestRaftId.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.protocol;
+
+import org.apache.ratis.BaseTest;
+import org.apache.ratis.thirdparty.com.google.protobuf.ByteString;
+import org.junit.Assert;
+import org.junit.Test;
+
+public class TestRaftId extends BaseTest {
+  @Override
+  public int getGlobalTimeoutSeconds() {
+    return 1;
+  }
+
+  @Test
+  public void testClientId() {
+    final ClientId id = ClientId.randomId();
+    final ByteString bytes = id.toByteString();
+    Assert.assertEquals(bytes, id.toByteString());
+    Assert.assertEquals(id, ClientId.valueOf(bytes));
+  }
+
+  @Test
+  public void testRaftGroupId() {
+    final RaftGroupId id = RaftGroupId.randomId();
+    final ByteString bytes = id.toByteString();
+    Assert.assertEquals(bytes, id.toByteString());
+    Assert.assertEquals(id, RaftGroupId.valueOf(bytes));
+  }
+
+  @Test
+  public void testRaftPeerId() {
+    final RaftPeerId id = RaftPeerId.valueOf("abc");
+    final ByteString bytes = id.toByteString();
+    Assert.assertEquals(bytes, id.toByteString());
+    Assert.assertEquals(id, RaftPeerId.valueOf(bytes));
+  }
+}

http://git-wip-us.apache.org/repos/asf/incubator-ratis/blob/5c37675f/ratis-test/src/test/java/org/apache/ratis/server/TestRaftLogMetrics.java
----------------------------------------------------------------------
diff --git a/ratis-test/src/test/java/org/apache/ratis/server/TestRaftLogMetrics.java b/ratis-test/src/test/java/org/apache/ratis/server/TestRaftLogMetrics.java
new file mode 100644
index 0000000..58e319d
--- /dev/null
+++ b/ratis-test/src/test/java/org/apache/ratis/server/TestRaftLogMetrics.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
+ * <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;
+
+import com.codahale.metrics.Timer;
+import org.apache.log4j.Level;
+import org.apache.ratis.BaseTest;
+import org.apache.ratis.MiniRaftCluster;
+import org.apache.ratis.RaftTestUtil;
+import org.apache.ratis.client.RaftClient;
+import org.apache.ratis.metrics.RatisMetricsRegistry;
+import org.apache.ratis.server.impl.RaftServerImpl;
+import org.apache.ratis.server.simulation.MiniRaftClusterWithSimulatedRpc;
+import org.apache.ratis.server.storage.RaftStorageTestUtils;
+import org.apache.ratis.statemachine.StateMachine;
+import org.apache.ratis.statemachine.impl.BaseStateMachine;
+import org.apache.ratis.util.JavaUtils;
+import org.apache.ratis.util.LogUtils;
+import org.junit.Assert;
+import org.junit.Test;
+
+import javax.management.ObjectName;
+import java.lang.management.ManagementFactory;
+import java.util.concurrent.CompletableFuture;
+import java.util.concurrent.atomic.AtomicInteger;
+
+public class TestRaftLogMetrics extends BaseTest
+    implements MiniRaftClusterWithSimulatedRpc.FactoryGet {
+
+  {
+    LogUtils.setLogLevel(RaftServerImpl.LOG, Level.DEBUG);
+  }
+
+  public static final int NUM_SERVERS = 3;
+
+  {
+    getProperties().setClass(MiniRaftCluster.STATEMACHINE_CLASS_KEY,
+        MetricsStateMachine.class, StateMachine.class);
+  }
+
+  static class MetricsStateMachine extends BaseStateMachine {
+    static MetricsStateMachine get(RaftServerImpl s) {
+      return (MetricsStateMachine)s.getStateMachine();
+    }
+
+    private final AtomicInteger flushCount = new AtomicInteger();
+
+    int getFlushCount() {
+      return flushCount.get();
+    }
+
+    @Override
+    public CompletableFuture<Void> flushStateMachineData(long index) {
+      flushCount.incrementAndGet();
+      return super.flushStateMachineData(index);
+    }
+  }
+
+  @Test
+  public void testFlushMetric() throws Exception {
+    try(final MiniRaftCluster cluster = newCluster(NUM_SERVERS)) {
+      cluster.start();
+      runTestFlushMetric(cluster);
+    }
+  }
+
+  static void runTestFlushMetric(MiniRaftCluster cluster) throws Exception {
+    int numMsg = 2;
+    final RaftTestUtil.SimpleMessage[] messages = RaftTestUtil.SimpleMessage.create(numMsg);
+
+    try (final RaftClient client = cluster.createClient()) {
+      for (RaftTestUtil.SimpleMessage message : messages) {
+        client.send(message);
+      }
+    }
+
+    // For leader, flush must happen before client can get replies.
+    assertFlushCount(cluster.getLeader());
+
+    // For followers, flush can be lagged behind.  Attempt multiple times.
+    for(RaftServerImpl f : cluster.getFollowers()) {
+      JavaUtils.attempt(() -> assertFlushCount(f), 10, 100, f.getId() + "-assertFlushCount", null);
+    }
+  }
+
+  static void assertFlushCount(RaftServerImpl server) throws Exception {
+      final String flushTimeMetric = RaftStorageTestUtils.getLogFlushTimeMetric(server.getId());
+      Timer tm = RatisMetricsRegistry.getRegistry().getTimers().get(flushTimeMetric);
+      Assert.assertNotNull(tm);
+
+      final MetricsStateMachine stateMachine = MetricsStateMachine.get(server);
+      final int expectedFlush = stateMachine.getFlushCount();
+
+      Assert.assertEquals(expectedFlush, tm.getCount());
+      Assert.assertTrue(tm.getMeanRate() > 0);
+
+      // Test jmx
+      ObjectName oname = new ObjectName("metrics", "name", flushTimeMetric);
+      Assert.assertEquals(expectedFlush,
+          ((Long) ManagementFactory.getPlatformMBeanServer().getAttribute(oname, "Count"))
+              .intValue());
+  }
+}

http://git-wip-us.apache.org/repos/asf/incubator-ratis/blob/5c37675f/ratis-test/src/test/java/org/apache/ratis/server/TestRaftServerConfigKeys.java
----------------------------------------------------------------------
diff --git a/ratis-test/src/test/java/org/apache/ratis/server/TestRaftServerConfigKeys.java b/ratis-test/src/test/java/org/apache/ratis/server/TestRaftServerConfigKeys.java
new file mode 100644
index 0000000..53c8871
--- /dev/null
+++ b/ratis-test/src/test/java/org/apache/ratis/server/TestRaftServerConfigKeys.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;
+
+import org.apache.ratis.BaseTest;
+import org.apache.ratis.conf.RaftProperties;
+import org.apache.ratis.util.FileUtils;
+import org.apache.ratis.util.JavaUtils;
+import org.junit.AfterClass;
+import org.junit.Assert;
+import org.junit.Test;
+
+import java.io.File;
+import java.io.IOException;
+import java.util.ArrayList;
+import java.util.List;
+import java.util.UUID;
+import java.util.concurrent.ThreadLocalRandom;
+import java.util.function.Supplier;
+import java.util.stream.Collectors;
+import java.util.stream.IntStream;
+
+/**
+ * Test cases to verify RaftServerConfigKeys.
+ */
+public class TestRaftServerConfigKeys {
+
+  private static final Supplier<File> rootTestDir = JavaUtils.memoize(
+      () -> new File(BaseTest.getRootTestDir(),
+          TestRaftServerConfigKeys.class.getSimpleName() +
+              Integer.toHexString(ThreadLocalRandom.current().nextInt())));
+
+  @AfterClass
+  public static void tearDown() throws IOException {
+    FileUtils.deleteFully(rootTestDir.get());
+  }
+
+  /**
+   * Sets the value to <code>raft.server.storage.dir</code> via
+   * RaftServerConfigKeys and verifies it by reading directly from property.
+   */
+  @Test
+  public void testStorageDirsProperty() {
+    final File testDir = new File(
+        rootTestDir.get(), UUID.randomUUID().toString());
+    final List<File> directories = new ArrayList<>();
+    final  RaftProperties properties = new RaftProperties();
+
+    IntStream.range(0, 10).mapToObj((i) -> new File(testDir,
+        Integer.toString(i))).forEach(directories::add);
+    RaftServerConfigKeys.setStorageDirs(properties, directories);
+
+    final String expected = directories.stream().map(File::getAbsolutePath)
+        .collect(Collectors.joining(","));
+    final String actual = properties.get(RaftServerConfigKeys.STORAGE_DIR_KEY);
+    Assert.assertEquals(expected, actual);
+  }
+
+  /**
+   * Sets the value to <code>raft.server.storage.dir</code> via
+   * RaftServerConfigKeys and also verifies the same via RaftServerConfigKeys.
+   */
+  @Test
+  public void testStorageDirs() {
+    final File testDir = new File(
+        rootTestDir.get(), UUID.randomUUID().toString());
+    final List<File> directories = new ArrayList<>();
+    IntStream.range(0, 10).mapToObj((i) -> new File(testDir,
+        Integer.toString(i))).forEach(directories::add);
+    RaftProperties properties = new RaftProperties();
+    RaftServerConfigKeys.setStorageDirs(properties, directories);
+
+    final List<File> storageDirs = RaftServerConfigKeys.storageDirs(properties);
+    final List<String> expectedDirs = directories.stream()
+        .map(File::getAbsolutePath).collect(Collectors.toList());
+    final List<String> actualDirs = storageDirs.stream()
+        .map(File::getAbsolutePath).collect(Collectors.toList());
+    actualDirs.removeAll(expectedDirs);
+    Assert.assertEquals(directories.size(), storageDirs.size());
+    Assert.assertEquals(0, actualDirs.size());
+  }
+}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/incubator-ratis/blob/5c37675f/ratis-test/src/test/java/org/apache/ratis/server/impl/TestRaftServerJmx.java
----------------------------------------------------------------------
diff --git a/ratis-test/src/test/java/org/apache/ratis/server/impl/TestRaftServerJmx.java b/ratis-test/src/test/java/org/apache/ratis/server/impl/TestRaftServerJmx.java
new file mode 100644
index 0000000..f060645
--- /dev/null
+++ b/ratis-test/src/test/java/org/apache/ratis/server/impl/TestRaftServerJmx.java
@@ -0,0 +1,109 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.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 org.apache.ratis.BaseTest;
+import org.apache.ratis.conf.RaftProperties;
+import org.apache.ratis.protocol.RaftGroupId;
+import org.apache.ratis.protocol.RaftPeerId;
+import org.apache.ratis.server.RaftServerMXBean;
+import org.apache.ratis.server.simulation.MiniRaftClusterWithSimulatedRpc;
+import org.apache.ratis.util.JmxRegister;
+import org.junit.Assert;
+import org.junit.Test;
+
+import javax.management.JMException;
+import javax.management.MBeanServer;
+import javax.management.ObjectInstance;
+import javax.management.ObjectName;
+import java.lang.management.ManagementFactory;
+import java.util.List;
+import java.util.Set;
+
+import static org.apache.ratis.RaftTestUtil.waitForLeader;
+
+public class TestRaftServerJmx extends BaseTest {
+  @Test(timeout = 30000)
+  public void testJmxBeans() throws Exception {
+    final int NUM_SERVERS = 3;
+    final MiniRaftClusterWithSimulatedRpc cluster
+        = MiniRaftClusterWithSimulatedRpc.FACTORY.newCluster(3, new RaftProperties());
+    cluster.start();
+    waitForLeader(cluster);
+
+    MBeanServer platformMBeanServer = ManagementFactory.getPlatformMBeanServer();
+    Set<ObjectInstance> objectInstances = platformMBeanServer.queryMBeans(new ObjectName("Ratis:*"), null);
+    Assert.assertEquals(NUM_SERVERS, objectInstances.size());
+
+    for (ObjectInstance instance : objectInstances) {
+      Object groupId = platformMBeanServer.getAttribute(instance.getObjectName(), "GroupId");
+      Assert.assertEquals(cluster.getGroupId().toString(), groupId);
+    }
+    cluster.shutdown();
+  }
+
+  @Test(timeout = 30000)
+  public void testRegister() throws JMException {
+    {
+      final JmxRegister jmx = new JmxRegister();
+      runUnregister(false, jmx);
+
+      runRegister(true, "abc", jmx);
+      runRegister(false, "abc", jmx);
+      runUnregister(true, jmx);
+      runUnregister(false, jmx);
+
+      runRegister(true, "abc", jmx);
+      runUnregister(true, jmx);
+      runUnregister(false, jmx);
+    }
+
+    {
+      final JmxRegister jmx = new JmxRegister();
+      runRegister(true, "host:1234", jmx);
+      runUnregister(true, jmx);
+      runUnregister(false, jmx);
+    }
+  }
+
+  static void runRegister(boolean expectToSucceed, String name, JmxRegister jmx) {
+    final RaftServerMXBean mBean = new RaftServerMXBean() {
+      @Override
+      public String getId() { return null; }
+      @Override
+      public String getLeaderId() { return null; }
+      @Override
+      public long getCurrentTerm() { return 0; }
+      @Override
+      public String getGroupId() { return null; }
+      @Override
+      public String getRole() { return null; }
+      @Override
+      public List<String> getFollowers() { return null; }
+    };
+    final RaftPeerId id = RaftPeerId.valueOf(name);
+    final RaftGroupId groupId = RaftGroupId.randomId();
+    final boolean succeeded = RaftServerImpl.registerMBean(id, groupId, mBean, jmx);
+    Assert.assertEquals(expectToSucceed, succeeded);
+  }
+
+  static void runUnregister(boolean expectToSucceed, JmxRegister jmx) throws JMException {
+    final boolean succeeded = jmx.unregister();
+    Assert.assertEquals(expectToSucceed, succeeded);
+  }
+}

http://git-wip-us.apache.org/repos/asf/incubator-ratis/blob/5c37675f/ratis-test/src/test/java/org/apache/ratis/server/impl/TestServerState.java
----------------------------------------------------------------------
diff --git a/ratis-test/src/test/java/org/apache/ratis/server/impl/TestServerState.java b/ratis-test/src/test/java/org/apache/ratis/server/impl/TestServerState.java
new file mode 100644
index 0000000..5801d2b
--- /dev/null
+++ b/ratis-test/src/test/java/org/apache/ratis/server/impl/TestServerState.java
@@ -0,0 +1,127 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS 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.BaseTest;
+import org.apache.ratis.util.FileUtils;
+import org.apache.ratis.util.JavaUtils;
+import org.junit.AfterClass;
+import org.junit.Assert;
+import org.junit.Test;
+
+import java.io.File;
+import java.io.IOException;
+import java.util.ArrayList;
+import java.util.Collections;
+import java.util.List;
+import java.util.UUID;
+import java.util.concurrent.ThreadLocalRandom;
+import java.util.function.Supplier;
+import java.util.stream.IntStream;
+
+/**
+ * Test cases to verify ServerState.
+ */
+public class TestServerState {
+
+  private static final Supplier<File> rootTestDir = JavaUtils.memoize(
+      () -> new File(BaseTest.getRootTestDir(),
+          TestServerState.class.getSimpleName() +
+              Integer.toHexString(ThreadLocalRandom.current().nextInt())));
+
+  @AfterClass
+  public static void tearDown() throws IOException {
+    FileUtils.deleteFully(rootTestDir.get());
+  }
+
+  /**
+   * Tests choosing of storage directory when only one volume is configured.
+   *
+   * @throws IOException in case of exception.
+   */
+  @Test
+  public void testChooseStorageDirWithOneVolume() throws IOException {
+    File testDir = new File(rootTestDir.get(), UUID.randomUUID().toString());
+    List<File> directories = Collections.singletonList(testDir);
+    String subDirOne = UUID.randomUUID().toString();
+    String subDirTwo = UUID.randomUUID().toString();
+    File storageDirOne = ServerState.chooseStorageDir(directories, subDirOne);
+    File storageDirTwo = ServerState.chooseStorageDir(directories, subDirTwo);
+    File expectedOne = new File(testDir, subDirOne);
+    File expectedTwo = new File(testDir, subDirTwo);
+    Assert.assertEquals(expectedOne.getCanonicalPath(),
+        storageDirOne.getCanonicalPath());
+    Assert.assertEquals(expectedTwo.getCanonicalPath(),
+        storageDirTwo.getCanonicalPath());
+  }
+
+  /**
+   * Tests choosing of storage directory when multiple volumes are configured.
+   *
+   * @throws IOException in case of exception.
+   */
+  @Test
+  public void testChooseStorageDirWithMultipleVolumes() throws IOException {
+    File testDir = new File(rootTestDir.get(), UUID.randomUUID().toString());
+    List<File> directories = new ArrayList<>();
+    IntStream.range(0, 10).mapToObj((i) -> new File(testDir,
+        Integer.toString(i))).forEach((dir) -> {
+      try {
+        FileUtils.createDirectories(dir);
+        directories.add(dir);
+      } catch (IOException e) {
+        throw new RuntimeException(e);
+      }
+    });
+
+    directories.stream().filter((dir) -> Integer.parseInt(dir.getName()) != 6)
+        .forEach(
+            (dir) -> {
+              try {
+                FileUtils.createDirectories(
+                    new File(dir, UUID.randomUUID().toString()));
+              } catch (IOException e) {
+                throw new RuntimeException(e);
+              }
+            });
+    String subDir = UUID.randomUUID().toString();
+    File storageDirectory = ServerState.chooseStorageDir(directories, subDir);
+    File expected = new File(directories.get(6), subDir);
+    Assert.assertEquals(expected.getCanonicalPath(),
+        storageDirectory.getCanonicalPath());
+  }
+
+  /**
+   * Tests choosing of storage directory when only no volume is configured.
+   *
+   * @throws IOException in case of exception.
+   */
+  @Test
+  public void testChooseStorageDirWithNoVolume() {
+    try {
+      ServerState.chooseStorageDir(
+          Collections.emptyList(), UUID.randomUUID().toString());
+      Assert.fail();
+    } catch (IOException ex) {
+      String expectedErrMsg = "No storage directory found.";
+      Assert.assertEquals(expectedErrMsg, ex.getMessage());
+    }
+  }
+
+}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/incubator-ratis/blob/5c37675f/ratis-test/src/test/java/org/apache/ratis/server/simulation/TestGroupManagementWithSimulatedRpc.java
----------------------------------------------------------------------
diff --git a/ratis-test/src/test/java/org/apache/ratis/server/simulation/TestGroupManagementWithSimulatedRpc.java b/ratis-test/src/test/java/org/apache/ratis/server/simulation/TestGroupManagementWithSimulatedRpc.java
new file mode 100644
index 0000000..73fbae9
--- /dev/null
+++ b/ratis-test/src/test/java/org/apache/ratis/server/simulation/TestGroupManagementWithSimulatedRpc.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.server.simulation;
+
+import org.apache.ratis.MiniRaftCluster;
+import org.apache.ratis.server.impl.GroupManagementBaseTest;
+
+public class TestGroupManagementWithSimulatedRpc extends GroupManagementBaseTest {
+  @Override
+  public MiniRaftCluster.Factory<? extends MiniRaftCluster> getClusterFactory() {
+    return MiniRaftClusterWithSimulatedRpc.FACTORY;
+  }
+}

http://git-wip-us.apache.org/repos/asf/incubator-ratis/blob/5c37675f/ratis-test/src/test/java/org/apache/ratis/server/simulation/TestLeaderElectionWithSimulatedRpc.java
----------------------------------------------------------------------
diff --git a/ratis-test/src/test/java/org/apache/ratis/server/simulation/TestLeaderElectionWithSimulatedRpc.java b/ratis-test/src/test/java/org/apache/ratis/server/simulation/TestLeaderElectionWithSimulatedRpc.java
new file mode 100644
index 0000000..7ea273b
--- /dev/null
+++ b/ratis-test/src/test/java/org/apache/ratis/server/simulation/TestLeaderElectionWithSimulatedRpc.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.ratis.server.simulation;
+
+import org.apache.ratis.server.impl.LeaderElectionTests;
+
+public class TestLeaderElectionWithSimulatedRpc
+    extends LeaderElectionTests<MiniRaftClusterWithSimulatedRpc>
+    implements MiniRaftClusterWithSimulatedRpc.FactoryGet {
+}

http://git-wip-us.apache.org/repos/asf/incubator-ratis/blob/5c37675f/ratis-test/src/test/java/org/apache/ratis/server/simulation/TestRaftExceptionWithSimulation.java
----------------------------------------------------------------------
diff --git a/ratis-test/src/test/java/org/apache/ratis/server/simulation/TestRaftExceptionWithSimulation.java b/ratis-test/src/test/java/org/apache/ratis/server/simulation/TestRaftExceptionWithSimulation.java
new file mode 100644
index 0000000..f328ea3
--- /dev/null
+++ b/ratis-test/src/test/java/org/apache/ratis/server/simulation/TestRaftExceptionWithSimulation.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.ratis.server.simulation;
+
+import org.apache.ratis.RaftExceptionBaseTest;
+
+public class TestRaftExceptionWithSimulation
+    extends RaftExceptionBaseTest<MiniRaftClusterWithSimulatedRpc>
+    implements MiniRaftClusterWithSimulatedRpc.FactoryGet {
+}

http://git-wip-us.apache.org/repos/asf/incubator-ratis/blob/5c37675f/ratis-test/src/test/java/org/apache/ratis/server/simulation/TestRaftReconfigurationWithSimulatedRpc.java
----------------------------------------------------------------------
diff --git a/ratis-test/src/test/java/org/apache/ratis/server/simulation/TestRaftReconfigurationWithSimulatedRpc.java b/ratis-test/src/test/java/org/apache/ratis/server/simulation/TestRaftReconfigurationWithSimulatedRpc.java
new file mode 100644
index 0000000..b1bb26a
--- /dev/null
+++ b/ratis-test/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 MiniRaftClusterWithSimulatedRpc.FACTORY.newCluster(peerNum, prop);
+  }
+}

http://git-wip-us.apache.org/repos/asf/incubator-ratis/blob/5c37675f/ratis-test/src/test/java/org/apache/ratis/server/simulation/TestRaftSnapshotWithSimulatedRpc.java
----------------------------------------------------------------------
diff --git a/ratis-test/src/test/java/org/apache/ratis/server/simulation/TestRaftSnapshotWithSimulatedRpc.java b/ratis-test/src/test/java/org/apache/ratis/server/simulation/TestRaftSnapshotWithSimulatedRpc.java
new file mode 100644
index 0000000..8345872
--- /dev/null
+++ b/ratis-test/src/test/java/org/apache/ratis/server/simulation/TestRaftSnapshotWithSimulatedRpc.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 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.Factory<?> getFactory() {
+    return MiniRaftClusterWithSimulatedRpc.FACTORY;
+  }
+}

http://git-wip-us.apache.org/repos/asf/incubator-ratis/blob/5c37675f/ratis-test/src/test/java/org/apache/ratis/server/simulation/TestRaftStateMachineExceptionWithSimulatedRpc.java
----------------------------------------------------------------------
diff --git a/ratis-test/src/test/java/org/apache/ratis/server/simulation/TestRaftStateMachineExceptionWithSimulatedRpc.java b/ratis-test/src/test/java/org/apache/ratis/server/simulation/TestRaftStateMachineExceptionWithSimulatedRpc.java
new file mode 100644
index 0000000..d8ca86f
--- /dev/null
+++ b/ratis-test/src/test/java/org/apache/ratis/server/simulation/TestRaftStateMachineExceptionWithSimulatedRpc.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.ratis.server.simulation;
+
+import org.apache.ratis.server.impl.RaftStateMachineExceptionTests;
+
+public class TestRaftStateMachineExceptionWithSimulatedRpc extends
+    RaftStateMachineExceptionTests<MiniRaftClusterWithSimulatedRpc>
+    implements MiniRaftClusterWithSimulatedRpc.FactoryGet {
+}

http://git-wip-us.apache.org/repos/asf/incubator-ratis/blob/5c37675f/ratis-test/src/test/java/org/apache/ratis/server/simulation/TestRaftWithSimulatedRpc.java
----------------------------------------------------------------------
diff --git a/ratis-test/src/test/java/org/apache/ratis/server/simulation/TestRaftWithSimulatedRpc.java b/ratis-test/src/test/java/org/apache/ratis/server/simulation/TestRaftWithSimulatedRpc.java
new file mode 100644
index 0000000..391a6fa
--- /dev/null
+++ b/ratis-test/src/test/java/org/apache/ratis/server/simulation/TestRaftWithSimulatedRpc.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.ratis.server.simulation;
+
+import org.apache.ratis.RaftBasicTests;
+
+public class TestRaftWithSimulatedRpc
+    extends RaftBasicTests<MiniRaftClusterWithSimulatedRpc>
+    implements MiniRaftClusterWithSimulatedRpc.FactoryGet {
+}



[2/6] incubator-ratis git commit: RATIS-399. Move all tests to a new module.

Posted by sz...@apache.org.
http://git-wip-us.apache.org/repos/asf/incubator-ratis/blob/5c37675f/ratis-test/src/test/java/org/apache/ratis/server/simulation/TestRetryCacheWithSimulatedRpc.java
----------------------------------------------------------------------
diff --git a/ratis-test/src/test/java/org/apache/ratis/server/simulation/TestRetryCacheWithSimulatedRpc.java b/ratis-test/src/test/java/org/apache/ratis/server/simulation/TestRetryCacheWithSimulatedRpc.java
new file mode 100644
index 0000000..a088578
--- /dev/null
+++ b/ratis-test/src/test/java/org/apache/ratis/server/simulation/TestRetryCacheWithSimulatedRpc.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 java.io.IOException;
+
+import org.apache.log4j.Level;
+import org.apache.ratis.RetryCacheTests;
+import org.apache.ratis.client.RaftClient;
+import org.apache.ratis.server.impl.RaftServerImpl;
+import org.apache.ratis.util.LogUtils;
+
+public class TestRetryCacheWithSimulatedRpc extends RetryCacheTests {
+  static {
+    LogUtils.setLogLevel(RaftServerImpl.LOG, Level.DEBUG);
+    LogUtils.setLogLevel(RaftClient.LOG, Level.DEBUG);
+  }
+
+  private final MiniRaftClusterWithSimulatedRpc cluster;
+
+  public TestRetryCacheWithSimulatedRpc() throws IOException {
+    cluster = MiniRaftClusterWithSimulatedRpc.FACTORY.newCluster(
+        NUM_SERVERS, getProperties());
+  }
+
+  @Override
+  public MiniRaftClusterWithSimulatedRpc getCluster() {
+    return cluster;
+  }
+}

http://git-wip-us.apache.org/repos/asf/incubator-ratis/blob/5c37675f/ratis-test/src/test/java/org/apache/ratis/server/simulation/TestServerInformationWithSimulatedRpc.java
----------------------------------------------------------------------
diff --git a/ratis-test/src/test/java/org/apache/ratis/server/simulation/TestServerInformationWithSimulatedRpc.java b/ratis-test/src/test/java/org/apache/ratis/server/simulation/TestServerInformationWithSimulatedRpc.java
new file mode 100644
index 0000000..f7025a5
--- /dev/null
+++ b/ratis-test/src/test/java/org/apache/ratis/server/simulation/TestServerInformationWithSimulatedRpc.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.ratis.server.simulation;
+
+import org.apache.ratis.server.impl.ServerInformationBaseTest;
+
+public class TestServerInformationWithSimulatedRpc
+    extends ServerInformationBaseTest<MiniRaftClusterWithSimulatedRpc>
+    implements MiniRaftClusterWithSimulatedRpc.FactoryGet {
+}

http://git-wip-us.apache.org/repos/asf/incubator-ratis/blob/5c37675f/ratis-test/src/test/java/org/apache/ratis/server/simulation/TestServerRestartWithSimulatedRpc.java
----------------------------------------------------------------------
diff --git a/ratis-test/src/test/java/org/apache/ratis/server/simulation/TestServerRestartWithSimulatedRpc.java b/ratis-test/src/test/java/org/apache/ratis/server/simulation/TestServerRestartWithSimulatedRpc.java
new file mode 100644
index 0000000..306e5e7
--- /dev/null
+++ b/ratis-test/src/test/java/org/apache/ratis/server/simulation/TestServerRestartWithSimulatedRpc.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.ratis.server.simulation;
+
+import org.apache.ratis.server.ServerRestartTests;
+
+public class TestServerRestartWithSimulatedRpc
+    extends ServerRestartTests<MiniRaftClusterWithSimulatedRpc>
+    implements MiniRaftClusterWithSimulatedRpc.FactoryGet {
+}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/incubator-ratis/blob/5c37675f/ratis-test/src/test/java/org/apache/ratis/server/storage/TestCacheEviction.java
----------------------------------------------------------------------
diff --git a/ratis-test/src/test/java/org/apache/ratis/server/storage/TestCacheEviction.java b/ratis-test/src/test/java/org/apache/ratis/server/storage/TestCacheEviction.java
new file mode 100644
index 0000000..1cd41a5
--- /dev/null
+++ b/ratis-test/src/test/java/org/apache/ratis/server/storage/TestCacheEviction.java
@@ -0,0 +1,198 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS 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.BaseTest;
+import org.apache.ratis.MiniRaftCluster;
+import org.apache.ratis.RaftTestUtil.SimpleOperation;
+import org.apache.ratis.conf.RaftProperties;
+import org.apache.ratis.protocol.RaftPeerId;
+import org.apache.ratis.server.RaftServerConfigKeys;
+import org.apache.ratis.server.impl.RaftServerConstants;
+import org.apache.ratis.server.impl.RaftServerImpl;
+import org.apache.ratis.server.impl.ServerProtoUtils;
+import org.apache.ratis.server.impl.ServerState;
+import org.apache.ratis.server.storage.CacheInvalidationPolicy.CacheInvalidationPolicyDefault;
+import org.apache.ratis.server.storage.TestSegmentedRaftLog.SegmentRange;
+import org.apache.ratis.proto.RaftProtos.LogEntryProto;
+import org.apache.ratis.statemachine.SimpleStateMachine4Testing;
+import org.apache.ratis.statemachine.StateMachine;
+import org.apache.ratis.util.SizeInBytes;
+import org.junit.Assert;
+import org.junit.Test;
+import org.mockito.Mockito;
+
+import java.io.File;
+import java.util.ArrayList;
+import java.util.Collections;
+import java.util.List;
+import java.util.concurrent.CompletableFuture;
+
+public class TestCacheEviction extends BaseTest {
+  private static final CacheInvalidationPolicy policy = new CacheInvalidationPolicyDefault();
+
+  private List<LogSegment> prepareSegments(int numSegments, boolean[] cached, long start, long size) {
+    Assert.assertEquals(numSegments, cached.length);
+    List<LogSegment> segments = new ArrayList<>(numSegments);
+    for (int i = 0; i < numSegments; i++) {
+      LogSegment s = LogSegment.newCloseSegment(null, start, start + size - 1);
+      if (cached[i]) {
+        s = Mockito.spy(s);
+        Mockito.when(s.hasCache()).thenReturn(true);
+      }
+      segments.add(s);
+      start += size;
+    }
+    return segments;
+  }
+
+  @Test
+  public void testBasicEviction() throws Exception {
+    final int maxCached = 5;
+    List<LogSegment> segments = prepareSegments(5,
+        new boolean[]{true, true, true, true, true}, 0, 10);
+
+    // case 1, make sure we do not evict cache for segments behind local flushed index
+    List<LogSegment> evicted = policy.evict(null, 5, 15, segments, maxCached);
+    Assert.assertEquals(0, evicted.size());
+
+    // case 2, suppose the local flushed index is in the 3rd segment, then we
+    // can evict the first two segment
+    evicted = policy.evict(null, 25, 30, segments, maxCached);
+    Assert.assertEquals(2, evicted.size());
+    Assert.assertSame(evicted.get(0), segments.get(0));
+    Assert.assertSame(evicted.get(1), segments.get(1));
+
+    // case 3, similar with case 2, but the local applied index is less than
+    // the local flushed index.
+    evicted = policy.evict(null, 25, 15, segments, maxCached);
+    Assert.assertEquals(1, evicted.size());
+    Assert.assertSame(evicted.get(0), segments.get(0));
+
+    // case 4, the local applied index is very small, then evict cache behind it
+    // first and let the state machine load the segments later
+    evicted = policy.evict(null, 35, 5, segments, maxCached);
+    Assert.assertEquals(1, evicted.size());
+    Assert.assertSame(evicted.get(0), segments.get(2));
+
+    Mockito.when(segments.get(2).hasCache()).thenReturn(false);
+    evicted = policy.evict(null, 35, 5, segments, maxCached);
+    Assert.assertEquals(1, evicted.size());
+    Assert.assertSame(evicted.get(0), segments.get(1));
+
+    Mockito.when(segments.get(1).hasCache()).thenReturn(false);
+    evicted = policy.evict(null, 35, 5, segments, maxCached);
+    Assert.assertEquals(0, evicted.size());
+  }
+
+  @Test
+  public void testEvictionWithFollowerIndices() throws Exception {
+    final int maxCached = 6;
+    List<LogSegment> segments = prepareSegments(6,
+        new boolean[]{true, true, true, true, true, true}, 0, 10);
+
+    // case 1, no matter where the followers are, we do not evict segments behind local
+    // flushed index
+    List<LogSegment> evicted = policy.evict(new long[]{20, 40, 40}, 5, 15, segments,
+        maxCached);
+    Assert.assertEquals(0, evicted.size());
+
+    // case 2, the follower indices are behind the local flushed index
+    evicted = policy.evict(new long[]{30, 40, 45}, 25, 30, segments, maxCached);
+    Assert.assertEquals(2, evicted.size());
+    Assert.assertSame(evicted.get(0), segments.get(0));
+    Assert.assertSame(evicted.get(1), segments.get(1));
+
+    // case 3, similar with case 3 in basic eviction test
+    evicted = policy.evict(new long[]{30, 40, 45}, 25, 15, segments, maxCached);
+    Assert.assertEquals(1, evicted.size());
+    Assert.assertSame(evicted.get(0), segments.get(0));
+
+    // case 4, the followers are slower than local flush
+    evicted = policy.evict(new long[]{15, 45, 45}, 55, 50, segments, maxCached);
+    Assert.assertEquals(1, evicted.size());
+    Assert.assertSame(evicted.get(0), segments.get(0));
+
+    Mockito.when(segments.get(0).hasCache()).thenReturn(false);
+    evicted = policy.evict(new long[]{15, 45, 45}, 55, 50, segments, maxCached);
+    Assert.assertEquals(1, evicted.size());
+    Assert.assertSame(evicted.get(0), segments.get(2));
+
+    Mockito.when(segments.get(2).hasCache()).thenReturn(false);
+    evicted = policy.evict(new long[]{15, 45, 45}, 55, 50, segments, maxCached);
+    Assert.assertEquals(1, evicted.size());
+    Assert.assertSame(evicted.get(0), segments.get(3));
+
+    Mockito.when(segments.get(3).hasCache()).thenReturn(false);
+    evicted = policy.evict(new long[]{15, 45, 45}, 55, 50, segments, maxCached);
+    Assert.assertEquals(0, evicted.size());
+  }
+
+  @Test
+  public void testEvictionInSegmentedLog() throws Exception {
+    final RaftProperties prop = new RaftProperties();
+    prop.setClass(MiniRaftCluster.STATEMACHINE_CLASS_KEY,
+        SimpleStateMachine4Testing.class, StateMachine.class);
+    RaftServerConfigKeys.Log.setSegmentSizeMax(prop, SizeInBytes.valueOf("8KB"));
+    RaftServerConfigKeys.Log.setPreallocatedSize(prop, SizeInBytes.valueOf("8KB"));
+    final RaftPeerId peerId = RaftPeerId.valueOf("s0");
+    final int maxCachedNum = RaftServerConfigKeys.Log.maxCachedSegmentNum(prop);
+
+    File storageDir = getTestDir();
+    RaftServerConfigKeys.setStorageDirs(prop,  Collections.singletonList(storageDir));
+    RaftStorage storage = new RaftStorage(storageDir, RaftServerConstants.StartupOption.REGULAR);
+
+    RaftServerImpl server = Mockito.mock(RaftServerImpl.class);
+    ServerState state = Mockito.mock(ServerState.class);
+    Mockito.when(server.getState()).thenReturn(state);
+    Mockito.when(server.getFollowerNextIndices()).thenReturn(new long[]{});
+    Mockito.when(state.getLastAppliedIndex()).thenReturn(0L);
+
+    SegmentedRaftLog raftLog = new SegmentedRaftLog(peerId, server, storage, -1, prop);
+    raftLog.open(RaftServerConstants.INVALID_LOG_INDEX, null);
+    List<SegmentRange> slist = TestSegmentedRaftLog.prepareRanges(0, maxCachedNum, 7, 0);
+    LogEntryProto[] entries = generateEntries(slist);
+    raftLog.append(entries).forEach(CompletableFuture::join);
+
+    // check the current cached segment number: the last segment is still open
+    Assert.assertEquals(maxCachedNum - 1,
+        raftLog.getRaftLogCache().getCachedSegmentNum());
+
+    Mockito.when(server.getFollowerNextIndices()).thenReturn(new long[]{21, 40, 40});
+    Mockito.when(state.getLastAppliedIndex()).thenReturn(35L);
+    slist = TestSegmentedRaftLog.prepareRanges(maxCachedNum, maxCachedNum + 2, 7, 7 * maxCachedNum);
+    entries = generateEntries(slist);
+    raftLog.append(entries).forEach(CompletableFuture::join);
+
+    // check the cached segment number again. since the slowest follower is on
+    // index 21, the eviction should happen and evict 3 segments
+    Assert.assertEquals(maxCachedNum + 1 - 3,
+        raftLog.getRaftLogCache().getCachedSegmentNum());
+  }
+
+  private LogEntryProto[] generateEntries(List<SegmentRange> slist) {
+    List<LogEntryProto> eList = new ArrayList<>();
+    for (SegmentRange range : slist) {
+      for (long index = range.start; index <= range.end; index++) {
+        SimpleOperation m = new SimpleOperation(new String(new byte[1024]));
+        eList.add(ServerProtoUtils.toLogEntryProto(m.getLogEntryContent(), range.term, index));
+      }
+    }
+    return eList.toArray(new LogEntryProto[eList.size()]);
+  }
+}

http://git-wip-us.apache.org/repos/asf/incubator-ratis/blob/5c37675f/ratis-test/src/test/java/org/apache/ratis/server/storage/TestRaftLogCache.java
----------------------------------------------------------------------
diff --git a/ratis-test/src/test/java/org/apache/ratis/server/storage/TestRaftLogCache.java b/ratis-test/src/test/java/org/apache/ratis/server/storage/TestRaftLogCache.java
new file mode 100644
index 0000000..d3c216d
--- /dev/null
+++ b/ratis-test/src/test/java/org/apache/ratis/server/storage/TestRaftLogCache.java
@@ -0,0 +1,257 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS 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.util.Iterator;
+
+import org.apache.ratis.RaftTestUtil.SimpleOperation;
+import org.apache.ratis.conf.RaftProperties;
+import org.apache.ratis.server.impl.ServerProtoUtils;
+import org.apache.ratis.server.protocol.TermIndex;
+import org.apache.ratis.server.storage.RaftLogCache.TruncationSegments;
+import org.apache.ratis.proto.RaftProtos.LogEntryProto;
+import org.junit.Assert;
+import org.junit.Before;
+import org.junit.Test;
+
+public class TestRaftLogCache {
+  private static final RaftProperties prop = new RaftProperties();
+
+  private RaftLogCache cache;
+
+  @Before
+  public void setup() {
+    cache = new RaftLogCache(null, null, prop);
+  }
+
+  private LogSegment prepareLogSegment(long start, long end, boolean isOpen) {
+    LogSegment s = LogSegment.newOpenSegment(null, start);
+    for (long i = start; i <= end; i++) {
+      SimpleOperation m = new SimpleOperation("m" + i);
+      LogEntryProto entry = ServerProtoUtils.toLogEntryProto(m.getLogEntryContent(), 0, i);
+      s.appendToOpenSegment(entry);
+    }
+    if (!isOpen) {
+      s.close();
+    }
+    return s;
+  }
+
+  private void checkCache(long start, long end, int segmentSize) throws IOException {
+    Assert.assertEquals(start, cache.getStartIndex());
+    Assert.assertEquals(end, cache.getEndIndex());
+
+    for (long index = start; index <= end; index++) {
+      LogEntryProto entry = cache.getSegment(index).getEntryWithoutLoading(index).getEntry();
+      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) {
+    TermIndex[] entries = cache.getTermIndices(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 = ServerProtoUtils.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 = ServerProtoUtils.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) throws IOException {
+    Iterator<TermIndex> iterator = cache.iterator(startIndex);
+    TermIndex prev = null;
+    while (iterator.hasNext()) {
+      TermIndex termIndex = iterator.next();
+      Assert.assertEquals(cache.getLogRecord(termIndex.getIndex()).getTermIndex(), termIndex);
+      if (prev != null) {
+        Assert.assertEquals(prev.getIndex() + 1, termIndex.getIndex());
+      }
+      prev = termIndex;
+    }
+    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<TermIndex> iterator = cache.iterator(300);
+    Assert.assertFalse(iterator.hasNext());
+  }
+}

http://git-wip-us.apache.org/repos/asf/incubator-ratis/blob/5c37675f/ratis-test/src/test/java/org/apache/ratis/server/storage/TestRaftLogReadWrite.java
----------------------------------------------------------------------
diff --git a/ratis-test/src/test/java/org/apache/ratis/server/storage/TestRaftLogReadWrite.java b/ratis-test/src/test/java/org/apache/ratis/server/storage/TestRaftLogReadWrite.java
new file mode 100644
index 0000000..7d9fdf5
--- /dev/null
+++ b/ratis-test/src/test/java/org/apache/ratis/server/storage/TestRaftLogReadWrite.java
@@ -0,0 +1,272 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS 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.BaseTest;
+import org.apache.ratis.RaftTestUtil.SimpleOperation;
+import org.apache.ratis.conf.RaftProperties;
+import org.apache.ratis.protocol.ChecksumException;
+import org.apache.ratis.server.RaftServerConfigKeys;
+import org.apache.ratis.server.impl.RaftServerConstants;
+import org.apache.ratis.server.impl.RaftServerConstants.StartupOption;
+import org.apache.ratis.server.impl.ServerProtoUtils;
+import org.apache.ratis.thirdparty.com.google.protobuf.CodedOutputStream;
+import org.apache.ratis.proto.RaftProtos.LogEntryProto;
+import org.apache.ratis.util.FileUtils;
+import org.junit.After;
+import org.junit.Assert;
+import org.junit.Before;
+import org.junit.Test;
+
+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.Collections;
+import java.util.List;
+
+/**
+ * Test basic functionality of LogReader, LogInputStream, and LogOutputStream.
+ */
+public class TestRaftLogReadWrite extends BaseTest {
+  private File storageDir;
+  private long segmentMaxSize;
+  private long preallocatedSize;
+  private int bufferSize;
+
+  @Before
+  public void setup() throws Exception {
+    storageDir = getTestDir();
+    RaftProperties properties = new RaftProperties();
+    RaftServerConfigKeys.setStorageDirs(properties,  Collections.singletonList(storageDir));
+    this.segmentMaxSize =
+        RaftServerConfigKeys.Log.segmentSizeMax(properties).getSize();
+    this.preallocatedSize =
+        RaftServerConfigKeys.Log.preallocatedSize(properties).getSize();
+    this.bufferSize =
+        RaftServerConfigKeys.Log.writeBufferSize(properties).getSizeInt();
+  }
+
+  @After
+  public void tearDown() throws Exception {
+    if (storageDir != null) {
+      FileUtils.deleteFully(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] = ServerProtoUtils.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 {
+    final RaftStorage storage = new RaftStorage(storageDir, StartupOption.REGULAR);
+    File openSegment = storage.getStorageDir().getOpenLogFile(0);
+    long size = SegmentedRaftLogFormat.getHeaderLength();
+
+    final LogEntryProto[] entries = new LogEntryProto[100];
+    try (LogOutputStream out =
+             new LogOutputStream(openSegment, false, segmentMaxSize,
+                 preallocatedSize, bufferSize)) {
+      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 {
+    final RaftStorage storage = new RaftStorage(storageDir, StartupOption.REGULAR);
+    File openSegment = storage.getStorageDir().getOpenLogFile(0);
+    LogEntryProto[] entries = new LogEntryProto[200];
+    try (LogOutputStream out =
+             new LogOutputStream(openSegment, false, segmentMaxSize,
+                 preallocatedSize, bufferSize)) {
+      for (int i = 0; i < 100; i++) {
+        SimpleOperation m = new SimpleOperation("m" + i);
+        entries[i] = ServerProtoUtils.toLogEntryProto(m.getLogEntryContent(), 0, i);
+        out.write(entries[i]);
+      }
+    }
+
+    try (LogOutputStream out =
+             new LogOutputStream(openSegment, true, segmentMaxSize,
+                 preallocatedSize, bufferSize)) {
+      for (int i = 100; i < 200; i++) {
+        SimpleOperation m = new SimpleOperation("m" + i);
+        entries[i] = ServerProtoUtils.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 {
+    final RaftStorage storage = new RaftStorage(storageDir, StartupOption.REGULAR);
+    File openSegment = storage.getStorageDir().getOpenLogFile(0);
+    long size = SegmentedRaftLogFormat.getHeaderLength();
+
+    LogEntryProto[] entries = new LogEntryProto[100];
+    LogOutputStream out = new LogOutputStream(openSegment, false,
+        segmentMaxSize, preallocatedSize, bufferSize);
+    size += writeMessages(entries, out);
+    out.flush();
+
+    // make sure the file contains padding
+    Assert.assertEquals(
+        RaftServerConfigKeys.Log.PREALLOCATED_SIZE_DEFAULT.getSize(),
+        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 {
+    final RaftStorage storage = new RaftStorage(storageDir, StartupOption.REGULAR);
+    File openSegment = storage.getStorageDir().getOpenLogFile(0);
+
+    LogEntryProto[] entries = new LogEntryProto[10];
+    LogOutputStream out = new LogOutputStream(openSegment, false,
+        16 * 1024 * 1024, 4 * 1024 * 1024, bufferSize);
+    for (int i = 0; i < 10; i++) {
+      SimpleOperation m = new SimpleOperation("m" + i);
+      entries[i] = ServerProtoUtils.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(storageDir, StartupOption.REGULAR);
+    File openSegment = storage.getStorageDir().getOpenLogFile(0);
+    try (LogOutputStream out =
+             new LogOutputStream(openSegment, false, segmentMaxSize,
+                 preallocatedSize, bufferSize)) {
+      for (int i = 0; i < 100; i++) {
+        LogEntryProto entry = ServerProtoUtils.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/5c37675f/ratis-test/src/test/java/org/apache/ratis/server/storage/TestRaftLogSegment.java
----------------------------------------------------------------------
diff --git a/ratis-test/src/test/java/org/apache/ratis/server/storage/TestRaftLogSegment.java b/ratis-test/src/test/java/org/apache/ratis/server/storage/TestRaftLogSegment.java
new file mode 100644
index 0000000..270e279
--- /dev/null
+++ b/ratis-test/src/test/java/org/apache/ratis/server/storage/TestRaftLogSegment.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 org.apache.ratis.BaseTest;
+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.server.impl.ServerProtoUtils;
+import org.apache.ratis.server.storage.LogSegment.LogRecordWithEntry;
+import org.apache.ratis.proto.RaftProtos.LogEntryProto;
+import org.apache.ratis.proto.RaftProtos.StateMachineLogEntryProto;
+import org.apache.ratis.thirdparty.com.google.protobuf.CodedOutputStream;
+import org.apache.ratis.util.FileUtils;
+import org.apache.ratis.util.Preconditions;
+import org.apache.ratis.util.SizeInBytes;
+import org.apache.ratis.util.TraditionalBinaryPrefix;
+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.nio.file.Files;
+import java.nio.file.Path;
+import java.util.ArrayList;
+import java.util.Arrays;
+import java.util.Collections;
+import java.util.List;
+import java.util.concurrent.ThreadLocalRandom;
+
+import static org.apache.ratis.server.impl.RaftServerConstants.INVALID_LOG_INDEX;
+import static org.apache.ratis.server.storage.LogSegment.getEntrySize;
+
+/**
+ * Test basic functionality of {@link LogSegment}
+ */
+public class TestRaftLogSegment extends BaseTest {
+  private File storageDir;
+  private long segmentMaxSize;
+  private long preallocatedSize;
+  private int bufferSize;
+
+  @Before
+  public void setup() throws Exception {
+    RaftProperties properties = new RaftProperties();
+    storageDir = getTestDir();
+    RaftServerConfigKeys.setStorageDirs(properties,  Collections.singletonList(storageDir));
+    this.segmentMaxSize =
+        RaftServerConfigKeys.Log.segmentSizeMax(properties).getSize();
+    this.preallocatedSize =
+        RaftServerConfigKeys.Log.preallocatedSize(properties).getSize();
+    this.bufferSize =
+        RaftServerConfigKeys.Log.writeBufferSize(properties).getSizeInt();
+  }
+
+  @After
+  public void tearDown() throws Exception {
+    if (storageDir != null) {
+      FileUtils.deleteFully(storageDir.getParentFile());
+    }
+  }
+
+  File prepareLog(boolean isOpen, long startIndex, int numEntries, long term, boolean isLastEntryPartiallyWritten)
+      throws IOException {
+    if (!isOpen) {
+      Preconditions.assertTrue(!isLastEntryPartiallyWritten, "For closed log, the last entry cannot be partially written.");
+    }
+    RaftStorage storage = new RaftStorage(storageDir, StartupOption.REGULAR);
+    final File file = isOpen ?
+        storage.getStorageDir().getOpenLogFile(startIndex) :
+        storage.getStorageDir().getClosedLogFile(startIndex, startIndex + numEntries - 1);
+
+    final LogEntryProto[] entries = new LogEntryProto[numEntries];
+    try (LogOutputStream out = new LogOutputStream(file, false,
+        segmentMaxSize, preallocatedSize, bufferSize)) {
+      for (int i = 0; i < entries.length; i++) {
+        SimpleOperation op = new SimpleOperation("m" + i);
+        entries[i] = ServerProtoUtils.toLogEntryProto(op.getLogEntryContent(), term, i + startIndex);
+        out.write(entries[i]);
+      }
+    }
+
+    if (isLastEntryPartiallyWritten) {
+      final int entrySize = size(entries[entries.length - 1]);
+      final int truncatedEntrySize = ThreadLocalRandom.current().nextInt(entrySize - 1) + 1;
+      // 0 < truncatedEntrySize < entrySize
+      final long fileLength = file.length();
+      final long truncatedFileLength = fileLength - (entrySize - truncatedEntrySize);
+      LOG.info("truncate last entry: entry(size={}, truncated={}), file(length={}, truncated={})",
+          entrySize, truncatedEntrySize, fileLength, truncatedFileLength);
+      FileUtils.truncateFile(file, truncatedFileLength);
+    }
+
+    storage.close();
+    return file;
+  }
+
+  static int size(LogEntryProto entry) {
+    final int n = entry.getSerializedSize();
+    return CodedOutputStream.computeUInt32SizeNoTag(n) + n + 4;
+  }
+
+  static void checkLogSegment(LogSegment segment, long start, long end,
+      boolean isOpen, long totalSize, long term) throws Exception {
+    Assert.assertEquals(start, segment.getStartIndex());
+    Assert.assertEquals(end, segment.getEndIndex());
+    Assert.assertEquals(isOpen, segment.isOpen());
+    Assert.assertEquals(totalSize, segment.getTotalSize());
+
+    long offset = SegmentedRaftLogFormat.getHeaderLength();
+    for (long i = start; i <= end; i++) {
+      LogSegment.LogRecord record = segment.getLogRecord(i);
+      LogRecordWithEntry lre = segment.getEntryWithoutLoading(i);
+      Assert.assertEquals(i, lre.getRecord().getTermIndex().getIndex());
+      Assert.assertEquals(term, lre.getRecord().getTermIndex().getTerm());
+      Assert.assertEquals(offset, record.getOffset());
+
+      LogEntryProto entry = lre.hasEntry() ?
+          lre.getEntry() : segment.loadCache(lre.getRecord());
+      offset += getEntrySize(entry);
+    }
+  }
+
+  @Test
+  public void testLoadLogSegment() throws Exception {
+    testLoadSegment(true, false);
+  }
+
+  @Test
+  public void testLoadLogSegmentLastEntryPartiallyWritten() throws Exception {
+    testLoadSegment(true, true);
+  }
+
+  @Test
+  public void testLoadCache() throws Exception {
+    testLoadSegment(false, false);
+  }
+
+  @Test
+  public void testLoadCacheLastEntryPartiallyWritten() throws Exception {
+    testLoadSegment(false, true);
+  }
+
+  private void testLoadSegment(boolean loadInitial, boolean isLastEntryPartiallyWritten) throws Exception {
+    // load an open segment
+    final File openSegmentFile = prepareLog(true, 0, 100, 0, isLastEntryPartiallyWritten);
+    RaftStorage storage = new RaftStorage(storageDir, StartupOption.REGULAR);
+    LogSegment openSegment = LogSegment.loadSegment(storage, openSegmentFile, 0,
+        INVALID_LOG_INDEX, true, loadInitial, null);
+    final int delta = isLastEntryPartiallyWritten? 1: 0;
+    checkLogSegment(openSegment, 0, 99 - delta, true, openSegmentFile.length(), 0);
+    storage.close();
+    // for open segment we currently always keep log entries in the memory
+    Assert.assertEquals(0, openSegment.getLoadingTimes());
+
+    // load a closed segment (1000-1099)
+    final File closedSegmentFile = prepareLog(false, 1000, 100, 1, false);
+    LogSegment closedSegment = LogSegment.loadSegment(storage, closedSegmentFile,
+        1000, 1099, false, loadInitial, null);
+    checkLogSegment(closedSegment, 1000, 1099, false,
+        closedSegment.getTotalSize(), 1);
+    Assert.assertEquals(loadInitial ? 0 : 1, closedSegment.getLoadingTimes());
+  }
+
+  @Test
+  public void testAppendEntries() throws Exception {
+    final long start = 1000;
+    LogSegment segment = LogSegment.newOpenSegment(null, start);
+    long size = SegmentedRaftLogFormat.getHeaderLength();
+    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 = ServerProtoUtils.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(null, 1000);
+    SimpleOperation op = new SimpleOperation("m");
+    final StateMachineLogEntryProto m = op.getLogEntryContent();
+    try {
+      LogEntryProto entry = ServerProtoUtils.toLogEntryProto(m, 0, 1001);
+      segment.appendToOpenSegment(entry);
+      Assert.fail("should fail since the entry's index needs to be 1000");
+    } catch (IllegalStateException e) {
+      // the exception is expected.
+    }
+
+    LogEntryProto entry = ServerProtoUtils.toLogEntryProto(m, 0, 1000);
+    segment.appendToOpenSegment(entry);
+
+    try {
+      entry = ServerProtoUtils.toLogEntryProto(m, 0, 1002);
+      segment.appendToOpenSegment(entry);
+      Assert.fail("should fail since the entry's index needs to be 1001");
+    } catch (IllegalStateException e) {
+      // the exception is expected.
+    }
+
+    LogEntryProto[] entries = new LogEntryProto[2];
+    for (int i = 0; i < 2; i++) {
+      entries[i] = ServerProtoUtils.toLogEntryProto(m, 0, 1001 + i * 2);
+    }
+    try {
+      segment.appendToOpenSegment(entries);
+      Assert.fail("should fail since there is gap between entries");
+    } catch (IllegalStateException e) {
+      // the exception is expected.
+    }
+  }
+
+  @Test
+  public void testTruncate() throws Exception {
+    final long term = 1;
+    final long start = 1000;
+    LogSegment segment = LogSegment.newOpenSegment(null, start);
+    for (int i = 0; i < 100; i++) {
+      LogEntryProto entry = ServerProtoUtils.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).getOffset();
+    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).getOffset();
+    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,
+        SegmentedRaftLogFormat.getHeaderLength(), term);
+  }
+
+  @Test
+  public void testPreallocateSegment() throws Exception {
+    RaftStorage storage = new RaftStorage(storageDir, 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, max, a, bufferSize)) {
+          Assert.assertEquals("max=" + max + ", a=" + a, 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,
+        1024, 1024, bufferSize)) {
+      SimpleOperation op = new SimpleOperation(new String(content));
+      LogEntryProto entry = ServerProtoUtils.toLogEntryProto(op.getLogEntryContent(), 0, 0);
+      size = LogSegment.getEntrySize(entry);
+      out.write(entry);
+    }
+    Assert.assertEquals(file.length(),
+        size + SegmentedRaftLogFormat.getHeaderLength());
+    try (LogInputStream in = new LogInputStream(file, 0,
+        INVALID_LOG_INDEX, true)) {
+      LogEntryProto entry = in.nextEntry();
+      Assert.assertArrayEquals(content,
+          entry.getStateMachineLogEntry().getLogData().toByteArray());
+      Assert.assertNull(in.nextEntry());
+    }
+  }
+
+  /**
+   * Keep appending and check if pre-allocation is correct
+   */
+  @Test
+  public void testPreallocationAndAppend() throws Exception {
+    final SizeInBytes max = SizeInBytes.valueOf(2, TraditionalBinaryPrefix.MEGA);
+    RaftStorage storage = new RaftStorage(storageDir, 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 = ServerProtoUtils.toLogEntryProto(op.getLogEntryContent(), 0, 0);
+    final long entrySize = LogSegment.getEntrySize(entry);
+
+    long totalSize = SegmentedRaftLogFormat.getHeaderLength();
+    long preallocated = 16 * 1024;
+    try (LogOutputStream out = new LogOutputStream(file, false,
+        max.getSize(), 16 * 1024, 10 * 1024)) {
+      Assert.assertEquals(preallocated, file.length());
+      while (totalSize + entrySize < max.getSize()) {
+        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());
+  }
+
+  @Test
+  public void testZeroSizeInProgressFile() throws Exception {
+    final RaftStorage storage = new RaftStorage(storageDir, StartupOption.REGULAR);
+    final File file = storage.getStorageDir().getOpenLogFile(0);
+    storage.close();
+
+    // create zero size in-progress file
+    LOG.info("file: " + file);
+    Assert.assertTrue(file.createNewFile());
+    final Path path = file.toPath();
+    Assert.assertTrue(Files.exists(path));
+    Assert.assertEquals(0, Files.size(path));
+
+    // getLogSegmentFiles should remove it.
+    final List<RaftStorageDirectory.LogPathAndIndex> logs = storage.getStorageDir().getLogSegmentFiles();
+    Assert.assertEquals(0, logs.size());
+    Assert.assertFalse(Files.exists(path));
+  }
+}

http://git-wip-us.apache.org/repos/asf/incubator-ratis/blob/5c37675f/ratis-test/src/test/java/org/apache/ratis/server/storage/TestRaftStorage.java
----------------------------------------------------------------------
diff --git a/ratis-test/src/test/java/org/apache/ratis/server/storage/TestRaftStorage.java b/ratis-test/src/test/java/org/apache/ratis/server/storage/TestRaftStorage.java
new file mode 100644
index 0000000..4a26f8c
--- /dev/null
+++ b/ratis-test/src/test/java/org/apache/ratis/server/storage/TestRaftStorage.java
@@ -0,0 +1,206 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS 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.BaseTest;
+import org.apache.ratis.server.impl.RaftServerConstants.StartupOption;
+import org.apache.ratis.server.protocol.TermIndex;
+import org.apache.ratis.server.storage.RaftStorageDirectory.StorageState;
+import org.apache.ratis.statemachine.impl.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 extends BaseTest {
+  private File storageDir;
+
+  @Before
+  public void setup() throws Exception {
+    storageDir = getTestDir();
+  }
+
+  @After
+  public void tearDown() throws Exception {
+    if (storageDir != null) {
+      FileUtils.deleteFully(storageDir.getParentFile());
+    }
+  }
+
+  @Test
+  public void testNotExistent() throws IOException {
+    FileUtils.deleteFully(storageDir);
+
+    // we will format the empty directory
+    RaftStorage storage = new RaftStorage(storageDir, StartupOption.REGULAR);
+    Assert.assertEquals(StorageState.NORMAL, storage.getState());
+
+    try {
+      new RaftStorage(storageDir, 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.deleteFully(storageDir);
+    Assert.assertTrue(storageDir.createNewFile());
+    try {
+      new RaftStorage(storageDir, 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(storageDir, 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(storageDir, 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(storageDir, 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(storageDir, StartupOption.REGULAR);
+    Assert.assertEquals(StorageState.NORMAL, storage.getState());
+    storage.close();
+
+    RaftStorageDirectory sd = new RaftStorageDirectory(storageDir);
+    File metaFile = sd.getMetaFile();
+    FileUtils.move(metaFile, sd.getMetaTmpFile());
+
+    Assert.assertEquals(StorageState.NOT_FORMATTED, sd.analyzeStorage(false));
+
+    try {
+      new RaftStorage(storageDir, 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(storageDir, StartupOption.FORMAT).close();
+    Assert.assertTrue(sd.getMetaFile().exists());
+    Assert.assertTrue(sd.getMetaTmpFile().createNewFile());
+    Assert.assertTrue(sd.getMetaTmpFile().exists());
+    try {
+      storage = new RaftStorage(storageDir, 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/5c37675f/ratis-test/src/test/java/org/apache/ratis/server/storage/TestSegmentedRaftLog.java
----------------------------------------------------------------------
diff --git a/ratis-test/src/test/java/org/apache/ratis/server/storage/TestSegmentedRaftLog.java b/ratis-test/src/test/java/org/apache/ratis/server/storage/TestSegmentedRaftLog.java
new file mode 100644
index 0000000..bcbfa73
--- /dev/null
+++ b/ratis-test/src/test/java/org/apache/ratis/server/storage/TestSegmentedRaftLog.java
@@ -0,0 +1,524 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS 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.log4j.Level;
+import org.apache.ratis.BaseTest;
+import org.apache.ratis.RaftTestUtil.SimpleOperation;
+import org.apache.ratis.conf.RaftProperties;
+import org.apache.ratis.protocol.RaftPeerId;
+import org.apache.ratis.protocol.TimeoutIOException;
+import org.apache.ratis.server.RaftServerConfigKeys;
+import org.apache.ratis.server.impl.RaftServerConstants;
+import org.apache.ratis.server.impl.RetryCacheTestUtil;
+import org.apache.ratis.server.impl.RetryCache;
+import org.apache.ratis.server.impl.RaftServerImpl;
+import org.apache.ratis.server.impl.ServerProtoUtils;
+import org.apache.ratis.server.protocol.TermIndex;
+import org.apache.ratis.proto.RaftProtos.LogEntryProto;
+import org.apache.ratis.statemachine.SimpleStateMachine4Testing;
+import org.apache.ratis.statemachine.StateMachine;
+import org.apache.ratis.statemachine.impl.BaseStateMachine;
+import org.apache.ratis.util.ExitUtils;
+import org.apache.ratis.util.FileUtils;
+import org.apache.ratis.util.JavaUtils;
+import org.apache.ratis.util.LogUtils;
+import org.apache.ratis.util.SizeInBytes;
+import org.apache.ratis.util.TimeDuration;
+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.nio.charset.StandardCharsets;
+import java.util.ArrayList;
+import java.util.Arrays;
+import java.util.Collections;
+import java.util.List;
+import java.util.Objects;
+import java.util.concurrent.CompletableFuture;
+import java.util.concurrent.TimeUnit;
+import java.util.function.Supplier;
+
+import static org.mockito.Matchers.any;
+import static org.mockito.Mockito.doCallRealMethod;
+import static org.mockito.Mockito.mock;
+import static org.mockito.Mockito.when;
+
+public class TestSegmentedRaftLog extends BaseTest {
+  static {
+    LogUtils.setLogLevel(RaftLogWorker.LOG, Level.DEBUG);
+  }
+
+  private static final RaftPeerId peerId = RaftPeerId.valueOf("s0");
+
+  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 long segmentMaxSize;
+  private long preallocatedSize;
+  private int bufferSize;
+
+  @Before
+  public void setup() throws Exception {
+    storageDir = getTestDir();
+    properties = new RaftProperties();
+    RaftServerConfigKeys.setStorageDirs(properties,  Collections.singletonList(storageDir));
+    storage = new RaftStorage(storageDir, RaftServerConstants.StartupOption.REGULAR);
+    this.segmentMaxSize =
+        RaftServerConfigKeys.Log.segmentSizeMax(properties).getSize();
+    this.preallocatedSize =
+        RaftServerConfigKeys.Log.preallocatedSize(properties).getSize();
+    this.bufferSize =
+        RaftServerConfigKeys.Log.writeBufferSize(properties).getSizeInt();
+  }
+
+  @After
+  public void tearDown() throws Exception {
+    if (storageDir != null) {
+      FileUtils.deleteFully(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,
+          segmentMaxSize, preallocatedSize, bufferSize)) {
+        for (int i = 0; i < size; i++) {
+          SimpleOperation m = new SimpleOperation("m" + (i + range.start));
+          entries[i] = ServerProtoUtils.toLogEntryProto(m.getLogEntryContent(), range.term, i + range.start);
+          out.write(entries[i]);
+        }
+      }
+      Collections.addAll(entryList, entries);
+    }
+    return entryList.toArray(new LogEntryProto[entryList.size()]);
+  }
+
+  static List<SegmentRange> prepareRanges(int startTerm, int endTerm, int segmentSize,
+      long startIndex) {
+    List<SegmentRange> list = new ArrayList<>(endTerm - startTerm);
+    for (int i = startTerm; i < endTerm; i++) {
+      list.add(new SegmentRange(startIndex, startIndex + segmentSize - 1, i,
+          i == endTerm - 1));
+      startIndex += segmentSize;
+    }
+    return list;
+  }
+
+  private LogEntryProto getLastEntry(SegmentedRaftLog raftLog)
+      throws IOException {
+    return raftLog.get(raftLog.getLastEntryTermIndex().getIndex());
+  }
+
+  @Test
+  public void testLoadLogSegments() throws Exception {
+    // first generate log files
+    List<SegmentRange> ranges = prepareRanges(0, 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(RaftServerConstants.INVALID_LOG_INDEX, null);
+      // check if log entries are loaded correctly
+      for (LogEntryProto e : entries) {
+        LogEntryProto entry = raftLog.get(e.getIndex());
+        Assert.assertEquals(e, entry);
+      }
+
+      TermIndex[] termIndices = raftLog.getEntries(0, 500);
+      LogEntryProto[] entriesFromLog = Arrays.stream(termIndices)
+          .map(ti -> {
+            try {
+              return raftLog.get(ti.getIndex());
+            } catch (IOException e) {
+              throw new RuntimeException(e);
+            }
+          })
+          .toArray(LogEntryProto[]::new);
+      Assert.assertArrayEquals(entries, entriesFromLog);
+      Assert.assertEquals(entries[entries.length - 1], getLastEntry(raftLog));
+    }
+  }
+
+  static List<LogEntryProto> prepareLogEntries(List<SegmentRange> slist,
+      Supplier<String> stringSupplier) {
+    List<LogEntryProto> eList = new ArrayList<>();
+    for (SegmentRange range : slist) {
+      prepareLogEntries(range, stringSupplier, false, eList);
+    }
+    return eList;
+  }
+
+  static List<LogEntryProto> prepareLogEntries(SegmentRange range,
+      Supplier<String> stringSupplier, boolean hasStataMachineData, List<LogEntryProto> eList) {
+    for(long index = range.start; index <= range.end; index++) {
+      eList.add(prepareLogEntry(range.term, index, stringSupplier, hasStataMachineData));
+    }
+    return eList;
+  }
+
+  static LogEntryProto prepareLogEntry(long term, long index, Supplier<String> stringSupplier, boolean hasStataMachineData) {
+    final SimpleOperation m = stringSupplier == null?
+        new SimpleOperation("m" + index, hasStataMachineData):
+        new SimpleOperation(stringSupplier.get(), hasStataMachineData);
+    return ServerProtoUtils.toLogEntryProto(m.getLogEntryContent(), term, index);
+  }
+
+  /**
+   * Append entry one by one and check if log state is correct.
+   */
+  @Test
+  public void testAppendEntry() throws Exception {
+    List<SegmentRange> ranges = prepareRanges(0, 5, 200, 0);
+    List<LogEntryProto> entries = prepareLogEntries(ranges, null);
+
+    try (SegmentedRaftLog raftLog =
+             new SegmentedRaftLog(peerId, null, storage, -1, properties)) {
+      raftLog.open(RaftServerConstants.INVALID_LOG_INDEX, null);
+      // append entries to the raftlog
+      entries.stream().map(raftLog::appendEntry).forEach(CompletableFuture::join);
+    }
+
+    try (SegmentedRaftLog raftLog =
+             new SegmentedRaftLog(peerId, null, storage, -1, properties)) {
+      raftLog.open(RaftServerConstants.INVALID_LOG_INDEX, null);
+      // check if the raft log is correct
+      checkEntries(raftLog, entries, 0, entries.size());
+    }
+
+    try (SegmentedRaftLog raftLog =
+        new SegmentedRaftLog(peerId, null, storage, -1, properties)) {
+      raftLog.open(RaftServerConstants.INVALID_LOG_INDEX, null);
+      TermIndex lastTermIndex  = raftLog.getLastEntryTermIndex();
+      IllegalStateException ex = null;
+      try {
+        // append entry fails if append entry term is lower than log's last entry term
+        raftLog.appendEntry(LogEntryProto.newBuilder(entries.get(0))
+            .setTerm(lastTermIndex.getTerm() - 1)
+            .setIndex(lastTermIndex.getIndex() + 1).build());
+      } catch (IllegalStateException e) {
+        ex = e;
+      }
+      Assert.assertTrue(ex.getMessage().contains("term less than RaftLog's last term"));
+      try {
+        // append entry fails if difference between append entry index and log's last entry index is greater than 1
+        raftLog.appendEntry(LogEntryProto.newBuilder(entries.get(0))
+            .setTerm(lastTermIndex.getTerm())
+            .setIndex(lastTermIndex.getIndex() + 2).build());
+      } catch (IllegalStateException e) {
+        ex = e;
+      }
+      Assert.assertTrue(ex.getMessage().contains("and RaftLog's last index " + lastTermIndex.getIndex() + " greater than 1"));
+    }
+  }
+
+  /**
+   * Keep appending entries, make sure the rolling is correct.
+   */
+  @Test
+  public void testAppendAndRoll() throws Exception {
+    RaftServerConfigKeys.Log.setPreallocatedSize(properties, SizeInBytes.valueOf("16KB"));
+    RaftServerConfigKeys.Log.setSegmentSizeMax(properties, SizeInBytes.valueOf("128KB"));
+
+    List<SegmentRange> ranges = prepareRanges(0, 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(RaftServerConstants.INVALID_LOG_INDEX, null);
+      // append entries to the raftlog
+      entries.stream().map(raftLog::appendEntry).forEach(CompletableFuture::join);
+    }
+
+    try (SegmentedRaftLog raftLog =
+             new SegmentedRaftLog(peerId, null, storage, -1, properties)) {
+      raftLog.open(RaftServerConstants.INVALID_LOG_INDEX, null);
+      // 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(0, 5, 200, 0);
+    List<LogEntryProto> entries = prepareLogEntries(ranges, null);
+
+    try (SegmentedRaftLog raftLog =
+             new SegmentedRaftLog(peerId, null, storage, -1, properties)) {
+      raftLog.open(RaftServerConstants.INVALID_LOG_INDEX, null);
+      // append entries to the raftlog
+      entries.stream().map(raftLog::appendEntry).forEach(CompletableFuture::join);
+    }
+
+    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(RaftServerConstants.INVALID_LOG_INDEX, null);
+      // truncate the log
+      raftLog.truncate(fromIndex).join();
+
+
+      checkEntries(raftLog, entries, 0, (int) fromIndex);
+    }
+
+    try (SegmentedRaftLog raftLog =
+             new SegmentedRaftLog(peerId, null, storage, -1, properties)) {
+      raftLog.open(RaftServerConstants.INVALID_LOG_INDEX, null);
+      // check if the raft log is correct
+      if (fromIndex > 0) {
+        Assert.assertEquals(entries.get((int) (fromIndex - 1)),
+            getLastEntry(raftLog));
+      } else {
+        Assert.assertNull(raftLog.getLastEntryTermIndex());
+      }
+      checkEntries(raftLog, entries, 0, (int) fromIndex);
+    }
+  }
+
+  private void checkEntries(RaftLog raftLog, List<LogEntryProto> expected,
+      int offset, int size) throws IOException {
+    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);
+      }
+      TermIndex[] termIndices = raftLog.getEntries(
+          expected.get(offset).getIndex(),
+          expected.get(offset + size - 1).getIndex() + 1);
+      LogEntryProto[] entriesFromLog = Arrays.stream(termIndices)
+          .map(ti -> {
+            try {
+              return raftLog.get(ti.getIndex());
+            } catch (IOException e) {
+              throw new RuntimeException(e);
+            }
+          })
+          .toArray(LogEntryProto[]::new);
+      LogEntryProto[] expectedArray = expected.subList(offset, offset + size)
+          .stream().toArray(LogEntryProto[]::new);
+      Assert.assertArrayEquals(expectedArray, entriesFromLog);
+    }
+  }
+
+  private void checkFailedEntries(List<LogEntryProto> entries, long fromIndex, RetryCache retryCache) {
+    for (int i = 0; i < entries.size(); i++) {
+      if (i < fromIndex) {
+        RetryCacheTestUtil.assertFailure(retryCache, entries.get(i), false);
+      } else {
+        RetryCacheTestUtil.assertFailure(retryCache, entries.get(i), true);
+      }
+    }
+  }
+
+  /**
+   * Test append with inconsistent entries
+   */
+  @Test
+  public void testAppendEntriesWithInconsistency() throws Exception {
+    // prepare the log for truncation
+    List<SegmentRange> ranges = prepareRanges(0, 5, 200, 0);
+    List<LogEntryProto> entries = prepareLogEntries(ranges, null);
+
+    RaftServerImpl server = mock(RaftServerImpl.class);
+    RetryCache retryCache = RetryCacheTestUtil.createRetryCache();
+    when(server.getRetryCache()).thenReturn(retryCache);
+    doCallRealMethod().when(server).failClientRequest(any(LogEntryProto.class));
+    try (SegmentedRaftLog raftLog =
+             new SegmentedRaftLog(peerId, server, storage, -1, properties)) {
+      raftLog.open(RaftServerConstants.INVALID_LOG_INDEX, null);
+      entries.stream().forEach(entry -> RetryCacheTestUtil.createEntry(retryCache, entry));
+      // append entries to the raftlog
+      entries.stream().map(raftLog::appendEntry).forEach(CompletableFuture::join);
+    }
+
+    // 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, server, storage, -1, properties)) {
+      raftLog.open(RaftServerConstants.INVALID_LOG_INDEX, null);
+      raftLog.append(newEntries.toArray(new LogEntryProto[newEntries.size()])).forEach(CompletableFuture::join);
+
+      checkFailedEntries(entries, 650, retryCache);
+      checkEntries(raftLog, entries, 0, 650);
+      checkEntries(raftLog, newEntries, 100, 100);
+      Assert.assertEquals(newEntries.get(newEntries.size() - 1),
+          getLastEntry(raftLog));
+      Assert.assertEquals(newEntries.get(newEntries.size() - 1).getIndex(),
+          raftLog.getLatestFlushedIndex());
+    }
+
+    // load the raftlog again and check
+    try (SegmentedRaftLog raftLog =
+             new SegmentedRaftLog(peerId, server, storage, -1, properties)) {
+      raftLog.open(RaftServerConstants.INVALID_LOG_INDEX, null);
+      checkEntries(raftLog, entries, 0, 650);
+      checkEntries(raftLog, newEntries, 100, 100);
+      Assert.assertEquals(newEntries.get(newEntries.size() - 1),
+          getLastEntry(raftLog));
+      Assert.assertEquals(newEntries.get(newEntries.size() - 1).getIndex(),
+          raftLog.getLatestFlushedIndex());
+
+      RaftLogCache cache = raftLog.getRaftLogCache();
+      Assert.assertEquals(5, cache.getNumOfSegments());
+    }
+  }
+
+  @Test
+  public void testSegmentedRaftLogStateMachineData() throws Exception {
+    final SegmentRange range = new SegmentRange(0, 10, 1, true);
+    final List<LogEntryProto> entries = prepareLogEntries(range, null, true, new ArrayList<>());
+
+    final SimpleStateMachine4Testing sm = new SimpleStateMachine4Testing();
+    try (SegmentedRaftLog raftLog = new SegmentedRaftLog(peerId, null, sm, null, storage, -1, properties)) {
+      raftLog.open(RaftServerConstants.INVALID_LOG_INDEX, null);
+
+      int next = 0;
+      long flush = -1;
+      assertIndices(raftLog, flush, next);
+      raftLog.appendEntry(entries.get(next++));
+      assertIndices(raftLog, flush, next);
+      raftLog.appendEntry(entries.get(next++));
+      assertIndices(raftLog, flush, next);
+      raftLog.appendEntry(entries.get(next++));
+      assertIndicesMultipleAttempts(raftLog, flush += 3, next);
+
+      sm.blockFlushStateMachineData();
+      raftLog.appendEntry(entries.get(next++));
+      {
+        sm.blockWriteStateMachineData();
+        final Thread t = startAppendEntryThread(raftLog, entries.get(next++));
+        TimeUnit.SECONDS.sleep(1);
+        Assert.assertTrue(t.isAlive());
+        sm.unblockWriteStateMachineData();
+        t.join();
+      }
+      assertIndices(raftLog, flush, next);
+      TimeUnit.SECONDS.sleep(1);
+      assertIndices(raftLog, flush, next);
+      sm.unblockFlushStateMachineData();
+      assertIndicesMultipleAttempts(raftLog, flush + 2, next);
+    }
+  }
+
+  @Test
+  public void testSegmentedRaftLogStateMachineDataTimeoutIOException() throws Exception {
+    RaftServerConfigKeys.Log.StateMachineData.setSync(properties, true);
+    final TimeDuration syncTimeout = TimeDuration.valueOf(100, TimeUnit.MILLISECONDS);
+    RaftServerConfigKeys.Log.StateMachineData.setSyncTimeout(properties, syncTimeout);
+    final int numRetries = 2;
+    RaftServerConfigKeys.Log.StateMachineData.setSyncTimeoutRetry(properties, numRetries);
+    ExitUtils.disableSystemExit();
+
+    final LogEntryProto entry = prepareLogEntry(0, 0, null, true);
+    final StateMachine sm = new BaseStateMachine() {
+      @Override
+      public CompletableFuture<?> writeStateMachineData(LogEntryProto entry) {
+        return new CompletableFuture<>(); // the future never completes
+      }
+    };
+
+    try (SegmentedRaftLog raftLog = new SegmentedRaftLog(peerId, null, sm, null, storage, -1, properties)) {
+      raftLog.open(RaftServerConstants.INVALID_LOG_INDEX, null);
+      raftLog.appendEntry(entry);  // RaftLogWorker should catch TimeoutIOException
+
+      JavaUtils.attempt(() -> {
+        final ExitUtils.ExitException exitException = ExitUtils.getFirstExitException();
+        Objects.requireNonNull(exitException, "exitException == null");
+        Assert.assertEquals(TimeoutIOException.class, exitException.getCause().getClass());
+      }, 3*numRetries, syncTimeout, "RaftLogWorker should catch TimeoutIOException and exit", LOG);
+    }
+  }
+
+  static Thread startAppendEntryThread(RaftLog raftLog, LogEntryProto entry) {
+    final Thread t = new Thread(() -> raftLog.appendEntry(entry));
+    t.start();
+    return t;
+  }
+
+  void assertIndices(RaftLog raftLog, long expectedFlushIndex, long expectedNextIndex) {
+    LOG.info("assert expectedFlushIndex={}", expectedFlushIndex);
+    Assert.assertEquals(expectedFlushIndex, raftLog.getLatestFlushedIndex());
+    LOG.info("assert expectedNextIndex={}", expectedNextIndex);
+    Assert.assertEquals(expectedNextIndex, raftLog.getNextIndex());
+  }
+
+  void assertIndicesMultipleAttempts(RaftLog raftLog, long expectedFlushIndex, long expectedNextIndex) throws Exception {
+    JavaUtils.attempt(() -> assertIndices(raftLog, expectedFlushIndex, expectedNextIndex),
+        10, 100, "assertIndices", LOG);
+  }
+
+  @Test
+  public void testSegmentedRaftLogFormatInternalHeader() throws Exception {
+    testFailureCase("testSegmentedRaftLogFormatInternalHeader",
+        () -> SegmentedRaftLogFormat.applyHeaderTo(header -> {
+          LOG.info("header  = " + new String(header, StandardCharsets.UTF_8));
+          header[0] += 1; // try changing the internal header
+          LOG.info("header' = " + new String(header, StandardCharsets.UTF_8));
+          return null;
+        }), IllegalStateException.class);
+
+    // reset the header
+    SegmentedRaftLogFormat.applyHeaderTo(header -> {
+      LOG.info("header'  = " + new String(header, StandardCharsets.UTF_8));
+      header[0] -= 1; // try changing the internal header
+      LOG.info("header'' = " + new String(header, StandardCharsets.UTF_8));
+      return null;
+    });
+  }
+}