smjn opened a new pull request, #17467: URL: https://github.com/apache/kafka/pull/17467
As part of KIP-932, a new internal topic `__share_group_state` was introduced. There are 2 types of records which are currently being added in this topic. In light of this, we must make the existing tooling like `kafka-console-consumer` and `kafka-dump-logs` aware of these records for debugging and introspection purposes. This PR introduces `ShareRecordFormatter` to be used used with `kafka-console-consumer` and adds an internal class `ShareGroupStateMessageParser` in `DumpLogSegments.scala`. Unit tests have been added to `DumpLogSegmentsTest.scala` ### Testing **kafka-console-consumer** ``` ➜ bin/kafka-console-consumer.sh --bootstrap-server localhost:9092 --from-beginning --topic __share_group_state --property print.key=true --property print.timestamp=true --property print.headers=true --property print.partition=true --timeout-ms 6000 --formatter org.apache.kafka.tools.consumer.group.share.ShareRecordFormatter {"key":{"version":0,"data":{"groupId":"gs1","topicId":"Uj5wn_FqTXirEASvVZRY1w","partition":0}},"value":{"version":0,"data":{"snapshotEpoch":0,"stateEpoch":0,"leaderEpoch":0,"startOffset":0,"stateBatches":[{"firstOffset":0,"lastOffset":4,"deliveryState":2,"deliveryCount":1}]}}}{"key":{"version":1,"data":{"groupId":"gs1","topicId":"Uj5wn_FqTXirEASvVZRY1w","partition":0}},"value":{"version":0,"data":{"snapshotEpoch":0,"leaderEpoch":0,"startOffset":5,"stateBatches":[{"firstOffset":5,"lastOffset":7,"deliveryState":2,"deliveryCount":1}]}}}^CProcessed a total of 2 messages ``` **kafka-dump-log** ``` ➜ bin/kafka-dump-log.sh --files /tmp/kraft-combined-logs/__share_group_state-34/00000000000000000000.log --share-group-state-decoder Dumping /tmp/kraft-combined-logs/__share_group_state-34/00000000000000000000.log Log starting offset: 0 baseOffset: 0 lastOffset: 0 count: 1 baseSequence: 0 lastSequence: 0 producerId: -1 producerEpoch: -1 partitionLeaderEpoch: 0 isTransactional: false isControl: false deleteHorizonMs: OptionalLong.empty position: 0 CreateTime: 1728629394220 size: 138 magic: 2 compresscodec: none crc: 2528359694 isvalid: true | offset: 0 CreateTime: 1728629394220 keySize: 27 valueSize: 42 sequence: 0 headerKeys: [] key: {"type":"0","data":{"groupId":"gs1","topicId":"Uj5wn_FqTXirEASvVZRY1w","partition":0}} payload: {"type":"0","data":{"snapshotEpoch":0,"stateEpoch":0,"leaderEpoch":0,"startOffset":0,"stateBatches":[{"firstOffset":0,"lastOffset":4,"deliveryState":2,"deliveryCount":1}]}} baseOffset: 1 lastOffset: 1 count: 1 baseSequence: 0 lastSequence: 0 producerId: -1 producerEpoch: -1 partitionLeaderEpoch: 0 isTransactional: false isControl: false deleteHorizonMs: OptionalLong.empty position: 138 CreateTime: 1728629410581 size: 134 magic: 2 compresscodec: none crc: 1581677863 isvalid: true | offset: 1 CreateTime: 1728629410581 keySize: 27 valueSize: 38 sequence: 0 headerKeys: [] key: {"type":"1","data":{"groupId":"gs1","topicId":"Uj5wn_FqTXirEASvVZRY1w","partition":0}} payload: {"type":"0","data":{"snapshotEpoch":0,"leaderEpoch":0,"startOffset":5,"stateBatches":[{"firstOffset":5,"lastOffset":7,"deliveryState":2,"deliveryCount":1}]}} ``` -- This is an automated message from the Apache Git Service. To respond to the message, please log on to GitHub and use the URL above to go to the specific comment. To unsubscribe, e-mail: jira-unsubscr...@kafka.apache.org For queries about this service, please contact Infrastructure at: us...@infra.apache.org