Skip to content

Commit

Permalink
RATIS-1946. slf4j: Exception may not be printed.
Browse files Browse the repository at this point in the history
  • Loading branch information
szetszwo committed Nov 30, 2023
1 parent dd36de2 commit 1e720dc
Show file tree
Hide file tree
Showing 13 changed files with 25 additions and 36 deletions.
Original file line number Diff line number Diff line change
Expand Up @@ -181,7 +181,7 @@ private static <E> E parseElement(String[] elements, int i, String input, Functi
try {
return parser.apply(s);
} catch(Exception t) {
LOG.warn("Failed to parse \"{}\", which is the index {} element in \"{}\"", s, i, input, t);
LOG.warn("Failed to parse \"" + s + "\", which is the index " + i + " element in \"" + input + "\"", t);
return null;
}
}
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -175,7 +175,7 @@ private void handle(TimeDuration extraSleep) {
try {
handler.accept(extraSleep);
} catch (Throwable t) {
LOG.error("{}: Failed to handle extra sleep {}", this, extraSleep, t);
LOG.error(this + ": Failed to handle extra sleep " + extraSleep, t);
}
}

Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -167,8 +167,7 @@ private void closeProxy(PROXY proxy, PeerAndProxy pp) {
LOG.debug("{}: Closing proxy for peer {}", name, pp);
proxy.close();
} catch (IOException e) {
LOG.warn("{}: Failed to close proxy for peer {}, proxy class: {}",
name, pp, proxy.getClass(), e);
LOG.warn(name + ": Failed to close proxy for peer " + pp + ", proxy class: " + proxy.getClass(), e);
}
}
}
2 changes: 1 addition & 1 deletion ratis-common/src/test/java/org/apache/ratis/BaseTest.java
Original file line number Diff line number Diff line change
Expand Up @@ -130,7 +130,7 @@ public static void assertThrowable(
Class<? extends Throwable> expectedThrowableClass, Logger log,
Class<? extends Throwable>... expectedCauseClasses) {
if (log != null) {
log.info("The test \"{}\" throws {}", description, JavaUtils.getClassSimpleName(t.getClass()), t);
log.info("The test \"" + description + "\" failed", t);
}
Assert.assertEquals(expectedThrowableClass, t.getClass());

Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -408,7 +408,7 @@ static void sendDataStreamException(Throwable throwable, DataStreamRequestByteBu
try {
ctx.writeAndFlush(newDataStreamReplyByteBuffer(request, reply));
} catch (Throwable t) {
LOG.warn("Failed to sendDataStreamException {} for {}", throwable, request, t);
LOG.warn("Failed to sendDataStreamException " + throwable + " for " + request, t);
} finally {
request.release();
}
Expand Down Expand Up @@ -524,8 +524,7 @@ static boolean checkSuccessRemoteWrite(List<CompletableFuture<DataStreamReply>>
return false;
}
if (reply.getBytesWritten() != bytesWritten) {
LOG.warn(
"reply written bytes not match, local size: {} remote size: {} request: {}",
LOG.warn("reply written bytes not match, local size: {} remote size: {} request: {}",
bytesWritten, reply.getBytesWritten(), request);
return false;
}
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -254,14 +254,12 @@ public void run() {
}
final LifeCycle.State state = lifeCycle.getCurrentState();
if (state.isClosingOrClosed()) {
LOG.info("{}: {} is safely ignored since this is already {}",
this, JavaUtils.getClassSimpleName(e.getClass()), state, e);
LOG.info(this + ": since this is already " + state + ", safely ignore " + e);
} else {
if (!server.getInfo().isAlive()) {
LOG.info("{}: {} is safely ignored since the server is not alive: {}",
this, JavaUtils.getClassSimpleName(e.getClass()), server, e);
LOG.info(this + ": since the server is not alive, safely ignore " + e);
} else {
LOG.error("{}: Failed, state={}", this, state, e);
LOG.error(this + ": Failed, state=" + state, e);
}
shutdown();
}
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -486,9 +486,9 @@ void groupRemove(boolean deleteDirectory, boolean renameDirectory) {
LOG.info("{}: Succeed to remove RaftStorageDirectory {}", getMemberId(), dir);
break;
} catch (NoSuchFileException e) {
LOG.warn("{}: Some file does not exist {}", getMemberId(), dir, e);
LOG.warn(getMemberId() + ": Some file does not exist " + dir, e);
} catch (Exception e) {
LOG.error("{}: Failed to remove RaftStorageDirectory {}", getMemberId(), dir, e);
LOG.error(getMemberId() + ": Failed to remove RaftStorageDirectory " + dir, e);
break;
}
}
Expand All @@ -504,8 +504,7 @@ void groupRemove(boolean deleteDirectory, boolean renameDirectory) {

LOG.info("{}: Group {} is renamed successfully", getMemberId(), getGroup());
} catch (IOException e) {
LOG.warn("{}: Failed to remove group {}", getMemberId(),
dir.getRoot().getName(), e);
LOG.warn(getMemberId() + ": Failed to remove group " + dir.getRoot().getName(), e);
}
}
}
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -129,13 +129,13 @@ private void close(RaftGroupId groupId, CompletableFuture<RaftServerImpl> future
try {
impl = future.join();
} catch (Throwable t) {
LOG.warn("{}: Failed to join the division for {}", getId(), groupId, t);
LOG.warn(getId() + ": Failed to join the division for " + groupId, t);
return;
}
try {
impl.close();
} catch (Throwable t) {
LOG.warn("{}: Failed to close the division for {}", getId(), groupId, t);
LOG.warn(getId() + ": Failed to close the division for " + groupId, t);
}
}

Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -87,7 +87,7 @@ private AppendEntriesReplyProto sendAppendEntriesWithRetries()
} catch (IOException ioe) {
// TODO should have more detailed retry policy here.
if (retry++ % 10 == 0) { // to reduce the number of messages
LOG.warn("{}: Failed to appendEntries (retry={}): {}", this, retry++, ioe);
LOG.warn(this + ": Failed to appendEntries for retry " + retry, ioe);
}
handleException(ioe);
}
Expand All @@ -114,7 +114,7 @@ private InstallSnapshotReplyProto installSnapshot(SnapshotInfo snapshot) throws
} catch (InterruptedIOException iioe) {
throw iioe;
} catch (Exception ioe) {
LOG.warn("{}: Failed to installSnapshot {}: {}", this, snapshot, ioe);
LOG.warn(this + ": Failed to installSnapshot " + snapshot, ioe);
handleException(ioe);
return null;
}
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -143,8 +143,8 @@ public static int readSegmentFile(File file, LogSegmentStartEnd startEnd, SizeIn
switch (corruptionPolicy) {
case EXCEPTION: throw ioe;
case WARN_AND_RETURN:
LOG.warn("Failed to read segment file {} ({}): only {} entries read successfully",
file, startEnd, count, ioe);
LOG.warn("Failed to read segment file " + file + " (" + startEnd
+ "): only " + count + " entries read successfully", ioe);
break;
default:
throw new IllegalStateException("Unexpected enum value: " + corruptionPolicy
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -243,9 +243,8 @@ static LogValidation scanEditLog(SegmentedRaftLogInputStream in, long maxIndexTo
hitError = false;
}
} catch (Exception e) {
LOG.warn("Caught exception after scanning through {} ops from {}"
+ " while determining its valid length. Position was "
+ lastPos, numValid, in, e);
LOG.warn("Failed to scanEditLog at offset " + lastPos + " in " + in
+ ": #valid entries scanned=" + numValid, e);
hitError = true;
continue;
}
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -157,10 +157,10 @@ StorageState analyzeStorage(boolean toLock) throws IOException {
}

// check enough space
if (!hasEnoughSpace()) {
LOG.warn("There are not enough space left for directory {}"
+ " free space min required: {} free space actual: {}",
rootPath, freeSpaceMin, root.getFreeSpace());
final long freeSpace = root.getFreeSpace();
if (freeSpace < freeSpaceMin.getSize()) {
LOG.warn("{} in directory {}: free space = {} < required = {}",
StorageState.NO_SPACE, rootPath, freeSpace, freeSpaceMin);
return StorageState.NO_SPACE;
}

Expand All @@ -177,10 +177,6 @@ public boolean isHealthy() {
return getMetaFile().exists();
}

private boolean hasEnoughSpace() {
return root.getFreeSpace() >= freeSpaceMin.getSize();
}

/**
* Lock storage to provide exclusive access.
*
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -211,8 +211,7 @@ public RaftClientReply handleRequest(RaftClientRequest request)
: IOUtils.asIOException(JavaUtils.unwrapCompletionException(exception));
clientHandler.getRpc().sendReply(request, reply, e);
} catch (IOException e) {
LOG.warn("Failed to send reply {} for request {} due to exception {}",
reply, request, e);
LOG.warn("Failed to send reply " + reply + " for request " + request + " due to exception ", e);
}
}, executor);
return null;
Expand Down

0 comments on commit 1e720dc

Please sign in to comment.