mjsax commented on code in PR #20293:
URL: https://github.com/apache/kafka/pull/20293#discussion_r2374350694


##########
tools/src/main/java/org/apache/kafka/tools/streams/StreamsGroupCommand.java:
##########
@@ -83,27 +85,46 @@ public class StreamsGroupCommand {
     static final String MISSING_COLUMN_VALUE = "-";
 
     public static void main(String[] args) {
-        StreamsGroupCommandOptions opts = new StreamsGroupCommandOptions(args);
+        Exit.exit(execute(args));
+    }
+
+    public static int execute(String[] args) {
+        StreamsGroupCommandOptions opts = null;
+        int exitCode = 0;
         try {
-            opts.checkArgs();
+            opts = new StreamsGroupCommandOptions(args);
+            Objects.requireNonNull(opts).checkArgs();
             // should have exactly one action
             long numberOfActions = Stream.of(
-                opts.listOpt,
-                opts.describeOpt,
-                opts.resetOffsetsOpt,
-                opts.deleteOpt,
-                opts.deleteOffsetsOpt
+                    opts.listOpt,
+                    opts.describeOpt,
+                    opts.resetOffsetsOpt,
+                    opts.deleteOpt,
+                    opts.deleteOffsetsOpt

Review Comment:
   nit: avoid unnecessary re-formatting (might be your IDE auto-formatting; for 
this case, either disable auto-formatting, or adjust the formatting rules).



##########
tools/src/test/java/org/apache/kafka/tools/streams/DeleteStreamsGroupOffsetTest.java:
##########
@@ -132,6 +132,7 @@ public void testDeleteOffsetsNonExistingGroup() {
             Map.Entry<Errors, Map<TopicPartition, Throwable>> res = 
service.deleteOffsets();
             assertEquals(Errors.GROUP_ID_NOT_FOUND, res.getKey());
         }
+        assertEquals(0, StreamsGroupCommand.execute(args));

Review Comment:
   Not sure why we need to add this?
   
   This test does not verify `StreamsGroupCommand` itself, so it seems we don't 
need any change to this test?



##########
tools/src/test/java/org/apache/kafka/tools/streams/DeleteStreamsGroupTest.java:
##########
@@ -129,6 +129,7 @@ public static void closeCluster() {
     public void testDeleteWithUnrecognizedOption() {
         final String[] args = new String[]{"--unrecognized-option", 
"--bootstrap-server", bootstrapServers, "--delete", "--all-groups"};
         assertThrows(OptionException.class, () -> 
getStreamsGroupService(args));
+        assertEquals(1, StreamsGroupCommand.execute(args));

Review Comment:
   Same. Not sure why we need to add this?
   
   This test does not verify `StreamsGroupCommand` itself, so it seems we don't 
need any change to this test?



##########
tools/src/test/java/org/apache/kafka/tools/streams/ListStreamsGroupTest.java:
##########
@@ -22,6 +22,7 @@
 import org.apache.kafka.common.GroupState;
 import org.apache.kafka.common.GroupType;
 import org.apache.kafka.common.serialization.Serdes;
+import org.apache.kafka.common.utils.Exit;

Review Comment:
   Same



##########
tools/src/main/java/org/apache/kafka/tools/streams/StreamsGroupCommand.java:
##########
@@ -83,27 +85,46 @@ public class StreamsGroupCommand {
     static final String MISSING_COLUMN_VALUE = "-";
 
     public static void main(String[] args) {
-        StreamsGroupCommandOptions opts = new StreamsGroupCommandOptions(args);
+        Exit.exit(execute(args));
+    }
+
+    public static int execute(String[] args) {
+        StreamsGroupCommandOptions opts = null;
+        int exitCode = 0;
         try {
-            opts.checkArgs();
+            opts = new StreamsGroupCommandOptions(args);
+            Objects.requireNonNull(opts).checkArgs();

Review Comment:
   We just assign `opts = ...` the line before. Using `requireNonNull()` seem 
unnecessary, as we _know_ it won't be `null`.



##########
tools/src/test/java/org/apache/kafka/tools/streams/ResetStreamsGroupOffsetTest.java:
##########
@@ -39,9 +39,11 @@
 import org.apache.kafka.streams.kstream.KTable;
 import org.apache.kafka.streams.kstream.Materialized;
 import org.apache.kafka.test.TestUtils;
+import org.apache.kafka.tools.ToolsTestUtils;

Review Comment:
   Same



##########
tools/src/test/java/org/apache/kafka/tools/streams/DescribeStreamsGroupTest.java:
##########
@@ -101,6 +101,7 @@ public static void closeCluster() {
     public void testDescribeWithUnrecognizedOption() {
         String[] args = new String[]{"--unrecognized-option", 
"--bootstrap-server", bootstrapServers, "--describe", "--group", APP_ID};
         assertThrows(OptionException.class, () -> 
getStreamsGroupService(args));
+        assertEquals(1, StreamsGroupCommand.execute(args));

Review Comment:
   Same



##########
tools/src/main/java/org/apache/kafka/tools/streams/StreamsGroupCommand.java:
##########
@@ -83,27 +85,46 @@ public class StreamsGroupCommand {
     static final String MISSING_COLUMN_VALUE = "-";
 
     public static void main(String[] args) {
-        StreamsGroupCommandOptions opts = new StreamsGroupCommandOptions(args);
+        Exit.exit(execute(args));
+    }
+
+    public static int execute(String[] args) {
+        StreamsGroupCommandOptions opts = null;
+        int exitCode = 0;
         try {
-            opts.checkArgs();
+            opts = new StreamsGroupCommandOptions(args);
+            Objects.requireNonNull(opts).checkArgs();
             // should have exactly one action
             long numberOfActions = Stream.of(
-                opts.listOpt,
-                opts.describeOpt,
-                opts.resetOffsetsOpt,
-                opts.deleteOpt,
-                opts.deleteOffsetsOpt
+                    opts.listOpt,
+                    opts.describeOpt,
+                    opts.resetOffsetsOpt,
+                    opts.deleteOpt,
+                    opts.deleteOffsetsOpt
             ).filter(opts.options::has).count();
             if (numberOfActions != 1)
-                CommandLineUtils.printUsageAndExit(opts.parser, "Command must 
include exactly one action: --list, --describe, --delete, --reset-offsets, or 
--delete-offsets.");
+                throw new IllegalArgumentException("Command must include 
exactly one action: --list, --describe, --delete, --reset-offsets, or 
--delete-offsets.");
 
             run(opts);
-        } catch (OptionException e) {
-            CommandLineUtils.printUsageAndExit(opts.parser, e.getMessage());
+        } catch (IllegalArgumentException | OptionException e) {
+            System.err.println(e.getMessage());
+            if (opts != null) {
+                try {
+                    opts.parser.printHelpOn(System.err);
+                } catch (IOException ex) {
+                    throw new RuntimeException(ex);
+                }
+            }
+            exitCode = 1;
+        } catch (Throwable e) {
+            printError("Executing streams group command failed due to " + 
e.getMessage(), Optional.of(e));
+            exitCode = 1;
         }
+
+        return exitCode;
     }
 
-    public static void run(StreamsGroupCommandOptions opts) {
+    public static void run(StreamsGroupCommandOptions opts) throws 
IllegalArgumentException, ExecutionException, InterruptedException {

Review Comment:
   I think we can omit `IllegalArgumentException`, as it's a `RuntimeException` 
?



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: [email protected]

For queries about this service, please contact Infrastructure at:
[email protected]

Reply via email to