From commits-return-482-archive-asf-public=cust-asf.ponee.io@ratis.incubator.apache.org Thu Dec 6 20:32:50 2018 Return-Path: X-Original-To: archive-asf-public@cust-asf.ponee.io Delivered-To: archive-asf-public@cust-asf.ponee.io Received: from mail.apache.org (hermes.apache.org [140.211.11.3]) by mx-eu-01.ponee.io (Postfix) with SMTP id B0780180674 for ; Thu, 6 Dec 2018 20:32:48 +0100 (CET) Received: (qmail 49200 invoked by uid 500); 6 Dec 2018 19:32:47 -0000 Mailing-List: contact commits-help@ratis.incubator.apache.org; run by ezmlm Precedence: bulk List-Help: List-Unsubscribe: List-Post: List-Id: Reply-To: dev@ratis.incubator.apache.org Delivered-To: mailing list commits@ratis.incubator.apache.org Received: (qmail 49191 invoked by uid 99); 6 Dec 2018 19:32:47 -0000 Received: from pnap-us-west-generic-nat.apache.org (HELO spamd4-us-west.apache.org) (209.188.14.142) by apache.org (qpsmtpd/0.29) with ESMTP; Thu, 06 Dec 2018 19:32:47 +0000 Received: from localhost (localhost [127.0.0.1]) by spamd4-us-west.apache.org (ASF Mail Server at spamd4-us-west.apache.org) with ESMTP id 4E230C1CDC for ; Thu, 6 Dec 2018 19:32:47 +0000 (UTC) X-Virus-Scanned: Debian amavisd-new at spamd4-us-west.apache.org X-Spam-Flag: NO X-Spam-Score: -11.701 X-Spam-Level: X-Spam-Status: No, score=-11.701 tagged_above=-999 required=6.31 tests=[KAM_ASCII_DIVIDERS=0.8, RCVD_IN_DNSWL_HI=-5, SPF_PASS=-0.001, USER_IN_DEF_SPF_WL=-7.5] autolearn=disabled Received: from mx1-lw-eu.apache.org ([10.40.0.8]) by localhost (spamd4-us-west.apache.org [10.40.0.11]) (amavisd-new, port 10024) with ESMTP id oLamK7rJ4N3e for ; Thu, 6 Dec 2018 19:32:43 +0000 (UTC) Received: from mail.apache.org (hermes.apache.org [140.211.11.3]) by mx1-lw-eu.apache.org (ASF Mail Server at mx1-lw-eu.apache.org) with SMTP id DAEC460E03 for ; Thu, 6 Dec 2018 19:32:41 +0000 (UTC) Received: (qmail 47190 invoked by uid 99); 6 Dec 2018 19:32:41 -0000 Received: from git1-us-west.apache.org (HELO git1-us-west.apache.org) (140.211.11.23) by apache.org (qpsmtpd/0.29) with ESMTP; Thu, 06 Dec 2018 19:32:41 +0000 Received: by git1-us-west.apache.org (ASF Mail Server at git1-us-west.apache.org, from userid 33) id C9BDBE1285; Thu, 6 Dec 2018 19:32:40 +0000 (UTC) Content-Type: text/plain; charset="us-ascii" MIME-Version: 1.0 Content-Transfer-Encoding: 7bit From: szetszwo@apache.org To: commits@ratis.incubator.apache.org Message-Id: <1f9e7231d3d74dd3abf7609794a19760@git.apache.org> X-Mailer: ASF-Git Admin Mailer Subject: incubator-ratis git commit: RATIS-449. Move the functional interfaces to a new package. Date: Thu, 6 Dec 2018 19:32:40 +0000 (UTC) Repository: incubator-ratis Updated Branches: refs/heads/master bef9a72e3 -> 087652d47 RATIS-449. Move the functional interfaces to a new package. Project: http://git-wip-us.apache.org/repos/asf/incubator-ratis/repo Commit: http://git-wip-us.apache.org/repos/asf/incubator-ratis/commit/087652d4 Tree: http://git-wip-us.apache.org/repos/asf/incubator-ratis/tree/087652d4 Diff: http://git-wip-us.apache.org/repos/asf/incubator-ratis/diff/087652d4 Branch: refs/heads/master Commit: 087652d47704dc8b9edf7d28f8e6268af1364b1b Parents: bef9a72 Author: Tsz Wo Nicholas Sze Authored: Thu Dec 6 11:32:13 2018 -0800 Committer: Tsz Wo Nicholas Sze Committed: Thu Dec 6 11:32:13 2018 -0800 ---------------------------------------------------------------------- .../java/org/apache/ratis/conf/ConfUtils.java | 4 +-- .../apache/ratis/util/CheckedBiConsumer.java | 30 ---------------- .../org/apache/ratis/util/CheckedConsumer.java | 37 -------------------- .../org/apache/ratis/util/CheckedFunction.java | 30 ---------------- .../org/apache/ratis/util/CheckedRunnable.java | 36 ------------------- .../org/apache/ratis/util/CheckedSupplier.java | 30 ---------------- .../java/org/apache/ratis/util/JavaUtils.java | 2 ++ .../java/org/apache/ratis/util/LifeCycle.java | 9 ++--- .../java/org/apache/ratis/util/LogUtils.java | 34 +++++++++--------- .../org/apache/ratis/util/PeerProxyMap.java | 3 +- .../java/org/apache/ratis/util/TaskQueue.java | 3 +- .../org/apache/ratis/util/TimeoutScheduler.java | 3 +- .../ratis/util/function/CheckedBiConsumer.java | 28 +++++++++++++++ .../ratis/util/function/CheckedConsumer.java | 37 ++++++++++++++++++++ .../ratis/util/function/CheckedFunction.java | 28 +++++++++++++++ .../ratis/util/function/CheckedRunnable.java | 36 +++++++++++++++++++ .../ratis/util/function/CheckedSupplier.java | 28 +++++++++++++++ .../test/java/org/apache/ratis/BaseTest.java | 7 ++-- .../ratis/examples/filestore/FileInfo.java | 11 ++++-- .../ratis/examples/filestore/FileStore.java | 17 ++++++--- .../examples/filestore/FileStoreClient.java | 13 +++++-- .../org/apache/ratis/TestMultiRaftGroup.java | 6 ++-- .../examples/filestore/FileStoreBaseTest.java | 17 ++++++--- .../java/org/apache/ratis/grpc/GrpcUtil.java | 8 +++-- .../grpc/client/GrpcClientProtocolClient.java | 31 ++++++++++------ ...nedClientProtocolClientSideTranslatorPB.java | 13 +++++-- .../hadooprpc/server/HadoopRpcService.java | 15 +++++--- .../ratis/server/impl/RaftServerProxy.java | 13 +++++-- .../ratis/server/storage/LogOutputStream.java | 6 ++-- .../server/storage/RaftLogSequentialOps.java | 4 +-- .../server/storage/SegmentedRaftLogFormat.java | 4 +-- .../java/org/apache/ratis/MiniRaftCluster.java | 32 ++++++++++++++--- .../org/apache/ratis/WatchRequestTests.java | 5 ++- .../server/impl/GroupManagementBaseTest.java | 4 +-- 34 files changed, 338 insertions(+), 246 deletions(-) ---------------------------------------------------------------------- http://git-wip-us.apache.org/repos/asf/incubator-ratis/blob/087652d4/ratis-common/src/main/java/org/apache/ratis/conf/ConfUtils.java ---------------------------------------------------------------------- diff --git a/ratis-common/src/main/java/org/apache/ratis/conf/ConfUtils.java b/ratis-common/src/main/java/org/apache/ratis/conf/ConfUtils.java index 98ee162..cbc3b5c 100644 --- a/ratis-common/src/main/java/org/apache/ratis/conf/ConfUtils.java +++ b/ratis-common/src/main/java/org/apache/ratis/conf/ConfUtils.java @@ -1,4 +1,4 @@ -/** +/* * Licensed to the Apache Software Foundation (ASF) under one * or more contributor license agreements. See the NOTICE file * distributed with this work for additional information @@ -18,10 +18,10 @@ package org.apache.ratis.conf; import org.apache.ratis.thirdparty.com.google.common.base.Objects; -import org.apache.ratis.util.CheckedBiConsumer; import org.apache.ratis.util.NetUtils; import org.apache.ratis.util.SizeInBytes; import org.apache.ratis.util.TimeDuration; +import org.apache.ratis.util.function.CheckedBiConsumer; import org.slf4j.Logger; import org.slf4j.LoggerFactory; http://git-wip-us.apache.org/repos/asf/incubator-ratis/blob/087652d4/ratis-common/src/main/java/org/apache/ratis/util/CheckedBiConsumer.java ---------------------------------------------------------------------- diff --git a/ratis-common/src/main/java/org/apache/ratis/util/CheckedBiConsumer.java b/ratis-common/src/main/java/org/apache/ratis/util/CheckedBiConsumer.java deleted file mode 100644 index 03256b2..0000000 --- a/ratis-common/src/main/java/org/apache/ratis/util/CheckedBiConsumer.java +++ /dev/null @@ -1,30 +0,0 @@ -/** - * Licensed to the Apache Software Foundation (ASF) under one - * or more contributor license agreements. See the NOTICE file - * distributed with this work for additional information - * regarding copyright ownership. The ASF licenses this file - * to you under the Apache License, Version 2.0 (the - * "License"); you may not use this file except in compliance - * with the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ -package org.apache.ratis.util; - -import java.util.function.BiConsumer; - -/** {@link BiConsumer} with a throws-clause. */ -@FunctionalInterface -public interface CheckedBiConsumer { - /** - * The same as {@link BiConsumer#accept(Object, Object)} - * except that this method is declared with a throws-clause. - */ - void accept(LEFT left, RIGHT right) throws THROWABLE; -} http://git-wip-us.apache.org/repos/asf/incubator-ratis/blob/087652d4/ratis-common/src/main/java/org/apache/ratis/util/CheckedConsumer.java ---------------------------------------------------------------------- diff --git a/ratis-common/src/main/java/org/apache/ratis/util/CheckedConsumer.java b/ratis-common/src/main/java/org/apache/ratis/util/CheckedConsumer.java deleted file mode 100644 index 0532ea1..0000000 --- a/ratis-common/src/main/java/org/apache/ratis/util/CheckedConsumer.java +++ /dev/null @@ -1,37 +0,0 @@ -/** - * Licensed to the Apache Software Foundation (ASF) under one - * or more contributor license agreements. See the NOTICE file - * distributed with this work for additional information - * regarding copyright ownership. The ASF licenses this file - * to you under the Apache License, Version 2.0 (the - * "License"); you may not use this file except in compliance - * with the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ -package org.apache.ratis.util; - -/** Consumer with a throws-clause. */ -@FunctionalInterface -public interface CheckedConsumer { - /** - * The same as {@link java.util.function.Consumer#accept(Object)} - * except that this method is declared with a throws-clause. - */ - void accept(INPUT input) throws THROWABLE; - - /** @return a {@link CheckedFunction} with {@link Void} return type. */ - static CheckedFunction asCheckedFunction( - CheckedConsumer consumer) { - return input -> { - consumer.accept(input); - return null; - }; - } -} http://git-wip-us.apache.org/repos/asf/incubator-ratis/blob/087652d4/ratis-common/src/main/java/org/apache/ratis/util/CheckedFunction.java ---------------------------------------------------------------------- diff --git a/ratis-common/src/main/java/org/apache/ratis/util/CheckedFunction.java b/ratis-common/src/main/java/org/apache/ratis/util/CheckedFunction.java deleted file mode 100644 index 4badc66..0000000 --- a/ratis-common/src/main/java/org/apache/ratis/util/CheckedFunction.java +++ /dev/null @@ -1,30 +0,0 @@ -/** - * Licensed to the Apache Software Foundation (ASF) under one - * or more contributor license agreements. See the NOTICE file - * distributed with this work for additional information - * regarding copyright ownership. The ASF licenses this file - * to you under the Apache License, Version 2.0 (the - * "License"); you may not use this file except in compliance - * with the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ -package org.apache.ratis.util; - -import java.util.function.Function; - -/** Function with a throws-clause. */ -@FunctionalInterface -public interface CheckedFunction { - /** - * The same as {@link Function#apply(Object)} - * except that this method is declared with a throws-clause. - */ - OUTPUT apply(INPUT input) throws THROWABLE; -} http://git-wip-us.apache.org/repos/asf/incubator-ratis/blob/087652d4/ratis-common/src/main/java/org/apache/ratis/util/CheckedRunnable.java ---------------------------------------------------------------------- diff --git a/ratis-common/src/main/java/org/apache/ratis/util/CheckedRunnable.java b/ratis-common/src/main/java/org/apache/ratis/util/CheckedRunnable.java deleted file mode 100644 index 2911254..0000000 --- a/ratis-common/src/main/java/org/apache/ratis/util/CheckedRunnable.java +++ /dev/null @@ -1,36 +0,0 @@ -/* - * Licensed to the Apache Software Foundation (ASF) under one - * or more contributor license agreements. See the NOTICE file - * distributed with this work for additional information - * regarding copyright ownership. The ASF licenses this file - * to you under the Apache License, Version 2.0 (the - * "License"); you may not use this file except in compliance - * with the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ -package org.apache.ratis.util; - -/** Runnable with a throws-clause. */ -@FunctionalInterface -public interface CheckedRunnable { - /** - * The same as {@link Runnable#run()} - * except that this method is declared with a throws-clause. - */ - void run() throws THROWABLE; - - static CheckedSupplier asCheckedSupplier( - CheckedRunnable runnable) { - return () -> { - runnable.run(); - return null; - }; - } -} \ No newline at end of file http://git-wip-us.apache.org/repos/asf/incubator-ratis/blob/087652d4/ratis-common/src/main/java/org/apache/ratis/util/CheckedSupplier.java ---------------------------------------------------------------------- diff --git a/ratis-common/src/main/java/org/apache/ratis/util/CheckedSupplier.java b/ratis-common/src/main/java/org/apache/ratis/util/CheckedSupplier.java deleted file mode 100644 index 9bbb009..0000000 --- a/ratis-common/src/main/java/org/apache/ratis/util/CheckedSupplier.java +++ /dev/null @@ -1,30 +0,0 @@ -/* - * Licensed to the Apache Software Foundation (ASF) under one - * or more contributor license agreements. See the NOTICE file - * distributed with this work for additional information - * regarding copyright ownership. The ASF licenses this file - * to you under the Apache License, Version 2.0 (the - * "License"); you may not use this file except in compliance - * with the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ -package org.apache.ratis.util; - -import java.util.function.Supplier; - -/** Function with a throws-clause. */ -@FunctionalInterface -public interface CheckedSupplier { - /** - * The same as {@link Supplier#get()} - * except that this method is declared with a throws-clause. - */ - OUTPUT get() throws THROWABLE; -} http://git-wip-us.apache.org/repos/asf/incubator-ratis/blob/087652d4/ratis-common/src/main/java/org/apache/ratis/util/JavaUtils.java ---------------------------------------------------------------------- diff --git a/ratis-common/src/main/java/org/apache/ratis/util/JavaUtils.java b/ratis-common/src/main/java/org/apache/ratis/util/JavaUtils.java index 769e12f..1572aa8 100644 --- a/ratis-common/src/main/java/org/apache/ratis/util/JavaUtils.java +++ b/ratis-common/src/main/java/org/apache/ratis/util/JavaUtils.java @@ -17,6 +17,8 @@ */ package org.apache.ratis.util; +import org.apache.ratis.util.function.CheckedRunnable; +import org.apache.ratis.util.function.CheckedSupplier; import org.slf4j.Logger; import org.slf4j.LoggerFactory; http://git-wip-us.apache.org/repos/asf/incubator-ratis/blob/087652d4/ratis-common/src/main/java/org/apache/ratis/util/LifeCycle.java ---------------------------------------------------------------------- diff --git a/ratis-common/src/main/java/org/apache/ratis/util/LifeCycle.java b/ratis-common/src/main/java/org/apache/ratis/util/LifeCycle.java index 824e409..dd91c28 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 @@ -1,4 +1,4 @@ -/** +/* * Licensed to the Apache Software Foundation (ASF) under one * or more contributor license agreements. See the NOTICE file * distributed with this work for additional information @@ -17,6 +17,10 @@ */ package org.apache.ratis.util; +import org.apache.ratis.util.function.CheckedRunnable; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + import java.util.Arrays; import java.util.Collections; import java.util.EnumMap; @@ -25,9 +29,6 @@ import java.util.Map; import java.util.concurrent.atomic.AtomicReference; import java.util.function.BiFunction; -import org.slf4j.Logger; -import org.slf4j.LoggerFactory; - /** * The life cycle of a machine. *

