This is an automated email from the ASF dual-hosted git repository.
huxihx pushed a commit to branch trunk
in repository https://gitbox.apache.org/repos/asf/kafka.git
The following commit(s) were added to refs/heads/trunk by this push:
new 206b69a KAFKA-10305: Print usage when parsing fails for
ConsumerPerformance (#9071)
206b69a is described below
commit 206b69af8b1267a5022d22d1535bd1db02702be1
Author: huxi <[email protected]>
AuthorDate: Sat Jul 25 19:04:16 2020 +0800
KAFKA-10305: Print usage when parsing fails for ConsumerPerformance (#9071)
https://issues.apache.org/jira/browse/KAFKA-10305
When `kafka-consumer-perf-test.sh` is executed without required options or
no options at all, only the error message is displayed. It's better off showing
the usage as well like what we did for kafka-console-producer.sh.
---
core/src/main/scala/kafka/tools/ConsumerPerformance.scala | 8 +++++++-
.../scala/unit/kafka/tools/ConsumerPerformanceTest.scala | 14 +++++++++-----
2 files changed, 16 insertions(+), 6 deletions(-)
diff --git a/core/src/main/scala/kafka/tools/ConsumerPerformance.scala
b/core/src/main/scala/kafka/tools/ConsumerPerformance.scala
index d2cd20d..a329b4a 100644
--- a/core/src/main/scala/kafka/tools/ConsumerPerformance.scala
+++ b/core/src/main/scala/kafka/tools/ConsumerPerformance.scala
@@ -24,6 +24,7 @@ import java.util.concurrent.atomic.AtomicLong
import java.util.{Properties, Random}
import com.typesafe.scalalogging.LazyLogging
+import joptsimple.OptionException
import kafka.utils.{CommandLineUtils, ToolsUtils}
import org.apache.kafka.clients.consumer.{ConsumerRebalanceListener,
KafkaConsumer}
import org.apache.kafka.common.serialization.ByteArrayDeserializer
@@ -255,7 +256,12 @@ object ConsumerPerformance extends LazyLogging {
.ofType(classOf[Long])
.defaultsTo(10000)
- options = parser.parse(args: _*)
+ try
+ options = parser.parse(args: _*)
+ catch {
+ case e: OptionException =>
+ CommandLineUtils.printUsageAndDie(parser, e.getMessage)
+ }
if(options.has(numThreadsOpt) || options.has(numFetchersOpt))
println("WARNING: option [threads] and [num-fetch-threads] have been
deprecated and will be ignored by the test")
diff --git a/core/src/test/scala/unit/kafka/tools/ConsumerPerformanceTest.scala
b/core/src/test/scala/unit/kafka/tools/ConsumerPerformanceTest.scala
index 530d191..94b732d 100644
--- a/core/src/test/scala/unit/kafka/tools/ConsumerPerformanceTest.scala
+++ b/core/src/test/scala/unit/kafka/tools/ConsumerPerformanceTest.scala
@@ -20,7 +20,7 @@ package kafka.tools
import java.io.ByteArrayOutputStream
import java.text.SimpleDateFormat
-import joptsimple.OptionException
+import kafka.utils.Exit
import org.junit.Assert.assertEquals
import org.junit.Test
@@ -97,8 +97,9 @@ class ConsumerPerformanceTest {
assertEquals(10, config.numMessages)
}
- @Test(expected = classOf[OptionException])
+ @Test(expected = classOf[IllegalArgumentException])
def testConfigWithUnrecognizedOption(): Unit = {
+ Exit.setExitProcedure((_, message) => throw new
IllegalArgumentException(message.orNull))
//Given
val args: Array[String] = Array(
"--broker-list", "localhost:9092",
@@ -106,9 +107,12 @@ class ConsumerPerformanceTest {
"--messages", "10",
"--new-consumer"
)
-
- //When
- new ConsumerPerformance.ConsumerPerfConfig(args)
+ try {
+ //When
+ new ConsumerPerformance.ConsumerPerfConfig(args)
+ } finally {
+ Exit.resetExitProcedure()
+ }
}
private def testHeaderMatchContent(detailed: Boolean,
expectedOutputLineCount: Int, fun: () => Unit): Unit = {