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

incubator-ratis git commit: RATIS-314. Use different loggers for different confs.

Repository: incubator-ratis
Updated Branches:
  refs/heads/master 6369d4e65 -> 50588bde3


RATIS-314. Use different loggers for different confs.


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

Branch: refs/heads/master
Commit: 50588bde3e3d39757f1893c1451ba9a30d66459f
Parents: 6369d4e
Author: Tsz Wo Nicholas Sze <sz...@apache.org>
Authored: Tue Sep 11 14:00:10 2018 -0700
Committer: Tsz Wo Nicholas Sze <sz...@apache.org>
Committed: Tue Sep 11 14:00:10 2018 -0700

----------------------------------------------------------------------
 .../org/apache/ratis/client/RaftClient.java     |  2 +-
 .../ratis/client/RaftClientConfigKeys.java      | 17 +++++--
 .../java/org/apache/ratis/RaftConfigKeys.java   |  6 ++-
 .../java/org/apache/ratis/conf/ConfUtils.java   | 39 ++++++++-------
 .../filestore/FileStoreStateMachine.java        |  2 +-
 .../org/apache/ratis/grpc/GrpcConfigKeys.java   | 34 +++++++++----
 .../org/apache/ratis/grpc/RaftGRpcService.java  |  4 +-
 .../ratis/grpc/client/AppendStreamer.java       |  2 +-
 .../apache/ratis/grpc/client/GrpcClientRpc.java |  2 +-
 .../grpc/client/RaftClientProtocolClient.java   |  8 ++--
 .../ratis/hadooprpc/HadoopConfigKeys.java       | 14 ++++--
 .../org/apache/ratis/netty/NettyConfigKeys.java | 11 ++++-
 .../ratis/server/RaftServerConfigKeys.java      | 50 ++++++++++++--------
 .../ratis/server/impl/RaftServerProxy.java      |  2 +-
 .../MiniRaftClusterWithSimulatedRpc.java        |  2 +-
 15 files changed, 123 insertions(+), 72 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-ratis/blob/50588bde/ratis-client/src/main/java/org/apache/ratis/client/RaftClient.java
