[ https://issues.apache.org/jira/browse/BEAM-5442?focusedWorklogId=153140&page=com.atlassian.jira.plugin.system.issuetabpanels:worklog-tabpanel#worklog-153140 ]
ASF GitHub Bot logged work on BEAM-5442: ---------------------------------------- Author: ASF GitHub Bot Created on: 10/Oct/18 13:54 Start Date: 10/Oct/18 13:54 Worklog Time Spent: 10m Work Description: tweise closed pull request #6600: [BEAM-5442] Store duplicate unknown options in a list argument URL: https://github.com/apache/beam/pull/6600 This is a PR merged from a forked repository. As GitHub hides the original diff on merge, it is displayed below for the sake of provenance: As this is a foreign pull request (from a fork), the diff is supplied below (as it won't show otherwise due to GitHub magic): diff --git a/sdks/python/apache_beam/options/pipeline_options.py b/sdks/python/apache_beam/options/pipeline_options.py index a86cab9a84c..a172535b100 100644 --- a/sdks/python/apache_beam/options/pipeline_options.py +++ b/sdks/python/apache_beam/options/pipeline_options.py @@ -23,6 +23,7 @@ import logging from builtins import list from builtins import object +from collections import OrderedDict from apache_beam.options.value_provider import RuntimeValueProvider from apache_beam.options.value_provider import StaticValueProvider @@ -219,10 +220,25 @@ def get_all_options(self, drop_default=False): # at a later point in time, i.e. by the actual Runner. if unknown_args and unknown_args[0] != '': logging.info("Parsing unknown args: %s", unknown_args) - for arg in unknown_args: - # https://issues.apache.org/jira/browse/BEAM-5442 - if arg.startswith('--') and not arg.startswith('--beam_plugins'): - parser.add_argument(arg.split('=', 1)[0], nargs='?') + + def enumerate_args(args): + cleaned_args = OrderedDict() + for arg in args: + if arg.startswith('--'): + # split argument name if it's in arg_name=value syntax + arg_name = arg.split('=', 1)[0] + # count identical arg names + if arg_name not in cleaned_args: + cleaned_args[arg_name] = 1 + else: + cleaned_args[arg_name] += 1 + return cleaned_args + + for arg_name, num_times in enumerate_args(unknown_args).items(): + parser.add_argument(arg_name, + nargs='?', + action='append' if num_times > 1 else 'store') + # repeat parsing with unknown options added known_args, unknown_args = parser.parse_known_args(self._flags) if unknown_args: diff --git a/sdks/python/apache_beam/options/pipeline_options_test.py b/sdks/python/apache_beam/options/pipeline_options_test.py index 651e7336b39..65997536370 100644 --- a/sdks/python/apache_beam/options/pipeline_options_test.py +++ b/sdks/python/apache_beam/options/pipeline_options_test.py @@ -289,6 +289,15 @@ def _add_argparse_args(cls, parser): with self.assertRaises(RuntimeError): options.pot_non_vp_arg1.get() + # Converts duplicate unknown argument values to a single argument + # with a list value. + def test_unknown_duplicate_args_converted_to_list(self): + options = PipelineOptions(['--dup_arg', 'val1', + '--dup_arg', 'val2', + '--dup_arg=val3']) + self.assertEqual(options.get_all_options()['dup_arg'], + ['val1', 'val2', 'val3']) + if __name__ == '__main__': logging.getLogger().setLevel(logging.INFO) ---------------------------------------------------------------- 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 Issue Time Tracking ------------------- Worklog Id: (was: 153140) Time Spent: 8h 10m (was: 8h) > PortableRunner swallows custom options for Runner > ------------------------------------------------- > > Key: BEAM-5442 > URL: https://issues.apache.org/jira/browse/BEAM-5442 > Project: Beam > Issue Type: Bug > Components: sdk-java-core, sdk-py-core > Reporter: Maximilian Michels > Assignee: Maximilian Michels > Priority: Major > Labels: portability, portability-flink > Fix For: 2.8.0 > > Time Spent: 8h 10m > Remaining Estimate: 0h > > The PortableRunner doesn't pass custom PipelineOptions to the executing > Runner. > Example: {{--parallelism=4}} won't be forwarded to the FlinkRunner. > (The option is just removed during proto translation without any warning) > We should allow some form of customization through the options, even for the > PortableRunner. -- This message was sent by Atlassian JIRA (v7.6.3#76005)