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 2017/03/31 05:27:22 UTC

[2/2] incubator-ratis git commit: RATIS-52. Refactor RaftUtils into different classes. Contributed by Enis Soztutar

RATIS-52. Refactor RaftUtils into different classes.  Contributed by Enis Soztutar


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

Branch: refs/heads/master
Commit: 43ad0def61944b108bb4f2a474a60a4c566d9821
Parents: 67f25fe
Author: Tsz-Wo Nicholas Sze <sz...@hortonworks.com>
Authored: Fri Mar 31 13:26:16 2017 +0800
Committer: Tsz-Wo Nicholas Sze <sz...@hortonworks.com>
Committed: Fri Mar 31 13:26:16 2017 +0800

----------------------------------------------------------------------
 .../ratis/client/impl/ClientProtoUtils.java     |   4 +-
 .../ratis/client/impl/RaftClientImpl.java       |   7 +-
 .../java/org/apache/ratis/RaftConfigKeys.java   |  12 +-
 .../org/apache/ratis/io/nativeio/NativeIO.java  |  19 +-
 .../ratis/io/nativeio/NativeIOException.java    |   4 +-
 .../org/apache/ratis/protocol/ClientId.java     |   4 +-
 .../org/apache/ratis/protocol/RaftPeerId.java   |   4 +-
 .../org/apache/ratis/rpc/SupportedRpcType.java  |   7 +-
 .../ratis/util/AtomicFileOutputStream.java      |   2 +-
 .../org/apache/ratis/util/CollectionUtils.java  |  63 +++
 .../java/org/apache/ratis/util/FileUtils.java   |   2 +-
 .../java/org/apache/ratis/util/IOUtils.java     | 144 +++++++
 .../java/org/apache/ratis/util/LifeCycle.java   |  14 +-
 .../java/org/apache/ratis/util/LogUtils.java    |  36 ++
 .../java/org/apache/ratis/util/MD5FileUtil.java |   6 +-
 .../org/apache/ratis/util/PlatformUtils.java    |  74 ++++
 .../org/apache/ratis/util/Preconditions.java    |  75 ++++
 .../java/org/apache/ratis/util/RaftUtils.java   | 386 -------------------
 .../org/apache/ratis/util/ReflectionUtils.java  | 220 +++++++++++
 .../arithmetic/ArithmeticStateMachine.java      |   4 +-
 .../arithmetic/expression/BinaryExpression.java |   6 +-
 .../arithmetic/expression/DoubleValue.java      |   6 +-
 .../arithmetic/expression/Expression.java       |   8 +-
 .../arithmetic/expression/NullValue.java        |   4 +-
 .../arithmetic/expression/UnaryExpression.java  |   6 +-
 .../arithmetic/expression/Variable.java         |   6 +-
 .../java/org/apache/ratis/TestBatchAppend.java  |   8 +-
 .../org/apache/ratis/TestRestartRaftPeer.java   |  10 +-
 .../examples/arithmetic/TestArithmetic.java     |   4 +-
 .../TestRaftStateMachineException.java          |  10 +-
 .../org/apache/ratis/grpc/RaftGrpcUtil.java     |   9 +-
 .../ratis/grpc/client/AppendStreamer.java       |  11 +-
 .../apache/ratis/grpc/client/GrpcClientRpc.java |   6 +-
 .../grpc/client/RaftClientProtocolService.java  |   8 +-
 .../ratis/grpc/server/GRpcLogAppender.java      |  12 +-
 .../grpc/TestRaftReconfigurationWithGRpc.java   |   4 +-
 .../org/apache/ratis/grpc/TestRaftStream.java   |   4 +-
 .../org/apache/ratis/grpc/TestRaftWithGrpc.java |   4 +-
 .../TestRaftReconfigurationWithHadoopRpc.java   |   2 -
 .../ratis/hadooprpc/TestRaftWithHadoopRpc.java  |   8 +-
 .../org/apache/ratis/netty/NettyRpcProxy.java   |   8 +-
 .../apache/ratis/netty/TestRaftWithNetty.java   |   6 +-
 .../ratis/server/impl/ConfigurationManager.java |   4 +-
 .../ratis/server/impl/LeaderElection.java       |   4 +-
 .../apache/ratis/server/impl/LeaderState.java   |  10 +-
 .../apache/ratis/server/impl/LogAppender.java   |   7 +-
 .../ratis/server/impl/PeerConfiguration.java    |   6 +-
 .../ratis/server/impl/PendingRequest.java       |   6 +-
 .../ratis/server/impl/PendingRequests.java      |  12 +-
 .../ratis/server/impl/RaftConfiguration.java    |  24 +-
 .../ratis/server/impl/RaftServerImpl.java       |  25 +-
 .../apache/ratis/server/impl/ServerState.java   |   4 +-
 .../ratis/server/impl/StateMachineUpdater.java  |   8 +-
 .../ratis/server/storage/LogInputStream.java    |  19 +-
 .../ratis/server/storage/LogOutputStream.java   |   6 +-
 .../apache/ratis/server/storage/LogReader.java  |  15 +-
 .../apache/ratis/server/storage/LogSegment.java |  26 +-
 .../ratis/server/storage/MemoryRaftLog.java     |   4 +-
 .../apache/ratis/server/storage/MetaFile.java   |   4 +-
 .../apache/ratis/server/storage/RaftLog.java    |   4 +-
 .../ratis/server/storage/RaftLogCache.java      |  14 +-
 .../ratis/server/storage/RaftLogWorker.java     |  27 +-
 .../ratis/server/storage/RaftStorage.java       |   4 +-
 .../ratis/server/storage/SegmentedRaftLog.java  |   6 +-
 .../ratis/server/storage/SnapshotManager.java   |   7 +-
 .../java/org/apache/ratis/MiniRaftCluster.java  |  41 +-
 .../ratis/RaftNotLeaderExceptionBaseTest.java   |  10 +-
 .../impl/RaftReconfigurationBaseTest.java       |   8 +-
 .../simulation/SimulatedRequestReply.java       |  17 +-
 .../simulation/TestRaftWithSimulatedRpc.java    |   8 +-
 .../server/storage/TestSegmentedRaftLog.java    |   4 +-
 .../statemachine/RaftSnapshotBaseTest.java      |  10 +-
 .../SimpleStateMachine4Testing.java             |   4 +-
 .../ratis/statemachine/TermIndexTracker.java    |   4 +-
 .../ratis/statemachine/TestStateMachine.java    |   6 +-
 75 files changed, 922 insertions(+), 673 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-ratis/blob/43ad0def/ratis-client/src/main/java/org/apache/ratis/client/impl/ClientProtoUtils.java
----------------------------------------------------------------------
diff --git a/ratis-client/src/main/java/org/apache/ratis/client/impl/ClientProtoUtils.java b/ratis-client/src/main/java/org/apache/ratis/client/impl/ClientProtoUtils.java
index ddecad6..a6e99c6 100644
--- a/ratis-client/src/main/java/org/apache/ratis/client/impl/ClientProtoUtils.java
+++ b/ratis-client/src/main/java/org/apache/ratis/client/impl/ClientProtoUtils.java
@@ -21,7 +21,7 @@ import org.apache.ratis.shaded.com.google.protobuf.ByteString;
 import org.apache.ratis.shaded.proto.RaftProtos.*;
 import org.apache.ratis.protocol.*;
 import org.apache.ratis.util.ProtoUtils;
-import org.apache.ratis.util.RaftUtils;
+import org.apache.ratis.util.ReflectionUtils;
 
 import java.util.Arrays;
 
