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
KAFKA-14795: Provide message formatter for RemoteLogMetadata #13362
KAFKA-14795: Provide message formatter for RemoteLogMetadata #13362
Conversation
|
||
@Override | ||
public void writeTo(ConsumerRecord<byte[], byte[]> consumerRecord, PrintStream output) { | ||
output.println(remoteLogMetadataSerde.deserialize(consumerRecord.value()).toString()); |
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
Please add a comment here that key
for a RemoteLogMetadataRecord
is null.
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
Done
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
It is good to include offset and partition like below, which will be helpful for debugging.
output.printf("partition: %d, offset: %d, value: %s%n",record.partition(), record.offset(), remoteLogMetadataSerde.deserialize(consumerRecord.value()).toString());
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
Makes sense, done
This commit introduces a formatter for `RemoteLogMetadata`. Example usage: ```bash bin/kafka-console-consumer.sh --bootstrap-server localhost:9092 --topic __remote_log_metadata --from-beginning --formatter 'org.apache.kafka.server.log.remote.metadata.storage.serialization.RemoteLogMetadataSerde$RemoteLogMetadataFormatter' RemoteLogSegmentMetadata{remoteLogSegmentId=RemoteLogSegmentId{topicIdPartition=M1z1YtfhQ5i7oqLNve_0UQ:topic1-0, id=iWtc1Z6xQu2_DJXTklzKxQ}, startOffset=97990, endOffset=98467, brokerId=0, maxTimestampMs=1678292889855, eventTimestampMs=1678292938280, segmentLeaderEpochs={0=97990}, segmentSizeInBytes=511460, state=COPY_SEGMENT_STARTED} ```
ce4cc73
to
b85c79f
Compare
@satishd perhaps you would like to review this since it's associated with KIP405? |
I'll take a look this week. |
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
Thanks @ivanyu for the PR, left a minor comment.
|
||
@Override | ||
public void writeTo(ConsumerRecord<byte[], byte[]> consumerRecord, PrintStream output) { | ||
output.println(remoteLogMetadataSerde.deserialize(consumerRecord.value()).toString()); |
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
It is good to include offset and partition like below, which will be helpful for debugging.
output.printf("partition: %d, offset: %d, value: %s%n",record.partition(), record.offset(), remoteLogMetadataSerde.deserialize(consumerRecord.value()).toString());
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
Thanks @ivanyu for addressing the review comments. LGTM
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
LGTM, the only comment is should we make the formatter as a separate file? @satishd , thoughts?
Failed tests are unrelated
|
…13362) * KAFKA-14795: Provide message formatter for RemoteLogMetadata This commit introduces a formatter for `RemoteLogMetadata`. Example usage: ```bash bin/kafka-console-consumer.sh --bootstrap-server localhost:9092 --topic __remote_log_metadata --from-beginning --formatter 'org.apache.kafka.server.log.remote.metadata.storage.serialization.RemoteLogMetadataSerde$RemoteLogMetadataFormatter' RemoteLogSegmentMetadata{remoteLogSegmentId=RemoteLogSegmentId{topicIdPartition=M1z1YtfhQ5i7oqLNve_0UQ:topic1-0, id=iWtc1Z6xQu2_DJXTklzKxQ}, startOffset=97990, endOffset=98467, brokerId=0, maxTimestampMs=1678292889855, eventTimestampMs=1678292938280, segmentLeaderEpochs={0=97990}, segmentSizeInBytes=511460, state=COPY_SEGMENT_STARTED} ``` Reviewers: Luke Chen <showuon@gmail.com>, Divij Vaidya <diviv@amazon.com>, Satish Duggana <satishd@apache.org>
This commit introduces a formatter for
RemoteLogMetadata
.Example usage:
bin/kafka-console-consumer.sh --bootstrap-server localhost:9092 --topic __remote_log_metadata --from-beginning --formatter 'org.apache.kafka.server.log.remote.metadata.storage.serialization.RemoteLogMetadataSerde$RemoteLogMetadataFormatter' RemoteLogSegmentMetadata{remoteLogSegmentId=RemoteLogSegmentId{topicIdPartition=M1z1YtfhQ5i7oqLNve_0UQ:topic1-0, id=iWtc1Z6xQu2_DJXTklzKxQ}, startOffset=97990, endOffset=98467, brokerId=0, maxTimestampMs=1678292889855, eventTimestampMs=1678292938280, segmentLeaderEpochs={0=97990}, segmentSizeInBytes=511460, state=COPY_SEGMENT_STARTED}
Committer Checklist (excluded from commit message)