KAFKA-17387 Remove broker-list in VerifiableConsumer (#17406)

Reviewers: Chia-Ping Tsai <chia7712@gmail.com>
This commit is contained in:
PoAn Yang 2024-10-10 11:41:53 +08:00 committed by GitHub
parent c36b993af0
commit 9bbf0950f9
No known key found for this signature in database
GPG Key ID: B5690EEEBB952194
3 changed files with 13 additions and 23 deletions

View File

@ -102,6 +102,9 @@
</li>
<li>The <code>--topic-white-list</code> option was removed from the <code>kafka-replica-verification</code> command line tool.
Please use <code>--topics-include</code> instead.
</li>li>
<li>The <code>--broker-list</code> option was removed from the <code>kafka-verifiable-consumer</code> command line tool.
Please use <code>--bootstrap-server</code> instead.
</li>
</ul>
</li>

View File

@ -21,7 +21,7 @@ from ducktape.services.background_thread import BackgroundThreadService
from kafkatest.directory_layout.kafka_path import KafkaPathResolverMixin
from kafkatest.services.kafka import TopicPartition, consumer_group
from kafkatest.services.verifiable_client import VerifiableClientMixin
from kafkatest.version import DEV_BRANCH, V_2_3_0, V_2_3_1, V_3_7_0, V_0_10_0_0
from kafkatest.version import DEV_BRANCH, V_2_3_0, V_2_3_1, V_3_7_0, V_0_10_0_0, V_4_0_0
class ConsumerState:
@ -416,9 +416,13 @@ class VerifiableConsumer(KafkaPathResolverMixin, VerifiableClientMixin, Backgrou
if self.enable_autocommit:
cmd += " --enable-autocommit "
cmd += " --reset-policy %s --group-id %s --topic %s --broker-list %s --session-timeout %s" % \
if node.version < V_4_0_0:
cmd += " --broker-list %s" % self.kafka.bootstrap_servers(self.security_config.security_protocol)
else:
cmd += " --bootstrap-server %s" % self.kafka.bootstrap_servers(self.security_config.security_protocol)
cmd += " --reset-policy %s --group-id %s --topic %s --session-timeout %s" % \
(self.reset_policy, self.group_id, self.topic,
self.kafka.bootstrap_servers(self.security_config.security_protocol),
self.session_timeout_sec*1000)
if self.max_messages > 0:

View File

@ -514,18 +514,11 @@ public class VerifiableConsumer implements Closeable, OffsetCommitCallback, Cons
.required(true);
connectionGroup.addArgument("--bootstrap-server")
.action(store())
.required(false)
.required(true)
.type(String.class)
.metavar("HOST1:PORT1[,HOST2:PORT2[...]]")
.dest("bootstrapServer")
.help("REQUIRED unless --broker-list(deprecated) is specified. The server(s) to connect to. Comma-separated list of Kafka brokers in the form HOST1:PORT1,HOST2:PORT2,...");
connectionGroup.addArgument("--broker-list")
.action(store())
.required(false)
.type(String.class)
.metavar("HOST1:PORT1[,HOST2:PORT2[...]]")
.dest("brokerList")
.help("DEPRECATED, use --bootstrap-server instead; ignored if --bootstrap-server is specified. Comma-separated list of Kafka brokers in the form HOST1:PORT1,HOST2:PORT2,...");
.help("The server(s) to connect to. Comma-separated list of Kafka brokers in the form HOST1:PORT1,HOST2:PORT2,...");
parser.addArgument("--topic")
.action(store())
@ -631,7 +624,7 @@ public class VerifiableConsumer implements Closeable, OffsetCommitCallback, Cons
boolean useAutoCommit = res.getBoolean("useAutoCommit");
String configFile = res.getString("consumer.config");
String brokerHostandPort = null;
String brokerHostandPort = res.getString("bootstrapServer");
Properties consumerProps = new Properties();
if (configFile != null) {
@ -667,16 +660,6 @@ public class VerifiableConsumer implements Closeable, OffsetCommitCallback, Cons
consumerProps.put(ConsumerConfig.GROUP_INSTANCE_ID_CONFIG, groupInstanceId);
}
if (res.get("bootstrapServer") != null) {
brokerHostandPort = res.getString("bootstrapServer");
} else if (res.getString("brokerList") != null) {
brokerHostandPort = res.getString("brokerList");
} else {
parser.printHelp();
// Can't use `Exit.exit` here because it didn't exist until 0.11.0.0.
System.exit(0);
}
consumerProps.put(ConsumerConfig.BOOTSTRAP_SERVERS_CONFIG, brokerHostandPort);
consumerProps.put(ConsumerConfig.ENABLE_AUTO_COMMIT_CONFIG, useAutoCommit);