From c0524939c768ea61cf3a62f9c7c8c1b954e4b679 Mon Sep 17 00:00:00 2001 From: Cyrus Date: Sun, 26 Jul 2020 12:24:39 +0530 Subject: [PATCH 1/5] Changes Log message formatter --- .../ratis/client/impl/OrderedAsync.java | 2 +- .../ratis/client/impl/UnorderedAsync.java | 4 ++-- .../org/apache/ratis/conf/RaftProperties.java | 15 ++++++------ .../ratis/util/AtomicFileOutputStream.java | 6 ++--- .../java/org/apache/ratis/util/IOUtils.java | 2 +- .../java/org/apache/ratis/util/JavaUtils.java | 4 ++-- .../org/apache/ratis/util/JmxRegister.java | 6 ++--- .../java/org/apache/ratis/util/LogUtils.java | 14 +++++------ .../org/apache/ratis/util/MD5FileUtil.java | 4 ++-- .../arithmetic/ArithmeticStateMachine.java | 4 ++-- .../counter/client/CounterClient.java | 2 +- .../counter/server/CounterStateMachine.java | 6 ++--- .../filestore/FileStoreStateMachine.java | 3 ++- .../client/GrpcClientProtocolService.java | 9 +++---- .../ratis/grpc/client/GrpcClientRpc.java | 4 ++-- .../ratis/grpc/client/GrpcClientStreamer.java | 2 +- .../ratis/grpc/server/GrpcLogAppender.java | 6 ++--- ...dClientProtocolClientSideTranslatorPB.java | 2 +- .../hadooprpc/server/HadoopRpcService.java | 4 ++-- .../logservice/server/LogStateMachine.java | 10 ++++---- .../logservice/tool/VerificationTool.java | 20 +++++++++------- .../ratis/metrics/MetricRegistriesLoader.java | 6 ++--- .../ratis/server/impl/FollowerState.java | 4 ++-- .../apache/ratis/server/impl/LeaderState.java | 2 +- .../apache/ratis/server/impl/LogAppender.java | 8 +++---- .../ratis/server/impl/RaftServerImpl.java | 2 +- .../ratis/server/impl/RaftServerProxy.java | 16 +++++++------ .../server/impl/StateMachineUpdater.java | 5 ++-- .../apache/ratis/server/raftlog/RaftLog.java | 4 ++-- .../SegmentedRaftLogInputStream.java | 10 ++++---- .../segmented/SegmentedRaftLogReader.java | 4 ++-- .../segmented/SegmentedRaftLogWorker.java | 18 +++++++------- .../ratis/server/storage/RaftStorage.java | 8 +++---- .../server/storage/RaftStorageDirectory.java | 24 +++++++++---------- 34 files changed, 123 insertions(+), 117 deletions(-) diff --git a/ratis-client/src/main/java/org/apache/ratis/client/impl/OrderedAsync.java b/ratis-client/src/main/java/org/apache/ratis/client/impl/OrderedAsync.java index 3b6c396398..061a9bcfe1 100644 --- a/ratis-client/src/main/java/org/apache/ratis/client/impl/OrderedAsync.java +++ b/ratis-client/src/main/java/org/apache/ratis/client/impl/OrderedAsync.java @@ -238,7 +238,7 @@ private CompletableFuture sendRequest(PendingOrderedRequest pen return reply; }).exceptionally(e -> { if (LOG.isTraceEnabled()) { - LOG.trace(client.getId() + ": Failed* " + request, e); + LOG.trace("{} : Failed* {}", client.getId(), request, e); } else { LOG.debug("{}: Failed* {} with {}", client.getId(), request, e); } diff --git a/ratis-client/src/main/java/org/apache/ratis/client/impl/UnorderedAsync.java b/ratis-client/src/main/java/org/apache/ratis/client/impl/UnorderedAsync.java index eeb6b96e8d..04033e6186 100644 --- a/ratis-client/src/main/java/org/apache/ratis/client/impl/UnorderedAsync.java +++ b/ratis-client/src/main/java/org/apache/ratis/client/impl/UnorderedAsync.java @@ -96,7 +96,7 @@ static void sendRequestWithRetry(PendingClientRequest pending, RaftClientImpl cl if (e != null) { if (LOG.isTraceEnabled()) { - LOG.trace(clientId + ": attempt #" + attemptCount + " failed~ " + request, e); + LOG.trace("{}: attempt #{} failed~ {}", clientId, attemptCount, request, e); } else { LOG.debug("{}: attempt #{} failed {} with {}", clientId, attemptCount, request, e); } @@ -123,7 +123,7 @@ static void sendRequestWithRetry(PendingClientRequest pending, RaftClientImpl cl client.getScheduler().onTimeout(sleepTime, () -> sendRequestWithRetry(pending, client), LOG, () -> clientId + ": Failed~ to retry " + request); } catch (Throwable t) { - LOG.error(clientId + ": Failed " + request, t); + LOG.error("{}: Failed {}",clientId, request, t); f.completeExceptionally(t); } }); diff --git a/ratis-common/src/main/java/org/apache/ratis/conf/RaftProperties.java b/ratis-common/src/main/java/org/apache/ratis/conf/RaftProperties.java index 928ee92687..1bdf73f1dc 100644 --- a/ratis-common/src/main/java/org/apache/ratis/conf/RaftProperties.java +++ b/ratis-common/src/main/java/org/apache/ratis/conf/RaftProperties.java @@ -911,8 +911,8 @@ public Pattern getPattern(String name, Pattern defaultValue) { try { return Pattern.compile(valString); } catch (PatternSyntaxException pse) { - LOG.warn("Regular expression '" + valString + "' for property '" + - name + "' not valid. Using default", pse); + LOG.warn("Regular expression {} for property {} not valid. Using default", + valString, name, pse); return defaultValue; } } @@ -1088,7 +1088,7 @@ public void clear() { private Document parse(DocumentBuilder builder, URL url) throws IOException, SAXException { - LOG.debug("parsing URL " + url); + LOG.debug("parsing URL {}", url); if (url == null) { return null; } @@ -1104,7 +1104,7 @@ private Document parse(DocumentBuilder builder, URL url) private Document parse(DocumentBuilder builder, InputStream is, String systemId) throws IOException, SAXException { - LOG.debug("parsing input stream " + is); + LOG.debug("parsing input stream {}", is); if (is == null) { return null; } @@ -1146,7 +1146,7 @@ private Resource loadResource(Properties propts, Resource wrapper) { try { docBuilderFactory.setXIncludeAware(true); } catch (UnsupportedOperationException e) { - LOG.error("Failed to set setXIncludeAware(true) for parser " + docBuilderFactory + ":" + e, e); + LOG.error("Failed to set setXIncludeAware(true) for parser {}:{}", docBuilderFactory, e, e); } DocumentBuilder builder = docBuilderFactory.newDocumentBuilder(); Document doc = null; @@ -1257,7 +1257,7 @@ private Resource loadResource(Properties propts, Resource wrapper) { return null; } catch (IOException | DOMException | SAXException | ParserConfigurationException e) { - LOG.error("error parsing conf " + name, e); + LOG.error("error parsing conf {}", name, e); throw new RuntimeException(e); } } @@ -1277,8 +1277,7 @@ private void loadProperty(Properties prop, String name, String attr, updatingResource.put(attr, source); } } else if (!value.equals(prop.getProperty(attr))) { - LOG.warn(name+":an attempt to override final parameter: "+attr - +"; Ignoring."); + LOG.warn("{}:an attempt to override final parameter: {}; Ignoring.", name, attr); } } if (finalParameter && attr != null) { diff --git a/ratis-common/src/main/java/org/apache/ratis/util/AtomicFileOutputStream.java b/ratis-common/src/main/java/org/apache/ratis/util/AtomicFileOutputStream.java index c2d626f12a..88edcc448c 100644 --- a/ratis-common/src/main/java/org/apache/ratis/util/AtomicFileOutputStream.java +++ b/ratis-common/src/main/java/org/apache/ratis/util/AtomicFileOutputStream.java @@ -84,7 +84,7 @@ public void close() throws IOException { } // close wasn't successful, try to delete the tmp file if (!tmpFile.delete()) { - LOG.warn("Unable to delete tmp file " + tmpFile); + LOG.warn("Unable to delete tmp file {}", tmpFile); } } } @@ -99,10 +99,10 @@ public void abort() { try { super.close(); } catch (IOException ioe) { - LOG.warn("Unable to abort file " + tmpFile, ioe); + LOG.warn("Unable to abort file {}", tmpFile, ioe); } if (!tmpFile.delete()) { - LOG.warn("Unable to delete tmp file during abort " + tmpFile); + LOG.warn("Unable to delete tmp file during abort {}", tmpFile); } } diff --git a/ratis-common/src/main/java/org/apache/ratis/util/IOUtils.java b/ratis-common/src/main/java/org/apache/ratis/util/IOUtils.java index b075b3f8ec..9693172e5a 100644 --- a/ratis-common/src/main/java/org/apache/ratis/util/IOUtils.java +++ b/ratis-common/src/main/java/org/apache/ratis/util/IOUtils.java @@ -196,7 +196,7 @@ static void cleanup(Logger log, Closeable... closeables) { c.close(); } catch(Throwable e) { if (log != null && log.isDebugEnabled()) { - log.debug("Exception in closing " + c, e); + log.debug("Exception in closing {}", c, e); } } } 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 22215afb7f..b2bd62ee07 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 @@ -163,8 +163,8 @@ static RETURN attempt( throw t; } if (log != null && log.isWarnEnabled()) { - log.warn("FAILED \"" + name.get() + "\", attempt #" + i + "/" + numAttempts - + ": " + t + ", sleep " + sleepTime + " and then retry.", t); + log.warn("FAILED \"{}\", attempt #{}/{}: {}, sleep {} and then " + + "retry.", name.get(), i, numAttempts, t, sleepTime, t); } } diff --git a/ratis-common/src/main/java/org/apache/ratis/util/JmxRegister.java b/ratis-common/src/main/java/org/apache/ratis/util/JmxRegister.java index 54f7989245..48106e7c63 100644 --- a/ratis-common/src/main/java/org/apache/ratis/util/JmxRegister.java +++ b/ratis-common/src/main/java/org/apache/ratis/util/JmxRegister.java @@ -37,11 +37,11 @@ static ObjectName tryRegister(String name, Object mBean) { objectName = new ObjectName(name); ManagementFactory.getPlatformMBeanServer().registerMBean(mBean, objectName); } catch (Exception e) { - LOG.error("Failed to register JMX Bean with name " + name, e); + LOG.error("Failed to register JMX Bean with name {}", name, e); return null; } - LOG.info("Successfully registered JMX Bean with object name " + objectName); + LOG.info("Successfully registered JMX Bean with object name {}", objectName); return objectName; } @@ -70,7 +70,7 @@ public synchronized boolean unregister() throws JMException { return false; } ManagementFactory.getPlatformMBeanServer().unregisterMBean(registeredName); - LOG.info("Successfully un-registered JMX Bean with object name " + registeredName); + LOG.info("Successfully un-registered JMX Bean with object name {}", registeredName); registeredName = null; return true; } 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 eb05ae1cf2..f6da98fcdb 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 @@ -38,15 +38,15 @@ static void runAndLog( op.run(); } catch (Throwable t) { if (log.isTraceEnabled()) { - log.trace("Failed to " + opName.get(), t); + log.trace("Failed to {}", opName.get(), t); } else if (log.isWarnEnabled()){ - log.warn("Failed to " + opName.get() + ": " + t); + log.warn("Failed to {}: {}", opName.get(), t); } throw t; } if (log.isTraceEnabled()) { - log.trace("Successfully ran " + opName.get()); + log.trace("Successfully ran {}", opName.get()); } } @@ -58,16 +58,16 @@ static OUTPUT supplyAndLog( output = supplier.get(); } catch (Throwable t) { if (log.isTraceEnabled()) { - log.trace("Failed to " + name.get(), t); + log.trace("Failed to {}", name.get(), t); } else if (log.isWarnEnabled()){ - log.warn("Failed to " + name.get() + ": " + t); + log.warn("Failed to {}: {}", name.get(), t); } final THROWABLE throwable = JavaUtils.cast(t); throw throwable; } if (log.isTraceEnabled()) { - log.trace("Successfully supplied " + name.get() + ": " + output); + log.trace("Successfully supplied {}: {}", name.get(), output); } return output; } @@ -119,7 +119,7 @@ static void warn(Logger log, Supplier message, Throwable t, Class... if (log.isWarnEnabled()) { if (ReflectionUtils.isInstance(t, exceptionClasses)) { // do not print stack trace for known exceptions. - log.warn(message.get() + ": " + t); + log.warn("{}: {}", message.get(), t); } else { log.warn(message.get(), t); } diff --git a/ratis-common/src/main/java/org/apache/ratis/util/MD5FileUtil.java b/ratis-common/src/main/java/org/apache/ratis/util/MD5FileUtil.java index 3bd4df0881..3d2a1d3621 100644 --- a/ratis-common/src/main/java/org/apache/ratis/util/MD5FileUtil.java +++ b/ratis-common/src/main/java/org/apache/ratis/util/MD5FileUtil.java @@ -155,7 +155,7 @@ private static void saveMD5File(File dataFile, String digestString) afos.close(); if (LOG.isDebugEnabled()) { - LOG.debug("Saved MD5 " + digestString + " to " + md5File); + LOG.debug("Saved MD5 {} to {}", digestString, md5File); } } @@ -170,7 +170,7 @@ public static void renameMD5File(File oldDataFile, File newDataFile) saveMD5File(newDataFile, digestString); if (!fromFile.delete()) { - LOG.warn("deleting " + fromFile.getAbsolutePath() + " FAILED"); + LOG.warn("deleting {} FAILED", fromFile.getAbsolutePath()); } } diff --git a/ratis-examples/src/main/java/org/apache/ratis/examples/arithmetic/ArithmeticStateMachine.java b/ratis-examples/src/main/java/org/apache/ratis/examples/arithmetic/ArithmeticStateMachine.java index b7f5925586..b566d0d6b4 100644 --- a/ratis-examples/src/main/java/org/apache/ratis/examples/arithmetic/ArithmeticStateMachine.java +++ b/ratis-examples/src/main/java/org/apache/ratis/examples/arithmetic/ArithmeticStateMachine.java @@ -98,8 +98,8 @@ public long takeSnapshot() { new BufferedOutputStream(new FileOutputStream(snapshotFile)))) { out.writeObject(copy); } catch(IOException ioe) { - LOG.warn("Failed to write snapshot file \"" + snapshotFile - + "\", last applied index=" + last); + LOG.warn("Failed to write snapshot file \"{}\", last applied index={}", + snapshotFile, last); } return last.getIndex(); diff --git a/ratis-examples/src/main/java/org/apache/ratis/examples/counter/client/CounterClient.java b/ratis-examples/src/main/java/org/apache/ratis/examples/counter/client/CounterClient.java index 9607178c96..88cf9f9adf 100644 --- a/ratis-examples/src/main/java/org/apache/ratis/examples/counter/client/CounterClient.java +++ b/ratis-examples/src/main/java/org/apache/ratis/examples/counter/client/CounterClient.java @@ -59,7 +59,7 @@ public static void main(String[] args) ExecutorService executorService = Executors.newFixedThreadPool(10); //send INCREMENT commands concurrently - System.out.printf("Sending %d increment command...\n", increment); + System.out.printf("Sending %d increment command...%n", increment); for (int i = 0; i < increment; i++) { executorService.submit(() -> raftClient.send(Message.valueOf("INCREMENT"))); diff --git a/ratis-examples/src/main/java/org/apache/ratis/examples/counter/server/CounterStateMachine.java b/ratis-examples/src/main/java/org/apache/ratis/examples/counter/server/CounterStateMachine.java index 7159ec1c51..1ef88793e0 100644 --- a/ratis-examples/src/main/java/org/apache/ratis/examples/counter/server/CounterStateMachine.java +++ b/ratis-examples/src/main/java/org/apache/ratis/examples/counter/server/CounterStateMachine.java @@ -105,8 +105,8 @@ public long takeSnapshot() { new BufferedOutputStream(new FileOutputStream(snapshotFile)))) { out.writeObject(counter); } catch (IOException ioe) { - LOG.warn("Failed to write snapshot file \"" + snapshotFile - + "\", last applied index=" + last); + LOG.warn("Failed to write snapshot file \"{}\", last applied index={}", + snapshotFile, last); } //return the index of the stored snapshot (which is the last applied one) @@ -202,7 +202,7 @@ public CompletableFuture applyTransaction(TransactionContext trx) { //if leader, log the incremented value and it's log index if (trx.getServerRole() == RaftProtos.RaftPeerRole.LEADER) { - LOG.info("{}: Increment to {}", index, counter.toString()); + LOG.info("{}: Increment to {}", index, counter); } return f; 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 383f2cb165..7b38025837 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 @@ -178,7 +178,8 @@ public CompletableFuture applyTransaction(TransactionContext trx) { // WRITE should not happen here since // startTransaction converts WRITE requests to WRITEHEADER requests. default: - LOG.error(getId() + ": Unexpected request case " + request.getRequestCase()); + LOG.error("{}: Unexpected request case {}", getId(), + request.getRequestCase()); return FileStoreCommon.completeExceptionally(index, "Unexpected request case " + request.getRequestCase()); } diff --git a/ratis-grpc/src/main/java/org/apache/ratis/grpc/client/GrpcClientProtocolService.java b/ratis-grpc/src/main/java/org/apache/ratis/grpc/client/GrpcClientProtocolService.java index d878c8dbe0..6852cbd281 100644 --- a/ratis-grpc/src/main/java/org/apache/ratis/grpc/client/GrpcClientProtocolService.java +++ b/ratis-grpc/src/main/java/org/apache/ratis/grpc/client/GrpcClientProtocolService.java @@ -194,7 +194,8 @@ synchronized void responseCompleted() { } catch(Exception e) { // response stream may possibly be already closed/failed so that the exception can be safely ignored. if (LOG.isTraceEnabled()) { - LOG.trace(getName() + ": Failed onCompleted, exception is ignored", e); + LOG.trace("{}: Failed onCompleted, exception is ignored", + getName(), e); } } } @@ -205,7 +206,7 @@ synchronized void responseError(Throwable t) { } catch(Exception e) { // response stream may possibly be already closed/failed so that the exception can be safely ignored. if (LOG.isTraceEnabled()) { - LOG.trace(getName() + ": Failed onError, exception is ignored", e); + LOG.trace("{}: Failed onError, exception is ignored", getName(), e); } } } @@ -259,7 +260,7 @@ boolean responseError(Throwable t, Supplier message) { if (setClose()) { t = JavaUtils.unwrapCompletionException(t); if (LOG.isDebugEnabled()) { - LOG.debug(name + ": Failed " + message.get(), t); + LOG.debug("{}: Failed {}", name, message.get(), t); } responseError(GrpcUtil.wrapException(t)); return true; @@ -280,7 +281,7 @@ private class UnorderedRequestStreamObserver extends RequestStreamObserver { void processClientRequest(RaftClientRequest request) { final CompletableFuture f = processClientRequest(request, reply -> { if (!reply.isSuccess()) { - LOG.info("Failed " + request + ", reply=" + reply); + LOG.info("Failed {}, reply={}", request, reply); } final RaftClientReplyProto proto = ClientProtoUtils.toRaftClientReplyProto(reply); responseNext(proto); 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 0450fa0e38..331129357f 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 @@ -79,7 +79,7 @@ public CompletableFuture sendRequestAsyncUnordered(RaftClientRe // Reuse the same grpc stream for all async calls. return proxy.getUnorderedAsyncStreamObservers().onNext(request); } catch (Throwable t) { - LOG.error(clientId + ": XXX Failed " + request, t); + LOG.error("{}: XXX Failed {}", clientId, request, t); return JavaUtils.completeExceptionally(t); } } @@ -115,7 +115,7 @@ public RaftClientReply sendRequest(RaftClientRequest request) "Interrupted while waiting for response of request " + request); } catch (ExecutionException e) { if (LOG.isTraceEnabled()) { - LOG.trace(clientId + ": failed " + request, e); + LOG.trace("{}: failed {}", clientId, request, e); } throw IOUtils.toIOException(e); } diff --git a/ratis-grpc/src/main/java/org/apache/ratis/grpc/client/GrpcClientStreamer.java b/ratis-grpc/src/main/java/org/apache/ratis/grpc/client/GrpcClientStreamer.java index a93a62209b..57c1e95373 100644 --- a/ratis-grpc/src/main/java/org/apache/ratis/grpc/client/GrpcClientStreamer.java +++ b/ratis-grpc/src/main/java/org/apache/ratis/grpc/client/GrpcClientStreamer.java @@ -292,7 +292,7 @@ public void onNext(RaftClientReplyProto reply) { @Override public void onError(Throwable t) { - LOG.warn(this + " onError", t); + LOG.warn("{} onError", this, t); if (active) { synchronized (GrpcClientStreamer.this) { handleError(t, this); diff --git a/ratis-grpc/src/main/java/org/apache/ratis/grpc/server/GrpcLogAppender.java b/ratis-grpc/src/main/java/org/apache/ratis/grpc/server/GrpcLogAppender.java index 4eb5cb9d59..4204d8ae14 100644 --- a/ratis-grpc/src/main/java/org/apache/ratis/grpc/server/GrpcLogAppender.java +++ b/ratis-grpc/src/main/java/org/apache/ratis/grpc/server/GrpcLogAppender.java @@ -150,7 +150,7 @@ private void mayWait() { LOG.trace("{}: wait {}ms", this, waitTimeMs); wait(waitTimeMs); } catch(InterruptedException ie) { - LOG.warn(this + ": Wait interrupted by " + ie); + LOG.warn("{}: Wait interrupted by {}", this, ie); } } } @@ -262,8 +262,8 @@ public void onNext(AppendEntriesReplyProto reply) { try { onNextImpl(reply); } catch(Throwable t) { - LOG.error("Failed onNext request=" + request - + ", reply=" + ServerProtoUtils.toString(reply), t); + LOG.error("Failed onNext request={}, reply={}", request, + ServerProtoUtils.toString(reply), t); } } 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 e092818254..8ec872bec8 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 @@ -126,7 +126,7 @@ PROTO_REP extends GeneratedMessageV3> REPLY handleRequest( PROTO_REP replyProto = byteToProto.apply(reply); return repToProto.apply(replyProto); } catch (ServiceException se) { - LOG.trace("Failed to handle " + request, se); + LOG.trace("Failed to handle {}", request, se); throw new IOException(se); } } 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 daff7e4d31..35a0561736 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 @@ -108,8 +108,8 @@ private HadoopRpcService(RaftServer server, final Configuration conf) { addRaftClientProtocol(server, conf); - LOG.info(getClass().getSimpleName() + " created RPC.Server at " - + ipcServerAddress); + LOG.info("{} created RPC.Server at {}", getClass().getSimpleName(), + ipcServerAddress); } @Override diff --git a/ratis-logservice/src/main/java/org/apache/ratis/logservice/server/LogStateMachine.java b/ratis-logservice/src/main/java/org/apache/ratis/logservice/server/LogStateMachine.java index 02e9234465..f74f157b5e 100644 --- a/ratis-logservice/src/main/java/org/apache/ratis/logservice/server/LogStateMachine.java +++ b/ratis-logservice/src/main/java/org/apache/ratis/logservice/server/LogStateMachine.java @@ -209,8 +209,8 @@ public long takeSnapshot() { out.writeLong(dataRecordsSize); out.writeObject(state); } catch(IOException ioe) { - LOG.warn("Failed to write snapshot file \"" + snapshotFile - + "\", last applied index=" + last); + LOG.warn("Failed to write snapshot file \"{}\", last applied index={}", + snapshotFile, last); } return last.getIndex(); @@ -489,7 +489,7 @@ public void close() { try { client.close(); } catch (Exception ignored) { - LOG.warn(ignored.getClass().getSimpleName() + " is ignored", ignored); + LOG.warn("{} is ignored", ignored.getClass().getSimpleName(), ignored); } } } @@ -684,7 +684,7 @@ private CompletableFuture processArchiveLog( } return true; } catch (Exception e) { - LOG.error("Archival failed for the log:" + logName, e); + LOG.error("Archival failed for the log:{}", logName, e); failArchival(recordId, logName, location); } finally { timerContext.stop(); @@ -695,7 +695,7 @@ private CompletableFuture processArchiveLog( archiveExportFutures.put(location, executorService.submit(callable)); } }catch (Exception e){ - LOG.warn("Exception while processing archival request for " + logName, e); + LOG.warn("Exception while processing archival request for {}", logName, e); t = e; } return CompletableFuture.completedFuture( diff --git a/ratis-logservice/src/main/java/org/apache/ratis/logservice/tool/VerificationTool.java b/ratis-logservice/src/main/java/org/apache/ratis/logservice/tool/VerificationTool.java index 0c778dfc10..48a4fad2be 100644 --- a/ratis-logservice/src/main/java/org/apache/ratis/logservice/tool/VerificationTool.java +++ b/ratis-logservice/src/main/java/org/apache/ratis/logservice/tool/VerificationTool.java @@ -188,7 +188,7 @@ private static void waitForCompletion(List> futures) { try { Object object = future.get(); if (object != null) { - LOG.error("Operation failed with error ", object); + LOG.error("Operation failed with error {}", object); System.exit(-1); } } catch (Exception e) { @@ -315,7 +315,7 @@ public void run() { for (int i = 0; i < getNumRecords(); i++) { String message = MESSAGE_PREFIX + i; if (i % getLogFreq() == 0) { - LOG.info(getLogName() + " Writing " + message); + LOG.info("{} Writing {}", getLogName(), message); } writer.write(createValue(message)); } @@ -346,7 +346,8 @@ public void run() { String message = MESSAGE_PREFIX + (i * batchSize + j); messages.add(createValue(message)); if((i * batchSize + j) % getLogFreq() == 0) { - LOG.info(getLogName() + " batching write " + message); + LOG.info("{} batching write {}", getLogName(), + message); } } try { @@ -363,7 +364,8 @@ public void run() { String message = MESSAGE_PREFIX + i; lastBatch.add(createValue(message)); } - LOG.info(getLogName() + " writing last mini-batch of " + lastBatch.size() + " records"); + LOG.info("{} writing last mini-batch of {} records", + getLogName(), lastBatch.size()); try { writer.write(lastBatch); } catch (IOException e) { @@ -389,19 +391,19 @@ public void run() { LogReader reader = logStream.createReader(); long size = logStream.getLength(); if(size != getNumRecords()) { - LOG.error("There is mismatch is number of records. Expected Records: "+ - getNumRecords() +", Actual Records: " + size); + LOG.error("There is mismatch is number of records. Expected Records: {}, Actual Records: {}", + getNumRecords(), size); System.exit(-1); } for (int i = 0; i < size; i++) { ByteBuffer buffer = reader.readNext(); String message = parseValue(buffer); if (i % getLogFreq() == 0) { - LOG.info(getLogName() + " Read " + message); + LOG.info("{} Read {}", getLogName(), message); } if(!message.equals(MESSAGE_PREFIX + i)) { - LOG.error("Message is not correct. Expected: "+(MESSAGE_PREFIX + i) - +". Actual:" +message); + LOG.error("Message is not correct. Expected: {}. " + + "Actual:{}", (MESSAGE_PREFIX + i), message); System.exit(-1); } } diff --git a/ratis-metrics/src/main/java/org/apache/ratis/metrics/MetricRegistriesLoader.java b/ratis-metrics/src/main/java/org/apache/ratis/metrics/MetricRegistriesLoader.java index e2da686cf3..c5ed078655 100644 --- a/ratis-metrics/src/main/java/org/apache/ratis/metrics/MetricRegistriesLoader.java +++ b/ratis-metrics/src/main/java/org/apache/ratis/metrics/MetricRegistriesLoader.java @@ -61,7 +61,7 @@ static MetricRegistries load(List availableImplementations) { if (availableImplementations.size() == 1) { // One and only one instance -- what we want/expect MetricRegistries impl = availableImplementations.get(0); - LOG.info("Loaded MetricRegistries " + impl.getClass()); + LOG.info("Loaded MetricRegistries {}", impl.getClass()); return impl; } else if (availableImplementations.isEmpty()) { try { @@ -78,8 +78,8 @@ static MetricRegistries load(List availableImplementations) { } sb.append(factory.getClass()); } - LOG.warn("Found multiple MetricRegistries implementations: " + sb - + ". Using first found implementation: " + availableImplementations.get(0)); + LOG.warn("Found multiple MetricRegistries implementations: {}." + + " Using first found implementation: {}", sb, availableImplementations.get(0)); return availableImplementations.get(0); } } diff --git a/ratis-server/src/main/java/org/apache/ratis/server/impl/FollowerState.java b/ratis-server/src/main/java/org/apache/ratis/server/impl/FollowerState.java index 13ef103de9..2dc35c7af6 100644 --- a/ratis-server/src/main/java/org/apache/ratis/server/impl/FollowerState.java +++ b/ratis-server/src/main/java/org/apache/ratis/server/impl/FollowerState.java @@ -114,11 +114,11 @@ public void run() { } } } catch (InterruptedException e) { - LOG.info(this + " was interrupted: " + e); + LOG.info("{} was interrupted: {}", this, e); LOG.trace("TRACE", e); return; } catch (Exception e) { - LOG.warn(this + " caught an exception", e); + LOG.warn("{} caught an exception", this, e); } } } diff --git a/ratis-server/src/main/java/org/apache/ratis/server/impl/LeaderState.java b/ratis-server/src/main/java/org/apache/ratis/server/impl/LeaderState.java index 0bd3ab4d66..79552d7151 100644 --- a/ratis-server/src/main/java/org/apache/ratis/server/impl/LeaderState.java +++ b/ratis-server/src/main/java/org/apache/ratis/server/impl/LeaderState.java @@ -124,7 +124,7 @@ StateUpdateEvent poll() { } catch(InterruptedException ie) { String s = this + ": poll() is interrupted"; if (!running) { - LOG.info(s + " gracefully"); + LOG.info("{} gracefully", s); return null; } else { throw new IllegalStateException(s + " UNEXPECTEDLY", ie); diff --git a/ratis-server/src/main/java/org/apache/ratis/server/impl/LogAppender.java b/ratis-server/src/main/java/org/apache/ratis/server/impl/LogAppender.java index b796099e48..bea2dda4be 100644 --- a/ratis-server/src/main/java/org/apache/ratis/server/impl/LogAppender.java +++ b/ratis-server/src/main/java/org/apache/ratis/server/impl/LogAppender.java @@ -78,15 +78,15 @@ void run() { try { runAppenderImpl(); } catch (InterruptedException | InterruptedIOException e) { - LOG.info(this + " was interrupted: " + e); + LOG.info("{} was interrupted: {}", this, e); } catch (RaftLogIOException e) { - LOG.error(this + " failed RaftLog", e); + LOG.error("{} failed RaftLog", this, e); transitionLifeCycle(EXCEPTION); } catch (IOException e) { - LOG.error(this + " failed IOException", e); + LOG.error("{} failed IOException", this, e); transitionLifeCycle(EXCEPTION); } catch (Throwable e) { - LOG.error(this + " unexpected exception", e); + LOG.error("{} unexpected exception", this, e); transitionLifeCycle(EXCEPTION); } finally { synchronized (lifeCycle) { diff --git a/ratis-server/src/main/java/org/apache/ratis/server/impl/RaftServerImpl.java b/ratis-server/src/main/java/org/apache/ratis/server/impl/RaftServerImpl.java index 3c4a3d8856..35b53ae41e 100644 --- a/ratis-server/src/main/java/org/apache/ratis/server/impl/RaftServerImpl.java +++ b/ratis-server/src/main/java/org/apache/ratis/server/impl/RaftServerImpl.java @@ -934,7 +934,7 @@ public CompletableFuture appendEntriesAsync(AppendEntri static void logAppendEntries(boolean isHeartbeat, Supplier message) { if (isHeartbeat) { if (LOG.isTraceEnabled()) { - LOG.trace("HEARTBEAT: " + message.get()); + LOG.trace("HEARTBEAT: {}", message.get()); } } else { if (LOG.isDebugEnabled()) { 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 71a095d491..fe2c36d574 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 @@ -199,8 +199,8 @@ void initGroups(RaftGroup group) { addGroup(RaftGroup.valueOf(groupId)); } } catch (Throwable t) { - LOG.warn(getId() + ": Failed to initialize the group directory " - + sub.getAbsolutePath() + ". Ignoring it", t); + LOG.warn("{}: Failed to initialize the group directory {}. " + + "Ignoring it", getId(), sub.getAbsolutePath(), t); } })); raftGroup.ifPresent(this::addGroup); @@ -222,7 +222,8 @@ private static String getIdStringFrom(RaftServerRpc rpc) { try { address = rpc.getInetSocketAddress(); } catch(Exception e) { - LOG.warn("Failed to get InetSocketAddress from " + rpc.getRpcType() + " rpc server", e); + LOG.warn("Failed to get InetSocketAddress from {} rpc server", + rpc.getRpcType(), e); } return address != null? address.getHostName() + "_" + address.getPort() : rpc.getRpcType() + "-" + UUID.randomUUID(); @@ -312,7 +313,7 @@ public void close() { implExecutor.shutdown(); implExecutor.awaitTermination(1, TimeUnit.DAYS); } catch (Exception e) { - LOG.warn(getId() + ": Failed to shutdown " + getRpcType() + " server"); + LOG.warn("{}: Failed to shutdown {} server", getId(), getRpcType()); } lifeCycle.checkStateAndClose(() -> { @@ -322,7 +323,8 @@ public void close() { try { getServerRpc().close(); } catch(IOException ignored) { - LOG.warn(getId() + ": Failed to close " + getRpcType() + " server", ignored); + LOG.warn("{}: Failed to close {} server", getId(), + getRpcType(), ignored); } }); } @@ -391,10 +393,10 @@ private CompletableFuture groupAddAsync(GroupManagementRequest if (throwable != null) { if (!(throwable.getCause() instanceof AlreadyExistsException)) { impls.remove(newGroup.getGroupId()); - LOG.warn(getId() + ": Failed groupAdd* " + request, throwable); + LOG.warn("{}: Failed groupAdd* {}", getId(), request, throwable); } else { if (LOG.isDebugEnabled()) { - LOG.debug(getId() + ": Failed groupAdd* " + request, throwable); + LOG.debug("{}: Failed groupAdd* {}", getId(), request, throwable); } } } diff --git a/ratis-server/src/main/java/org/apache/ratis/server/impl/StateMachineUpdater.java b/ratis-server/src/main/java/org/apache/ratis/server/impl/StateMachineUpdater.java index 50b56e8b9c..9fbf424f41 100644 --- a/ratis-server/src/main/java/org/apache/ratis/server/impl/StateMachineUpdater.java +++ b/ratis-server/src/main/java/org/apache/ratis/server/impl/StateMachineUpdater.java @@ -128,7 +128,8 @@ private void stop() { stateMachine.close(); stateMachineMetrics.unregister(); } catch(Throwable t) { - LOG.warn(name + ": Failed to close " + stateMachine.getClass().getSimpleName() + " " + stateMachine, t); + LOG.warn("{}: Failed to close {} {}", + name, stateMachine.getClass().getSimpleName(), stateMachine, t); } } @@ -269,7 +270,7 @@ private void takeSnapshot() { } stateMachine.getStateMachineStorage().cleanupOldSnapshots(snapshotRetentionPolicy); } catch (IOException e) { - LOG.error(name + ": Failed to take snapshot", e); + LOG.error("{}: Failed to take snapshot", name, e); return; } diff --git a/ratis-server/src/main/java/org/apache/ratis/server/raftlog/RaftLog.java b/ratis-server/src/main/java/org/apache/ratis/server/raftlog/RaftLog.java index 4122262994..8c32cf29f9 100644 --- a/ratis-server/src/main/java/org/apache/ratis/server/raftlog/RaftLog.java +++ b/ratis-server/src/main/java/org/apache/ratis/server/raftlog/RaftLog.java @@ -252,7 +252,7 @@ private boolean shouldAppendMetadata(long newCommitIndex) { return false; } } catch(RaftLogIOException e) { - LOG.error("Failed to get log entry for index " + newCommitIndex, e); + LOG.error("Failed to get log entry for index {}", newCommitIndex, e); } return true; } @@ -386,7 +386,7 @@ public final CompletableFuture purge(long suggestedIndex) { purgeIndex.updateToMax(purged, infoIndexChange); } if (e != null) { - LOG.warn(getName() + ": Failed to purge " + suggestedIndex, e); + LOG.warn("{}: Failed to purge {}", getName(), suggestedIndex, e); } }); } diff --git a/ratis-server/src/main/java/org/apache/ratis/server/raftlog/segmented/SegmentedRaftLogInputStream.java b/ratis-server/src/main/java/org/apache/ratis/server/raftlog/segmented/SegmentedRaftLogInputStream.java index 1db790fa25..8852354ac8 100644 --- a/ratis-server/src/main/java/org/apache/ratis/server/raftlog/segmented/SegmentedRaftLogInputStream.java +++ b/ratis-server/src/main/java/org/apache/ratis/server/raftlog/segmented/SegmentedRaftLogInputStream.java @@ -119,7 +119,7 @@ public LogEntryProto nextEntry() throws IOException { try { init(); } catch (Throwable e) { - LOG.error("caught exception initializing " + this, e); + LOG.error("caught exception initializing {}", this, e); throw IOUtils.asIOException(e); } } @@ -196,7 +196,7 @@ static LogValidation scanEditLog(File file, long maxTxIdToScan) // read the header, initialize the inputstream in.init(); } catch (EOFException e) { - LOG.warn("Log file " + file + " has no valid header", e); + LOG.warn("Log file {} has no valid header", file, e); return new LogValidation(0, INVALID_LOG_INDEX, true); } @@ -231,7 +231,7 @@ static LogValidation scanEditLog(SegmentedRaftLogInputStream in, long maxIndexTo if (hitError) { LogEntryProto entry = in.nextEntry(); index = entry != null ? entry.getIndex() : INVALID_LOG_INDEX; - LOG.warn("After resync, position is " + in.getPosition()); + LOG.warn("After resync, position is {}", in.getPosition()); } else { index = in.scanNextEntry(); } @@ -242,8 +242,8 @@ static LogValidation scanEditLog(SegmentedRaftLogInputStream in, long maxIndexTo } } catch (Throwable t) { LOG.warn("Caught exception after scanning through {} ops from {}" - + " while determining its valid length. Position was " - + lastPos, numValid, in, t); + + " while determining its valid length. Position was {}", + numValid, in, lastPos, t); hitError = true; continue; } diff --git a/ratis-server/src/main/java/org/apache/ratis/server/raftlog/segmented/SegmentedRaftLogReader.java b/ratis-server/src/main/java/org/apache/ratis/server/raftlog/segmented/SegmentedRaftLogReader.java index b529ad03cc..52652e2929 100644 --- a/ratis-server/src/main/java/org/apache/ratis/server/raftlog/segmented/SegmentedRaftLogReader.java +++ b/ratis-server/src/main/java/org/apache/ratis/server/raftlog/segmented/SegmentedRaftLogReader.java @@ -203,9 +203,9 @@ LogEntryProto readEntry() throws IOException { // The last entry is partially written. // It is okay to ignore it since this entry is never committed in this server. if (LOG.isWarnEnabled()) { - LOG.warn("Ignoring the last partial written log entry in " + file + ": " + eof); + LOG.warn("Ignoring the last partial written log entry in {}: {}", file, eof); } else if (LOG.isTraceEnabled()) { - LOG.trace("Ignoring the last partial written log entry in " + file , eof); + LOG.trace("Ignoring the last partial written log entry in {}", file, eof); } return null; } catch (IOException e) { diff --git a/ratis-server/src/main/java/org/apache/ratis/server/raftlog/segmented/SegmentedRaftLogWorker.java b/ratis-server/src/main/java/org/apache/ratis/server/raftlog/segmented/SegmentedRaftLogWorker.java index fa359959ec..764bff9f4a 100644 --- a/ratis-server/src/main/java/org/apache/ratis/server/raftlog/segmented/SegmentedRaftLogWorker.java +++ b/ratis-server/src/main/java/org/apache/ratis/server/raftlog/segmented/SegmentedRaftLogWorker.java @@ -266,8 +266,8 @@ private Task addIOTask(Task task) { task.startTimerOnEnqueue(raftLogQueueingTimer); } catch (Throwable t) { if (t instanceof InterruptedException && !running) { - LOG.info("Got InterruptedException when adding task " + task - + ". The SegmentedRaftLogWorker already stopped."); + LOG.info("Got InterruptedException when adding task {}. " + + "The SegmentedRaftLogWorker already stopped.", task); } else { LOG.error("Failed to add IO task {}", task, t); if (server != null) { @@ -304,9 +304,9 @@ public void run() { } } catch (IOException e) { if (task.getEndIndex() < lastWrittenIndex) { - LOG.info("Ignore IOException when handling task " + task + LOG.info("Ignore IOException when handling task {}" + " which is smaller than the lastWrittenIndex." - + " There should be a snapshot installed.", e); + + " There should be a snapshot installed.", task, e); } else { task.failed(e); if (logIOException == null) { @@ -324,9 +324,8 @@ public void run() { LOG.warn("{} got interrupted while still running", Thread.currentThread().getName()); } - LOG.info(Thread.currentThread().getName() - + " was interrupted, exiting. There are " + queue.getNumElements() - + " tasks remaining in the queue."); + LOG.info("{} was interrupted, exiting. There are {} tasks remaining in " + + "the queue.", Thread.currentThread().getName(), queue.getNumElements()); Thread.currentThread().interrupt(); return; } catch (Throwable t) { @@ -466,8 +465,9 @@ private class WriteLog extends Task { // this.entry != entry iff the entry has state machine data this.stateMachineFuture = stateMachine.data().write(entry); } catch (Throwable e) { - LOG.error(name + ": writeStateMachineData failed for index " + entry.getIndex() - + ", entry=" + ServerProtoUtils.toLogEntryString(entry, stateMachine::toStateMachineLogEntryString), e); + LOG.error("{}: writeStateMachineData failed for index {}, entry={}", + name, entry.getIndex(), ServerProtoUtils.toLogEntryString( + entry, stateMachine::toStateMachineLogEntryString), e); throw e; } } diff --git a/ratis-server/src/main/java/org/apache/ratis/server/storage/RaftStorage.java b/ratis-server/src/main/java/org/apache/ratis/server/storage/RaftStorage.java index f98987557a..64e752f311 100644 --- a/ratis-server/src/main/java/org/apache/ratis/server/storage/RaftStorage.java +++ b/ratis-server/src/main/java/org/apache/ratis/server/storage/RaftStorage.java @@ -81,8 +81,8 @@ public CorruptionPolicy getLogCorruptionPolicy() { private void format() throws IOException { storageDir.clearDirectory(); metaFile = writeMetaFile(MetaFile.DEFAULT_TERM, MetaFile.EMPTY_VOTEFOR); - LOG.info("Storage directory " + storageDir.getRoot() - + " has been successfully formatted."); + LOG.info("Storage directory {} has been successfully formatted.", + storageDir.getRoot()); } private MetaFile writeMetaFile(long term, String votedFor) throws IOException { @@ -134,7 +134,7 @@ public void writeRaftConfiguration(LogEntryProto conf) { try (FileOutputStream fio = new FileOutputStream(confFile)) { conf.writeTo(fio); } catch (Exception e) { - LOG.error("Failed writing configuration to file:" + confFile, e); + LOG.error("Failed writing configuration to file:{}", confFile, e); } } @@ -144,7 +144,7 @@ public RaftConfiguration readRaftConfiguration() { LogEntryProto confProto = LogEntryProto.newBuilder().mergeFrom(fio).build(); return ServerProtoUtils.toRaftConfiguration(confProto); } catch (Exception e) { - LOG.error("Failed reading configuration from file:" + confFile, e); + LOG.error("Failed reading configuration from file:{}", confFile, e); return null; } } diff --git a/ratis-server/src/main/java/org/apache/ratis/server/storage/RaftStorageDirectory.java b/ratis-server/src/main/java/org/apache/ratis/server/storage/RaftStorageDirectory.java index c9d8cc3165..e0cfc75418 100644 --- a/ratis-server/src/main/java/org/apache/ratis/server/storage/RaftStorageDirectory.java +++ b/ratis-server/src/main/java/org/apache/ratis/server/storage/RaftStorageDirectory.java @@ -132,7 +132,7 @@ void clearDirectory() throws IOException { private static void clearDirectory(File dir) throws IOException { if (dir.exists()) { - LOG.info(dir + " already exists. Deleting it ..."); + LOG.info("{} already exists. Deleting it ...", dir); FileUtils.deleteFully(dir); } FileUtils.createDirectories(dir); @@ -217,7 +217,7 @@ public static LogPathAndIndex processOnePath(Path path) throws IOException { if (matcher.matches()) { if (pattern == OPEN_SEGMENT_REGEX && Files.size(path) == 0L) { Files.delete(path); - LOG.info("Delete zero size file " + path); + LOG.info("Delete zero size file {}", path); return null; } final long startIndex = Long.parseLong(matcher.group(1)); @@ -258,20 +258,20 @@ StorageState analyzeStorage(boolean toLock) throws IOException { String rootPath = root.getCanonicalPath(); try { // check that storage exists if (!root.exists()) { - LOG.info("The storage directory " + rootPath + " does not exist. Creating ..."); + LOG.info("The storage directory {} does not exist. Creating ...", rootPath); FileUtils.createDirectories(root); } // or is inaccessible if (!root.isDirectory()) { - LOG.warn(rootPath + " is not a directory"); + LOG.warn("{} is not a directory", rootPath); return StorageState.NON_EXISTENT; } if (!Files.isWritable(root.toPath())) { - LOG.warn("The storage directory " + rootPath + " is not writable."); + LOG.warn("The storage directory {} is not writable.", rootPath); return StorageState.NON_EXISTENT; } } catch(SecurityException ex) { - LOG.warn("Cannot access storage directory " + rootPath, ex); + LOG.warn("Cannot access storage directory {}", rootPath, ex); return StorageState.NON_EXISTENT; } @@ -336,21 +336,21 @@ private FileLock tryLock(File lockF) throws IOException { try { res = file.getChannel().tryLock(); if (null == res) { - LOG.error("Unable to acquire file lock on path " + lockF.toString()); + LOG.error("Unable to acquire file lock on path {}", lockF.toString()); throw new OverlappingFileLockException(); } file.write(jvmName.getBytes(StandardCharsets.UTF_8)); - LOG.info("Lock on " + lockF + " acquired by nodename " + jvmName); + LOG.info("Lock on {} acquired by nodename {}", lockF, jvmName); } catch (OverlappingFileLockException oe) { // Cannot read from the locked file on Windows. LOG.error("It appears that another process " - + "has already locked the storage directory: " + root, oe); + + "has already locked the storage directory: {}", root, oe); file.close(); throw new IOException("Failed to lock storage " + this.root + ". The directory is already locked", oe); } catch(IOException e) { - LOG.error("Failed to acquire lock on " + lockF - + ". If this storage directory is mounted via NFS, " - + "ensure that the appropriate nfs lock services are running.", e); + LOG.error("Failed to acquire lock on {}. " + + "If this storage directory is mounted via NFS, " + + "ensure that the appropriate nfs lock services are running.", lockF, e); file.close(); throw e; } From 90e368c3775adc538843170c775b857cf646cb9f Mon Sep 17 00:00:00 2001 From: Cyrus Date: Sun, 26 Jul 2020 12:53:53 +0530 Subject: [PATCH 2/5] Made few toString corrections --- .../src/main/java/org/apache/ratis/conf/RaftProperties.java | 2 +- .../ratis/examples/counter/server/CounterStateMachine.java | 2 +- .../org/apache/ratis/server/storage/RaftStorageDirectory.java | 4 ++-- 3 files changed, 4 insertions(+), 4 deletions(-) diff --git a/ratis-common/src/main/java/org/apache/ratis/conf/RaftProperties.java b/ratis-common/src/main/java/org/apache/ratis/conf/RaftProperties.java index 1bdf73f1dc..f409da3704 100644 --- a/ratis-common/src/main/java/org/apache/ratis/conf/RaftProperties.java +++ b/ratis-common/src/main/java/org/apache/ratis/conf/RaftProperties.java @@ -911,7 +911,7 @@ public Pattern getPattern(String name, Pattern defaultValue) { try { return Pattern.compile(valString); } catch (PatternSyntaxException pse) { - LOG.warn("Regular expression {} for property {} not valid. Using default", + LOG.warn("Regular expression '{}' for property {} not valid. Using default", valString, name, pse); return defaultValue; } diff --git a/ratis-examples/src/main/java/org/apache/ratis/examples/counter/server/CounterStateMachine.java b/ratis-examples/src/main/java/org/apache/ratis/examples/counter/server/CounterStateMachine.java index 1ef88793e0..835167c698 100644 --- a/ratis-examples/src/main/java/org/apache/ratis/examples/counter/server/CounterStateMachine.java +++ b/ratis-examples/src/main/java/org/apache/ratis/examples/counter/server/CounterStateMachine.java @@ -202,7 +202,7 @@ public CompletableFuture applyTransaction(TransactionContext trx) { //if leader, log the incremented value and it's log index if (trx.getServerRole() == RaftProtos.RaftPeerRole.LEADER) { - LOG.info("{}: Increment to {}", index, counter); + LOG.info("{}: Increment to {}", index, counter.toString()); } return f; diff --git a/ratis-server/src/main/java/org/apache/ratis/server/storage/RaftStorageDirectory.java b/ratis-server/src/main/java/org/apache/ratis/server/storage/RaftStorageDirectory.java index e0cfc75418..8261cd616a 100644 --- a/ratis-server/src/main/java/org/apache/ratis/server/storage/RaftStorageDirectory.java +++ b/ratis-server/src/main/java/org/apache/ratis/server/storage/RaftStorageDirectory.java @@ -340,7 +340,7 @@ private FileLock tryLock(File lockF) throws IOException { throw new OverlappingFileLockException(); } file.write(jvmName.getBytes(StandardCharsets.UTF_8)); - LOG.info("Lock on {} acquired by nodename {}", lockF, jvmName); + LOG.info("Lock on {} acquired by nodename {}", lockF.toString(), jvmName); } catch (OverlappingFileLockException oe) { // Cannot read from the locked file on Windows. LOG.error("It appears that another process " @@ -350,7 +350,7 @@ private FileLock tryLock(File lockF) throws IOException { } catch(IOException e) { LOG.error("Failed to acquire lock on {}. " + "If this storage directory is mounted via NFS, " - + "ensure that the appropriate nfs lock services are running.", lockF, e); + + "ensure that the appropriate nfs lock services are running.", lockF.toString(), e); file.close(); throw e; } From df15155944db96f78e29d69023279292399162f3 Mon Sep 17 00:00:00 2001 From: Cyrus Date: Sun, 26 Jul 2020 15:00:20 +0530 Subject: [PATCH 3/5] triger ci From ab845fc691e40ced8b4bad9110b1e6c0872e7166 Mon Sep 17 00:00:00 2001 From: Cyrus Date: Sun, 16 Aug 2020 18:41:18 +0530 Subject: [PATCH 4/5] Trigger Build From d0dce5cdc4badb719df933ac9409b885864dc82e Mon Sep 17 00:00:00 2001 From: Cyrus Date: Tue, 25 Aug 2020 17:15:12 +0530 Subject: [PATCH 5/5] Trigger notification