From 7d122bf3840a85a3f6d99cb5054a684eb4748921 Mon Sep 17 00:00:00 2001 From: Anand Inguva Date: Tue, 8 Feb 2022 12:43:55 -0500 Subject: [PATCH 1/7] Map to define relation between flags Discard options with value False passed to PipelineOptions class --- .../apache_beam/options/pipeline_options.py | 45 ++++++++++- .../options/pipeline_options_test.py | 81 +++++++++++++++++++ 2 files changed, 125 insertions(+), 1 deletion(-) diff --git a/sdks/python/apache_beam/options/pipeline_options.py b/sdks/python/apache_beam/options/pipeline_options.py index eb5235b796b..136cea70e1f 100644 --- a/sdks/python/apache_beam/options/pipeline_options.py +++ b/sdks/python/apache_beam/options/pipeline_options.py @@ -55,6 +55,18 @@ _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. +# Defining a map with their dest would maintain consistency +# across PipelineOptions(**dict), PipelineOptions.from_dictionary(dict), +# and argparse. +_STORE_FALSE_OPTIONS_WITH_DIFFERENT_DEST = { + 'no_use_public_ips': 'use_public_ips', + 'no_pipeline_type_check': 'pipeline_type_check', + 'no_direct_runner_use_stacked_bundle': 'direct_runner_use_stacked_bundle' +} + def _static_value_provider_of(value_type): """"Helper function to plug a ValueProvider into argparse. @@ -212,6 +224,24 @@ def __init__(self, flags=None, **kwargs): # Users access this dictionary store via __getattr__ / __setattr__ methods. self._all_options = kwargs + # filter bool options which are False to raise warnings. + # Beam doesn't support --=False on the command line. + # If a boolean option specified in kwargs programmatically, + # Eg: {no_use_public_ips: True}, implicitly it means the option + # is provided on the command line as --no_use_public_ips. + + self.invalid_options = {} + for option_name, option_value in self._all_options.items(): + if isinstance(option_value, bool) and not option_value: + self.invalid_options[option_name] = option_value + + for k in self.invalid_options: + if k in self._all_options: + del self._all_options[k] + + if self.invalid_options: + _LOGGER.warning("Discarding invalid overrides: %s", self.invalid_options) + # Initialize values of keys defined by this class. for option_name in self._visible_option_list(): # Note that options specified in kwargs will not be overwritten. @@ -219,6 +249,13 @@ def __init__(self, flags=None, **kwargs): self._all_options[option_name] = getattr( self._visible_options, option_name) + # flags like --no_use_public_ips have a different dest. + # Get the dest name from _STORE_FALSE_OPTIONS_WITH_DIFFERENT_DEST + for option_name, option_dest in ( + _STORE_FALSE_OPTIONS_WITH_DIFFERENT_DEST.items()): + if option_name in self._all_options: + self._all_options[option_dest] = False + @classmethod def _add_argparse_args(cls, parser): # type: (_BeamArgumentParser) -> None @@ -236,10 +273,13 @@ def from_dictionary(cls, options): A PipelineOptions object representing the given arguments. """ flags = [] + invalid_options = {} for k, v in options.items(): if isinstance(v, bool): if v: flags.append('--%s' % k) + else: + invalid_options[k] = v elif isinstance(v, list): for i in v: flags.append('--%s=%s' % (k, i)) @@ -248,6 +288,8 @@ def from_dictionary(cls, options): else: flags.append('--%s=%s' % (k, v)) + if invalid_options: + _LOGGER.warning("Discarding invalid overrides: %s", invalid_options) return cls(flags) def get_all_options( @@ -390,7 +432,8 @@ def __getattr__(self, name): "'%s' object has no attribute '%s'" % (type(self).__name__, name)) def __setattr__(self, name, value): - if name in ('_flags', '_all_options', '_visible_options'): + if name in ('_flags', '_all_options', '_visible_options', + 'invalid_options'): super().__setattr__(name, value) elif name in self._visible_option_list(): self._all_options[name] = value diff --git a/sdks/python/apache_beam/options/pipeline_options_test.py b/sdks/python/apache_beam/options/pipeline_options_test.py index 9392055799c..52d87692ebd 100644 --- a/sdks/python/apache_beam/options/pipeline_options_test.py +++ b/sdks/python/apache_beam/options/pipeline_options_test.py @@ -25,6 +25,7 @@ import hamcrest as hc +from apache_beam.options.pipeline_options import _BeamArgumentParser from apache_beam.options.pipeline_options import DebugOptions from apache_beam.options.pipeline_options import GoogleCloudOptions from apache_beam.options.pipeline_options import PipelineOptions @@ -36,6 +37,8 @@ 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): @@ -647,6 +650,84 @@ 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_dest = {} + options_diff_dest_store_true = {} # action == store_true, + # dest is different name + 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: + if actions[i].const: + options_diff_dest_store_true[option_name] = dest + continue + options_to_dest[option_name] = dest + + assert len(options_diff_dest_store_true) == 0, ( + _LOGGER.error("There should be no options that have a dest " + "different from option_name and action as " + "store_true. It would be confusing " + "to the user. Please specify the dest as the " + "option_name/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 == options_to_dest, ( + "If you are adding a new option with default=None, action=store_false," + " with dest different from option name, please add the option_name and " + "dest of the option: %s to variable " + " _STORE_FALSE_OPTIONS_WITH_DIFFERENT_DEST in PipelineOptions.py" % ( + get_options_not_present_in_map(options_to_dest, + _STORE_FALSE_OPTIONS_WITH_DIFFERENT_DEST)) + ) + + def test_pipelineoptions_store_false_with_different_dest(self): + + flags = ['--no_use_public_ips'] + params = {'no_use_public_ips': True} + options_1 = PipelineOptions(flags).view_as(WorkerOptions) + options_2 = PipelineOptions(**params).view_as(WorkerOptions) + options_3 = PipelineOptions.from_dictionary(params).view_as(WorkerOptions) + + assert options_1.use_public_ips == options_2.use_public_ips == ( + options_3.use_public_ips) == False + + flags = ['--use_public_ips'] + params = {'use_public_ips': True} + options_1 = PipelineOptions(flags).view_as(WorkerOptions) + options_2 = PipelineOptions(**params).view_as(WorkerOptions) + options_3 = PipelineOptions.from_dictionary(params).view_as(WorkerOptions) + + assert options_1.use_public_ips == options_2.use_public_ips == ( + options_3.use_public_ips) == True + params = {'use_public_ips': False} # use_public_ips = False represents + # that this option is not provided to the command line. + flags = [] + options_1 = PipelineOptions(flags).view_as(WorkerOptions) + options_2 = PipelineOptions(**params).view_as(WorkerOptions) + options_3 = PipelineOptions.from_dictionary(params).view_as(WorkerOptions) + + # Invalid override flags in params. + assert options_1.use_public_ips == options_2.use_public_ips == ( + options_3.use_public_ips) == None + if __name__ == '__main__': logging.getLogger().setLevel(logging.INFO) From 8ac7657f575899e8595bd2deea81c5de5160f8c8 Mon Sep 17 00:00:00 2001 From: Anand Inguva Date: Wed, 23 Feb 2022 13:33:17 -0500 Subject: [PATCH 2/7] Add warnings to the discarded flags --- .../apache_beam/options/pipeline_options.py | 74 ++++++++++--------- .../options/pipeline_options_test.py | 60 +++++---------- 2 files changed, 59 insertions(+), 75 deletions(-) diff --git a/sdks/python/apache_beam/options/pipeline_options.py b/sdks/python/apache_beam/options/pipeline_options.py index 136cea70e1f..f9205372def 100644 --- a/sdks/python/apache_beam/options/pipeline_options.py +++ b/sdks/python/apache_beam/options/pipeline_options.py @@ -62,9 +62,7 @@ # across PipelineOptions(**dict), PipelineOptions.from_dictionary(dict), # and argparse. _STORE_FALSE_OPTIONS_WITH_DIFFERENT_DEST = { - 'no_use_public_ips': 'use_public_ips', - 'no_pipeline_type_check': 'pipeline_type_check', - 'no_direct_runner_use_stacked_bundle': 'direct_runner_use_stacked_bundle' + 'use_public_ips': 'no_use_public_ips' } @@ -193,6 +191,15 @@ def __init__(self, flags=None, **kwargs): then sys.argv will be used as input for parsing arguments. **kwargs: Add overrides for arguments passed in flags. + + Note: 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 {no_use_public_ips: True}, which is not defined + to any destination(dest) in parser, would be discarded/ignored. Instead, + pass the dest of the flag(dest of no_use_public_ips is use_public_ips), + Eg: {use_public_ips: False} to get the desired behavior. + + """ # Initializing logging configuration in case the user did not set it up. logging.basicConfig() @@ -224,23 +231,14 @@ def __init__(self, flags=None, **kwargs): # Users access this dictionary store via __getattr__ / __setattr__ methods. self._all_options = kwargs - # filter bool options which are False to raise warnings. - # Beam doesn't support --=False on the command line. - # If a boolean option specified in kwargs programmatically, - # Eg: {no_use_public_ips: True}, implicitly it means the option - # is provided on the command line as --no_use_public_ips. - - self.invalid_options = {} - for option_name, option_value in self._all_options.items(): - if isinstance(option_value, bool) and not option_value: - self.invalid_options[option_name] = option_value - - for k in self.invalid_options: - if k in self._all_options: - del self._all_options[k] + 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 self.invalid_options: - _LOGGER.warning("Discarding invalid overrides: %s", self.invalid_options) + if _invalid_options: + _LOGGER.warning("Discarding invalid overrides: %s", _invalid_options) # Initialize values of keys defined by this class. for option_name in self._visible_option_list(): @@ -249,13 +247,6 @@ def __init__(self, flags=None, **kwargs): self._all_options[option_name] = getattr( self._visible_options, option_name) - # flags like --no_use_public_ips have a different dest. - # Get the dest name from _STORE_FALSE_OPTIONS_WITH_DIFFERENT_DEST - for option_name, option_dest in ( - _STORE_FALSE_OPTIONS_WITH_DIFFERENT_DEST.items()): - if option_name in self._all_options: - self._all_options[option_dest] = False - @classmethod def _add_argparse_args(cls, parser): # type: (_BeamArgumentParser) -> None @@ -271,15 +262,26 @@ 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 = [] - invalid_options = {} for k, v in options.items(): if isinstance(v, bool): if v: flags.append('--%s' % k) - else: - invalid_options[k] = v + # 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]) elif isinstance(v, list): for i in v: flags.append('--%s=%s' % (k, i)) @@ -288,8 +290,6 @@ def from_dictionary(cls, options): else: flags.append('--%s=%s' % (k, v)) - if invalid_options: - _LOGGER.warning("Discarding invalid overrides: %s", invalid_options) return cls(flags) def get_all_options( @@ -395,6 +395,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) + for option_name in view._visible_option_list(): # Initialize values of keys defined by a cls. # @@ -432,8 +441,7 @@ def __getattr__(self, name): "'%s' object has no attribute '%s'" % (type(self).__name__, name)) def __setattr__(self, name, value): - if name in ('_flags', '_all_options', '_visible_options', - 'invalid_options'): + if name in ('_flags', '_all_options', '_visible_options'): super().__setattr__(name, value) elif name in self._visible_option_list(): self._all_options[name] = value diff --git a/sdks/python/apache_beam/options/pipeline_options_test.py b/sdks/python/apache_beam/options/pipeline_options_test.py index 52d87692ebd..39aa65d26d8 100644 --- a/sdks/python/apache_beam/options/pipeline_options_test.py +++ b/sdks/python/apache_beam/options/pipeline_options_test.py @@ -656,28 +656,34 @@ def test_options_store_false_with_different_dest(self): cls._add_argparse_args(parser) actions = parser._actions.copy() - options_to_dest = {} - options_diff_dest_store_true = {} # action == store_true, - # dest is different name + 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 - options_to_dest[option_name] = dest + # 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 options that have a dest " - "different from option_name and action as " + _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 " - "option_name/flag_name instead.") + "flag_name instead.") ) from apache_beam.options.pipeline_options import ( _STORE_FALSE_OPTIONS_WITH_DIFFERENT_DEST) @@ -689,45 +695,15 @@ def get_options_not_present_in_map(d1, d2): d[k] = d1[k] return d - assert _STORE_FALSE_OPTIONS_WITH_DIFFERENT_DEST == options_to_dest, ( - "If you are adding a new option with default=None, action=store_false," - " with dest different from option name, please add the option_name and " - "dest of the option: %s to variable " + 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" % ( - get_options_not_present_in_map(options_to_dest, + get_options_not_present_in_map(dest_to_options, _STORE_FALSE_OPTIONS_WITH_DIFFERENT_DEST)) ) - def test_pipelineoptions_store_false_with_different_dest(self): - - flags = ['--no_use_public_ips'] - params = {'no_use_public_ips': True} - options_1 = PipelineOptions(flags).view_as(WorkerOptions) - options_2 = PipelineOptions(**params).view_as(WorkerOptions) - options_3 = PipelineOptions.from_dictionary(params).view_as(WorkerOptions) - - assert options_1.use_public_ips == options_2.use_public_ips == ( - options_3.use_public_ips) == False - - flags = ['--use_public_ips'] - params = {'use_public_ips': True} - options_1 = PipelineOptions(flags).view_as(WorkerOptions) - options_2 = PipelineOptions(**params).view_as(WorkerOptions) - options_3 = PipelineOptions.from_dictionary(params).view_as(WorkerOptions) - - assert options_1.use_public_ips == options_2.use_public_ips == ( - options_3.use_public_ips) == True - params = {'use_public_ips': False} # use_public_ips = False represents - # that this option is not provided to the command line. - flags = [] - options_1 = PipelineOptions(flags).view_as(WorkerOptions) - options_2 = PipelineOptions(**params).view_as(WorkerOptions) - options_3 = PipelineOptions.from_dictionary(params).view_as(WorkerOptions) - - # Invalid override flags in params. - assert options_1.use_public_ips == options_2.use_public_ips == ( - options_3.use_public_ips) == None - if __name__ == '__main__': logging.getLogger().setLevel(logging.INFO) From 3af5a6848c77469f143f79a0b60bd060328a58e5 Mon Sep 17 00:00:00 2001 From: Anand Inguva Date: Wed, 23 Feb 2022 15:39:35 -0500 Subject: [PATCH 3/7] Fixup: lint --- sdks/python/apache_beam/options/pipeline_options_test.py | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/sdks/python/apache_beam/options/pipeline_options_test.py b/sdks/python/apache_beam/options/pipeline_options_test.py index 39aa65d26d8..f93156dc188 100644 --- a/sdks/python/apache_beam/options/pipeline_options_test.py +++ b/sdks/python/apache_beam/options/pipeline_options_test.py @@ -25,13 +25,13 @@ import hamcrest as hc -from apache_beam.options.pipeline_options import _BeamArgumentParser from apache_beam.options.pipeline_options import DebugOptions from apache_beam.options.pipeline_options import GoogleCloudOptions from apache_beam.options.pipeline_options import PipelineOptions 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 10cff75bb4d5530aa882790793c28aad900a907e Mon Sep 17 00:00:00 2001 From: Anand Inguva Date: Wed, 23 Feb 2022 16:51:59 -0500 Subject: [PATCH 4/7] Fixup: docstring --- .../apache_beam/options/pipeline_options.py | 27 ++++++++++--------- 1 file changed, 14 insertions(+), 13 deletions(-) diff --git a/sdks/python/apache_beam/options/pipeline_options.py b/sdks/python/apache_beam/options/pipeline_options.py index f9205372def..77dc0ae43db 100644 --- a/sdks/python/apache_beam/options/pipeline_options.py +++ b/sdks/python/apache_beam/options/pipeline_options.py @@ -190,15 +190,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. - - Note: 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 {no_use_public_ips: True}, which is not defined - to any destination(dest) in parser, would be discarded/ignored. Instead, - pass the dest of the flag(dest of no_use_public_ips is use_public_ips), - Eg: {use_public_ips: False} to get the desired behavior. - + **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 + {no_use_public_ips: True}, which is not defined to any + destination(dest) in parser, would be discarded/ignored. + Instead, pass the dest of the flag + (dest of no_use_public_ips is use_public_ips), + Eg: {use_public_ips: False} to get the desired behavior. """ # Initializing logging configuration in case the user did not set it up. @@ -263,10 +263,11 @@ 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. - + 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. """ From eedce2fd8ce5df1bd6d2046628f71a5be15e6002 Mon Sep 17 00:00:00 2001 From: Anand Inguva Date: Sun, 6 Mar 2022 19:53:17 -0500 Subject: [PATCH 5/7] Refactor docstring, unittest --- .../apache_beam/options/pipeline_options.py | 73 +++++++------------ .../options/pipeline_options_test.py | 61 +++++++--------- 2 files changed, 52 insertions(+), 82 deletions(-) diff --git a/sdks/python/apache_beam/options/pipeline_options.py b/sdks/python/apache_beam/options/pipeline_options.py index 77dc0ae43db..0465509ee7a 100644 --- a/sdks/python/apache_beam/options/pipeline_options.py +++ b/sdks/python/apache_beam/options/pipeline_options.py @@ -55,15 +55,10 @@ _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. -# Defining a map with their dest would maintain consistency -# across PipelineOptions(**dict), PipelineOptions.from_dictionary(dict), -# and argparse. -_STORE_FALSE_OPTIONS_WITH_DIFFERENT_DEST = { - 'use_public_ips': 'no_use_public_ips' -} +# Map the boolean option with the flag_name for the flags that have a +# 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): @@ -190,14 +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. For kwargs, - please pass the option names instead of flag names. + **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(). Passing flag names like - {no_use_public_ips: True}, which is not defined to any - destination(dest) in parser, would be discarded/ignored. - Instead, pass the dest of the flag - (dest of no_use_public_ips is use_public_ips), + parser.add_argument() for each flag. Passing flags + like {no_use_public_ips: True}, for which the flag name + defined to a different destination(dest) in parser, + would be discarded. Instead, pass the dest of + the flag(dest of no_use_public_ips is use_public_ips), Eg: {use_public_ips: False} to get the desired behavior. """ @@ -231,15 +227,6 @@ 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) - # Initialize values of keys defined by this class. for option_name in self._visible_option_list(): # Note that options specified in kwargs will not be overwritten. @@ -262,27 +249,25 @@ 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(): + # 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): 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]) + 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)) @@ -397,14 +382,6 @@ 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) - for option_name in view._visible_option_list(): # Initialize values of keys defined by a cls. # diff --git a/sdks/python/apache_beam/options/pipeline_options_test.py b/sdks/python/apache_beam/options/pipeline_options_test.py index f93156dc188..d49ecec6670 100644 --- a/sdks/python/apache_beam/options/pipeline_options_test.py +++ b/sdks/python/apache_beam/options/pipeline_options_test.py @@ -656,53 +656,46 @@ def test_options_store_false_with_different_dest(self): cls._add_argparse_args(parser) actions = parser._actions.copy() - dest_to_options = {} + options_to_flags = {} options_diff_dest_store_true = {} for i in range(len(actions)): - options_name = actions[i].option_strings - dest = actions[i].dest + flag_names = actions[i].option_strings + option_name = 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: + 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[option_name] = dest + 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: - 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.") - ) + 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 ( - _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" % ( - get_options_not_present_in_map(dest_to_options, - _STORE_FALSE_OPTIONS_WITH_DIFFERENT_DEST)) - ) + _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__': From cb24183001ba3d52eff780b2687794ca7638666a Mon Sep 17 00:00:00 2001 From: Anand Inguva Date: Mon, 7 Mar 2022 13:42:30 -0500 Subject: [PATCH 6/7] Update docstring --- sdks/python/apache_beam/options/pipeline_options.py | 8 +++----- 1 file changed, 3 insertions(+), 5 deletions(-) diff --git a/sdks/python/apache_beam/options/pipeline_options.py b/sdks/python/apache_beam/options/pipeline_options.py index 0465509ee7a..2c2748400a2 100644 --- a/sdks/python/apache_beam/options/pipeline_options.py +++ b/sdks/python/apache_beam/options/pipeline_options.py @@ -190,12 +190,10 @@ def __init__(self, flags=None, **kwargs): 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 flag name - defined to a different destination(dest) in parser, + 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), - Eg: {use_public_ips: False} to get the desired behavior. - + 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() From 078f58bff03daea77e3f713f128f239608799c90 Mon Sep 17 00:00:00 2001 From: Anand Inguva Date: Tue, 8 Mar 2022 00:01:24 -0500 Subject: [PATCH 7/7] Change Map name and update comments --- sdks/python/apache_beam/options/pipeline_options.py | 12 ++++++------ .../apache_beam/options/pipeline_options_test.py | 6 +++--- 2 files changed, 9 insertions(+), 9 deletions(-) diff --git a/sdks/python/apache_beam/options/pipeline_options.py b/sdks/python/apache_beam/options/pipeline_options.py index 2c2748400a2..bea4d55e174 100644 --- a/sdks/python/apache_beam/options/pipeline_options.py +++ b/sdks/python/apache_beam/options/pipeline_options.py @@ -55,10 +55,10 @@ _LOGGER = logging.getLogger(__name__) -# Map the boolean option with the flag_name for the flags that have a -# 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'} +# Map defined with option names to flag names for boolean 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 = {'use_public_ips': 'no_use_public_ips'} def _static_value_provider_of(value_type): @@ -259,12 +259,12 @@ def from_dictionary(cls, options): if isinstance(v, bool): if v: flags.append('--%s' % k) - elif k in _FLAGS_WITH_DIFFERENT_DEST: + elif k in _FLAG_THAT_SETS_FALSE_VALUE: # 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]) + flag_that_disables_the_option = (_FLAG_THAT_SETS_FALSE_VALUE[k]) flags.append('--%s' % flag_that_disables_the_option) elif isinstance(v, list): for i in v: diff --git a/sdks/python/apache_beam/options/pipeline_options_test.py b/sdks/python/apache_beam/options/pipeline_options_test.py index d49ecec6670..3195f3594fe 100644 --- a/sdks/python/apache_beam/options/pipeline_options_test.py +++ b/sdks/python/apache_beam/options/pipeline_options_test.py @@ -687,15 +687,15 @@ def test_options_store_false_with_different_dest(self): "to the user. Please specify the dest as the " "flag_name instead.")) from apache_beam.options.pipeline_options import ( - _FLAGS_WITH_DIFFERENT_DEST) + _FLAG_THAT_SETS_FALSE_VALUE) self.assertDictEqual( - _FLAGS_WITH_DIFFERENT_DEST, + _FLAG_THAT_SETS_FALSE_VALUE, 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") + "_FLAG_THAT_SETS_FALSE_VALUE in PipelineOptions.py") if __name__ == '__main__':