http://git-wip-us.apache.org/repos/asf/incubator-ratis/blob/087652d4/ratis-common/src/main/java/org/apache/ratis/util/LogUtils.java
----------------------------------------------------------------------
diff --git a/ratis-common/src/main/java/org/apache/ratis/util/LogUtils.java b/ratis-common/src/main/java/org/apache/ratis/util/LogUtils.java
index f76852e..d30bae1 100644
--- a/ratis-common/src/main/java/org/apache/ratis/util/LogUtils.java
+++ b/ratis-common/src/main/java/org/apache/ratis/util/LogUtils.java
@@ -1,32 +1,30 @@
 /*
- * *
- *  * Licensed to the Apache Software Foundation (ASF) under one
- *  * or more contributor license agreements.  See the NOTICE file
- *  * distributed with this work for additional information
- *  * regarding copyright ownership.  The ASF licenses this file
- *  * to you under the Apache License, Version 2.0 (the
- *  * "License"); you may not use this file except in compliance
- *  * with the License.  You may obtain a copy of the License at
- *  *
- *  *     http://www.apache.org/licenses/LICENSE-2.0
- *  *
- *  * Unless required by applicable law or agreed to in writing, software
- *  * distributed under the License is distributed on an "AS IS" BASIS,
- *  * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- *  * See the License for the specific language governing permissions and
- *  * limitations under the License.
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
  *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF 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.util.function.CheckedRunnable;
+import org.apache.ratis.util.function.CheckedSupplier;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
 import java.util.concurrent.Callable;
-import java.util.concurrent.CompletableFuture;
 import java.util.function.Supplier;
 
 /**

http://git-wip-us.apache.org/repos/asf/incubator-ratis/blob/087652d4/ratis-common/src/main/java/org/apache/ratis/util/PeerProxyMap.java
----------------------------------------------------------------------
diff --git a/ratis-common/src/main/java/org/apache/ratis/util/PeerProxyMap.java b/ratis-common/src/main/java/org/apache/ratis/util/PeerProxyMap.java
index d3d80ef..c100d44 100644
--- a/ratis-common/src/main/java/org/apache/ratis/util/PeerProxyMap.java
+++ b/ratis-common/src/main/java/org/apache/ratis/util/PeerProxyMap.java
@@ -1,4 +1,4 @@
-/**
+/*
  * Licensed to the Apache Software Foundation (ASF) under one
  * or more contributor license agreements.  See the NOTICE file
  * distributed with this work for additional information
@@ -19,6 +19,7 @@ package org.apache.ratis.util;
 
 import org.apache.ratis.protocol.RaftPeer;
 import org.apache.ratis.protocol.RaftPeerId;
+import org.apache.ratis.util.function.CheckedFunction;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 

http://git-wip-us.apache.org/repos/asf/incubator-ratis/blob/087652d4/ratis-common/src/main/java/org/apache/ratis/util/TaskQueue.java
----------------------------------------------------------------------
diff --git a/ratis-common/src/main/java/org/apache/ratis/util/TaskQueue.java b/ratis-common/src/main/java/org/apache/ratis/util/TaskQueue.java
index c72067b..47f902c 100644
--- a/ratis-common/src/main/java/org/apache/ratis/util/TaskQueue.java
+++ b/ratis-common/src/main/java/org/apache/ratis/util/TaskQueue.java
@@ -1,4 +1,4 @@
-/**
+/*
  * Licensed to the Apache Software Foundation (ASF) under one
  * or more contributor license agreements.  See the NOTICE file
  * distributed with this work for additional information
@@ -17,6 +17,7 @@
  */
 package org.apache.ratis.util;
 
