Skip to content
New issue

Have a question about this project? Sign up for a free GitHub account to open an issue and contact its maintainers and the community.

By clicking “Sign up for GitHub”, you agree to our terms of service and privacy statement. We’ll occasionally send you account related emails.

Already on GitHub? Sign in to your account

[BEAM-13709] Inconsistent behavior when parsing boolean flags across different APIs in Python SDK #16929

Merged
merged 7 commits into from
Mar 8, 2022
Merged
Show file tree
Hide file tree
Changes from 6 commits
Commits
File filter

Filter by extension

Filter by extension

Conversations
Failed to load comments.
Loading
Jump to
Jump to file
Failed to load files.
Loading
Diff view
Diff view
29 changes: 28 additions & 1 deletion sdks/python/apache_beam/options/pipeline_options.py
Original file line number Diff line number Diff line change
Expand Up @@ -55,6 +55,11 @@

_LOGGER = logging.getLogger(__name__)

# Map the boolean option with the flag_name for the flags that have a
Copy link
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Consider the following for readability:

# Map of Boolean options to flag names for options that have a destination(dest)
# in parser.add_argument() different from the flag name 
# and whose default value  is `None`. 
_FLAG_THAT_SETS_FALSE_VALUE =  {...}

Copy link
Contributor Author

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Variable name sounds good. Changed the comments a little bit

# destination(dest) different from the flag name and the
# default value is None in parser.add_argument().
_FLAGS_WITH_DIFFERENT_DEST = {'use_public_ips': 'no_use_public_ips'}


def _static_value_provider_of(value_type):
""""Helper function to plug a ValueProvider into argparse.
Expand Down Expand Up @@ -180,7 +185,15 @@ 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 overrides
of arguments, please pass the `option names` instead of
flag names.
Option names: These are defined as dest in the
parser.add_argument() for each flag. Passing flags
like {no_use_public_ips: True}, for which the dest is
defined to a different flag name in the parser,
would be discarded. Instead, pass the dest of
the flag (dest of no_use_public_ips is use_public_ips).
"""
# Initializing logging configuration in case the user did not set it up.
logging.basicConfig()
Expand Down Expand Up @@ -237,9 +250,22 @@ def from_dictionary(cls, options):
"""
flags = []
for k, v in options.items():
# 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.
if isinstance(v, bool):
Copy link
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Let's move the Note: here.

Copy link
Contributor Author

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Done

if v:
flags.append('--%s' % k)
elif k in _FLAGS_WITH_DIFFERENT_DEST:
# Capture overriding flags, which have a different dest
# from the flag name defined in the parser.add_argument
# Eg: no_use_public_ips, which has the dest=use_public_ips
# different from flag name
flag_that_disables_the_option = (_FLAGS_WITH_DIFFERENT_DEST[k])
flags.append('--%s' % flag_that_disables_the_option)
elif isinstance(v, list):
for i in v:
flags.append('--%s=%s' % (k, i))
Expand Down Expand Up @@ -353,6 +379,7 @@ def view_as(self, cls):

"""
view = cls(self._flags)

for option_name in view._visible_option_list():
# Initialize values of keys defined by a cls.
#
Expand Down
50 changes: 50 additions & 0 deletions sdks/python/apache_beam/options/pipeline_options_test.py
Original file line number Diff line number Diff line change
Expand Up @@ -31,11 +31,14 @@
from apache_beam.options.pipeline_options import ProfilingOptions
from apache_beam.options.pipeline_options import TypeOptions
from apache_beam.options.pipeline_options import WorkerOptions
from apache_beam.options.pipeline_options import _BeamArgumentParser
from apache_beam.options.value_provider import RuntimeValueProvider
from apache_beam.options.value_provider import StaticValueProvider
from apache_beam.transforms.display import DisplayData
from apache_beam.transforms.display_test import DisplayDataItemMatcher

_LOGGER = logging.getLogger(__name__)


class PipelineOptionsTest(unittest.TestCase):
def setUp(self):
Expand Down Expand Up @@ -647,6 +650,53 @@ 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()
options_to_flags = {}
options_diff_dest_store_true = {}

for i in range(len(actions)):
flag_names = actions[i].option_strings
option_name = actions[i].dest

if isinstance(actions[i].const, bool):
for flag_name in flag_names:
flag_name = flag_name.strip('-')
if flag_name != option_name:
# 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[flag_name] = option_name
continue
# check the flags like no_use_public_ips
# default is None, action is {True, False}
if actions[i].default is None:
options_to_flags[option_name] = flag_name

self.assertEqual(
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 (
_FLAGS_WITH_DIFFERENT_DEST)

self.assertDictEqual(
_FLAGS_WITH_DIFFERENT_DEST,
options_to_flags,
"If you are adding a new boolean flag with default=None,"
" with different dest/option_name from the flag name, please add "
"the dest and the flag name to the map "
"_FLAGS_WITH_DIFFERENT_DEST in PipelineOptions.py")


if __name__ == '__main__':
logging.getLogger().setLevel(logging.INFO)
Expand Down