Skip to content

Commit 899cdb5

Browse files
KAFKA-18811: Added command configs to admin client as well in VerifiableShareConsumer (apache#19130)
This PR includes a new flag in VerifiableShareConsumer.java called command.config to include a properties file for admin client related configs Co-authored-by: Andrew Schofield <[email protected]> Reviewers: Apoorv Mittal <[email protected]>, Andrew Schofield <[email protected]>
1 parent 6ad9ca5 commit 899cdb5

File tree

1 file changed

+22
-14
lines changed

1 file changed

+22
-14
lines changed

tools/src/main/java/org/apache/kafka/tools/VerifiableShareConsumer.java

Lines changed: 22 additions & 14 deletions
Original file line numberDiff line numberDiff line change
@@ -84,13 +84,13 @@ public class VerifiableShareConsumer implements Closeable, AcknowledgementCommit
8484
private final ObjectMapper mapper = new ObjectMapper();
8585
private final PrintStream out;
8686
private final KafkaShareConsumer<String, String> consumer;
87+
private final Admin adminClient;
8788
private final String topic;
8889
private final AcknowledgementMode acknowledgementMode;
8990
private final String offsetResetStrategy;
9091
private final Boolean verbose;
9192
private final int maxMessages;
9293
private Integer totalAcknowledged = 0;
93-
private final String brokerHostandPort;
9494
private final String groupId;
9595
private final CountDownLatch shutdownLatch = new CountDownLatch(1);
9696

@@ -317,22 +317,22 @@ public long offset() {
317317
}
318318

319319
public VerifiableShareConsumer(KafkaShareConsumer<String, String> consumer,
320+
Admin adminClient,
320321
PrintStream out,
321322
Integer maxMessages,
322323
String topic,
323324
AcknowledgementMode acknowledgementMode,
324325
String offsetResetStrategy,
325-
String brokerHostandPort,
326326
String groupId,
327327
Boolean verbose) {
328328
this.out = out;
329329
this.consumer = consumer;
330+
this.adminClient = adminClient;
330331
this.topic = topic;
331332
this.acknowledgementMode = acknowledgementMode;
332333
this.offsetResetStrategy = offsetResetStrategy;
333334
this.verbose = verbose;
334335
this.maxMessages = maxMessages;
335-
this.brokerHostandPort = brokerHostandPort;
336336
this.groupId = groupId;
337337
addKafkaSerializerModule();
338338
}
@@ -407,11 +407,6 @@ public void run() {
407407
ShareGroupAutoOffsetResetStrategy offsetResetStrategy =
408408
ShareGroupAutoOffsetResetStrategy.fromString(this.offsetResetStrategy);
409409

410-
Properties adminClientProps = new Properties();
411-
adminClientProps.put(CommonClientConfigs.BOOTSTRAP_SERVERS_CONFIG, brokerHostandPort);
412-
413-
Admin adminClient = Admin.create(adminClientProps);
414-
415410
ConfigResource configResource = new ConfigResource(ConfigResource.Type.GROUP, groupId);
416411
Map<ConfigResource, Collection<AlterConfigOp>> alterEntries = new HashMap<>();
417412
alterEntries.put(configResource, List.of(new AlterConfigOp(new ConfigEntry(
@@ -555,12 +550,13 @@ private static ArgumentParser argParser() {
555550
.dest("offsetResetStrategy")
556551
.help("Set share group reset strategy (must be either 'earliest' or 'latest')");
557552

558-
parser.addArgument("--consumer.config")
553+
parser.addArgument("--command-config")
559554
.action(store())
560555
.required(false)
561556
.type(String.class)
557+
.dest("commandConfig")
562558
.metavar("CONFIG_FILE")
563-
.help("Consumer config properties file (config options shared with command line parameters will be overridden).");
559+
.help("Config properties file (config options shared with command line parameters will be overridden).");
564560

565561
return parser;
566562
}
@@ -571,8 +567,8 @@ public static VerifiableShareConsumer createFromArgs(ArgumentParser parser, Stri
571567
AcknowledgementMode acknowledgementMode =
572568
AcknowledgementMode.valueOf(res.getString("acknowledgementMode").toUpperCase(Locale.ROOT));
573569
String offsetResetStrategy = res.getString("offsetResetStrategy").toLowerCase(Locale.ROOT);
574-
String configFile = res.getString("consumer.config");
575-
String brokerHostandPort = res.getString("bootstrapServer");
570+
String configFile = res.getString("commandConfig");
571+
String brokerHostAndPort = res.getString("bootstrapServer");
576572

577573
Properties consumerProps = new Properties();
578574
if (configFile != null) {
@@ -587,7 +583,7 @@ public static VerifiableShareConsumer createFromArgs(ArgumentParser parser, Stri
587583

588584
consumerProps.put(ConsumerConfig.GROUP_ID_CONFIG, groupId);
589585

590-
consumerProps.put(ConsumerConfig.BOOTSTRAP_SERVERS_CONFIG, brokerHostandPort);
586+
consumerProps.put(ConsumerConfig.BOOTSTRAP_SERVERS_CONFIG, brokerHostAndPort);
591587

592588
String topic = res.getString("topic");
593589
int maxMessages = res.getInt("maxMessages");
@@ -596,14 +592,26 @@ public static VerifiableShareConsumer createFromArgs(ArgumentParser parser, Stri
596592
StringDeserializer deserializer = new StringDeserializer();
597593
KafkaShareConsumer<String, String> consumer = new KafkaShareConsumer<>(consumerProps, deserializer, deserializer);
598594

595+
Properties adminClientProps = new Properties();
596+
if (configFile != null) {
597+
try {
598+
adminClientProps.putAll(Utils.loadProps(configFile));
599+
} catch (IOException e) {
600+
throw new ArgumentParserException(e.getMessage(), parser);
601+
}
602+
}
603+
604+
adminClientProps.put(CommonClientConfigs.BOOTSTRAP_SERVERS_CONFIG, brokerHostAndPort);
605+
Admin adminClient = Admin.create(adminClientProps);
606+
599607
return new VerifiableShareConsumer(
600608
consumer,
609+
adminClient,
601610
System.out,
602611
maxMessages,
603612
topic,
604613
acknowledgementMode,
605614
offsetResetStrategy,
606-
brokerHostandPort,
607615
groupId,
608616
verbose);
609617
}

0 commit comments

Comments
 (0)