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

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

                Author: ASF GitHub Bot
            Created on: 05/Oct/18 13:19
            Start Date: 05/Oct/18 13:19
    Worklog Time Spent: 10m 
      Work Description: mxm closed pull request #6557: [BEAM-5442] Pass unknown 
SDK pipeline options to Runner
URL: https://github.com/apache/beam/pull/6557
 
 
   

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 a0059dbb381..bc8c962bcd4 100644
--- a/sdks/python/apache_beam/options/pipeline_options.py
+++ b/sdks/python/apache_beam/options/pipeline_options.py
@@ -20,6 +20,7 @@
 from __future__ import absolute_import
 
 import argparse
+import logging
 from builtins import list
 from builtins import object
 
@@ -213,7 +214,19 @@ def get_all_options(self, drop_default=False):
       subset[str(cls)] = cls
     for cls in subset.values():
       cls._add_argparse_args(parser)  # pylint: disable=protected-access
-    known_args, _ = parser.parse_known_args(self._flags)
+    known_args, unknown_args = parser.parse_known_args(self._flags)
+    # Parse args which are not known at this point but might be recognized
+    # 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:
+        if arg.startswith('--'):
+          parser.add_argument(arg.split('=', 1)[0], nargs='?')
+      # repeat parsing with unknown options added
+      known_args, unknown_args = parser.parse_known_args(self._flags)
+      if unknown_args:
+        logging.warn("Discarding unparseable args: %s", unknown_args)
+
     result = vars(known_args)
 
     # Apply the overrides if any
diff --git a/sdks/python/apache_beam/options/pipeline_options_test.py 
b/sdks/python/apache_beam/options/pipeline_options_test.py
index 9c14c25668e..651e7336b39 100644
--- a/sdks/python/apache_beam/options/pipeline_options_test.py
+++ b/sdks/python/apache_beam/options/pipeline_options_test.py
@@ -45,16 +45,24 @@ def tearDown(self):
        'display_data': [DisplayDataItemMatcher('num_workers', 5)]},
       {
           'flags': [
-              '--profile_cpu', '--profile_location', 'gs://bucket/', 
'ignored'],
+              '--profile_cpu', '--profile_location', 'gs://bucket/',
+              'ignored', '-invalid=arg', '--unknown_arg', 'unknown_value',
+              '--unknown_flag'
+          ],
           'expected': {
               'profile_cpu': True, 'profile_location': 'gs://bucket/',
               'mock_flag': False, 'mock_option': None,
-              'mock_multi_option': None},
+              'mock_multi_option': None,
+              'unknown_arg': 'unknown_value',
+              'unknown_flag': None},
           'display_data': [
               DisplayDataItemMatcher('profile_cpu',
                                      True),
               DisplayDataItemMatcher('profile_location',
-                                     'gs://bucket/')]
+                                     'gs://bucket/'),
+              DisplayDataItemMatcher('unknown_arg',
+                                     'unknown_value')
+          ]
       },
       {'flags': ['--num_workers', '5', '--mock_flag'],
        'expected': {'num_workers': 5,
diff --git a/sdks/python/apache_beam/runners/worker/sdk_worker_main_test.py 
b/sdks/python/apache_beam/runners/worker/sdk_worker_main_test.py
index 6b5972e4ac4..e9b584a339f 100644
--- a/sdks/python/apache_beam/runners/worker/sdk_worker_main_test.py
+++ b/sdks/python/apache_beam/runners/worker/sdk_worker_main_test.py
@@ -56,8 +56,8 @@ def wrapped_method_for_test():
   def test_work_count_default_value(self):
     self._check_worker_count('{}', 12)
 
-  def test_parse_pipeine_options(self):
-    expected_options = PipelineOptions()
+  def test_parse_pipeline_options(self):
+    expected_options = PipelineOptions([])
     expected_options.view_as(
         SdkWorkerMainTest.MockOptions).m_m_option = [
             'worker_threads=1', 'beam_fn_api'


 

----------------------------------------------------------------
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:
[email protected]


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

    Worklog Id:     (was: 151608)
    Time Spent: 4h  (was: 3h 50m)

> 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: 4h
>  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)

Reply via email to