From 43bc6ebf4f415ecf23317ef4c38fa6c58afde29a Mon Sep 17 00:00:00 2001 From: Enis Soztutar Date: Mon, 10 Apr 2017 00:32:17 -0700 Subject: [PATCH] RATIS-52 Refactor RaftUtils into different classes --- .../ratis/client/impl/ClientProtoUtils.java | 4 +- .../ratis/client/impl/RaftClientImpl.java | 7 +- .../java/org/apache/ratis/RaftConfigKeys.java | 12 +- .../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 +- .../apache/ratis/rpc/SupportedRpcType.java | 6 +- .../ratis/util/AtomicFileOutputStream.java | 2 +- .../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 ++ .../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 ------------------ .../apache/ratis/util/ReflectionUtils.java | 136 ++++++ .../arithmetic/ArithmeticStateMachine.java | 4 +- .../expression/BinaryExpression.java | 6 +- .../arithmetic/expression/DoubleValue.java | 6 +- .../arithmetic/expression/Expression.java | 8 +- .../arithmetic/expression/NullValue.java | 4 +- .../expression/UnaryExpression.java | 6 +- .../arithmetic/expression/Variable.java | 6 +- .../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 +- .../ratis/grpc/client/GrpcClientRpc.java | 6 +- .../client/RaftClientProtocolService.java | 8 +- .../ratis/grpc/server/GRpcLogAppender.java | 12 +- .../grpc/TestRaftReconfigurationWithGRpc.java | 4 +- .../org/apache/ratis/grpc/TestRaftStream.java | 4 +- .../apache/ratis/grpc/TestRaftWithGrpc.java | 4 +- .../ratis/grpc/TestRetryCacheWithGrpc.java | 8 +- .../TestRaftReconfigurationWithHadoopRpc.java | 2 - .../hadooprpc/TestRaftWithHadoopRpc.java | 8 +- .../TestRetryCacheWithHadoopRpc.java | 8 +- .../org/apache/ratis/netty/NettyRpcProxy.java | 8 +- .../apache/ratis/netty/TestRaftWithNetty.java | 6 +- .../netty/TestRetryCacheWithNettyRpc.java | 10 +- .../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 | 82 ++-- .../apache/ratis/server/impl/RetryCache.java | 14 +- .../apache/ratis/server/impl/ServerState.java | 4 +- .../server/impl/StateMachineUpdater.java | 8 +- .../ratis/server/storage/LogInputStream.java | 19 +- .../ratis/server/storage/LogOutputStream.java | 6 +- .../ratis/server/storage/LogReader.java | 15 +- .../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 +- .../server/storage/SegmentedRaftLog.java | 6 +- .../ratis/server/storage/SnapshotManager.java | 7 +- .../org/apache/ratis/MiniRaftCluster.java | 41 +- .../ratis/RaftNotLeaderExceptionBaseTest.java | 10 +- .../impl/RaftReconfigurationBaseTest.java | 48 ++- .../simulation/SimulatedRequestReply.java | 17 +- .../simulation/TestRaftWithSimulatedRpc.java | 8 +- .../TestRetryCacheWithSimulatedRpc.java | 10 +- .../server/storage/TestSegmentedRaftLog.java | 4 +- .../statemachine/RaftSnapshotBaseTest.java | 10 +- .../SimpleStateMachine4Testing.java | 4 +- .../ratis/statemachine/TermIndexTracker.java | 4 +- .../ratis/statemachine/TestStateMachine.java | 6 +- 80 files changed, 926 insertions(+), 731 deletions(-) create mode 100644 ratis-common/src/main/java/org/apache/ratis/util/CollectionUtils.java create mode 100644 ratis-common/src/main/java/org/apache/ratis/util/IOUtils.java create mode 100644 ratis-common/src/main/java/org/apache/ratis/util/LogUtils.java create mode 100644 ratis-common/src/main/java/org/apache/ratis/util/PlatformUtils.java create mode 100644 ratis-common/src/main/java/org/apache/ratis/util/Preconditions.java delete mode 100644 ratis-common/src/main/java/org/apache/ratis/util/RaftUtils.java create mode 100644 ratis-common/src/main/java/org/apache/ratis/util/ReflectionUtils.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 ddecad681f..a6e99c6f02 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.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 @@ private static StateMachineException wrapStateMachineException( } 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) { 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 eae42a511a..2125ce0fa3 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 @@ 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 @@ private RaftClientReply sendRequestWithRetry( retryInterval.sleep(); } catch (InterruptedException ie) { Thread.currentThread().interrupt(); - throw RaftUtils.toInterruptedIOException( + throw IOUtils.toInterruptedIOException( "Interrupted when sending " + request, ie); } } @@ -158,7 +159,7 @@ private void handleIOException(RaftClientRequest request, IOException ioe, 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); 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 b67acafacc..665a7f5760 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 @@ static RpcType type(RaftProperties properties) { } // 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) { 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 e096e84cc8..921fb32cd4 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 @@ private static void assertCodeLoaded() throws IOException { 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 static class Stat { } 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 @@ private static String stripDomain(String name) { */ 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 static FileInputStream getShareDeleteFileInputStream(File f) */ 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 static FileInputStream getShareDeleteFileInputStream(File f, long seekOff */ 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 @@ private static native void link0(String src, String dst) * @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 static void copyFileUnbuffered(File src, File dst) throws IOException { position += transferred; } } finally { - RaftUtils.cleanup(LOG, output, fos, input, fis); + IOUtils.cleanup(LOG, output, fos, input, fis); } } } 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 58b83e7515..15f69cfaae 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 @@ import java.io.IOException; -import org.apache.ratis.util.RaftUtils; +import org.apache.ratis.util.PlatformUtils; /** @@ -61,7 +61,7 @@ public Errno getErrno() { @Override public String toString() { - if (RaftUtils.WINDOWS) + if (PlatformUtils.WINDOWS) return errorCode + ": " + super.getMessage(); else return errno.toString() + ": " + super.getMessage(); 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 7e15ee3b7a..310e2078c7 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 @@ private ClientId(UUID id) { 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()); 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 92d984619e..dff355ed96 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 static RaftPeerId getRaftPeerId(String id) { 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); } 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 f1d8fac6b4..92f70f61f8 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 @@ 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,8 @@ public static SupportedRpcType valueOfIgnoreCase(String s) { @Override public RpcFactory newFactory(RaftProperties properties, Parameters parameters) { - final Class clazz = RaftUtils.getClass( + final Class clazz = ReflectionUtils.getClass( factoryClassName, properties, RpcFactory.class); - return RaftUtils.newInstance(clazz, ARG_CLASSES, parameters); + return ReflectionUtils.newInstance(clazz, ARG_CLASSES, parameters); } } 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 e8e267e561..e181e4487f 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 void close() throws IOException { } 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()) { 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 0000000000..05fa2fb204 --- /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 next(final T given, final Iterable iteration) { + Objects.requireNonNull(given, "given == null"); + final Iterator 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 Iterable as( + Iterable iteration, Function converter) { + return () -> new Iterator() { + final Iterator i = iteration.iterator(); + @Override + public boolean hasNext() { + return i.hasNext(); + } + + @Override + public OUTPUT next() { + return converter.apply(i.next()); + } + }; + } +} 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 5dc509d526..732e4c8fff 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 static File[] listFiles(File dir) throws IOException { * 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); 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 0000000000..ba5e78eecb --- /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 ignore 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); + } + } + } + } + } +} 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 166f0dc73d..5246abab3c 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. *
@@ -104,7 +108,7 @@ static void validate(Object name, State from, State to) {
         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 final  void startAndTransition(
       startImpl.run();
       transition(State.RUNNING);
     } catch (Throwable t) {
-      transition(RaftUtils.isInstance(t, exceptionClasses)?
+      transition(ReflectionUtils.isInstance(t, exceptionClasses)?
           State.NEW: State.EXCEPTION);
       throw t;
     }
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 0000000000..addc2ff1db
--- /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);
+  }
+
+}
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 35c7136535..a5ded8c599 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 @@ private static Matcher readStoredMd5(File md5File) throws IOException {
     } 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 static MD5Hash computeMd5ForFile(File dataFile) throws IOException {
     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);
     }
   }
 
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 0000000000..6d4bf54b4f
--- /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 os.name 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
+  }
+}
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 0000000000..f1d55b0535
--- /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 message) {
+    if (!value) {
+      throw new IllegalStateException(String.valueOf(message.get()));
+    }
+  }
+}
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 ae3e857ad0..0000000000
--- 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 os.name 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>, 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 newInstance(Class 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  class type of clazz
-   * @return a new object
-   */
-  public static  T newInstance(Class clazz, Class[] argClasses, Object... args) {
-    Objects.requireNonNull(clazz, "clazz == null");
-    try {
-      final List> key = new ArrayList<>();
-      key.add(clazz);
-      key.addAll(Arrays.asList(argClasses));
-
-      @SuppressWarnings("unchecked")
-      Constructor ctor = (Constructor) 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  Class getClass(
-      String subClassName, RaftProperties properties, Class 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  The supplier result type.
-   * @return a memoized supplier which is thread-safe.
-   */
-  public static  Supplier memoize(Supplier initializer) {
-    Objects.requireNonNull(initializer, "initializer == null");
-    return new Supplier() {
-      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 ignore 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 next(final T given, final Iterable iteration) {
-    Objects.requireNonNull(given, "given == null");
-    final Iterator 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  Iterable as(
-      Iterable iteration, Function converter) {
-    return () -> new Iterator() {
-      final Iterator 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 message) {
-    if (!value) {
-      throw new IllegalStateException(String.valueOf(message.get()));
-    }
-  }
-
-  public static Exception instantiateException(Class cls,
-      String message, Exception from) throws Exception {
-    Constructor cn = cls.getConstructor(String.class);
-    cn.setAccessible(true);
-    Exception ex = cn.newInstance(message);
-    if (from != null) {
-      ex.initCause(from);
-    }
-    return ex;
-  }
-}
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 0000000000..d0762c3168
--- /dev/null
+++ b/ratis-common/src/main/java/org/apache/ratis/util/ReflectionUtils.java
@@ -0,0 +1,136 @@
+/*
+ * *
+ *  * Licensed to the Apache Software Foundation (ASF) under one
+ *  * or more contributor license agreements.  See the NOTICE file
+ *  * distributed with this work for additional information
+ *  * regarding copyright ownership.  The ASF licenses this file
+ *  * to you under the Apache License, Version 2.0 (the
+ *  * "License"); you may not use this file except in compliance
+ *  * with the License.  You may obtain a copy of the License at
+ *  *
+ *  *     http://www.apache.org/licenses/LICENSE-2.0
+ *  *
+ *  * Unless required by applicable law or agreed to in writing, software
+ *  * distributed under the License is distributed on an "AS IS" BASIS,
+ *  * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ *  * See the License for the specific language governing permissions and
+ *  * limitations under the License.
+ *
+ */
+
+package org.apache.ratis.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 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>, Constructor> CONSTRUCTOR_CACHE
+      = new ConcurrentHashMap<>();
+
+  /**
+   * Create an object for the given class using its default constructor.
+   */
+  public static  T newInstance(Class 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  class type of clazz
+   * @return a new object
+   */
+  public static  T newInstance(Class clazz, Class[] argClasses, Object... args) {
+    Objects.requireNonNull(clazz, "clazz == null");
+    try {
+      final List> key = new ArrayList<>();
+      key.add(clazz);
+      key.addAll(Arrays.asList(argClasses));
+
+      @SuppressWarnings("unchecked")
+      Constructor ctor = (Constructor) 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);
+    }
+  }
+
+  private static  T instantiate(final String className, Constructor 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);
+    }
+  }
+
+  /** 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;
+  }
+
+  public static  Class getClass(
+      String subClassName, RaftProperties properties, Class 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 cls,
+      String message, Exception from) throws Exception {
+    Constructor cn = cls.getConstructor(String.class);
+    cn.setAccessible(true);
+    Exception ex = cn.newInstance(message);
+    if (from != null) {
+      ex.initCause(from);
+    }
+    return ex;
+  }
+}
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 89580dfd2a..53e8ab7984 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.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 @@ private void updateLatestTermIndex(long term, long index) {
     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);
     }
   }
 }
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 6486cfd5d9..121bb7a0bc 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 String toString() {
     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 @@ static Op valueOf(byte b) {
   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());
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 175318696a..66862f001a 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 DoubleValue(double value) {
 
   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();
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 2d5a490932..12818f77cf 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 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 @@ byte byteValue() {
     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 static Message toMessage(final Expression e) {
       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);
       };
     }
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 9167839ff3..6f589fadf4 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 @@ private NullValue() {
 
   @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();
   }
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 2311064b92..abe329df59 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 String toString() {
     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 @@ static Op valueOf(byte b) {
   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);
   }
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 4c5608b978..248b24a5de 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.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 @@ static byte[] string2bytes(String s) {
   }
 
   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 @@ static String extractString(byte[] buf, int offset) {
   }
 
   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);
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 cc699842bb..b494b677aa 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.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.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 @@
 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
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 34aec44d7f..346d7c0901 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.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.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 @@
 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
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 f4bcd0adb2..5ffb70fd2e 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.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 @@
 @RunWith(Parameterized.class)
 public class TestArithmetic {
   static {
-    RaftUtils.setLogLevel(ArithmeticStateMachine.LOG, Level.ALL);
+    LogUtils.setLogLevel(ArithmeticStateMachine.LOG, Level.ALL);
   }
 
   @Parameterized.Parameters
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 5ce10e2d46..3dbd24876f 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
@@ -33,7 +33,7 @@
 import org.apache.ratis.server.impl.RetryCache;
 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;
@@ -52,10 +52,10 @@ public class TestRaftStateMachineException {
   public static final Logger LOG = LoggerFactory.getLogger(TestRaftStateMachineException.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);
   }
 
   protected static boolean failPreAppend = false;
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 fb3cc114a4..b89c2974a2 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 @@
 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 static IOException unwrapException(StatusRuntimeException se) {
       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 static IOException unwrapIOException(Throwable t) {
     if (t instanceof StatusRuntimeException) {
       e = RaftGrpcUtil.unwrapException((StatusRuntimeException) t);
     } else {
-      e = RaftUtils.asIOException(t);
+      e = IOUtils.asIOException(t);
     }
     return e;
   }
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 d42f4e1d1b..c92820cc81 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.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 @@ private synchronized void refreshLeaderProxy(RaftPeerId suggested,
       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 void onNext(RaftClientReplyProto reply) {
         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 @@ private void throwException(String msg) throws IOException {
 
   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 @@ private void handleNotLeader(NotLeaderException nle,
   }
 
   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);
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 3f7343a85b..b28415c7c8 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.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 void onError(Throwable t) {
               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 void onCompleted() {
         throw new InterruptedIOException(
             "Interrupted while waiting for response of request " + request);
       } catch (ExecutionException e) {
-        throw RaftUtils.toIOException(e);
+        throw IOUtils.toIOException(e);
       }
     }
   }
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 5ebeb66027..97e32c17ae 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.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 void onNext(RaftClientRequestProto request) {
           } 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 void onNext(RaftClientRequestProto request) {
 
     private void sendReadyReplies(Collection readySet) {
       readySet.forEach(ready -> {
-        RaftUtils.assertTrue(ready.isReady());
+        Preconditions.assertTrue(ready.isReady());
         if (ready == COMPLETED) {
           responseObserver.onCompleted();
         } else {
diff --git a/ratis-grpc/src/main/java/org/apache/ratis/grpc/server/GRpcLogAppender.java b/ratis-grpc/src/main/java/org/apache/ratis/grpc/server/GRpcLogAppender.java
index e29da00c80..656adc204f 100644
--- a/ratis-grpc/src/main/java/org/apache/ratis/grpc/server/GRpcLogAppender.java
+++ b/ratis-grpc/src/main/java/org/apache/ratis/grpc/server/GRpcLogAppender.java
@@ -31,7 +31,7 @@
 import org.apache.ratis.server.impl.RaftServerImpl;
 import org.apache.ratis.statemachine.SnapshotInfo;
 import org.apache.ratis.util.CodeInjectionForTesting;
-import org.apache.ratis.util.RaftUtils;
+import org.apache.ratis.util.Preconditions;
 
 import static org.apache.ratis.grpc.RaftGRpcService.GRPC_SEND_SERVER_REQUEST;
 
@@ -130,7 +130,7 @@ private void appendLog() {
         // together and protected by the lock
         pending = createRequest();
         if (pending != null) {
-          RaftUtils.assertTrue(pendingRequests.offer(pending));
+          Preconditions.assertTrue(pendingRequests.offer(pending));
           updateNextIndex(pending);
         }
       }
@@ -261,7 +261,7 @@ private void onSuccess(AppendEntriesReplyProto reply) {
             + " but the pending queue is empty");
 
     if (request.getEntriesCount() == 0) {
-      RaftUtils.assertTrue(!request.hasPreviousLog() ||
+      Preconditions.assertTrue(!request.hasPreviousLog() ||
               replyNextIndex - 1 == request.getPreviousLog().getIndex(),
           "reply's next index is %s, request's previous is %s",
           replyNextIndex, request.getPreviousLog());
@@ -269,7 +269,7 @@ private void onSuccess(AppendEntriesReplyProto reply) {
       // check if the reply and the pending request is consistent
       final long lastEntryIndex = request
           .getEntries(request.getEntriesCount() - 1).getIndex();
-      RaftUtils.assertTrue(replyNextIndex == lastEntryIndex + 1,
+      Preconditions.assertTrue(replyNextIndex == lastEntryIndex + 1,
           "reply's next index is %s, request's last entry index is %s",
           replyNextIndex, lastEntryIndex);
       follower.updateMatchIndex(lastEntryIndex);
@@ -284,7 +284,7 @@ private void onNotLeader(AppendEntriesReplyProto reply) {
 
   private synchronized void onInconsistency(AppendEntriesReplyProto reply) {
     AppendEntriesRequestProto request = pendingRequests.peek();
-    RaftUtils.assertTrue(request.hasPreviousLog());
+    Preconditions.assertTrue(request.hasPreviousLog());
     if (request.getPreviousLog().getIndex() >= reply.getNextIndex()) {
       clearPendingRequests(reply.getNextIndex());
     }
@@ -305,7 +305,7 @@ synchronized void addPending(InstallSnapshotRequestProto request) {
 
     synchronized void removePending(InstallSnapshotReplyProto reply) {
       int index = pending.poll();
-      RaftUtils.assertTrue(index == reply.getRequestIndex());
+      Preconditions.assertTrue(index == reply.getRequestIndex());
     }
 
     boolean isDone() {
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
index 4f54fa82a9..822b923589 100644
--- a/ratis-grpc/src/test/java/org/apache/ratis/grpc/TestRaftReconfigurationWithGRpc.java
+++ b/ratis-grpc/src/test/java/org/apache/ratis/grpc/TestRaftReconfigurationWithGRpc.java
@@ -20,13 +20,13 @@
 import org.apache.log4j.Level;
 import org.apache.ratis.grpc.server.RaftServerProtocolService;
 import org.apache.ratis.server.impl.RaftReconfigurationBaseTest;
-import org.apache.ratis.util.RaftUtils;
+import org.apache.ratis.util.LogUtils;
 
 import java.io.IOException;
 
 public class TestRaftReconfigurationWithGRpc extends RaftReconfigurationBaseTest {
   static {
-    RaftUtils.setLogLevel(RaftServerProtocolService.LOG, Level.DEBUG);
+    LogUtils.setLogLevel(RaftServerProtocolService.LOG, Level.DEBUG);
   }
 
   @Override
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
index 4bae5a9377..16bc221e15 100644
--- a/ratis-grpc/src/test/java/org/apache/ratis/grpc/TestRaftStream.java
+++ b/ratis-grpc/src/test/java/org/apache/ratis/grpc/TestRaftStream.java
@@ -20,6 +20,7 @@
 import org.apache.log4j.Level;
 import org.apache.ratis.RaftTestUtil;
 import org.apache.ratis.conf.RaftProperties;
+import org.apache.ratis.util.LogUtils;
 import org.apache.ratis.util.SizeInBytes;
 import org.apache.ratis.grpc.client.AppendStreamer;
 import org.apache.ratis.grpc.client.RaftOutputStream;
@@ -27,7 +28,6 @@
 import org.apache.ratis.server.impl.RaftServerImpl;
 import org.apache.ratis.server.storage.RaftLog;
 import org.apache.ratis.shaded.proto.RaftProtos.LogEntryProto;
-import org.apache.ratis.util.RaftUtils;
 import org.junit.After;
 import org.junit.Assert;
 import org.junit.Test;
@@ -47,7 +47,7 @@
 
 public class TestRaftStream {
   static {
-    RaftUtils.setLogLevel(AppendStreamer.LOG, Level.ALL);
+    LogUtils.setLogLevel(AppendStreamer.LOG, Level.ALL);
   }
   static final Logger LOG = LoggerFactory.getLogger(TestRaftStream.class);
 
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
index bfaf819133..d7b72c2fad 100644
--- a/ratis-grpc/src/test/java/org/apache/ratis/grpc/TestRaftWithGrpc.java
+++ b/ratis-grpc/src/test/java/org/apache/ratis/grpc/TestRaftWithGrpc.java
@@ -21,7 +21,7 @@
 import org.apache.ratis.RaftBasicTests;
 import org.apache.ratis.server.impl.BlockRequestHandlingInjection;
 import org.apache.ratis.server.impl.RaftServerImpl;
-import org.apache.ratis.util.RaftUtils;
+import org.apache.ratis.util.LogUtils;
 import org.junit.Assert;
 import org.junit.Test;
 
@@ -29,7 +29,7 @@
 
 public class TestRaftWithGrpc extends RaftBasicTests {
   static {
-    RaftUtils.setLogLevel(RaftServerImpl.LOG, Level.DEBUG);
+    LogUtils.setLogLevel(RaftServerImpl.LOG, Level.DEBUG);
   }
 
   private final MiniRaftClusterWithGRpc cluster;
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
index d709c1cf96..cc0e2ccc45 100644
--- a/ratis-grpc/src/test/java/org/apache/ratis/grpc/TestRetryCacheWithGrpc.java
+++ b/ratis-grpc/src/test/java/org/apache/ratis/grpc/TestRetryCacheWithGrpc.java
@@ -17,17 +17,17 @@
  */
 package org.apache.ratis.grpc;
 
+import java.io.IOException;
+
 import org.apache.log4j.Level;
 import org.apache.ratis.RaftRetryCacheTests;
 import org.apache.ratis.server.impl.RaftServerImpl;
-import org.apache.ratis.util.RaftUtils;
+import org.apache.ratis.util.LogUtils;
 import org.junit.Assert;
 
-import java.io.IOException;
-
 public class TestRetryCacheWithGrpc extends RaftRetryCacheTests {
   static {
-    RaftUtils.setLogLevel(RaftServerImpl.LOG, Level.DEBUG);
+    LogUtils.setLogLevel(RaftServerImpl.LOG, Level.DEBUG);
   }
 
   private final MiniRaftClusterWithGRpc cluster;
diff --git a/ratis-hadoop/src/test/java/org/apache/ratis/hadooprpc/TestRaftReconfigurationWithHadoopRpc.java b/ratis-hadoop/src/test/java/org/apache/ratis/hadooprpc/TestRaftReconfigurationWithHadoopRpc.java
index 7a36fa1105..df432fc369 100644
--- a/ratis-hadoop/src/test/java/org/apache/ratis/hadooprpc/TestRaftReconfigurationWithHadoopRpc.java
+++ b/ratis-hadoop/src/test/java/org/apache/ratis/hadooprpc/TestRaftReconfigurationWithHadoopRpc.java
@@ -22,9 +22,7 @@
 import org.apache.hadoop.ipc.Client;
 import org.apache.log4j.Level;
 import org.apache.ratis.MiniRaftCluster;
-import org.apache.ratis.client.RaftClient;
 import org.apache.ratis.server.impl.RaftReconfigurationBaseTest;
-import org.apache.ratis.util.RaftUtils;
 
 import java.io.IOException;
 
diff --git a/ratis-hadoop/src/test/java/org/apache/ratis/hadooprpc/TestRaftWithHadoopRpc.java b/ratis-hadoop/src/test/java/org/apache/ratis/hadooprpc/TestRaftWithHadoopRpc.java
index 5c03129989..124e7ee57f 100644
--- a/ratis-hadoop/src/test/java/org/apache/ratis/hadooprpc/TestRaftWithHadoopRpc.java
+++ b/ratis-hadoop/src/test/java/org/apache/ratis/hadooprpc/TestRaftWithHadoopRpc.java
@@ -22,7 +22,7 @@
 import org.apache.ratis.client.RaftClient;
 import org.apache.ratis.server.impl.BlockRequestHandlingInjection;
 import org.apache.ratis.server.impl.RaftServerImpl;
-import org.apache.ratis.util.RaftUtils;
+import org.apache.ratis.util.LogUtils;
 import org.junit.Test;
 
 import java.io.IOException;
@@ -31,9 +31,9 @@
 
 public class TestRaftWithHadoopRpc extends RaftBasicTests {
   static {
-    RaftUtils.setLogLevel(RaftServerImpl.LOG, Level.DEBUG);
-    RaftUtils.setLogLevel(RaftClient.LOG, Level.DEBUG);
-    RaftUtils.setLogLevel(MiniRaftClusterWithHadoopRpc.LOG, Level.DEBUG);
+    LogUtils.setLogLevel(RaftServerImpl.LOG, Level.DEBUG);
+    LogUtils.setLogLevel(RaftClient.LOG, Level.DEBUG);
+    LogUtils.setLogLevel(MiniRaftClusterWithHadoopRpc.LOG, Level.DEBUG);
   }
 
   private final MiniRaftClusterWithHadoopRpc cluster;
diff --git a/ratis-hadoop/src/test/java/org/apache/ratis/hadooprpc/TestRetryCacheWithHadoopRpc.java b/ratis-hadoop/src/test/java/org/apache/ratis/hadooprpc/TestRetryCacheWithHadoopRpc.java
index fade34b80d..6e5ba8cbd2 100644
--- a/ratis-hadoop/src/test/java/org/apache/ratis/hadooprpc/TestRetryCacheWithHadoopRpc.java
+++ b/ratis-hadoop/src/test/java/org/apache/ratis/hadooprpc/TestRetryCacheWithHadoopRpc.java
@@ -21,15 +21,15 @@
 import org.apache.ratis.RaftRetryCacheTests;
 import org.apache.ratis.client.RaftClient;
 import org.apache.ratis.server.impl.RaftServerImpl;
-import org.apache.ratis.util.RaftUtils;
+import org.apache.ratis.util.LogUtils;
 
 import java.io.IOException;
 
 public class TestRetryCacheWithHadoopRpc extends RaftRetryCacheTests {
   static {
-    RaftUtils.setLogLevel(RaftServerImpl.LOG, Level.DEBUG);
-    RaftUtils.setLogLevel(RaftClient.LOG, Level.DEBUG);
-    RaftUtils.setLogLevel(MiniRaftClusterWithHadoopRpc.LOG, Level.DEBUG);
+    LogUtils.setLogLevel(RaftServerImpl.LOG, Level.DEBUG);
+    LogUtils.setLogLevel(RaftClient.LOG, Level.DEBUG);
+    LogUtils.setLogLevel(MiniRaftClusterWithHadoopRpc.LOG, Level.DEBUG);
   }
 
   private final MiniRaftClusterWithHadoopRpc cluster;
diff --git a/ratis-netty/src/main/java/org/apache/ratis/netty/NettyRpcProxy.java b/ratis-netty/src/main/java/org/apache/ratis/netty/NettyRpcProxy.java
index 8b647844de..5b3492b1a5 100644
--- a/ratis-netty/src/main/java/org/apache/ratis/netty/NettyRpcProxy.java
+++ b/ratis-netty/src/main/java/org/apache/ratis/netty/NettyRpcProxy.java
@@ -28,9 +28,9 @@
 import org.apache.ratis.shaded.proto.RaftProtos.RaftRpcRequestProto;
 import org.apache.ratis.shaded.proto.netty.NettyProtos.RaftNettyServerReplyProto;
 import org.apache.ratis.shaded.proto.netty.NettyProtos.RaftNettyServerRequestProto;
+import org.apache.ratis.util.IOUtils;
 import org.apache.ratis.util.PeerProxyMap;
 import org.apache.ratis.util.ProtoUtils;
-import org.apache.ratis.util.RaftUtils;
 
 import java.io.Closeable;
 import java.io.IOException;
@@ -51,7 +51,7 @@ public NettyRpcProxy createProxyImpl(RaftPeer peer)
       try {
         return new NettyRpcProxy(peer, group);
       } catch (InterruptedException e) {
-        throw RaftUtils.toInterruptedIOException("Failed connecting to " + peer, e);
+        throw IOUtils.toInterruptedIOException("Failed connecting to " + peer, e);
       }
     }
 
@@ -170,10 +170,10 @@ public RaftNettyServerReplyProto send(
       channelFuture.sync();
       return reply.get();
     } catch (InterruptedException e) {
-      throw RaftUtils.toInterruptedIOException(ProtoUtils.toString(request)
+      throw IOUtils.toInterruptedIOException(ProtoUtils.toString(request)
           + " sending from " + peer + " is interrupted.", e);
     } catch (ExecutionException e) {
-      throw RaftUtils.toIOException(e);
+      throw IOUtils.toIOException(e);
     }
   }
 }
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
index a48e608348..afb7c2bd1e 100644
--- a/ratis-netty/src/test/java/org/apache/ratis/netty/TestRaftWithNetty.java
+++ b/ratis-netty/src/test/java/org/apache/ratis/netty/TestRaftWithNetty.java
@@ -22,15 +22,15 @@
 import org.apache.ratis.client.RaftClient;
 import org.apache.ratis.server.impl.BlockRequestHandlingInjection;
 import org.apache.ratis.server.impl.RaftServerImpl;
-import org.apache.ratis.util.RaftUtils;
+import org.apache.ratis.util.LogUtils;
 import org.junit.Test;
 
 import java.io.IOException;
 
 public class TestRaftWithNetty extends RaftBasicTests {
   static {
-    RaftUtils.setLogLevel(RaftServerImpl.LOG, Level.DEBUG);
-    RaftUtils.setLogLevel(RaftClient.LOG, Level.DEBUG);
+    LogUtils.setLogLevel(RaftServerImpl.LOG, Level.DEBUG);
+    LogUtils.setLogLevel(RaftClient.LOG, Level.DEBUG);
   }
 
   private final MiniRaftClusterWithNetty cluster;
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
index 540cc16a7d..cb4c31b4e5 100644
--- a/ratis-netty/src/test/java/org/apache/ratis/netty/TestRetryCacheWithNettyRpc.java
+++ b/ratis-netty/src/test/java/org/apache/ratis/netty/TestRetryCacheWithNettyRpc.java
@@ -17,18 +17,18 @@
  */
 package org.apache.ratis.netty;
 
+import java.io.IOException;
+
 import org.apache.log4j.Level;
 import org.apache.ratis.RaftRetryCacheTests;
 import org.apache.ratis.client.RaftClient;
 import org.apache.ratis.server.impl.RaftServerImpl;
-import org.apache.ratis.util.RaftUtils;
-
-import java.io.IOException;
+import org.apache.ratis.util.LogUtils;
 
 public class TestRetryCacheWithNettyRpc extends RaftRetryCacheTests {
   static {
-    RaftUtils.setLogLevel(RaftServerImpl.LOG, Level.DEBUG);
-    RaftUtils.setLogLevel(RaftClient.LOG, Level.DEBUG);
+    LogUtils.setLogLevel(RaftServerImpl.LOG, Level.DEBUG);
+    LogUtils.setLogLevel(RaftClient.LOG, Level.DEBUG);
   }
 
   private final MiniRaftClusterWithNetty cluster;
diff --git a/ratis-server/src/main/java/org/apache/ratis/server/impl/ConfigurationManager.java b/ratis-server/src/main/java/org/apache/ratis/server/impl/ConfigurationManager.java
index 85b41d0c7e..6aed1d7942 100644
--- a/ratis-server/src/main/java/org/apache/ratis/server/impl/ConfigurationManager.java
+++ b/ratis-server/src/main/java/org/apache/ratis/server/impl/ConfigurationManager.java
@@ -17,7 +17,7 @@
  */
 package org.apache.ratis.server.impl;
 
-import org.apache.ratis.util.RaftUtils;
+import org.apache.ratis.util.Preconditions;
 
 import java.util.*;
 
@@ -44,7 +44,7 @@ public class ConfigurationManager {
 
   public synchronized void addConfiguration(long logIndex,
       RaftConfiguration conf) {
-    RaftUtils.assertTrue(configurations.isEmpty() ||
+    Preconditions.assertTrue(configurations.isEmpty() ||
         configurations.lastEntry().getKey() < logIndex);
     configurations.put(logIndex, conf);
     this.currentConf = conf;
diff --git a/ratis-server/src/main/java/org/apache/ratis/server/impl/LeaderElection.java b/ratis-server/src/main/java/org/apache/ratis/server/impl/LeaderElection.java
index db899f1384..d026db6536 100644
--- a/ratis-server/src/main/java/org/apache/ratis/server/impl/LeaderElection.java
+++ b/ratis-server/src/main/java/org/apache/ratis/server/impl/LeaderElection.java
@@ -24,8 +24,8 @@
 import org.apache.ratis.shaded.proto.RaftProtos.RequestVoteRequestProto;
 import org.apache.ratis.statemachine.SnapshotInfo;
 import org.apache.ratis.util.Daemon;
+import org.apache.ratis.util.Preconditions;
 import org.apache.ratis.util.ProtoUtils;
-import org.apache.ratis.util.RaftUtils;
 import org.apache.ratis.util.Timestamp;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
@@ -91,7 +91,7 @@ void stopRunning() {
   }
 
   private void initExecutor() {
-    RaftUtils.assertTrue(!others.isEmpty());
+    Preconditions.assertTrue(!others.isEmpty());
     executor = Executors.newFixedThreadPool(others.size(), Daemon::new);
     service = new ExecutorCompletionService<>(executor);
   }
diff --git a/ratis-server/src/main/java/org/apache/ratis/server/impl/LeaderState.java b/ratis-server/src/main/java/org/apache/ratis/server/impl/LeaderState.java
index e7704fcb57..2d1ab5226b 100644
--- a/ratis-server/src/main/java/org/apache/ratis/server/impl/LeaderState.java
+++ b/ratis-server/src/main/java/org/apache/ratis/server/impl/LeaderState.java
@@ -180,7 +180,7 @@ TimeDuration getSyncInterval() {
    * Start bootstrapping new peers
    */
   PendingRequest startSetConfiguration(SetConfigurationRequest request) {
-    RaftUtils.assertTrue(running && !inStagingState());
+    Preconditions.assertTrue(running && !inStagingState());
 
     RaftPeer[] peersInNewConf = request.getPeersInNewConf();
     Collection peersToBootStrap = RaftConfiguration
@@ -246,7 +246,7 @@ void addSenders(Collection newMembers) {
    * Update the RpcSender list based on the current configuration
    */
   private void updateSenders(RaftConfiguration conf) {
-    RaftUtils.assertTrue(conf.isStable() && !inStagingState());
+    Preconditions.assertTrue(conf.isStable() && !inStagingState());
     Iterator iterator = senders.iterator();
     while (iterator.hasNext()) {
       LogAppender sender = iterator.next();
@@ -312,7 +312,7 @@ public void run() {
           LOG.warn("Failed to persist new votedFor/term.", e);
           // the failure should happen while changing the state to follower
           // thus the in-memory state should have been updated
-          RaftUtils.assertTrue(!running);
+          Preconditions.assertTrue(!running);
         }
       }
     }
@@ -346,7 +346,7 @@ private void handleEvent(StateUpdateEvent e) throws IOException {
    */
   private BootStrapProgress checkProgress(FollowerInfo follower,
       long committed) {
-    RaftUtils.assertTrue(!follower.isAttendingVote());
+    Preconditions.assertTrue(!follower.isAttendingVote());
     final Timestamp progressTime = new Timestamp().addTimeMs(-server.getMaxTimeoutMs());
     final Timestamp timeoutTime = new Timestamp().addTimeMs(-3*server.getMaxTimeoutMs());
     if (follower.getLastRpcResponseTime().compareTo(timeoutTime) < 0) {
@@ -362,7 +362,7 @@ private BootStrapProgress checkProgress(FollowerInfo follower,
   }
 
   private Collection checkAllProgress(long committed) {
-    RaftUtils.assertTrue(inStagingState());
+    Preconditions.assertTrue(inStagingState());
     return senders.stream()
         .filter(sender -> !sender.getFollower().isAttendingVote())
         .map(sender -> checkProgress(sender.getFollower(), committed))
diff --git a/ratis-server/src/main/java/org/apache/ratis/server/impl/LogAppender.java b/ratis-server/src/main/java/org/apache/ratis/server/impl/LogAppender.java
index 4ae4726493..5417e92f19 100644
--- a/ratis-server/src/main/java/org/apache/ratis/server/impl/LogAppender.java
+++ b/ratis-server/src/main/java/org/apache/ratis/server/impl/LogAppender.java
@@ -27,8 +27,9 @@
 import org.apache.ratis.shaded.proto.RaftProtos.*;
 import org.apache.ratis.statemachine.SnapshotInfo;
 import org.apache.ratis.util.Daemon;
+import org.apache.ratis.util.IOUtils;
+import org.apache.ratis.util.Preconditions;
 import org.apache.ratis.util.ProtoUtils;
-import org.apache.ratis.util.RaftUtils;
 import org.apache.ratis.util.Timestamp;
 import org.slf4j.Logger;
 
@@ -141,7 +142,7 @@ private TermIndex getPrevious() {
     if (previous == null) {
       // if previous is null, nextIndex must be equal to the log start
       // index (otherwise we will install snapshot).
-      RaftUtils.assertTrue(follower.getNextIndex() == raftLog.getStartIndex(),
+      Preconditions.assertTrue(follower.getNextIndex() == raftLog.getStartIndex(),
           "follower's next index %s, local log start index %s",
           follower.getNextIndex(), raftLog.getStartIndex());
       SnapshotInfo snapshot = server.getState().getLatestSnapshot();
@@ -312,7 +313,7 @@ private FileChunkProto readFileChunk(FileInfo fileInfo,
       throws IOException {
     FileChunkProto.Builder builder = FileChunkProto.newBuilder()
         .setOffset(offset).setChunkIndex(chunkIndex);
-    RaftUtils.readFully(in, buf, 0, length);
+    IOUtils.readFully(in, buf, 0, length);
     Path relativePath = server.getState().getStorage().getStorageDir()
         .relativizeToRoot(fileInfo.getPath());
     builder.setFilename(relativePath.toString());
diff --git a/ratis-server/src/main/java/org/apache/ratis/server/impl/PeerConfiguration.java b/ratis-server/src/main/java/org/apache/ratis/server/impl/PeerConfiguration.java
index 704a7d241c..82f546b609 100644
--- a/ratis-server/src/main/java/org/apache/ratis/server/impl/PeerConfiguration.java
+++ b/ratis-server/src/main/java/org/apache/ratis/server/impl/PeerConfiguration.java
@@ -21,7 +21,7 @@
 
 import org.apache.ratis.protocol.RaftPeer;
 import org.apache.ratis.protocol.RaftPeerId;
-import org.apache.ratis.util.RaftUtils;
+import org.apache.ratis.util.Preconditions;
 
 /**
  * The peer configuration of a raft cluster.
@@ -38,7 +38,7 @@ class PeerConfiguration {
       map.put(p.getId(), p);
     }
     this.peers = Collections.unmodifiableMap(map);
-    RaftUtils.assertTrue(!this.peers.isEmpty());
+    Preconditions.assertTrue(!this.peers.isEmpty());
   }
 
   Collection getPeers() {
@@ -73,7 +73,7 @@ List getOtherPeers(RaftPeerId selfId) {
   }
 
   boolean hasMajority(Collection others, RaftPeerId selfId) {
-    RaftUtils.assertTrue(!others.contains(selfId));
+    Preconditions.assertTrue(!others.contains(selfId));
     int num = 0;
     if (contains(selfId)) {
       num++;
diff --git a/ratis-server/src/main/java/org/apache/ratis/server/impl/PendingRequest.java b/ratis-server/src/main/java/org/apache/ratis/server/impl/PendingRequest.java
index 1d51e54eef..59f870ea7c 100644
--- a/ratis-server/src/main/java/org/apache/ratis/server/impl/PendingRequest.java
+++ b/ratis-server/src/main/java/org/apache/ratis/server/impl/PendingRequest.java
@@ -22,7 +22,7 @@
 import org.apache.ratis.protocol.RaftClientRequest;
 import org.apache.ratis.protocol.SetConfigurationRequest;
 import org.apache.ratis.statemachine.TransactionContext;
-import org.apache.ratis.util.RaftUtils;
+import org.apache.ratis.util.Preconditions;
 
 import java.util.concurrent.CompletableFuture;
 
@@ -64,12 +64,12 @@ TransactionContext getEntry() {
    * This is only used when setting new raft configuration.
    */
   synchronized void setException(Throwable e) {
-    RaftUtils.assertTrue(e != null);
+    Preconditions.assertTrue(e != null);
     future.completeExceptionally(e);
   }
 
   synchronized void setReply(RaftClientReply r) {
-    RaftUtils.assertTrue(r != null);
+    Preconditions.assertTrue(r != null);
     future.complete(r);
   }
 
diff --git a/ratis-server/src/main/java/org/apache/ratis/server/impl/PendingRequests.java b/ratis-server/src/main/java/org/apache/ratis/server/impl/PendingRequests.java
index c56d7a860d..98bc0a7dcf 100644
--- a/ratis-server/src/main/java/org/apache/ratis/server/impl/PendingRequests.java
+++ b/ratis-server/src/main/java/org/apache/ratis/server/impl/PendingRequests.java
@@ -19,7 +19,7 @@
 
 import org.apache.ratis.protocol.*;
 import org.apache.ratis.statemachine.TransactionContext;
-import org.apache.ratis.util.RaftUtils;
+import org.apache.ratis.util.Preconditions;
 import org.slf4j.Logger;
 
 import java.io.IOException;
@@ -44,8 +44,8 @@ class PendingRequests {
   PendingRequest addPendingRequest(long index, RaftClientRequest request,
       TransactionContext entry) {
     // externally synced for now
-    RaftUtils.assertTrue(!request.isReadOnly());
-    RaftUtils.assertTrue(last == null || index == last.getIndex() + 1);
+    Preconditions.assertTrue(!request.isReadOnly());
+    Preconditions.assertTrue(last == null || index == last.getIndex() + 1);
     return add(index, request, entry);
   }
 
@@ -58,7 +58,7 @@ private PendingRequest add(long index, RaftClientRequest request,
   }
 
   PendingRequest addConfRequest(SetConfigurationRequest request) {
-    RaftUtils.assertTrue(pendingSetConf == null);
+    Preconditions.assertTrue(pendingSetConf == null);
     pendingSetConf = new PendingRequest(request);
     return pendingSetConf;
   }
@@ -76,7 +76,7 @@ void replySetConfiguration() {
   }
 
   void failSetConfiguration(RaftException e) {
-    RaftUtils.assertTrue(pendingSetConf != null);
+    Preconditions.assertTrue(pendingSetConf != null);
     pendingSetConf.setException(e);
     pendingSetConf = null;
   }
@@ -91,7 +91,7 @@ TransactionContext getTransactionContext(long index) {
   void replyPendingRequest(long index, RaftClientReply reply) {
     final PendingRequest pending = pendingRequests.get(index);
     if (pending != null) {
-      RaftUtils.assertTrue(pending.getIndex() == index);
+      Preconditions.assertTrue(pending.getIndex() == index);
       pending.setReply(reply);
     }
   }
diff --git a/ratis-server/src/main/java/org/apache/ratis/server/impl/RaftConfiguration.java b/ratis-server/src/main/java/org/apache/ratis/server/impl/RaftConfiguration.java
index a70abf91d7..6ce7ecde11 100644
--- a/ratis-server/src/main/java/org/apache/ratis/server/impl/RaftConfiguration.java
+++ b/ratis-server/src/main/java/org/apache/ratis/server/impl/RaftConfiguration.java
@@ -19,7 +19,7 @@
 
 import org.apache.ratis.protocol.RaftPeer;
 import org.apache.ratis.protocol.RaftPeerId;
-import org.apache.ratis.util.RaftUtils;
+import org.apache.ratis.util.Preconditions;
 
 import java.util.*;
 import java.util.concurrent.ThreadLocalRandom;
@@ -51,7 +51,7 @@ private Builder() {}
 
     public Builder setConf(PeerConfiguration conf) {
       Objects.requireNonNull(conf);
-      RaftUtils.assertTrue(this.conf == null, "conf is already set.");
+      Preconditions.assertTrue(this.conf == null, "conf is already set.");
       this.conf = conf;
       return this;
     }
@@ -66,9 +66,9 @@ public Builder setConf(RaftPeer[] peers) {
 
     Builder setConf(RaftConfiguration transitionalConf) {
       Objects.requireNonNull(transitionalConf);
-      RaftUtils.assertTrue(transitionalConf.isTransitional());
+      Preconditions.assertTrue(transitionalConf.isTransitional());
 
-      RaftUtils.assertTrue(!forceTransitional);
+      Preconditions.assertTrue(!forceTransitional);
       forceStable = true;
       return setConf(transitionalConf.conf);
     }
@@ -76,7 +76,7 @@ Builder setConf(RaftConfiguration transitionalConf) {
 
     public Builder setOldConf(PeerConfiguration oldConf) {
       Objects.requireNonNull(oldConf);
-      RaftUtils.assertTrue(this.oldConf == null, "oldConf is already set.");
+      Preconditions.assertTrue(this.oldConf == null, "oldConf is already set.");
       this.oldConf = oldConf;
       return this;
     }
@@ -91,17 +91,17 @@ public Builder setOldConf(RaftPeer[] oldPeers) {
 
     Builder setOldConf(RaftConfiguration stableConf) {
       Objects.requireNonNull(stableConf);
-      RaftUtils.assertTrue(stableConf.isStable());
+      Preconditions.assertTrue(stableConf.isStable());
 
-      RaftUtils.assertTrue(!forceStable);
+      Preconditions.assertTrue(!forceStable);
       forceTransitional = true;
       return setOldConf(stableConf.conf);
     }
 
     public Builder setLogEntryIndex(long logEntryIndex) {
-      RaftUtils.assertTrue(
+      Preconditions.assertTrue(
           logEntryIndex != RaftServerConstants.INVALID_LOG_INDEX);
-      RaftUtils.assertTrue(
+      Preconditions.assertTrue(
           this.logEntryIndex == RaftServerConstants.INVALID_LOG_INDEX,
           "logEntryIndex is already set.");
       this.logEntryIndex = logEntryIndex;
@@ -111,10 +111,10 @@ public Builder setLogEntryIndex(long logEntryIndex) {
     /** Build a {@link RaftConfiguration}. */
     public RaftConfiguration build() {
       if (forceTransitional) {
-        RaftUtils.assertTrue(oldConf != null);
+        Preconditions.assertTrue(oldConf != null);
       }
       if (forceStable) {
-        RaftUtils.assertTrue(oldConf == null);
+        Preconditions.assertTrue(oldConf == null);
       }
       return new RaftConfiguration(conf, oldConf, logEntryIndex);
     }
@@ -204,7 +204,7 @@ public Collection getOtherPeers(RaftPeerId selfId) {
 
   /** @return true if the self id together with the others are in the majority. */
   boolean hasMajority(Collection others, RaftPeerId selfId) {
-    RaftUtils.assertTrue(!others.contains(selfId));
+    Preconditions.assertTrue(!others.contains(selfId));
     return conf.hasMajority(others, selfId) &&
         (oldConf == null || oldConf.hasMajority(others, selfId));
   }
diff --git a/ratis-server/src/main/java/org/apache/ratis/server/impl/RaftServerImpl.java b/ratis-server/src/main/java/org/apache/ratis/server/impl/RaftServerImpl.java
index 6a1fbd4edf..0a6a68db51 100644
--- a/ratis-server/src/main/java/org/apache/ratis/server/impl/RaftServerImpl.java
+++ b/ratis-server/src/main/java/org/apache/ratis/server/impl/RaftServerImpl.java
@@ -17,43 +17,68 @@
  */
 package org.apache.ratis.server.impl;
 
+import static org.apache.ratis.shaded.proto.RaftProtos.AppendEntriesReplyProto.AppendResult.INCONSISTENCY;
+import static org.apache.ratis.shaded.proto.RaftProtos.AppendEntriesReplyProto.AppendResult.NOT_LEADER;
+import static org.apache.ratis.shaded.proto.RaftProtos.AppendEntriesReplyProto.AppendResult.SUCCESS;
+import static org.apache.ratis.util.LifeCycle.State.CLOSED;
+import static org.apache.ratis.util.LifeCycle.State.CLOSING;
+import static org.apache.ratis.util.LifeCycle.State.RUNNING;
+import static org.apache.ratis.util.LifeCycle.State.STARTING;
+
+import java.io.IOException;
+import java.io.InterruptedIOException;
+import java.util.Arrays;
+import java.util.Collection;
+import java.util.List;
+import java.util.OptionalLong;
+import java.util.concurrent.CompletableFuture;
+import java.util.concurrent.ExecutionException;
+import java.util.concurrent.ThreadLocalRandom;
+import java.util.concurrent.TimeUnit;
+
 import org.apache.ratis.RaftConfigKeys;
 import org.apache.ratis.conf.Parameters;
-import org.apache.ratis.rpc.RpcType;
 import org.apache.ratis.conf.RaftProperties;
-import org.apache.ratis.protocol.*;
+import org.apache.ratis.protocol.ClientId;
+import org.apache.ratis.protocol.LeaderNotReadyException;
+import org.apache.ratis.protocol.Message;
+import org.apache.ratis.protocol.NotLeaderException;
+import org.apache.ratis.protocol.RaftClientReply;
+import org.apache.ratis.protocol.RaftClientRequest;
+import org.apache.ratis.protocol.RaftException;
+import org.apache.ratis.protocol.RaftPeer;
+import org.apache.ratis.protocol.RaftPeerId;
+import org.apache.ratis.protocol.ReconfigurationInProgressException;
+import org.apache.ratis.protocol.SetConfigurationRequest;
+import org.apache.ratis.protocol.StateMachineException;
+import org.apache.ratis.rpc.RpcType;
 import org.apache.ratis.server.RaftServer;
 import org.apache.ratis.server.RaftServerConfigKeys;
 import org.apache.ratis.server.RaftServerRpc;
 import org.apache.ratis.server.protocol.TermIndex;
 import org.apache.ratis.server.storage.FileInfo;
 import org.apache.ratis.shaded.com.google.common.annotations.VisibleForTesting;
-import org.apache.ratis.shaded.proto.RaftProtos.*;
+import org.apache.ratis.shaded.proto.RaftProtos.AppendEntriesReplyProto;
+import org.apache.ratis.shaded.proto.RaftProtos.AppendEntriesRequestProto;
+import org.apache.ratis.shaded.proto.RaftProtos.FileChunkProto;
+import org.apache.ratis.shaded.proto.RaftProtos.InstallSnapshotReplyProto;
+import org.apache.ratis.shaded.proto.RaftProtos.InstallSnapshotRequestProto;
+import org.apache.ratis.shaded.proto.RaftProtos.InstallSnapshotResult;
+import org.apache.ratis.shaded.proto.RaftProtos.LogEntryProto;
+import org.apache.ratis.shaded.proto.RaftProtos.RequestVoteReplyProto;
+import org.apache.ratis.shaded.proto.RaftProtos.RequestVoteRequestProto;
 import org.apache.ratis.statemachine.SnapshotInfo;
 import org.apache.ratis.statemachine.StateMachine;
 import org.apache.ratis.statemachine.TransactionContext;
 import org.apache.ratis.util.CodeInjectionForTesting;
+import org.apache.ratis.util.IOUtils;
 import org.apache.ratis.util.LifeCycle;
+import org.apache.ratis.util.Preconditions;
 import org.apache.ratis.util.ProtoUtils;
-import org.apache.ratis.util.RaftUtils;
 import org.apache.ratis.util.TimeDuration;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
-import java.io.IOException;
-import java.io.InterruptedIOException;
-import java.util.Arrays;
-import java.util.Collection;
-import java.util.List;
-import java.util.OptionalLong;
-import java.util.concurrent.CompletableFuture;
-import java.util.concurrent.ExecutionException;
-import java.util.concurrent.ThreadLocalRandom;
-import java.util.concurrent.TimeUnit;
-
-import static org.apache.ratis.shaded.proto.RaftProtos.AppendEntriesReplyProto.AppendResult.*;
-import static org.apache.ratis.util.LifeCycle.State.*;
-
 public class RaftServerImpl implements RaftServer {
   public static final Logger LOG = LoggerFactory.getLogger(RaftServerImpl.class);
 
@@ -98,7 +123,7 @@ enum Role {
     this.lifeCycle = new LifeCycle(id);
     minTimeoutMs = RaftServerConfigKeys.Rpc.timeoutMin(properties).toInt(TimeUnit.MILLISECONDS);
     maxTimeoutMs = RaftServerConfigKeys.Rpc.timeoutMax(properties).toInt(TimeUnit.MILLISECONDS);
-    RaftUtils.assertTrue(maxTimeoutMs > minTimeoutMs,
+    Preconditions.assertTrue(maxTimeoutMs > minTimeoutMs,
         "max timeout: %s, min timeout: %s", maxTimeoutMs, minTimeoutMs);
     this.properties = properties;
     this.stateMachine = stateMachine;
@@ -302,7 +327,7 @@ private void shutdownElectionDaemon() {
   }
 
   synchronized void changeToLeader() {
-    RaftUtils.assertTrue(isCandidate());
+    Preconditions.assertTrue(isCandidate());
     shutdownElectionDaemon();
     role = Role.LEADER;
     state.becomeLeader();
@@ -321,7 +346,7 @@ private void shutdownHeartbeatMonitor() {
   }
 
   synchronized void changeToCandidate() {
-    RaftUtils.assertTrue(isFollower());
+    Preconditions.assertTrue(isFollower());
     shutdownHeartbeatMonitor();
     role = Role.CANDIDATE;
     // start election
@@ -455,7 +480,6 @@ public CompletableFuture submitClientRequestAsync(
       cacheEntry.failWithReply(exceptionReply);
       return CompletableFuture.completedFuture(exceptionReply);
     }
-
     return appendTransaction(request, context, cacheEntry);
   }
 
@@ -473,7 +497,7 @@ private static RaftClientReply waitForReply(RaftPeerId id,
     } catch (InterruptedException e) {
       final String s = id + ": Interrupted when waiting for reply, request=" + request;
       LOG.info(s, e);
-      throw RaftUtils.toInterruptedIOException(s, e);
+      throw IOUtils.toInterruptedIOException(s, e);
     } catch (ExecutionException e) {
       final Throwable cause = e.getCause();
       if (cause == null) {
@@ -483,7 +507,7 @@ private static RaftClientReply waitForReply(RaftPeerId id,
           cause instanceof StateMachineException) {
         return new RaftClientReply(request, (RaftException) cause);
       } else {
-        throw RaftUtils.asIOException(cause);
+        throw IOUtils.asIOException(cause);
       }
     }
   }
@@ -617,23 +641,23 @@ private void validateEntries(long expectedTerm, TermIndex previous,
       final long index0 = entries[0].getIndex();
 
       if (previous == null || previous.getTerm() == 0) {
-        RaftUtils.assertTrue(index0 == 0,
+        Preconditions.assertTrue(index0 == 0,
             "Unexpected Index: previous is null but entries[%s].getIndex()=%s",
             0, index0);
       } else {
-        RaftUtils.assertTrue(previous.getIndex() == index0 - 1,
+        Preconditions.assertTrue(previous.getIndex() == index0 - 1,
             "Unexpected Index: previous is %s but entries[%s].getIndex()=%s",
             previous, 0, index0);
       }
 
       for (int i = 0; i < entries.length; i++) {
         final long t = entries[i].getTerm();
-        RaftUtils.assertTrue(expectedTerm >= t,
+        Preconditions.assertTrue(expectedTerm >= t,
             "Unexpected Term: entries[%s].getTerm()=%s but expectedTerm=%s",
             i, t, expectedTerm);
 
         final long indexi = entries[i].getIndex();
-        RaftUtils.assertTrue(indexi == index0 + i,
+        Preconditions.assertTrue(indexi == index0 + i,
             "Unexpected Index: entries[%s].getIndex()=%s but entries[0].getIndex()=%s",
             i, indexi, index0);
       }
@@ -786,7 +810,7 @@ public InstallSnapshotReplyProto installSnapshot(
       // Check and append the snapshot chunk. We simply put this in lock
       // considering a follower peer requiring a snapshot installation does not
       // have a lot of requests
-      RaftUtils.assertTrue(
+      Preconditions.assertTrue(
           state.getLog().getNextIndex() <= lastIncludedIndex,
           "%s log's next id is %s, last included index in snapshot is %s",
           getId(),  state.getLog().getNextIndex(), lastIncludedIndex);
diff --git a/ratis-server/src/main/java/org/apache/ratis/server/impl/RetryCache.java b/ratis-server/src/main/java/org/apache/ratis/server/impl/RetryCache.java
index 03725586ee..bf2e94c32b 100644
--- a/ratis-server/src/main/java/org/apache/ratis/server/impl/RetryCache.java
+++ b/ratis-server/src/main/java/org/apache/ratis/server/impl/RetryCache.java
@@ -17,21 +17,21 @@
  */
 package org.apache.ratis.server.impl;
 
+import java.io.Closeable;
+import java.util.concurrent.CompletableFuture;
+import java.util.concurrent.ExecutionException;
+import java.util.concurrent.TimeUnit;
+
 import org.apache.ratis.protocol.ClientId;
 import org.apache.ratis.protocol.RaftClientReply;
 import org.apache.ratis.shaded.com.google.common.annotations.VisibleForTesting;
 import org.apache.ratis.shaded.com.google.common.cache.Cache;
 import org.apache.ratis.shaded.com.google.common.cache.CacheBuilder;
-import org.apache.ratis.util.RaftUtils;
+import org.apache.ratis.util.Preconditions;
 import org.apache.ratis.util.TimeDuration;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
-import java.io.Closeable;
-import java.util.concurrent.CompletableFuture;
-import java.util.concurrent.ExecutionException;
-import java.util.concurrent.TimeUnit;
-
 public class RetryCache implements Closeable {
   static final Logger LOG = LoggerFactory.getLogger(RetryCache.class);
   private static final int MIN_CAPACITY = 128;
@@ -162,7 +162,7 @@ CacheEntry getOrCreateEntry(ClientId clientId, long callId) {
     } catch (ExecutionException e) {
       throw new IllegalStateException(e);
     }
-    RaftUtils.assertTrue(entry != null && !entry.isDone(),
+    Preconditions.assertTrue(entry != null && !entry.isDone(),
         "retry cache entry should be pending: %s", entry);
     return entry;
   }
diff --git a/ratis-server/src/main/java/org/apache/ratis/server/impl/ServerState.java b/ratis-server/src/main/java/org/apache/ratis/server/impl/ServerState.java
index ff75237802..e9d8ef8fbc 100644
--- a/ratis-server/src/main/java/org/apache/ratis/server/impl/ServerState.java
+++ b/ratis-server/src/main/java/org/apache/ratis/server/impl/ServerState.java
@@ -29,8 +29,8 @@
 import org.apache.ratis.statemachine.SnapshotInfo;
 import org.apache.ratis.statemachine.StateMachine;
 import org.apache.ratis.statemachine.TransactionContext;
+import org.apache.ratis.util.Preconditions;
 import org.apache.ratis.util.ProtoUtils;
-import org.apache.ratis.util.RaftUtils;
 
 import java.io.Closeable;
 import java.io.IOException;
@@ -215,7 +215,7 @@ boolean recognizeLeader(RaftPeerId leaderId, long leaderTerm) {
       // leader and term later
       return true;
     }
-    RaftUtils.assertTrue(this.leaderId.equals(leaderId),
+    Preconditions.assertTrue(this.leaderId.equals(leaderId),
         "selfId:%s, this.leaderId:%s, received leaderId:%s",
         selfId, this.leaderId, leaderId);
     return true;
diff --git a/ratis-server/src/main/java/org/apache/ratis/server/impl/StateMachineUpdater.java b/ratis-server/src/main/java/org/apache/ratis/server/impl/StateMachineUpdater.java
index 38436a25e5..95af956ac4 100644
--- a/ratis-server/src/main/java/org/apache/ratis/server/impl/StateMachineUpdater.java
+++ b/ratis-server/src/main/java/org/apache/ratis/server/impl/StateMachineUpdater.java
@@ -35,7 +35,7 @@
 import org.apache.ratis.util.Daemon;
 import org.apache.ratis.util.ExitUtils;
 import org.apache.ratis.util.LifeCycle;
-import org.apache.ratis.util.RaftUtils;
+import org.apache.ratis.util.Preconditions;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
@@ -126,15 +126,15 @@ public void run() {
         }
 
         final long committedIndex = raftLog.getLastCommittedIndex();
-        RaftUtils.assertTrue(lastAppliedIndex < committedIndex);
+        Preconditions.assertTrue(lastAppliedIndex < committedIndex);
 
         if (state == State.RELOAD) {
-          RaftUtils.assertTrue(stateMachine.getLifeCycleState() == LifeCycle.State.PAUSED);
+          Preconditions.assertTrue(stateMachine.getLifeCycleState() == LifeCycle.State.PAUSED);
 
           stateMachine.reinitialize(server.getId(), properties, storage);
 
           SnapshotInfo snapshot = stateMachine.getLatestSnapshot();
-          RaftUtils.assertTrue(snapshot != null && snapshot.getIndex() > lastAppliedIndex,
+          Preconditions.assertTrue(snapshot != null && snapshot.getIndex() > lastAppliedIndex,
               "Snapshot: %s, lastAppliedIndex: %s", snapshot, lastAppliedIndex);
 
           lastAppliedIndex = snapshot.getIndex();
diff --git a/ratis-server/src/main/java/org/apache/ratis/server/storage/LogInputStream.java b/ratis-server/src/main/java/org/apache/ratis/server/storage/LogInputStream.java
index 4a36430420..0edea13ff4 100644
--- a/ratis-server/src/main/java/org/apache/ratis/server/storage/LogInputStream.java
+++ b/ratis-server/src/main/java/org/apache/ratis/server/storage/LogInputStream.java
@@ -25,7 +25,8 @@
 import java.io.IOException;
 
 import org.apache.ratis.shaded.proto.RaftProtos.LogEntryProto;
-import org.apache.ratis.util.RaftUtils;
+import org.apache.ratis.util.IOUtils;
+import org.apache.ratis.util.Preconditions;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
@@ -72,9 +73,9 @@ private enum State {
   public LogInputStream(File log, long startIndex, long endIndex,
       boolean isOpen) {
     if (isOpen) {
-      RaftUtils.assertTrue(endIndex == INVALID_LOG_INDEX);
+      Preconditions.assertTrue(endIndex == INVALID_LOG_INDEX);
     } else {
-      RaftUtils.assertTrue(endIndex >= startIndex);
+      Preconditions.assertTrue(endIndex >= startIndex);
     }
 
     this.logFile = log;
@@ -84,12 +85,12 @@ public LogInputStream(File log, long startIndex, long endIndex,
   }
 
   private void init() throws IOException {
-    RaftUtils.assertTrue(state == State.UNINIT);
+    Preconditions.assertTrue(state == State.UNINIT);
     try {
       reader = new LogReader(logFile);
       // read the log header
       String header = reader.readLogHeader();
-      RaftUtils.assertTrue(SegmentedRaftLog.HEADER_STR.equals(header),
+      Preconditions.assertTrue(SegmentedRaftLog.HEADER_STR.equals(header),
           "Corrupted log header: %s", header);
       state = State.OPEN;
     } finally {
@@ -119,9 +120,9 @@ public LogEntryProto nextEntry() throws IOException {
           init();
         } catch (Throwable e) {
           LOG.error("caught exception initializing " + this, e);
-          throw RaftUtils.asIOException(e);
+          throw IOUtils.asIOException(e);
         }
-        RaftUtils.assertTrue(state != State.UNINIT);
+        Preconditions.assertTrue(state != State.UNINIT);
         return nextEntry();
       case OPEN:
         entry = reader.readEntry();
@@ -149,7 +150,7 @@ public LogEntryProto nextEntry() throws IOException {
   }
 
   long scanNextEntry() throws IOException {
-    RaftUtils.assertTrue(state == State.OPEN);
+    Preconditions.assertTrue(state == State.OPEN);
     return reader.scanEntry();
   }
 
@@ -202,7 +203,7 @@ static LogValidation scanEditLog(File file, long maxTxIdToScan)
     try {
       return scanEditLog(in, maxTxIdToScan);
     } finally {
-      RaftUtils.cleanup(LOG, in);
+      IOUtils.cleanup(LOG, in);
     }
   }
 
diff --git a/ratis-server/src/main/java/org/apache/ratis/server/storage/LogOutputStream.java b/ratis-server/src/main/java/org/apache/ratis/server/storage/LogOutputStream.java
index 8563549f13..db0789e4a9 100644
--- a/ratis-server/src/main/java/org/apache/ratis/server/storage/LogOutputStream.java
+++ b/ratis-server/src/main/java/org/apache/ratis/server/storage/LogOutputStream.java
@@ -22,8 +22,8 @@
 import org.apache.ratis.server.impl.RaftServerConstants;
 import org.apache.ratis.shaded.com.google.protobuf.CodedOutputStream;
 import org.apache.ratis.shaded.proto.RaftProtos.LogEntryProto;
+import org.apache.ratis.util.IOUtils;
 import org.apache.ratis.util.PureJavaCrc32C;
-import org.apache.ratis.util.RaftUtils;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
@@ -126,7 +126,7 @@ public void close() throws IOException {
         fc.truncate(fc.position());
       }
     } finally {
-      RaftUtils.cleanup(LOG, fc, out);
+      IOUtils.cleanup(LOG, fc, out);
       fc = null;
       out = null;
     }
@@ -151,7 +151,7 @@ private void preallocate() throws IOException {
       int size = (int) Math.min(BUFFER_SIZE, targetSize - allocated);
       ByteBuffer buffer = fill.slice();
       buffer.limit(size);
-      RaftUtils.writeFully(fc, buffer, preallocatedPos);
+      IOUtils.writeFully(fc, buffer, preallocatedPos);
       preallocatedPos += size;
       allocated += size;
     }
diff --git a/ratis-server/src/main/java/org/apache/ratis/server/storage/LogReader.java b/ratis-server/src/main/java/org/apache/ratis/server/storage/LogReader.java
index 829bd3a7bc..48a00e1fa4 100644
--- a/ratis-server/src/main/java/org/apache/ratis/server/storage/LogReader.java
+++ b/ratis-server/src/main/java/org/apache/ratis/server/storage/LogReader.java
@@ -22,8 +22,9 @@
 import org.apache.ratis.shaded.com.google.protobuf.CodedInputStream;
 import org.apache.ratis.shaded.com.google.protobuf.CodedOutputStream;
 import org.apache.ratis.shaded.proto.RaftProtos.LogEntryProto;
+import org.apache.ratis.util.IOUtils;
+import org.apache.ratis.util.Preconditions;
 import org.apache.ratis.util.PureJavaCrc32C;
-import org.apache.ratis.util.RaftUtils;
 
 import java.io.*;
 import java.nio.charset.StandardCharsets;
@@ -196,9 +197,9 @@ void verifyTerminator() throws IOException {
         // want to reposition the mark one byte before the error
         if (numRead != -1) {
           in.reset();
-          RaftUtils.skipFully(in, idx);
+          IOUtils.skipFully(in, idx);
           in.mark(temp.length + 1);
-          RaftUtils.skipFully(in, 1);
+          IOUtils.skipFully(in, 1);
         }
       }
     }
@@ -246,7 +247,7 @@ private LogEntryProto decodeEntry() throws IOException {
     checkBufferSize(totalLength);
     in.reset();
     in.mark(maxOpSize);
-    RaftUtils.readFully(in, temp, 0, totalLength);
+    IOUtils.readFully(in, temp, 0, totalLength);
 
     // verify checksum
     checksum.reset();
@@ -265,7 +266,7 @@ private LogEntryProto decodeEntry() throws IOException {
   }
 
   private void checkBufferSize(int entryLength) {
-    RaftUtils.assertTrue(entryLength <= maxOpSize);
+    Preconditions.assertTrue(entryLength <= maxOpSize);
     int length = temp.length;
     if (length < entryLength) {
       while (length < entryLength) {
@@ -281,11 +282,11 @@ long getPos() {
 
   void skipFully(long length) throws IOException {
     limiter.clearLimit();
-    RaftUtils.skipFully(limiter, length);
+    IOUtils.skipFully(limiter, length);
   }
 
   @Override
   public void close() throws IOException {
-    RaftUtils.cleanup(null, in);
+    IOUtils.cleanup(null, in);
   }
 }
diff --git a/ratis-server/src/main/java/org/apache/ratis/server/storage/LogSegment.java b/ratis-server/src/main/java/org/apache/ratis/server/storage/LogSegment.java
index 89c0ff529f..46f9f4f761 100644
--- a/ratis-server/src/main/java/org/apache/ratis/server/storage/LogSegment.java
+++ b/ratis-server/src/main/java/org/apache/ratis/server/storage/LogSegment.java
@@ -22,7 +22,7 @@
 import org.apache.ratis.shaded.com.google.protobuf.CodedOutputStream;
 import org.apache.ratis.shaded.proto.RaftProtos.LogEntryProto;
 import org.apache.ratis.util.FileUtils;
-import org.apache.ratis.util.RaftUtils;
+import org.apache.ratis.util.Preconditions;
 
 import java.io.File;
 import java.io.IOException;
@@ -85,12 +85,12 @@ private LogSegment(boolean isOpen, long start, long end) {
   }
 
   static LogSegment newOpenSegment(long start) {
-    RaftUtils.assertTrue(start >= 0);
+    Preconditions.assertTrue(start >= 0);
     return new LogSegment(true, start, start - 1);
   }
 
   private static LogSegment newCloseSegment(long start, long end) {
-    RaftUtils.assertTrue(start >= 0 && end >= start);
+    Preconditions.assertTrue(start >= 0 && end >= start);
     return new LogSegment(false, start, end);
   }
 
@@ -104,7 +104,7 @@ static LogSegment loadSegment(File file, long start, long end, boolean isOpen,
       LogEntryProto prev = null;
       while ((next = in.nextEntry()) != null) {
         if (prev != null) {
-          RaftUtils.assertTrue(next.getIndex() == prev.getIndex() + 1,
+          Preconditions.assertTrue(next.getIndex() == prev.getIndex() + 1,
               "gap between entry %s and entry %s", prev, next);
         }
         segment.append(next);
@@ -123,9 +123,9 @@ static LogSegment loadSegment(File file, long start, long end, boolean isOpen,
       FileUtils.truncateFile(file, segment.getTotalSize());
     }
 
-    RaftUtils.assertTrue(start == segment.records.get(0).entry.getIndex());
+    Preconditions.assertTrue(start == segment.records.get(0).entry.getIndex());
     if (!isOpen) {
-      RaftUtils.assertTrue(segment.getEndIndex() == end);
+      Preconditions.assertTrue(segment.getEndIndex() == end);
     }
     return segment;
   }
@@ -147,26 +147,26 @@ int numOfEntries() {
   }
 
   void appendToOpenSegment(LogEntryProto... entries) {
-    RaftUtils.assertTrue(isOpen(),
+    Preconditions.assertTrue(isOpen(),
         "The log segment %s is not open for append", this.toString());
     append(entries);
   }
 
   private void append(LogEntryProto... entries) {
-    RaftUtils.assertTrue(entries != null && entries.length > 0);
+    Preconditions.assertTrue(entries != null && entries.length > 0);
     final long term = entries[0].getTerm();
     if (records.isEmpty()) {
-      RaftUtils.assertTrue(entries[0].getIndex() == startIndex,
+      Preconditions.assertTrue(entries[0].getIndex() == startIndex,
           "gap between start index %s and first entry to append %s",
           startIndex, entries[0].getIndex());
     }
     for (LogEntryProto entry : entries) {
       // all these entries should be of the same term
-      RaftUtils.assertTrue(entry.getTerm() == term,
+      Preconditions.assertTrue(entry.getTerm() == term,
           "expected term:%s, term of the entry:%s", term, entry.getTerm());
       final LogRecord currentLast = getLastRecord();
       if (currentLast != null) {
-        RaftUtils.assertTrue(
+        Preconditions.assertTrue(
             entry.getIndex() == currentLast.entry.getIndex() + 1,
             "gap between entries %s and %s", entry.getIndex(),
             currentLast.entry.getIndex());
@@ -198,7 +198,7 @@ long getTotalSize() {
    * Remove records from the given index (inclusive)
    */
   void truncate(long fromIndex) {
-    RaftUtils.assertTrue(fromIndex >= startIndex && fromIndex <= endIndex);
+    Preconditions.assertTrue(fromIndex >= startIndex && fromIndex <= endIndex);
     LogRecord record = records.get((int) (fromIndex - startIndex));
     for (long index = endIndex; index >= fromIndex; index--) {
       records.remove((int)(index - startIndex));
@@ -209,7 +209,7 @@ void truncate(long fromIndex) {
   }
 
   void close() {
-    RaftUtils.assertTrue(isOpen());
+    Preconditions.assertTrue(isOpen());
     isOpen = false;
   }
 
diff --git a/ratis-server/src/main/java/org/apache/ratis/server/storage/MemoryRaftLog.java b/ratis-server/src/main/java/org/apache/ratis/server/storage/MemoryRaftLog.java
index 17914cd412..a49db9a85a 100644
--- a/ratis-server/src/main/java/org/apache/ratis/server/storage/MemoryRaftLog.java
+++ b/ratis-server/src/main/java/org/apache/ratis/server/storage/MemoryRaftLog.java
@@ -27,7 +27,7 @@
 import org.apache.ratis.shaded.proto.RaftProtos.LogEntryProto;
 import org.apache.ratis.util.AutoCloseableLock;
 import org.apache.ratis.util.CodeInjectionForTesting;
-import org.apache.ratis.util.RaftUtils;
+import org.apache.ratis.util.Preconditions;
 
 /**
  * A simple RaftLog implementation in memory. Used only for testing.
@@ -65,7 +65,7 @@ public LogEntryProto[] getEntries(long startIndex, long endIndex) {
   void truncate(long index) {
     checkLogState();
     try(AutoCloseableLock writeLock = writeLock()) {
-      RaftUtils.assertTrue(index >= 0);
+      Preconditions.assertTrue(index >= 0);
       final int truncateIndex = (int) index;
       for (int i = entries.size() - 1; i >= truncateIndex; i--) {
         entries.remove(i);
diff --git a/ratis-server/src/main/java/org/apache/ratis/server/storage/MetaFile.java b/ratis-server/src/main/java/org/apache/ratis/server/storage/MetaFile.java
index ff010e15f8..0e4ccbd979 100644
--- a/ratis-server/src/main/java/org/apache/ratis/server/storage/MetaFile.java
+++ b/ratis-server/src/main/java/org/apache/ratis/server/storage/MetaFile.java
@@ -18,7 +18,7 @@
 package org.apache.ratis.server.storage;
 
 import org.apache.ratis.util.AtomicFileOutputStream;
-import org.apache.ratis.util.RaftUtils;
+import org.apache.ratis.util.IOUtils;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
@@ -123,7 +123,7 @@ void readFile() throws IOException {
         LOG.warn("Cannot load term/votedFor properties from {}", file, e);
         throw e;
       } finally {
-        RaftUtils.cleanup(LOG, br);
+        IOUtils.cleanup(LOG, br);
       }
     }
   }
diff --git a/ratis-server/src/main/java/org/apache/ratis/server/storage/RaftLog.java b/ratis-server/src/main/java/org/apache/ratis/server/storage/RaftLog.java
index 77e554b06f..e72249a95e 100644
--- a/ratis-server/src/main/java/org/apache/ratis/server/storage/RaftLog.java
+++ b/ratis-server/src/main/java/org/apache/ratis/server/storage/RaftLog.java
@@ -33,8 +33,8 @@
 import org.apache.ratis.shaded.proto.RaftProtos.LogEntryProto;
 import org.apache.ratis.statemachine.TransactionContext;
 import org.apache.ratis.util.AutoCloseableLock;
+import org.apache.ratis.util.Preconditions;
 import org.apache.ratis.util.ProtoUtils;
-import org.apache.ratis.util.RaftUtils;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
@@ -71,7 +71,7 @@ public long getLastCommittedIndex() {
   }
 
   public void checkLogState() {
-    RaftUtils.assertTrue(isOpen,
+    Preconditions.assertTrue(isOpen,
         "The RaftLog has not been opened or has been closed");
   }
 
diff --git a/ratis-server/src/main/java/org/apache/ratis/server/storage/RaftLogCache.java b/ratis-server/src/main/java/org/apache/ratis/server/storage/RaftLogCache.java
index 2b9972c3f9..0a21846678 100644
--- a/ratis-server/src/main/java/org/apache/ratis/server/storage/RaftLogCache.java
+++ b/ratis-server/src/main/java/org/apache/ratis/server/storage/RaftLogCache.java
@@ -30,7 +30,7 @@
 import org.apache.ratis.server.storage.LogSegment.SegmentFileInfo;
 import org.apache.ratis.shaded.proto.RaftProtos.LogEntryProto;
 
-import org.apache.ratis.util.RaftUtils;
+import org.apache.ratis.util.Preconditions;
 
 /**
  * In-memory RaftLog Cache. Currently we provide a simple implementation that
@@ -57,10 +57,10 @@ private LogSegment getLastClosedSegment() {
   private void validateAdding(LogSegment segment) {
     final LogSegment lastClosed = getLastClosedSegment();
     if (!segment.isOpen()) {
-      RaftUtils.assertTrue(lastClosed == null ||
+      Preconditions.assertTrue(lastClosed == null ||
           areConsecutiveSegments(lastClosed, segment));
     } else {
-      RaftUtils.assertTrue(openSegment == null &&
+      Preconditions.assertTrue(openSegment == null &&
           (lastClosed == null || areConsecutiveSegments(lastClosed, segment)));
     }
   }
@@ -168,7 +168,7 @@ LogSegment getOpenSegment() {
   void appendEntry(LogEntryProto entry) {
     // SegmentedRaftLog does the segment creation/rolling work. Here we just
     // simply append the entry into the open segment.
-    RaftUtils.assertTrue(openSegment != null);
+    Preconditions.assertTrue(openSegment != null);
     openSegment.appendToOpenSegment(entry);
   }
 
@@ -176,7 +176,7 @@ void appendEntry(LogEntryProto entry) {
    * finalize the current open segment, and start a new open segment
    */
   void rollOpenSegment(boolean createNewOpen) {
-    RaftUtils.assertTrue(openSegment != null
+    Preconditions.assertTrue(openSegment != null
         && openSegment.numOfEntries() > 0);
     final long nextIndex = openSegment.getEndIndex() + 1;
     openSegment.close();
@@ -211,7 +211,7 @@ TruncationSegments truncate(long index) {
               Collections.singletonList(deleteOpenSegment()));
         } else {
           openSegment.truncate(index);
-          RaftUtils.assertTrue(!openSegment.isOpen());
+          Preconditions.assertTrue(!openSegment.isOpen());
           SegmentFileInfo info = new SegmentFileInfo(openSegment.getStartIndex(),
               oldEnd, true, openSegment.getTotalSize(),
               openSegment.getEndIndex());
@@ -280,7 +280,7 @@ private class EntryIterator implements Iterator {
           // the start index is smaller than the first closed segment's start
           // index. We no longer keep the log entry (because of the snapshot) or
           // the start index is invalid.
-          RaftUtils.assertTrue(segmentIndex == 0);
+          Preconditions.assertTrue(segmentIndex == 0);
           throw new IndexOutOfBoundsException();
         }
       }
diff --git a/ratis-server/src/main/java/org/apache/ratis/server/storage/RaftLogWorker.java b/ratis-server/src/main/java/org/apache/ratis/server/storage/RaftLogWorker.java
index d0fc3ce2d2..5add8ae56e 100644
--- a/ratis-server/src/main/java/org/apache/ratis/server/storage/RaftLogWorker.java
+++ b/ratis-server/src/main/java/org/apache/ratis/server/storage/RaftLogWorker.java
@@ -34,7 +34,8 @@
 import org.apache.ratis.shaded.proto.RaftProtos.LogEntryProto;
 import org.apache.ratis.util.ExitUtils;
 import org.apache.ratis.util.FileUtils;
-import org.apache.ratis.util.RaftUtils;
+import org.apache.ratis.util.IOUtils;
+import org.apache.ratis.util.Preconditions;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
@@ -83,7 +84,7 @@ void start(long latestIndex, File openSegmentFile) throws IOException {
     lastWrittenIndex = latestIndex;
     flushedIndex = latestIndex;
     if (openSegmentFile != null) {
-      RaftUtils.assertTrue(openSegmentFile.exists());
+      Preconditions.assertTrue(openSegmentFile.exists());
       out = new LogOutputStream(openSegmentFile, true, properties);
     }
     workerThread.start();
@@ -122,7 +123,7 @@ private Task addIOTask(Task task) {
     LOG.debug("add task {}", task);
     try {
       if (!queue.offer(task, 1, TimeUnit.SECONDS)) {
-        RaftUtils.assertTrue(isAlive(),
+        Preconditions.assertTrue(isAlive(),
             "the worker thread is not alive");
         queue.put(task);
       }
@@ -225,8 +226,8 @@ private class WriteLog extends Task {
 
     @Override
     public void execute() throws IOException {
-      RaftUtils.assertTrue(out != null);
-      RaftUtils.assertTrue(lastWrittenIndex + 1 == entry.getIndex(),
+      Preconditions.assertTrue(out != null);
+      Preconditions.assertTrue(lastWrittenIndex + 1 == entry.getIndex(),
           "lastWrittenIndex == %s, entry == %s", lastWrittenIndex, entry);
       out.write(entry);
       lastWrittenIndex = entry.getIndex();
@@ -251,19 +252,19 @@ private class FinalizeLogSegment extends Task {
 
     @Override
     public void execute() throws IOException {
-      RaftUtils.cleanup(null, out);
+      IOUtils.cleanup(null, out);
       out = null;
-      RaftUtils.assertTrue(segmentToClose != null);
+      Preconditions.assertTrue(segmentToClose != null);
 
       File openFile = storage.getStorageDir()
           .getOpenLogFile(segmentToClose.getStartIndex());
-      RaftUtils.assertTrue(openFile.exists(),
+      Preconditions.assertTrue(openFile.exists(),
           "File %s does not exist.", openFile);
       if (segmentToClose.numOfEntries() > 0) {
         // finalize the current open segment
         File dstFile = storage.getStorageDir().getClosedLogFile(
             segmentToClose.getStartIndex(), segmentToClose.getEndIndex());
-        RaftUtils.assertTrue(!dstFile.exists());
+        Preconditions.assertTrue(!dstFile.exists());
 
         NativeIO.renameTo(openFile, dstFile);
       } else { // delete the file of the empty segment
@@ -288,9 +289,9 @@ private class StartLogSegment extends Task {
     @Override
     void execute() throws IOException {
       File openFile = storage.getStorageDir().getOpenLogFile(newStartIndex);
-      RaftUtils.assertTrue(!openFile.exists(), "open file %s exists for %s",
+      Preconditions.assertTrue(!openFile.exists(), "open file %s exists for %s",
           openFile.getAbsolutePath(), RaftLogWorker.this.toString());
-      RaftUtils.assertTrue(out == null && pendingFlushNum == 0);
+      Preconditions.assertTrue(out == null && pendingFlushNum == 0);
       out = new LogOutputStream(openFile, false, properties);
     }
 
@@ -309,7 +310,7 @@ private class TruncateLog extends Task {
 
     @Override
     void execute() throws IOException {
-      RaftUtils.cleanup(null, out);
+      IOUtils.cleanup(null, out);
       out = null;
       if (segments.toTruncate != null) {
         File fileToTruncate = segments.toTruncate.isOpen ?
@@ -323,7 +324,7 @@ void execute() throws IOException {
         // rename the file
         File dstFile = storage.getStorageDir().getClosedLogFile(
             segments.toTruncate.startIndex, segments.toTruncate.newEndIndex);
-        RaftUtils.assertTrue(!dstFile.exists());
+        Preconditions.assertTrue(!dstFile.exists());
         NativeIO.renameTo(fileToTruncate, dstFile);
 
         // update lastWrittenIndex
diff --git a/ratis-server/src/main/java/org/apache/ratis/server/storage/RaftStorage.java b/ratis-server/src/main/java/org/apache/ratis/server/storage/RaftStorage.java
index a7bc47cdbd..db8a196404 100644
--- a/ratis-server/src/main/java/org/apache/ratis/server/storage/RaftStorage.java
+++ b/ratis-server/src/main/java/org/apache/ratis/server/storage/RaftStorage.java
@@ -24,7 +24,7 @@
 import org.apache.ratis.statemachine.SnapshotInfo;
 import org.apache.ratis.statemachine.StateMachineStorage;
 import org.apache.ratis.util.FileUtils;
-import org.apache.ratis.util.RaftUtils;
+import org.apache.ratis.util.Preconditions;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
@@ -54,7 +54,7 @@ public RaftStorage(RaftProperties prop, RaftServerConstants.StartupOption option
       storageDir.lock();
       format();
       state = storageDir.analyzeStorage(false);
-      RaftUtils.assertTrue(state == StorageState.NORMAL);
+      Preconditions.assertTrue(state == StorageState.NORMAL);
     } else {
       state = analyzeAndRecoverStorage(true); // metaFile is initialized here
       if (state != StorageState.NORMAL) {
diff --git a/ratis-server/src/main/java/org/apache/ratis/server/storage/SegmentedRaftLog.java b/ratis-server/src/main/java/org/apache/ratis/server/storage/SegmentedRaftLog.java
index fc35535ec4..2fd4dd25d9 100644
--- a/ratis-server/src/main/java/org/apache/ratis/server/storage/SegmentedRaftLog.java
+++ b/ratis-server/src/main/java/org/apache/ratis/server/storage/SegmentedRaftLog.java
@@ -27,7 +27,7 @@
 import org.apache.ratis.shaded.proto.RaftProtos.LogEntryProto;
 import org.apache.ratis.util.AutoCloseableLock;
 import org.apache.ratis.util.CodeInjectionForTesting;
-import org.apache.ratis.util.RaftUtils;
+import org.apache.ratis.util.Preconditions;
 
 import java.io.File;
 import java.io.IOException;
@@ -213,7 +213,7 @@ void appendEntry(LogEntryProto entry) {
         // the term changes
         final long currentTerm = currentOpenSegment.getLastRecord().entry
             .getTerm();
-        RaftUtils.assertTrue(currentTerm < entry.getTerm(),
+        Preconditions.assertTrue(currentTerm < entry.getTerm(),
             "open segment's term %s is larger than the new entry's term %s",
             currentTerm, entry.getTerm());
         cache.rollOpenSegment(true);
@@ -249,7 +249,7 @@ public void append(LogEntryProto... entries) {
       long truncateIndex = -1;
       for (; iter.hasNext() && index < entries.length; index++) {
         LogEntryProto storedEntry = iter.next();
-        RaftUtils.assertTrue(
+        Preconditions.assertTrue(
             storedEntry.getIndex() == entries[index].getIndex(),
             "The stored entry's index %s is not consistent with" +
                 " the received entries[%s]'s index %s", storedEntry.getIndex(),
diff --git a/ratis-server/src/main/java/org/apache/ratis/server/storage/SnapshotManager.java b/ratis-server/src/main/java/org/apache/ratis/server/storage/SnapshotManager.java
index 0b4e8fa73a..bd25d076e0 100644
--- a/ratis-server/src/main/java/org/apache/ratis/server/storage/SnapshotManager.java
+++ b/ratis-server/src/main/java/org/apache/ratis/server/storage/SnapshotManager.java
@@ -29,8 +29,9 @@
 import org.apache.ratis.statemachine.SnapshotInfo;
 import org.apache.ratis.statemachine.StateMachine;
 import org.apache.ratis.util.FileUtils;
+import org.apache.ratis.util.IOUtils;
 import org.apache.ratis.util.MD5FileUtil;
-import org.apache.ratis.util.RaftUtils;
+import org.apache.ratis.util.Preconditions;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
@@ -89,7 +90,7 @@ public void installSnapshot(StateMachine stateMachine,
           // create the temp snapshot file and put padding inside
           out = new FileOutputStream(tmpSnapshotFile);
         } else {
-          RaftUtils.assertTrue(tmpSnapshotFile.exists());
+          Preconditions.assertTrue(tmpSnapshotFile.exists());
           out = new FileOutputStream(tmpSnapshotFile, true);
           FileChannel fc = out.getChannel();
           fc.position(chunk.getOffset());
@@ -98,7 +99,7 @@ public void installSnapshot(StateMachine stateMachine,
         // write data to the file
         out.write(chunk.getData().toByteArray());
       } finally {
-        RaftUtils.cleanup(null, out);
+        IOUtils.cleanup(null, out);
       }
 
       // rename the temp snapshot file if this is the last chunk. also verify
diff --git a/ratis-server/src/test/java/org/apache/ratis/MiniRaftCluster.java b/ratis-server/src/test/java/org/apache/ratis/MiniRaftCluster.java
index 2a5f314388..8c4a7c687f 100644
--- a/ratis-server/src/test/java/org/apache/ratis/MiniRaftCluster.java
+++ b/ratis-server/src/test/java/org/apache/ratis/MiniRaftCluster.java
@@ -17,6 +17,19 @@
  */
 package org.apache.ratis;
 
+import java.io.File;
+import java.io.IOException;
+import java.net.InetSocketAddress;
+import java.util.ArrayList;
+import java.util.Arrays;
+import java.util.Collection;
+import java.util.List;
+import java.util.Map;
+import java.util.concurrent.ConcurrentHashMap;
+import java.util.concurrent.TimeUnit;
+import java.util.stream.Collectors;
+import java.util.stream.StreamSupport;
+
 import org.apache.ratis.client.ClientFactory;
 import org.apache.ratis.client.RaftClient;
 import org.apache.ratis.conf.Parameters;
@@ -25,28 +38,24 @@
 import org.apache.ratis.protocol.RaftPeerId;
 import org.apache.ratis.rpc.RpcType;
 import org.apache.ratis.server.RaftServerConfigKeys;
-import org.apache.ratis.server.impl.*;
+import org.apache.ratis.server.impl.DelayLocalExecutionInjection;
+import org.apache.ratis.server.impl.LeaderState;
+import org.apache.ratis.server.impl.RaftConfiguration;
+import org.apache.ratis.server.impl.RaftServerImpl;
 import org.apache.ratis.server.storage.MemoryRaftLog;
 import org.apache.ratis.server.storage.RaftLog;
 import org.apache.ratis.statemachine.BaseStateMachine;
 import org.apache.ratis.statemachine.StateMachine;
 import org.apache.ratis.util.ExitUtils;
 import org.apache.ratis.util.FileUtils;
+import org.apache.ratis.util.CollectionUtils;
 import org.apache.ratis.util.NetUtils;
-import org.apache.ratis.util.RaftUtils;
+import org.apache.ratis.util.Preconditions;
+import org.apache.ratis.util.ReflectionUtils;
 import org.junit.Assert;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
-import java.io.File;
-import java.io.IOException;
-import java.net.InetSocketAddress;
-import java.util.*;
-import java.util.concurrent.ConcurrentHashMap;
-import java.util.concurrent.TimeUnit;
-import java.util.stream.Collectors;
-import java.util.stream.StreamSupport;
-
 public abstract class MiniRaftCluster {
   public static final Logger LOG = LoggerFactory.getLogger(MiniRaftCluster.class);
   public static final DelayLocalExecutionInjection logSyncDelay =
@@ -117,7 +126,7 @@ private static String getBaseDirectory() {
 
   private static void formatDir(String dirStr) {
     final File serverDir = new File(dirStr);
-    RaftUtils.assertTrue(FileUtils.fullyDelete(serverDir),
+    Preconditions.assertTrue(FileUtils.fullyDelete(serverDir),
         "Failed to format directory %s", dirStr);
     LOG.info("Formatted directory {}", dirStr);
   }
@@ -152,14 +161,14 @@ protected MiniRaftCluster(String[] ids, RaftProperties properties, Parameters pa
 
   public MiniRaftCluster initServers() {
     if (servers.isEmpty()) {
-      putNewServers(RaftUtils.as(conf.getPeers(), RaftPeer::getId), true);
+      putNewServers(CollectionUtils.as(conf.getPeers(), RaftPeer::getId), true);
     }
     return this;
   }
 
   private RaftServerImpl putNewServer(RaftPeerId id, boolean format) {
     final RaftServerImpl s = newRaftServer(id, format);
-    RaftUtils.assertTrue(servers.put(id, s) == null);
+    Preconditions.assertTrue(servers.put(id, s) == null);
     return s;
   }
 
@@ -228,7 +237,7 @@ static StateMachine getStateMachine4Test(RaftProperties properties) {
         STATEMACHINE_CLASS_KEY,
         STATEMACHINE_CLASS_DEFAULT,
         StateMachine.class);
-    return RaftUtils.newInstance(smClass);
+    return ReflectionUtils.newInstance(smClass);
   }
 
   public static Collection toRaftPeers(
@@ -253,7 +262,7 @@ public PeerChanges addNewPeers(String[] ids,
 
     // create and add new RaftServers
     final Collection newServers = putNewServers(
-        RaftUtils.as(Arrays.asList(ids), RaftPeerId::new), true);
+        CollectionUtils.as(Arrays.asList(ids), RaftPeerId::new), true);
     newServers.forEach(s -> startServer(s, startNewPeer));
 
     final Collection newPeers = toRaftPeers(newServers);
diff --git a/ratis-server/src/test/java/org/apache/ratis/RaftNotLeaderExceptionBaseTest.java b/ratis-server/src/test/java/org/apache/ratis/RaftNotLeaderExceptionBaseTest.java
index 54cfa4de73..d60f8ebf39 100644
--- a/ratis-server/src/test/java/org/apache/ratis/RaftNotLeaderExceptionBaseTest.java
+++ b/ratis-server/src/test/java/org/apache/ratis/RaftNotLeaderExceptionBaseTest.java
@@ -30,7 +30,7 @@
 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.*;
 import org.junit.rules.Timeout;
 import org.slf4j.Logger;
@@ -44,10 +44,10 @@
 
 public abstract class RaftNotLeaderExceptionBaseTest {
   static {
-    RaftUtils.setLogLevel(RaftServerImpl.LOG, Level.DEBUG);
-    RaftUtils.setLogLevel(RaftLog.LOG, Level.DEBUG);
-    RaftUtils.setLogLevel(RequestHandler.LOG, Level.DEBUG);
-    RaftUtils.setLogLevel(RaftClient.LOG, Level.DEBUG);
+    LogUtils.setLogLevel(RaftServerImpl.LOG, Level.DEBUG);
+    LogUtils.setLogLevel(RaftLog.LOG, Level.DEBUG);
+    LogUtils.setLogLevel(RequestHandler.LOG, Level.DEBUG);
+    LogUtils.setLogLevel(RaftClient.LOG, Level.DEBUG);
   }
 
   public static final Logger LOG =
diff --git a/ratis-server/src/test/java/org/apache/ratis/server/impl/RaftReconfigurationBaseTest.java b/ratis-server/src/test/java/org/apache/ratis/server/impl/RaftReconfigurationBaseTest.java
index bf7de9f523..6884e40afb 100644
--- a/ratis-server/src/test/java/org/apache/ratis/server/impl/RaftReconfigurationBaseTest.java
+++ b/ratis-server/src/test/java/org/apache/ratis/server/impl/RaftReconfigurationBaseTest.java
@@ -17,6 +17,21 @@
  */
 package org.apache.ratis.server.impl;
 
+import static java.util.Arrays.asList;
+import static org.apache.ratis.MiniRaftCluster.leaderPlaceHolderDelay;
+import static org.apache.ratis.MiniRaftCluster.logSyncDelay;
+import static org.apache.ratis.server.impl.RaftServerConstants.DEFAULT_CALLID;
+import static org.apache.ratis.server.impl.RaftServerTestUtil.waitAndCheckNewConf;
+import static org.apache.ratis.shaded.proto.RaftProtos.LogEntryProto.LogEntryBodyCase.CONFIGURATIONENTRY;
+
+import java.io.IOException;
+import java.util.ArrayList;
+import java.util.Arrays;
+import java.util.Collections;
+import java.util.concurrent.CountDownLatch;
+import java.util.concurrent.atomic.AtomicBoolean;
+import java.util.concurrent.atomic.AtomicReference;
+
 import org.apache.log4j.Level;
 import org.apache.ratis.MiniRaftCluster;
 import org.apache.ratis.MiniRaftCluster.PeerChanges;
@@ -25,11 +40,19 @@
 import org.apache.ratis.client.RaftClient;
 import org.apache.ratis.client.RaftClientRpc;
 import org.apache.ratis.conf.RaftProperties;
-import org.apache.ratis.protocol.*;
+import org.apache.ratis.protocol.ClientId;
+import org.apache.ratis.protocol.LeaderNotReadyException;
+import org.apache.ratis.protocol.RaftClientReply;
+import org.apache.ratis.protocol.RaftClientRequest;
+import org.apache.ratis.protocol.RaftPeer;
+import org.apache.ratis.protocol.RaftPeerId;
+import org.apache.ratis.protocol.ReconfigurationInProgressException;
+import org.apache.ratis.protocol.ReconfigurationTimeoutException;
+import org.apache.ratis.protocol.SetConfigurationRequest;
 import org.apache.ratis.server.RaftServerConfigKeys;
 import org.apache.ratis.server.simulation.RequestHandler;
 import org.apache.ratis.server.storage.RaftLog;
-import org.apache.ratis.util.RaftUtils;
+import org.apache.ratis.util.LogUtils;
 import org.junit.Assert;
 import org.junit.BeforeClass;
 import org.junit.Test;
@@ -37,26 +60,11 @@
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
-import java.io.IOException;
-import java.util.ArrayList;
-import java.util.Arrays;
-import java.util.Collections;
-import java.util.concurrent.CountDownLatch;
-import java.util.concurrent.atomic.AtomicBoolean;
-import java.util.concurrent.atomic.AtomicReference;
-
-import static java.util.Arrays.asList;
-import static org.apache.ratis.MiniRaftCluster.leaderPlaceHolderDelay;
-import static org.apache.ratis.MiniRaftCluster.logSyncDelay;
-import static org.apache.ratis.server.impl.RaftServerConstants.DEFAULT_CALLID;
-import static org.apache.ratis.server.impl.RaftServerTestUtil.waitAndCheckNewConf;
-import static org.apache.ratis.shaded.proto.RaftProtos.LogEntryProto.LogEntryBodyCase.CONFIGURATIONENTRY;
-
 public abstract class RaftReconfigurationBaseTest {
   static {
-    RaftUtils.setLogLevel(RaftServerImpl.LOG, Level.DEBUG);
-    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);
   }
   static final Logger LOG = LoggerFactory.getLogger(RaftReconfigurationBaseTest.class);
 
diff --git a/ratis-server/src/test/java/org/apache/ratis/server/simulation/SimulatedRequestReply.java b/ratis-server/src/test/java/org/apache/ratis/server/simulation/SimulatedRequestReply.java
index 48c9d5580a..528cc1a5d1 100644
--- a/ratis-server/src/test/java/org/apache/ratis/server/simulation/SimulatedRequestReply.java
+++ b/ratis-server/src/test/java/org/apache/ratis/server/simulation/SimulatedRequestReply.java
@@ -21,7 +21,8 @@
 import org.apache.ratis.protocol.RaftPeerId;
 import org.apache.ratis.protocol.RaftRpcMessage;
 import org.apache.ratis.server.RaftServerConfigKeys;
-import org.apache.ratis.util.RaftUtils;
+import org.apache.ratis.util.IOUtils;
+import org.apache.ratis.util.Preconditions;
 import org.apache.ratis.util.Timestamp;
 
 import java.io.IOException;
@@ -43,7 +44,7 @@ private static class ReplyOrException {
     private final IOException ioe;
 
     ReplyOrException(REPLY reply, IOException ioe) {
-      RaftUtils.assertTrue(reply == null ^ ioe == null);
+      Preconditions.assertTrue(reply == null ^ ioe == null);
       this.reply = reply;
       this.ioe = ioe;
     }
@@ -120,7 +121,7 @@ public REPLY sendRequest(REQUEST request) throws IOException {
       RaftTestUtil.block(q.blockSendRequestTo::get);
       return q.request(request);
     } catch (InterruptedException e) {
-      throw RaftUtils.toInterruptedIOException("", e);
+      throw IOUtils.toInterruptedIOException("", e);
     }
   }
 
@@ -136,7 +137,7 @@ public REQUEST takeRequest(String qid) throws IOException {
       RaftTestUtil.delay(q.delayTakeRequestTo::get);
 
       request = q.takeRequest();
-      RaftUtils.assertTrue(qid.equals(request.getReplierId()));
+      Preconditions.assertTrue(qid.equals(request.getReplierId()));
 
       // block request for testing
       final EventQueue reqQ = queues.get(request.getRequestorId());
@@ -145,7 +146,7 @@ public REQUEST takeRequest(String qid) throws IOException {
         RaftTestUtil.block(reqQ.blockTakeRequestFrom::get);
       }
     } catch (InterruptedException e) {
-      throw RaftUtils.toInterruptedIOException("", e);
+      throw IOUtils.toInterruptedIOException("", e);
     }
     return request;
   }
@@ -153,9 +154,9 @@ public REQUEST takeRequest(String qid) throws IOException {
   public void sendReply(REQUEST request, REPLY reply, IOException ioe)
       throws IOException {
     if (reply != null) {
-      RaftUtils.assertTrue(
+      Preconditions.assertTrue(
           request.getRequestorId().equals(reply.getRequestorId()));
-      RaftUtils.assertTrue(
+      Preconditions.assertTrue(
           request.getReplierId().equals(reply.getReplierId()));
     }
     simulateLatency();
@@ -187,7 +188,7 @@ private void simulateLatency() throws IOException {
       try {
         Thread.sleep(randomSleepMs);
       } catch (InterruptedException ie) {
-        throw RaftUtils.toInterruptedIOException("", ie);
+        throw IOUtils.toInterruptedIOException("", ie);
       }
     }
   }
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
index 5332064d6b..c1136b7a9d 100644
--- 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
@@ -20,17 +20,15 @@
 import org.apache.log4j.Level;
 import org.apache.ratis.RaftBasicTests;
 import org.apache.ratis.client.RaftClient;
-import org.apache.ratis.conf.RaftProperties;
 import org.apache.ratis.server.impl.RaftServerImpl;
-import org.apache.ratis.util.RaftUtils;
+import org.apache.ratis.util.LogUtils;
 
 import java.io.IOException;
-import java.util.concurrent.ThreadLocalRandom;
 
 public class TestRaftWithSimulatedRpc extends RaftBasicTests {
   static {
-    RaftUtils.setLogLevel(RaftServerImpl.LOG, Level.DEBUG);
-    RaftUtils.setLogLevel(RaftClient.LOG, Level.DEBUG);
+    LogUtils.setLogLevel(RaftServerImpl.LOG, Level.DEBUG);
+    LogUtils.setLogLevel(RaftClient.LOG, Level.DEBUG);
   }
 
   private final MiniRaftClusterWithSimulatedRpc cluster;
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
index 5e896e2f0a..64cf0d7bd4 100644
--- 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
@@ -17,18 +17,18 @@
  */
 package org.apache.ratis.server.simulation;
 
+import java.io.IOException;
+
 import org.apache.log4j.Level;
 import org.apache.ratis.RaftRetryCacheTests;
 import org.apache.ratis.client.RaftClient;
 import org.apache.ratis.server.impl.RaftServerImpl;
-import org.apache.ratis.util.RaftUtils;
-
-import java.io.IOException;
+import org.apache.ratis.util.LogUtils;
 
 public class TestRetryCacheWithSimulatedRpc extends RaftRetryCacheTests {
   static {
-    RaftUtils.setLogLevel(RaftServerImpl.LOG, Level.DEBUG);
-    RaftUtils.setLogLevel(RaftClient.LOG, Level.DEBUG);
+    LogUtils.setLogLevel(RaftServerImpl.LOG, Level.DEBUG);
+    LogUtils.setLogLevel(RaftClient.LOG, Level.DEBUG);
   }
 
   private final MiniRaftClusterWithSimulatedRpc cluster;
diff --git a/ratis-server/src/test/java/org/apache/ratis/server/storage/TestSegmentedRaftLog.java b/ratis-server/src/test/java/org/apache/ratis/server/storage/TestSegmentedRaftLog.java
index 1fcb54b060..ccf769061d 100644
--- a/ratis-server/src/test/java/org/apache/ratis/server/storage/TestSegmentedRaftLog.java
+++ b/ratis-server/src/test/java/org/apache/ratis/server/storage/TestSegmentedRaftLog.java
@@ -22,6 +22,7 @@
 import org.apache.ratis.RaftTestUtil;
 import org.apache.ratis.RaftTestUtil.SimpleOperation;
 import org.apache.ratis.conf.RaftProperties;
+import org.apache.ratis.util.LogUtils;
 import org.apache.ratis.util.SizeInBytes;
 import org.apache.ratis.protocol.ClientId;
 import org.apache.ratis.protocol.RaftPeerId;
@@ -32,7 +33,6 @@
 import org.apache.ratis.shaded.proto.RaftProtos.LogEntryProto;
 import org.apache.ratis.util.FileUtils;
 import org.apache.ratis.util.ProtoUtils;
-import org.apache.ratis.util.RaftUtils;
 import org.junit.After;
 import org.junit.Assert;
 import org.junit.Before;
@@ -48,7 +48,7 @@
 
 public class TestSegmentedRaftLog {
   static {
-    RaftUtils.setLogLevel(RaftLogWorker.LOG, Level.DEBUG);
+    LogUtils.setLogLevel(RaftLogWorker.LOG, Level.DEBUG);
   }
 
   private static final RaftPeerId peerId = new RaftPeerId("s0");
diff --git a/ratis-server/src/test/java/org/apache/ratis/statemachine/RaftSnapshotBaseTest.java b/ratis-server/src/test/java/org/apache/ratis/statemachine/RaftSnapshotBaseTest.java
index 68ab8c3c83..f5e23788c9 100644
--- a/ratis-server/src/test/java/org/apache/ratis/statemachine/RaftSnapshotBaseTest.java
+++ b/ratis-server/src/test/java/org/apache/ratis/statemachine/RaftSnapshotBaseTest.java
@@ -36,7 +36,7 @@
 import org.apache.ratis.server.storage.RaftStorageDirectory.LogPathAndIndex;
 import org.apache.ratis.shaded.proto.RaftProtos.LogEntryProto;
 import org.apache.ratis.util.FileUtils;
-import org.apache.ratis.util.RaftUtils;
+import org.apache.ratis.util.LogUtils;
 import org.junit.After;
 import org.junit.Assert;
 import org.junit.Before;
@@ -51,10 +51,10 @@
 
 public abstract class RaftSnapshotBaseTest {
   static {
-    RaftUtils.setLogLevel(RaftServerImpl.LOG, Level.DEBUG);
-    RaftUtils.setLogLevel(RaftLog.LOG, Level.DEBUG);
-    RaftUtils.setLogLevel(RequestHandler.LOG, Level.DEBUG);
-    RaftUtils.setLogLevel(RaftClient.LOG, Level.DEBUG);
+    LogUtils.setLogLevel(RaftServerImpl.LOG, Level.DEBUG);
+    LogUtils.setLogLevel(RaftLog.LOG, Level.DEBUG);
+    LogUtils.setLogLevel(RequestHandler.LOG, Level.DEBUG);
+    LogUtils.setLogLevel(RaftClient.LOG, Level.DEBUG);
   }
 
   static final Logger LOG = LoggerFactory.getLogger(RaftSnapshotBaseTest.class);
diff --git a/ratis-server/src/test/java/org/apache/ratis/statemachine/SimpleStateMachine4Testing.java b/ratis-server/src/test/java/org/apache/ratis/statemachine/SimpleStateMachine4Testing.java
index 44c420c86d..6cb0234619 100644
--- a/ratis-server/src/test/java/org/apache/ratis/statemachine/SimpleStateMachine4Testing.java
+++ b/ratis-server/src/test/java/org/apache/ratis/statemachine/SimpleStateMachine4Testing.java
@@ -36,7 +36,7 @@
 import org.apache.ratis.util.Daemon;
 import org.apache.ratis.util.LifeCycle;
 import org.apache.ratis.util.MD5FileUtil;
-import org.apache.ratis.util.RaftUtils;
+import org.apache.ratis.util.Preconditions;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
@@ -197,7 +197,7 @@ private synchronized long loadSnapshot(SingleFileSnapshotInfo snapshot)
           termIndexTracker.update(ServerProtoUtils.toTermIndex(entry));
         }
       }
-      RaftUtils.assertTrue(
+      Preconditions.assertTrue(
           !list.isEmpty() && endIndex == list.get(list.size() - 1).getIndex(),
           "endIndex=%s, list=%s", endIndex, list);
       this.endIndexLastCkpt = endIndex;
diff --git a/ratis-server/src/test/java/org/apache/ratis/statemachine/TermIndexTracker.java b/ratis-server/src/test/java/org/apache/ratis/statemachine/TermIndexTracker.java
index a282d97db1..bb359d7c70 100644
--- a/ratis-server/src/test/java/org/apache/ratis/statemachine/TermIndexTracker.java
+++ b/ratis-server/src/test/java/org/apache/ratis/statemachine/TermIndexTracker.java
@@ -18,7 +18,7 @@
 package org.apache.ratis.statemachine;
 
 import org.apache.ratis.server.protocol.TermIndex;
-import org.apache.ratis.util.RaftUtils;
+import org.apache.ratis.util.Preconditions;
 
 import java.util.Objects;
 
@@ -53,7 +53,7 @@ public synchronized void reset() {
    */
   public synchronized void update(TermIndex termIndex) {
     Objects.requireNonNull(termIndex);
-    RaftUtils.assertTrue(termIndex.compareTo(current) >= 0);
+    Preconditions.assertTrue(termIndex.compareTo(current) >= 0);
     this.current = termIndex;
   }
 
diff --git a/ratis-server/src/test/java/org/apache/ratis/statemachine/TestStateMachine.java b/ratis-server/src/test/java/org/apache/ratis/statemachine/TestStateMachine.java
index b81e367582..c1f79a3cbf 100644
--- a/ratis-server/src/test/java/org/apache/ratis/statemachine/TestStateMachine.java
+++ b/ratis-server/src/test/java/org/apache/ratis/statemachine/TestStateMachine.java
@@ -28,7 +28,7 @@
 import org.apache.ratis.server.impl.RaftServerImpl;
 import org.apache.ratis.server.simulation.MiniRaftClusterWithSimulatedRpc;
 import org.apache.ratis.shaded.proto.RaftProtos.SMLogEntryProto;
-import org.apache.ratis.util.RaftUtils;
+import org.apache.ratis.util.LogUtils;
 import org.junit.*;
 import org.junit.rules.Timeout;
 
@@ -50,8 +50,8 @@
  */
 public class TestStateMachine {
   static {
-    RaftUtils.setLogLevel(RaftServerImpl.LOG, Level.DEBUG);
-    RaftUtils.setLogLevel(RaftClient.LOG, Level.DEBUG);
+    LogUtils.setLogLevel(RaftServerImpl.LOG, Level.DEBUG);
+    LogUtils.setLogLevel(RaftClient.LOG, Level.DEBUG);
   }
 
   public static final int NUM_SERVERS = 5;