Skip to content
New issue

Have a question about this project? Sign up for a free GitHub account to open an issue and contact its maintainers and the community.

By clicking “Sign up for GitHub”, you agree to our terms of service and privacy statement. We’ll occasionally send you account related emails.

Already on GitHub? Sign in to your account

Ratis 956 - Use format specifier instead of string concat #156

Open
wants to merge 6 commits into
base: master
Choose a base branch
from
Open
Show file tree
Hide file tree
Changes from 3 commits
Commits
File filter

Filter by extension

Filter by extension

Conversations
Failed to load comments.
Loading
Jump to
Jump to file
Failed to load files.
Loading
Diff view
Diff view
Original file line number Diff line number Diff line change
Expand Up @@ -238,7 +238,7 @@ private CompletableFuture<RaftClientReply> 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);
}
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -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);
}
Expand All @@ -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);
}
});
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -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;
}
}
Expand Down Expand Up @@ -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;
}
Expand All @@ -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;
}
Expand Down Expand Up @@ -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;
Expand Down Expand Up @@ -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);
}
}
Expand All @@ -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) {
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -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);
}
}
}
Expand All @@ -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);
}
}

Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -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);
}
}
}
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -163,8 +163,8 @@ static <RETURN, THROWABLE extends Throwable> 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);
}
}

Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -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;
}

Expand Down Expand Up @@ -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;
}
Expand Down
14 changes: 7 additions & 7 deletions ratis-common/src/main/java/org/apache/ratis/util/LogUtils.java
Original file line number Diff line number Diff line change
Expand Up @@ -38,15 +38,15 @@ static <THROWABLE extends Throwable> 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());
}
}

Expand All @@ -58,16 +58,16 @@ static <OUTPUT, THROWABLE extends Throwable> 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;
}
Expand Down Expand Up @@ -119,7 +119,7 @@ static void warn(Logger log, Supplier<String> 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);
}
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -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);
}
}

Expand All @@ -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());
}
}

Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -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();
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -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")));
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -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)
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -178,7 +178,8 @@ public CompletableFuture<Message> 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());
}
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -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);
}
}
}
Expand All @@ -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);
}
}
}
Expand Down Expand Up @@ -259,7 +260,7 @@ boolean responseError(Throwable t, Supplier<String> 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;
Expand All @@ -280,7 +281,7 @@ private class UnorderedRequestStreamObserver extends RequestStreamObserver {
void processClientRequest(RaftClientRequest request) {
final CompletableFuture<Void> 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);
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -79,7 +79,7 @@ public CompletableFuture<RaftClientReply> 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);
}
}
Expand Down Expand Up @@ -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);
}
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -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);
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -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);
}
}
}
Expand Down Expand Up @@ -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);
}
}

Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -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);
}
}
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -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
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -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();
Expand Down Expand Up @@ -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);
}
}
}
Expand Down Expand Up @@ -684,7 +684,7 @@ private CompletableFuture<Message> 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();
Expand All @@ -695,7 +695,7 @@ private CompletableFuture<Message> 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(
Expand Down
Loading