+import org.apache.ratis.util.function.CheckedSupplier;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 

http://git-wip-us.apache.org/repos/asf/incubator-ratis/blob/087652d4/ratis-common/src/main/java/org/apache/ratis/util/TimeoutScheduler.java
----------------------------------------------------------------------
diff --git a/ratis-common/src/main/java/org/apache/ratis/util/TimeoutScheduler.java b/ratis-common/src/main/java/org/apache/ratis/util/TimeoutScheduler.java
index a298cea..db33b0f 100644
--- a/ratis-common/src/main/java/org/apache/ratis/util/TimeoutScheduler.java
+++ b/ratis-common/src/main/java/org/apache/ratis/util/TimeoutScheduler.java
@@ -1,4 +1,4 @@
-/**
+/*
  * Licensed to the Apache Software Foundation (ASF) under one
  * or more contributor license agreements.  See the NOTICE file
  * distributed with this work for additional information
@@ -17,6 +17,7 @@
  */
 package org.apache.ratis.util;
 
+import org.apache.ratis.util.function.CheckedRunnable;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 

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

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

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

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

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

http://git-wip-us.apache.org/repos/asf/incubator-ratis/blob/087652d4/ratis-common/src/test/java/org/apache/ratis/BaseTest.java
----------------------------------------------------------------------
diff --git a/ratis-common/src/test/java/org/apache/ratis/BaseTest.java b/ratis-common/src/test/java/org/apache/ratis/BaseTest.java
index fa15d55..3612d21 100644
--- a/ratis-common/src/test/java/org/apache/ratis/BaseTest.java
+++ b/ratis-common/src/test/java/org/apache/ratis/BaseTest.java
@@ -1,4 +1,4 @@
-/**
+/*
  * Licensed to the Apache Software Foundation (ASF) under one
  * or more contributor license agreements.  See the NOTICE file
  * distributed with this work for additional information
@@ -19,7 +19,10 @@ package org.apache.ratis;
 
 import org.apache.log4j.Level;
 import org.apache.ratis.conf.ConfUtils;
-import org.apache.ratis.util.*;
+import org.apache.ratis.util.FileUtils;
+import org.apache.ratis.util.JavaUtils;
+import org.apache.ratis.util.LogUtils;
+import org.apache.ratis.util.function.CheckedRunnable;
 import org.junit.Assert;
 import org.junit.Rule;
 import org.junit.rules.TestName;

http://git-wip-us.apache.org/repos/asf/incubator-ratis/blob/087652d4/ratis-examples/src/main/java/org/apache/ratis/examples/filestore/FileInfo.java
----------------------------------------------------------------------
diff --git a/ratis-examples/src/main/java/org/apache/ratis/examples/filestore/FileInfo.java b/ratis-examples/src/main/java/org/apache/ratis/examples/filestore/FileInfo.java
index cdbd361..8e63f1f 100644
--- a/ratis-examples/src/main/java/org/apache/ratis/examples/filestore/FileInfo.java
+++ b/ratis-examples/src/main/java/org/apache/ratis/examples/filestore/FileInfo.java
@@ -1,4 +1,4 @@
-/**
+/*
  * Licensed to the Apache Software Foundation (ASF) under one
  * or more contributor license agreements.  See the NOTICE file
  * distributed with this work for additional information
@@ -19,7 +19,14 @@ package org.apache.ratis.examples.filestore;
 
 import org.apache.ratis.protocol.RaftPeerId;
 import org.apache.ratis.thirdparty.com.google.protobuf.ByteString;
-import org.apache.ratis.util.*;
+import org.apache.ratis.util.CollectionUtils;
+import org.apache.ratis.util.FileUtils;
+import org.apache.ratis.util.JavaUtils;
+import org.apache.ratis.util.LogUtils;
+import org.apache.ratis.util.Preconditions;
+import org.apache.ratis.util.TaskQueue;
+import org.apache.ratis.util.function.CheckedFunction;
+import org.apache.ratis.util.function.CheckedSupplier;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 

http://git-wip-us.apache.org/repos/asf/incubator-ratis/blob/087652d4/ratis-examples/src/main/java/org/apache/ratis/examples/filestore/FileStore.java
----------------------------------------------------------------------
diff --git a/ratis-examples/src/main/java/org/apache/ratis/examples/filestore/FileStore.java b/ratis-examples/src/main/java/org/apache/ratis/examples/filestore/FileStore.java
index 8dda656..ceab594 100644
--- a/ratis-examples/src/main/java/org/apache/ratis/examples/filestore/FileStore.java
+++ b/ratis-examples/src/main/java/org/apache/ratis/examples/filestore/FileStore.java
@@ -1,4 +1,4 @@
-/**
+/*
  * Licensed to the Apache Software Foundation (ASF) under one
  * or more contributor license agreements.  See the NOTICE file
  * distributed with this work for additional information
@@ -19,10 +19,16 @@ package org.apache.ratis.examples.filestore;
 
 import org.apache.ratis.examples.filestore.FileInfo.ReadOnly;
 import org.apache.ratis.examples.filestore.FileInfo.UnderConstruction;
+import org.apache.ratis.proto.ExamplesProtos.ReadReplyProto;
+import org.apache.ratis.proto.ExamplesProtos.WriteReplyProto;
 import org.apache.ratis.protocol.RaftPeerId;
 import org.apache.ratis.thirdparty.com.google.protobuf.ByteString;
-import org.apache.ratis.proto.ExamplesProtos.*;
-import org.apache.ratis.util.*;
+import org.apache.ratis.util.CollectionUtils;
+import org.apache.ratis.util.FileUtils;
+import org.apache.ratis.util.JavaUtils;
+import org.apache.ratis.util.LogUtils;
+import org.apache.ratis.util.StringUtils;
+import org.apache.ratis.util.function.CheckedSupplier;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
@@ -33,7 +39,10 @@ import java.nio.file.Path;
 import java.nio.file.Paths;
 import java.util.Map;
 import java.util.Objects;
-import java.util.concurrent.*;
+import java.util.concurrent.CompletableFuture;
+import java.util.concurrent.ConcurrentHashMap;
+import java.util.concurrent.ExecutorService;
+import java.util.concurrent.Executors;
 import java.util.function.Function;
 import java.util.function.Supplier;
 

http://git-wip-us.apache.org/repos/asf/incubator-ratis/blob/087652d4/ratis-examples/src/main/java/org/apache/ratis/examples/filestore/FileStoreClient.java
----------------------------------------------------------------------
diff --git a/ratis-examples/src/main/java/org/apache/ratis/examples/filestore/FileStoreClient.java b/ratis-examples/src/main/java/org/apache/ratis/examples/filestore/FileStoreClient.java
index 439dd6a..49feeb8 100644
--- a/ratis-examples/src/main/java/org/apache/ratis/examples/filestore/FileStoreClient.java
+++ b/ratis-examples/src/main/java/org/apache/ratis/examples/filestore/FileStoreClient.java
@@ -1,4 +1,4 @@
-/**
+/*
  * Licensed to the Apache Software Foundation (ASF) under one
  * or more contributor license agreements.  See the NOTICE file
  * distributed with this work for additional information
@@ -19,16 +19,23 @@ package org.apache.ratis.examples.filestore;
 
 import org.apache.ratis.client.RaftClient;
 import org.apache.ratis.conf.RaftProperties;
+import org.apache.ratis.proto.ExamplesProtos.DeleteReplyProto;
+import org.apache.ratis.proto.ExamplesProtos.DeleteRequestProto;
+import org.apache.ratis.proto.ExamplesProtos.FileStoreRequestProto;
+import org.apache.ratis.proto.ExamplesProtos.ReadReplyProto;
+import org.apache.ratis.proto.ExamplesProtos.ReadRequestProto;
+import org.apache.ratis.proto.ExamplesProtos.WriteReplyProto;
+import org.apache.ratis.proto.ExamplesProtos.WriteRequestHeaderProto;
+import org.apache.ratis.proto.ExamplesProtos.WriteRequestProto;
 import org.apache.ratis.protocol.Message;
 import org.apache.ratis.protocol.RaftClientReply;
 import org.apache.ratis.protocol.RaftGroup;
 import org.apache.ratis.protocol.StateMachineException;
 import org.apache.ratis.thirdparty.com.google.protobuf.ByteString;
-import org.apache.ratis.proto.ExamplesProtos.*;
-import org.apache.ratis.util.CheckedFunction;
 import org.apache.ratis.util.JavaUtils;
 import org.apache.ratis.util.Preconditions;
 import org.apache.ratis.util.ProtoUtils;
+import org.apache.ratis.util.function.CheckedFunction;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 

http://git-wip-us.apache.org/repos/asf/incubator-ratis/blob/087652d4/ratis-examples/src/test/java/org/apache/ratis/TestMultiRaftGroup.java
----------------------------------------------------------------------
diff --git a/ratis-examples/src/test/java/org/apache/ratis/TestMultiRaftGroup.java b/ratis-examples/src/test/java/org/apache/ratis/TestMultiRaftGroup.java
index 030badd..68d19be 100644
--- a/ratis-examples/src/test/java/org/apache/ratis/TestMultiRaftGroup.java
+++ b/ratis-examples/src/test/java/org/apache/ratis/TestMultiRaftGroup.java
@@ -1,4 +1,4 @@
-/**
+/*
  * Licensed to the Apache Software Foundation (ASF) under one
  * or more contributor license agreements.  See the NOTICE file
  * distributed with this work for additional information
@@ -24,10 +24,10 @@ import org.apache.ratis.examples.ParameterizedBaseTest;
 import org.apache.ratis.examples.arithmetic.ArithmeticStateMachine;
 import org.apache.ratis.examples.arithmetic.TestArithmetic;
 import org.apache.ratis.protocol.RaftGroup;
-import org.apache.ratis.server.impl.RaftServerImpl;
 import org.apache.ratis.server.impl.GroupManagementBaseTest;
-import org.apache.ratis.util.CheckedBiConsumer;
+import org.apache.ratis.server.impl.RaftServerImpl;
 import org.apache.ratis.util.LogUtils;
+import org.apache.ratis.util.function.CheckedBiConsumer;
 import org.junit.Test;
 import org.junit.runner.RunWith;
 import org.junit.runners.Parameterized;

http://git-wip-us.apache.org/repos/asf/incubator-ratis/blob/087652d4/ratis-examples/src/test/java/org/apache/ratis/examples/filestore/FileStoreBaseTest.java
----------------------------------------------------------------------
diff --git a/ratis-examples/src/test/java/org/apache/ratis/examples/filestore/FileStoreBaseTest.java b/ratis-examples/src/test/java/org/apache/ratis/examples/filestore/FileStoreBaseTest.java
index f746da2..92c81f6 100644
--- a/ratis-examples/src/test/java/org/apache/ratis/examples/filestore/FileStoreBaseTest.java
+++ b/ratis-examples/src/test/java/org/apache/ratis/examples/filestore/FileStoreBaseTest.java
@@ -1,4 +1,4 @@
-/**
+/*
  * Licensed to the Apache Software Foundation (ASF) under one
  * or more contributor license agreements.  See the NOTICE file
  * distributed with this work for additional information
@@ -22,10 +22,14 @@ import org.apache.ratis.MiniRaftCluster;
 import org.apache.ratis.RaftTestUtil;
 import org.apache.ratis.conf.ConfUtils;
 import org.apache.ratis.conf.RaftProperties;
+import org.apache.ratis.statemachine.StateMachine;
 import org.apache.ratis.thirdparty.com.google.protobuf.ByteString;
 import org.apache.ratis.thirdparty.io.netty.util.internal.ThreadLocalRandom;
-import org.apache.ratis.statemachine.StateMachine;
-import org.apache.ratis.util.*;
+import org.apache.ratis.util.LogUtils;
+import org.apache.ratis.util.Preconditions;
+import org.apache.ratis.util.SizeInBytes;
+import org.apache.ratis.util.StringUtils;
+import org.apache.ratis.util.function.CheckedSupplier;
 import org.junit.Assert;
 import org.junit.Test;
 import org.slf4j.Logger;
@@ -39,7 +43,12 @@ import java.util.ArrayList;
 import java.util.List;
 import java.util.Objects;
 import java.util.Random;
-import java.util.concurrent.*;
+import java.util.concurrent.Callable;
+import java.util.concurrent.CompletableFuture;
+import java.util.concurrent.Executor;
+import java.util.concurrent.ExecutorService;
+import java.util.concurrent.Executors;
+import java.util.concurrent.Future;
 import java.util.concurrent.atomic.AtomicInteger;
 
 public abstract class FileStoreBaseTest

http://git-wip-us.apache.org/repos/asf/incubator-ratis/blob/087652d4/ratis-grpc/src/main/java/org/apache/ratis/grpc/GrpcUtil.java
----------------------------------------------------------------------
diff --git a/ratis-grpc/src/main/java/org/apache/ratis/grpc/GrpcUtil.java b/ratis-grpc/src/main/java/org/apache/ratis/grpc/GrpcUtil.java
index e8ca1ef..482a8de 100644
--- a/ratis-grpc/src/main/java/org/apache/ratis/grpc/GrpcUtil.java
+++ b/ratis-grpc/src/main/java/org/apache/ratis/grpc/GrpcUtil.java
@@ -1,4 +1,4 @@
-/**
+/*
  * Licensed to the Apache Software Foundation (ASF) under one
  * or more contributor license agreements.  See the NOTICE file
  * distributed with this work for additional information
@@ -23,7 +23,11 @@ import org.apache.ratis.thirdparty.io.grpc.Metadata;
 import org.apache.ratis.thirdparty.io.grpc.Status;
 import org.apache.ratis.thirdparty.io.grpc.StatusRuntimeException;
 import org.apache.ratis.thirdparty.io.grpc.stub.StreamObserver;
-import org.apache.ratis.util.*;
+import org.apache.ratis.util.IOUtils;
+import org.apache.ratis.util.JavaUtils;
+import org.apache.ratis.util.LogUtils;
+import org.apache.ratis.util.ReflectionUtils;
+import org.apache.ratis.util.function.CheckedSupplier;
 import org.slf4j.Logger;
 
 import java.io.IOException;

http://git-wip-us.apache.org/repos/asf/incubator-ratis/blob/087652d4/ratis-grpc/src/main/java/org/apache/ratis/grpc/client/GrpcClientProtocolClient.java
----------------------------------------------------------------------
diff --git a/ratis-grpc/src/main/java/org/apache/ratis/grpc/client/GrpcClientProtocolClient.java b/ratis-grpc/src/main/java/org/apache/ratis/grpc/client/GrpcClientProtocolClient.java
index 7aaf4a8..cf239b6 100644
--- a/ratis-grpc/src/main/java/org/apache/ratis/grpc/client/GrpcClientProtocolClient.java
+++ b/ratis-grpc/src/main/java/org/apache/ratis/grpc/client/GrpcClientProtocolClient.java
@@ -1,4 +1,4 @@
-/**
+/*
  * Licensed to the Apache Software Foundation (ASF) under one
  * or more contributor license agreements.  See the NOTICE file
  * distributed with this work for additional information
@@ -22,24 +22,35 @@ import org.apache.ratis.client.impl.ClientProtoUtils;
 import org.apache.ratis.conf.RaftProperties;
 import org.apache.ratis.grpc.GrpcConfigKeys;
 import org.apache.ratis.grpc.GrpcUtil;
-import org.apache.ratis.protocol.*;
-import org.apache.ratis.util.TimeoutScheduler;
-import org.apache.ratis.thirdparty.io.grpc.ManagedChannel;
-import org.apache.ratis.thirdparty.io.grpc.StatusRuntimeException;
-import org.apache.ratis.thirdparty.io.grpc.netty.NegotiationType;
-import org.apache.ratis.thirdparty.io.grpc.netty.NettyChannelBuilder;
-import org.apache.ratis.thirdparty.io.grpc.stub.StreamObserver;
-import org.apache.ratis.proto.RaftProtos.*;
+import org.apache.ratis.proto.RaftProtos.GroupInfoReplyProto;
+import org.apache.ratis.proto.RaftProtos.GroupInfoRequestProto;
+import org.apache.ratis.proto.RaftProtos.GroupListReplyProto;
+import org.apache.ratis.proto.RaftProtos.GroupListRequestProto;
+import org.apache.ratis.proto.RaftProtos.GroupManagementRequestProto;
+import org.apache.ratis.proto.RaftProtos.RaftClientReplyProto;
+import org.apache.ratis.proto.RaftProtos.RaftClientRequestProto;
+import org.apache.ratis.proto.RaftProtos.SetConfigurationRequestProto;
 import org.apache.ratis.proto.grpc.AdminProtocolServiceGrpc;
 import org.apache.ratis.proto.grpc.AdminProtocolServiceGrpc.AdminProtocolServiceBlockingStub;
 import org.apache.ratis.proto.grpc.RaftClientProtocolServiceGrpc;
 import org.apache.ratis.proto.grpc.RaftClientProtocolServiceGrpc.RaftClientProtocolServiceBlockingStub;
 import org.apache.ratis.proto.grpc.RaftClientProtocolServiceGrpc.RaftClientProtocolServiceStub;
-import org.apache.ratis.util.CheckedSupplier;
+import org.apache.ratis.protocol.ClientId;
+import org.apache.ratis.protocol.NotLeaderException;
+import org.apache.ratis.protocol.RaftClientReply;
+import org.apache.ratis.protocol.RaftClientRequest;
+import org.apache.ratis.protocol.RaftPeer;
+import org.apache.ratis.thirdparty.io.grpc.ManagedChannel;
+import org.apache.ratis.thirdparty.io.grpc.StatusRuntimeException;
+import org.apache.ratis.thirdparty.io.grpc.netty.NegotiationType;
+import org.apache.ratis.thirdparty.io.grpc.netty.NettyChannelBuilder;
+import org.apache.ratis.thirdparty.io.grpc.stub.StreamObserver;
 import org.apache.ratis.util.CollectionUtils;
 import org.apache.ratis.util.JavaUtils;
 import org.apache.ratis.util.SizeInBytes;
 import org.apache.ratis.util.TimeDuration;
+import org.apache.ratis.util.TimeoutScheduler;
+import org.apache.ratis.util.function.CheckedSupplier;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 

http://git-wip-us.apache.org/repos/asf/incubator-ratis/blob/087652d4/ratis-hadoop/src/main/java/org/apache/ratis/hadooprpc/client/CombinedClientProtocolClientSideTranslatorPB.java
----------------------------------------------------------------------
diff --git a/ratis-hadoop/src/main/java/org/apache/ratis/hadooprpc/client/CombinedClientProtocolClientSideTranslatorPB.java b/ratis-hadoop/src/main/java/org/apache/ratis/hadooprpc/client/CombinedClientProtocolClientSideTranslatorPB.java
index fc28a4e..f54d5a7 100644
--- a/ratis-hadoop/src/main/java/org/apache/ratis/hadooprpc/client/CombinedClientProtocolClientSideTranslatorPB.java
+++ b/ratis-hadoop/src/main/java/org/apache/ratis/hadooprpc/client/CombinedClientProtocolClientSideTranslatorPB.java
@@ -1,4 +1,4 @@
-/**
+/*
  * Licensed to the Apache Software Foundation (ASF) under one
  * or more contributor license agreements.  See the NOTICE file
  * distributed with this work for additional information
@@ -21,10 +21,17 @@ import org.apache.hadoop.classification.InterfaceAudience;
 import org.apache.hadoop.conf.Configuration;
 import org.apache.ratis.client.impl.ClientProtoUtils;
 import org.apache.ratis.hadooprpc.Proxy;
-import org.apache.ratis.protocol.*;
+import org.apache.ratis.protocol.GroupInfoReply;
+import org.apache.ratis.protocol.GroupInfoRequest;
+import org.apache.ratis.protocol.GroupListReply;
+import org.apache.ratis.protocol.GroupListRequest;
+import org.apache.ratis.protocol.GroupManagementRequest;
+import org.apache.ratis.protocol.RaftClientReply;
+import org.apache.ratis.protocol.RaftClientRequest;
+import org.apache.ratis.protocol.SetConfigurationRequest;
 import org.apache.ratis.thirdparty.com.google.protobuf.ServiceException;
-import org.apache.ratis.util.CheckedFunction;
 import org.apache.ratis.util.ProtoUtils;
+import org.apache.ratis.util.function.CheckedFunction;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 

http://git-wip-us.apache.org/repos/asf/incubator-ratis/blob/087652d4/ratis-hadoop/src/main/java/org/apache/ratis/hadooprpc/server/HadoopRpcService.java
----------------------------------------------------------------------
diff --git a/ratis-hadoop/src/main/java/org/apache/ratis/hadooprpc/server/HadoopRpcService.java b/ratis-hadoop/src/main/java/org/apache/ratis/hadooprpc/server/HadoopRpcService.java
index 891bf73..f1d8b22 100644
--- a/ratis-hadoop/src/main/java/org/apache/ratis/hadooprpc/server/HadoopRpcService.java
+++ b/ratis-hadoop/src/main/java/org/apache/ratis/hadooprpc/server/HadoopRpcService.java
@@ -1,4 +1,4 @@
-/**
+/*
  * Licensed to the Apache Software Foundation (ASF) under one
  * or more contributor license agreements.  See the NOTICE file
  * distributed with this work for additional information
@@ -24,6 +24,14 @@ import org.apache.ratis.hadooprpc.HadoopConfigKeys;
 import org.apache.ratis.hadooprpc.Proxy;
 import org.apache.ratis.hadooprpc.client.CombinedClientProtocolPB;
 import org.apache.ratis.hadooprpc.client.CombinedClientProtocolServerSideTranslatorPB;
+import org.apache.ratis.proto.RaftProtos.AppendEntriesReplyProto;
+import org.apache.ratis.proto.RaftProtos.AppendEntriesRequestProto;
+import org.apache.ratis.proto.RaftProtos.InstallSnapshotReplyProto;
+import org.apache.ratis.proto.RaftProtos.InstallSnapshotRequestProto;
+import org.apache.ratis.proto.RaftProtos.RequestVoteReplyProto;
+import org.apache.ratis.proto.RaftProtos.RequestVoteRequestProto;
+import org.apache.ratis.proto.hadoop.HadoopProtos.CombinedClientProtocolService;
+import org.apache.ratis.proto.hadoop.HadoopProtos.RaftServerProtocolService;
 import org.apache.ratis.protocol.RaftPeerId;
 import org.apache.ratis.rpc.SupportedRpcType;
 import org.apache.ratis.server.RaftServer;
@@ -33,13 +41,10 @@ import org.apache.ratis.server.protocol.RaftServerProtocol;
 import org.apache.ratis.thirdparty.com.google.protobuf.BlockingService;
 import org.apache.ratis.thirdparty.com.google.protobuf.ByteString;
 import org.apache.ratis.thirdparty.com.google.protobuf.ServiceException;
-import org.apache.ratis.proto.RaftProtos.*;
-import org.apache.ratis.proto.hadoop.HadoopProtos.CombinedClientProtocolService;
-import org.apache.ratis.proto.hadoop.HadoopProtos.RaftServerProtocolService;
-import org.apache.ratis.util.CheckedFunction;
 import org.apache.ratis.util.CodeInjectionForTesting;
 import org.apache.ratis.util.PeerProxyMap;
 import org.apache.ratis.util.ProtoUtils;
+import org.apache.ratis.util.function.CheckedFunction;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 

http://git-wip-us.apache.org/repos/asf/incubator-ratis/blob/087652d4/ratis-server/src/main/java/org/apache/ratis/server/impl/RaftServerProxy.java
----------------------------------------------------------------------
diff --git a/ratis-server/src/main/java/org/apache/ratis/server/impl/RaftServerProxy.java b/ratis-server/src/main/java/org/apache/ratis/server/impl/RaftServerProxy.java
index c3ca1f7..6353902 100644
--- a/ratis-server/src/main/java/org/apache/ratis/server/impl/RaftServerProxy.java
+++ b/ratis-server/src/main/java/org/apache/ratis/server/impl/RaftServerProxy.java
@@ -1,4 +1,4 @@
-/**
+/*
  * Licensed to the Apache Software Foundation (ASF) under one
  * or more contributor license agreements.  See the NOTICE file
  * distributed with this work for additional information
@@ -20,19 +20,26 @@ package org.apache.ratis.server.impl;
 import org.apache.ratis.RaftConfigKeys;
 import org.apache.ratis.conf.Parameters;
 import org.apache.ratis.conf.RaftProperties;
+import org.apache.ratis.proto.RaftProtos.AppendEntriesReplyProto;
+import org.apache.ratis.proto.RaftProtos.AppendEntriesRequestProto;
+import org.apache.ratis.proto.RaftProtos.CommitInfoProto;
+import org.apache.ratis.proto.RaftProtos.InstallSnapshotReplyProto;
+import org.apache.ratis.proto.RaftProtos.InstallSnapshotRequestProto;
+import org.apache.ratis.proto.RaftProtos.RaftRpcRequestProto;
+import org.apache.ratis.proto.RaftProtos.RequestVoteReplyProto;
+import org.apache.ratis.proto.RaftProtos.RequestVoteRequestProto;
 import org.apache.ratis.protocol.*;
 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.proto.RaftProtos.*;
 import org.apache.ratis.statemachine.StateMachine;
-import org.apache.ratis.util.CheckedFunction;
 import org.apache.ratis.util.IOUtils;
 import org.apache.ratis.util.JavaUtils;
 import org.apache.ratis.util.LifeCycle;
 import org.apache.ratis.util.Preconditions;
 import org.apache.ratis.util.ProtoUtils;
+import org.apache.ratis.util.function.CheckedFunction;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 

http://git-wip-us.apache.org/repos/asf/incubator-ratis/blob/087652d4/ratis-server/src/main/java/org/apache/ratis/server/storage/LogOutputStream.java
----------------------------------------------------------------------
diff --git a/ratis-server/src/main/java/org/apache/ratis/server/storage/LogOutputStream.java b/ratis-server/src/main/java/org/apache/ratis/server/storage/LogOutputStream.java
index 46da9da..5854e01 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
@@ -1,4 +1,4 @@
-/**
+/*
  * Licensed to the Apache Software Foundation (ASF) under one
  * or more contributor license agreements.  See the NOTICE file
  * distributed with this work for additional information
@@ -17,12 +17,12 @@
  */
 package org.apache.ratis.server.storage;
 
