KAFKA-16037: Update VerifiableConsumer to support KIP-848’s group protocol config (#15325)

Add the optional --group-protocol command line option that can be set in the system tests

Reviewers: Andrew Schofield <aschofield@confluent.io>, Lucas Brutschy <lbrutschy@confluent.io>
This commit is contained in:
Kirk True 2024-02-07 01:06:13 -08:00 committed by GitHub
parent af0aceb4d7
commit 08b68583fa
No known key found for this signature in database
GPG Key ID: B5690EEEBB952194
1 changed files with 11 additions and 0 deletions

View File

@ -34,6 +34,7 @@ import org.apache.kafka.clients.consumer.ConsumerConfig;
import org.apache.kafka.clients.consumer.ConsumerRebalanceListener;
import org.apache.kafka.clients.consumer.ConsumerRecord;
import org.apache.kafka.clients.consumer.ConsumerRecords;
import org.apache.kafka.clients.consumer.GroupProtocol;
import org.apache.kafka.clients.consumer.KafkaConsumer;
import org.apache.kafka.clients.consumer.OffsetAndMetadata;
import org.apache.kafka.clients.consumer.OffsetCommitCallback;
@ -528,6 +529,15 @@ public class VerifiableConsumer implements Closeable, OffsetCommitCallback, Cons
.metavar("TOPIC")
.help("Consumes messages from this topic.");
parser.addArgument("--group-protocol")
.action(store())
.required(false)
.type(String.class)
.setDefault(GroupProtocol.CLASSIC.name)
.metavar("GROUP_PROTOCOL")
.dest("groupProtocol")
.help(String.format("Group protocol (must be one of %s)", Utils.join(GroupProtocol.values(), ", ")));
parser.addArgument("--group-id")
.action(store())
.required(true)
@ -617,6 +627,7 @@ public class VerifiableConsumer implements Closeable, OffsetCommitCallback, Cons
}
}
consumerProps.put(ConsumerConfig.GROUP_PROTOCOL_CONFIG, res.getString("groupProtocol"));
consumerProps.put(ConsumerConfig.GROUP_ID_CONFIG, res.getString("groupId"));
String groupInstanceId = res.getString("groupInstanceId");