[ 
https://issues.apache.org/jira/browse/FLINK-10399?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=16641964#comment-16641964
 ] 

ASF GitHub Bot commented on FLINK-10399:
----------------------------------------

GJL commented on a change in pull request #6737: [FLINK-10399] Refactor 
ParameterTool#fromArgs
URL: https://github.com/apache/flink/pull/6737#discussion_r223391917
 
 

 ##########
 File path: 
flink-java/src/main/java/org/apache/flink/api/java/utils/ParameterTool.java
 ##########
 @@ -68,81 +68,46 @@
         * @return A {@link ParameterTool}
         */
        public static ParameterTool fromArgs(String[] args) {
-               Map<String, String> map = new HashMap<String, 
String>(args.length / 2);
-
-               String key = null;
-               String value = null;
-               boolean expectValue = false;
-               for (String arg : args) {
-                       // check for -- argument
-                       if (arg.startsWith("--")) {
-                               if (expectValue) {
-                                       // we got into a new key, even though 
we were a value --> current key is one without value
-                                       if (value != null) {
-                                               throw new 
IllegalStateException("Unexpected state");
-                                       }
-                                       map.put(key, NO_VALUE_KEY);
-                                       // key will be overwritten in the next 
step
-                               }
-                               key = arg.substring(2);
-                               expectValue = true;
-                       } // check for - argument
-                       else if (arg.startsWith("-")) {
-                               // we are waiting for a value, so this is a - 
prefixed value (negative number)
-                               if (expectValue) {
-
-                                       if (NumberUtils.isNumber(arg)) {
-                                               // negative number
-                                               value = arg;
-                                               expectValue = false;
-                                       } else {
-                                               if (value != null) {
-                                                       throw new 
IllegalStateException("Unexpected state");
-                                               }
-                                               // We waited for a value but 
found a new key. So the previous key doesnt have a value.
-                                               map.put(key, NO_VALUE_KEY);
-                                               key = arg.substring(1);
-                                               expectValue = true;
-                                       }
-                               } else {
-                                       // we are not waiting for a value, so 
its an argument
-                                       key = arg.substring(1);
-                                       expectValue = true;
-                               }
+               final Map<String, String> map = new HashMap<>(args.length / 2);
+
+               final String errorMessage = "Error parsing arguments '%s' on 
'%s'.";
+
+               int i = 0;
+               while (i < args.length) {
+                       final String key;
+
+                       if (args[i].startsWith("--")) {
+                               key = args[i].substring(2);
+                       } else if (args[i].startsWith("-")) {
+                               key = args[i].substring(1);
                        } else {
-                               if (expectValue) {
-                                       value = arg;
-                                       expectValue = false;
-                               } else {
-                                       throw new RuntimeException("Error 
parsing arguments '" + Arrays.toString(args) + "' on '" + arg + "'. Unexpected 
value. Please prefix values with -- or -.");
-                               }
+                               throw new IllegalArgumentException(
+                                       String.format(errorMessage + " Please 
prefix keys with -- or -.",
+                                               Arrays.toString(args), 
args[i]));
                        }
 
-                       if (value == null && key == null) {
-                               throw new IllegalStateException("Value and key 
can not be null at the same time");
-                       }
-                       if (key != null && value == null && !expectValue) {
-                               throw new IllegalStateException("Value expected 
but flag not set");
-                       }
-                       if (key != null && value != null) {
-                               map.put(key, value);
-                               key = null;
-                               value = null;
-                               expectValue = false;
-                       }
-                       if (key != null && key.length() == 0) {
-                               throw new IllegalArgumentException("The input " 
+ Arrays.toString(args) + " contains an empty argument");
+                       if (key.length() == 0) {
+                               throw new IllegalArgumentException(
+                                       String.format(errorMessage + " Keys 
should never be empty",
+                                               Arrays.toString(args), 
args[i]));
                        }
 
-                       if (key != null && !expectValue) {
+                       i += 1; // try to find the value
+
+                       if (i >= args.length) {
+                               map.put(key, NO_VALUE_KEY);
+                       } else if (NumberUtils.isNumber(args[i])) {
+                               map.put(key, args[i]);
+                               i += 1;
+                       } else if (args[i].startsWith("--")) {
+                               map.put(key, NO_VALUE_KEY);
+                       } else if (args[i].startsWith("-")) { // negate number 
guarded by precondition
 
 Review comment:
   It's not clear to me what the comment means, and you have two branches that 
are equal. Maybe rewrite to:
   
   ```
   else if (args[i].startsWith("--") || args[i].startsWith("-")) {
        // the argument cannot be a negative number because we checked earlier
        // -> the next argument is a parameter name
        map.put(key, NO_VALUE_KEY);
   }
   ```

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
us...@infra.apache.org


> Refactor ParameterTool#fromArgs
> -------------------------------
>
>                 Key: FLINK-10399
>                 URL: https://issues.apache.org/jira/browse/FLINK-10399
>             Project: Flink
>          Issue Type: Improvement
>          Components: Client
>    Affects Versions: 1.7.0
>            Reporter: TisonKun
>            Assignee: TisonKun
>            Priority: Major
>              Labels: pull-request-available
>             Fix For: 1.7.0
>
>
> {{ParameterTool#fromArgs}} uses a weird implement which flink developer would 
> fail to parse it fast.
> The main problem is that, when parse args, we always try to get a key-value 
> pair, but the implement iterate by a {{for}} loop, thus introduce weird 
> flag/mutable variable and branches.



--
This message was sent by Atlassian JIRA
(v7.6.3#76005)

Reply via email to