-import org.apache.ratis.thirdparty.com.google.protobuf.CodedOutputStream;
 import org.apache.ratis.proto.RaftProtos.LogEntryProto;
-import org.apache.ratis.util.CheckedConsumer;
+import org.apache.ratis.thirdparty.com.google.protobuf.CodedOutputStream;
 import org.apache.ratis.util.FileUtils;
 import org.apache.ratis.util.IOUtils;
 import org.apache.ratis.util.PureJavaCrc32C;
+import org.apache.ratis.util.function.CheckedConsumer;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 

http://git-wip-us.apache.org/repos/asf/incubator-ratis/blob/087652d4/ratis-server/src/main/java/org/apache/ratis/server/storage/RaftLogSequentialOps.java
----------------------------------------------------------------------
diff --git a/ratis-server/src/main/java/org/apache/ratis/server/storage/RaftLogSequentialOps.java b/ratis-server/src/main/java/org/apache/ratis/server/storage/RaftLogSequentialOps.java
index 0d4df99..8ed92b8 100644
--- a/ratis-server/src/main/java/org/apache/ratis/server/storage/RaftLogSequentialOps.java
+++ b/ratis-server/src/main/java/org/apache/ratis/server/storage/RaftLogSequentialOps.java
@@ -1,4 +1,4 @@
-/**
+/*
  * Licensed to the Apache Software Foundation (ASF) under one
  * or more contributor license agreements.  See the NOTICE file
  * distributed with this work for additional information
@@ -21,9 +21,9 @@ import org.apache.ratis.proto.RaftProtos.LogEntryProto;
 import org.apache.ratis.protocol.StateMachineException;
 import org.apache.ratis.server.impl.RaftConfiguration;
 import org.apache.ratis.statemachine.TransactionContext;
-import org.apache.ratis.util.CheckedSupplier;
 import org.apache.ratis.util.Preconditions;
 import org.apache.ratis.util.StringUtils;
+import org.apache.ratis.util.function.CheckedSupplier;
 
 import java.util.List;
 import java.util.concurrent.CompletableFuture;

http://git-wip-us.apache.org/repos/asf/incubator-ratis/blob/087652d4/ratis-server/src/main/java/org/apache/ratis/server/storage/SegmentedRaftLogFormat.java
----------------------------------------------------------------------
diff --git a/ratis-server/src/main/java/org/apache/ratis/server/storage/SegmentedRaftLogFormat.java b/ratis-server/src/main/java/org/apache/ratis/server/storage/SegmentedRaftLogFormat.java
index 1a00a84..37b3cc6 100644
--- a/ratis-server/src/main/java/org/apache/ratis/server/storage/SegmentedRaftLogFormat.java
+++ b/ratis-server/src/main/java/org/apache/ratis/server/storage/SegmentedRaftLogFormat.java
@@ -1,4 +1,4 @@
-/**
+/*
  * Licensed to the Apache Software Foundation (ASF) under one
  * or more contributor license agreements.  See the NOTICE file
  * distributed with this work for additional information
@@ -17,8 +17,8 @@
  */
 package org.apache.ratis.server.storage;
 
