You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@flink.apache.org by gy...@apache.org on 2022/12/29 14:53:14 UTC

[flink] branch master updated: [FLINK-30429][client] Fix IllegalArgumentException when no argument in flink executable

This is an automated email from the ASF dual-hosted git repository.

gyfora pushed a commit to branch master
in repository https://gitbox.apache.org/repos/asf/flink.git


The following commit(s) were added to refs/heads/master by this push:
     new 259c7a4776a [FLINK-30429][client] Fix IllegalArgumentException when no argument in flink executable
259c7a4776a is described below

commit 259c7a4776af4d9d7f174f871fb2d4145787f794
Author: Gabor Somogyi <ga...@apple.com>
AuthorDate: Thu Dec 15 15:54:57 2022 +0100

    [FLINK-30429][client] Fix IllegalArgumentException when no argument in flink executable
---
 .../org/apache/flink/client/cli/CliFrontend.java   | 22 +++++++++++++---
 .../apache/flink/client/cli/CliFrontendITCase.java | 30 ++++++++++++++++++++++
 2 files changed, 48 insertions(+), 4 deletions(-)

diff --git a/flink-clients/src/main/java/org/apache/flink/client/cli/CliFrontend.java b/flink-clients/src/main/java/org/apache/flink/client/cli/CliFrontend.java
index c4a3b85b58d..2b70407b7a1 100644
--- a/flink-clients/src/main/java/org/apache/flink/client/cli/CliFrontend.java
+++ b/flink-clients/src/main/java/org/apache/flink/client/cli/CliFrontend.java
@@ -83,6 +83,7 @@ import java.util.concurrent.TimeUnit;
 import java.util.stream.Collectors;
 import java.util.stream.Stream;
 
+import static java.lang.Math.min;
 import static org.apache.flink.client.cli.CliFrontendParser.HELP_OPTION;
 import static org.apache.flink.util.Preconditions.checkNotNull;
 
@@ -91,6 +92,8 @@ public class CliFrontend {
 
     private static final Logger LOG = LoggerFactory.getLogger(CliFrontend.class);
 
+    private static final int INITIAL_RET_CODE = 31;
+
     // actions
     private static final String ACTION_RUN = "run";
     private static final String ACTION_RUN_APPLICATION = "run-application";
@@ -1149,6 +1152,16 @@ public class CliFrontend {
 
     /** Submits the job based on the arguments. */
     public static void main(final String[] args) {
+        int retCode = INITIAL_RET_CODE;
+        try {
+            retCode = mainInternal(args);
+        } finally {
+            System.exit(retCode);
+        }
+    }
+
+    @VisibleForTesting
+    static int mainInternal(final String[] args) {
         EnvironmentInformation.logEnvironmentInfo(LOG, "Command Line Client", args);
 
         // 1. find the configuration directory
@@ -1162,12 +1175,14 @@ public class CliFrontend {
         final List<CustomCommandLine> customCommandLines =
                 loadCustomCommandLines(configuration, configurationDirectory);
 
-        int retCode = 31;
+        int retCode = INITIAL_RET_CODE;
         try {
             final CliFrontend cli = new CliFrontend(configuration, customCommandLines);
             CommandLine commandLine =
                     cli.getCommandLine(
-                            new Options(), Arrays.copyOfRange(args, 1, args.length), true);
+                            new Options(),
+                            Arrays.copyOfRange(args, min(args.length, 1), args.length),
+                            true);
             Configuration securityConfig = new Configuration(cli.configuration);
             DynamicPropertiesUtil.encodeDynamicProperties(commandLine, securityConfig);
             SecurityUtils.install(new SecurityConfiguration(securityConfig));
@@ -1177,9 +1192,8 @@ public class CliFrontend {
                     ExceptionUtils.stripException(t, UndeclaredThrowableException.class);
             LOG.error("Fatal error while running command line interface.", strippedThrowable);
             strippedThrowable.printStackTrace();
-        } finally {
-            System.exit(retCode);
         }
+        return retCode;
     }
 
     // --------------------------------------------------------------------------------------------
diff --git a/flink-clients/src/test/java/org/apache/flink/client/cli/CliFrontendITCase.java b/flink-clients/src/test/java/org/apache/flink/client/cli/CliFrontendITCase.java
index 309dbbdc0d6..dbffe57783b 100644
--- a/flink-clients/src/test/java/org/apache/flink/client/cli/CliFrontendITCase.java
+++ b/flink-clients/src/test/java/org/apache/flink/client/cli/CliFrontendITCase.java
@@ -21,17 +21,26 @@ package org.apache.flink.client.cli;
 import org.apache.flink.client.deployment.executors.LocalExecutor;
 import org.apache.flink.configuration.Configuration;
 import org.apache.flink.configuration.PipelineOptions;
+import org.apache.flink.core.testutils.CommonTestUtils;
 import org.apache.flink.streaming.api.environment.StreamExecutionEnvironment;
 
 import org.junit.jupiter.api.AfterEach;
 import org.junit.jupiter.api.BeforeEach;
 import org.junit.jupiter.api.Test;
+import org.junit.jupiter.api.io.TempDir;
 
 import java.io.ByteArrayOutputStream;
+import java.io.File;
+import java.io.IOException;
 import java.io.PrintStream;
+import java.nio.file.Files;
+import java.nio.file.Path;
 import java.time.Duration;
 import java.util.Collections;
+import java.util.HashMap;
+import java.util.Map;
 
+import static org.apache.flink.configuration.ConfigConstants.ENV_FLINK_CONF_DIR;
 import static org.assertj.core.api.Assertions.assertThat;
 
 /** Integration tests for {@link CliFrontend}. */
@@ -99,6 +108,27 @@ class CliFrontendITCase {
         assertThat(getStdoutString()).contains("Watermark interval is 142");
     }
 
+    @Test
+    void mainShouldPrintHelpWithoutArgs(@TempDir Path tempFolder) throws Exception {
+        Map<String, String> originalEnv = System.getenv();
+        try {
+            File confFolder = Files.createTempDirectory(tempFolder, "conf").toFile();
+            File confYaml = new File(confFolder, "flink-conf.yaml");
+            if (!confYaml.createNewFile()) {
+                throw new IOException("Can't create testing flink-conf.yaml file.");
+            }
+
+            Map<String, String> map = new HashMap<>(System.getenv());
+            map.put(ENV_FLINK_CONF_DIR, confFolder.getAbsolutePath());
+            CommonTestUtils.setEnv(map);
+
+            assertThat(CliFrontend.mainInternal(new String[0])).isEqualTo(1);
+            assertThat(getStdoutString()).contains("The following actions are available");
+        } finally {
+            CommonTestUtils.setEnv(originalEnv);
+        }
+    }
+
     /**
      * Testing job that the watermark interval from the {@link
      * org.apache.flink.api.common.ExecutionConfig}.