KAFKA-8507; Unify connection name flag for command line tool [KIP-499] (#8023)

This change updates ConsoleProducer, ConsumerPerformance, VerifiableProducer, and VerifiableConsumer classes to add and prefer the --bootstrap-server flag for defining the connection point of the Kafka cluster. This change is part of KIP-499: https://cwiki.apache.org/confluence/display/KAFKA/KIP-499+-+Unify+connection+name+flag+for+command+line+tool.

Reviewers: Ron Dagostino <rdagostino@confluent.io>, Stanislav Kozlovski <stanislav_kozlovski@outlook.com>,  Chia-Ping Tsai <chia7712@gmail.com>, Jason Gustafson <jason@confluent.io>
This commit is contained in:
Mitch 2020-02-13 15:44:51 -06:00 committed by GitHub
parent 7e1c39f75a
commit 96c69da8c1
No known key found for this signature in database
GPG Key ID: 4AEE18F83AFDEB23
9 changed files with 178 additions and 36 deletions

View File

@ -86,7 +86,11 @@ object ConsoleProducer {
props ++= config.extraProducerProps props ++= config.extraProducerProps
props.put(ProducerConfig.BOOTSTRAP_SERVERS_CONFIG, config.brokerList) if(config.bootstrapServer != null)
props.put(ProducerConfig.BOOTSTRAP_SERVERS_CONFIG, config.bootstrapServer)
else
props.put(ProducerConfig.BOOTSTRAP_SERVERS_CONFIG, config.brokerList)
props.put(ProducerConfig.COMPRESSION_TYPE_CONFIG, config.compressionCodec) props.put(ProducerConfig.COMPRESSION_TYPE_CONFIG, config.compressionCodec)
props.put(ProducerConfig.CLIENT_ID_CONFIG, "console-producer") props.put(ProducerConfig.CLIENT_ID_CONFIG, "console-producer")
props.put(ProducerConfig.KEY_SERIALIZER_CLASS_CONFIG, "org.apache.kafka.common.serialization.ByteArraySerializer") props.put(ProducerConfig.KEY_SERIALIZER_CLASS_CONFIG, "org.apache.kafka.common.serialization.ByteArraySerializer")
@ -121,10 +125,15 @@ object ConsoleProducer {
.withRequiredArg .withRequiredArg
.describedAs("topic") .describedAs("topic")
.ofType(classOf[String]) .ofType(classOf[String])
val brokerListOpt = parser.accepts("broker-list", "REQUIRED: The broker list string in the form HOST1:PORT1,HOST2:PORT2.") val brokerListOpt = parser.accepts("broker-list", "DEPRECATED, use --bootstrap-server instead; ignored if --bootstrap-server is specified. The broker list string in the form HOST1:PORT1,HOST2:PORT2.")
.withRequiredArg .withRequiredArg
.describedAs("broker-list") .describedAs("broker-list")
.ofType(classOf[String]) .ofType(classOf[String])
val bootstrapServerOpt = parser.accepts("bootstrap-server", "REQUIRED unless --broker-list(deprecated) is specified. The server(s) to connect to. The broker list string in the form HOST1:PORT1,HOST2:PORT2.")
.requiredUnless("broker-list")
.withRequiredArg
.describedAs("server to connect to")
.ofType(classOf[String])
val syncOpt = parser.accepts("sync", "If set message send requests to the brokers are synchronously, one at a time as they arrive.") val syncOpt = parser.accepts("sync", "If set message send requests to the brokers are synchronously, one at a time as they arrive.")
val compressionCodecOpt = parser.accepts("compression-codec", "The compression codec: either 'none', 'gzip', 'snappy', 'lz4', or 'zstd'." + val compressionCodecOpt = parser.accepts("compression-codec", "The compression codec: either 'none', 'gzip', 'snappy', 'lz4', or 'zstd'." +
"If specified without value, then it defaults to 'gzip'") "If specified without value, then it defaults to 'gzip'")
@ -216,11 +225,17 @@ object ConsoleProducer {
options = tryParse(parser, args) options = tryParse(parser, args)
CommandLineUtils.printHelpAndExitIfNeeded(this, "This tool helps to read data from standard input and publish it to Kafka.") CommandLineUtils.printHelpAndExitIfNeeded(this, "This tool helps to read data from standard input and publish it to Kafka.")
CommandLineUtils.checkRequiredArgs(parser, options, topicOpt, brokerListOpt)
CommandLineUtils.checkRequiredArgs(parser, options, topicOpt)
val topic = options.valueOf(topicOpt) val topic = options.valueOf(topicOpt)
val bootstrapServer = options.valueOf(bootstrapServerOpt)
val brokerList = options.valueOf(brokerListOpt) val brokerList = options.valueOf(brokerListOpt)
ToolsUtils.validatePortOrDie(parser,brokerList)
val brokerHostsAndPorts = options.valueOf(if (options.has(bootstrapServerOpt)) bootstrapServerOpt else brokerListOpt)
ToolsUtils.validatePortOrDie(parser, brokerHostsAndPorts)
val sync = options.has(syncOpt) val sync = options.has(syncOpt)
val compressionCodecOptionValue = options.valueOf(compressionCodecOpt) val compressionCodecOptionValue = options.valueOf(compressionCodecOpt)
val compressionCodec = if (options.has(compressionCodecOpt)) val compressionCodec = if (options.has(compressionCodecOpt))

View File

@ -24,6 +24,7 @@ import java.util.concurrent.atomic.AtomicLong
import java.util.{Properties, Random} import java.util.{Properties, Random}
import com.typesafe.scalalogging.LazyLogging import com.typesafe.scalalogging.LazyLogging
import joptsimple.{OptionException, OptionParser, OptionSet}
import kafka.utils.{CommandLineUtils, ToolsUtils} import kafka.utils.{CommandLineUtils, ToolsUtils}
import org.apache.kafka.clients.consumer.{ConsumerRebalanceListener, KafkaConsumer} import org.apache.kafka.clients.consumer.{ConsumerRebalanceListener, KafkaConsumer}
import org.apache.kafka.common.serialization.ByteArrayDeserializer import org.apache.kafka.common.serialization.ByteArrayDeserializer
@ -202,9 +203,14 @@ object ConsumerPerformance extends LazyLogging {
} }
class ConsumerPerfConfig(args: Array[String]) extends PerfConfig(args) { class ConsumerPerfConfig(args: Array[String]) extends PerfConfig(args) {
val bootstrapServersOpt = parser.accepts("broker-list", "REQUIRED: The server(s) to connect to.") val brokerListOpt = parser.accepts("broker-list", "DEPRECATED, use --bootstrap-server instead; ignored if --bootstrap-server is specified. The broker list string in the form HOST1:PORT1,HOST2:PORT2.")
.withRequiredArg() .withRequiredArg
.describedAs("host") .describedAs("broker-list")
.ofType(classOf[String])
val bootstrapServerOpt = parser.accepts("bootstrap-server", "REQUIRED unless --broker-list(deprecated) is specified. The server(s) to connect to.")
.requiredUnless("broker-list")
.withRequiredArg
.describedAs("server to connect to")
.ofType(classOf[String]) .ofType(classOf[String])
val topicOpt = parser.accepts("topic", "REQUIRED: The topic to consume from.") val topicOpt = parser.accepts("topic", "REQUIRED: The topic to consume from.")
.withRequiredArg .withRequiredArg
@ -250,11 +256,10 @@ object ConsumerPerformance extends LazyLogging {
.ofType(classOf[Long]) .ofType(classOf[Long])
.defaultsTo(10000) .defaultsTo(10000)
options = parser.parse(args: _*) options = tryParse(parser, args)
CommandLineUtils.printHelpAndExitIfNeeded(this, "This tool helps in performance test for the full zookeeper consumer") CommandLineUtils.printHelpAndExitIfNeeded(this, "This tool helps in performance test for the full zookeeper consumer")
CommandLineUtils.checkRequiredArgs(parser, options, topicOpt, numMessagesOpt, bootstrapServersOpt) CommandLineUtils.checkRequiredArgs(parser, options, topicOpt, numMessagesOpt)
val printMetrics = options.has(printMetricsOpt) val printMetrics = options.has(printMetricsOpt)
@ -264,7 +269,10 @@ object ConsumerPerformance extends LazyLogging {
new Properties new Properties
import org.apache.kafka.clients.consumer.ConsumerConfig import org.apache.kafka.clients.consumer.ConsumerConfig
props.put(ConsumerConfig.BOOTSTRAP_SERVERS_CONFIG, options.valueOf(bootstrapServersOpt))
val brokerHostsAndPorts = options.valueOf(if (options.has(bootstrapServerOpt)) bootstrapServerOpt else brokerListOpt)
props.put(ConsumerConfig.BOOTSTRAP_SERVERS_CONFIG, brokerHostsAndPorts)
props.put(ConsumerConfig.GROUP_ID_CONFIG, options.valueOf(groupIdOpt)) props.put(ConsumerConfig.GROUP_ID_CONFIG, options.valueOf(groupIdOpt))
props.put(ConsumerConfig.RECEIVE_BUFFER_CONFIG, options.valueOf(socketBufferSizeOpt).toString) props.put(ConsumerConfig.RECEIVE_BUFFER_CONFIG, options.valueOf(socketBufferSizeOpt).toString)
props.put(ConsumerConfig.MAX_PARTITION_FETCH_BYTES_CONFIG, options.valueOf(fetchSizeOpt).toString) props.put(ConsumerConfig.MAX_PARTITION_FETCH_BYTES_CONFIG, options.valueOf(fetchSizeOpt).toString)
@ -283,5 +291,14 @@ object ConsumerPerformance extends LazyLogging {
val dateFormat = new SimpleDateFormat(options.valueOf(dateFormatOpt)) val dateFormat = new SimpleDateFormat(options.valueOf(dateFormatOpt))
val hideHeader = options.has(hideHeaderOpt) val hideHeader = options.has(hideHeaderOpt)
val recordFetchTimeoutMs = options.valueOf(recordFetchTimeoutOpt).longValue() val recordFetchTimeoutMs = options.valueOf(recordFetchTimeoutOpt).longValue()
def tryParse(parser: OptionParser, args: Array[String]): OptionSet = {
try
parser.parse(args: _*)
catch {
case e: OptionException =>
CommandLineUtils.printUsageAndDie(parser, e.getMessage)
}
}
} }
} }

View File

@ -27,7 +27,7 @@ import kafka.utils.Exit
class ConsoleProducerTest { class ConsoleProducerTest {
val validArgs: Array[String] = Array( val brokerListValidArgs: Array[String] = Array(
"--broker-list", "--broker-list",
"localhost:1001,localhost:1002", "localhost:1001,localhost:1002",
"--topic", "--topic",
@ -37,20 +37,45 @@ class ConsoleProducerTest {
"--property", "--property",
"key.separator=#" "key.separator=#"
) )
val bootstrapServerValidArgs: Array[String] = Array(
"--bootstrap-server",
"localhost:1003,localhost:1004",
"--topic",
"t3",
"--property",
"parse.key=true",
"--property",
"key.separator=#"
)
val invalidArgs: Array[String] = Array( val invalidArgs: Array[String] = Array(
"--t", // not a valid argument "--t", // not a valid argument
"t3" "t3"
) )
val bootstrapServerOverride: Array[String] = Array(
"--broker-list",
"localhost:1001",
"--bootstrap-server",
"localhost:1002",
"--topic",
"t3",
)
@Test @Test
def testValidConfigs(): Unit = { def testValidConfigsBrokerList(): Unit = {
val config = new ConsoleProducer.ProducerConfig(validArgs) val config = new ConsoleProducer.ProducerConfig(brokerListValidArgs)
val producerConfig = new ProducerConfig(ConsoleProducer.producerProps(config)) val producerConfig = new ProducerConfig(ConsoleProducer.producerProps(config))
assertEquals(util.Arrays.asList("localhost:1001", "localhost:1002"), assertEquals(util.Arrays.asList("localhost:1001", "localhost:1002"),
producerConfig.getList(ProducerConfig.BOOTSTRAP_SERVERS_CONFIG)) producerConfig.getList(ProducerConfig.BOOTSTRAP_SERVERS_CONFIG))
} }
@Test
def testValidConfigsBootstrapServer(): Unit = {
val config = new ConsoleProducer.ProducerConfig(bootstrapServerValidArgs)
val producerConfig = new ProducerConfig(ConsoleProducer.producerProps(config))
assertEquals(util.Arrays.asList("localhost:1003", "localhost:1004"),
producerConfig.getList(ProducerConfig.BOOTSTRAP_SERVERS_CONFIG))
}
@Test(expected = classOf[IllegalArgumentException]) @Test(expected = classOf[IllegalArgumentException])
def testInvalidConfigs(): Unit = { def testInvalidConfigs(): Unit = {
Exit.setExitProcedure((_, message) => throw new IllegalArgumentException(message.orNull)) Exit.setExitProcedure((_, message) => throw new IllegalArgumentException(message.orNull))
@ -63,11 +88,18 @@ class ConsoleProducerTest {
@Test @Test
def testParseKeyProp(): Unit = { def testParseKeyProp(): Unit = {
val config = new ConsoleProducer.ProducerConfig(validArgs) val config = new ConsoleProducer.ProducerConfig(brokerListValidArgs)
val reader = Class.forName(config.readerClass).getDeclaredConstructor().newInstance().asInstanceOf[LineMessageReader] val reader = Class.forName(config.readerClass).getDeclaredConstructor().newInstance().asInstanceOf[LineMessageReader]
reader.init(System.in,ConsoleProducer.getReaderProps(config)) reader.init(System.in,ConsoleProducer.getReaderProps(config))
assert(reader.keySeparator == "#") assert(reader.keySeparator == "#")
assert(reader.parseKey) assert(reader.parseKey)
} }
@Test
def testBootstrapServerOverride(): Unit = {
val config = new ConsoleProducer.ProducerConfig(bootstrapServerOverride)
val producerConfig = new ProducerConfig(ConsoleProducer.producerProps(config))
assertEquals(util.Arrays.asList("localhost:1002"),
producerConfig.getList(ProducerConfig.BOOTSTRAP_SERVERS_CONFIG))
}
} }

View File

@ -42,7 +42,7 @@ class ConsumerPerformanceTest {
} }
@Test @Test
def testConfig(): Unit = { def testConfigBrokerList(): Unit = {
//Given //Given
val args: Array[String] = Array( val args: Array[String] = Array(
"--broker-list", "localhost:9092", "--broker-list", "localhost:9092",
@ -54,7 +54,44 @@ class ConsumerPerformanceTest {
val config = new ConsumerPerformance.ConsumerPerfConfig(args) val config = new ConsumerPerformance.ConsumerPerfConfig(args)
//Then //Then
assertEquals("localhost:9092", config.options.valueOf(config.bootstrapServersOpt)) assertEquals("localhost:9092", config.options.valueOf(config.brokerHostsAndPorts))
assertEquals("test", config.topic)
assertEquals(10, config.numMessages)
}
@Test
def testConfigBootStrapServer(): Unit = {
//Given
val args: Array[String] = Array(
"--bootstrap-server", "localhost:9092",
"--topic", "test",
"--messages", "10"
)
//When
val config = new ConsumerPerformance.ConsumerPerfConfig(args)
//Then
assertEquals("localhost:9092", config.options.valueOf(config.brokerHostsAndPorts))
assertEquals("test", config.topic)
assertEquals(10, config.numMessages)
}
@Test
def testBrokerListOverride(): Unit = {
//Given
val args: Array[String] = Array(
"--broker-list", "localhost:9094",
"--bootstrap-server", "localhost:9092",
"--topic", "test",
"--messages", "10"
)
//When
val config = new ConsumerPerformance.ConsumerPerfConfig(args)
//Then
assertEquals("localhost:9092", config.brokerHostsAndPorts)
assertEquals("test", config.topic) assertEquals("test", config.topic)
assertEquals(10, config.numMessages) assertEquals(10, config.numMessages)
} }

View File

@ -73,7 +73,7 @@ test
Run the producer and then type a few messages into the console to send to the server.</p> Run the producer and then type a few messages into the console to send to the server.</p>
<pre class="brush: bash;"> <pre class="brush: bash;">
&gt; bin/kafka-console-producer.sh --broker-list localhost:9092 --topic test &gt; bin/kafka-console-producer.sh --bootstrap-server localhost:9092 --topic test
This is a message This is a message
This is another message This is another message
</pre> </pre>
@ -162,7 +162,7 @@ Topic:test PartitionCount:1 ReplicationFactor:1 Configs:
Let's publish a few messages to our new topic: Let's publish a few messages to our new topic:
</p> </p>
<pre class="brush: bash;"> <pre class="brush: bash;">
&gt; bin/kafka-console-producer.sh --broker-list localhost:9092 --topic my-replicated-topic &gt; bin/kafka-console-producer.sh --bootstrap-server localhost:9092 --topic my-replicated-topic
... ...
my test message 1 my test message 1
my test message 2 my test message 2

View File

@ -232,7 +232,7 @@
<br> <br>
Examples using console-producer and console-consumer: Examples using console-producer and console-consumer:
<pre class="brush: bash;"> <pre class="brush: bash;">
kafka-console-producer.sh --broker-list localhost:9093 --topic test --producer.config client-ssl.properties kafka-console-producer.sh --bootstrap-server localhost:9093 --topic test --producer.config client-ssl.properties
kafka-console-consumer.sh --bootstrap-server localhost:9093 --topic test --consumer.config client-ssl.properties</pre> kafka-console-consumer.sh --bootstrap-server localhost:9093 --topic test --consumer.config client-ssl.properties</pre>
</li> </li>
</ol> </ol>

View File

@ -188,7 +188,7 @@ Hence there won't be any STDOUT output except log entries as the results are wri
Now we can start the console producer in a separate terminal to write some input data to this topic: Now we can start the console producer in a separate terminal to write some input data to this topic:
<pre class="brush: bash;"> <pre class="brush: bash;">
&gt; bin/kafka-console-producer.sh --broker-list localhost:9092 --topic streams-plaintext-input &gt; bin/kafka-console-producer.sh --bootstrap-server localhost:9092 --topic streams-plaintext-input
</pre> </pre>
and inspect the output of the WordCount demo application by reading from its output topic with the console consumer in a separate terminal: and inspect the output of the WordCount demo application by reading from its output topic with the console consumer in a separate terminal:
@ -212,7 +212,7 @@ This will send a new message to the input topic, where the message key is null a
(in practice, input data for applications will typically be streaming continuously into Kafka, rather than being manually entered as we do in this quickstart): (in practice, input data for applications will typically be streaming continuously into Kafka, rather than being manually entered as we do in this quickstart):
<pre class="brush: bash;"> <pre class="brush: bash;">
&gt; bin/kafka-console-producer.sh --broker-list localhost:9092 --topic streams-plaintext-input &gt; bin/kafka-console-producer.sh --bootstrap-server localhost:9092 --topic streams-plaintext-input
all streams lead to kafka all streams lead to kafka
</pre> </pre>
@ -246,7 +246,7 @@ Enter the text line "hello kafka streams" and hit &lt;RETURN&gt;.
Your terminal should look as follows: Your terminal should look as follows:
<pre class="brush: bash;"> <pre class="brush: bash;">
&gt; bin/kafka-console-producer.sh --broker-list localhost:9092 --topic streams-plaintext-input &gt; bin/kafka-console-producer.sh --bootstrap-server localhost:9092 --topic streams-plaintext-input
all streams lead to kafka all streams lead to kafka
hello kafka streams hello kafka streams
</pre> </pre>
@ -279,7 +279,7 @@ representing the most recent word counts as computed by the WordCount applicatio
Let's enter one final input text line "join kafka summit" and hit &lt;RETURN&gt; in the console producer to the input topic <b>streams-plaintext-input</b> before we wrap up this quickstart: Let's enter one final input text line "join kafka summit" and hit &lt;RETURN&gt; in the console producer to the input topic <b>streams-plaintext-input</b> before we wrap up this quickstart:
<pre class="brush: bash;"> <pre class="brush: bash;">
&gt; bin/kafka-console-producer.sh --broker-list localhost:9092 --topic streams-plaintext-input &gt; bin/kafka-console-producer.sh --bootstrap-server localhost:9092 --topic streams-plaintext-input
all streams lead to kafka all streams lead to kafka
hello kafka streams hello kafka streams
join kafka summit join kafka summit

View File

@ -28,6 +28,7 @@ import com.fasterxml.jackson.databind.module.SimpleModule;
import net.sourceforge.argparse4j.ArgumentParsers; import net.sourceforge.argparse4j.ArgumentParsers;
import net.sourceforge.argparse4j.inf.ArgumentParser; import net.sourceforge.argparse4j.inf.ArgumentParser;
import net.sourceforge.argparse4j.inf.ArgumentParserException; import net.sourceforge.argparse4j.inf.ArgumentParserException;
import net.sourceforge.argparse4j.inf.MutuallyExclusiveGroup;
import net.sourceforge.argparse4j.inf.Namespace; import net.sourceforge.argparse4j.inf.Namespace;
import org.apache.kafka.clients.consumer.ConsumerConfig; import org.apache.kafka.clients.consumer.ConsumerConfig;
import org.apache.kafka.clients.consumer.ConsumerRebalanceListener; import org.apache.kafka.clients.consumer.ConsumerRebalanceListener;
@ -504,14 +505,23 @@ public class VerifiableConsumer implements Closeable, OffsetCommitCallback, Cons
.newArgumentParser("verifiable-consumer") .newArgumentParser("verifiable-consumer")
.defaultHelp(true) .defaultHelp(true)
.description("This tool consumes messages from a specific topic and emits consumer events (e.g. group rebalances, received messages, and offsets committed) as JSON objects to STDOUT."); .description("This tool consumes messages from a specific topic and emits consumer events (e.g. group rebalances, received messages, and offsets committed) as JSON objects to STDOUT.");
MutuallyExclusiveGroup connectionGroup = parser.addMutuallyExclusiveGroup("Connection Group")
parser.addArgument("--broker-list") .description("Group of arguments for connection to brokers")
.required(true);
connectionGroup.addArgument("--bootstrap-server")
.action(store()) .action(store())
.required(true) .required(false)
.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) .type(String.class)
.metavar("HOST1:PORT1[,HOST2:PORT2[...]]") .metavar("HOST1:PORT1[,HOST2:PORT2[...]]")
.dest("brokerList") .dest("brokerList")
.help("Comma-separated list of Kafka brokers in the form HOST1:PORT1,HOST2:PORT2,..."); .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,...");
parser.addArgument("--topic") parser.addArgument("--topic")
.action(store()) .action(store())
@ -598,6 +608,7 @@ public class VerifiableConsumer implements Closeable, OffsetCommitCallback, Cons
boolean useAutoCommit = res.getBoolean("useAutoCommit"); boolean useAutoCommit = res.getBoolean("useAutoCommit");
String configFile = res.getString("consumer.config"); String configFile = res.getString("consumer.config");
String brokerHostandPort = null;
Properties consumerProps = new Properties(); Properties consumerProps = new Properties();
if (configFile != null) { if (configFile != null) {
@ -614,7 +625,18 @@ public class VerifiableConsumer implements Closeable, OffsetCommitCallback, Cons
if (groupInstanceId != null) { if (groupInstanceId != null) {
consumerProps.put(ConsumerConfig.GROUP_INSTANCE_ID_CONFIG, groupInstanceId); consumerProps.put(ConsumerConfig.GROUP_INSTANCE_ID_CONFIG, groupInstanceId);
} }
consumerProps.put(ConsumerConfig.BOOTSTRAP_SERVERS_CONFIG, res.getString("brokerList"));
if (res.get("bootstrapServer") != null) {
brokerHostandPort = res.getString("bootstrapServer");
} else if (res.getString("brokerList") != null) {
brokerHostandPort = res.getString("brokerList");
} else {
parser.printHelp();
Exit.exit(0);
}
consumerProps.put(ConsumerConfig.BOOTSTRAP_SERVERS_CONFIG, brokerHostandPort);
consumerProps.put(ConsumerConfig.ENABLE_AUTO_COMMIT_CONFIG, useAutoCommit); consumerProps.put(ConsumerConfig.ENABLE_AUTO_COMMIT_CONFIG, useAutoCommit);
consumerProps.put(ConsumerConfig.AUTO_OFFSET_RESET_CONFIG, res.getString("resetPolicy")); consumerProps.put(ConsumerConfig.AUTO_OFFSET_RESET_CONFIG, res.getString("resetPolicy"));
consumerProps.put(ConsumerConfig.SESSION_TIMEOUT_MS_CONFIG, Integer.toString(res.getInt("sessionTimeout"))); consumerProps.put(ConsumerConfig.SESSION_TIMEOUT_MS_CONFIG, Integer.toString(res.getInt("sessionTimeout")));
@ -643,7 +665,6 @@ public class VerifiableConsumer implements Closeable, OffsetCommitCallback, Cons
parser.printHelp(); parser.printHelp();
Exit.exit(0); Exit.exit(0);
} }
try { try {
final VerifiableConsumer consumer = createFromArgs(parser, args); final VerifiableConsumer consumer = createFromArgs(parser, args);
Exit.addShutdownHook("verifiable-consumer-shutdown-hook", () -> consumer.close()); Exit.addShutdownHook("verifiable-consumer-shutdown-hook", () -> consumer.close());

View File

@ -24,6 +24,7 @@ import com.fasterxml.jackson.databind.ObjectMapper;
import net.sourceforge.argparse4j.ArgumentParsers; import net.sourceforge.argparse4j.ArgumentParsers;
import net.sourceforge.argparse4j.inf.ArgumentParser; import net.sourceforge.argparse4j.inf.ArgumentParser;
import net.sourceforge.argparse4j.inf.ArgumentParserException; import net.sourceforge.argparse4j.inf.ArgumentParserException;
import net.sourceforge.argparse4j.inf.MutuallyExclusiveGroup;
import net.sourceforge.argparse4j.inf.Namespace; import net.sourceforge.argparse4j.inf.Namespace;
import org.apache.kafka.clients.producer.Callback; import org.apache.kafka.clients.producer.Callback;
@ -119,14 +120,24 @@ public class VerifiableProducer implements AutoCloseable {
.type(String.class) .type(String.class)
.metavar("TOPIC") .metavar("TOPIC")
.help("Produce messages to this topic."); .help("Produce messages to this topic.");
MutuallyExclusiveGroup connectionGroup = parser.addMutuallyExclusiveGroup("Connection Group")
parser.addArgument("--broker-list") .description("Group of arguments for connection to brokers")
.required(true);
connectionGroup.addArgument("--bootstrap-server")
.action(store()) .action(store())
.required(true) .required(false)
.type(String.class)
.metavar("HOST1:PORT1[,HOST2:PORT2[...]]")
.dest("bootstrapServer")
.help("REQUIRED: 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) .type(String.class)
.metavar("HOST1:PORT1[,HOST2:PORT2[...]]") .metavar("HOST1:PORT1[,HOST2:PORT2[...]]")
.dest("brokerList") .dest("brokerList")
.help("Comma-separated list of Kafka brokers in the form HOST1:PORT1,HOST2:PORT2,..."); .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,...");
parser.addArgument("--max-messages") parser.addArgument("--max-messages")
.action(store()) .action(store())
@ -222,7 +233,16 @@ public class VerifiableProducer implements AutoCloseable {
createTime = null; createTime = null;
Properties producerProps = new Properties(); Properties producerProps = new Properties();
producerProps.put(ProducerConfig.BOOTSTRAP_SERVERS_CONFIG, res.getString("brokerList"));
if (res.get("bootstrapServer") != null) {
producerProps.put(ProducerConfig.BOOTSTRAP_SERVERS_CONFIG, res.getString("bootstrapServer"));
} else if (res.getString("brokerList") != null) {
producerProps.put(ProducerConfig.BOOTSTRAP_SERVERS_CONFIG, res.getString("brokerList"));
} else {
parser.printHelp();
Exit.exit(0);
}
producerProps.put(ProducerConfig.KEY_SERIALIZER_CLASS_CONFIG, producerProps.put(ProducerConfig.KEY_SERIALIZER_CLASS_CONFIG,
"org.apache.kafka.common.serialization.StringSerializer"); "org.apache.kafka.common.serialization.StringSerializer");
producerProps.put(ProducerConfig.VALUE_SERIALIZER_CLASS_CONFIG, producerProps.put(ProducerConfig.VALUE_SERIALIZER_CLASS_CONFIG,