KAFKA-16408 kafka-get-offsets / GetOffsetShell doesn't handle --version or --help (#15583)

Reviewers: Chia-Ping Tsai <chia7712@gmail.com>
This commit is contained in:
Dmitry Werner 2024-03-25 09:12:23 +05:00 committed by GitHub
parent bf9a27fefd
commit 0434c29e58
No known key found for this signature in database
GPG Key ID: B5690EEEBB952194
2 changed files with 17 additions and 1 deletions

View File

@ -59,6 +59,7 @@ import java.util.regex.Pattern;
import java.util.stream.Collectors; import java.util.stream.Collectors;
public class GetOffsetShell { public class GetOffsetShell {
static final String USAGE_TEXT = "An interactive shell for getting topic-partition offsets.";
private static final Pattern TOPIC_PARTITION_PATTERN = Pattern.compile("([^:,]*)(?::(?:([0-9]*)|(?:([0-9]*)-([0-9]*))))?"); private static final Pattern TOPIC_PARTITION_PATTERN = Pattern.compile("([^:,]*)(?::(?:([0-9]*)|(?:([0-9]*)-([0-9]*))))?");
public static void main(String... args) { public static void main(String... args) {
@ -142,7 +143,7 @@ public class GetOffsetShell {
excludeInternalTopicsOpt = parser.accepts("exclude-internal-topics", "By default, internal topics are included. If specified, internal topics are excluded."); excludeInternalTopicsOpt = parser.accepts("exclude-internal-topics", "By default, internal topics are included. If specified, internal topics are excluded.");
if (args.length == 0) { if (args.length == 0) {
CommandLineUtils.printUsageAndExit(parser, "An interactive shell for getting topic-partition offsets."); CommandLineUtils.printUsageAndExit(parser, USAGE_TEXT);
} }
try { try {
@ -157,6 +158,8 @@ public class GetOffsetShell {
effectiveBrokerListOpt = brokerListOpt; effectiveBrokerListOpt = brokerListOpt;
} }
CommandLineUtils.maybePrintHelpOrVersion(this, USAGE_TEXT);
CommandLineUtils.checkRequiredArgs(parser, options, effectiveBrokerListOpt); CommandLineUtils.checkRequiredArgs(parser, options, effectiveBrokerListOpt);
String brokerList = options.valueOf(effectiveBrokerListOpt); String brokerList = options.valueOf(effectiveBrokerListOpt);

View File

@ -29,6 +29,7 @@ import org.apache.kafka.clients.producer.KafkaProducer;
import org.apache.kafka.clients.producer.ProducerConfig; import org.apache.kafka.clients.producer.ProducerConfig;
import org.apache.kafka.clients.producer.ProducerRecord; import org.apache.kafka.clients.producer.ProducerRecord;
import org.apache.kafka.common.serialization.StringSerializer; import org.apache.kafka.common.serialization.StringSerializer;
import org.apache.kafka.common.utils.AppInfoParser;
import org.apache.kafka.common.utils.Exit; import org.apache.kafka.common.utils.Exit;
import org.junit.jupiter.api.BeforeEach; import org.junit.jupiter.api.BeforeEach;
import org.junit.jupiter.api.Tag; import org.junit.jupiter.api.Tag;
@ -317,6 +318,18 @@ public class GetOffsetShellTest {
assertExitCodeIsOne("--topic-partitions", "__consumer_offsets", "--partitions", "0"); assertExitCodeIsOne("--topic-partitions", "__consumer_offsets", "--partitions", "0");
} }
@ClusterTest
public void testPrintHelp() {
String out = ToolsTestUtils.captureStandardErr(() -> GetOffsetShell.mainNoExit("--help"));
assertTrue(out.startsWith(GetOffsetShell.USAGE_TEXT));
}
@ClusterTest
public void testPrintVersion() {
String out = ToolsTestUtils.captureStandardOut(() -> GetOffsetShell.mainNoExit("--version"));
assertEquals(AppInfoParser.getVersion(), out);
}
private void assertExitCodeIsOne(String... args) { private void assertExitCodeIsOne(String... args) {
final int[] exitStatus = new int[1]; final int[] exitStatus = new int[1];