Skip to content

RATIS-2068. Avoid logging raw StateMachine data body.#1074

Merged
szetszwo merged 5 commits intoapache:masterfrom
duongkame:RATIS-2068
May 3, 2024
Merged

RATIS-2068. Avoid logging raw StateMachine data body.#1074
szetszwo merged 5 commits intoapache:masterfrom
duongkame:RATIS-2068

Conversation

@duongkame
Copy link
Copy Markdown
Contributor

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

Copy link
Copy Markdown
Contributor

@szetszwo szetszwo left a comment

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

@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 -> "");
Copy link
Copy Markdown
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

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;
Copy link
Copy Markdown
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Let don't add StateMachine dependency RaftLogBase . The subclasses can override toLogEntryString(..).

Comment on lines +202 to +214
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));
Copy link
Copy Markdown
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Use this.toLogEntryString(..).

Comment on lines -539 to -541
public String toLogEntryString(LogEntryProto logEntry) {
return LogProtoUtils.toLogEntryString(logEntry);
}
Copy link
Copy Markdown
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Let's keep this method.

} catch (Exception e) {
final String err = getName() + ": Failed readStateMachineData for " +
LogProtoUtils.toLogEntryString(entry);
final String err = getName() + ": Failed readStateMachineData for " + toString(entry);
Copy link
Copy Markdown
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

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);
Copy link
Copy Markdown
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Use this.toLogEntryString(..).

@duongkame
Copy link
Copy Markdown
Contributor Author

@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

Thanks for the quick review @szetszwo . Updated the PR.

@duongkame duongkame marked this pull request as ready for review May 3, 2024 16:36
Copy link
Copy Markdown
Contributor

@szetszwo szetszwo left a comment

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

+1 the change looks good.

@szetszwo szetszwo merged commit 51244e4 into apache:master May 3, 2024
szetszwo pushed a commit to szetszwo/ratis that referenced this pull request Jun 16, 2024
Sign up for free to join this conversation on GitHub. Already have an account? Sign in to comment

Labels

None yet

Projects

None yet

Development

Successfully merging this pull request may close these issues.

2 participants