-import org.apache.ratis.util.CheckedFunction;
 import org.apache.ratis.util.Preconditions;
+import org.apache.ratis.util.function.CheckedFunction;
 
 import java.io.IOException;
 import java.nio.charset.StandardCharsets;

http://git-wip-us.apache.org/repos/asf/incubator-ratis/blob/087652d4/ratis-server/src/test/java/org/apache/ratis/MiniRaftCluster.java
----------------------------------------------------------------------
diff --git a/ratis-server/src/test/java/org/apache/ratis/MiniRaftCluster.java b/ratis-server/src/test/java/org/apache/ratis/MiniRaftCluster.java
index 18358a2..ea5730e 100644
--- a/ratis-server/src/test/java/org/apache/ratis/MiniRaftCluster.java
+++ b/ratis-server/src/test/java/org/apache/ratis/MiniRaftCluster.java
@@ -20,7 +20,15 @@ package org.apache.ratis;
 import org.apache.ratis.client.RaftClient;
 import org.apache.ratis.conf.Parameters;
 import org.apache.ratis.conf.RaftProperties;
-import org.apache.ratis.protocol.*;
+import org.apache.ratis.protocol.ClientId;
+import org.apache.ratis.protocol.Message;
+import org.apache.ratis.protocol.RaftClientReply;
+import org.apache.ratis.protocol.RaftClientRequest;
+import org.apache.ratis.protocol.RaftGroup;
+import org.apache.ratis.protocol.RaftGroupId;
+import org.apache.ratis.protocol.RaftPeer;
+import org.apache.ratis.protocol.RaftPeerId;
+import org.apache.ratis.protocol.SetConfigurationRequest;
 import org.apache.ratis.retry.RetryPolicies;
 import org.apache.ratis.retry.RetryPolicy;
 import org.apache.ratis.server.RaftServer;
