This is an automated email from the ASF dual-hosted git repository.

damccorm pushed a commit to branch users/damccorm/depLogging
in repository https://gitbox.apache.org/repos/asf/beam.git

commit a20766bb9521b199436a644edf4f284ec0304e08
Author: Danny Mccormick <[email protected]>
AuthorDate: Mon Jun 23 15:13:46 2025 -0400

    Disable dependency logging in local envs
---
 sdks/python/apache_beam/options/pipeline_options.py              | 9 +++++++++
 .../apache_beam/runners/portability/sdk_container_builder.py     | 3 ++-
 sdks/python/apache_beam/transforms/environments.py               | 8 +++++++-
 3 files changed, 18 insertions(+), 2 deletions(-)

diff --git a/sdks/python/apache_beam/options/pipeline_options.py 
b/sdks/python/apache_beam/options/pipeline_options.py
index 3554e4cfe95..dad905fec79 100644
--- a/sdks/python/apache_beam/options/pipeline_options.py
+++ b/sdks/python/apache_beam/options/pipeline_options.py
@@ -615,6 +615,15 @@ class StandardOptions(PipelineOptions):
       'apache_beam.runners.test.TestDataflowRunner',
   )
 
+  LOCAL_RUNNERS = (
+      'BundleBasedDirectRunner',
+      'DirectRunner',
+      'SwitchingDirectRunner',
+      'FnApiRunner',
+      'PrismRunner',
+      'TestDirectRunner',
+  )
+
   KNOWN_RUNNER_NAMES = [path.split('.')[-1] for path in ALL_KNOWN_RUNNERS]
 
   @classmethod
diff --git 
a/sdks/python/apache_beam/runners/portability/sdk_container_builder.py 
b/sdks/python/apache_beam/runners/portability/sdk_container_builder.py
index 0b7e1f28d00..9237e0c9f82 100644
--- a/sdks/python/apache_beam/runners/portability/sdk_container_builder.py
+++ b/sdks/python/apache_beam/runners/portability/sdk_container_builder.py
@@ -93,7 +93,8 @@ class SdkContainerImageBuilder(plugin.BeamPlugin):
 
   def _prepare_dependencies(self):
     with tempfile.TemporaryDirectory() as tmp:
-      artifacts = Stager.create_job_resources(self._options, tmp)
+      artifacts = Stager.create_job_resources(
+          self._options, tmp, log_submission_env_dependencies=False)
       resources = Stager.extract_staging_tuple_iter(artifacts)
       # make a copy of the staged artifacts into the temp source folder.
       file_names = []
diff --git a/sdks/python/apache_beam/transforms/environments.py 
b/sdks/python/apache_beam/transforms/environments.py
index ee9c3230292..6f17ca270eb 100644
--- a/sdks/python/apache_beam/transforms/environments.py
+++ b/sdks/python/apache_beam/transforms/environments.py
@@ -42,6 +42,7 @@ from google.protobuf import message
 from apache_beam import coders
 from apache_beam.options.pipeline_options import PortableOptions
 from apache_beam.options.pipeline_options import SetupOptions
+from apache_beam.options.pipeline_options import StandardOptions
 from apache_beam.portability import common_urns
 from apache_beam.portability import python_urns
 from apache_beam.portability.api import beam_runner_api_pb2
@@ -923,6 +924,10 @@ def python_sdk_dependencies(options, tmp_dir=None):
     tmp_dir = tempfile.mkdtemp()
   skip_prestaged_dependencies = options.view_as(
       SetupOptions).prebuild_sdk_container_engine is not None
+  runner = options.view_as(
+      StandardOptions).runner or StandardOptions.DEFAULT_RUNNER
+  log_submission_env_dependencies = runner.split(
+      '.')[-1] not in StandardOptions.LOCAL_RUNNERS
   return stager.Stager.create_job_resources(
       options,
       tmp_dir,
@@ -930,4 +935,5 @@ def python_sdk_dependencies(options, tmp_dir=None):
           artifact[0] + artifact[1]
           for artifact in PyPIArtifactRegistry.get_artifacts()
       ],
-      skip_prestaged_dependencies=skip_prestaged_dependencies)
+      skip_prestaged_dependencies=skip_prestaged_dependencies,
+      log_submission_env_dependencies=log_submission_env_dependencies)

Reply via email to