----------------------------------------------------------------------
diff --git a/ratis-client/src/main/java/org/apache/ratis/client/RaftClient.java b/ratis-client/src/main/java/org/apache/ratis/client/RaftClient.java
index d28d50a..20d746b 100644
--- a/ratis-client/src/main/java/org/apache/ratis/client/RaftClient.java
+++ b/ratis-client/src/main/java/org/apache/ratis/client/RaftClient.java
@@ -124,7 +124,7 @@ public interface RaftClient extends Closeable {
       }
       if (properties != null) {
         if (clientRpc == null) {
-          final RpcType rpcType = RaftConfigKeys.Rpc.type(properties);
+          final RpcType rpcType = RaftConfigKeys.Rpc.type(properties, LOG::debug);
           final ClientFactory factory = ClientFactory.cast(rpcType.newFactory(parameters));
           clientRpc = factory.newRaftClientRpc(clientId, properties);
         }

http://git-wip-us.apache.org/repos/asf/incubator-ratis/blob/50588bde/ratis-client/src/main/java/org/apache/ratis/client/RaftClientConfigKeys.java
----------------------------------------------------------------------
diff --git a/ratis-client/src/main/java/org/apache/ratis/client/RaftClientConfigKeys.java b/ratis-client/src/main/java/org/apache/ratis/client/RaftClientConfigKeys.java
index cc4e5ec..b54f15c 100644
--- a/ratis-client/src/main/java/org/apache/ratis/client/RaftClientConfigKeys.java
+++ b/ratis-client/src/main/java/org/apache/ratis/client/RaftClientConfigKeys.java
@@ -19,12 +19,21 @@ package org.apache.ratis.client;
 
 import org.apache.ratis.conf.RaftProperties;
 import org.apache.ratis.util.TimeDuration;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
 
 import java.util.concurrent.TimeUnit;
+import java.util.function.Consumer;
 
 import static org.apache.ratis.conf.ConfUtils.*;
 
 public interface RaftClientConfigKeys {
+  Logger LOG = LoggerFactory.getLogger(RaftClientConfigKeys.class);
+
+  static Consumer<String> getDefaultLog() {
+    return LOG::debug;
+  }
+
   String PREFIX = "raft.client";
 
   interface Rpc {
@@ -34,14 +43,14 @@ public interface RaftClientConfigKeys {
     TimeDuration RETRY_INTERVAL_DEFAULT = TimeDuration.valueOf(300, TimeUnit.MILLISECONDS);
     static TimeDuration retryInterval(RaftProperties properties) {
       return getTimeDuration(properties.getTimeDuration(RETRY_INTERVAL_DEFAULT.getUnit()),
-          RETRY_INTERVAL_KEY, RETRY_INTERVAL_DEFAULT);
+          RETRY_INTERVAL_KEY, RETRY_INTERVAL_DEFAULT, getDefaultLog());
     }
 
     String REQUEST_TIMEOUT_KEY = PREFIX + ".request.timeout";
     TimeDuration REQUEST_TIMEOUT_DEFAULT = TimeDuration.valueOf(3000, TimeUnit.MILLISECONDS);
     static TimeDuration requestTimeout(RaftProperties properties) {
       return getTimeDuration(properties.getTimeDuration(REQUEST_TIMEOUT_DEFAULT.getUnit()),
-          REQUEST_TIMEOUT_KEY, REQUEST_TIMEOUT_DEFAULT);
+          REQUEST_TIMEOUT_KEY, REQUEST_TIMEOUT_DEFAULT, getDefaultLog());
     }
     static void setRequestTimeout(RaftProperties properties, TimeDuration timeoutDuration) {
       setTimeDuration(properties::setTimeDuration, REQUEST_TIMEOUT_KEY, timeoutDuration);
@@ -55,7 +64,7 @@ public interface RaftClientConfigKeys {
     int MAX_OUTSTANDING_REQUESTS_DEFAULT = 100;
     static int maxOutstandingRequests(RaftProperties properties) {
       return getInt(properties::getInt, MAX_OUTSTANDING_REQUESTS_KEY,
-          MAX_OUTSTANDING_REQUESTS_DEFAULT, requireMin(2));
+          MAX_OUTSTANDING_REQUESTS_DEFAULT, getDefaultLog(), requireMin(2));
     }
     static void setMaxOutstandingRequests(RaftProperties properties, int outstandingRequests) {
       setInt(properties::setInt, MAX_OUTSTANDING_REQUESTS_KEY, outstandingRequests);
@@ -65,7 +74,7 @@ public interface RaftClientConfigKeys {
     int SCHEDULER_THREADS_DEFAULT = 3;
     static int schedulerThreads(RaftProperties properties) {
       return getInt(properties::getInt, SCHEDULER_THREADS_KEY,
-          SCHEDULER_THREADS_DEFAULT, requireMin(1));
+          SCHEDULER_THREADS_DEFAULT, getDefaultLog(), requireMin(1));
     }
     static void setSchedulerThreads(RaftProperties properties, int schedulerThreads) {
       setInt(properties::setInt, SCHEDULER_THREADS_KEY, schedulerThreads);

http://git-wip-us.apache.org/repos/asf/incubator-ratis/blob/50588bde/ratis-common/src/main/java/org/apache/ratis/RaftConfigKeys.java
----------------------------------------------------------------------
diff --git a/ratis-common/src/main/java/org/apache/ratis/RaftConfigKeys.java b/ratis-common/src/main/java/org/apache/ratis/RaftConfigKeys.java
index 3de95ab..a12de55 100644
--- a/ratis-common/src/main/java/org/apache/ratis/RaftConfigKeys.java
+++ b/ratis-common/src/main/java/org/apache/ratis/RaftConfigKeys.java
@@ -25,6 +25,8 @@ import org.apache.ratis.conf.RaftProperties;
 import org.apache.ratis.rpc.RpcType;
 import org.apache.ratis.rpc.SupportedRpcType;
 
+import java.util.function.Consumer;
+
 public interface RaftConfigKeys {
   String PREFIX = "raft";
 
@@ -34,8 +36,8 @@ public interface RaftConfigKeys {
     String TYPE_KEY = PREFIX + ".type";
     String TYPE_DEFAULT = SupportedRpcType.GRPC.name();
 
-    static RpcType type(RaftProperties properties) {
-      final String t = get(properties::get, TYPE_KEY, TYPE_DEFAULT);
+    static RpcType type(RaftProperties properties, Consumer<String> logger) {
+      final String t = get(properties::get, TYPE_KEY, TYPE_DEFAULT, logger);
       return RpcType.valueOf(t);
     }
 

http://git-wip-us.apache.org/repos/asf/incubator-ratis/blob/50588bde/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 cfef928..6e7ae6d 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
@@ -37,8 +37,11 @@ import java.util.function.Consumer;
 public interface ConfUtils {
   Logger LOG = LoggerFactory.getLogger(ConfUtils.class);
 
-  static <T> void logGet(String key, T value, T defaultValue) {
-    LOG.info("{} = {} ({})", key, value, Objects.equal(value, defaultValue)? "default": "custom");
+  static <T> void logGet(String key, T value, T defaultValue, Consumer<String> logger) {
+    if (logger != null) {
+      logger.accept(String.format("%s = %s (%s)", key, value,
+          Objects.equal(value, defaultValue)? "default": "custom"));
+    }
   }
 
   static void logSet(String key, Object value) {
@@ -104,37 +107,37 @@ public interface ConfUtils {
   @SafeVarargs
   static boolean getBoolean(
       BiFunction<String, Boolean, Boolean> booleanGetter,
-      String key, boolean defaultValue, BiConsumer<String, Boolean>... assertions) {
-    return get(booleanGetter, key, defaultValue, assertions);
+      String key, boolean defaultValue, Consumer<String> logger, BiConsumer<String, Boolean>... assertions) {
+    return get(booleanGetter, key, defaultValue, logger, assertions);
   }
 
   @SafeVarargs
   static int getInt(
       BiFunction<String, Integer, Integer> integerGetter,
-      String key, int defaultValue, BiConsumer<String, Integer>... assertions) {
-    return get(integerGetter, key, defaultValue, assertions);
+      String key, int defaultValue, Consumer<String> logger, BiConsumer<String, Integer>... assertions) {
+    return get(integerGetter, key, defaultValue, logger, assertions);
   }
 
   @SafeVarargs
   static long getLong(
       BiFunction<String, Long, Long> longGetter,
-      String key, long defaultValue, BiConsumer<String, Long>... assertions) {
-    return get(longGetter, key, defaultValue, assertions);
+      String key, long defaultValue, Consumer<String> logger, BiConsumer<String, Long>... assertions) {
+    return get(longGetter, key, defaultValue, logger, assertions);
   }
 
   @SafeVarargs
   static File getFile(
       BiFunction<String, File, File> fileGetter,
-      String key, File defaultValue, BiConsumer<String, File>... assertions) {
-    return get(fileGetter, key, defaultValue, assertions);
+      String key, File defaultValue, Consumer<String> logger, BiConsumer<String, File>... assertions) {
+    return get(fileGetter, key, defaultValue, logger, assertions);
   }
 
 
   @SafeVarargs
   static SizeInBytes getSizeInBytes(
       BiFunction<String, SizeInBytes, SizeInBytes> getter,
-      String key, SizeInBytes defaultValue, BiConsumer<String, SizeInBytes>... assertions) {
-    final SizeInBytes value = get(getter, key, defaultValue, assertions);
+      String key, SizeInBytes defaultValue, Consumer<String> logger, BiConsumer<String, SizeInBytes>... assertions) {
+    final SizeInBytes value = get(getter, key, defaultValue, logger, assertions);
     requireMin(0L).accept(key, value.getSize());
     return value;
   }
@@ -142,25 +145,25 @@ public interface ConfUtils {
   @SafeVarargs
   static TimeDuration getTimeDuration(
       BiFunction<String, TimeDuration, TimeDuration> getter,
-      String key, TimeDuration defaultValue, BiConsumer<String, TimeDuration>... assertions) {
-    final TimeDuration value = get(getter, key, defaultValue, assertions);
+      String key, TimeDuration defaultValue, Consumer<String> logger, BiConsumer<String, TimeDuration>... assertions) {
+    final TimeDuration value = get(getter, key, defaultValue, logger, assertions);
     requireNonNegativeTimeDuration().accept(key, value);
     return value;
   }
 
   @SafeVarargs
   static <T> T get(BiFunction<String, T, T> getter,
-      String key, T defaultValue, BiConsumer<String, T>... assertions) {
+      String key, T defaultValue, Consumer<String> logger, BiConsumer<String, T>... assertions) {
     final T value = getter.apply(key, defaultValue);
-    logGet(key, value, defaultValue);
+    logGet(key, value, defaultValue, logger);
     Arrays.asList(assertions).forEach(a -> a.accept(key, value));
     return value;
   }
 
   static InetSocketAddress getInetSocketAddress(
       BiFunction<String, String, String> stringGetter,
-      String key, String defaultValue) {
-    return NetUtils.createSocketAddr(get(stringGetter, key, defaultValue));
+      String key, String defaultValue, Consumer<String> logger) {
+    return NetUtils.createSocketAddr(get(stringGetter, key, defaultValue, logger));
   }
 
   @SafeVarargs

http://git-wip-us.apache.org/repos/asf/incubator-ratis/blob/50588bde/ratis-examples/src/main/java/org/apache/ratis/examples/filestore/FileStoreStateMachine.java
----------------------------------------------------------------------
diff --git a/ratis-examples/src/main/java/org/apache/ratis/examples/filestore/FileStoreStateMachine.java b/ratis-examples/src/main/java/org/apache/ratis/examples/filestore/FileStoreStateMachine.java
index 05e31f9..e6f56f9 100644
--- a/ratis-examples/src/main/java/org/apache/ratis/examples/filestore/FileStoreStateMachine.java
+++ b/ratis-examples/src/main/java/org/apache/ratis/examples/filestore/FileStoreStateMachine.java
@@ -47,7 +47,7 @@ public class FileStoreStateMachine extends BaseStateMachine {
   private final FileStore files;
 
   public FileStoreStateMachine(RaftProperties properties) {
-    final File dir = ConfUtils.getFile(properties::getFile, FileStoreCommon.STATEMACHINE_DIR_KEY, null);
+    final File dir = ConfUtils.getFile(properties::getFile, FileStoreCommon.STATEMACHINE_DIR_KEY, null, LOG::info);
     Objects.requireNonNull(dir, FileStoreCommon.STATEMACHINE_DIR_KEY + " is not set.");
     this.files = new FileStore(this::getId, dir.toPath());
   }

http://git-wip-us.apache.org/repos/asf/incubator-ratis/blob/50588bde/ratis-grpc/src/main/java/org/apache/ratis/grpc/GrpcConfigKeys.java
----------------------------------------------------------------------
diff --git a/ratis-grpc/src/main/java/org/apache/ratis/grpc/GrpcConfigKeys.java b/ratis-grpc/src/main/java/org/apache/ratis/grpc/GrpcConfigKeys.java
index 9d18f3e..ab9ea8c 100644
--- a/ratis-grpc/src/main/java/org/apache/ratis/grpc/GrpcConfigKeys.java
+++ b/ratis-grpc/src/main/java/org/apache/ratis/grpc/GrpcConfigKeys.java
@@ -21,6 +21,10 @@ import org.apache.ratis.client.RaftClientConfigKeys;
 import org.apache.ratis.conf.RaftProperties;
 import org.apache.ratis.util.SizeInBytes;
 import org.apache.ratis.util.TimeDuration;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.util.function.Consumer;
 
 import static org.apache.ratis.conf.ConfUtils.*;
 
@@ -28,13 +32,18 @@ public interface GrpcConfigKeys {
   String PREFIX = "raft.grpc";
 
   interface Server {
+    Logger LOG = LoggerFactory.getLogger(Server.class);
+    static Consumer<String> getDefaultLog() {
+      return LOG::info;
+    }
+
     String PREFIX = GrpcConfigKeys.PREFIX + ".server";
 
     String PORT_KEY = PREFIX + ".port";
     int PORT_DEFAULT = 0;
     static int port(RaftProperties properties) {
       return getInt(properties::getInt,
-          PORT_KEY, PORT_DEFAULT, requireMin(0), requireMax(65536));
+          PORT_KEY, PORT_DEFAULT, getDefaultLog(), requireMin(0), requireMax(65536));
     }
     static void setPort(RaftProperties properties, int port) {
       setInt(properties::setInt, PORT_KEY, port);
@@ -44,7 +53,7 @@ public interface GrpcConfigKeys {
     int LEADER_OUTSTANDING_APPENDS_MAX_DEFAULT = 128;
     static int leaderOutstandingAppendsMax(RaftProperties properties) {
       return getInt(properties::getInt,
-          LEADER_OUTSTANDING_APPENDS_MAX_KEY, LEADER_OUTSTANDING_APPENDS_MAX_DEFAULT, requireMin(0));
+          LEADER_OUTSTANDING_APPENDS_MAX_KEY, LEADER_OUTSTANDING_APPENDS_MAX_DEFAULT, getDefaultLog(), requireMin(0));
     }
     static void setLeaderOutstandingAppendsMax(RaftProperties properties, int maxAppend) {
       setInt(properties::setInt, LEADER_OUTSTANDING_APPENDS_MAX_KEY, maxAppend);
@@ -52,13 +61,18 @@ public interface GrpcConfigKeys {
   }
 
   interface OutputStream {
+    Logger LOG = LoggerFactory.getLogger(OutputStream.class);
+    static Consumer<String> getDefaultLog() {
+      return LOG::debug;
+    }
+
     String PREFIX = GrpcConfigKeys.PREFIX + ".outputstream";
 
     String BUFFER_SIZE_KEY = PREFIX + ".buffer.size";
     SizeInBytes BUFFER_SIZE_DEFAULT = SizeInBytes.valueOf("64KB");
     static SizeInBytes bufferSize(RaftProperties properties) {
       return getSizeInBytes(properties::getSizeInBytes,
-          BUFFER_SIZE_KEY, BUFFER_SIZE_DEFAULT);
+          BUFFER_SIZE_KEY, BUFFER_SIZE_DEFAULT, getDefaultLog());
     }
     static void setBufferSize(RaftProperties properties, SizeInBytes bufferSize) {
       setSizeInBytes(properties::set, BUFFER_SIZE_KEY, bufferSize);
@@ -68,7 +82,7 @@ public interface GrpcConfigKeys {
     int RETRY_TIMES_DEFAULT = 5;
     static int retryTimes(RaftProperties properties) {
       return getInt(properties::getInt,
-          RETRY_TIMES_KEY, RETRY_TIMES_DEFAULT, requireMin(1));
+          RETRY_TIMES_KEY, RETRY_TIMES_DEFAULT, getDefaultLog(), requireMin(1));
     }
     static void setRetryTimes(RaftProperties properties, int retryTimes) {
       setInt(properties::setInt, RETRY_TIMES_KEY, retryTimes);
@@ -78,7 +92,7 @@ public interface GrpcConfigKeys {
     TimeDuration RETRY_INTERVAL_DEFAULT = RaftClientConfigKeys.Rpc.RETRY_INTERVAL_DEFAULT;
     static TimeDuration retryInterval(RaftProperties properties) {
       return getTimeDuration(properties.getTimeDuration(RETRY_INTERVAL_DEFAULT.getUnit()),
-          RETRY_INTERVAL_KEY, RETRY_INTERVAL_DEFAULT);
+          RETRY_INTERVAL_KEY, RETRY_INTERVAL_DEFAULT, getDefaultLog());
     }
     static void setRetryInterval(RaftProperties properties, TimeDuration retryInterval) {
       setTimeDuration(properties::setTimeDuration, RETRY_INTERVAL_KEY, retryInterval);
@@ -88,7 +102,7 @@ public interface GrpcConfigKeys {
     int OUTSTANDING_APPENDS_MAX_DEFAULT = 128;
     static int outstandingAppendsMax(RaftProperties properties) {
       return getInt(properties::getInt,
-          OUTSTANDING_APPENDS_MAX_KEY, OUTSTANDING_APPENDS_MAX_DEFAULT, requireMin(0));
+          OUTSTANDING_APPENDS_MAX_KEY, OUTSTANDING_APPENDS_MAX_DEFAULT, getDefaultLog(), requireMin(0));
     }
     static void setOutstandingAppendsMax(RaftProperties properties, int maxOutstandingAppends) {
       setInt(properties::setInt, OUTSTANDING_APPENDS_MAX_KEY, maxOutstandingAppends);
@@ -97,9 +111,9 @@ public interface GrpcConfigKeys {
 
   String MESSAGE_SIZE_MAX_KEY = PREFIX + ".message.size.max";
   SizeInBytes MESSAGE_SIZE_MAX_DEFAULT = SizeInBytes.valueOf("64MB");
-  static SizeInBytes messageSizeMax(RaftProperties properties) {
+  static SizeInBytes messageSizeMax(RaftProperties properties, Consumer<String> logger) {
     return getSizeInBytes(properties::getSizeInBytes,
-        MESSAGE_SIZE_MAX_KEY, MESSAGE_SIZE_MAX_DEFAULT);
+        MESSAGE_SIZE_MAX_KEY, MESSAGE_SIZE_MAX_DEFAULT, logger);
   }
   static void setMessageSizeMax(RaftProperties properties, SizeInBytes maxMessageSize) {
     setSizeInBytes(properties::set, MESSAGE_SIZE_MAX_KEY, maxMessageSize);
@@ -107,9 +121,9 @@ public interface GrpcConfigKeys {
 
   String FLOW_CONTROL_WINDOW_KEY = PREFIX + ".flow.control.window";
   SizeInBytes FLOW_CONTROL_WINDOW_DEFAULT = SizeInBytes.valueOf("1MB");
-  static SizeInBytes flowControlWindow(RaftProperties properties) {
+  static SizeInBytes flowControlWindow(RaftProperties properties, Consumer<String> logger) {
     return getSizeInBytes(properties::getSizeInBytes,
-        FLOW_CONTROL_WINDOW_KEY, FLOW_CONTROL_WINDOW_DEFAULT);
+        FLOW_CONTROL_WINDOW_KEY, FLOW_CONTROL_WINDOW_DEFAULT, logger);
   }
   static void setFlowControlWindow(RaftProperties properties, SizeInBytes flowControlWindowSize) {
     setSizeInBytes(properties::set, FLOW_CONTROL_WINDOW_KEY, flowControlWindowSize);

http://git-wip-us.apache.org/repos/asf/incubator-ratis/blob/50588bde/ratis-grpc/src/main/java/org/apache/ratis/grpc/RaftGRpcService.java
----------------------------------------------------------------------
diff --git a/ratis-grpc/src/main/java/org/apache/ratis/grpc/RaftGRpcService.java b/ratis-grpc/src/main/java/org/apache/ratis/grpc/RaftGRpcService.java
index b3e514c..929b564 100644
--- a/ratis-grpc/src/main/java/org/apache/ratis/grpc/RaftGRpcService.java
+++ b/ratis-grpc/src/main/java/org/apache/ratis/grpc/RaftGRpcService.java
@@ -69,9 +69,9 @@ public class RaftGRpcService extends RaftServerRpcWithProxy<RaftServerProtocolCl
   private RaftGRpcService(RaftServer server) {
     this(server, server::getId,
         GrpcConfigKeys.Server.port(server.getProperties()),
-        GrpcConfigKeys.messageSizeMax(server.getProperties()),
+        GrpcConfigKeys.messageSizeMax(server.getProperties(), LOG::info),
         RaftServerConfigKeys.Log.Appender.bufferCapacity(server.getProperties()),
-        GrpcConfigKeys.flowControlWindow(server.getProperties()),
+        GrpcConfigKeys.flowControlWindow(server.getProperties(), LOG::info),
         RaftServerConfigKeys.Rpc.requestTimeout(server.getProperties()));
   }
   private RaftGRpcService(RaftServer raftServer, Supplier<RaftPeerId> idSupplier, int port,

http://git-wip-us.apache.org/repos/asf/incubator-ratis/blob/50588bde/ratis-grpc/src/main/java/org/apache/ratis/grpc/client/AppendStreamer.java
----------------------------------------------------------------------
diff --git a/ratis-grpc/src/main/java/org/apache/ratis/grpc/client/AppendStreamer.java b/ratis-grpc/src/main/java/org/apache/ratis/grpc/client/AppendStreamer.java
index c1228d0..3068751 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
@@ -88,7 +88,7 @@ public class AppendStreamer implements Closeable {
       RaftPeerId leaderId, ClientId clientId) {
     this.clientId = clientId;
     maxPendingNum = GrpcConfigKeys.OutputStream.outstandingAppendsMax(prop);
-    maxMessageSize = GrpcConfigKeys.messageSizeMax(prop);
+    maxMessageSize = GrpcConfigKeys.messageSizeMax(prop, LOG::debug);
     dataQueue = new ConcurrentLinkedDeque<>();
     ackQueue = new ConcurrentLinkedDeque<>();
     exceptionAndRetry = new ExceptionAndRetry(prop);

http://git-wip-us.apache.org/repos/asf/incubator-ratis/blob/50588bde/ratis-grpc/src/main/java/org/apache/ratis/grpc/client/GrpcClientRpc.java
----------------------------------------------------------------------
diff --git a/ratis-grpc/src/main/java/org/apache/ratis/grpc/client/GrpcClientRpc.java b/ratis-grpc/src/main/java/org/apache/ratis/grpc/client/GrpcClientRpc.java
index 160ae16..48ab95d 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
@@ -55,7 +55,7 @@ public class GrpcClientRpc extends RaftClientRpcWithProxy<RaftClientProtocolClie
   public GrpcClientRpc(ClientId clientId, RaftProperties properties) {
     super(new PeerProxyMap<>(clientId.toString(), p -> new RaftClientProtocolClient(clientId, p, properties)));
     this.clientId = clientId;
-    maxMessageSize = GrpcConfigKeys.messageSizeMax(properties).getSizeInt();
+    this.maxMessageSize = GrpcConfigKeys.messageSizeMax(properties, LOG::debug).getSizeInt();
   }
 
   @Override

http://git-wip-us.apache.org/repos/asf/incubator-ratis/blob/50588bde/ratis-grpc/src/main/java/org/apache/ratis/grpc/client/RaftClientProtocolClient.java
----------------------------------------------------------------------
diff --git a/ratis-grpc/src/main/java/org/apache/ratis/grpc/client/RaftClientProtocolClient.java b/ratis-grpc/src/main/java/org/apache/ratis/grpc/client/RaftClientProtocolClient.java
index 11f2676..7d444e6 100644
--- a/ratis-grpc/src/main/java/org/apache/ratis/grpc/client/RaftClientProtocolClient.java
+++ b/ratis-grpc/src/main/java/org/apache/ratis/grpc/client/RaftClientProtocolClient.java
@@ -68,12 +68,12 @@ public class RaftClientProtocolClient implements Closeable {
 
   private final AtomicReference<AsyncStreamObservers> appendStreamObservers = new AtomicReference<>();
 
-  public RaftClientProtocolClient(ClientId id, RaftPeer target,
-      RaftProperties properties) {
+  public RaftClientProtocolClient(ClientId id, RaftPeer target, RaftProperties properties) {
     this.name = JavaUtils.memoize(() -> id + "->" + target.getId());
     this.target = target;
-    SizeInBytes flowControlWindow = GrpcConfigKeys.flowControlWindow(properties);
-    SizeInBytes maxMessageSize = GrpcConfigKeys.messageSizeMax(properties);
+
+    final SizeInBytes flowControlWindow = GrpcConfigKeys.flowControlWindow(properties, LOG::debug);
+    final SizeInBytes maxMessageSize = GrpcConfigKeys.messageSizeMax(properties, LOG::debug);
     channel = NettyChannelBuilder.forTarget(target.getAddress())
         .usePlaintext(true).flowControlWindow(flowControlWindow.getSizeInt())
         .maxMessageSize(maxMessageSize.getSizeInt())

http://git-wip-us.apache.org/repos/asf/incubator-ratis/blob/50588bde/ratis-hadoop/src/main/java/org/apache/ratis/hadooprpc/HadoopConfigKeys.java
----------------------------------------------------------------------
diff --git a/ratis-hadoop/src/main/java/org/apache/ratis/hadooprpc/HadoopConfigKeys.java b/ratis-hadoop/src/main/java/org/apache/ratis/hadooprpc/HadoopConfigKeys.java
index a838416..b399ec4 100644
--- a/ratis-hadoop/src/main/java/org/apache/ratis/hadooprpc/HadoopConfigKeys.java
+++ b/ratis-hadoop/src/main/java/org/apache/ratis/hadooprpc/HadoopConfigKeys.java
@@ -19,8 +19,11 @@ package org.apache.ratis.hadooprpc;
 
 import org.apache.hadoop.conf.Configuration;
 import org.apache.ratis.conf.Parameters;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
 
 import java.net.InetSocketAddress;
+import java.util.function.Consumer;
 
 import static org.apache.ratis.conf.ConfUtils.*;
 
@@ -41,6 +44,11 @@ public interface HadoopConfigKeys {
 
   /** IPC server configurations */
   interface Ipc {
+    Logger LOG = LoggerFactory.getLogger(Ipc.class);
+    static Consumer<String> getDefaultLog() {
+      return LOG::info;
+    }
+
     String PREFIX = HadoopConfigKeys.PREFIX + ".ipc";
 
     String ADDRESS_KEY = PREFIX + ".address";
@@ -51,8 +59,7 @@ public interface HadoopConfigKeys {
     int HANDLERS_DEFAULT = 10;
 
     static int handlers(Configuration conf) {
-      return getInt(conf::getInt,
-          HANDLERS_KEY, HANDLERS_DEFAULT, requireMin(1));
+      return getInt(conf::getInt, HANDLERS_KEY, HANDLERS_DEFAULT, getDefaultLog(), requireMin(1));
     }
 
     static void setHandlers(Configuration conf, int handers) {
@@ -60,8 +67,7 @@ public interface HadoopConfigKeys {
     }
 
     static InetSocketAddress address(Configuration conf) {
-      return getInetSocketAddress(conf::getTrimmed,
-          ADDRESS_KEY, ADDRESS_DEFAULT);
+      return getInetSocketAddress(conf::getTrimmed, ADDRESS_KEY, ADDRESS_DEFAULT, getDefaultLog());
     }
 
     static void setAddress(Configuration conf, String address) {

http://git-wip-us.apache.org/repos/asf/incubator-ratis/blob/50588bde/ratis-netty/src/main/java/org/apache/ratis/netty/NettyConfigKeys.java
----------------------------------------------------------------------
diff --git a/ratis-netty/src/main/java/org/apache/ratis/netty/NettyConfigKeys.java b/ratis-netty/src/main/java/org/apache/ratis/netty/NettyConfigKeys.java
index aa0d9f6..2bfcca6 100644
--- a/ratis-netty/src/main/java/org/apache/ratis/netty/NettyConfigKeys.java
+++ b/ratis-netty/src/main/java/org/apache/ratis/netty/NettyConfigKeys.java
@@ -18,6 +18,10 @@
 package org.apache.ratis.netty;
 
 import org.apache.ratis.conf.RaftProperties;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.util.function.Consumer;
 
 import static org.apache.ratis.conf.ConfUtils.*;
 
@@ -25,6 +29,11 @@ public interface NettyConfigKeys {
   String PREFIX = "raft.netty";
 
   interface Server {
+    Logger LOG = LoggerFactory.getLogger(Server.class);
+    static Consumer<String> getDefaultLog() {
+      return LOG::info;
+    }
+
     String PREFIX = NettyConfigKeys.PREFIX + ".server";
 
     String PORT_KEY = PREFIX + ".port";
@@ -32,7 +41,7 @@ public interface NettyConfigKeys {
 
     static int port(RaftProperties properties) {
       return getInt(properties::getInt,
-          PORT_KEY, PORT_DEFAULT, requireMin(0), requireMax(65536));
+          PORT_KEY, PORT_DEFAULT, getDefaultLog(), requireMin(0), requireMax(65536));
     }
 
     static void setPort(RaftProperties properties, int port) {

http://git-wip-us.apache.org/repos/asf/incubator-ratis/blob/50588bde/ratis-server/src/main/java/org/apache/ratis/server/RaftServerConfigKeys.java
----------------------------------------------------------------------
diff --git a/ratis-server/src/main/java/org/apache/ratis/server/RaftServerConfigKeys.java b/ratis-server/src/main/java/org/apache/ratis/server/RaftServerConfigKeys.java
index 314c506..c878c8f 100644
--- a/ratis-server/src/main/java/org/apache/ratis/server/RaftServerConfigKeys.java
+++ b/ratis-server/src/main/java/org/apache/ratis/server/RaftServerConfigKeys.java
@@ -21,19 +21,27 @@ import org.apache.ratis.conf.ConfUtils;
 import org.apache.ratis.conf.RaftProperties;
 import org.apache.ratis.util.SizeInBytes;
 import org.apache.ratis.util.TimeDuration;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
 
 import java.io.File;
 import java.util.concurrent.TimeUnit;
+import java.util.function.Consumer;
 
 import static org.apache.ratis.conf.ConfUtils.*;
 
 public interface RaftServerConfigKeys {
+  Logger LOG = LoggerFactory.getLogger(RaftServerConfigKeys.class);
+  static Consumer<String> getDefaultLog() {
+    return LOG::info;
+  }
+
   String PREFIX = "raft.server";
 
   String STORAGE_DIR_KEY = PREFIX + ".storage.dir";
   File STORAGE_DIR_DEFAULT = new File("/tmp/raft-server/");
   static File storageDir(RaftProperties properties) {
-    return getFile(properties::getFile, STORAGE_DIR_KEY, STORAGE_DIR_DEFAULT);
+    return getFile(properties::getFile, STORAGE_DIR_KEY, STORAGE_DIR_DEFAULT, getDefaultLog());
   }
   static void setStorageDir(RaftProperties properties, File storageDir) {
     setFile(properties::setFile, STORAGE_DIR_KEY, storageDir);
@@ -48,7 +56,7 @@ public interface RaftServerConfigKeys {
   int STAGING_CATCHUP_GAP_DEFAULT = 1000; // increase this number when write throughput is high
   static int stagingCatchupGap(RaftProperties properties) {
     return getInt(properties::getInt,
-        STAGING_CATCHUP_GAP_KEY, STAGING_CATCHUP_GAP_DEFAULT, requireMin(0));
+        STAGING_CATCHUP_GAP_KEY, STAGING_CATCHUP_GAP_DEFAULT, getDefaultLog(), requireMin(0));
   }
   static void setStagingCatchupGap(RaftProperties properties, int stagingCatchupGap) {
     setInt(properties::setInt, STAGING_CATCHUP_GAP_KEY, stagingCatchupGap);
@@ -62,7 +70,7 @@ public interface RaftServerConfigKeys {
   TimeDuration LEADER_ELECTION_TIMEOUT_DEFAULT = TimeDuration.valueOf(60, TimeUnit.SECONDS);
   static TimeDuration leaderElectionTimeout(RaftProperties properties) {
     return getTimeDuration(properties.getTimeDuration(LEADER_ELECTION_TIMEOUT_DEFAULT.getUnit()),
-        LEADER_ELECTION_TIMEOUT_KEY, LEADER_ELECTION_TIMEOUT_DEFAULT);
+        LEADER_ELECTION_TIMEOUT_KEY, LEADER_ELECTION_TIMEOUT_DEFAULT, getDefaultLog());
   }
   static void setLeaderElectionTimeout(RaftProperties properties, TimeDuration leaderElectionTimeout) {
     setTimeDuration(properties::setTimeDuration, LEADER_ELECTION_TIMEOUT_KEY, leaderElectionTimeout);
@@ -75,7 +83,7 @@ public interface RaftServerConfigKeys {
     String USE_MEMORY_KEY = PREFIX + ".use.memory";
     boolean USE_MEMORY_DEFAULT = false;
     static boolean useMemory(RaftProperties properties) {
-      return getBoolean(properties::getBoolean, USE_MEMORY_KEY, USE_MEMORY_DEFAULT);
+      return getBoolean(properties::getBoolean, USE_MEMORY_KEY, USE_MEMORY_DEFAULT, getDefaultLog());
     }
     static void setUseMemory(RaftProperties properties, boolean useMemory) {
       setBoolean(properties::setBoolean, USE_MEMORY_KEY, useMemory);
@@ -85,7 +93,7 @@ public interface RaftServerConfigKeys {
     SizeInBytes SEGMENT_SIZE_MAX_DEFAULT = SizeInBytes.valueOf("8MB");
     static SizeInBytes segmentSizeMax(RaftProperties properties) {
       return getSizeInBytes(properties::getSizeInBytes,
-          SEGMENT_SIZE_MAX_KEY, SEGMENT_SIZE_MAX_DEFAULT);
+          SEGMENT_SIZE_MAX_KEY, SEGMENT_SIZE_MAX_DEFAULT, getDefaultLog());
     }
     static void setSegmentSizeMax(RaftProperties properties, SizeInBytes segmentSizeMax) {
       setSizeInBytes(properties::set, SEGMENT_SIZE_MAX_KEY, segmentSizeMax);
@@ -98,7 +106,7 @@ public interface RaftServerConfigKeys {
     int SEGMENT_CACHE_MAX_NUM_DEFAULT = 6;
     static int maxCachedSegmentNum(RaftProperties properties) {
       return getInt(properties::getInt, SEGMENT_CACHE_MAX_NUM_KEY,
-          SEGMENT_CACHE_MAX_NUM_DEFAULT, requireMin(0));
+          SEGMENT_CACHE_MAX_NUM_DEFAULT, getDefaultLog(), requireMin(0));
     }
     static void setMaxCachedSegmentNum(RaftProperties properties, int maxCachedSegmentNum) {
       setInt(properties::setInt, SEGMENT_CACHE_MAX_NUM_KEY, maxCachedSegmentNum);
@@ -108,7 +116,7 @@ public interface RaftServerConfigKeys {
     SizeInBytes PREALLOCATED_SIZE_DEFAULT = SizeInBytes.valueOf("4MB");
     static SizeInBytes preallocatedSize(RaftProperties properties) {
       return getSizeInBytes(properties::getSizeInBytes,
-          PREALLOCATED_SIZE_KEY, PREALLOCATED_SIZE_DEFAULT);
+          PREALLOCATED_SIZE_KEY, PREALLOCATED_SIZE_DEFAULT, getDefaultLog());
     }
     static void setPreallocatedSize(RaftProperties properties, SizeInBytes preallocatedSize) {
       setSizeInBytes(properties::set, PREALLOCATED_SIZE_KEY, preallocatedSize);
@@ -118,7 +126,7 @@ public interface RaftServerConfigKeys {
     SizeInBytes WRITE_BUFFER_SIZE_DEFAULT =SizeInBytes.valueOf("64KB");
     static SizeInBytes writeBufferSize(RaftProperties properties) {
       return getSizeInBytes(properties::getSizeInBytes,
-          WRITE_BUFFER_SIZE_KEY, WRITE_BUFFER_SIZE_DEFAULT);
+          WRITE_BUFFER_SIZE_KEY, WRITE_BUFFER_SIZE_DEFAULT, getDefaultLog());
     }
     static void setWriteBufferSize(RaftProperties properties, SizeInBytes writeBufferSize) {
       setSizeInBytes(properties::set, WRITE_BUFFER_SIZE_KEY, writeBufferSize);
@@ -128,7 +136,7 @@ public interface RaftServerConfigKeys {
     int FORCE_SYNC_NUM_DEFAULT = 128;
     static int forceSyncNum(RaftProperties properties) {
       return getInt(properties::getInt,
-          FORCE_SYNC_NUM_KEY, FORCE_SYNC_NUM_DEFAULT, requireMin(0));
+          FORCE_SYNC_NUM_KEY, FORCE_SYNC_NUM_DEFAULT, getDefaultLog(), requireMin(0));
     }
     static void setForceSyncNum(RaftProperties properties, int forceSyncNum) {
       setInt(properties::setInt, FORCE_SYNC_NUM_KEY, forceSyncNum);
@@ -141,7 +149,7 @@ public interface RaftServerConfigKeys {
       SizeInBytes BUFFER_CAPACITY_DEFAULT =SizeInBytes.valueOf("4MB");
       static SizeInBytes bufferCapacity(RaftProperties properties) {
         return getSizeInBytes(properties::getSizeInBytes,
-            BUFFER_CAPACITY_KEY, BUFFER_CAPACITY_DEFAULT);
+            BUFFER_CAPACITY_KEY, BUFFER_CAPACITY_DEFAULT, getDefaultLog());
       }
       static void setBufferCapacity(RaftProperties properties, SizeInBytes bufferCapacity) {
         setSizeInBytes(properties::set, BUFFER_CAPACITY_KEY, bufferCapacity);
@@ -150,7 +158,7 @@ public interface RaftServerConfigKeys {
       String BATCH_ENABLED_KEY = PREFIX + ".batch.enabled";
       boolean BATCH_ENABLED_DEFAULT = false;
       static boolean batchEnabled(RaftProperties properties) {
-        return getBoolean(properties::getBoolean, BATCH_ENABLED_KEY, BATCH_ENABLED_DEFAULT);
+        return getBoolean(properties::getBoolean, BATCH_ENABLED_KEY, BATCH_ENABLED_DEFAULT, getDefaultLog());
       }
       static void setBatchEnabled(RaftProperties properties, boolean batchEnabled) {
         setBoolean(properties::setBoolean, BATCH_ENABLED_KEY, batchEnabled);
@@ -160,7 +168,7 @@ public interface RaftServerConfigKeys {
       SizeInBytes SNAPSHOT_CHUNK_SIZE_MAX_DEFAULT =SizeInBytes.valueOf("16MB");
       static SizeInBytes snapshotChunkSizeMax(RaftProperties properties) {
         return getSizeInBytes(properties::getSizeInBytes,
-            SNAPSHOT_CHUNK_SIZE_MAX_KEY, SNAPSHOT_CHUNK_SIZE_MAX_DEFAULT);
+            SNAPSHOT_CHUNK_SIZE_MAX_KEY, SNAPSHOT_CHUNK_SIZE_MAX_DEFAULT, getDefaultLog());
       }
       static void setSnapshotChunkSizeMax(RaftProperties properties, SizeInBytes maxChunkSize) {
         setSizeInBytes(properties::set, SNAPSHOT_CHUNK_SIZE_MAX_KEY, maxChunkSize);
@@ -177,7 +185,7 @@ public interface RaftServerConfigKeys {
     boolean AUTO_TRIGGER_ENABLED_DEFAULT = false;
     static boolean autoTriggerEnabled(RaftProperties properties) {
       return getBoolean(properties::getBoolean,
-          AUTO_TRIGGER_ENABLED_KEY, AUTO_TRIGGER_ENABLED_DEFAULT);
+          AUTO_TRIGGER_ENABLED_KEY, AUTO_TRIGGER_ENABLED_DEFAULT, getDefaultLog());
     }
     static void setAutoTriggerEnabled(RaftProperties properties, boolean autoTriggerThreshold) {
       setBoolean(properties::setBoolean, AUTO_TRIGGER_ENABLED_KEY, autoTriggerThreshold);
@@ -188,7 +196,7 @@ public interface RaftServerConfigKeys {
     long AUTO_TRIGGER_THRESHOLD_DEFAULT = 400000L;
     static long autoTriggerThreshold(RaftProperties properties) {
       return getLong(properties::getLong,
-          AUTO_TRIGGER_THRESHOLD_KEY, AUTO_TRIGGER_THRESHOLD_DEFAULT, requireMin(0L));
+          AUTO_TRIGGER_THRESHOLD_KEY, AUTO_TRIGGER_THRESHOLD_DEFAULT, getDefaultLog(), requireMin(0L));
     }
     static void setAutoTriggerThreshold(RaftProperties properties, long autoTriggerThreshold) {
       setLong(properties::setLong, AUTO_TRIGGER_THRESHOLD_KEY, autoTriggerThreshold);
@@ -203,7 +211,7 @@ public interface RaftServerConfigKeys {
     TimeDuration TIMEOUT_MIN_DEFAULT = TimeDuration.valueOf(150, TimeUnit.MILLISECONDS);
     static TimeDuration timeoutMin(RaftProperties properties) {
       return getTimeDuration(properties.getTimeDuration(TIMEOUT_MIN_DEFAULT.getUnit()),
-          TIMEOUT_MIN_KEY, TIMEOUT_MIN_DEFAULT);
+          TIMEOUT_MIN_KEY, TIMEOUT_MIN_DEFAULT, getDefaultLog());
     }
     static void setTimeoutMin(RaftProperties properties, TimeDuration minDuration) {
       setTimeDuration(properties::setTimeDuration, TIMEOUT_MIN_KEY, minDuration);
@@ -213,7 +221,7 @@ public interface RaftServerConfigKeys {
     TimeDuration TIMEOUT_MAX_DEFAULT = TimeDuration.valueOf(300, TimeUnit.MILLISECONDS);
     static TimeDuration timeoutMax(RaftProperties properties) {
       return getTimeDuration(properties.getTimeDuration(TIMEOUT_MAX_DEFAULT.getUnit()),
-          TIMEOUT_MAX_KEY, TIMEOUT_MAX_DEFAULT);
+          TIMEOUT_MAX_KEY, TIMEOUT_MAX_DEFAULT, getDefaultLog());
     }
     static void setTimeoutMax(RaftProperties properties, TimeDuration maxDuration) {
       setTimeDuration(properties::setTimeDuration, TIMEOUT_MAX_KEY, maxDuration);
@@ -223,7 +231,7 @@ public interface RaftServerConfigKeys {
     TimeDuration REQUEST_TIMEOUT_DEFAULT = TimeDuration.valueOf(3000, TimeUnit.MILLISECONDS);
     static TimeDuration requestTimeout(RaftProperties properties) {
       return getTimeDuration(properties.getTimeDuration(REQUEST_TIMEOUT_DEFAULT.getUnit()),
-          REQUEST_TIMEOUT_KEY, REQUEST_TIMEOUT_DEFAULT);
+          REQUEST_TIMEOUT_KEY, REQUEST_TIMEOUT_DEFAULT, getDefaultLog());
     }
     static void setRequestTimeout(RaftProperties properties, TimeDuration timeoutDuration) {
       setTimeDuration(properties::setTimeDuration, REQUEST_TIMEOUT_KEY, timeoutDuration);
@@ -233,7 +241,7 @@ public interface RaftServerConfigKeys {
     TimeDuration SLEEP_TIME_DEFAULT = TimeDuration.valueOf(25, TimeUnit.MILLISECONDS);
     static TimeDuration sleepTime(RaftProperties properties) {
       return getTimeDuration(properties.getTimeDuration(SLEEP_TIME_DEFAULT.getUnit()),
-          SLEEP_TIME_KEY, SLEEP_TIME_DEFAULT);
+          SLEEP_TIME_KEY, SLEEP_TIME_DEFAULT, getDefaultLog());
     }
     static void setSleepTime(RaftProperties properties, TimeDuration sleepTime) {
       setTimeDuration(properties::setTimeDuration, SLEEP_TIME_KEY, sleepTime);
@@ -243,7 +251,7 @@ public interface RaftServerConfigKeys {
     TimeDuration SLOWNESS_TIMEOUT_DEFAULT = TimeDuration.valueOf(60, TimeUnit.SECONDS);
     static TimeDuration slownessTimeout(RaftProperties properties) {
       return getTimeDuration(properties.getTimeDuration(SLOWNESS_TIMEOUT_DEFAULT.getUnit()),
-          SLOWNESS_TIMEOUT_KEY, SLOWNESS_TIMEOUT_DEFAULT);
+          SLOWNESS_TIMEOUT_KEY, SLOWNESS_TIMEOUT_DEFAULT, getDefaultLog());
     }
     static void setSlownessTimeout(RaftProperties properties, TimeDuration expiryTime) {
       setTimeDuration(properties::setTimeDuration, SLOWNESS_TIMEOUT_KEY, expiryTime);
@@ -257,7 +265,7 @@ public interface RaftServerConfigKeys {
     String CAPACITY_KEY = PREFIX + ".capacity";
     int CAPACITY_DEFAULT = 4096;
     static int capacity(RaftProperties properties) {
-      return ConfUtils.getInt(properties::getInt, CAPACITY_KEY, CAPACITY_DEFAULT,
+      return ConfUtils.getInt(properties::getInt, CAPACITY_KEY, CAPACITY_DEFAULT, getDefaultLog(),
           ConfUtils.requireMin(0));
     }
 
@@ -269,7 +277,7 @@ public interface RaftServerConfigKeys {
     TimeDuration EXPIRY_TIME_DEFAULT = TimeDuration.valueOf(60, TimeUnit.SECONDS);
     static TimeDuration expiryTime(RaftProperties properties) {
       return getTimeDuration(properties.getTimeDuration(EXPIRY_TIME_DEFAULT.getUnit()),
-          EXPIRY_TIME_KEY, EXPIRY_TIME_DEFAULT);
+          EXPIRY_TIME_KEY, EXPIRY_TIME_DEFAULT, getDefaultLog());
     }
     static void setExpiryTime(RaftProperties properties, TimeDuration expiryTime) {
       setTimeDuration(properties::setTimeDuration, EXPIRY_TIME_KEY, expiryTime);

http://git-wip-us.apache.org/repos/asf/incubator-ratis/blob/50588bde/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 acec50a..aea2767 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
@@ -154,7 +154,7 @@ public class RaftServerProxy implements RaftServer {
     this.properties = properties;
     this.stateMachineRegistry = stateMachineRegistry;
 
-    final RpcType rpcType = RaftConfigKeys.Rpc.type(properties);
+    final RpcType rpcType = RaftConfigKeys.Rpc.type(properties, LOG::info);
     this.factory = ServerFactory.cast(rpcType.newFactory(parameters));
 
     this.serverRpc = factory.newRaftServerRpc(this);

http://git-wip-us.apache.org/repos/asf/incubator-ratis/blob/50588bde/ratis-server/src/test/java/org/apache/ratis/server/simulation/MiniRaftClusterWithSimulatedRpc.java
----------------------------------------------------------------------
diff --git a/ratis-server/src/test/java/org/apache/ratis/server/simulation/MiniRaftClusterWithSimulatedRpc.java b/ratis-server/src/test/java/org/apache/ratis/server/simulation/MiniRaftClusterWithSimulatedRpc.java
index c2851d1..2287174 100644
--- a/ratis-server/src/test/java/org/apache/ratis/server/simulation/MiniRaftClusterWithSimulatedRpc.java
+++ b/ratis-server/src/test/java/org/apache/ratis/server/simulation/MiniRaftClusterWithSimulatedRpc.java
@@ -49,7 +49,7 @@ public class MiniRaftClusterWithSimulatedRpc extends MiniRaftCluster {
       }
       final int simulateLatencyMs = ConfUtils.getInt(prop::getInt,
           SimulatedRequestReply.SIMULATE_LATENCY_KEY,
-          SimulatedRequestReply.SIMULATE_LATENCY_DEFAULT, requireMin(0));
+          SimulatedRequestReply.SIMULATE_LATENCY_DEFAULT, LOG::info, requireMin(0));
       final SimulatedRequestReply<RaftServerRequest, RaftServerReply> serverRequestReply
           = new SimulatedRequestReply<>(simulateLatencyMs);
       final SimulatedClientRpc client2serverRequestReply