Skip to content

Commit

Permalink
Refactor docstring, unittest
Browse files Browse the repository at this point in the history
  • Loading branch information
AnandInguva committed Mar 7, 2022
1 parent 10cff75 commit eedce2f
Show file tree
Hide file tree
Showing 2 changed files with 52 additions and 82 deletions.
73 changes: 25 additions & 48 deletions sdks/python/apache_beam/options/pipeline_options.py
Original file line number Diff line number Diff line change
Expand Up @@ -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):
Expand Down Expand Up @@ -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.
"""
Expand Down Expand Up @@ -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.
Expand All @@ -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))
Expand Down Expand Up @@ -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.
#
Expand Down
61 changes: 27 additions & 34 deletions sdks/python/apache_beam/options/pipeline_options_test.py
Original file line number Diff line number Diff line change
Expand Up @@ -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__':
Expand Down

0 comments on commit eedce2f

Please sign in to comment.