Thanks to visit codestin.com
Credit goes to github.com

Skip to content

Commit 88da381

Browse files
authored
Fail the pipeline when a mismatched Python or Beam version is detected. (#25313)
1 parent a9e80d2 commit 88da381

3 files changed

Lines changed: 46 additions & 2 deletions

File tree

sdks/python/apache_beam/pipeline_test.py

Lines changed: 16 additions & 0 deletions
Original file line numberDiff line numberDiff line change
@@ -23,6 +23,7 @@
2323
import platform
2424
import unittest
2525

26+
import mock
2627
import pytest
2728

2829
import apache_beam as beam
@@ -680,6 +681,21 @@ def test_track_pcoll_unbounded_flatten(self):
680681
self.assertIs(pcoll2_unbounded.is_bounded, False)
681682
self.assertIs(merged.is_bounded, False)
682683

684+
def test_incompatible_submission_and_runtime_envs_fail_pipeline(self):
685+
with mock.patch(
686+
'apache_beam.transforms.environments.sdk_base_version_capability'
687+
) as base_version:
688+
base_version.side_effect = [
689+
f"beam:version:sdk_base:apache/beam_python3.5_sdk:2.{i}.0"
690+
for i in range(100)
691+
]
692+
with self.assertRaisesRegex(
693+
RuntimeError,
694+
'Pipeline construction environment and pipeline runtime '
695+
'environment are not compatible.'):
696+
with TestPipeline() as p:
697+
_ = p | Create([None])
698+
683699

684700
class DoFnTest(unittest.TestCase):
685701
def test_element(self):

sdks/python/apache_beam/runners/worker/bundle_processor.py

Lines changed: 23 additions & 0 deletions
Original file line numberDiff line numberDiff line change
@@ -67,6 +67,7 @@
6767
from apache_beam.runners.worker import statesampler
6868
from apache_beam.transforms import TimeDomain
6969
from apache_beam.transforms import core
70+
from apache_beam.transforms import environments
7071
from apache_beam.transforms import sideinputs
7172
from apache_beam.transforms import userstate
7273
from apache_beam.transforms import window
@@ -823,6 +824,27 @@ def only_element(iterable):
823824
return element
824825

825826

827+
def _verify_descriptor_created_in_a_compatible_env(process_bundle_descriptor):
828+
# type: (beam_fn_api_pb2.ProcessBundleDescriptor) -> None
829+
830+
runtime_sdk = environments.sdk_base_version_capability()
831+
for t in process_bundle_descriptor.transforms.values():
832+
env = process_bundle_descriptor.environments[t.environment_id]
833+
for c in env.capabilities:
834+
if (c.startswith(environments.SDK_VERSION_CAPABILITY_PREFIX) and
835+
c != runtime_sdk):
836+
raise RuntimeError(
837+
"Pipeline construction environment and pipeline runtime "
838+
"environment are not compatible. If you use a custom "
839+
"container image, check that the Python interpreter minor version "
840+
"and the Apache Beam version in your image match the versions "
841+
"used at pipeline construction time. "
842+
f"Submission environment: {c}. "
843+
f"Runtime environment: {runtime_sdk}.")
844+
845+
# TODO: Consider warning on mismatches in versions of installed packages.
846+
847+
826848
class BundleProcessor(object):
827849
""" A class for processing bundles of elements. """
828850

@@ -846,6 +868,7 @@ def __init__(self,
846868
self.data_channel_factory = data_channel_factory
847869
self.current_instruction_id = None # type: Optional[str]
848870

871+
_verify_descriptor_created_in_a_compatible_env(process_bundle_descriptor)
849872
# There is no guarantee that the runner only set
850873
# timer_api_service_descriptor when having timers. So this field cannot be
851874
# used as an indicator of timers.

sdks/python/apache_beam/transforms/environments.py

Lines changed: 7 additions & 2 deletions
Original file line numberDiff line numberDiff line change
@@ -90,8 +90,8 @@ def looks_like_json(s):
9090

9191

9292
APACHE_BEAM_DOCKER_IMAGE_PREFIX = 'apache/beam'
93-
9493
APACHE_BEAM_JAVA_CONTAINER_NAME_PREFIX = 'beam_java'
94+
SDK_VERSION_CAPABILITY_PREFIX = 'beam:version:sdk_base:'
9595

9696

9797
def is_apache_beam_container(container_image):
@@ -777,6 +777,11 @@ def python_sdk_docker_capabilities():
777777
return python_sdk_capabilities() + [common_urns.protocols.SIBLING_WORKERS.urn]
778778

779779

780+
def sdk_base_version_capability():
781+
return (
782+
SDK_VERSION_CAPABILITY_PREFIX + DockerEnvironment.default_docker_image())
783+
784+
780785
def _python_sdk_capabilities_iter():
781786
# type: () -> Iterator[str]
782787
for urn_spec in common_urns.coders.__dict__.values():
@@ -786,7 +791,7 @@ def _python_sdk_capabilities_iter():
786791
yield common_urns.protocols.HARNESS_MONITORING_INFOS.urn
787792
yield common_urns.protocols.WORKER_STATUS.urn
788793
yield python_urns.PACKED_COMBINE_FN
789-
yield 'beam:version:sdk_base:' + DockerEnvironment.default_docker_image()
794+
yield sdk_base_version_capability()
790795
yield common_urns.sdf_components.TRUNCATE_SIZED_RESTRICTION.urn
791796
yield common_urns.primitives.TO_STRING.urn
792797

0 commit comments

Comments
 (0)