@@ -143,7 +143,7 @@ public class ClientProtoUtils {
     } else {
       try {
         Class<?> clazz = Class.forName(className);
-        final Exception e = RaftUtils.instantiateException(
+        final Exception e = ReflectionUtils.instantiateException(
             clazz.asSubclass(Exception.class), errorMsg, null);
         sme = new StateMachineException(serverId, e);
       } catch (Exception e) {

http://git-wip-us.apache.org/repos/asf/incubator-ratis/blob/43ad0def/ratis-client/src/main/java/org/apache/ratis/client/impl/RaftClientImpl.java
----------------------------------------------------------------------
diff --git a/ratis-client/src/main/java/org/apache/ratis/client/impl/RaftClientImpl.java b/ratis-client/src/main/java/org/apache/ratis/client/impl/RaftClientImpl.java
index eae42a5..2125ce0 100644
--- a/ratis-client/src/main/java/org/apache/ratis/client/impl/RaftClientImpl.java
+++ b/ratis-client/src/main/java/org/apache/ratis/client/impl/RaftClientImpl.java
@@ -19,9 +19,10 @@ package org.apache.ratis.client.impl;
 
 import org.apache.ratis.client.RaftClient;
 import org.apache.ratis.client.RaftClientRpc;
+import org.apache.ratis.util.IOUtils;
+import org.apache.ratis.util.CollectionUtils;
 import org.apache.ratis.util.TimeDuration;
 import org.apache.ratis.protocol.*;
-import org.apache.ratis.util.RaftUtils;
 
 import java.io.IOException;
 import java.io.InterruptedIOException;
@@ -108,7 +109,7 @@ final class RaftClientImpl implements RaftClient {
         retryInterval.sleep();
       } catch (InterruptedException ie) {
         Thread.currentThread().interrupt();
-        throw RaftUtils.toInterruptedIOException(
+        throw IOUtils.toInterruptedIOException(
             "Interrupted when sending " + request, ie);
       }
     }
@@ -158,7 +159,7 @@ final class RaftClientImpl implements RaftClient {
         newLeader, ioe);
     final RaftPeerId oldLeader = request.getServerId();
     if (newLeader == null && oldLeader.equals(leaderId)) {
-      newLeader = RaftUtils.next(oldLeader, RaftUtils.as(peers, RaftPeer::getId));
+      newLeader = CollectionUtils.next(oldLeader, CollectionUtils.as(peers, RaftPeer::getId));
     }
     if (newLeader != null && oldLeader.equals(leaderId)) {
       LOG.debug("{}: change Leader from {} to {}", clientId, oldLeader, newLeader);

http://git-wip-us.apache.org/repos/asf/incubator-ratis/blob/43ad0def/ratis-common/src/main/java/org/apache/ratis/RaftConfigKeys.java
----------------------------------------------------------------------
diff --git a/ratis-common/src/main/java/org/apache/ratis/RaftConfigKeys.java b/ratis-common/src/main/java/org/apache/ratis/RaftConfigKeys.java
index b67acaf..665a7f5 100644
--- a/ratis-common/src/main/java/org/apache/ratis/RaftConfigKeys.java
+++ b/ratis-common/src/main/java/org/apache/ratis/RaftConfigKeys.java
@@ -17,12 +17,14 @@
  */
 package org.apache.ratis;
 
+import static org.apache.ratis.conf.ConfUtils.get;
+import static org.apache.ratis.conf.ConfUtils.printAll;
+import static org.apache.ratis.conf.ConfUtils.set;
+
 import org.apache.ratis.conf.RaftProperties;
 import org.apache.ratis.rpc.RpcType;
 import org.apache.ratis.rpc.SupportedRpcType;
-import org.apache.ratis.util.RaftUtils;
-
-import static org.apache.ratis.conf.ConfUtils.*;
+import org.apache.ratis.util.ReflectionUtils;
 
 public interface RaftConfigKeys {
   String PREFIX = "raft";
@@ -42,8 +44,8 @@ public interface RaftConfigKeys {
       }
 
       // Try using it as a class name
-      return RaftUtils.newInstance(
-          RaftUtils.getClass(t, properties, RpcType.class));
+      return ReflectionUtils.newInstance(
+          ReflectionUtils.getClass(t, properties, RpcType.class));
     }
 
     static void setType(RaftProperties properties, RpcType type) {

http://git-wip-us.apache.org/repos/asf/incubator-ratis/blob/43ad0def/ratis-common/src/main/java/org/apache/ratis/io/nativeio/NativeIO.java
----------------------------------------------------------------------
diff --git a/ratis-common/src/main/java/org/apache/ratis/io/nativeio/NativeIO.java b/ratis-common/src/main/java/org/apache/ratis/io/nativeio/NativeIO.java
index e096e84..921fb32 100644
--- a/ratis-common/src/main/java/org/apache/ratis/io/nativeio/NativeIO.java
+++ b/ratis-common/src/main/java/org/apache/ratis/io/nativeio/NativeIO.java
@@ -18,8 +18,9 @@
 package org.apache.ratis.io.nativeio;
 
 import org.apache.ratis.protocol.AlreadyExistsException;
+import org.apache.ratis.util.IOUtils;
 import org.apache.ratis.util.NativeCodeLoader;
-import org.apache.ratis.util.RaftUtils;
+import org.apache.ratis.util.PlatformUtils;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 import sun.misc.Unsafe;
@@ -183,7 +184,7 @@ public class NativeIO {
     private static native void chmodImpl(String path, int mode) throws IOException;
 
     public static void chmod(String path, int mode) throws IOException {
-      if (!RaftUtils.WINDOWS) {
+      if (!PlatformUtils.WINDOWS) {
         chmodImpl(path, mode);
       } else {
         try {
@@ -321,12 +322,12 @@ public class NativeIO {
       }
 
       Stat(String owner, String group, int mode) {
-        if (!RaftUtils.WINDOWS) {
+        if (!PlatformUtils.WINDOWS) {
           this.owner = owner;
         } else {
           this.owner = stripDomain(owner);
         }
-        if (!RaftUtils.WINDOWS) {
+        if (!PlatformUtils.WINDOWS) {
           this.group = group;
         } else {
           this.group = stripDomain(group);
@@ -604,7 +605,7 @@ public class NativeIO {
    */
   public static FileInputStream getShareDeleteFileInputStream(File f)
       throws IOException {
-    if (!RaftUtils.WINDOWS) {
+    if (!PlatformUtils.WINDOWS) {
       // On Linux the default FileInputStream shares delete permission
       // on the file opened.
       //
@@ -632,7 +633,7 @@ public class NativeIO {
    */
   public static FileInputStream getShareDeleteFileInputStream(File f, long seekOffset)
       throws IOException {
-    if (!RaftUtils.WINDOWS) {
+    if (!PlatformUtils.WINDOWS) {
       RandomAccessFile rf = new RandomAccessFile(f, "r");
       if (seekOffset > 0) {
         rf.seek(seekOffset);
@@ -666,7 +667,7 @@ public class NativeIO {
    */
   public static FileOutputStream getCreateForWriteFileOutputStream(File f, int permissions)
       throws IOException {
-    if (!RaftUtils.WINDOWS) {
+    if (!PlatformUtils.WINDOWS) {
       // Use the native wrapper around open(2)
       try {
         FileDescriptor fd = NativeIO.POSIX.open(f.getAbsolutePath(),
@@ -770,7 +771,7 @@ public class NativeIO {
    * @param dst                  The destination path
    */
   public static void copyFileUnbuffered(File src, File dst) throws IOException {
-    if (nativeLoaded && RaftUtils.WINDOWS) {
+    if (nativeLoaded && PlatformUtils.WINDOWS) {
       copyFileUnbuffered0(src.getAbsolutePath(), dst.getAbsolutePath());
     } else {
       FileInputStream fis = null;
@@ -791,7 +792,7 @@ public class NativeIO {
           position += transferred;
         }
       } finally {
-        RaftUtils.cleanup(LOG, output, fos, input, fis);
+        IOUtils.cleanup(LOG, output, fos, input, fis);
       }
     }
   }

http://git-wip-us.apache.org/repos/asf/incubator-ratis/blob/43ad0def/ratis-common/src/main/java/org/apache/ratis/io/nativeio/NativeIOException.java
----------------------------------------------------------------------
diff --git a/ratis-common/src/main/java/org/apache/ratis/io/nativeio/NativeIOException.java b/ratis-common/src/main/java/org/apache/ratis/io/nativeio/NativeIOException.java
index 58b83e7..15f69cf 100644
--- a/ratis-common/src/main/java/org/apache/ratis/io/nativeio/NativeIOException.java
+++ b/ratis-common/src/main/java/org/apache/ratis/io/nativeio/NativeIOException.java
@@ -19,7 +19,7 @@ package org.apache.ratis.io.nativeio;
 
 import java.io.IOException;
 
-import org.apache.ratis.util.RaftUtils;
+import org.apache.ratis.util.PlatformUtils;
 
 
 /**
@@ -61,7 +61,7 @@ public class NativeIOException extends IOException {
 
   @Override
   public String toString() {
-    if (RaftUtils.WINDOWS)
+    if (PlatformUtils.WINDOWS)
       return errorCode + ": " + super.getMessage();
     else
       return errno.toString() + ": " + super.getMessage();

http://git-wip-us.apache.org/repos/asf/incubator-ratis/blob/43ad0def/ratis-common/src/main/java/org/apache/ratis/protocol/ClientId.java
----------------------------------------------------------------------
diff --git a/ratis-common/src/main/java/org/apache/ratis/protocol/ClientId.java b/ratis-common/src/main/java/org/apache/ratis/protocol/ClientId.java
index 7e15ee3..310e207 100644
--- a/ratis-common/src/main/java/org/apache/ratis/protocol/ClientId.java
+++ b/ratis-common/src/main/java/org/apache/ratis/protocol/ClientId.java
@@ -17,7 +17,7 @@
  */
 package org.apache.ratis.protocol;
 
-import org.apache.ratis.util.RaftUtils;
+import org.apache.ratis.util.Preconditions;
 
 import java.nio.ByteBuffer;
 import java.util.Objects;
@@ -43,7 +43,7 @@ public class ClientId {
 
   public ClientId(byte[] data) {
     Objects.requireNonNull(data, "data == null");
-    RaftUtils.assertTrue(data.length == BYTE_LENGTH,
+    Preconditions.assertTrue(data.length == BYTE_LENGTH,
         "data.length = %s != BYTE_LENGTH = %s", data.length, BYTE_LENGTH);
     ByteBuffer buffer = ByteBuffer.wrap(data);
     this.uuid = new UUID(buffer.getLong(), buffer.getLong());

http://git-wip-us.apache.org/repos/asf/incubator-ratis/blob/43ad0def/ratis-common/src/main/java/org/apache/ratis/protocol/RaftPeerId.java
----------------------------------------------------------------------
diff --git a/ratis-common/src/main/java/org/apache/ratis/protocol/RaftPeerId.java b/ratis-common/src/main/java/org/apache/ratis/protocol/RaftPeerId.java
index 92d9846..dff355e 100644
--- a/ratis-common/src/main/java/org/apache/ratis/protocol/RaftPeerId.java
+++ b/ratis-common/src/main/java/org/apache/ratis/protocol/RaftPeerId.java
@@ -18,7 +18,7 @@
 package org.apache.ratis.protocol;
 
 import org.apache.ratis.shaded.com.google.protobuf.ByteString;
-import org.apache.ratis.util.RaftUtils;
+import org.apache.ratis.util.Preconditions;
 
 import java.nio.charset.StandardCharsets;
 import java.util.Arrays;
@@ -37,7 +37,7 @@ public class RaftPeerId {
 
   public RaftPeerId(String id) {
     Objects.requireNonNull(id, "id == null");
-    RaftUtils.assertTrue(!id.isEmpty(), "id is an empty string.");
+    Preconditions.assertTrue(!id.isEmpty(), "id is an empty string.");
     this.id = id.getBytes(StandardCharsets.UTF_8);
   }
 

http://git-wip-us.apache.org/repos/asf/incubator-ratis/blob/43ad0def/ratis-common/src/main/java/org/apache/ratis/rpc/SupportedRpcType.java
----------------------------------------------------------------------
diff --git a/ratis-common/src/main/java/org/apache/ratis/rpc/SupportedRpcType.java b/ratis-common/src/main/java/org/apache/ratis/rpc/SupportedRpcType.java
index f1d8fac..665b2f4 100644
--- a/ratis-common/src/main/java/org/apache/ratis/rpc/SupportedRpcType.java
+++ b/ratis-common/src/main/java/org/apache/ratis/rpc/SupportedRpcType.java
@@ -19,7 +19,7 @@ package org.apache.ratis.rpc;
 
 import org.apache.ratis.conf.Parameters;
 import org.apache.ratis.conf.RaftProperties;
-import org.apache.ratis.util.RaftUtils;
+import org.apache.ratis.util.ReflectionUtils;
 
 /** The RPC types supported. */
 public enum SupportedRpcType implements RpcType {
@@ -42,8 +42,9 @@ public enum SupportedRpcType implements RpcType {
 
   @Override
   public RpcFactory newFactory(RaftProperties properties, Parameters parameters) {
-    final Class<? extends RpcFactory> clazz = RaftUtils.getClass(
+    final Class<? extends RpcFactory> clazz = ReflectionUtils.getClass(
         factoryClassName, properties, RpcFactory.class);
-    return RaftUtils.newInstance(clazz, ARG_CLASSES, parameters);
+    return ReflectionUtils.newInstance(clazz,
+        RpcFactory.class, ARG_CLASSES, parameters);
   }
 }

http://git-wip-us.apache.org/repos/asf/incubator-ratis/blob/43ad0def/ratis-common/src/main/java/org/apache/ratis/util/AtomicFileOutputStream.java
----------------------------------------------------------------------
diff --git a/ratis-common/src/main/java/org/apache/ratis/util/AtomicFileOutputStream.java b/ratis-common/src/main/java/org/apache/ratis/util/AtomicFileOutputStream.java
index e8e267e..e181e44 100644
--- a/ratis-common/src/main/java/org/apache/ratis/util/AtomicFileOutputStream.java
+++ b/ratis-common/src/main/java/org/apache/ratis/util/AtomicFileOutputStream.java
@@ -83,7 +83,7 @@ public class AtomicFileOutputStream extends FilterOutputStream {
       } else {
         if (!triedToClose) {
           // If we failed when flushing, try to close it to not leak an FD
-          RaftUtils.cleanup(LOG, out);
+          IOUtils.cleanup(LOG, out);
         }
         // close wasn't successful, try to delete the tmp file
         if (!tmpFile.delete()) {

http://git-wip-us.apache.org/repos/asf/incubator-ratis/blob/43ad0def/ratis-common/src/main/java/org/apache/ratis/util/CollectionUtils.java
----------------------------------------------------------------------
diff --git a/ratis-common/src/main/java/org/apache/ratis/util/CollectionUtils.java b/ratis-common/src/main/java/org/apache/ratis/util/CollectionUtils.java
new file mode 100644
index 0000000..05fa2fb
--- /dev/null
+++ b/ratis-common/src/main/java/org/apache/ratis/util/CollectionUtils.java
@@ -0,0 +1,63 @@
+/*
+ * *
+ *  * Licensed to the Apache Software Foundation (ASF) under one
+ *  * or more contributor license agreements.  See the NOTICE file
+ *  * distributed with this work for additional information
+ *  * regarding copyright ownership.  The ASF licenses this file
+ *  * to you under the Apache License, Version 2.0 (the
+ *  * "License"); you may not use this file except in compliance
+ *  * with the License.  You may obtain a copy of the License at
+ *  *
+ *  *     http://www.apache.org/licenses/LICENSE-2.0
+ *  *
+ *  * Unless required by applicable law or agreed to in writing, software
+ *  * distributed under the License is distributed on an "AS IS" BASIS,
+ *  * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ *  * See the License for the specific language governing permissions and
+ *  * limitations under the License.
+ *
+ */
+
+package org.apache.ratis.util;
+
+import java.util.Iterator;
+import java.util.Objects;
+import java.util.function.Function;
+
+public interface CollectionUtils {
+  /**
+   *  @return the next element in the iteration right after the given element;
+   *          if the given element is not in the iteration, return the first one
+   */
+  static <T> T next(final T given, final Iterable<T> iteration) {
+    Objects.requireNonNull(given, "given == null");
+    final Iterator<T> i = Objects.requireNonNull(iteration, "iteration == null").iterator();
+    Preconditions.assertTrue(i.hasNext(), "iteration is empty.");
+
+    final T first = i.next();
+    for(T current = first; i.hasNext(); ) {
+      final T next = i.next();
+      if (given.equals(current)) {
+        return next;
+      }
+      current = next;
+    }
+    return first;
+  }
+
+  static <INPUT, OUTPUT> Iterable<OUTPUT> as(
+      Iterable<INPUT> iteration, Function<INPUT, OUTPUT> converter) {
+    return () -> new Iterator<OUTPUT>() {
+      final Iterator<INPUT> i = iteration.iterator();
+      @Override
+      public boolean hasNext() {
+        return i.hasNext();
+      }
+
+      @Override
+      public OUTPUT next() {
+        return converter.apply(i.next());
+      }
+    };
+  }
+}

http://git-wip-us.apache.org/repos/asf/incubator-ratis/blob/43ad0def/ratis-common/src/main/java/org/apache/ratis/util/FileUtils.java
----------------------------------------------------------------------
diff --git a/ratis-common/src/main/java/org/apache/ratis/util/FileUtils.java b/ratis-common/src/main/java/org/apache/ratis/util/FileUtils.java
index 5dc509d..732e4c8 100644
--- a/ratis-common/src/main/java/org/apache/ratis/util/FileUtils.java
+++ b/ratis-common/src/main/java/org/apache/ratis/util/FileUtils.java
@@ -193,7 +193,7 @@ public class FileUtils {
    *         On Windows, true if process has write access on the path
    */
   public static boolean canWrite(File f) {
-    if (RaftUtils.WINDOWS) {
+    if (PlatformUtils.WINDOWS) {
       try {
         return NativeIO.Windows.access(f.getCanonicalPath(),
             NativeIO.Windows.AccessRight.ACCESS_WRITE);

http://git-wip-us.apache.org/repos/asf/incubator-ratis/blob/43ad0def/ratis-common/src/main/java/org/apache/ratis/util/IOUtils.java
----------------------------------------------------------------------
diff --git a/ratis-common/src/main/java/org/apache/ratis/util/IOUtils.java b/ratis-common/src/main/java/org/apache/ratis/util/IOUtils.java
new file mode 100644
index 0000000..ba5e78e
--- /dev/null
+++ b/ratis-common/src/main/java/org/apache/ratis/util/IOUtils.java
@@ -0,0 +1,144 @@
+/*
+ * *
+ *  * Licensed to the Apache Software Foundation (ASF) under one
+ *  * or more contributor license agreements.  See the NOTICE file
+ *  * distributed with this work for additional information
+ *  * regarding copyright ownership.  The ASF licenses this file
+ *  * to you under the Apache License, Version 2.0 (the
+ *  * "License"); you may not use this file except in compliance
+ *  * with the License.  You may obtain a copy of the License at
+ *  *
+ *  *     http://www.apache.org/licenses/LICENSE-2.0
+ *  *
+ *  * Unless required by applicable law or agreed to in writing, software
+ *  * distributed under the License is distributed on an "AS IS" BASIS,
+ *  * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ *  * See the License for the specific language governing permissions and
+ *  * limitations under the License.
+ *
+ */
+
+package org.apache.ratis.util;
+
+import java.io.Closeable;
+import java.io.EOFException;
+import java.io.IOException;
+import java.io.InputStream;
+import java.io.InterruptedIOException;
+import java.nio.ByteBuffer;
+import java.nio.channels.FileChannel;
+import java.util.concurrent.ExecutionException;
+
+import org.slf4j.Logger;
+
+/**
+ * IO related utility methods.
+ */
+public interface IOUtils {
+  static InterruptedIOException toInterruptedIOException(
+      String message, InterruptedException e) {
+    final InterruptedIOException iioe = new InterruptedIOException(message);
+    iioe.initCause(e);
+    return iioe;
+  }
+
+  static IOException asIOException(Throwable t) {
+    return t instanceof IOException? (IOException)t : new IOException(t);
+  }
+
+  static IOException toIOException(ExecutionException e) {
+    final Throwable cause = e.getCause();
+    return cause != null? asIOException(cause): new IOException(e);
+  }
+
+  static void readFully(InputStream in, int buffSize) throws IOException {
+    final byte buf[] = new byte[buffSize];
+    for(int bytesRead = in.read(buf); bytesRead >= 0; ) {
+      bytesRead = in.read(buf);
+    }
+  }
+
+  /**
+   * Reads len bytes in a loop.
+   *
+   * @param in InputStream to read from
+   * @param buf The buffer to fill
+   * @param off offset from the buffer
+   * @param len the length of bytes to read
+   * @throws IOException if it could not read requested number of bytes
+   * for any reason (including EOF)
+   */
+  static void readFully(InputStream in, byte[] buf, int off, int len)
+      throws IOException {
+    for(int toRead = len; toRead > 0; ) {
+      final int ret = in.read(buf, off, toRead);
+      if (ret < 0) {
+        throw new IOException( "Premature EOF from inputStream");
+      }
+      toRead -= ret;
+      off += ret;
+    }
+  }
+
+  /**
+   * Write a ByteBuffer to a FileChannel at a given offset,
+   * handling short writes.
+   *
+   * @param fc               The FileChannel to write to
+   * @param buf              The input buffer
+   * @param offset           The offset in the file to start writing at
+   * @throws IOException     On I/O error
+   */
+  static void writeFully(FileChannel fc, ByteBuffer buf, long offset)
+      throws IOException {
+    do {
+      offset += fc.write(buf, offset);
+    } while (buf.remaining() > 0);
+  }
+
+  /**
+   * Similar to readFully(). Skips bytes in a loop.
+   * @param in The InputStream to skip bytes from
+   * @param len number of bytes to skip.
+   * @throws IOException if it could not skip requested number of bytes
+   * for any reason (including EOF)
+   */
+  static void skipFully(InputStream in, long len) throws IOException {
+    long amt = len;
+    while (amt > 0) {
+      long ret = in.skip(amt);
+      if (ret == 0) {
+        // skip may return 0 even if we're not at EOF.  Luckily, we can
+        // use the read() method to figure out if we're at the end.
+        int b = in.read();
+        if (b == -1) {
+          throw new EOFException( "Premature EOF from inputStream after " +
+              "skipping " + (len - amt) + " byte(s).");
+        }
+        ret = 1;
+      }
+      amt -= ret;
+    }
+  }
+
+  /**
+   * Close the Closeable objects and <b>ignore</b> any {@link Throwable} or
+   * null pointers. Must only be used for cleanup in exception handlers.
+   *
+   * @param log the log to record problems to at debug level. Can be null.
+   * @param closeables the objects to close
+   */
+  static void cleanup(Logger log, Closeable... closeables) {
+    for (Closeable c : closeables) {
+      if (c != null) {
+        try {
+          c.close();
+        } catch(Throwable e) {
+          if (log != null && log.isDebugEnabled()) {
+            log.debug("Exception in closing " + c, e);
+          }
+        }
+      }
+    }
+  }
+}

http://git-wip-us.apache.org/repos/asf/incubator-ratis/blob/43ad0def/ratis-common/src/main/java/org/apache/ratis/util/LifeCycle.java
----------------------------------------------------------------------
diff --git a/ratis-common/src/main/java/org/apache/ratis/util/LifeCycle.java b/ratis-common/src/main/java/org/apache/ratis/util/LifeCycle.java
index 166f0dc..5246aba 100644
--- a/ratis-common/src/main/java/org/apache/ratis/util/LifeCycle.java
+++ b/ratis-common/src/main/java/org/apache/ratis/util/LifeCycle.java
@@ -17,12 +17,16 @@
  */
 package org.apache.ratis.util;
 
+import java.util.Arrays;
+import java.util.Collections;
+import java.util.EnumMap;
+import java.util.List;
+import java.util.Map;
+import java.util.concurrent.atomic.AtomicReference;
+
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
-import java.util.*;
-import java.util.concurrent.atomic.AtomicReference;
-
 /**
  * The life cycle of a machine.
  * <pre>
@@ -104,7 +108,7 @@ public class LifeCycle {
         LOG.trace("TRACE", new Throwable());
       }
 
-      RaftUtils.assertTrue(isValid(from, to),
+      Preconditions.assertTrue(isValid(from, to),
           "ILLEGAL TRANSITION: In %s, %s -> %s", name, from, to);
     }
   }
@@ -171,7 +175,7 @@ public class LifeCycle {
       startImpl.run();
       transition(State.RUNNING);
     } catch (Throwable t) {
-      transition(RaftUtils.isInstance(t, exceptionClasses)?
+      transition(ReflectionUtils.isInstance(t, exceptionClasses)?
           State.NEW: State.EXCEPTION);
       throw t;
     }

http://git-wip-us.apache.org/repos/asf/incubator-ratis/blob/43ad0def/ratis-common/src/main/java/org/apache/ratis/util/LogUtils.java
----------------------------------------------------------------------
diff --git a/ratis-common/src/main/java/org/apache/ratis/util/LogUtils.java b/ratis-common/src/main/java/org/apache/ratis/util/LogUtils.java
new file mode 100644
index 0000000..addc2ff
--- /dev/null
+++ b/ratis-common/src/main/java/org/apache/ratis/util/LogUtils.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.util;
+
+import org.apache.log4j.Level;
+import org.apache.log4j.LogManager;
+import org.slf4j.Logger;
+
+/**
+ * Logging (as in log4j) related utility methods.
+ */
+public interface LogUtils {
+
+  static void setLogLevel(Logger logger, Level level) {
+    LogManager.getLogger(logger.getName()).setLevel(level);
+  }
+
+}

http://git-wip-us.apache.org/repos/asf/incubator-ratis/blob/43ad0def/ratis-common/src/main/java/org/apache/ratis/util/MD5FileUtil.java
----------------------------------------------------------------------
diff --git a/ratis-common/src/main/java/org/apache/ratis/util/MD5FileUtil.java b/ratis-common/src/main/java/org/apache/ratis/util/MD5FileUtil.java
index 35c7136..a5ded8c 100644
--- a/ratis-common/src/main/java/org/apache/ratis/util/MD5FileUtil.java
+++ b/ratis-common/src/main/java/org/apache/ratis/util/MD5FileUtil.java
@@ -73,7 +73,7 @@ public abstract class MD5FileUtil {
     } catch (IOException ioe) {
       throw new IOException("Error reading md5 file at " + md5File, ioe);
     } finally {
-      RaftUtils.cleanup(LOG, reader);
+      IOUtils.cleanup(LOG, reader);
     }
 
     Matcher matcher = LINE_REGEX.matcher(md5Line);
@@ -118,11 +118,11 @@ public abstract class MD5FileUtil {
     try {
       MessageDigest digester = MD5Hash.getDigester();
       DigestInputStream dis = new DigestInputStream(in, digester);
-      RaftUtils.readFully(dis, 128*1024);
+      IOUtils.readFully(dis, 128*1024);
 
       return new MD5Hash(digester.digest());
     } finally {
-      RaftUtils.cleanup(LOG, in);
+      IOUtils.cleanup(LOG, in);
     }
   }
 

http://git-wip-us.apache.org/repos/asf/incubator-ratis/blob/43ad0def/ratis-common/src/main/java/org/apache/ratis/util/PlatformUtils.java
----------------------------------------------------------------------
diff --git a/ratis-common/src/main/java/org/apache/ratis/util/PlatformUtils.java b/ratis-common/src/main/java/org/apache/ratis/util/PlatformUtils.java
new file mode 100644
index 0000000..6d4bf54
--- /dev/null
+++ b/ratis-common/src/main/java/org/apache/ratis/util/PlatformUtils.java
@@ -0,0 +1,74 @@
+/*
+ * *
+ *  * Licensed to the Apache Software Foundation (ASF) under one
+ *  * or more contributor license agreements.  See the NOTICE file
+ *  * distributed with this work for additional information
+ *  * regarding copyright ownership.  The ASF licenses this file
+ *  * to you under the Apache License, Version 2.0 (the
+ *  * "License"); you may not use this file except in compliance
+ *  * with the License.  You may obtain a copy of the License at
+ *  *
+ *  *     http://www.apache.org/licenses/LICENSE-2.0
+ *  *
+ *  * Unless required by applicable law or agreed to in writing, software
+ *  * distributed under the License is distributed on an "AS IS" BASIS,
+ *  * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ *  * See the License for the specific language governing permissions and
+ *  * limitations under the License.
+ *
+ */
+
+package org.apache.ratis.util;
+
+/**
+ * Platform and architecture related utility methods.
+ */
+public class PlatformUtils {
+
+  private PlatformUtils() {
+    // Utility class, cannot instantiate
+  }
+
+  public static final boolean PPC_64
+      = System.getProperties().getProperty("os.arch").contains("ppc64");
+  /**
+   * Get the type of the operating system, as determined from parsing
+   * the <code>os.name</code> property.
+   */
+  private static final OSType osType = getOSType();
+  public static final boolean OTHER   = (osType == OSType.OS_TYPE_OTHER);
+  public static final boolean LINUX   = (osType == OSType.OS_TYPE_LINUX);
+  public static final boolean FREEBSD = (osType == OSType.OS_TYPE_FREEBSD);
+  public static final boolean MAC     = (osType == OSType.OS_TYPE_MAC);
+  public static final boolean SOLARIS = (osType == OSType.OS_TYPE_SOLARIS);
+  // Helper static vars for each platform
+  public static final boolean WINDOWS = (osType == OSType.OS_TYPE_WIN);
+
+  private static OSType getOSType() {
+    String osName = System.getProperty("os.name");
+    if (osName.startsWith("Windows")) {
+      return OSType.OS_TYPE_WIN;
+    } else if (osName.contains("SunOS") || osName.contains("Solaris")) {
+      return OSType.OS_TYPE_SOLARIS;
+    } else if (osName.contains("Mac")) {
+      return OSType.OS_TYPE_MAC;
+    } else if (osName.contains("FreeBSD")) {
+      return OSType.OS_TYPE_FREEBSD;
+    } else if (osName.startsWith("Linux")) {
+      return OSType.OS_TYPE_LINUX;
+    } else {
+      // Some other form of Unix
+      return OSType.OS_TYPE_OTHER;
+    }
+  }
+
+  // OSType detection
+  public enum OSType {
+    OS_TYPE_LINUX,
+    OS_TYPE_WIN,
+    OS_TYPE_SOLARIS,
+    OS_TYPE_MAC,
+    OS_TYPE_FREEBSD,
+    OS_TYPE_OTHER
+  }
+}

http://git-wip-us.apache.org/repos/asf/incubator-ratis/blob/43ad0def/ratis-common/src/main/java/org/apache/ratis/util/Preconditions.java
----------------------------------------------------------------------
diff --git a/ratis-common/src/main/java/org/apache/ratis/util/Preconditions.java b/ratis-common/src/main/java/org/apache/ratis/util/Preconditions.java
new file mode 100644
index 0000000..f1d55b0
--- /dev/null
+++ b/ratis-common/src/main/java/org/apache/ratis/util/Preconditions.java
@@ -0,0 +1,75 @@
+/*
+ * *
+ *  * Licensed to the Apache Software Foundation (ASF) under one
+ *  * or more contributor license agreements.  See the NOTICE file
+ *  * distributed with this work for additional information
+ *  * regarding copyright ownership.  The ASF licenses this file
+ *  * to you under the Apache License, Version 2.0 (the
+ *  * "License"); you may not use this file except in compliance
+ *  * with the License.  You may obtain a copy of the License at
+ *  *
+ *  *     http://www.apache.org/licenses/LICENSE-2.0
+ *  *
+ *  * Unless required by applicable law or agreed to in writing, software
+ *  * distributed under the License is distributed on an "AS IS" BASIS,
+ *  * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ *  * See the License for the specific language governing permissions and
+ *  * limitations under the License.
+ *
+ */
+
+package org.apache.ratis.util;
+
+import java.util.function.Supplier;
+
+public interface Preconditions {
+  /**
+   * Assert if the given value is true.
+   * @param value the value to be asserted.
+   * @throws IllegalStateException if the given value is false.
+   */
+  static void assertTrue(boolean value) {
+    if (!value) {
+      throw new IllegalStateException();
+    }
+  }
+
+  /**
+   * Assert if the given value is true.
+   * @param value the value to be asserted.
+   * @param message The exception message.
+   * @throws IllegalStateException with the given message if the given value is false.
+   */
+  static void assertTrue(boolean value, Object message) {
+    if (!value) {
+      throw new IllegalStateException(String.valueOf(message));
+    }
+  }
+
+  /**
+   * Assert if the given value is true.
+   * @param value the value to be asserted.
+   * @param format exception message format.
+   * @param args exception message arguments.
+   * @throws IllegalStateException if the given value is false.
+   * The exception message is constructed by {@link String#format(String, Object...)}
+   * with the given format and arguments.
+   */
+  static void assertTrue(boolean value, String format, Object... args) {
+    if (!value) {
+      throw new IllegalStateException(String.format(format, args));
+    }
+  }
+
+  /**
+   * Assert if the given value is true.
+   * @param value the value to be asserted.
+   * @param message The exception message supplier.
+   * @throws IllegalStateException with the given message if the given value is false.
+   */
+  static void assertTrue(boolean value, Supplier<Object> message) {
+    if (!value) {
+      throw new IllegalStateException(String.valueOf(message.get()));
+    }
+  }
+}

http://git-wip-us.apache.org/repos/asf/incubator-ratis/blob/43ad0def/ratis-common/src/main/java/org/apache/ratis/util/RaftUtils.java
----------------------------------------------------------------------
diff --git a/ratis-common/src/main/java/org/apache/ratis/util/RaftUtils.java b/ratis-common/src/main/java/org/apache/ratis/util/RaftUtils.java
deleted file mode 100644
index ae3e857..0000000
--- a/ratis-common/src/main/java/org/apache/ratis/util/RaftUtils.java
+++ /dev/null
@@ -1,386 +0,0 @@
-/**
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- *     http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF 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.apache.log4j.LogManager;
-import org.apache.ratis.conf.RaftProperties;
-import org.slf4j.Logger;
-import org.slf4j.LoggerFactory;
-
-import java.io.*;
-import java.lang.reflect.Constructor;
-import java.nio.ByteBuffer;
-import java.nio.channels.FileChannel;
-import java.util.*;
-import java.util.concurrent.ConcurrentHashMap;
-import java.util.concurrent.ExecutionException;
-import java.util.function.Function;
-import java.util.function.Supplier;
-
-public abstract class RaftUtils {
-  public static final Logger LOG = LoggerFactory.getLogger(RaftUtils.class);
-
-  // OSType detection
-  public enum OSType {
-    OS_TYPE_LINUX,
-    OS_TYPE_WIN,
-    OS_TYPE_SOLARIS,
-    OS_TYPE_MAC,
-    OS_TYPE_FREEBSD,
-    OS_TYPE_OTHER
-  }
-
-  /**
-   * Get the type of the operating system, as determined from parsing
-   * the <code>os.name</code> property.
-   */
-  private static final OSType osType = getOSType();
-
-  private static OSType getOSType() {
-    String osName = System.getProperty("os.name");
-    if (osName.startsWith("Windows")) {
-      return OSType.OS_TYPE_WIN;
-    } else if (osName.contains("SunOS") || osName.contains("Solaris")) {
-      return OSType.OS_TYPE_SOLARIS;
-    } else if (osName.contains("Mac")) {
-      return OSType.OS_TYPE_MAC;
-    } else if (osName.contains("FreeBSD")) {
-      return OSType.OS_TYPE_FREEBSD;
-    } else if (osName.startsWith("Linux")) {
-      return OSType.OS_TYPE_LINUX;
-    } else {
-      // Some other form of Unix
-      return OSType.OS_TYPE_OTHER;
-    }
-  }
-
-  // Helper static vars for each platform
-  public static final boolean WINDOWS = (osType == OSType.OS_TYPE_WIN);
-  public static final boolean SOLARIS = (osType == OSType.OS_TYPE_SOLARIS);
-  public static final boolean MAC     = (osType == OSType.OS_TYPE_MAC);
-  public static final boolean FREEBSD = (osType == OSType.OS_TYPE_FREEBSD);
-  public static final boolean LINUX   = (osType == OSType.OS_TYPE_LINUX);
-  public static final boolean OTHER   = (osType == OSType.OS_TYPE_OTHER);
-
-  public static final boolean PPC_64
-      = System.getProperties().getProperty("os.arch").contains("ppc64");
-
-  public static final Class<?>[] EMPTY_CLASSES = {};
-  /**
-   * Cache of constructors for each class. Pins the classes so they
-   * can't be garbage collected until ReflectionUtils can be collected.
-   */
-  private static final Map<List<Class<?>>, Constructor<?>> CONSTRUCTOR_CACHE
-      = new ConcurrentHashMap<>();
-
-  public static InterruptedIOException toInterruptedIOException(
-      String message, InterruptedException e) {
-    final InterruptedIOException iioe = new InterruptedIOException(message);
-    iioe.initCause(e);
-    return iioe;
-  }
-
-  public static IOException asIOException(Throwable t) {
-    return t instanceof IOException? (IOException)t : new IOException(t);
-  }
-
-  public static IOException toIOException(ExecutionException e) {
-    final Throwable cause = e.getCause();
-    return cause != null? asIOException(cause): new IOException(e);
-  }
-
-  /** Is the given object an instance of one of the given classes? */
-  public static boolean isInstance(Object obj, Class<?>... classes) {
-    for(Class<?> c : classes) {
-      if (c.isInstance(obj)) {
-        return true;
-      }
-    }
-    return false;
-  }
-
-  /**
-   * Create an object for the given class using its default constructor.
-   */
-  public static <T> T newInstance(Class<T> clazz) {
-    return newInstance(clazz, EMPTY_CLASSES);
-  }
-
-  /**
-   * Create an object for the given class using the specified constructor.
-   *
-   * @param clazz class of which an object is created
-   * @param argClasses argument classes of the constructor
-   * @param args actual arguments to be passed to the constructor
-   * @param <T> class type of clazz
-   * @return a new object
-   */
-  public static <T> T newInstance(Class<T> clazz, Class<?>[] argClasses, Object... args) {
-    Objects.requireNonNull(clazz, "clazz == null");
-    try {
-      final List<Class<?>> key = new ArrayList<>();
-      key.add(clazz);
-      key.addAll(Arrays.asList(argClasses));
-
-      @SuppressWarnings("unchecked")
-      Constructor<T> ctor = (Constructor<T>) CONSTRUCTOR_CACHE.get(key);
-      if (ctor == null) {
-        ctor = clazz.getDeclaredConstructor(argClasses);
-        ctor.setAccessible(true);
-        CONSTRUCTOR_CACHE.put(key, ctor);
-      }
-      return ctor.newInstance(args);
-    } catch (Exception e) {
-      throw new RuntimeException(e);
-    }
-  }
-
-  public static <BASE> Class<? extends BASE> getClass(
-      String subClassName, RaftProperties properties, Class<BASE> base) {
-    try {
-      return properties.getClassByName(subClassName).asSubclass(base);
-    } catch (ClassNotFoundException e) {
-      throw new IllegalArgumentException("Failed to get class "
-          + subClassName + " as a subclass of " + base, e);
-    }
-  }
-
-  /**
-   * Create a memoized supplier which gets a value by invoking the initializer once
-   * and then keeps returning the same value as its supplied results.
-   *
-   * @param initializer to supply at most one non-null value.
-   * @param <T> The supplier result type.
-   * @return a memoized supplier which is thread-safe.
-   */
-  public static <T> Supplier<T> memoize(Supplier<T> initializer) {
-    Objects.requireNonNull(initializer, "initializer == null");
-    return new Supplier<T>() {
-      private volatile T value = null;
-
-      @Override
-      public T get() {
-        T v = value;
-        if (v == null) {
-          synchronized (this) {
-            v = value;
-            if (v == null) {
-              v = value = Objects.requireNonNull(initializer.get(),
-                  "initializer.get() returns null");
-            }
-          }
-        }
-        return v;
-      }
-    };
-  }
-
-  public static void setLogLevel(Logger logger, Level level) {
-    LogManager.getLogger(logger.getName()).setLevel(level);
-  }
-
-
-  public static void readFully(InputStream in, int buffSize) throws IOException {
-    final byte buf[] = new byte[buffSize];
-    for(int bytesRead = in.read(buf); bytesRead >= 0; ) {
-      bytesRead = in.read(buf);
-    }
-  }
-
-  /**
-   * Reads len bytes in a loop.
-   *
-   * @param in InputStream to read from
-   * @param buf The buffer to fill
-   * @param off offset from the buffer
-   * @param len the length of bytes to read
-   * @throws IOException if it could not read requested number of bytes
-   * for any reason (including EOF)
-   */
-  public static void readFully(InputStream in, byte[] buf, int off, int len)
-      throws IOException {
-    for(int toRead = len; toRead > 0; ) {
-      final int ret = in.read(buf, off, toRead);
-      if (ret < 0) {
-        throw new IOException( "Premature EOF from inputStream");
-      }
-      toRead -= ret;
-      off += ret;
-    }
-  }
-
-  /**
-   * Write a ByteBuffer to a FileChannel at a given offset,
-   * handling short writes.
-   *
-   * @param fc               The FileChannel to write to
-   * @param buf              The input buffer
-   * @param offset           The offset in the file to start writing at
-   * @throws IOException     On I/O error
-   */
-  public static void writeFully(FileChannel fc, ByteBuffer buf, long offset)
-      throws IOException {
-    do {
-      offset += fc.write(buf, offset);
-    } while (buf.remaining() > 0);
-  }
-
-  /**
-   * Similar to readFully(). Skips bytes in a loop.
-   * @param in The InputStream to skip bytes from
-   * @param len number of bytes to skip.
-   * @throws IOException if it could not skip requested number of bytes
-   * for any reason (including EOF)
-   */
-  public static void skipFully(InputStream in, long len) throws IOException {
-    long amt = len;
-    while (amt > 0) {
-      long ret = in.skip(amt);
-      if (ret == 0) {
-        // skip may return 0 even if we're not at EOF.  Luckily, we can
-        // use the read() method to figure out if we're at the end.
-        int b = in.read();
-        if (b == -1) {
-          throw new EOFException( "Premature EOF from inputStream after " +
-              "skipping " + (len - amt) + " byte(s).");
-        }
-        ret = 1;
-      }
-      amt -= ret;
-    }
-  }
-
-  /**
-   * Close the Closeable objects and <b>ignore</b> any {@link Throwable} or
-   * null pointers. Must only be used for cleanup in exception handlers.
-   *
-   * @param log the log to record problems to at debug level. Can be null.
-   * @param closeables the objects to close
-   */
-  public static void cleanup(Logger log, Closeable... closeables) {
-    for (Closeable c : closeables) {
-      if (c != null) {
-        try {
-          c.close();
-        } catch(Throwable e) {
-          if (log != null && log.isDebugEnabled()) {
-            log.debug("Exception in closing " + c, e);
-          }
-        }
-      }
-    }
-  }
-
-  /**
-   *  @return the next element in the iteration right after the given element;
-   *          if the given element is not in the iteration, return the first one
-   */
-  public static <T> T next(final T given, final Iterable<T> iteration) {
-    Objects.requireNonNull(given, "given == null");
-    final Iterator<T> i = Objects.requireNonNull(iteration, "iteration == null").iterator();
-    assertTrue(i.hasNext(), "iteration is empty.");
-
-    final T first = i.next();
-    for(T current = first; i.hasNext(); ) {
-      final T next = i.next();
-      if (given.equals(current)) {
-        return next;
-      }
-      current = next;
-    }
-    return first;
-  }
-
-  public static <INPUT, OUTPUT> Iterable<OUTPUT> as(
-      Iterable<INPUT> iteration, Function<INPUT, OUTPUT> converter) {
-    return () -> new Iterator<OUTPUT>() {
-      final Iterator<INPUT> i = iteration.iterator();
-      @Override
-      public boolean hasNext() {
-        return i.hasNext();
-      }
-
-      @Override
-      public OUTPUT next() {
-        return converter.apply(i.next());
-      }
-    };
-  }
-
-  /**
-   * Assert if the given value is true.
-   * @param value the value to be asserted.
-   * @throws IllegalStateException if the given value is false.
-   */
-  public static void assertTrue(boolean value) {
-    if (!value) {
-      throw new IllegalStateException();
-    }
-  }
-
-  /**
-   * Assert if the given value is true.
-   * @param value the value to be asserted.
-   * @param message The exception message.
-   * @throws IllegalStateException with the given message if the given value is false.
-   */
-  public static void assertTrue(boolean value, Object message) {
-    if (!value) {
-      throw new IllegalStateException(String.valueOf(message));
-    }
-  }
-
-  /**
-   * Assert if the given value is true.
-   * @param value the value to be asserted.
-   * @param format exception message format.
-   * @param args exception message arguments.
-   * @throws IllegalStateException if the given value is false.
-   * The exception message is constructed by {@link String#format(String, Object...)}
-   * with the given format and arguments.
-   */
-  public static void assertTrue(boolean value, String format, Object... args) {
-    if (!value) {
-      throw new IllegalStateException(String.format(format, args));
-    }
-  }
-
-  /**
-   * Assert if the given value is true.
-   * @param value the value to be asserted.
-   * @param message The exception message supplier.
-   * @throws IllegalStateException with the given message if the given value is false.
-   */
-  public static void assertTrue(boolean value, Supplier<Object> message) {
-    if (!value) {
-      throw new IllegalStateException(String.valueOf(message.get()));
-    }
-  }
-
-  public static Exception instantiateException(Class<? extends Exception> cls,
-      String message, Exception from) throws Exception {
-    Constructor<? extends Exception> cn = cls.getConstructor(String.class);
-    cn.setAccessible(true);
-    Exception ex = cn.newInstance(message);
-    if (from != null) {
-      ex.initCause(from);
-    }
-    return ex;
-  }
-}

http://git-wip-us.apache.org/repos/asf/incubator-ratis/blob/43ad0def/ratis-common/src/main/java/org/apache/ratis/util/ReflectionUtils.java
----------------------------------------------------------------------
diff --git a/ratis-common/src/main/java/org/apache/ratis/util/ReflectionUtils.java b/ratis-common/src/main/java/org/apache/ratis/util/ReflectionUtils.java
new file mode 100644
index 0000000..5c7ada6
--- /dev/null
+++ b/ratis-common/src/main/java/org/apache/ratis/util/ReflectionUtils.java
@@ -0,0 +1,220 @@
+/*
+ * *
+ *  * Licensed to the Apache Software Foundation (ASF) under one
+ *  * or more contributor license agreements.  See the NOTICE file
+ *  * distributed with this work for additional information
+ *  * regarding copyright ownership.  The ASF licenses this file
+ *  * to you under the Apache License, Version 2.0 (the
+ *  * "License"); you may not use this file except in compliance
+ *  * with the License.  You may obtain a copy of the License at
+ *  *
+ *  *     http://www.apache.org/licenses/LICENSE-2.0
+ *  *
+ *  * Unless required by applicable law or agreed to in writing, software
+ *  * distributed under the License is distributed on an "AS IS" BASIS,
+ *  * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ *  * See the License for the specific language governing permissions and
+ *  * limitations under the License.
+ *
+ */
+
+package org.apache.ratis.util;
+
+
+import java.lang.reflect.Constructor;
+import java.lang.reflect.InvocationTargetException;
+import java.util.ArrayList;
+import java.util.Arrays;
+import java.util.List;
+import java.util.Map;
+import java.util.Objects;
+import java.util.concurrent.ConcurrentHashMap;
+import java.util.function.Supplier;
+
+import org.apache.ratis.conf.RaftProperties;
+
+/**
+ * Reflection related utility methods.
+ */
+public class ReflectionUtils {
+
+  private ReflectionUtils() {
+    // Utility class, cannot instantiate
+  }
+
+  private static final Class<?>[] EMPTY_CLASSES = {};
+
+  /**
+   * Cache of constructors for each class. Pins the classes so they
+   * can't be garbage collected until ReflectionUtils can be collected.
+   */
+  private static final Map<List<Class<?>>, Constructor<?>> CONSTRUCTOR_CACHE
+      = new ConcurrentHashMap<>();
+
+  /**
+   * Create an object for the given class using its default constructor.
+   */
+  public static <T> T newInstance(Class<T> clazz) {
+    return newInstance(clazz, EMPTY_CLASSES);
+  }
+
+  /**
+   * Create an object for the given class using its default constructor.
+   */
+  public static <T> T newInstance(String className) {
+    return newInstance(className, EMPTY_CLASSES);
+  }
+
+  /**
+   * Create an object for the given class using the specified constructor.
+   *
+   * @param clazz class of which an object is created
+   * @param argClasses argument classes of the constructor
+   * @param args actual arguments to be passed to the constructor
+   * @param <T> class type of clazz
+   * @return a new object
+   */
+  public static <T> T newInstance(Class<T> clazz, Class<?>[] argClasses, Object... args) {
+    Objects.requireNonNull(clazz, "clazz == null");
+    try {
+      final List<Class<?>> key = new ArrayList<>();
+      key.add(clazz);
+      key.addAll(Arrays.asList(argClasses));
+
+      @SuppressWarnings("unchecked")
+      Constructor<T> ctor = (Constructor<T>) CONSTRUCTOR_CACHE.get(key);
+      if (ctor == null) {
+        ctor = clazz.getDeclaredConstructor(argClasses);
+        ctor.setAccessible(true);
+        CONSTRUCTOR_CACHE.put(key, ctor);
+      }
+      return instantiate(clazz.getName(), ctor, args);
+    } catch (NoSuchMethodException e) {
+      throw new UnsupportedOperationException(
+          "Unable to find suitable constructor for class " + clazz.getName(), e);
+    }
+  }
+
+  @SuppressWarnings("unchecked")
+  public static <T> T newInstance(String className, Class<? >[] argClasses,
+                                        Object... args) {
+    try {
+      Class<? extends T> resultType = (Class<? extends T>) Class.forName(className);
+      return newInstance(resultType, argClasses, args);
+    } catch (ClassNotFoundException e) {
+      throw new UnsupportedOperationException(
+          "Unable to find " + className, e);
+    }
+  }
+
+  public static <T> T newInstance(Class<T> type, Object... params) {
+    return newInstance(type, findConstructor(type, params), params);
+  }
+
+  private static <T> T instantiate(final String className, Constructor<T> ctor, Object[] ctorArgs) {
+    try {
+      ctor.setAccessible(true);
+      return ctor.newInstance(ctorArgs);
+    } catch (IllegalAccessException e) {
+      throw new UnsupportedOperationException(
+          "Unable to access specified class " + className, e);
+    } catch (InstantiationException e) {
+      throw new UnsupportedOperationException(
+          "Unable to instantiate specified class " + className, e);
+    } catch (InvocationTargetException e) {
+      throw new UnsupportedOperationException(
+          "Constructor threw an exception for " + className, e);
+    }
+  }
+
+  @SuppressWarnings("unchecked")
+  public static <T> Constructor<T> findConstructor(Class<T> type, Object... paramTypes) {
+    Constructor<T>[] constructors = (Constructor<T>[]) type.getDeclaredConstructors();
+    for (Constructor<T> ctor : constructors) {
+      Class<?>[] ctorParamTypes = ctor.getParameterTypes();
+      if (ctorParamTypes.length != paramTypes.length) {
+        continue;
+      }
+
+      boolean match = true;
+      for (int i = 0; i < ctorParamTypes.length && match; ++i) {
+        Class<?> paramType = paramTypes[i].getClass();
+        match = (!ctorParamTypes[i].isPrimitive()) ? ctorParamTypes[i].isAssignableFrom(paramType) :
+            ((int.class.equals(ctorParamTypes[i]) && Integer.class.equals(paramType)) ||
+                (long.class.equals(ctorParamTypes[i]) && Long.class.equals(paramType)) ||
+                (double.class.equals(ctorParamTypes[i]) && Double.class.equals(paramType)) ||
+                (char.class.equals(ctorParamTypes[i]) && Character.class.equals(paramType)) ||
+                (short.class.equals(ctorParamTypes[i]) && Short.class.equals(paramType)) ||
+                (boolean.class.equals(ctorParamTypes[i]) && Boolean.class.equals(paramType)) ||
+                (byte.class.equals(ctorParamTypes[i]) && Byte.class.equals(paramType)));
+      }
+
+      if (match) {
+        return ctor;
+      }
+    }
+    throw new UnsupportedOperationException(
+        "Unable to find suitable constructor for class " + type.getName());
+  }
+
+  /** Is the given object an instance of one of the given classes? */
+  public static boolean isInstance(Object obj, Class<?>... classes) {
+    for(Class<?> c : classes) {
+      if (c.isInstance(obj)) {
+        return true;
+      }
+    }
+    return false;
+  }
+
+  /**
+   * Create a memoized supplier which gets a value by invoking the initializer once
+   * and then keeps returning the same value as its supplied results.
+   *
+   * @param initializer to supply at most one non-null value.
+   * @param <T> The supplier result type.
+   * @return a memoized supplier which is thread-safe.
+   */
+  public static <T> Supplier<T> memoize(Supplier<T> initializer) {
+    Objects.requireNonNull(initializer, "initializer == null");
+    return new Supplier<T>() {
+      private volatile T value = null;
+
+      @Override
+      public T get() {
+        T v = value;
+        if (v == null) {
+          synchronized (this) {
+            v = value;
+            if (v == null) {
+              v = value = Objects.requireNonNull(initializer.get(),
+                  "initializer.get() returns null");
+            }
+          }
+        }
+        return v;
+      }
+    };
+  }
+
+  public static <BASE> Class<? extends BASE> getClass(
+      String subClassName, RaftProperties properties, Class<BASE> base) {
+    try {
+      return properties.getClassByName(subClassName).asSubclass(base);
+    } catch (ClassNotFoundException e) {
+      throw new IllegalArgumentException("Failed to get class "
+          + subClassName + " as a subclass of " + base, e);
+    }
+  }
+
+  public static Exception instantiateException(Class<? extends Exception> cls,
+      String message, Exception from) throws Exception {
+    Constructor<? extends Exception> cn = cls.getConstructor(String.class);
+    cn.setAccessible(true);
+    Exception ex = cn.newInstance(message);
+    if (from != null) {
+      ex.initCause(from);
+    }
+    return ex;
+  }
+}

http://git-wip-us.apache.org/repos/asf/incubator-ratis/blob/43ad0def/ratis-examples/src/main/java/org/apache/ratis/examples/arithmetic/ArithmeticStateMachine.java
----------------------------------------------------------------------
diff --git a/ratis-examples/src/main/java/org/apache/ratis/examples/arithmetic/ArithmeticStateMachine.java b/ratis-examples/src/main/java/org/apache/ratis/examples/arithmetic/ArithmeticStateMachine.java
index 89580df..53e8ab7 100644
--- a/ratis-examples/src/main/java/org/apache/ratis/examples/arithmetic/ArithmeticStateMachine.java
+++ b/ratis-examples/src/main/java/org/apache/ratis/examples/arithmetic/ArithmeticStateMachine.java
@@ -29,7 +29,7 @@ import org.apache.ratis.server.storage.RaftStorage;
 import org.apache.ratis.shaded.proto.RaftProtos.LogEntryProto;
 import org.apache.ratis.statemachine.*;
 import org.apache.ratis.util.AutoCloseableLock;
-import org.apache.ratis.util.RaftUtils;
+import org.apache.ratis.util.Preconditions;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
@@ -175,7 +175,7 @@ public class ArithmeticStateMachine extends BaseStateMachine {
     final TermIndex newTI = TermIndex.newTermIndex(term, index);
     final TermIndex oldTI = latestTermIndex.getAndSet(newTI);
     if (oldTI != null) {
-      RaftUtils.assertTrue(newTI.compareTo(oldTI) >= 0);
+      Preconditions.assertTrue(newTI.compareTo(oldTI) >= 0);
     }
   }
 }

http://git-wip-us.apache.org/repos/asf/incubator-ratis/blob/43ad0def/ratis-examples/src/main/java/org/apache/ratis/examples/arithmetic/expression/BinaryExpression.java
----------------------------------------------------------------------
diff --git a/ratis-examples/src/main/java/org/apache/ratis/examples/arithmetic/expression/BinaryExpression.java b/ratis-examples/src/main/java/org/apache/ratis/examples/arithmetic/expression/BinaryExpression.java
index 6486cfd..121bb7a 100644
--- a/ratis-examples/src/main/java/org/apache/ratis/examples/arithmetic/expression/BinaryExpression.java
+++ b/ratis-examples/src/main/java/org/apache/ratis/examples/arithmetic/expression/BinaryExpression.java
@@ -17,7 +17,7 @@
  */
 package org.apache.ratis.examples.arithmetic.expression;
 
-import org.apache.ratis.util.RaftUtils;
+import org.apache.ratis.util.Preconditions;
 
 import java.util.Map;
 
@@ -43,7 +43,7 @@ public class BinaryExpression implements Expression {
     static final Op[] VALUES = Op.values();
 
     static Op valueOf(byte b) {
-      RaftUtils.assertTrue(b < VALUES.length);
+      Preconditions.assertTrue(b < VALUES.length);
       return VALUES[b];
     }
   }
@@ -52,7 +52,7 @@ public class BinaryExpression implements Expression {
   private final Expression left, right;
 
   BinaryExpression(byte[] buf, final int offset) {
-    RaftUtils.assertTrue(buf[offset] == Type.BINARY.byteValue());
+    Preconditions.assertTrue(buf[offset] == Type.BINARY.byteValue());
     op = Op.valueOf(buf[offset + 1]);
     left = Utils.bytes2Expression(buf, offset + 2);
     right = Utils.bytes2Expression(buf, offset + 2 + left.length());

http://git-wip-us.apache.org/repos/asf/incubator-ratis/blob/43ad0def/ratis-examples/src/main/java/org/apache/ratis/examples/arithmetic/expression/DoubleValue.java
----------------------------------------------------------------------
diff --git a/ratis-examples/src/main/java/org/apache/ratis/examples/arithmetic/expression/DoubleValue.java b/ratis-examples/src/main/java/org/apache/ratis/examples/arithmetic/expression/DoubleValue.java
index 1753186..66862f0 100644
--- a/ratis-examples/src/main/java/org/apache/ratis/examples/arithmetic/expression/DoubleValue.java
+++ b/ratis-examples/src/main/java/org/apache/ratis/examples/arithmetic/expression/DoubleValue.java
@@ -17,7 +17,7 @@
  */
 package org.apache.ratis.examples.arithmetic.expression;
 
-import org.apache.ratis.util.RaftUtils;
+import org.apache.ratis.util.Preconditions;
 
 import java.util.Map;
 
@@ -33,12 +33,12 @@ public class DoubleValue implements Expression {
 
   DoubleValue(byte[] buf, int offset) {
     this(Utils.bytes2double(buf, offset + 1));
-    RaftUtils.assertTrue(buf[offset] == Type.DOUBLE.byteValue());
+    Preconditions.assertTrue(buf[offset] == Type.DOUBLE.byteValue());
   }
 
   @Override
   public int toBytes(byte[] buf, int offset) {
-    RaftUtils.assertTrue(offset + length() <= buf.length);
+    Preconditions.assertTrue(offset + length() <= buf.length);
     buf[offset++] = Type.DOUBLE.byteValue();
     Utils.double2bytes(value, buf, offset);
     return length();

http://git-wip-us.apache.org/repos/asf/incubator-ratis/blob/43ad0def/ratis-examples/src/main/java/org/apache/ratis/examples/arithmetic/expression/Expression.java
----------------------------------------------------------------------
diff --git a/ratis-examples/src/main/java/org/apache/ratis/examples/arithmetic/expression/Expression.java b/ratis-examples/src/main/java/org/apache/ratis/examples/arithmetic/expression/Expression.java
index 2d5a490..12818f7 100644
--- a/ratis-examples/src/main/java/org/apache/ratis/examples/arithmetic/expression/Expression.java
+++ b/ratis-examples/src/main/java/org/apache/ratis/examples/arithmetic/expression/Expression.java
@@ -21,7 +21,7 @@ import static org.apache.ratis.util.ProtoUtils.toByteString;
 
 import org.apache.ratis.examples.arithmetic.Evaluable;
 import org.apache.ratis.protocol.Message;
-import org.apache.ratis.util.RaftUtils;
+import org.apache.ratis.util.Preconditions;
 
 public interface Expression extends Evaluable {
   enum Type {
@@ -34,8 +34,8 @@ public interface Expression extends Evaluable {
     private static final Type[] VALUES = Type.values();
 
     static Type valueOf(byte b) {
-      RaftUtils.assertTrue(b >= 0);
-      RaftUtils.assertTrue(b < VALUES.length);
+      Preconditions.assertTrue(b >= 0);
+      Preconditions.assertTrue(b < VALUES.length);
       return VALUES[b];
     }
   }
@@ -49,7 +49,7 @@ public interface Expression extends Evaluable {
       return () -> {
         final byte[] buf = new byte[e.length()];
         final int length = e.toBytes(buf, 0);
-        RaftUtils.assertTrue(length == buf.length);
+        Preconditions.assertTrue(length == buf.length);
         return toByteString(buf);
       };
     }

http://git-wip-us.apache.org/repos/asf/incubator-ratis/blob/43ad0def/ratis-examples/src/main/java/org/apache/ratis/examples/arithmetic/expression/NullValue.java
----------------------------------------------------------------------
diff --git a/ratis-examples/src/main/java/org/apache/ratis/examples/arithmetic/expression/NullValue.java b/ratis-examples/src/main/java/org/apache/ratis/examples/arithmetic/expression/NullValue.java
index 9167839..6f589fa 100644
--- a/ratis-examples/src/main/java/org/apache/ratis/examples/arithmetic/expression/NullValue.java
+++ b/ratis-examples/src/main/java/org/apache/ratis/examples/arithmetic/expression/NullValue.java
@@ -17,7 +17,7 @@
  */
 package org.apache.ratis.examples.arithmetic.expression;
 
-import org.apache.ratis.util.RaftUtils;
+import org.apache.ratis.util.Preconditions;
 
 import java.util.Map;
 
@@ -33,7 +33,7 @@ public class NullValue implements Expression {
 
   @Override
   public int toBytes(byte[] buf, int offset) {
-    RaftUtils.assertTrue(offset + length() <= buf.length);
+    Preconditions.assertTrue(offset + length() <= buf.length);
     buf[offset++] = Type.NULL.byteValue();
     return length();
   }

http://git-wip-us.apache.org/repos/asf/incubator-ratis/blob/43ad0def/ratis-examples/src/main/java/org/apache/ratis/examples/arithmetic/expression/UnaryExpression.java
----------------------------------------------------------------------
diff --git a/ratis-examples/src/main/java/org/apache/ratis/examples/arithmetic/expression/UnaryExpression.java b/ratis-examples/src/main/java/org/apache/ratis/examples/arithmetic/expression/UnaryExpression.java
index 2311064..abe329d 100644
--- a/ratis-examples/src/main/java/org/apache/ratis/examples/arithmetic/expression/UnaryExpression.java
+++ b/ratis-examples/src/main/java/org/apache/ratis/examples/arithmetic/expression/UnaryExpression.java
@@ -17,7 +17,7 @@
  */
 package org.apache.ratis.examples.arithmetic.expression;
 
-import org.apache.ratis.util.RaftUtils;
+import org.apache.ratis.util.Preconditions;
 
 import java.util.Map;
 
@@ -43,7 +43,7 @@ public class UnaryExpression implements Expression {
     static final Op[] VALUES = Op.values();
 
     static Op valueOf(byte b) {
-      RaftUtils.assertTrue(b < VALUES.length);
+      Preconditions.assertTrue(b < VALUES.length);
       return VALUES[b];
     }
   }
@@ -52,7 +52,7 @@ public class UnaryExpression implements Expression {
   final Expression expression;
 
   UnaryExpression(byte[] buf, int offset) {
-    RaftUtils.assertTrue(buf[offset] == Type.UNARY.byteValue());
+    Preconditions.assertTrue(buf[offset] == Type.UNARY.byteValue());
     op = Op.valueOf(buf[offset + 1]);
     expression = Utils.bytes2Expression(buf, offset + 2);
   }

http://git-wip-us.apache.org/repos/asf/incubator-ratis/blob/43ad0def/ratis-examples/src/main/java/org/apache/ratis/examples/arithmetic/expression/Variable.java
----------------------------------------------------------------------
diff --git a/ratis-examples/src/main/java/org/apache/ratis/examples/arithmetic/expression/Variable.java b/ratis-examples/src/main/java/org/apache/ratis/examples/arithmetic/expression/Variable.java
index 4c5608b..248b24a 100644
--- a/ratis-examples/src/main/java/org/apache/ratis/examples/arithmetic/expression/Variable.java
+++ b/ratis-examples/src/main/java/org/apache/ratis/examples/arithmetic/expression/Variable.java
@@ -21,7 +21,7 @@ import java.util.Map;
 import java.util.regex.Pattern;
 
 import org.apache.ratis.examples.arithmetic.AssignmentMessage;
-import org.apache.ratis.util.RaftUtils;
+import org.apache.ratis.util.Preconditions;
 
 public class Variable implements Expression {
   static final int LENGTH_LIMIT = 32;
@@ -38,7 +38,7 @@ public class Variable implements Expression {
   }
 
   static String extractString(byte[] buf, int offset) {
-    RaftUtils.assertTrue(buf[offset] == Type.VARIABLE.byteValue());
+    Preconditions.assertTrue(buf[offset] == Type.VARIABLE.byteValue());
     final int length = buf[offset + 1];
     final byte[] stringBytes = new byte[length];
     System.arraycopy(buf, offset + 2, stringBytes, 0, length);
@@ -46,7 +46,7 @@ public class Variable implements Expression {
   }
 
   static byte[] copyBytes(byte[] buf, int offset) {
-    RaftUtils.assertTrue(buf[offset] == Type.VARIABLE.byteValue());
+    Preconditions.assertTrue(buf[offset] == Type.VARIABLE.byteValue());
     final int length = buf[offset + 1];
     final byte[] copy = new byte[length + 2];
     System.arraycopy(buf, offset, copy, 0, copy.length);

http://git-wip-us.apache.org/repos/asf/incubator-ratis/blob/43ad0def/ratis-examples/src/test/java/org/apache/ratis/TestBatchAppend.java
----------------------------------------------------------------------
diff --git a/ratis-examples/src/test/java/org/apache/ratis/TestBatchAppend.java b/ratis-examples/src/test/java/org/apache/ratis/TestBatchAppend.java
index cc69984..b494b67 100644
--- a/ratis-examples/src/test/java/org/apache/ratis/TestBatchAppend.java
+++ b/ratis-examples/src/test/java/org/apache/ratis/TestBatchAppend.java
@@ -21,6 +21,7 @@ import org.apache.log4j.Level;
 import org.apache.ratis.RaftTestUtil.SimpleMessage;
 import org.apache.ratis.client.RaftClient;
 import org.apache.ratis.conf.RaftProperties;
+import org.apache.ratis.util.LogUtils;
 import org.apache.ratis.util.SizeInBytes;
 import org.apache.ratis.examples.RaftExamplesTestUtil;
 import org.apache.ratis.protocol.RaftPeerId;
@@ -29,7 +30,6 @@ import org.apache.ratis.server.impl.RaftServerImpl;
 import org.apache.ratis.server.simulation.RequestHandler;
 import org.apache.ratis.statemachine.SimpleStateMachine4Testing;
 import org.apache.ratis.statemachine.StateMachine;
-import org.apache.ratis.util.RaftUtils;
 import org.junit.After;
 import org.junit.Assert;
 import org.junit.Rule;
@@ -56,9 +56,9 @@ import java.util.stream.Stream;
 public class TestBatchAppend {
   static Logger LOG = LoggerFactory.getLogger(TestBatchAppend.class);
   static {
-    RaftUtils.setLogLevel(RaftServerImpl.LOG, Level.DEBUG);
-    RaftUtils.setLogLevel(RequestHandler.LOG, Level.DEBUG);
-    RaftUtils.setLogLevel(RaftClient.LOG, Level.DEBUG);
+    LogUtils.setLogLevel(RaftServerImpl.LOG, Level.DEBUG);
+    LogUtils.setLogLevel(RequestHandler.LOG, Level.DEBUG);
+    LogUtils.setLogLevel(RaftClient.LOG, Level.DEBUG);
   }
 
   @Parameterized.Parameters

http://git-wip-us.apache.org/repos/asf/incubator-ratis/blob/43ad0def/ratis-examples/src/test/java/org/apache/ratis/TestRestartRaftPeer.java
----------------------------------------------------------------------
diff --git a/ratis-examples/src/test/java/org/apache/ratis/TestRestartRaftPeer.java b/ratis-examples/src/test/java/org/apache/ratis/TestRestartRaftPeer.java
index 34aec44..346d7c0 100644
--- a/ratis-examples/src/test/java/org/apache/ratis/TestRestartRaftPeer.java
+++ b/ratis-examples/src/test/java/org/apache/ratis/TestRestartRaftPeer.java
@@ -21,6 +21,7 @@ import org.apache.log4j.Level;
 import org.apache.ratis.RaftTestUtil.SimpleMessage;
 import org.apache.ratis.client.RaftClient;
 import org.apache.ratis.conf.RaftProperties;
+import org.apache.ratis.util.LogUtils;
 import org.apache.ratis.util.SizeInBytes;
 import org.apache.ratis.examples.RaftExamplesTestUtil;
 import org.apache.ratis.protocol.RaftPeerId;
@@ -30,7 +31,6 @@ import org.apache.ratis.server.simulation.RequestHandler;
 import org.apache.ratis.server.storage.RaftLog;
 import org.apache.ratis.statemachine.SimpleStateMachine4Testing;
 import org.apache.ratis.statemachine.StateMachine;
-import org.apache.ratis.util.RaftUtils;
 import org.junit.Assert;
 import org.junit.Rule;
 import org.junit.Test;
@@ -51,10 +51,10 @@ import java.util.Collection;
 public class TestRestartRaftPeer {
   static Logger LOG = LoggerFactory.getLogger(TestRestartRaftPeer.class);
   static {
-    RaftUtils.setLogLevel(RaftServerImpl.LOG, Level.DEBUG);
-    RaftUtils.setLogLevel(RaftLog.LOG, Level.DEBUG);
-    RaftUtils.setLogLevel(RequestHandler.LOG, Level.DEBUG);
-    RaftUtils.setLogLevel(RaftClient.LOG, Level.DEBUG);
+    LogUtils.setLogLevel(RaftServerImpl.LOG, Level.DEBUG);
+    LogUtils.setLogLevel(RaftLog.LOG, Level.DEBUG);
+    LogUtils.setLogLevel(RequestHandler.LOG, Level.DEBUG);
+    LogUtils.setLogLevel(RaftClient.LOG, Level.DEBUG);
   }
 
   @Parameterized.Parameters

http://git-wip-us.apache.org/repos/asf/incubator-ratis/blob/43ad0def/ratis-examples/src/test/java/org/apache/ratis/examples/arithmetic/TestArithmetic.java
----------------------------------------------------------------------
diff --git a/ratis-examples/src/test/java/org/apache/ratis/examples/arithmetic/TestArithmetic.java b/ratis-examples/src/test/java/org/apache/ratis/examples/arithmetic/TestArithmetic.java
index f4bcd0a..5ffb70f 100644
--- a/ratis-examples/src/test/java/org/apache/ratis/examples/arithmetic/TestArithmetic.java
+++ b/ratis-examples/src/test/java/org/apache/ratis/examples/arithmetic/TestArithmetic.java
@@ -26,7 +26,7 @@ import org.apache.ratis.examples.RaftExamplesTestUtil;
 import org.apache.ratis.examples.arithmetic.expression.*;
 import org.apache.ratis.protocol.RaftClientReply;
 import org.apache.ratis.protocol.RaftPeerId;
-import org.apache.ratis.util.RaftUtils;
+import org.apache.ratis.util.LogUtils;
 import org.junit.Assert;
 import org.junit.Test;
 import org.junit.runner.RunWith;
@@ -38,7 +38,7 @@ import java.util.Collection;
 @RunWith(Parameterized.class)
 public class TestArithmetic {
   static {
-    RaftUtils.setLogLevel(ArithmeticStateMachine.LOG, Level.ALL);
+    LogUtils.setLogLevel(ArithmeticStateMachine.LOG, Level.ALL);
   }
 
   @Parameterized.Parameters

http://git-wip-us.apache.org/repos/asf/incubator-ratis/blob/43ad0def/ratis-examples/src/test/java/org/apache/ratis/statemachine/TestRaftStateMachineException.java
----------------------------------------------------------------------
diff --git a/ratis-examples/src/test/java/org/apache/ratis/statemachine/TestRaftStateMachineException.java b/ratis-examples/src/test/java/org/apache/ratis/statemachine/TestRaftStateMachineException.java
index 447f2ea..c3b0184 100644
--- a/ratis-examples/src/test/java/org/apache/ratis/statemachine/TestRaftStateMachineException.java
+++ b/ratis-examples/src/test/java/org/apache/ratis/statemachine/TestRaftStateMachineException.java
@@ -28,7 +28,7 @@ import org.apache.ratis.protocol.StateMachineException;
 import org.apache.ratis.server.impl.RaftServerImpl;
 import org.apache.ratis.server.simulation.RequestHandler;
 import org.apache.ratis.server.storage.RaftLog;
-import org.apache.ratis.util.RaftUtils;
+import org.apache.ratis.util.LogUtils;
 import org.junit.Assert;
 import org.junit.Test;
 import org.junit.runner.RunWith;
@@ -43,10 +43,10 @@ import static org.junit.Assert.fail;
 @RunWith(Parameterized.class)
 public class TestRaftStateMachineException {
   static {
-    RaftUtils.setLogLevel(RaftServerImpl.LOG, Level.DEBUG);
-    RaftUtils.setLogLevel(RaftLog.LOG, Level.DEBUG);
-    RaftUtils.setLogLevel(RequestHandler.LOG, Level.DEBUG);
-    RaftUtils.setLogLevel(RaftClient.LOG, Level.DEBUG);
+    LogUtils.setLogLevel(RaftServerImpl.LOG, Level.DEBUG);
+    LogUtils.setLogLevel(RaftLog.LOG, Level.DEBUG);
+    LogUtils.setLogLevel(RequestHandler.LOG, Level.DEBUG);
+    LogUtils.setLogLevel(RaftClient.LOG, Level.DEBUG);
   }
 
   protected static class StateMachineWithException extends SimpleStateMachine4Testing {

http://git-wip-us.apache.org/repos/asf/incubator-ratis/blob/43ad0def/ratis-grpc/src/main/java/org/apache/ratis/grpc/RaftGrpcUtil.java
----------------------------------------------------------------------
diff --git a/ratis-grpc/src/main/java/org/apache/ratis/grpc/RaftGrpcUtil.java b/ratis-grpc/src/main/java/org/apache/ratis/grpc/RaftGrpcUtil.java
index fb3cc11..b89c297 100644
--- a/ratis-grpc/src/main/java/org/apache/ratis/grpc/RaftGrpcUtil.java
+++ b/ratis-grpc/src/main/java/org/apache/ratis/grpc/RaftGrpcUtil.java
@@ -20,7 +20,8 @@ package org.apache.ratis.grpc;
 import org.apache.ratis.shaded.io.grpc.Metadata;
 import org.apache.ratis.shaded.io.grpc.Status;
 import org.apache.ratis.shaded.io.grpc.StatusRuntimeException;
-import org.apache.ratis.util.RaftUtils;
+import org.apache.ratis.util.IOUtils;
+import org.apache.ratis.util.ReflectionUtils;
 import org.apache.ratis.util.StringUtils;
 
 import java.io.IOException;
@@ -45,9 +46,9 @@ public class RaftGrpcUtil {
       if (className != null) {
         try {
           Class<?> clazz = Class.forName(className);
-          final Exception unwrapped = RaftUtils.instantiateException(
+          final Exception unwrapped = ReflectionUtils.instantiateException(
               clazz.asSubclass(Exception.class), status.getDescription(), se);
-          return RaftUtils.asIOException(unwrapped);
+          return IOUtils.asIOException(unwrapped);
         } catch (Exception e) {
           return new IOException(se);
         }
@@ -61,7 +62,7 @@ public class RaftGrpcUtil {
     if (t instanceof StatusRuntimeException) {
       e = RaftGrpcUtil.unwrapException((StatusRuntimeException) t);
     } else {
-      e = RaftUtils.asIOException(t);
+      e = IOUtils.asIOException(t);
     }
     return e;
   }

http://git-wip-us.apache.org/repos/asf/incubator-ratis/blob/43ad0def/ratis-grpc/src/main/java/org/apache/ratis/grpc/client/AppendStreamer.java
----------------------------------------------------------------------
diff --git a/ratis-grpc/src/main/java/org/apache/ratis/grpc/client/AppendStreamer.java b/ratis-grpc/src/main/java/org/apache/ratis/grpc/client/AppendStreamer.java
index d42f4e1..c92820c 100644
--- a/ratis-grpc/src/main/java/org/apache/ratis/grpc/client/AppendStreamer.java
+++ b/ratis-grpc/src/main/java/org/apache/ratis/grpc/client/AppendStreamer.java
@@ -18,6 +18,8 @@
 package org.apache.ratis.grpc.client;
 
 import org.apache.ratis.conf.RaftProperties;
+import org.apache.ratis.util.CollectionUtils;
+import org.apache.ratis.util.Preconditions;
 import org.apache.ratis.util.TimeDuration;
 import org.apache.ratis.grpc.GrpcConfigKeys;
 import org.apache.ratis.grpc.RaftGrpcUtil;
@@ -28,7 +30,6 @@ import org.apache.ratis.shaded.proto.RaftProtos.RaftClientRequestProto;
 import org.apache.ratis.shaded.proto.RaftProtos.RaftRpcRequestProto;
 import org.apache.ratis.util.Daemon;
 import org.apache.ratis.util.PeerProxyMap;
-import org.apache.ratis.util.RaftUtils;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
@@ -114,7 +115,7 @@ public class AppendStreamer implements Closeable {
       if (oldLeader == null) {
         leaderId = peers.keySet().iterator().next();
       } else {
-        leaderId = RaftUtils.next(oldLeader, peers.keySet());
+        leaderId = CollectionUtils.next(oldLeader, peers.keySet());
       }
     }
     LOG.debug("{} switches leader from {} to {}. suggested leader: {}", this,
@@ -257,7 +258,7 @@ public class AppendStreamer implements Closeable {
         RaftClientRequestProto pending = Objects.requireNonNull(
             ackQueue.peek());
         if (reply.getRpcReply().getSuccess()) {
-          RaftUtils.assertTrue(pending.getRpcRequest().getCallId() ==
+          Preconditions.assertTrue(pending.getRpcRequest().getCallId() ==
               reply.getRpcReply().getCallId());
           ackQueue.poll();
           LOG.trace("{} received success ack for request {}", this,
@@ -311,7 +312,7 @@ public class AppendStreamer implements Closeable {
 
   private void handleNotLeader(NotLeaderException nle,
       RaftPeerId oldLeader) {
-    RaftUtils.assertTrue(Thread.holdsLock(AppendStreamer.this));
+    Preconditions.assertTrue(Thread.holdsLock(AppendStreamer.this));
     // handle NotLeaderException: refresh leader and RaftConfiguration
     refreshPeers(nle.getPeers());
 
@@ -319,7 +320,7 @@ public class AppendStreamer implements Closeable {
   }
 
   private void handleError(Throwable t, ResponseHandler handler) {
-    RaftUtils.assertTrue(Thread.holdsLock(AppendStreamer.this));
+    Preconditions.assertTrue(Thread.holdsLock(AppendStreamer.this));
     final IOException e = RaftGrpcUtil.unwrapIOException(t);
 
     exceptionAndRetry.addException(handler.targetId, e);

http://git-wip-us.apache.org/repos/asf/incubator-ratis/blob/43ad0def/ratis-grpc/src/main/java/org/apache/ratis/grpc/client/GrpcClientRpc.java
----------------------------------------------------------------------
diff --git a/ratis-grpc/src/main/java/org/apache/ratis/grpc/client/GrpcClientRpc.java b/ratis-grpc/src/main/java/org/apache/ratis/grpc/client/GrpcClientRpc.java
index 3f7343a..b28415c 100644
--- a/ratis-grpc/src/main/java/org/apache/ratis/grpc/client/GrpcClientRpc.java
+++ b/ratis-grpc/src/main/java/org/apache/ratis/grpc/client/GrpcClientRpc.java
@@ -25,8 +25,8 @@ import org.apache.ratis.shaded.io.grpc.stub.StreamObserver;
 import org.apache.ratis.shaded.proto.RaftProtos.RaftClientReplyProto;
 import org.apache.ratis.shaded.proto.RaftProtos.RaftClientRequestProto;
 import org.apache.ratis.shaded.proto.RaftProtos.SetConfigurationRequestProto;
+import org.apache.ratis.util.IOUtils;
 import org.apache.ratis.util.PeerProxyMap;
-import org.apache.ratis.util.RaftUtils;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
@@ -71,7 +71,7 @@ public class GrpcClientRpc implements RaftClientRpc {
               if (t instanceof StatusRuntimeException) {
                 e = RaftGrpcUtil.unwrapException((StatusRuntimeException) t);
               } else {
-                e = RaftUtils.asIOException(t);
+                e = IOUtils.asIOException(t);
               }
               replyFuture.completeExceptionally(e);
             }
@@ -94,7 +94,7 @@ public class GrpcClientRpc implements RaftClientRpc {
         throw new InterruptedIOException(
             "Interrupted while waiting for response of request " + request);
       } catch (ExecutionException e) {
-        throw RaftUtils.toIOException(e);
+        throw IOUtils.toIOException(e);
       }
     }
   }

http://git-wip-us.apache.org/repos/asf/incubator-ratis/blob/43ad0def/ratis-grpc/src/main/java/org/apache/ratis/grpc/client/RaftClientProtocolService.java
----------------------------------------------------------------------
diff --git a/ratis-grpc/src/main/java/org/apache/ratis/grpc/client/RaftClientProtocolService.java b/ratis-grpc/src/main/java/org/apache/ratis/grpc/client/RaftClientProtocolService.java
index 5ebeb66..97e32c1 100644
--- a/ratis-grpc/src/main/java/org/apache/ratis/grpc/client/RaftClientProtocolService.java
+++ b/ratis-grpc/src/main/java/org/apache/ratis/grpc/client/RaftClientProtocolService.java
@@ -27,7 +27,7 @@ import org.apache.ratis.client.impl.ClientProtoUtils;
 import org.apache.ratis.grpc.RaftGrpcUtil;
 import org.apache.ratis.protocol.RaftClientAsynchronousProtocol;
 import org.apache.ratis.protocol.RaftClientReply;
-import org.apache.ratis.util.RaftUtils;
+import org.apache.ratis.util.Preconditions;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
@@ -126,14 +126,14 @@ public class RaftClientProtocolService extends RaftClientProtocolServiceImplBase
           } else {
             final long replySeq = reply.getCallId();
             synchronized (pendingList) {
-              RaftUtils.assertTrue(!pendingList.isEmpty(),
+              Preconditions.assertTrue(!pendingList.isEmpty(),
                   "PendingList is empty when handling onNext for callId %s",
                   replySeq);
               final long headSeqNum = pendingList.get(0).callId;
               // we assume the callId is consecutive for a stream RPC call
               final PendingAppend pendingForReply = pendingList.get(
                   (int) (replySeq - headSeqNum));
-              RaftUtils.assertTrue(pendingForReply != null &&
+              Preconditions.assertTrue(pendingForReply != null &&
                       pendingForReply.callId == replySeq,
                   "pending for reply is: %s, the pending list: %s",
                   pendingForReply, pendingList);
@@ -162,7 +162,7 @@ public class RaftClientProtocolService extends RaftClientProtocolServiceImplBase
 
     private void sendReadyReplies(Collection<PendingAppend> readySet) {
       readySet.forEach(ready -> {
-        RaftUtils.assertTrue(ready.isReady());
+        Preconditions.assertTrue(ready.isReady());
         if (ready == COMPLETED) {
           responseObserver.onCompleted();
         } else {