@@ -31,10 +39,18 @@ import org.apache.ratis.server.impl.RaftServerProxy;
 import org.apache.ratis.server.impl.RaftServerTestUtil;
 import org.apache.ratis.server.storage.MemoryRaftLog;
 import org.apache.ratis.server.storage.RaftLog;
-import org.apache.ratis.proto.RaftProtos.ReplicationLevel;
-import org.apache.ratis.statemachine.impl.BaseStateMachine;
 import org.apache.ratis.statemachine.StateMachine;
-import org.apache.ratis.util.*;
+import org.apache.ratis.statemachine.impl.BaseStateMachine;
+import org.apache.ratis.util.CollectionUtils;
+import org.apache.ratis.util.Daemon;
+import org.apache.ratis.util.ExitUtils;
+import org.apache.ratis.util.FileUtils;
+import org.apache.ratis.util.JavaUtils;
+import org.apache.ratis.util.NetUtils;
+import org.apache.ratis.util.Preconditions;
+import org.apache.ratis.util.ReflectionUtils;
+import org.apache.ratis.util.TimeDuration;
+import org.apache.ratis.util.function.CheckedConsumer;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
@@ -42,7 +58,13 @@ import java.io.Closeable;
 import java.io.File;
 import java.io.IOException;
 import java.net.InetSocketAddress;
