Status

Current state: Accepted

Discussion thread: here

Voting thread: here

JIRA KAFKA-16228 - Getting issue details... STATUS

Please keep the discussion on the mailing list rather than commenting on the wiki (wiki discussions get unwieldy fast).

Motivation

The kafka-dump-log.sh supports a number of data and metadata decoder flags to be used with Kafka internal topics.

KIP-405 introduced support for tiered storage, that includes a topic based implementation of the RemoteLogMetadataManager (RLMM), which is used to store RemoteLogSegmentMetadata records.

Each metadata operation has an associated record type defined in storage/src/main/resources/message, which is serialized using the RemoteLogMetadataSerde, and then stored in the __remote_log_metadata internal topic.

When debugging tiered storage issues, it would be useful to have a decode option for remote log metadata record payloads, like we have for the other internal topics.

Public Interfaces

As shown below, the kafka-dump-log.sh script will support a new option called --remote-log-metadata-decoder which will allow to decode the remote log metadata record payloads.

$ bin/kafka-dump-log.sh 2> >(grep -A5 remote-log-metadata-decoder)
--remote-log-metadata-decoder        If set, log data will be parsed as         
                                       TopicBasedRemoteLogMetadataManager       
                                       (RLMM) metadata records. Instead, the    
                                       value-decoder-class option can be used   
                                       if a custom RLMM implementation is       
                                       configured.

$ bin/kafka-dump-log.sh --remote-log-metadata-decoder --max-bytes 500 --files /opt/kafka/data/__remote_log_metadata-0/00000000000000000000.log 
Dumping /home/fvaleri/.local/tmp/kafka/data/__remote_log_metadata-0/00000000000000000000.log
Log starting offset: 0
baseOffset: 0 lastOffset: 0 count: 1 baseSequence: 0 lastSequence: 0 producerId: 0 producerEpoch: 0 partitionLeaderEpoch: 0 isTransactional: false isControl: false deleteHorizonMs: OptionalLong.empty position: 0 CreateTime: 1718198522340 size: 177 magic: 2 compresscodec: none crc: 65027821 isvalid: true
| offset: 0 CreateTime: 1718198522340 keySize: -1 valueSize: 107 sequence: 0 headerKeys: [] payload: RemoteLogSegmentMetadata{remoteLogSegmentId=RemoteLogSegmentId{topicIdPartition=iaRqtGKpRxWv22RDNVl6XA:my-topic-0, id=WyySRmq7QLidlaHnt6NQCg}, startOffset=0, endOffset=2, brokerId=1, maxTimestampMs=1718198506346, eventTimestampMs=1718198522330, segmentLeaderEpochs={0=0}, segmentSizeInBytes=91, customMetadata=Optional.empty, state=COPY_SEGMENT_STARTED}
baseOffset: 1 lastOffset: 1 count: 1 baseSequence: 1 lastSequence: 1 producerId: 0 producerEpoch: 0 partitionLeaderEpoch: 0 isTransactional: false isControl: false deleteHorizonMs: OptionalLong.empty position: 177 CreateTime: 1718198522415 size: 135 magic: 2 compresscodec: none crc: 1090785376 isvalid: true
| offset: 1 CreateTime: 1718198522415 keySize: -1 valueSize: 65 sequence: 1 headerKeys: [] payload: RemoteLogSegmentMetadataUpdate{remoteLogSegmentId=RemoteLogSegmentId{topicIdPartition=iaRqtGKpRxWv22RDNVl6XA:my-topic-0, id=WyySRmq7QLidlaHnt6NQCg}, customMetadata=Optional.empty, state=COPY_SEGMENT_FINISHED, eventTimestampMs=1718198522415, brokerId=1}
baseOffset: 2 lastOffset: 2 count: 1 baseSequence: 2 lastSequence: 2 producerId: 0 producerEpoch: 0 partitionLeaderEpoch: 0 isTransactional: false isControl: false deleteHorizonMs: OptionalLong.empty position: 312 CreateTime: 1718198552486 size: 177 magic: 2 compresscodec: none crc: 2172145479 isvalid: true
| offset: 2 CreateTime: 1718198552486 keySize: -1 valueSize: 107 sequence: 2 headerKeys: [] payload: RemoteLogSegmentMetadata{remoteLogSegmentId=RemoteLogSegmentId{topicIdPartition=iaRqtGKpRxWv22RDNVl6XA:my-topic-0, id=EcPdVD8LSdyZuLrDcrv6rA}, startOffset=3, endOffset=962, brokerId=1, maxTimestampMs=1718198543964, eventTimestampMs=1718198552486, segmentLeaderEpochs={0=3}, segmentSizeInBytes=995584, customMetadata=Optional.empty, state=COPY_SEGMENT_STARTED}

Proposed Changes

The kafka-dump-log.sh uses the DumpLogSegments object that contains a set of MessageParser objects for deserializing the record's payload. A new RemoteMetadataLogMessageParser will be added to be used with the new decode option.

The RemoteMetadataLogMessageParser will use the RemoteLogMetadataSerde.deserialize method, which is able to extract apiKey and version from the record's payload, and select the correct RemoteLogMetadataTransform instance required for decoding.

Currently all RemoteLogSegmentMetadata schemas are at version 0, so we cannot have mixed versions of records. In the future, these schemas may evolve, so the RemoteLogMetadataSerde will need to be updated to decode all supported versions.

Compatibility, Deprecation, and Migration Plan

There is no impact on existing users, as this new flag is optional and does not change existing options.

Test Plan

A new unit test will be added to check the log dump tool's output when using the new decode option.

Rejected Alternatives

  1. Create a new class implementing the Decoder trait, and use it with the --value-decoder-class flag. We have dedicated decoder flags for all the other internal Kafka serializers, so this option would mostly be used for custom RLMM implementations.
  2. Use the kafka-console-consumer.sh tool with the RemoteLogMetadataSerde\$RemoteLogMetadataFormatter (this already works today). This approach is inconvenient for a remote support engineer who wants to inspect the remote log metadata records without having to run a client against a running broker. Also, you may have to deal with client security configuration, and it would be complicated to only dump specific segments.



  • No labels