...
The decommission command removes the registration of a specific broker ID. It will use make an UnregisterBrokerRequest in order to do this.
Changes to kafka-dump-log-segments.sh
kafka-dump-log-seguments.sh will have two new flags: --cluster-metadata-decoder, and --skip-record-metadata.
The --cluster-metadata-decoder flag will tell the DumpLogSegments tool to decode the records as KIP-500 metadata. Each record will be output in the following JSON format:
Code Block |
---|
payload: {"type":<record type enum name>, "version":<record version number>, "data":<record JSON>} |
Example output:
Code Block |
---|
payload: {"type":"TOPIC_RECORD","version":0,"data":{"name":"bar","topicId":"GU_rXds2FGppL1JqXYpx2g"}}
payload: {"type":"PARTITION_RECORD","version":0,"data":{"partitionId":0,"topicId":"GU_rXds2FGppL1JqXYpx2g","replicas":[1],"isr":[1],"removingReplicas":null,"addingReplicas":null,"leader":1,"leaderEpoch":0,"partitionEpoch":0}}
payload: {"type":"PARTITION_CHANGE_RECORD","version":0,"data":{"partitionId":0,"topicId":"GU_rXds2FGppL1JqXYpx2g","leader":-1}}
payload: {"type":"PARTITION_CHANGE_RECORD","version":0,"data":{"partitionId":0,"topicId":"WCnrza5uWKeerYa7HCNpOg","leader":-1}}
payload: {"type":"PARTITION_CHANGE_RECORD","version":0,"data":{"partitionId":0,"topicId":"GU_rXds2FGppL1JqXYpx2g","leader":-1}}
payload: {"type":"PARTITION_CHANGE_RECORD","version":0,"data":{"partitionId":0,"topicId":"WCnrza5uWKeerYa7HCNpOg","leader":-1}}
payload: {"type":"FENCE_BROKER_RECORD","version":0,"data":{"id":1,"epoch":0}} |
The --skip-record-metadata flag will skip printing metadata for each record. However, metadata for each record batch will still be printed when this flag is present.
kafka-shell.sh
The Kafka Metadata shell is a new command which allows users to interactively examine the metadata stored in a KIP-500 cluster.
...