-import java.util.*;
+import java.util.ArrayList;
+import java.util.Arrays;
+import java.util.Collection;
+import java.util.Collections;
+import java.util.List;
+import java.util.Map;
+import java.util.Timer;
 import java.util.concurrent.ConcurrentHashMap;
 import java.util.concurrent.ExecutorService;
 import java.util.concurrent.Executors;

http://git-wip-us.apache.org/repos/asf/incubator-ratis/blob/087652d4/ratis-server/src/test/java/org/apache/ratis/WatchRequestTests.java
----------------------------------------------------------------------
diff --git a/ratis-server/src/test/java/org/apache/ratis/WatchRequestTests.java b/ratis-server/src/test/java/org/apache/ratis/WatchRequestTests.java
index a43c42b..8d143eb 100644
--- a/ratis-server/src/test/java/org/apache/ratis/WatchRequestTests.java
+++ b/ratis-server/src/test/java/org/apache/ratis/WatchRequestTests.java
@@ -29,7 +29,10 @@ import org.apache.ratis.server.impl.RaftServerImpl;
 import org.apache.ratis.server.impl.RaftServerTestUtil;
 import org.apache.ratis.statemachine.SimpleStateMachine4Testing;
 import org.apache.ratis.statemachine.StateMachine;
-import org.apache.ratis.util.*;
+import org.apache.ratis.util.LogUtils;
+import org.apache.ratis.util.ProtoUtils;
+import org.apache.ratis.util.TimeDuration;
+import org.apache.ratis.util.function.CheckedConsumer;
 import org.junit.Assert;
 import org.junit.Before;
 import org.junit.Test;

