6767from apache_beam .runners .worker import statesampler
6868from apache_beam .transforms import TimeDomain
6969from apache_beam .transforms import core
70+ from apache_beam .transforms import environments
7071from apache_beam .transforms import sideinputs
7172from apache_beam .transforms import userstate
7273from 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+
826848class 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.
0 commit comments