From 96c69da8c164966c1281df7001d5d239053442fd Mon Sep 17 00:00:00 2001
From: Mitch Step 4: Send some messages
Run the producer and then type a few messages into the console to send to the server.
-> bin/kafka-console-producer.sh --broker-list localhost:9092 --topic test +> bin/kafka-console-producer.sh --bootstrap-server localhost:9092 --topic test This is a message This is another message@@ -162,7 +162,7 @@
-> bin/kafka-console-producer.sh --broker-list localhost:9092 --topic my-replicated-topic +> bin/kafka-console-producer.sh --bootstrap-server localhost:9092 --topic my-replicated-topic ... my test message 1 my test message 2 diff --git a/docs/security.html b/docs/security.html index d080e045c2e30..f9fca78b358fd 100644 --- a/docs/security.html +++ b/docs/security.html @@ -232,7 +232,7 @@Confi
Examples using console-producer and console-consumer:- 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.propertiesdiff --git a/docs/streams/quickstart.html b/docs/streams/quickstart.html index 4c0d0c92b732f..a6781f62b369a 100644 --- a/docs/streams/quickstart.html +++ b/docs/streams/quickstart.html @@ -188,7 +188,7 @@Step 4: St Now we can start the console producer in a separate terminal to write some input data to this topic:
-> bin/kafka-console-producer.sh --broker-list localhost:9092 --topic streams-plaintext-input +> bin/kafka-console-producer.sh --bootstrap-server localhost:9092 --topic streams-plaintext-inputand 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 @@Step 5 (in practice, input data for applications will typically be streaming continuously into Kafka, rather than being manually entered as we do in this quickstart):
-> bin/kafka-console-producer.sh --broker-list localhost:9092 --topic streams-plaintext-input +> bin/kafka-console-producer.sh --bootstrap-server localhost:9092 --topic streams-plaintext-input all streams lead to kafka@@ -246,7 +246,7 @@Step 5 Your terminal should look as follows:
-> bin/kafka-console-producer.sh --broker-list localhost:9092 --topic streams-plaintext-input +> bin/kafka-console-producer.sh --bootstrap-server localhost:9092 --topic streams-plaintext-input all streams lead to kafka hello kafka streams@@ -279,7 +279,7 @@Step 5 Let's enter one final input text line "join kafka summit" and hit <RETURN> in the console producer to the input topic streams-plaintext-input before we wrap up this quickstart:
-> bin/kafka-console-producer.sh --broker-list localhost:9092 --topic streams-plaintext-input +> bin/kafka-console-producer.sh --bootstrap-server localhost:9092 --topic streams-plaintext-input all streams lead to kafka hello kafka streams join kafka summit diff --git a/tools/src/main/java/org/apache/kafka/tools/VerifiableConsumer.java b/tools/src/main/java/org/apache/kafka/tools/VerifiableConsumer.java index f34b9e22d2633..b65eb9a2a7d13 100644 --- a/tools/src/main/java/org/apache/kafka/tools/VerifiableConsumer.java +++ b/tools/src/main/java/org/apache/kafka/tools/VerifiableConsumer.java @@ -28,6 +28,7 @@ import net.sourceforge.argparse4j.ArgumentParsers; import net.sourceforge.argparse4j.inf.ArgumentParser; import net.sourceforge.argparse4j.inf.ArgumentParserException; +import net.sourceforge.argparse4j.inf.MutuallyExclusiveGroup; import net.sourceforge.argparse4j.inf.Namespace; import org.apache.kafka.clients.consumer.ConsumerConfig; import org.apache.kafka.clients.consumer.ConsumerRebalanceListener; @@ -504,14 +505,23 @@ private static ArgumentParser argParser() { .newArgumentParser("verifiable-consumer") .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."); - - parser.addArgument("--broker-list") + MutuallyExclusiveGroup connectionGroup = parser.addMutuallyExclusiveGroup("Connection Group") + .description("Group of arguments for connection to brokers") + .required(true); + connectionGroup.addArgument("--bootstrap-server") .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) .metavar("HOST1:PORT1[,HOST2:PORT2[...]]") .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") .action(store()) @@ -598,6 +608,7 @@ public static VerifiableConsumer createFromArgs(ArgumentParser parser, String[] boolean useAutoCommit = res.getBoolean("useAutoCommit"); String configFile = res.getString("consumer.config"); + String brokerHostandPort = null; Properties consumerProps = new Properties(); if (configFile != null) { @@ -614,7 +625,18 @@ public static VerifiableConsumer createFromArgs(ArgumentParser parser, String[] if (groupInstanceId != null) { 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.AUTO_OFFSET_RESET_CONFIG, res.getString("resetPolicy")); consumerProps.put(ConsumerConfig.SESSION_TIMEOUT_MS_CONFIG, Integer.toString(res.getInt("sessionTimeout"))); @@ -643,7 +665,6 @@ public static void main(String[] args) { parser.printHelp(); Exit.exit(0); } - try { final VerifiableConsumer consumer = createFromArgs(parser, args); Exit.addShutdownHook("verifiable-consumer-shutdown-hook", () -> consumer.close()); diff --git a/tools/src/main/java/org/apache/kafka/tools/VerifiableProducer.java b/tools/src/main/java/org/apache/kafka/tools/VerifiableProducer.java index 7dbd215161905..befdfddf3f79c 100644 --- a/tools/src/main/java/org/apache/kafka/tools/VerifiableProducer.java +++ b/tools/src/main/java/org/apache/kafka/tools/VerifiableProducer.java @@ -24,6 +24,7 @@ import net.sourceforge.argparse4j.ArgumentParsers; import net.sourceforge.argparse4j.inf.ArgumentParser; import net.sourceforge.argparse4j.inf.ArgumentParserException; +import net.sourceforge.argparse4j.inf.MutuallyExclusiveGroup; import net.sourceforge.argparse4j.inf.Namespace; import org.apache.kafka.clients.producer.Callback; @@ -119,14 +120,24 @@ private static ArgumentParser argParser() { .type(String.class) .metavar("TOPIC") .help("Produce messages to this topic."); + MutuallyExclusiveGroup connectionGroup = parser.addMutuallyExclusiveGroup("Connection Group") + .description("Group of arguments for connection to brokers") + .required(true); + connectionGroup.addArgument("--bootstrap-server") + .action(store()) + .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,..."); - parser.addArgument("--broker-list") + connectionGroup.addArgument("--broker-list") .action(store()) - .required(true) + .required(false) .type(String.class) .metavar("HOST1:PORT1[,HOST2:PORT2[...]]") .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") .action(store()) @@ -222,7 +233,16 @@ public static VerifiableProducer createFromArgs(ArgumentParser parser, String[] createTime = null; 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, "org.apache.kafka.common.serialization.StringSerializer"); producerProps.put(ProducerConfig.VALUE_SERIALIZER_CLASS_CONFIG,