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 <[email protected]> Authored: Tue Sep 11 14:00:10 2018 -0700 Committer: Tsz Wo Nicholas Sze <[email protected]> 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
