You are viewing a plain text version of this content. The canonical link for it is here.
Posted to issues@beam.apache.org by "ASF GitHub Bot (Jira)" <ji...@apache.org> on 2022/03/01 18:51:00 UTC

[jira] [Work logged] (BEAM-13709) PipelineOptions() and from_dictionary parsing use_public_ips and no_use_public_ips differently

     [ https://issues.apache.org/jira/browse/BEAM-13709?focusedWorklogId=734864&page=com.atlassian.jira.plugin.system.issuetabpanels:worklog-tabpanel#worklog-734864 ]

ASF GitHub Bot logged work on BEAM-13709:
-----------------------------------------

                Author: ASF GitHub Bot
            Created on: 01/Mar/22 18:50
            Start Date: 01/Mar/22 18:50
    Worklog Time Spent: 10m 
      Work Description: tvalentyn commented on a change in pull request #16929:
URL: https://github.com/apache/beam/pull/16929#discussion_r817002170



##########
File path: sdks/python/apache_beam/options/pipeline_options.py
##########
@@ -180,7 +190,16 @@ def __init__(self, flags=None, **kwargs):
       flags: An iterable of command line arguments to be used. If not specified
         then sys.argv will be used as input for parsing arguments.
 
-      **kwargs: Add overrides for arguments passed in flags.
+      **kwargs: Add overrides for arguments passed in flags. For kwargs,
+                please pass the option names instead of flag names.
+                Option names: These are defined as dest in the
+                parser.add_argument(). Passing flag names like

Review comment:
       revise this as well.

##########
File path: sdks/python/apache_beam/options/pipeline_options.py
##########
@@ -234,12 +262,27 @@ def from_dictionary(cls, options):
 
     Returns:
       A PipelineOptions object representing the given arguments.
+
+    Note: If a boolean flag is True in the dictionary,
+          implicitly the method assumes the boolean flag is
+          specified as a command line argument. If the
+          boolean flag is False, this method simply discards
+          them.
+    Eg: {no_auth: True} is similar to python your_file.py --no_auth
+        {no_auth: False} is similar to python your_file.py.
     """
     flags = []
     for k, v in options.items():
       if isinstance(v, bool):
         if v:
           flags.append('--%s' % k)
+        # capture boolean flags with 3 values
+        # {default=None, True, False}
+        elif k in _STORE_FALSE_OPTIONS_WITH_DIFFERENT_DEST:
+          _LOGGER.warning(

Review comment:
       We don't need the warning, as the method should be supporting this use-case (it's a bug that it currently does not).

##########
File path: sdks/python/apache_beam/options/pipeline_options_test.py
##########
@@ -647,6 +650,60 @@ def test_dataflow_service_options(self):
     self.assertEqual(
         options.get_all_options()['dataflow_service_options'], None)
 
+  def test_options_store_false_with_different_dest(self):
+    parser = _BeamArgumentParser()
+    for cls in PipelineOptions.__subclasses__():
+      cls._add_argparse_args(parser)
+
+    actions = parser._actions.copy()
+    dest_to_options = {}
+    options_diff_dest_store_true = {}
+
+    for i in range(len(actions)):
+      options_name = actions[i].option_strings
+      dest = actions[i].dest
+
+      if isinstance(actions[i].const, bool):
+        for option_name in options_name:

Review comment:
       flag_name / flag_names

##########
File path: sdks/python/apache_beam/options/pipeline_options.py
##########
@@ -55,6 +55,16 @@
 
 _LOGGER = logging.getLogger(__name__)
 
+# These options have no dest and action is store_false in the
+# argparse and default is None. When parsing these options in a dict using
+# PipelineOptions,We either ignore/discard if these options are specified.

Review comment:
       ```suggestion
   # PipelineOptions(), we either ignore/discard if these options are specified.
   ```

##########
File path: sdks/python/apache_beam/options/pipeline_options.py
##########
@@ -234,12 +262,27 @@ def from_dictionary(cls, options):
 
     Returns:
       A PipelineOptions object representing the given arguments.
+
+    Note: If a boolean flag is True in the dictionary,
+          implicitly the method assumes the boolean flag is
+          specified as a command line argument. If the
+          boolean flag is False, this method simply discards
+          them.
+    Eg: {no_auth: True} is similar to python your_file.py --no_auth
+        {no_auth: False} is similar to python your_file.py.
     """
     flags = []
     for k, v in options.items():
       if isinstance(v, bool):

Review comment:
       Let's move the `Note:` here.

##########
File path: sdks/python/apache_beam/options/pipeline_options_test.py
##########
@@ -647,6 +650,60 @@ def test_dataflow_service_options(self):
     self.assertEqual(
         options.get_all_options()['dataflow_service_options'], None)
 
+  def test_options_store_false_with_different_dest(self):
+    parser = _BeamArgumentParser()
+    for cls in PipelineOptions.__subclasses__():
+      cls._add_argparse_args(parser)
+
+    actions = parser._actions.copy()
+    dest_to_options = {}
+    options_diff_dest_store_true = {}
+
+    for i in range(len(actions)):
+      options_name = actions[i].option_strings

Review comment:
       s/options_name/flag_names

##########
File path: sdks/python/apache_beam/options/pipeline_options.py
##########
@@ -212,6 +231,15 @@ def __init__(self, flags=None, **kwargs):
     # Users access this dictionary store via __getattr__ / __setattr__ methods.
     self._all_options = kwargs
 
+    if self.__class__ != PipelineOptions:
+      _invalid_options = {}
+      for option_name, option_value in self._all_options.items():
+        if option_name not in self._visible_option_list():
+          _invalid_options[option_name] = option_value
+
+      if _invalid_options:
+        _LOGGER.warning("Discarding invalid overrides: %s", _invalid_options)

Review comment:
       sounds like this warning will be still printed in get_all_options() call at pipeline submission, so we can omit it here.

##########
File path: sdks/python/apache_beam/options/pipeline_options.py
##########
@@ -353,6 +396,15 @@ def view_as(self, cls):
 
     """
     view = cls(self._flags)
+
+    _invalid_options = {}
+    for option_name, option_value in self._all_options.items():
+      if option_name not in self._visible_option_list():
+        _invalid_options[option_name] = option_value
+
+    if _invalid_options:
+      _LOGGER.warning("Discarding invalid overrides: %s", _invalid_options)

Review comment:
       We could also omit this. 

##########
File path: sdks/python/apache_beam/options/pipeline_options_test.py
##########
@@ -647,6 +650,60 @@ def test_dataflow_service_options(self):
     self.assertEqual(
         options.get_all_options()['dataflow_service_options'], None)
 
+  def test_options_store_false_with_different_dest(self):
+    parser = _BeamArgumentParser()
+    for cls in PipelineOptions.__subclasses__():
+      cls._add_argparse_args(parser)
+
+    actions = parser._actions.copy()
+    dest_to_options = {}
+    options_diff_dest_store_true = {}
+
+    for i in range(len(actions)):
+      options_name = actions[i].option_strings
+      dest = actions[i].dest

Review comment:
       consider: s/dest/option_name

##########
File path: sdks/python/apache_beam/options/pipeline_options_test.py
##########
@@ -647,6 +650,60 @@ def test_dataflow_service_options(self):
     self.assertEqual(
         options.get_all_options()['dataflow_service_options'], None)
 
+  def test_options_store_false_with_different_dest(self):
+    parser = _BeamArgumentParser()
+    for cls in PipelineOptions.__subclasses__():
+      cls._add_argparse_args(parser)
+
+    actions = parser._actions.copy()
+    dest_to_options = {}

Review comment:
       dest_to_flags

##########
File path: sdks/python/apache_beam/options/pipeline_options.py
##########
@@ -234,12 +262,27 @@ def from_dictionary(cls, options):
 
     Returns:
       A PipelineOptions object representing the given arguments.
+
+    Note: If a boolean flag is True in the dictionary,
+          implicitly the method assumes the boolean flag is
+          specified as a command line argument. If the
+          boolean flag is False, this method simply discards
+          them.
+    Eg: {no_auth: True} is similar to python your_file.py --no_auth
+        {no_auth: False} is similar to python your_file.py.
     """
     flags = []
     for k, v in options.items():
       if isinstance(v, bool):
         if v:
           flags.append('--%s' % k)
+        # capture boolean flags with 3 values
+        # {default=None, True, False}
+        elif k in _STORE_FALSE_OPTIONS_WITH_DIFFERENT_DEST:
+          _LOGGER.warning(
+              "Instead of %s=%s, please provide %s=%s" %
+              (k, v, _STORE_FALSE_OPTIONS_WITH_DIFFERENT_DEST[k], True))
+          flags.append('--%s' % _STORE_FALSE_OPTIONS_WITH_DIFFERENT_DEST[k])

Review comment:
       suggestion for better readability:
   ```
   flag_that_disables_the_option = _STORE_FALSE_OPTIONS_WITH_DIFFERENT_DEST[k]
   flags.append('--%s' % flag_that_disables_the_option)
   ```

##########
File path: sdks/python/apache_beam/options/pipeline_options.py
##########
@@ -55,6 +55,16 @@
 
 _LOGGER = logging.getLogger(__name__)
 
+# These options have no dest and action is store_false in the
+# argparse and default is None. When parsing these options in a dict using
+# PipelineOptions,We either ignore/discard if these options are specified.

Review comment:
       as discussed, please revise the comment.

##########
File path: sdks/python/apache_beam/options/pipeline_options_test.py
##########
@@ -647,6 +650,60 @@ def test_dataflow_service_options(self):
     self.assertEqual(
         options.get_all_options()['dataflow_service_options'], None)
 
+  def test_options_store_false_with_different_dest(self):
+    parser = _BeamArgumentParser()
+    for cls in PipelineOptions.__subclasses__():
+      cls._add_argparse_args(parser)
+
+    actions = parser._actions.copy()
+    dest_to_options = {}
+    options_diff_dest_store_true = {}
+
+    for i in range(len(actions)):
+      options_name = actions[i].option_strings
+      dest = actions[i].dest
+
+      if isinstance(actions[i].const, bool):
+        for option_name in options_name:
+          option_name = option_name.strip(
+              '--') if '--' in option_name else option_name
+          if option_name != dest:
+            # Capture flags which has store_action=True and has a
+            # different dest. This behavior would be confusing.
+            if actions[i].const:
+              options_diff_dest_store_true[option_name] = dest
+              continue
+            # check the flags like no_use_public_ips
+            # default is None, action is {True, False}
+            if actions[i].default is None:
+              dest_to_options[dest] = option_name
+
+    assert len(options_diff_dest_store_true) == 0, (
+      _LOGGER.error("There should be no flags that have a dest "
+                    "different from flag name and action as "
+                    "store_true. It would be confusing "
+                    "to the user. Please specify the dest as the "
+                    "flag_name instead.")
+    )
+    from apache_beam.options.pipeline_options import (
+        _STORE_FALSE_OPTIONS_WITH_DIFFERENT_DEST)
+
+    def get_options_not_present_in_map(d1, d2):
+      d = {}
+      for k in d1:
+        if k not in d2:
+          d[k] = d1[k]
+      return d
+
+    assert _STORE_FALSE_OPTIONS_WITH_DIFFERENT_DEST == dest_to_options, (
+      "If you are adding a new boolean flag with default=None,"
+      " with dest different from flag name, please add the flag and "
+      "dest of the flag: %s to variable "
+      " _STORE_FALSE_OPTIONS_WITH_DIFFERENT_DEST in PipelineOptions.py" % (

Review comment:
       I think we can simplify this to use `self.assertDictEqual()` with appropriate error message.




-- 
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: github-unsubscribe@beam.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


Issue Time Tracking
-------------------

    Worklog Id:     (was: 734864)
    Time Spent: 5h 10m  (was: 5h)

> PipelineOptions() and from_dictionary parsing use_public_ips and no_use_public_ips differently
> ----------------------------------------------------------------------------------------------
>
>                 Key: BEAM-13709
>                 URL: https://issues.apache.org/jira/browse/BEAM-13709
>             Project: Beam
>          Issue Type: Bug
>          Components: sdk-py-core
>            Reporter: Minbo Bae
>            Assignee: Anand Inguva
>            Priority: P2
>              Labels: starter, usability
>          Time Spent: 5h 10m
>  Remaining Estimate: 0h
>
> {{PipelineOptions}} in Python has two methods to pass a param dict: using in constructor {{PipelineOptions(**params)}} or {{{}PipelineOptions.from_dictionary(params){}}}.
> But, they work slightly differently:
>  * [PipelineOptions(**params)|https://github.com/apache/beam/blob/v2.35.0/sdks/python/apache_beam/options/pipeline_options.py#L313-L324] discards an option if it is not defined as a dest of {{argparse}} in an Option class. For example, {{no_use_public_ips=True}} is ignored and the Dataflow job will run with public IPs. To disable public IPs, the option dictionary must use {{{}use_public_ips{}}}.
>  * [PipelineOptions.from_dictionary()|https://github.com/apache/beam/blob/v2.35.0/sdks/python/apache_beam/options/pipeline_options.py#L229] skips an option if the option value is {{{}False{}}}. For example, {{use_public_ips=False}} is ignored and the Dataflow job will run with public IPs. To disable public IPs, the option dictionary must use {{no_use_public_ips.}}
> This makes the user very confused, and sometimes the pipeline works in an unexpected way. 
> We must have the consistent behavior between the two methods, or at least a warning about invalid ignored options.
> BEAM-9093 dealt with a similar issue for {{PipelineOptions()}}. Like the issue, I guess adding a warning in `PipelineOptions.from_dictionary()` for ignored options can help reducing the confusion, if we cannot have two methods have exactly the same behavior.
>  



--
This message was sent by Atlassian Jira
(v8.20.1#820001)