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

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

                Author: ASF GitHub Bot
            Created on: 27/Jan/21 18:18
            Start Date: 27/Jan/21 18:18
    Worklog Time Spent: 10m 
      Work Description: robertwb commented on a change in pull request #13763:
URL: https://github.com/apache/beam/pull/13763#discussion_r565530996



##########
File path: sdks/python/apache_beam/runners/dataflow/dataflow_runner.py
##########
@@ -462,6 +462,62 @@ def run_pipeline(self, pipeline, options):
 
     self._maybe_add_unified_worker_missing_options(options)
 
+    from apache_beam.transforms import environments
+    if options.view_as(SetupOptions).prebuild_sdk_container_engine:
+      # if prebuild_sdk_container_engine is specified we will build a new sdk
+      # container image with dependencies pre-installed and use that image,
+      # instead of using the inferred default container image.
+      self._default_environment = (
+          environments.DockerEnvironment.from_options(options))
+      options.view_as(WorkerOptions).worker_harness_container_image = (
+          self._default_environment.container_image)
+    else:
+      self._default_environment = (
+          environments.DockerEnvironment.from_container_image(
+              apiclient.get_container_image_from_options(options),
+              artifacts=environments.python_sdk_dependencies(options)))
+
+    # Optimize the pipeline if it not streaming and optimizations are enabled
+    # in options.
+    pre_optimize = options.view_as(DebugOptions).lookup_experiment(
+        'pre_optimize', 'default').lower()
+    if (not options.view_as(StandardOptions).streaming and
+        pre_optimize != 'none' and pre_optimize != 'default'):
+      from apache_beam.runners.portability.fn_api_runner import translations
+      if pre_optimize == 'all':
+        phases = [
+            translations.eliminate_common_key_with_none,
+            translations.pack_combiners,
+            translations.sort_stages
+        ]
+      else:

Review comment:
       What is the value of pre_optimize here? I would rather reject unknown 
values than have this be the fallthrough. 




----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to 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: 542979)
    Time Spent: 4h 20m  (was: 4h 10m)

> Remove translations.pack_combiners optimizer from defaults
> ----------------------------------------------------------
>
>                 Key: BEAM-11695
>                 URL: https://issues.apache.org/jira/browse/BEAM-11695
>             Project: Beam
>          Issue Type: Bug
>          Components: sdk-py-core
>            Reporter: Yifan Mai
>            Priority: P2
>             Fix For: 2.28.0
>
>          Time Spent: 4h 20m
>  Remaining Estimate: 0h
>
> Currently this optimizer is run by default inĀ  DataflowRunner, FnApiRunner 
> and PortableRunner. We should remove it and only enable it under an 
> experiment flag until BEAM-11644 is resolved.



--
This message was sent by Atlassian Jira
(v8.3.4#803005)

Reply via email to