http://git-wip-us.apache.org/repos/asf/incubator-ratis/blob/087652d4/ratis-server/src/test/java/org/apache/ratis/server/impl/GroupManagementBaseTest.java
----------------------------------------------------------------------
diff --git a/ratis-server/src/test/java/org/apache/ratis/server/impl/GroupManagementBaseTest.java b/ratis-server/src/test/java/org/apache/ratis/server/impl/GroupManagementBaseTest.java
index bc3b764..dd8f4a9 100644
--- a/ratis-server/src/test/java/org/apache/ratis/server/impl/GroupManagementBaseTest.java
+++ b/ratis-server/src/test/java/org/apache/ratis/server/impl/GroupManagementBaseTest.java
@@ -1,4 +1,4 @@
-/**
+/*
  * Licensed to the Apache Software Foundation (ASF) under one
  * or more contributor license agreements.  See the NOTICE file
  * distributed with this work for additional information
@@ -28,9 +28,9 @@ import org.apache.ratis.protocol.RaftGroup;
 import org.apache.ratis.protocol.RaftGroupId;
 import org.apache.ratis.protocol.RaftPeer;
 import org.apache.ratis.protocol.RaftPeerId;
-import org.apache.ratis.util.CheckedBiConsumer;
 import org.apache.ratis.util.JavaUtils;
 import org.apache.ratis.util.LogUtils;
+import org.apache.ratis.util.function.CheckedBiConsumer;
 import org.junit.Assert;
 import org.junit.Test;
 import org.slf4j.Logger;