mirror of https://github.com/apache/kafka.git
KAFKA-18234 DumpLogSegments cannot print ConsumerGroupRegularExpression record (#18173)
Reviewers: Chia-Ping Tsai <chia7712@gmail.com>
This commit is contained in:
parent
2966856049
commit
fef625cfa2
|
|
@ -37,7 +37,7 @@ import org.apache.kafka.common.metadata.{MetadataJsonConverters, MetadataRecordT
|
|||
import org.apache.kafka.common.protocol.{ByteBufferAccessor, Message}
|
||||
import org.apache.kafka.common.record._
|
||||
import org.apache.kafka.common.utils.Utils
|
||||
import org.apache.kafka.coordinator.group.generated.{ConsumerGroupCurrentMemberAssignmentKey, ConsumerGroupCurrentMemberAssignmentKeyJsonConverter, ConsumerGroupCurrentMemberAssignmentValue, ConsumerGroupCurrentMemberAssignmentValueJsonConverter, ConsumerGroupMemberMetadataKey, ConsumerGroupMemberMetadataKeyJsonConverter, ConsumerGroupMemberMetadataValue, ConsumerGroupMemberMetadataValueJsonConverter, ConsumerGroupMetadataKey, ConsumerGroupMetadataKeyJsonConverter, ConsumerGroupMetadataValue, ConsumerGroupMetadataValueJsonConverter, ConsumerGroupPartitionMetadataKey, ConsumerGroupPartitionMetadataKeyJsonConverter, ConsumerGroupPartitionMetadataValue, ConsumerGroupPartitionMetadataValueJsonConverter, ConsumerGroupTargetAssignmentMemberKey, ConsumerGroupTargetAssignmentMemberKeyJsonConverter, ConsumerGroupTargetAssignmentMemberValue, ConsumerGroupTargetAssignmentMemberValueJsonConverter, ConsumerGroupTargetAssignmentMetadataKey, ConsumerGroupTargetAssignmentMetadataKeyJsonConverter, ConsumerGroupTargetAssignmentMetadataValue, ConsumerGroupTargetAssignmentMetadataValueJsonConverter, GroupMetadataKey, GroupMetadataKeyJsonConverter, GroupMetadataValue, GroupMetadataValueJsonConverter, OffsetCommitKey, OffsetCommitKeyJsonConverter, OffsetCommitValue, OffsetCommitValueJsonConverter, ShareGroupCurrentMemberAssignmentKey, ShareGroupCurrentMemberAssignmentKeyJsonConverter, ShareGroupCurrentMemberAssignmentValue, ShareGroupCurrentMemberAssignmentValueJsonConverter, ShareGroupMemberMetadataKey, ShareGroupMemberMetadataKeyJsonConverter, ShareGroupMemberMetadataValue, ShareGroupMemberMetadataValueJsonConverter, ShareGroupMetadataKey, ShareGroupMetadataKeyJsonConverter, ShareGroupMetadataValue, ShareGroupMetadataValueJsonConverter, ShareGroupPartitionMetadataKey, ShareGroupPartitionMetadataKeyJsonConverter, ShareGroupPartitionMetadataValue, ShareGroupPartitionMetadataValueJsonConverter, ShareGroupStatePartitionMetadataKey, ShareGroupStatePartitionMetadataKeyJsonConverter, ShareGroupStatePartitionMetadataValue, ShareGroupStatePartitionMetadataValueJsonConverter, ShareGroupTargetAssignmentMemberKey, ShareGroupTargetAssignmentMemberKeyJsonConverter, ShareGroupTargetAssignmentMemberValue, ShareGroupTargetAssignmentMemberValueJsonConverter, ShareGroupTargetAssignmentMetadataKey, ShareGroupTargetAssignmentMetadataKeyJsonConverter, ShareGroupTargetAssignmentMetadataValue, ShareGroupTargetAssignmentMetadataValueJsonConverter}
|
||||
import org.apache.kafka.coordinator.group.generated.{ConsumerGroupCurrentMemberAssignmentKey, ConsumerGroupCurrentMemberAssignmentKeyJsonConverter, ConsumerGroupCurrentMemberAssignmentValue, ConsumerGroupCurrentMemberAssignmentValueJsonConverter, ConsumerGroupMemberMetadataKey, ConsumerGroupMemberMetadataKeyJsonConverter, ConsumerGroupMemberMetadataValue, ConsumerGroupMemberMetadataValueJsonConverter, ConsumerGroupMetadataKey, ConsumerGroupMetadataKeyJsonConverter, ConsumerGroupMetadataValue, ConsumerGroupMetadataValueJsonConverter, ConsumerGroupPartitionMetadataKey, ConsumerGroupPartitionMetadataKeyJsonConverter, ConsumerGroupPartitionMetadataValue, ConsumerGroupPartitionMetadataValueJsonConverter, ConsumerGroupRegularExpressionKey, ConsumerGroupRegularExpressionKeyJsonConverter, ConsumerGroupRegularExpressionValue, ConsumerGroupRegularExpressionValueJsonConverter, ConsumerGroupTargetAssignmentMemberKey, ConsumerGroupTargetAssignmentMemberKeyJsonConverter, ConsumerGroupTargetAssignmentMemberValue, ConsumerGroupTargetAssignmentMemberValueJsonConverter, ConsumerGroupTargetAssignmentMetadataKey, ConsumerGroupTargetAssignmentMetadataKeyJsonConverter, ConsumerGroupTargetAssignmentMetadataValue, ConsumerGroupTargetAssignmentMetadataValueJsonConverter, GroupMetadataKey, GroupMetadataKeyJsonConverter, GroupMetadataValue, GroupMetadataValueJsonConverter, OffsetCommitKey, OffsetCommitKeyJsonConverter, OffsetCommitValue, OffsetCommitValueJsonConverter, ShareGroupCurrentMemberAssignmentKey, ShareGroupCurrentMemberAssignmentKeyJsonConverter, ShareGroupCurrentMemberAssignmentValue, ShareGroupCurrentMemberAssignmentValueJsonConverter, ShareGroupMemberMetadataKey, ShareGroupMemberMetadataKeyJsonConverter, ShareGroupMemberMetadataValue, ShareGroupMemberMetadataValueJsonConverter, ShareGroupMetadataKey, ShareGroupMetadataKeyJsonConverter, ShareGroupMetadataValue, ShareGroupMetadataValueJsonConverter, ShareGroupPartitionMetadataKey, ShareGroupPartitionMetadataKeyJsonConverter, ShareGroupPartitionMetadataValue, ShareGroupPartitionMetadataValueJsonConverter, ShareGroupStatePartitionMetadataKey, ShareGroupStatePartitionMetadataKeyJsonConverter, ShareGroupStatePartitionMetadataValue, ShareGroupStatePartitionMetadataValueJsonConverter, ShareGroupTargetAssignmentMemberKey, ShareGroupTargetAssignmentMemberKeyJsonConverter, ShareGroupTargetAssignmentMemberValue, ShareGroupTargetAssignmentMemberValueJsonConverter, ShareGroupTargetAssignmentMetadataKey, ShareGroupTargetAssignmentMetadataKeyJsonConverter, ShareGroupTargetAssignmentMetadataValue, ShareGroupTargetAssignmentMetadataValueJsonConverter}
|
||||
import org.apache.kafka.coordinator.common.runtime.CoordinatorLoader.UnknownRecordTypeException
|
||||
import org.apache.kafka.coordinator.group.GroupCoordinatorRecordSerde
|
||||
import org.apache.kafka.coordinator.share.ShareCoordinatorRecordSerde
|
||||
|
|
@ -440,6 +440,8 @@ object DumpLogSegments {
|
|||
ConsumerGroupTargetAssignmentMemberKeyJsonConverter.write(m, version)
|
||||
case m: ConsumerGroupCurrentMemberAssignmentKey =>
|
||||
ConsumerGroupCurrentMemberAssignmentKeyJsonConverter.write(m, version)
|
||||
case m: ConsumerGroupRegularExpressionKey =>
|
||||
ConsumerGroupRegularExpressionKeyJsonConverter.write(m, version)
|
||||
case m: ShareGroupMetadataKey =>
|
||||
ShareGroupMetadataKeyJsonConverter.write(m, version)
|
||||
case m: ShareGroupPartitionMetadataKey =>
|
||||
|
|
@ -534,6 +536,8 @@ object DumpLogSegments {
|
|||
ConsumerGroupTargetAssignmentMemberValueJsonConverter.write(m, version)
|
||||
case m: ConsumerGroupCurrentMemberAssignmentValue =>
|
||||
ConsumerGroupCurrentMemberAssignmentValueJsonConverter.write(m, version)
|
||||
case m: ConsumerGroupRegularExpressionValue =>
|
||||
ConsumerGroupRegularExpressionValueJsonConverter.write(m, version)
|
||||
case m: ShareGroupMetadataValue =>
|
||||
ShareGroupMetadataValueJsonConverter.write(m, version)
|
||||
case m: ShareGroupPartitionMetadataValue =>
|
||||
|
|
|
|||
|
|
@ -52,6 +52,9 @@ import org.apache.kafka.coordinator.group.generated.ShareGroupTargetAssignmentMe
|
|||
import org.apache.kafka.coordinator.group.generated.ShareGroupTargetAssignmentMetadataKey;
|
||||
import org.apache.kafka.coordinator.group.generated.ShareGroupTargetAssignmentMetadataValue;
|
||||
|
||||
/**
|
||||
* Please ensure any new record added here stays in sync with DumpLogSegments.
|
||||
*/
|
||||
public class GroupCoordinatorRecordSerde extends CoordinatorRecordSerde {
|
||||
@Override
|
||||
protected ApiMessage apiMessageKeyFor(short recordVersion) {
|
||||
|
|
|
|||
Loading…
Reference in New Issue