RATIS-2068. Avoid logging raw StateMachine data body.#1074
RATIS-2068. Avoid logging raw StateMachine data body.#1074szetszwo merged 5 commits intoapache:masterfrom
Conversation
szetszwo
left a comment
There was a problem hiding this comment.
@duongkame , thanks a lot for fixing this! Please see the comments inlined and also https://issues.apache.org/jira/secure/attachment/13068605/1074_review.patch
| @Override | ||
| String requestToString(AppendEntriesRequestProto request) { | ||
| return ServerStringUtils.toAppendEntriesRequestString(request); | ||
| return ServerStringUtils.toAppendEntriesRequestString(request, e -> ""); |
There was a problem hiding this comment.
Let's pass null. The code currently can handle null.
BTW, could you also update stateMachineLogEntryProtoToString(..) to print only sizes and
toLogEntryString(..) to use stateMachineLogEntryProtoToString(..)?
@@ -70,7 +70,11 @@ public final class LogProtoUtils {
}
static String stateMachineLogEntryProtoToString(StateMachineLogEntryProto p) {
- return "logData:" + p.getLogData() + ", stateMachineEntry:" + p.getType() + ":" + p.getStateMachineEntry();
+ final StateMachineEntryProto stateMachineEntry = p.getStateMachineEntry();
+ return p.getType()
+ + ": logData.size=" + p.getLogData().size()
+ + ", stateMachineData.size=" + stateMachineEntry.getStateMachineData().size()
+ + ", logEntryProtoSerializedSize=" + stateMachineEntry.getLogEntryProtoSerializedSize();
} @@ -46,9 +45,10 @@ public final class LogProtoUtils {
}
final String s;
if (entry.hasStateMachineLogEntry()) {
- s = ", " + Optional.ofNullable(function)
- .orElseGet(() -> proto -> "" + ClientInvocationId.valueOf(proto))
- .apply(entry.getStateMachineLogEntry());
+ if (function == null) {
+ function = LogProtoUtils::stateMachineLogEntryProtoToString;
+ }
+ s = ", " + function.apply(entry.getStateMachineLogEntry());
} else if (entry.hasMetadataEntry()) {
final MetadataProto metadata = entry.getMetadataEntry();
s = "(c:" + metadata.getCommitIndex() + ")";| private final long purgePreservation; | ||
|
|
||
| private final AtomicReference<LogEntryProto> lastMetadataEntry = new AtomicReference<>(); | ||
| private final Function<StateMachineLogEntryProto, String> stateMachineToString; |
There was a problem hiding this comment.
Let don't add StateMachine dependency RaftLogBase . The subclasses can override toLogEntryString(..).
| LOG.error(name + ": Failed to write log entry " + LogProtoUtils.toLogEntryString(e), t); | ||
| LOG.error(name + ": Failed to write log entry " + toString(e), t); | ||
| } else if (returned != nextIndex) { | ||
| LOG.error("{}: Indices mismatched: returned index={} but nextIndex={} for log entry {}", | ||
| name, returned, nextIndex, LogProtoUtils.toLogEntryString(e)); | ||
| name, returned, nextIndex, toString(e)); |
There was a problem hiding this comment.
Use this.toLogEntryString(..).
| public String toLogEntryString(LogEntryProto logEntry) { | ||
| return LogProtoUtils.toLogEntryString(logEntry); | ||
| } |
| } catch (Exception e) { | ||
| final String err = getName() + ": Failed readStateMachineData for " + | ||
| LogProtoUtils.toLogEntryString(entry); | ||
| final String err = getName() + ": Failed readStateMachineData for " + toString(entry); |
There was a problem hiding this comment.
Use this.toLogEntryString(..).
| return write.getFuture().whenComplete((clientReply, exception) -> appendEntryTimerContext.stop()); | ||
| } catch (Exception e) { | ||
| LOG.error("{}: Failed to append {}", getName(), LogProtoUtils.toLogEntryString(entry), e); | ||
| LOG.error("{}: Failed to append {}", getName(), toString(entry), e); |
There was a problem hiding this comment.
Use this.toLogEntryString(..).
Thanks for the quick review @szetszwo . Updated the PR. |
szetszwo
left a comment
There was a problem hiding this comment.
+1 the change looks good.
What changes were proposed in this pull request?
See RATIS-2068.
What is the link to the Apache JIRA
https://issues.apache.org/jira/browse/RATIS-2068
How was this patch tested?
CI