From f4ba0b62dd6cbdeb6ed5a336eeb0ce7ba15ab1ab Mon Sep 17 00:00:00 2001 From: Claude Date: Tue, 29 Jul 2025 13:07:55 +0000 Subject: [PATCH 01/15] Pass update compat through as_deterministic_coder. --- sdks/python/apache_beam/coders/coders.py | 50 ++++++++++++++------- sdks/python/apache_beam/coders/row_coder.py | 19 ++++++-- 2 files changed, 48 insertions(+), 21 deletions(-) diff --git a/sdks/python/apache_beam/coders/coders.py b/sdks/python/apache_beam/coders/coders.py index 2691857bf0a6..40973e2a9a67 100644 --- a/sdks/python/apache_beam/coders/coders.py +++ b/sdks/python/apache_beam/coders/coders.py @@ -180,7 +180,8 @@ def is_deterministic(self): """ return False - def as_deterministic_coder(self, step_label, error_message=None): + def as_deterministic_coder( + self, step_label, error_message=None, update_compatibility_version=None): """Returns a deterministic version of self, if possible. Otherwise raises a value error. @@ -531,10 +532,13 @@ def is_deterministic(self): # Map ordering is non-deterministic return False - def as_deterministic_coder(self, step_label, error_message=None): + def as_deterministic_coder( + self, step_label, error_message=None, update_compatibility_version=None): return DeterministicMapCoder( - self._key_coder.as_deterministic_coder(step_label, error_message), - self._value_coder.as_deterministic_coder(step_label, error_message)) + self._key_coder.as_deterministic_coder( + step_label, error_message, update_compatibility_version), + self._value_coder.as_deterministic_coder( + step_label, error_message, update_compatibility_version)) def __eq__(self, other): return ( @@ -609,12 +613,13 @@ def is_deterministic(self): # type: () -> bool return self._value_coder.is_deterministic() - def as_deterministic_coder(self, step_label, error_message=None): + def as_deterministic_coder( + self, step_label, error_message=None, update_compatibility_version=None): if self.is_deterministic(): return self else: deterministic_value_coder = self._value_coder.as_deterministic_coder( - step_label, error_message) + step_label, error_message, update_compatibility_version) return NullableCoder(deterministic_value_coder) def __eq__(self, other): @@ -876,7 +881,8 @@ def _nonhashable_dumps(x): return coder_impl.CallbackCoderImpl(_nonhashable_dumps, pickler.loads) - def as_deterministic_coder(self, step_label, error_message=None): + def as_deterministic_coder( + self, step_label, error_message=None, update_compatibility_version=None): return FastPrimitivesCoder(self, requires_deterministic=step_label) def to_type_hint(self): @@ -891,7 +897,8 @@ def _create_impl(self): return coder_impl.CallbackCoderImpl( lambda x: dumps(x, protocol), pickle.loads) - def as_deterministic_coder(self, step_label, error_message=None): + def as_deterministic_coder( + self, step_label, error_message=None, update_compatibility_version=None): return FastPrimitivesCoder(self, requires_deterministic=step_label) def to_type_hint(self): @@ -956,7 +963,8 @@ def is_deterministic(self): # type: () -> bool return self._fallback_coder.is_deterministic() - def as_deterministic_coder(self, step_label, error_message=None): + def as_deterministic_coder( + self, step_label, error_message=None, update_compatibility_version=None): if self.is_deterministic(): return self else: @@ -1055,7 +1063,8 @@ def is_deterministic(self): # a Map. return False - def as_deterministic_coder(self, step_label, error_message=None): + def as_deterministic_coder( + self, step_label, error_message=None, update_compatibility_version=None): return DeterministicProtoCoder(self.proto_message_type) def __eq__(self, other): @@ -1101,7 +1110,8 @@ def is_deterministic(self): # type: () -> bool return True - def as_deterministic_coder(self, step_label, error_message=None): + def as_deterministic_coder( + self, step_label, error_message=None, update_compatibility_version=None): return self @@ -1188,12 +1198,14 @@ def is_deterministic(self): # type: () -> bool return all(c.is_deterministic() for c in self._coders) - def as_deterministic_coder(self, step_label, error_message=None): + def as_deterministic_coder( + self, step_label, error_message=None, update_compatibility_version=None): if self.is_deterministic(): return self else: return TupleCoder([ - c.as_deterministic_coder(step_label, error_message) + c.as_deterministic_coder( + step_label, error_message, update_compatibility_version) for c in self._coders ]) @@ -1267,12 +1279,14 @@ def is_deterministic(self): # type: () -> bool return self._elem_coder.is_deterministic() - def as_deterministic_coder(self, step_label, error_message=None): + def as_deterministic_coder( + self, step_label, error_message=None, update_compatibility_version=None): if self.is_deterministic(): return self else: return TupleSequenceCoder( - self._elem_coder.as_deterministic_coder(step_label, error_message)) + self._elem_coder.as_deterministic_coder( + step_label, error_message, update_compatibility_version)) @classmethod def from_type_hint(cls, typehint, registry): @@ -1307,12 +1321,14 @@ def is_deterministic(self): # type: () -> bool return self._elem_coder.is_deterministic() - def as_deterministic_coder(self, step_label, error_message=None): + def as_deterministic_coder( + self, step_label, error_message=None, update_compatibility_version=None): if self.is_deterministic(): return self else: return type(self)( - self._elem_coder.as_deterministic_coder(step_label, error_message)) + self._elem_coder.as_deterministic_coder( + step_label, error_message, update_compatibility_version)) def value_coder(self): return self._elem_coder diff --git a/sdks/python/apache_beam/coders/row_coder.py b/sdks/python/apache_beam/coders/row_coder.py index 1becf408cfbf..6ffd183f4e84 100644 --- a/sdks/python/apache_beam/coders/row_coder.py +++ b/sdks/python/apache_beam/coders/row_coder.py @@ -52,7 +52,11 @@ class RowCoder(FastCoder): Implements the beam:coder:row:v1 standard coder spec. """ - def __init__(self, schema, force_deterministic=False): + def __init__( + self, + schema, + force_deterministic=False, + update_compatibility_version=None): """Initializes a :class:`RowCoder`. Args: @@ -71,7 +75,10 @@ def __init__(self, schema, force_deterministic=False): ] if force_deterministic: self.components = [ - c.as_deterministic_coder(force_deterministic) for c in self.components + c.as_deterministic_coder( + force_deterministic, + update_compatibility_version=update_compatibility_version) + for c in self.components ] self.forced_deterministic = bool(force_deterministic) @@ -81,11 +88,15 @@ def _create_impl(self): def is_deterministic(self): return all(c.is_deterministic() for c in self.components) - def as_deterministic_coder(self, step_label, error_message=None): + def as_deterministic_coder( + self, step_label, error_message=None, update_compatibility_version=None): if self.is_deterministic(): return self else: - return RowCoder(self.schema, error_message or step_label) + return RowCoder( + self.schema, + error_message or step_label, + update_compatibility_version=update_compatibility_version) def to_type_hint(self): return self._type_hint From 513fac0e816e1461f8b1024b67256c0f9460d9cd Mon Sep 17 00:00:00 2001 From: Claude Date: Tue, 29 Jul 2025 15:58:40 +0000 Subject: [PATCH 02/15] Coder changes. --- sdks/python/apache_beam/coders/coder_impl.pxd | 2 + sdks/python/apache_beam/coders/coder_impl.py | 55 +++++++- sdks/python/apache_beam/coders/coders.py | 67 +++++++++- .../apache_beam/coders/coders_test_common.py | 117 +++++++++--------- .../internal/cloudpickle_pickler.py | 2 + sdks/python/apache_beam/transforms/util.py | 23 ++-- 6 files changed, 191 insertions(+), 75 deletions(-) diff --git a/sdks/python/apache_beam/coders/coder_impl.pxd b/sdks/python/apache_beam/coders/coder_impl.pxd index 27cffe7b62df..cd77db890dc7 100644 --- a/sdks/python/apache_beam/coders/coder_impl.pxd +++ b/sdks/python/apache_beam/coders/coder_impl.pxd @@ -81,6 +81,7 @@ cdef class FastPrimitivesCoderImpl(StreamCoderImpl): cdef CoderImpl iterable_coder_impl cdef object requires_deterministic_step_label cdef bint warn_deterministic_fallback + cdef bint force_use_dill @cython.locals(dict_value=dict, int_value=libc.stdint.int64_t, unicode_value=unicode) @@ -88,6 +89,7 @@ cdef class FastPrimitivesCoderImpl(StreamCoderImpl): @cython.locals(t=int) cpdef decode_from_stream(self, InputStream stream, bint nested) cdef encode_special_deterministic(self, value, OutputStream stream) + cdef encode_type_2_66_0(self, t, OutputStream stream) cdef encode_type(self, t, OutputStream stream) cdef decode_type(self, InputStream stream) diff --git a/sdks/python/apache_beam/coders/coder_impl.py b/sdks/python/apache_beam/coders/coder_impl.py index 807d083d8a38..5b43ae963a44 100644 --- a/sdks/python/apache_beam/coders/coder_impl.py +++ b/sdks/python/apache_beam/coders/coder_impl.py @@ -50,7 +50,6 @@ from typing import Tuple from typing import Type -import dill import numpy as np from fastavro import parse_schema from fastavro import schemaless_reader @@ -58,6 +57,7 @@ from apache_beam.coders import observable from apache_beam.coders.avro_record import AvroRecord +from apache_beam.internal import cloudpickle_pickler from apache_beam.typehints.schemas import named_tuple_from_schema from apache_beam.utils import proto_utils from apache_beam.utils import windowed_value @@ -71,6 +71,11 @@ except ImportError: dataclasses = None # type: ignore +try: + import dill +except ImportError: + dill = None # type: ignore + if TYPE_CHECKING: import proto from apache_beam.transforms import userstate @@ -354,14 +359,30 @@ def decode(self, value): _ITERABLE_LIKE_TYPES = set() # type: Set[Type] +def _verify_dill_compat(): + if not dill: + raise RuntimeError("Error importing dill for encoding deterministic" \ + " special dill types. Ensure dill version '0.3.1.1'" \ + " is installed in the execution environment.") + if dill.__version__ != "0.3.1.1": + raise RuntimeError("Error verifying dill compatibility forencoding " \ + "deterministic special dill types. Ensure dill " + "version '0.3.1.1' is installed in the execution " \ + "environment. Found dill version '{dill.__version__}") + + class FastPrimitivesCoderImpl(StreamCoderImpl): """For internal use only; no backwards-compatibility guarantees.""" def __init__( - self, fallback_coder_impl, requires_deterministic_step_label=None): + self, + fallback_coder_impl, + requires_deterministic_step_label=None, + force_use_dill=False): self.fallback_coder_impl = fallback_coder_impl self.iterable_coder_impl = IterableCoderImpl(self) self.requires_deterministic_step_label = requires_deterministic_step_label self.warn_deterministic_fallback = True + self.force_use_dill = force_use_dill @staticmethod def register_iterable_like_type(t): @@ -525,10 +546,20 @@ def _deterministic_encoding_error_msg(self, value): "please provide a type hint for the input of '%s'" % (value, type(value), self.requires_deterministic_step_label)) - def encode_type(self, t, stream): + def encode_type_2_66_0(self, t, stream): + _verify_dill_compat() stream.write(dill.dumps(t), True) + def encode_type(self, t, stream): + if self.force_use_dill: + return self.encode_type_2_66_0(t, stream) + bs = cloudpickle_pickler.dumps( + t, config=cloudpickle_pickler.NO_DYNAMIC_CLASS_TRACKING_CONFIG) + stream.write(bs, True) + def decode_type(self, stream): + if self.force_use_dill: + return _unpickle_type_2_66_0(stream.read_all(True)) return _unpickle_type(stream.read_all(True)) def decode_from_stream(self, stream, nested): @@ -589,19 +620,32 @@ def decode_from_stream(self, stream, nested): _unpickled_types = {} # type: Dict[bytes, type] -def _unpickle_type(bs): +def _unpickle_type_2_66_0(bs): t = _unpickled_types.get(bs, None) if t is None: + _verify_dill_compat() t = _unpickled_types[bs] = dill.loads(bs) # Fix unpicklable anonymous named tuples for Python 3.6. if t.__base__ is tuple and hasattr(t, '_fields'): try: pickle.loads(pickle.dumps(t)) except pickle.PicklingError: - t.__reduce__ = lambda self: (_unpickle_named_tuple, (bs, tuple(self))) + t.__reduce__ = lambda self: ( + _unpickle_named_tuple_2_66_0, (bs, tuple(self))) return t +def _unpickle_named_tuple_2_66_0(bs, items): + return _unpickle_type_2_66_0(bs)(*items) + + +def _unpickle_type(bs): + if not _unpickled_types.get(bs, None): + _unpickled_types[bs] = cloudpickle_pickler.loads(bs) + + return _unpickled_types[bs] + + def _unpickle_named_tuple(bs, items): return _unpickle_type(bs)(*items) @@ -837,6 +881,7 @@ def decode_from_stream(self, in_, nested): if IntervalWindow is None: from apache_beam.transforms.window import IntervalWindow # instantiating with None is not part of the public interface + # pylint: disable=too-many-function-args typed_value = IntervalWindow(None, None) # type: ignore[arg-type] typed_value._end_micros = ( 1000 * self._to_normal_time(in_.read_bigendian_uint64())) diff --git a/sdks/python/apache_beam/coders/coders.py b/sdks/python/apache_beam/coders/coders.py index 40973e2a9a67..4f5df3a78d02 100644 --- a/sdks/python/apache_beam/coders/coders.py +++ b/sdks/python/apache_beam/coders/coders.py @@ -918,6 +918,44 @@ def _create_impl(self): cloudpickle_pickler.dumps, cloudpickle_pickler.loads) +class DeterministicFastPrimitivesCoderV2(FastCoder): + """Throws runtime errors when encoding non-deterministic values.""" + def __init__(self, coder, step_label): + self._underlying_coder = coder + self._step_label = step_label + + def _create_impl(self): + + return coder_impl.FastPrimitivesCoderImpl( + self._underlying_coder.get_impl(), + requires_deterministic_step_label=self._step_label, + force_use_dill=False) + + def is_deterministic(self): + # type: () -> bool + return True + + def is_kv_coder(self): + # type: () -> bool + return True + + def key_coder(self): + return self + + def value_coder(self): + return self + + def to_type_hint(self): + return Any + + def to_runner_api_parameter(self, context): + # type: (Optional[PipelineContext]) -> Tuple[str, Any, Sequence[Coder]] + return ( + python_urns.PICKLED_CODER, + google.protobuf.wrappers_pb2.BytesValue(value=serialize_coder(self)), + ()) + + class DeterministicFastPrimitivesCoder(FastCoder): """Throws runtime errors when encoding non-deterministic values.""" def __init__(self, coder, step_label): @@ -927,7 +965,8 @@ def __init__(self, coder, step_label): def _create_impl(self): return coder_impl.FastPrimitivesCoderImpl( self._underlying_coder.get_impl(), - requires_deterministic_step_label=self._step_label) + requires_deterministic_step_label=self._step_label, + force_use_dill=True) def is_deterministic(self): # type: () -> bool @@ -947,6 +986,29 @@ def to_type_hint(self): return Any +def _should_force_use_dill(update_compatibility_version): + from apache_beam.transforms.util import is_v1_prior_to_v2 + + if not is_v1_prior_to_v2(v1=update_compatibility_version, v2="2.67.0"): + return False + + try: + import dill + assert dill.__version__ == "0.3.1.1" + except Exception as e: + raise RuntimeError("Error using dill for encoding special types. Ensure" \ + " dill version '0.3.1.1' is installed in execution and" \ + f" runtime environment. Error {e}") + return True + + +def _update_compatible_deterministic_fast_primitives_coder( + coder, step_label, update_compatibility_version): + if _should_force_use_dill(update_compatibility_version): + return DeterministicFastPrimitivesCoder(coder, step_label) + return DeterministicFastPrimitivesCoderV2(coder, step_label) + + class FastPrimitivesCoder(FastCoder): """Encodes simple primitives (e.g. str, int) efficiently. @@ -968,7 +1030,8 @@ def as_deterministic_coder( if self.is_deterministic(): return self else: - return DeterministicFastPrimitivesCoder(self, step_label) + return _update_compatible_deterministic_fast_primitives_coder( + self, step_label, update_compatibility_version) def to_type_hint(self): return Any diff --git a/sdks/python/apache_beam/coders/coders_test_common.py b/sdks/python/apache_beam/coders/coders_test_common.py index dbd0a301bb0d..56cb0f1d2a0b 100644 --- a/sdks/python/apache_beam/coders/coders_test_common.py +++ b/sdks/python/apache_beam/coders/coders_test_common.py @@ -34,6 +34,8 @@ from typing import NamedTuple import pytest +from parameterized import param +from parameterized import parameterized from apache_beam.coders import proto2_coder_test_messages_pb2 as test_message from apache_beam.coders import coders @@ -58,6 +60,7 @@ dataclasses = None # type: ignore MyNamedTuple = collections.namedtuple('A', ['x', 'y']) # type: ignore[name-match] +AnotherNamedTuple = collections.namedtuple('AnotherNamedTuple', ['x', 'y']) MyTypedNamedTuple = NamedTuple('MyTypedNamedTuple', [('f1', int), ('f2', str)]) @@ -230,9 +233,15 @@ def test_memoizing_pickle_coder(self): coder = coders._MemoizingPickleCoder() self.check_coder(coder, *self.test_values) - def test_deterministic_coder(self): + @parameterized.expand([ + param(compat_version=None), + param(compat_version="2.66.0"), + ]) + def test_deterministic_coder(self, compat_version): coder = coders.FastPrimitivesCoder() - deterministic_coder = coders.DeterministicFastPrimitivesCoder(coder, 'step') + deterministic_coder = coder.as_deterministic_coder( + step_label="step", update_compatibility_version=compat_version) + self.check_coder(deterministic_coder, *self.test_values_deterministic) for v in self.test_values_deterministic: self.check_coder(coders.TupleCoder((deterministic_coder, )), (v, )) @@ -254,8 +263,16 @@ def test_deterministic_coder(self): self.check_coder(deterministic_coder, test_message.MessageA(field1='value')) + # Skip this test during cloudpickle. Dill monkey patches the __reduce__ + # method for anonymous named tuples (MyNamedTuple) which is not pickleable. + # Since the test is parameterized the type gets colbbered. + if compat_version: + self.check_coder( + deterministic_coder, [MyNamedTuple(1, 2), MyTypedNamedTuple(1, 'a')]) + self.check_coder( - deterministic_coder, [MyNamedTuple(1, 2), MyTypedNamedTuple(1, 'a')]) + deterministic_coder, + [AnotherNamedTuple(1, 2), MyTypedNamedTuple(1, 'a')]) if dataclasses is not None: self.check_coder(deterministic_coder, FrozenDataClass(1, 2)) @@ -265,9 +282,10 @@ def test_deterministic_coder(self): with self.assertRaises(TypeError): self.check_coder( deterministic_coder, FrozenDataClass(UnFrozenDataClass(1, 2), 3)) - with self.assertRaises(TypeError): - self.check_coder( - deterministic_coder, MyNamedTuple(UnFrozenDataClass(1, 2), 3)) + with self.assertRaises(TypeError): + self.check_coder( + deterministic_coder, + AnotherNamedTuple(UnFrozenDataClass(1, 2), 3)) self.check_coder(deterministic_coder, list(MyEnum)) self.check_coder(deterministic_coder, list(MyIntEnum)) @@ -610,15 +628,21 @@ def test_param_windowed_value_coder(self): 1, (window.IntervalWindow(11, 21), ), PaneInfo(True, False, 1, 2, 3)))) - def test_cross_process_encoding_of_special_types_is_deterministic(self): + @parameterized.expand([ + param(compat_version="2.67.0"), + param(compat_version="2.66.0"), + ]) + def test_cross_process_encoding_of_special_types_is_deterministic( + self, compat_version): """Test cross-process determinism for all special deterministic types""" if sys.executable is None: self.skipTest('No Python interpreter found') + # pylint: disable=line-too-long script = textwrap.dedent( - '''\ + f'''\ import pickle import sys import collections @@ -626,13 +650,18 @@ def test_cross_process_encoding_of_special_types_is_deterministic(self): import logging from apache_beam.coders import coders - from apache_beam.coders import proto2_coder_test_messages_pb2 as test_message - from typing import NamedTuple + from apache_beam.coders.coders_test_common import MyNamedTuple + from apache_beam.coders.coders_test_common import MyTypedNamedTuple + from apache_beam.coders.coders_test_common import MyEnum + from apache_beam.coders.coders_test_common import MyIntEnum + from apache_beam.coders.coders_test_common import MyIntFlag + from apache_beam.coders.coders_test_common import MyFlag + from apache_beam.coders.coders_test_common import DefinesGetState + from apache_beam.coders.coders_test_common import DefinesGetAndSetState + from apache_beam.coders.coders_test_common import FrozenDataClass - try: - import dataclasses - except ImportError: - dataclasses = None + + from apache_beam.coders import proto2_coder_test_messages_pb2 as test_message logging.basicConfig( level=logging.INFO, @@ -640,38 +669,6 @@ def test_cross_process_encoding_of_special_types_is_deterministic(self): stream=sys.stderr, force=True ) - - # Define all the special types that encode_special_deterministic handles - MyNamedTuple = collections.namedtuple('A', ['x', 'y']) - MyTypedNamedTuple = NamedTuple('MyTypedNamedTuple', [('f1', int), ('f2', str)]) - - class MyEnum(enum.Enum): - E1 = 5 - E2 = enum.auto() - E3 = 'abc' - - MyIntEnum = enum.IntEnum('MyIntEnum', 'I1 I2 I3') - MyIntFlag = enum.IntFlag('MyIntFlag', 'F1 F2 F3') - MyFlag = enum.Flag('MyFlag', 'F1 F2 F3') - - if dataclasses is not None: - @dataclasses.dataclass(frozen=True) - class FrozenDataClass: - a: int - b: int - - class DefinesGetAndSetState: - def __init__(self, value): - self.value = value - - def __getstate__(self): - return self.value - - def __setstate__(self, value): - self.value = value - - def __eq__(self, other): - return type(other) is type(self) and other.value == self.value # Test cases for all special deterministic types # NOTE: When this script run in a subprocess the module is considered @@ -683,26 +680,27 @@ def __eq__(self, other): ("named_tuple_simple", MyNamedTuple(1, 2)), ("typed_named_tuple", MyTypedNamedTuple(1, 'a')), ("named_tuple_list", [MyNamedTuple(1, 2), MyTypedNamedTuple(1, 'a')]), - # ("enum_single", MyEnum.E1), - # ("enum_list", list(MyEnum)), - # ("int_enum_list", list(MyIntEnum)), - # ("int_flag_list", list(MyIntFlag)), - # ("flag_list", list(MyFlag)), + ("enum_single", MyEnum.E1), + ("enum_list", list(MyEnum)), + ("int_enum_list", list(MyIntEnum)), + ("int_flag_list", list(MyIntFlag)), + ("flag_list", list(MyFlag)), ("getstate_setstate_simple", DefinesGetAndSetState(1)), ("getstate_setstate_complex", DefinesGetAndSetState((1, 2, 3))), ("getstate_setstate_list", [DefinesGetAndSetState(1), DefinesGetAndSetState((1, 2, 3))]), ] - if dataclasses is not None: - test_cases.extend([ - ("frozen_dataclass", FrozenDataClass(1, 2)), - ("frozen_dataclass_list", [FrozenDataClass(1, 2), FrozenDataClass(3, 4)]), - ]) + + test_cases.extend([ + ("frozen_dataclass", FrozenDataClass(1, 2)), + ("frozen_dataclass_list", [FrozenDataClass(1, 2), FrozenDataClass(3, 4)]), + ]) + compat_version = "{compat_version}" coder = coders.FastPrimitivesCoder() - deterministic_coder = coders.DeterministicFastPrimitivesCoder(coder, 'step') + deterministic_coder = coder.as_deterministic_coder("step", update_compatibility_version=compat_version) - results = {} + results = dict() for test_name, value in test_cases: try: encoded = deterministic_coder.encode(value) @@ -730,7 +728,8 @@ def run_subprocess(): results2 = run_subprocess() coder = coders.FastPrimitivesCoder() - deterministic_coder = coders.DeterministicFastPrimitivesCoder(coder, 'step') + deterministic_coder = coder.as_deterministic_coder( + "step", update_compatibility_version=compat_version) for test_name in results1: data1 = results1[test_name] diff --git a/sdks/python/apache_beam/internal/cloudpickle_pickler.py b/sdks/python/apache_beam/internal/cloudpickle_pickler.py index 63038e770f27..e55818bfb226 100644 --- a/sdks/python/apache_beam/internal/cloudpickle_pickler.py +++ b/sdks/python/apache_beam/internal/cloudpickle_pickler.py @@ -39,6 +39,8 @@ DEFAULT_CONFIG = cloudpickle.CloudPickleConfig( skip_reset_dynamic_type_state=True) +NO_DYNAMIC_CLASS_TRACKING_CONFIG = cloudpickle.CloudPickleConfig( + id_generator=None, skip_reset_dynamic_type_state=True) try: from absl import flags diff --git a/sdks/python/apache_beam/transforms/util.py b/sdks/python/apache_beam/transforms/util.py index c60ded52df26..bbd2815b84a2 100644 --- a/sdks/python/apache_beam/transforms/util.py +++ b/sdks/python/apache_beam/transforms/util.py @@ -932,6 +932,18 @@ def get_window_coder(self): return self._window_coder +def is_v1_prior_to_v2(v1, v2): + if v1 is None: + return False + + v1 = tuple(map(int, v1.split('.')[0:3])) + v2 = tuple(map(int, v2.split('.')[0:3])) + for i in range(min(len(v1), len(v2))): + if v1[i] < v2[i]: + return True + return False + + def is_compat_version_prior_to(options, breaking_change_version): # This function is used in a branch statement to determine whether we should # keep the old behavior prior to a breaking change or use the new behavior. @@ -940,15 +952,8 @@ def is_compat_version_prior_to(options, breaking_change_version): update_compatibility_version = options.view_as( pipeline_options.StreamingOptions).update_compatibility_version - if update_compatibility_version is None: - return False - - compat_version = tuple(map(int, update_compatibility_version.split('.')[0:3])) - change_version = tuple(map(int, breaking_change_version.split('.')[0:3])) - for i in range(min(len(compat_version), len(change_version))): - if compat_version[i] < change_version[i]: - return True - return False + return is_v1_prior_to_v2( + v1=update_compatibility_version, v2=breaking_change_version) def reify_metadata_default_window( From 67c35470c290b08a81cf084e35d44f221a1faf02 Mon Sep 17 00:00:00 2001 From: Claude Date: Thu, 31 Jul 2025 02:45:39 +0000 Subject: [PATCH 03/15] Pass update compat through pipeline options. --- sdks/python/apache_beam/coders/coder_impl.py | 2 +- sdks/python/apache_beam/pipeline.py | 7 +++++-- sdks/python/apache_beam/runners/pipeline_context.py | 10 +++++++++- 3 files changed, 15 insertions(+), 4 deletions(-) diff --git a/sdks/python/apache_beam/coders/coder_impl.py b/sdks/python/apache_beam/coders/coder_impl.py index 5b43ae963a44..d46f7a0df7a8 100644 --- a/sdks/python/apache_beam/coders/coder_impl.py +++ b/sdks/python/apache_beam/coders/coder_impl.py @@ -74,7 +74,7 @@ try: import dill except ImportError: - dill = None # type: ignore + dill = None if TYPE_CHECKING: import proto diff --git a/sdks/python/apache_beam/pipeline.py b/sdks/python/apache_beam/pipeline.py index 83a0bee81456..f5a7d86288fc 100644 --- a/sdks/python/apache_beam/pipeline.py +++ b/sdks/python/apache_beam/pipeline.py @@ -989,7 +989,8 @@ def to_runner_api( context = pipeline_context.PipelineContext( use_fake_coders=use_fake_coders, component_id_map=self.component_id_map, - default_environment=default_environment) + default_environment=default_environment, + pipeline_options=self._options) elif default_environment is not None: raise ValueError( 'Only one of context or default_environment may be specified.') @@ -1092,7 +1093,9 @@ def from_runner_api( }) from apache_beam.runners import pipeline_context context = pipeline_context.PipelineContext( - proto.components, requirements=proto.requirements) + proto.components, + requirements=proto.requirements, + pipeline_options=options) if proto.root_transform_ids: root_transform_id, = proto.root_transform_ids p.transforms_stack = [context.transforms.get_by_id(root_transform_id)] diff --git a/sdks/python/apache_beam/runners/pipeline_context.py b/sdks/python/apache_beam/runners/pipeline_context.py index 132a1aedca33..8c17787ff48b 100644 --- a/sdks/python/apache_beam/runners/pipeline_context.py +++ b/sdks/python/apache_beam/runners/pipeline_context.py @@ -43,6 +43,7 @@ from apache_beam.coders.coder_impl import IterableStateReader from apache_beam.coders.coder_impl import IterableStateWriter from apache_beam.internal import pickler +from apache_beam.options import pipeline_options from apache_beam.pipeline import ComponentIdMap from apache_beam.portability.api import beam_fn_api_pb2 from apache_beam.portability.api import beam_runner_api_pb2 @@ -177,7 +178,9 @@ def __init__( iterable_state_write: Optional[IterableStateWriter] = None, namespace: str = 'ref', requirements: Iterable[str] = (), + pipeline_options: Optional[pipeline_options.PipelineOptions] = None ) -> None: + self.pipeline_options = pipeline_options if isinstance(proto, beam_fn_api_pb2.ProcessBundleDescriptor): proto = beam_runner_api_pb2.Components( coders=dict(proto.coders.items()), @@ -257,7 +260,12 @@ def coder_id_from_element_type( def deterministic_coder(self, coder: coders.Coder, msg: str) -> coders.Coder: if coder not in self.deterministic_coder_map: - self.deterministic_coder_map[coder] = coder.as_deterministic_coder(msg) # type: ignore + update_compatibility_version = self.pipeline_options.view_as( + pipeline_options.StreamingOptions + ).update_compatibility_version if self.pipeline_options else None + self.deterministic_coder_map[coder] = coder.as_deterministic_coder( + msg, + update_compatibility_version=update_compatibility_version) # type: ignore return self.deterministic_coder_map[coder] def element_type_from_coder_id(self, coder_id: str) -> Any: From fb59f29d3774e16eb888a74b6b3d53a1d2e22691 Mon Sep 17 00:00:00 2001 From: Claude Date: Thu, 31 Jul 2025 19:33:01 +0000 Subject: [PATCH 04/15] Add tests. --- .../apache_beam/coders/coders_test_common.py | 109 +++++++++++++++++- 1 file changed, 108 insertions(+), 1 deletion(-) diff --git a/sdks/python/apache_beam/coders/coders_test_common.py b/sdks/python/apache_beam/coders/coders_test_common.py index 56cb0f1d2a0b..b2a985950bd6 100644 --- a/sdks/python/apache_beam/coders/coders_test_common.py +++ b/sdks/python/apache_beam/coders/coders_test_common.py @@ -304,6 +304,113 @@ def test_deterministic_coder(self, compat_version): 1: 'x', 'y': 2 })) + @parameterized.expand([ + param(compat_version=None), + param(compat_version="2.66.0"), + ]) + def test_deterministic_map_coder_is_update_compatible(self, compat_version): + values = [{ + MyTypedNamedTuple(i, 'a'): MyTypedNamedTuple('a', i) + for i in range(10) + }] + + coder = coders.MapCoder( + coders.FastPrimitivesCoder(), coders.FastPrimitivesCoder()) + + deterministic_coder = coder.as_deterministic_coder( + step_label="step", update_compatibility_version=compat_version) + + assert isinstance( + deterministic_coder._key_coder, + coders.DeterministicFastPrimitivesCoderV2 + if not compat_version else coders.DeterministicFastPrimitivesCoder) + + self.check_coder(deterministic_coder, *values) + + @parameterized.expand([ + param(compat_version=None), + param(compat_version="2.66.0"), + ]) + def test_deterministic_nullable_coder_is_update_compatible( + self, compat_version): + values = [MyTypedNamedTuple(1, 'a'), None, MyTypedNamedTuple(2, 'a')] + + coder = coders.NullableCoder(coders.FastPrimitivesCoder()) + + deterministic_coder = coder.as_deterministic_coder( + step_label="step", update_compatibility_version=compat_version) + + assert isinstance( + deterministic_coder._value_coder, + coders.DeterministicFastPrimitivesCoderV2 + if not compat_version else coders.DeterministicFastPrimitivesCoder) + + self.check_coder(deterministic_coder, *values) + + @parameterized.expand([ + param(compat_version=None), + param(compat_version="2.66.0"), + ]) + def test_deterministic_tuple_coder_is_update_compatible(self, compat_version): + values = [MyTypedNamedTuple(1, 'a'), MyTypedNamedTuple(2, 'a')] + + coder = coders.TupleCoder( + [coders.FastPrimitivesCoder(), coders.FastPrimitivesCoder()]) + + deterministic_coder = coder.as_deterministic_coder( + step_label="step", update_compatibility_version=compat_version) + + assert all( + isinstance( + component_coder, + coders.DeterministicFastPrimitivesCoderV2 + if not compat_version else coders.DeterministicFastPrimitivesCoder) + for component_coder in deterministic_coder._coders) + + self.check_coder(deterministic_coder, *values) + + @parameterized.expand([ + param(compat_version=None), + param(compat_version="2.66.0"), + ]) + def test_deterministic_tuplesequence_coder_is_update_compatible( + self, compat_version): + values = [MyTypedNamedTuple(1, 'a'), MyTypedNamedTuple(2, 'a')] + + coder = coders.TupleSequenceCoder(coders.FastPrimitivesCoder()) + + deterministic_coder = coder.as_deterministic_coder( + step_label="step", update_compatibility_version=compat_version) + + assert isinstance( + deterministic_coder._elem_coder, + coders.DeterministicFastPrimitivesCoderV2 + if not compat_version else coders.DeterministicFastPrimitivesCoder) + + self.check_coder(deterministic_coder, *values) + + @parameterized.expand([ + param(compat_version=None), + param(compat_version="2.66.0"), + ]) + def test_deterministic_listlike_coder_is_update_compatible( + self, compat_version): + _ = [MyTypedNamedTuple(1, 'a'), MyTypedNamedTuple(2, 'a')] + + coder = coders.ListLikeCoder(coders.FastPrimitivesCoder()) + + deterministic_coder = coder.as_deterministic_coder( + step_label="step", update_compatibility_version=compat_version) + + assert isinstance( + deterministic_coder._elem_coder, + coders.DeterministicFastPrimitivesCoderV2 + if not compat_version else coders.DeterministicFastPrimitivesCoder) + + # This check fails for both compat options with + # AssertionError: MyTypedNamedTuple(f1=1, f2='a') != [1, 'a'] + # self.check_coder(deterministic_coder, *values) + def test_dill_coder(self): cell_value = (lambda x: lambda: x)(0).__closure__[0] self.check_coder(coders.DillCoder(), 'a', 1, cell_value) @@ -860,7 +967,7 @@ def test_map_coder(self): { i: str(i) for i in range(5000) - } + }, ] map_coder = coders.MapCoder(coders.VarIntCoder(), coders.StrUtf8Coder()) self.check_coder(map_coder, *values) From 97b0274c8b472e73104490861152de7e83fec433 Mon Sep 17 00:00:00 2001 From: Claude Date: Thu, 31 Jul 2025 20:13:40 +0000 Subject: [PATCH 05/15] asd --- sdks/python/apache_beam/coders/coder_impl.pxd | 2 +- sdks/python/apache_beam/coders/coder_impl.py | 33 +++++++++++-------- sdks/python/apache_beam/coders/coders.py | 9 ++--- 3 files changed, 25 insertions(+), 19 deletions(-) diff --git a/sdks/python/apache_beam/coders/coder_impl.pxd b/sdks/python/apache_beam/coders/coder_impl.pxd index cd77db890dc7..2db7b963d151 100644 --- a/sdks/python/apache_beam/coders/coder_impl.pxd +++ b/sdks/python/apache_beam/coders/coder_impl.pxd @@ -89,7 +89,7 @@ cdef class FastPrimitivesCoderImpl(StreamCoderImpl): @cython.locals(t=int) cpdef decode_from_stream(self, InputStream stream, bint nested) cdef encode_special_deterministic(self, value, OutputStream stream) - cdef encode_type_2_66_0(self, t, OutputStream stream) + cdef encode_type_2_67_0(self, t, OutputStream stream) cdef encode_type(self, t, OutputStream stream) cdef decode_type(self, InputStream stream) diff --git a/sdks/python/apache_beam/coders/coder_impl.py b/sdks/python/apache_beam/coders/coder_impl.py index d46f7a0df7a8..41ced15183bb 100644 --- a/sdks/python/apache_beam/coders/coder_impl.py +++ b/sdks/python/apache_beam/coders/coder_impl.py @@ -360,15 +360,14 @@ def decode(self, value): def _verify_dill_compat(): + base_error = ( + "This pipeline runs with the update_compatibility_version=2.67.0 flag. " + "When running with this flag on SDKs 2.68.0 or higher, you must " + "ensure dill==0.3.1.1 is installed. Dill is not installed.") if not dill: - raise RuntimeError("Error importing dill for encoding deterministic" \ - " special dill types. Ensure dill version '0.3.1.1'" \ - " is installed in the execution environment.") + raise RuntimeError(base_error + ". Dill is not installed.") if dill.__version__ != "0.3.1.1": - raise RuntimeError("Error verifying dill compatibility forencoding " \ - "deterministic special dill types. Ensure dill " - "version '0.3.1.1' is installed in the execution " \ - "environment. Found dill version '{dill.__version__}") + raise RuntimeError(base_error + f". Found dill version '{dill.__version__}") class FastPrimitivesCoderImpl(StreamCoderImpl): @@ -546,20 +545,23 @@ def _deterministic_encoding_error_msg(self, value): "please provide a type hint for the input of '%s'" % (value, type(value), self.requires_deterministic_step_label)) - def encode_type_2_66_0(self, t, stream): + def encode_type_2_67_0(self, t, stream): + """ + Encode special type with <=2.67.0 compatibility. + """ _verify_dill_compat() stream.write(dill.dumps(t), True) def encode_type(self, t, stream): if self.force_use_dill: - return self.encode_type_2_66_0(t, stream) + return self.encode_type_2_67_0(t, stream) bs = cloudpickle_pickler.dumps( t, config=cloudpickle_pickler.NO_DYNAMIC_CLASS_TRACKING_CONFIG) stream.write(bs, True) def decode_type(self, stream): if self.force_use_dill: - return _unpickle_type_2_66_0(stream.read_all(True)) + return _unpickle_type_2_67_0(stream.read_all(True)) return _unpickle_type(stream.read_all(True)) def decode_from_stream(self, stream, nested): @@ -620,7 +622,10 @@ def decode_from_stream(self, stream, nested): _unpickled_types = {} # type: Dict[bytes, type] -def _unpickle_type_2_66_0(bs): +def _unpickle_type_2_67_0(bs): + """ + Decode special type with <=2.67.0 compatibility. + """ t = _unpickled_types.get(bs, None) if t is None: _verify_dill_compat() @@ -631,12 +636,12 @@ def _unpickle_type_2_66_0(bs): pickle.loads(pickle.dumps(t)) except pickle.PicklingError: t.__reduce__ = lambda self: ( - _unpickle_named_tuple_2_66_0, (bs, tuple(self))) + _unpickle_named_tuple_2_67_0, (bs, tuple(self))) return t -def _unpickle_named_tuple_2_66_0(bs, items): - return _unpickle_type_2_66_0(bs)(*items) +def _unpickle_named_tuple_2_67_0(bs, items): + return _unpickle_type_2_67_0(bs)(*items) def _unpickle_type(bs): diff --git a/sdks/python/apache_beam/coders/coders.py b/sdks/python/apache_beam/coders/coders.py index 4f5df3a78d02..9ff130240aee 100644 --- a/sdks/python/apache_beam/coders/coders.py +++ b/sdks/python/apache_beam/coders/coders.py @@ -989,16 +989,17 @@ def to_type_hint(self): def _should_force_use_dill(update_compatibility_version): from apache_beam.transforms.util import is_v1_prior_to_v2 - if not is_v1_prior_to_v2(v1=update_compatibility_version, v2="2.67.0"): + if not is_v1_prior_to_v2(v1=update_compatibility_version, v2="2.68.0"): return False try: import dill assert dill.__version__ == "0.3.1.1" except Exception as e: - raise RuntimeError("Error using dill for encoding special types. Ensure" \ - " dill version '0.3.1.1' is installed in execution and" \ - f" runtime environment. Error {e}") + raise RuntimeError("This pipeline runs with the " \ + "update_compatibility_version=2.67.0 flag. When running with this flag " \ + "on SDKs 2.68.0 or higher, you must ensure dill==0.3.1.1 is installed. " \ + f"Error {e}") return True From 6961ceb7fec9cffa125857239bef510408717503 Mon Sep 17 00:00:00 2001 From: Claude Date: Thu, 31 Jul 2025 20:31:26 +0000 Subject: [PATCH 06/15] Fix test. --- sdks/python/apache_beam/runners/pipeline_context.py | 5 ++--- sdks/python/apache_beam/transforms/ptransform_test.py | 2 +- 2 files changed, 3 insertions(+), 4 deletions(-) diff --git a/sdks/python/apache_beam/runners/pipeline_context.py b/sdks/python/apache_beam/runners/pipeline_context.py index 8c17787ff48b..56267d7c57e9 100644 --- a/sdks/python/apache_beam/runners/pipeline_context.py +++ b/sdks/python/apache_beam/runners/pipeline_context.py @@ -225,7 +225,7 @@ def __init__( default_environment, label='default_environment') self.use_fake_coders = use_fake_coders - self.deterministic_coder_map: Mapping[coders.Coder, coders.Coder] = {} + self.deterministic_coder_map: Dict[coders.Coder, coders.Coder] = {} self.iterable_state_read = iterable_state_read self.iterable_state_write = iterable_state_write self._requirements = set(requirements) @@ -264,8 +264,7 @@ def deterministic_coder(self, coder: coders.Coder, msg: str) -> coders.Coder: pipeline_options.StreamingOptions ).update_compatibility_version if self.pipeline_options else None self.deterministic_coder_map[coder] = coder.as_deterministic_coder( - msg, - update_compatibility_version=update_compatibility_version) # type: ignore + msg, update_compatibility_version=update_compatibility_version) return self.deterministic_coder_map[coder] def element_type_from_coder_id(self, coder_id: str) -> Any: diff --git a/sdks/python/apache_beam/transforms/ptransform_test.py b/sdks/python/apache_beam/transforms/ptransform_test.py index e1c84c7dc9ae..5d9241265bca 100644 --- a/sdks/python/apache_beam/transforms/ptransform_test.py +++ b/sdks/python/apache_beam/transforms/ptransform_test.py @@ -572,7 +572,7 @@ def encode(self, o): def decode(self, encoded): return MyObject(pickle.loads(encoded)[0]) - def as_deterministic_coder(self, *args): + def as_deterministic_coder(self, *args, **kwargs): return MydeterministicObjectCoder() def to_type_hint(self): From 09d81da47cc16e4edc85f7f93ee27e2cb2fd8595 Mon Sep 17 00:00:00 2001 From: Claude Date: Fri, 1 Aug 2025 20:17:06 +0000 Subject: [PATCH 07/15] Trigger tests. --- ...ommit_Python_ValidatesRunner_Dataflow.json | 3 +- .../python/apache_beam/metrics/metric_test.py | 4 +- .../apache_beam/ml/inference/base_test.py | 2 +- sdks/python/apache_beam/pipeline_test.py | 4 +- ...xercise_streaming_metrics_pipeline_test.py | 2 +- .../fn_api_runner/translations_test.py | 8 +- .../apache_beam/transforms/ptransform_test.py | 93 ++++++++++++++++--- .../apache_beam/transforms/sideinputs_test.py | 24 ++--- .../apache_beam/transforms/util_test.py | 2 +- 9 files changed, 107 insertions(+), 35 deletions(-) diff --git a/.github/trigger_files/beam_PostCommit_Python_ValidatesRunner_Dataflow.json b/.github/trigger_files/beam_PostCommit_Python_ValidatesRunner_Dataflow.json index 19ebbfb9ad92..e3d6056a5de9 100644 --- a/.github/trigger_files/beam_PostCommit_Python_ValidatesRunner_Dataflow.json +++ b/.github/trigger_files/beam_PostCommit_Python_ValidatesRunner_Dataflow.json @@ -1,3 +1,4 @@ { - "https://github.com/apache/beam/pull/35951": "triggering sideinput test" + "comment": "Modify this file in a trivial way to cause this test suite to run", + "modification": 1 } diff --git a/sdks/python/apache_beam/metrics/metric_test.py b/sdks/python/apache_beam/metrics/metric_test.py index bdba0512dfa2..dc27eea93dc3 100644 --- a/sdks/python/apache_beam/metrics/metric_test.py +++ b/sdks/python/apache_beam/metrics/metric_test.py @@ -20,7 +20,7 @@ import unittest import hamcrest as hc -import pytest +# import pytest import apache_beam as beam from apache_beam import metrics @@ -151,7 +151,7 @@ def test_general_urn_metric_name_str(self): "urn=my_urn, labels={'key': 'value'})") self.assertEqual(str(mn), expected_str) - @pytest.mark.it_validatesrunner + # @pytest.mark.it_validatesrunner def test_user_counter_using_pardo(self): class SomeDoFn(beam.DoFn): """A custom dummy DoFn using yield.""" diff --git a/sdks/python/apache_beam/ml/inference/base_test.py b/sdks/python/apache_beam/ml/inference/base_test.py index 64fd73682e13..3e590f5253b2 100644 --- a/sdks/python/apache_beam/ml/inference/base_test.py +++ b/sdks/python/apache_beam/ml/inference/base_test.py @@ -1564,7 +1564,7 @@ def process(self, element): "SideInputs to RunInference are only supported in streaming mode.") @pytest.mark.it_postcommit @pytest.mark.sickbay_direct - @pytest.mark.it_validatesrunner + # @pytest.mark.it_validatesrunner def test_run_inference_with_side_inputin_streaming(self): test_pipeline = TestPipeline(is_integration_test=True) test_pipeline.options.view_as(StandardOptions).streaming = True diff --git a/sdks/python/apache_beam/pipeline_test.py b/sdks/python/apache_beam/pipeline_test.py index dc0d9a7cc58f..babff196659a 100644 --- a/sdks/python/apache_beam/pipeline_test.py +++ b/sdks/python/apache_beam/pipeline_test.py @@ -823,7 +823,7 @@ def process(self, element, prefix, suffix=DoFn.SideInputParam): TestDoFn(), prefix, suffix=AsSingleton(suffix)) assert_that(result, equal_to(['zyx-%s-xyz' % x for x in words_list])) - @pytest.mark.it_validatesrunner + # @pytest.mark.it_validatesrunner def test_element_param(self): pipeline = TestPipeline() input = [1, 2] @@ -834,7 +834,7 @@ def test_element_param(self): assert_that(pcoll, equal_to(input)) pipeline.run() - @pytest.mark.it_validatesrunner + # @pytest.mark.it_validatesrunner def test_key_param(self): pipeline = TestPipeline() pcoll = ( diff --git a/sdks/python/apache_beam/runners/dataflow/dataflow_exercise_streaming_metrics_pipeline_test.py b/sdks/python/apache_beam/runners/dataflow/dataflow_exercise_streaming_metrics_pipeline_test.py index 83bb35034642..6cbc2fc88b97 100644 --- a/sdks/python/apache_beam/runners/dataflow/dataflow_exercise_streaming_metrics_pipeline_test.py +++ b/sdks/python/apache_beam/runners/dataflow/dataflow_exercise_streaming_metrics_pipeline_test.py @@ -115,7 +115,7 @@ def run_pipeline(self): return dataflow_exercise_streaming_metrics_pipeline.run(argv) # Need not run streaming test in batch mode. - @pytest.mark.it_validatesrunner + # @pytest.mark.it_validatesrunner @pytest.mark.no_sickbay_batch @pytest.mark.no_xdist def test_streaming_pipeline_returns_expected_user_metrics_fnapi_it(self): diff --git a/sdks/python/apache_beam/runners/portability/fn_api_runner/translations_test.py b/sdks/python/apache_beam/runners/portability/fn_api_runner/translations_test.py index 8cbf1b610a5e..1eb26cdcaf3c 100644 --- a/sdks/python/apache_beam/runners/portability/fn_api_runner/translations_test.py +++ b/sdks/python/apache_beam/runners/portability/fn_api_runner/translations_test.py @@ -19,7 +19,7 @@ import logging import unittest -import pytest +# import pytest import apache_beam as beam from apache_beam import runners @@ -249,7 +249,7 @@ def assert_is_topologically_sorted(transform_id, visited_pcolls): assert_is_topologically_sorted( optimized_pipeline_proto.root_transform_ids[0], set()) - @pytest.mark.it_validatesrunner + # @pytest.mark.it_validatesrunner def test_run_packable_combine_per_key(self): class MultipleCombines(beam.PTransform): def annotations(self): @@ -279,7 +279,7 @@ def expand(self, pcoll): | Create([('a', x) for x in vals]) | 'multiple-combines' >> MultipleCombines()) - @pytest.mark.it_validatesrunner + # @pytest.mark.it_validatesrunner def test_run_packable_combine_globally(self): class MultipleCombines(beam.PTransform): def annotations(self): @@ -307,7 +307,7 @@ def expand(self, pcoll): vals = [6, 3, 1, -1, 9, 1, 5, 2, 0, 6] _ = pipeline | Create(vals) | 'multiple-combines' >> MultipleCombines() - @pytest.mark.it_validatesrunner + # @pytest.mark.it_validatesrunner def test_run_packable_combine_limit(self): class MultipleLargeCombines(beam.PTransform): def annotations(self): diff --git a/sdks/python/apache_beam/transforms/ptransform_test.py b/sdks/python/apache_beam/transforms/ptransform_test.py index 5d9241265bca..3908bfade578 100644 --- a/sdks/python/apache_beam/transforms/ptransform_test.py +++ b/sdks/python/apache_beam/transforms/ptransform_test.py @@ -34,6 +34,8 @@ import hamcrest as hc import numpy as np import pytest +from parameterized import param +from parameterized import parameterized from parameterized import parameterized_class import apache_beam as beam @@ -44,6 +46,11 @@ from apache_beam.metrics import Metrics from apache_beam.metrics.metric import MetricsFilter from apache_beam.options.pipeline_options import PipelineOptions +<<<<<<< HEAD +======= +from apache_beam.options.pipeline_options import StandardOptions +from apache_beam.options.pipeline_options import StreamingOptions +>>>>>>> Trigger tests. from apache_beam.options.pipeline_options import TypeOptions from apache_beam.portability import common_urns from apache_beam.testing.test_pipeline import TestPipeline @@ -63,6 +70,7 @@ from apache_beam.typehints.typehints_test import TypeHintTestCase from apache_beam.utils.timestamp import Timestamp from apache_beam.utils.windowed_value import WindowedValue +from apache_beam.coders import coders_test_common # Disable frequent lint warning due to pipe operator for chaining transforms. # pylint: disable=expression-not-assigned @@ -186,7 +194,7 @@ def test_do_with_multiple_outputs_maintains_unique_name(self): assert_that(r1.m, equal_to([2, 3, 4]), label='r1') assert_that(r2.m, equal_to([3, 4, 5]), label='r2') - @pytest.mark.it_validatesrunner + # @pytest.mark.it_validatesrunner def test_impulse(self): with TestPipeline() as pipeline: result = pipeline | beam.Impulse() | beam.Map(lambda _: 0) @@ -195,7 +203,7 @@ def test_impulse(self): # TODO(BEAM-3544): Disable this test in streaming temporarily. # Remove sickbay-streaming tag after it's resolved. @pytest.mark.no_sickbay_streaming - @pytest.mark.it_validatesrunner + # @pytest.mark.it_validatesrunner def test_read_metrics(self): from apache_beam.io.utils import CountingSource @@ -222,7 +230,7 @@ def process(self, element): self.assertEqual(outputs_counter.key.metric.name, 'recordsRead') self.assertEqual(outputs_counter.committed, 100) - @pytest.mark.it_validatesrunner + # @pytest.mark.it_validatesrunner def test_par_do_with_multiple_outputs_and_using_yield(self): class SomeDoFn(beam.DoFn): """A custom DoFn using yield.""" @@ -241,7 +249,7 @@ def process(self, element): assert_that(results.odd, equal_to([1, 3]), label='assert:odd') assert_that(results.even, equal_to([2, 4]), label='assert:even') - @pytest.mark.it_validatesrunner + # @pytest.mark.it_validatesrunner def test_par_do_with_multiple_outputs_and_using_return(self): def some_fn(v): if v % 2 == 0: @@ -256,7 +264,7 @@ def some_fn(v): assert_that(results.odd, equal_to([1, 3]), label='assert:odd') assert_that(results.even, equal_to([2, 4]), label='assert:even') - @pytest.mark.it_validatesrunner + # @pytest.mark.it_validatesrunner def test_undeclared_outputs(self): with TestPipeline() as pipeline: nums = pipeline | 'Some Numbers' >> beam.Create([1, 2, 3, 4]) @@ -269,7 +277,7 @@ def test_undeclared_outputs(self): assert_that(results.odd, equal_to([1, 3]), label='assert:odd') assert_that(results.even, equal_to([2, 4]), label='assert:even') - @pytest.mark.it_validatesrunner + # @pytest.mark.it_validatesrunner def test_multiple_empty_outputs(self): with TestPipeline() as pipeline: nums = pipeline | 'Some Numbers' >> beam.Create([1, 3, 5]) @@ -696,7 +704,7 @@ def test_partition_followed_by_flatten_and_groupbykey(self): grouped = flattened | 'D' >> beam.GroupByKey() | SortLists assert_that(grouped, equal_to([('aa', [1, 2]), ('bb', [2])])) - @pytest.mark.it_validatesrunner + # @pytest.mark.it_validatesrunner def test_flatten_pcollections(self): with TestPipeline() as pipeline: pcoll_1 = pipeline | 'Start 1' >> beam.Create([0, 1, 2, 3]) @@ -711,7 +719,7 @@ def test_flatten_no_pcollections(self): result = () | 'Empty' >> beam.Flatten(pipeline=pipeline) assert_that(result, equal_to([])) - @pytest.mark.it_validatesrunner + # @pytest.mark.it_validatesrunner def test_flatten_one_single_pcollection(self): with TestPipeline() as pipeline: input = [0, 1, 2, 3] @@ -719,10 +727,73 @@ def test_flatten_one_single_pcollection(self): result = (pcoll, ) | 'Single Flatten' >> beam.Flatten() assert_that(result, equal_to(input)) + @parameterized.expand([ + param(compat_version=None), + param(compat_version="2.66.0"), + ]) + @pytest.mark.it_validatesrunner + def test_group_by_key_importable_special_types(self, compat_version): + def generate(_): + for _ in range(100): + yield (coders_test_common.MyTypedNamedTuple(1, 'a'), 1) + + pipeline = TestPipeline(is_integration_test=True) + if compat_version: + pipeline.get_pipeline_options().view_as( + StreamingOptions).update_compatibility_version = compat_version + with pipeline as p: + result = ( + p + | 'Create' >> beam.Create([i for i in range(100)]) + | 'Generate' >> beam.ParDo(generate) + | 'Reshuffle' >> beam.Reshuffle() + | 'GBK' >> beam.GroupByKey()) + assert_that( + result, + equal_to([( + coders_test_common.MyTypedNamedTuple(1, 'a'), + [1 for i in range(10000)])])) + + @parameterized.expand([ + param(compat_version=None), + param(compat_version="2.66.0"), + ]) + @pytest.mark.it_validatesrunner + def test_group_by_key_dynamic_special_types(self, compat_version): + # All non-deterministic cases fall back to PickleCoder which cannot pickle + # dynamic types. This test should be re-enabled when the fallback coder is + # changed to CloudPickleCoder. + self.skipTest( + 'Skipping until default fallback FastPrimitivesCoder changed.') + + def dynamic_named_tuple(): + return collections.namedtuple('DynamicNamedTuple', ['x', 'y']) + + def generate(_): + for _ in range(100): + yield (dynamic_named_tuple()(1, 'a'), 1) + + pipeline = TestPipeline(is_integration_test=True) + if compat_version: + pipeline.get_pipeline_options().view_as( + StreamingOptions).update_compatibility_version = compat_version + + with pipeline as p: + result = ( + p + | 'Create' >> beam.Create([i for i in range(100)]) + | 'Generate' >> beam.ParDo(generate) + | 'Reshuffle' >> beam.Reshuffle() + | 'GBK' >> beam.GroupByKey()) + assert_that( + result, + equal_to([(dynamic_named_tuple()(1, + 'a'), [1 for i in range(10000)])])) + # TODO(https://github.com/apache/beam/issues/20067): Does not work in # streaming mode on Dataflow. @pytest.mark.no_sickbay_streaming - @pytest.mark.it_validatesrunner + # @pytest.mark.it_validatesrunner def test_flatten_same_pcollections(self): with TestPipeline() as pipeline: pc = pipeline | beam.Create(['a', 'b']) @@ -735,7 +806,7 @@ def test_flatten_pcollections_in_iterable(self): result = [pcoll for pcoll in (pcoll_1, pcoll_2)] | beam.Flatten() assert_that(result, equal_to([0, 1, 2, 3, 4, 5, 6, 7])) - @pytest.mark.it_validatesrunner + # @pytest.mark.it_validatesrunner def test_flatten_a_flattened_pcollection(self): with TestPipeline() as pipeline: pcoll_1 = pipeline | 'Start 1' >> beam.Create([0, 1, 2, 3]) @@ -757,7 +828,7 @@ def test_flatten_input_type_must_be_iterable_of_pcolls(self): with self.assertRaises(TypeError): set([1, 2, 3]) | beam.Flatten() - @pytest.mark.it_validatesrunner + # @pytest.mark.it_validatesrunner def test_flatten_multiple_pcollections_having_multiple_consumers(self): with TestPipeline() as pipeline: input = pipeline | 'Start' >> beam.Create(['AA', 'BBB', 'CC']) diff --git a/sdks/python/apache_beam/transforms/sideinputs_test.py b/sdks/python/apache_beam/transforms/sideinputs_test.py index 5f3cf761e1eb..598984c7874b 100644 --- a/sdks/python/apache_beam/transforms/sideinputs_test.py +++ b/sdks/python/apache_beam/transforms/sideinputs_test.py @@ -150,7 +150,7 @@ def test_windowed_dict(self): }), ]) - @pytest.mark.it_validatesrunner + # @pytest.mark.it_validatesrunner def test_empty_singleton_side_input(self): pipeline = self.create_pipeline() pcol = pipeline | 'start' >> beam.Create([1, 2]) @@ -169,7 +169,7 @@ def my_fn(k, s): # TODO(https://github.com/apache/beam/issues/19012): Disable this test in # streaming temporarily. Remove sickbay-streaming tag after it's fixed. @pytest.mark.no_sickbay_streaming - @pytest.mark.it_validatesrunner + # @pytest.mark.it_validatesrunner def test_multi_valued_singleton_side_input(self): pipeline = self.create_pipeline() pcol = pipeline | 'start' >> beam.Create([1, 2]) @@ -179,7 +179,7 @@ def test_multi_valued_singleton_side_input(self): with self.assertRaises(Exception): pipeline.run() - @pytest.mark.it_validatesrunner + # @pytest.mark.it_validatesrunner def test_default_value_singleton_side_input(self): pipeline = self.create_pipeline() pcol = pipeline | 'start' >> beam.Create([1, 2]) @@ -189,7 +189,7 @@ def test_default_value_singleton_side_input(self): assert_that(result, equal_to([10, 20])) pipeline.run() - @pytest.mark.it_validatesrunner + # @pytest.mark.it_validatesrunner def test_iterable_side_input(self): pipeline = self.create_pipeline() pcol = pipeline | 'start' >> beam.Create([1, 2]) @@ -199,7 +199,7 @@ def test_iterable_side_input(self): assert_that(result, equal_to([3, 4, 6, 8])) pipeline.run() - @pytest.mark.it_validatesrunner + # @pytest.mark.it_validatesrunner def test_reiterable_side_input(self): expected_side = frozenset(range(100)) @@ -226,7 +226,7 @@ def check_reiteration(main, side): _ = pcol | 'check' >> beam.Map(check_reiteration, beam.pvalue.AsIter(side)) pipeline.run() - @pytest.mark.it_validatesrunner + # @pytest.mark.it_validatesrunner def test_as_list_and_as_dict_side_inputs(self): a_list = [5, 1, 3, 2, 9] some_pairs = [('crouton', 17), ('supreme', None)] @@ -251,7 +251,7 @@ def match(actual): assert_that(results, matcher(1, a_list, some_pairs)) pipeline.run() - @pytest.mark.it_validatesrunner + # @pytest.mark.it_validatesrunner def test_as_singleton_without_unique_labels(self): # This should succeed as calling beam.pvalue.AsSingleton on the same # PCollection twice with the same defaults will return the same @@ -277,7 +277,7 @@ def match(actual): assert_that(results, matcher(1, 2)) pipeline.run() - @pytest.mark.it_validatesrunner + # @pytest.mark.it_validatesrunner def test_as_singleton_with_different_defaults(self): a_list = [] pipeline = self.create_pipeline() @@ -300,7 +300,7 @@ def match(actual): assert_that(results, matcher(1, 2, 3)) pipeline.run() - @pytest.mark.it_validatesrunner + # @pytest.mark.it_validatesrunner def test_as_list_twice(self): # This should succeed as calling beam.pvalue.AsList on the same # PCollection twice will return the same view. @@ -325,7 +325,7 @@ def match(actual): assert_that(results, matcher(1, [1, 2, 3])) pipeline.run() - @pytest.mark.it_validatesrunner + # @pytest.mark.it_validatesrunner def test_as_dict_twice(self): some_kvs = [('a', 1), ('b', 2)] pipeline = self.create_pipeline() @@ -348,7 +348,7 @@ def match(actual): assert_that(results, matcher(1, some_kvs)) pipeline.run() - @pytest.mark.it_validatesrunner + # @pytest.mark.it_validatesrunner def test_flattened_side_input(self): pipeline = self.create_pipeline() main_input = pipeline | 'main input' >> beam.Create([None]) @@ -364,7 +364,7 @@ def test_flattened_side_input(self): # TODO(BEAM-9499): Disable this test in streaming temporarily. @pytest.mark.no_sickbay_batch @pytest.mark.no_sickbay_streaming - @pytest.mark.it_validatesrunner + # @pytest.mark.it_validatesrunner def test_multi_triggered_gbk_side_input(self): """Test a GBK sideinput, with multiple triggering.""" # TODO(https://github.com/apache/beam/issues/20065): Remove use of this diff --git a/sdks/python/apache_beam/transforms/util_test.py b/sdks/python/apache_beam/transforms/util_test.py index ad185ac6a6d1..46f8efa10dcd 100644 --- a/sdks/python/apache_beam/transforms/util_test.py +++ b/sdks/python/apache_beam/transforms/util_test.py @@ -1176,7 +1176,7 @@ def test_reshuffle_default_window_preserves_metadata(self, compat_version): label='CheckMetadataPreserved', reify_windows=True) - @pytest.mark.it_validatesrunner + # @pytest.mark.it_validatesrunner def test_reshuffle_preserves_timestamps(self): with TestPipeline() as pipeline: From 6495db638c207aa7f38ac91d2b20f182b616b4a5 Mon Sep 17 00:00:00 2001 From: Claude Date: Mon, 25 Aug 2025 16:28:51 +0000 Subject: [PATCH 08/15] Undo disable tests. --- .../apache_beam/ml/inference/base_test.py | 2 +- sdks/python/apache_beam/pipeline_test.py | 4 +- ...xercise_streaming_metrics_pipeline_test.py | 2 +- .../apache_beam/transforms/ptransform_test.py | 50 +++++++++++-------- .../apache_beam/transforms/sideinputs_test.py | 24 ++++----- .../apache_beam/transforms/util_test.py | 2 +- 6 files changed, 45 insertions(+), 39 deletions(-) diff --git a/sdks/python/apache_beam/ml/inference/base_test.py b/sdks/python/apache_beam/ml/inference/base_test.py index 3e590f5253b2..64fd73682e13 100644 --- a/sdks/python/apache_beam/ml/inference/base_test.py +++ b/sdks/python/apache_beam/ml/inference/base_test.py @@ -1564,7 +1564,7 @@ def process(self, element): "SideInputs to RunInference are only supported in streaming mode.") @pytest.mark.it_postcommit @pytest.mark.sickbay_direct - # @pytest.mark.it_validatesrunner + @pytest.mark.it_validatesrunner def test_run_inference_with_side_inputin_streaming(self): test_pipeline = TestPipeline(is_integration_test=True) test_pipeline.options.view_as(StandardOptions).streaming = True diff --git a/sdks/python/apache_beam/pipeline_test.py b/sdks/python/apache_beam/pipeline_test.py index babff196659a..dc0d9a7cc58f 100644 --- a/sdks/python/apache_beam/pipeline_test.py +++ b/sdks/python/apache_beam/pipeline_test.py @@ -823,7 +823,7 @@ def process(self, element, prefix, suffix=DoFn.SideInputParam): TestDoFn(), prefix, suffix=AsSingleton(suffix)) assert_that(result, equal_to(['zyx-%s-xyz' % x for x in words_list])) - # @pytest.mark.it_validatesrunner + @pytest.mark.it_validatesrunner def test_element_param(self): pipeline = TestPipeline() input = [1, 2] @@ -834,7 +834,7 @@ def test_element_param(self): assert_that(pcoll, equal_to(input)) pipeline.run() - # @pytest.mark.it_validatesrunner + @pytest.mark.it_validatesrunner def test_key_param(self): pipeline = TestPipeline() pcoll = ( diff --git a/sdks/python/apache_beam/runners/dataflow/dataflow_exercise_streaming_metrics_pipeline_test.py b/sdks/python/apache_beam/runners/dataflow/dataflow_exercise_streaming_metrics_pipeline_test.py index 6cbc2fc88b97..83bb35034642 100644 --- a/sdks/python/apache_beam/runners/dataflow/dataflow_exercise_streaming_metrics_pipeline_test.py +++ b/sdks/python/apache_beam/runners/dataflow/dataflow_exercise_streaming_metrics_pipeline_test.py @@ -115,7 +115,7 @@ def run_pipeline(self): return dataflow_exercise_streaming_metrics_pipeline.run(argv) # Need not run streaming test in batch mode. - # @pytest.mark.it_validatesrunner + @pytest.mark.it_validatesrunner @pytest.mark.no_sickbay_batch @pytest.mark.no_xdist def test_streaming_pipeline_returns_expected_user_metrics_fnapi_it(self): diff --git a/sdks/python/apache_beam/transforms/ptransform_test.py b/sdks/python/apache_beam/transforms/ptransform_test.py index 3908bfade578..634b8bb091af 100644 --- a/sdks/python/apache_beam/transforms/ptransform_test.py +++ b/sdks/python/apache_beam/transforms/ptransform_test.py @@ -194,7 +194,7 @@ def test_do_with_multiple_outputs_maintains_unique_name(self): assert_that(r1.m, equal_to([2, 3, 4]), label='r1') assert_that(r2.m, equal_to([3, 4, 5]), label='r2') - # @pytest.mark.it_validatesrunner + @pytest.mark.it_validatesrunner def test_impulse(self): with TestPipeline() as pipeline: result = pipeline | beam.Impulse() | beam.Map(lambda _: 0) @@ -203,7 +203,7 @@ def test_impulse(self): # TODO(BEAM-3544): Disable this test in streaming temporarily. # Remove sickbay-streaming tag after it's resolved. @pytest.mark.no_sickbay_streaming - # @pytest.mark.it_validatesrunner + @pytest.mark.it_validatesrunner def test_read_metrics(self): from apache_beam.io.utils import CountingSource @@ -230,7 +230,7 @@ def process(self, element): self.assertEqual(outputs_counter.key.metric.name, 'recordsRead') self.assertEqual(outputs_counter.committed, 100) - # @pytest.mark.it_validatesrunner + @pytest.mark.it_validatesrunner def test_par_do_with_multiple_outputs_and_using_yield(self): class SomeDoFn(beam.DoFn): """A custom DoFn using yield.""" @@ -249,7 +249,7 @@ def process(self, element): assert_that(results.odd, equal_to([1, 3]), label='assert:odd') assert_that(results.even, equal_to([2, 4]), label='assert:even') - # @pytest.mark.it_validatesrunner + @pytest.mark.it_validatesrunner def test_par_do_with_multiple_outputs_and_using_return(self): def some_fn(v): if v % 2 == 0: @@ -264,7 +264,7 @@ def some_fn(v): assert_that(results.odd, equal_to([1, 3]), label='assert:odd') assert_that(results.even, equal_to([2, 4]), label='assert:even') - # @pytest.mark.it_validatesrunner + @pytest.mark.it_validatesrunner def test_undeclared_outputs(self): with TestPipeline() as pipeline: nums = pipeline | 'Some Numbers' >> beam.Create([1, 2, 3, 4]) @@ -277,7 +277,7 @@ def test_undeclared_outputs(self): assert_that(results.odd, equal_to([1, 3]), label='assert:odd') assert_that(results.even, equal_to([2, 4]), label='assert:even') - # @pytest.mark.it_validatesrunner + @pytest.mark.it_validatesrunner def test_multiple_empty_outputs(self): with TestPipeline() as pipeline: nums = pipeline | 'Some Numbers' >> beam.Create([1, 3, 5]) @@ -704,7 +704,7 @@ def test_partition_followed_by_flatten_and_groupbykey(self): grouped = flattened | 'D' >> beam.GroupByKey() | SortLists assert_that(grouped, equal_to([('aa', [1, 2]), ('bb', [2])])) - # @pytest.mark.it_validatesrunner + @pytest.mark.it_validatesrunner def test_flatten_pcollections(self): with TestPipeline() as pipeline: pcoll_1 = pipeline | 'Start 1' >> beam.Create([0, 1, 2, 3]) @@ -719,7 +719,7 @@ def test_flatten_no_pcollections(self): result = () | 'Empty' >> beam.Flatten(pipeline=pipeline) assert_that(result, equal_to([])) - # @pytest.mark.it_validatesrunner + @pytest.mark.it_validatesrunner def test_flatten_one_single_pcollection(self): with TestPipeline() as pipeline: input = [0, 1, 2, 3] @@ -760,18 +760,24 @@ def generate(_): ]) @pytest.mark.it_validatesrunner def test_group_by_key_dynamic_special_types(self, compat_version): - # All non-deterministic cases fall back to PickleCoder which cannot pickle - # dynamic types. This test should be re-enabled when the fallback coder is - # changed to CloudPickleCoder. - self.skipTest( - 'Skipping until default fallback FastPrimitivesCoder changed.') - - def dynamic_named_tuple(): + def create_dynamic_named_tuple(): return collections.namedtuple('DynamicNamedTuple', ['x', 'y']) + dynamic_named_tuple = create_dynamic_named_tuple() + + # Standard FastPrimitivesCoder falls back to python PickleCoder which + # cannot serialize dynamic types or types defined in __main__. Use + # CloudPickleCoder as fallbac coder for non-deterministic steps. + class FastPrimitivesCoderV2(beam.coders.FastPrimitivesCoder): + def __init__(self): + super().__init__(fallback_coder=beam.coders.CloudpickleCoder()) + + beam.coders.typecoders.registry.register_coder( + dynamic_named_tuple, FastPrimitivesCoderV2) + def generate(_): for _ in range(100): - yield (dynamic_named_tuple()(1, 'a'), 1) + yield (dynamic_named_tuple(1, 'a'), 1) pipeline = TestPipeline(is_integration_test=True) if compat_version: @@ -782,18 +788,18 @@ def generate(_): result = ( p | 'Create' >> beam.Create([i for i in range(100)]) - | 'Generate' >> beam.ParDo(generate) + | 'Generate' >> beam.ParDo(generate).with_output_types( + tuple[dynamic_named_tuple, int]) | 'Reshuffle' >> beam.Reshuffle() | 'GBK' >> beam.GroupByKey()) assert_that( result, - equal_to([(dynamic_named_tuple()(1, - 'a'), [1 for i in range(10000)])])) + equal_to([(dynamic_named_tuple(1, 'a'), [1 for i in range(10000)])])) # TODO(https://github.com/apache/beam/issues/20067): Does not work in # streaming mode on Dataflow. @pytest.mark.no_sickbay_streaming - # @pytest.mark.it_validatesrunner + @pytest.mark.it_validatesrunner def test_flatten_same_pcollections(self): with TestPipeline() as pipeline: pc = pipeline | beam.Create(['a', 'b']) @@ -806,7 +812,7 @@ def test_flatten_pcollections_in_iterable(self): result = [pcoll for pcoll in (pcoll_1, pcoll_2)] | beam.Flatten() assert_that(result, equal_to([0, 1, 2, 3, 4, 5, 6, 7])) - # @pytest.mark.it_validatesrunner + @pytest.mark.it_validatesrunner def test_flatten_a_flattened_pcollection(self): with TestPipeline() as pipeline: pcoll_1 = pipeline | 'Start 1' >> beam.Create([0, 1, 2, 3]) @@ -828,7 +834,7 @@ def test_flatten_input_type_must_be_iterable_of_pcolls(self): with self.assertRaises(TypeError): set([1, 2, 3]) | beam.Flatten() - # @pytest.mark.it_validatesrunner + @pytest.mark.it_validatesrunner def test_flatten_multiple_pcollections_having_multiple_consumers(self): with TestPipeline() as pipeline: input = pipeline | 'Start' >> beam.Create(['AA', 'BBB', 'CC']) diff --git a/sdks/python/apache_beam/transforms/sideinputs_test.py b/sdks/python/apache_beam/transforms/sideinputs_test.py index 598984c7874b..5f3cf761e1eb 100644 --- a/sdks/python/apache_beam/transforms/sideinputs_test.py +++ b/sdks/python/apache_beam/transforms/sideinputs_test.py @@ -150,7 +150,7 @@ def test_windowed_dict(self): }), ]) - # @pytest.mark.it_validatesrunner + @pytest.mark.it_validatesrunner def test_empty_singleton_side_input(self): pipeline = self.create_pipeline() pcol = pipeline | 'start' >> beam.Create([1, 2]) @@ -169,7 +169,7 @@ def my_fn(k, s): # TODO(https://github.com/apache/beam/issues/19012): Disable this test in # streaming temporarily. Remove sickbay-streaming tag after it's fixed. @pytest.mark.no_sickbay_streaming - # @pytest.mark.it_validatesrunner + @pytest.mark.it_validatesrunner def test_multi_valued_singleton_side_input(self): pipeline = self.create_pipeline() pcol = pipeline | 'start' >> beam.Create([1, 2]) @@ -179,7 +179,7 @@ def test_multi_valued_singleton_side_input(self): with self.assertRaises(Exception): pipeline.run() - # @pytest.mark.it_validatesrunner + @pytest.mark.it_validatesrunner def test_default_value_singleton_side_input(self): pipeline = self.create_pipeline() pcol = pipeline | 'start' >> beam.Create([1, 2]) @@ -189,7 +189,7 @@ def test_default_value_singleton_side_input(self): assert_that(result, equal_to([10, 20])) pipeline.run() - # @pytest.mark.it_validatesrunner + @pytest.mark.it_validatesrunner def test_iterable_side_input(self): pipeline = self.create_pipeline() pcol = pipeline | 'start' >> beam.Create([1, 2]) @@ -199,7 +199,7 @@ def test_iterable_side_input(self): assert_that(result, equal_to([3, 4, 6, 8])) pipeline.run() - # @pytest.mark.it_validatesrunner + @pytest.mark.it_validatesrunner def test_reiterable_side_input(self): expected_side = frozenset(range(100)) @@ -226,7 +226,7 @@ def check_reiteration(main, side): _ = pcol | 'check' >> beam.Map(check_reiteration, beam.pvalue.AsIter(side)) pipeline.run() - # @pytest.mark.it_validatesrunner + @pytest.mark.it_validatesrunner def test_as_list_and_as_dict_side_inputs(self): a_list = [5, 1, 3, 2, 9] some_pairs = [('crouton', 17), ('supreme', None)] @@ -251,7 +251,7 @@ def match(actual): assert_that(results, matcher(1, a_list, some_pairs)) pipeline.run() - # @pytest.mark.it_validatesrunner + @pytest.mark.it_validatesrunner def test_as_singleton_without_unique_labels(self): # This should succeed as calling beam.pvalue.AsSingleton on the same # PCollection twice with the same defaults will return the same @@ -277,7 +277,7 @@ def match(actual): assert_that(results, matcher(1, 2)) pipeline.run() - # @pytest.mark.it_validatesrunner + @pytest.mark.it_validatesrunner def test_as_singleton_with_different_defaults(self): a_list = [] pipeline = self.create_pipeline() @@ -300,7 +300,7 @@ def match(actual): assert_that(results, matcher(1, 2, 3)) pipeline.run() - # @pytest.mark.it_validatesrunner + @pytest.mark.it_validatesrunner def test_as_list_twice(self): # This should succeed as calling beam.pvalue.AsList on the same # PCollection twice will return the same view. @@ -325,7 +325,7 @@ def match(actual): assert_that(results, matcher(1, [1, 2, 3])) pipeline.run() - # @pytest.mark.it_validatesrunner + @pytest.mark.it_validatesrunner def test_as_dict_twice(self): some_kvs = [('a', 1), ('b', 2)] pipeline = self.create_pipeline() @@ -348,7 +348,7 @@ def match(actual): assert_that(results, matcher(1, some_kvs)) pipeline.run() - # @pytest.mark.it_validatesrunner + @pytest.mark.it_validatesrunner def test_flattened_side_input(self): pipeline = self.create_pipeline() main_input = pipeline | 'main input' >> beam.Create([None]) @@ -364,7 +364,7 @@ def test_flattened_side_input(self): # TODO(BEAM-9499): Disable this test in streaming temporarily. @pytest.mark.no_sickbay_batch @pytest.mark.no_sickbay_streaming - # @pytest.mark.it_validatesrunner + @pytest.mark.it_validatesrunner def test_multi_triggered_gbk_side_input(self): """Test a GBK sideinput, with multiple triggering.""" # TODO(https://github.com/apache/beam/issues/20065): Remove use of this diff --git a/sdks/python/apache_beam/transforms/util_test.py b/sdks/python/apache_beam/transforms/util_test.py index 46f8efa10dcd..ad185ac6a6d1 100644 --- a/sdks/python/apache_beam/transforms/util_test.py +++ b/sdks/python/apache_beam/transforms/util_test.py @@ -1176,7 +1176,7 @@ def test_reshuffle_default_window_preserves_metadata(self, compat_version): label='CheckMetadataPreserved', reify_windows=True) - # @pytest.mark.it_validatesrunner + @pytest.mark.it_validatesrunner def test_reshuffle_preserves_timestamps(self): with TestPipeline() as pipeline: From 4433332c48aa1f6ce7b4a2501f8d81b67c06eabf Mon Sep 17 00:00:00 2001 From: Claude Date: Tue, 26 Aug 2025 14:29:39 +0000 Subject: [PATCH 09/15] Fix test. --- sdks/python/apache_beam/coders/coder_impl.py | 2 +- sdks/python/apache_beam/metrics/metric_test.py | 4 ++-- .../portability/fn_api_runner/translations_test.py | 8 ++++---- sdks/python/apache_beam/transforms/ptransform_test.py | 11 +++++------ 4 files changed, 12 insertions(+), 13 deletions(-) diff --git a/sdks/python/apache_beam/coders/coder_impl.py b/sdks/python/apache_beam/coders/coder_impl.py index 41ced15183bb..88b1a8b75155 100644 --- a/sdks/python/apache_beam/coders/coder_impl.py +++ b/sdks/python/apache_beam/coders/coder_impl.py @@ -363,7 +363,7 @@ def _verify_dill_compat(): base_error = ( "This pipeline runs with the update_compatibility_version=2.67.0 flag. " "When running with this flag on SDKs 2.68.0 or higher, you must " - "ensure dill==0.3.1.1 is installed. Dill is not installed.") + "ensure dill==0.3.1.1 is installed.") if not dill: raise RuntimeError(base_error + ". Dill is not installed.") if dill.__version__ != "0.3.1.1": diff --git a/sdks/python/apache_beam/metrics/metric_test.py b/sdks/python/apache_beam/metrics/metric_test.py index dc27eea93dc3..bdba0512dfa2 100644 --- a/sdks/python/apache_beam/metrics/metric_test.py +++ b/sdks/python/apache_beam/metrics/metric_test.py @@ -20,7 +20,7 @@ import unittest import hamcrest as hc -# import pytest +import pytest import apache_beam as beam from apache_beam import metrics @@ -151,7 +151,7 @@ def test_general_urn_metric_name_str(self): "urn=my_urn, labels={'key': 'value'})") self.assertEqual(str(mn), expected_str) - # @pytest.mark.it_validatesrunner + @pytest.mark.it_validatesrunner def test_user_counter_using_pardo(self): class SomeDoFn(beam.DoFn): """A custom dummy DoFn using yield.""" diff --git a/sdks/python/apache_beam/runners/portability/fn_api_runner/translations_test.py b/sdks/python/apache_beam/runners/portability/fn_api_runner/translations_test.py index 1eb26cdcaf3c..8cbf1b610a5e 100644 --- a/sdks/python/apache_beam/runners/portability/fn_api_runner/translations_test.py +++ b/sdks/python/apache_beam/runners/portability/fn_api_runner/translations_test.py @@ -19,7 +19,7 @@ import logging import unittest -# import pytest +import pytest import apache_beam as beam from apache_beam import runners @@ -249,7 +249,7 @@ def assert_is_topologically_sorted(transform_id, visited_pcolls): assert_is_topologically_sorted( optimized_pipeline_proto.root_transform_ids[0], set()) - # @pytest.mark.it_validatesrunner + @pytest.mark.it_validatesrunner def test_run_packable_combine_per_key(self): class MultipleCombines(beam.PTransform): def annotations(self): @@ -279,7 +279,7 @@ def expand(self, pcoll): | Create([('a', x) for x in vals]) | 'multiple-combines' >> MultipleCombines()) - # @pytest.mark.it_validatesrunner + @pytest.mark.it_validatesrunner def test_run_packable_combine_globally(self): class MultipleCombines(beam.PTransform): def annotations(self): @@ -307,7 +307,7 @@ def expand(self, pcoll): vals = [6, 3, 1, -1, 9, 1, 5, 2, 0, 6] _ = pipeline | Create(vals) | 'multiple-combines' >> MultipleCombines() - # @pytest.mark.it_validatesrunner + @pytest.mark.it_validatesrunner def test_run_packable_combine_limit(self): class MultipleLargeCombines(beam.PTransform): def annotations(self): diff --git a/sdks/python/apache_beam/transforms/ptransform_test.py b/sdks/python/apache_beam/transforms/ptransform_test.py index 634b8bb091af..f3753dbf256f 100644 --- a/sdks/python/apache_beam/transforms/ptransform_test.py +++ b/sdks/python/apache_beam/transforms/ptransform_test.py @@ -767,7 +767,7 @@ def create_dynamic_named_tuple(): # Standard FastPrimitivesCoder falls back to python PickleCoder which # cannot serialize dynamic types or types defined in __main__. Use - # CloudPickleCoder as fallbac coder for non-deterministic steps. + # CloudPickleCoder as fallback coder for non-deterministic steps. class FastPrimitivesCoderV2(beam.coders.FastPrimitivesCoder): def __init__(self): super().__init__(fallback_coder=beam.coders.CloudpickleCoder()) @@ -788,13 +788,12 @@ def generate(_): result = ( p | 'Create' >> beam.Create([i for i in range(100)]) + | 'Reshuffle' >> beam.Reshuffle() | 'Generate' >> beam.ParDo(generate).with_output_types( tuple[dynamic_named_tuple, int]) - | 'Reshuffle' >> beam.Reshuffle() - | 'GBK' >> beam.GroupByKey()) - assert_that( - result, - equal_to([(dynamic_named_tuple(1, 'a'), [1 for i in range(10000)])])) + | 'GBK' >> beam.GroupByKey() + | 'Count Elements' >> beam.Map(lambda x: len(x[1]))) + assert_that(result, equal_to([10000])) # TODO(https://github.com/apache/beam/issues/20067): Does not work in # streaming mode on Dataflow. From be14f7ca46d34b48be01f39950fe306694efd99e Mon Sep 17 00:00:00 2001 From: Claude Date: Tue, 26 Aug 2025 17:26:07 +0000 Subject: [PATCH 10/15] Fix tests and lint. --- sdks/python/apache_beam/transforms/ptransform_test.py | 11 ++--------- 1 file changed, 2 insertions(+), 9 deletions(-) diff --git a/sdks/python/apache_beam/transforms/ptransform_test.py b/sdks/python/apache_beam/transforms/ptransform_test.py index f3753dbf256f..c044e0d4a562 100644 --- a/sdks/python/apache_beam/transforms/ptransform_test.py +++ b/sdks/python/apache_beam/transforms/ptransform_test.py @@ -42,6 +42,7 @@ import apache_beam.transforms.combiners as combine from apache_beam import pvalue from apache_beam import typehints +from apache_beam.coders import coders_test_common from apache_beam.io.iobase import Read from apache_beam.metrics import Metrics from apache_beam.metrics.metric import MetricsFilter @@ -70,7 +71,6 @@ from apache_beam.typehints.typehints_test import TypeHintTestCase from apache_beam.utils.timestamp import Timestamp from apache_beam.utils.windowed_value import WindowedValue -from apache_beam.coders import coders_test_common # Disable frequent lint warning due to pipe operator for chaining transforms. # pylint: disable=expression-not-assigned @@ -754,12 +754,8 @@ def generate(_): coders_test_common.MyTypedNamedTuple(1, 'a'), [1 for i in range(10000)])])) - @parameterized.expand([ - param(compat_version=None), - param(compat_version="2.66.0"), - ]) @pytest.mark.it_validatesrunner - def test_group_by_key_dynamic_special_types(self, compat_version): + def test_group_by_key_dynamic_special_types(self): def create_dynamic_named_tuple(): return collections.namedtuple('DynamicNamedTuple', ['x', 'y']) @@ -780,9 +776,6 @@ def generate(_): yield (dynamic_named_tuple(1, 'a'), 1) pipeline = TestPipeline(is_integration_test=True) - if compat_version: - pipeline.get_pipeline_options().view_as( - StreamingOptions).update_compatibility_version = compat_version with pipeline as p: result = ( From 2a045f58168e7c2f5c78b6a00bff7754bc56d5a3 Mon Sep 17 00:00:00 2001 From: Claude Date: Fri, 29 Aug 2025 14:53:13 -0400 Subject: [PATCH 11/15] Refactor. --- sdks/python/apache_beam/coders/coders.py | 69 +++++------ .../apache_beam/coders/coders_test_common.py | 112 +++--------------- sdks/python/apache_beam/coders/row_coder.py | 19 +-- sdks/python/apache_beam/coders/typecoders.py | 1 + sdks/python/apache_beam/pipeline.py | 12 +- .../apache_beam/runners/pipeline_context.py | 11 +- sdks/python/apache_beam/transforms/util.py | 11 +- .../apache_beam/transforms/util_test.py | 62 ++++++++++ 8 files changed, 123 insertions(+), 174 deletions(-) diff --git a/sdks/python/apache_beam/coders/coders.py b/sdks/python/apache_beam/coders/coders.py index 9ff130240aee..cdd93aff08fd 100644 --- a/sdks/python/apache_beam/coders/coders.py +++ b/sdks/python/apache_beam/coders/coders.py @@ -180,8 +180,7 @@ def is_deterministic(self): """ return False - def as_deterministic_coder( - self, step_label, error_message=None, update_compatibility_version=None): + def as_deterministic_coder(self, step_label, error_message=None): """Returns a deterministic version of self, if possible. Otherwise raises a value error. @@ -532,13 +531,10 @@ def is_deterministic(self): # Map ordering is non-deterministic return False - def as_deterministic_coder( - self, step_label, error_message=None, update_compatibility_version=None): + def as_deterministic_coder(self, step_label, error_message=None): return DeterministicMapCoder( - self._key_coder.as_deterministic_coder( - step_label, error_message, update_compatibility_version), - self._value_coder.as_deterministic_coder( - step_label, error_message, update_compatibility_version)) + self._key_coder.as_deterministic_coder(step_label, error_message), + self._value_coder.as_deterministic_coder(step_label, error_message)) def __eq__(self, other): return ( @@ -613,13 +609,12 @@ def is_deterministic(self): # type: () -> bool return self._value_coder.is_deterministic() - def as_deterministic_coder( - self, step_label, error_message=None, update_compatibility_version=None): + def as_deterministic_coder(self, step_label, error_message=None): if self.is_deterministic(): return self else: deterministic_value_coder = self._value_coder.as_deterministic_coder( - step_label, error_message, update_compatibility_version) + step_label, error_message) return NullableCoder(deterministic_value_coder) def __eq__(self, other): @@ -881,8 +876,7 @@ def _nonhashable_dumps(x): return coder_impl.CallbackCoderImpl(_nonhashable_dumps, pickler.loads) - def as_deterministic_coder( - self, step_label, error_message=None, update_compatibility_version=None): + def as_deterministic_coder(self, step_label, error_message=None): return FastPrimitivesCoder(self, requires_deterministic=step_label) def to_type_hint(self): @@ -897,8 +891,7 @@ def _create_impl(self): return coder_impl.CallbackCoderImpl( lambda x: dumps(x, protocol), pickle.loads) - def as_deterministic_coder( - self, step_label, error_message=None, update_compatibility_version=None): + def as_deterministic_coder(self, step_label, error_message=None): return FastPrimitivesCoder(self, requires_deterministic=step_label) def to_type_hint(self): @@ -986,10 +979,12 @@ def to_type_hint(self): return Any -def _should_force_use_dill(update_compatibility_version): +def _should_force_use_dill(): + from apache_beam.coders import typecoders from apache_beam.transforms.util import is_v1_prior_to_v2 - if not is_v1_prior_to_v2(v1=update_compatibility_version, v2="2.68.0"): + if not is_v1_prior_to_v2(v1=typecoders.registry.update_compatibility_version, + v2="2.68.0"): return False try: @@ -997,15 +992,14 @@ def _should_force_use_dill(update_compatibility_version): assert dill.__version__ == "0.3.1.1" except Exception as e: raise RuntimeError("This pipeline runs with the " \ - "update_compatibility_version=2.67.0 flag. When running with this flag " \ - "on SDKs 2.68.0 or higher, you must ensure dill==0.3.1.1 is installed. " \ - f"Error {e}") + "--update_compatibility_version=2.67.0 or earlier. When running with " \ + "this option on SDKs 2.68.0 or higher, you must ensure dill==0.3.1.1 " \ + f"is installed. Error {e}") return True -def _update_compatible_deterministic_fast_primitives_coder( - coder, step_label, update_compatibility_version): - if _should_force_use_dill(update_compatibility_version): +def _update_compatible_deterministic_fast_primitives_coder(coder, step_label): + if _should_force_use_dill(): return DeterministicFastPrimitivesCoder(coder, step_label) return DeterministicFastPrimitivesCoderV2(coder, step_label) @@ -1026,13 +1020,12 @@ def is_deterministic(self): # type: () -> bool return self._fallback_coder.is_deterministic() - def as_deterministic_coder( - self, step_label, error_message=None, update_compatibility_version=None): + def as_deterministic_coder(self, step_label, error_message=None): if self.is_deterministic(): return self else: return _update_compatible_deterministic_fast_primitives_coder( - self, step_label, update_compatibility_version) + self, step_label) def to_type_hint(self): return Any @@ -1127,8 +1120,7 @@ def is_deterministic(self): # a Map. return False - def as_deterministic_coder( - self, step_label, error_message=None, update_compatibility_version=None): + def as_deterministic_coder(self, step_label, error_message=None): return DeterministicProtoCoder(self.proto_message_type) def __eq__(self, other): @@ -1174,8 +1166,7 @@ def is_deterministic(self): # type: () -> bool return True - def as_deterministic_coder( - self, step_label, error_message=None, update_compatibility_version=None): + def as_deterministic_coder(self, step_label, error_message=None): return self @@ -1262,14 +1253,12 @@ def is_deterministic(self): # type: () -> bool return all(c.is_deterministic() for c in self._coders) - def as_deterministic_coder( - self, step_label, error_message=None, update_compatibility_version=None): + def as_deterministic_coder(self, step_label, error_message=None): if self.is_deterministic(): return self else: return TupleCoder([ - c.as_deterministic_coder( - step_label, error_message, update_compatibility_version) + c.as_deterministic_coder(step_label, error_message) for c in self._coders ]) @@ -1343,14 +1332,12 @@ def is_deterministic(self): # type: () -> bool return self._elem_coder.is_deterministic() - def as_deterministic_coder( - self, step_label, error_message=None, update_compatibility_version=None): + def as_deterministic_coder(self, step_label, error_message=None): if self.is_deterministic(): return self else: return TupleSequenceCoder( - self._elem_coder.as_deterministic_coder( - step_label, error_message, update_compatibility_version)) + self._elem_coder.as_deterministic_coder(step_label, error_message)) @classmethod def from_type_hint(cls, typehint, registry): @@ -1385,14 +1372,12 @@ def is_deterministic(self): # type: () -> bool return self._elem_coder.is_deterministic() - def as_deterministic_coder( - self, step_label, error_message=None, update_compatibility_version=None): + def as_deterministic_coder(self, step_label, error_message=None): if self.is_deterministic(): return self else: return type(self)( - self._elem_coder.as_deterministic_coder( - step_label, error_message, update_compatibility_version)) + self._elem_coder.as_deterministic_coder(step_label, error_message)) def value_coder(self): return self._elem_coder diff --git a/sdks/python/apache_beam/coders/coders_test_common.py b/sdks/python/apache_beam/coders/coders_test_common.py index b2a985950bd6..587e5d87522e 100644 --- a/sdks/python/apache_beam/coders/coders_test_common.py +++ b/sdks/python/apache_beam/coders/coders_test_common.py @@ -178,6 +178,9 @@ def tearDownClass(cls): assert not standard - cls.seen, str(standard - cls.seen) assert not cls.seen_nested - standard, str(cls.seen_nested - standard) + def tearDown(self): + typecoders.registry.update_compatibility_version = None + @classmethod def _observe(cls, coder): cls.seen.add(type(coder)) @@ -235,12 +238,12 @@ def test_memoizing_pickle_coder(self): @parameterized.expand([ param(compat_version=None), - param(compat_version="2.66.0"), + param(compat_version="2.67.0"), ]) def test_deterministic_coder(self, compat_version): + typecoders.registry.update_compatibility_version = compat_version coder = coders.FastPrimitivesCoder() - deterministic_coder = coder.as_deterministic_coder( - step_label="step", update_compatibility_version=compat_version) + deterministic_coder = coder.as_deterministic_coder(step_label="step") self.check_coder(deterministic_coder, *self.test_values_deterministic) for v in self.test_values_deterministic: @@ -306,9 +309,10 @@ def test_deterministic_coder(self, compat_version): @parameterized.expand([ param(compat_version=None), - param(compat_version="2.66.0"), + param(compat_version="2.67.0"), ]) def test_deterministic_map_coder_is_update_compatible(self, compat_version): + typecoders.registry.update_compatibility_version = compat_version values = [{ MyTypedNamedTuple(i, 'a'): MyTypedNamedTuple('a', i) for i in range(10) @@ -317,8 +321,7 @@ def test_deterministic_map_coder_is_update_compatible(self, compat_version): coder = coders.MapCoder( coders.FastPrimitivesCoder(), coders.FastPrimitivesCoder()) - deterministic_coder = coder.as_deterministic_coder( - step_label="step", update_compatibility_version=compat_version) + deterministic_coder = coder.as_deterministic_coder(step_label="step") assert isinstance( deterministic_coder._key_coder, @@ -327,90 +330,6 @@ def test_deterministic_map_coder_is_update_compatible(self, compat_version): self.check_coder(deterministic_coder, *values) - @parameterized.expand([ - param(compat_version=None), - param(compat_version="2.66.0"), - ]) - def test_deterministic_nullable_coder_is_update_compatible( - self, compat_version): - values = [MyTypedNamedTuple(1, 'a'), None, MyTypedNamedTuple(2, 'a')] - - coder = coders.NullableCoder(coders.FastPrimitivesCoder()) - - deterministic_coder = coder.as_deterministic_coder( - step_label="step", update_compatibility_version=compat_version) - - assert isinstance( - deterministic_coder._value_coder, - coders.DeterministicFastPrimitivesCoderV2 - if not compat_version else coders.DeterministicFastPrimitivesCoder) - - self.check_coder(deterministic_coder, *values) - - @parameterized.expand([ - param(compat_version=None), - param(compat_version="2.66.0"), - ]) - def test_deterministic_tuple_coder_is_update_compatible(self, compat_version): - values = [MyTypedNamedTuple(1, 'a'), MyTypedNamedTuple(2, 'a')] - - coder = coders.TupleCoder( - [coders.FastPrimitivesCoder(), coders.FastPrimitivesCoder()]) - - deterministic_coder = coder.as_deterministic_coder( - step_label="step", update_compatibility_version=compat_version) - - assert all( - isinstance( - component_coder, - coders.DeterministicFastPrimitivesCoderV2 - if not compat_version else coders.DeterministicFastPrimitivesCoder) - for component_coder in deterministic_coder._coders) - - self.check_coder(deterministic_coder, *values) - - @parameterized.expand([ - param(compat_version=None), - param(compat_version="2.66.0"), - ]) - def test_deterministic_tuplesequence_coder_is_update_compatible( - self, compat_version): - values = [MyTypedNamedTuple(1, 'a'), MyTypedNamedTuple(2, 'a')] - - coder = coders.TupleSequenceCoder(coders.FastPrimitivesCoder()) - - deterministic_coder = coder.as_deterministic_coder( - step_label="step", update_compatibility_version=compat_version) - - assert isinstance( - deterministic_coder._elem_coder, - coders.DeterministicFastPrimitivesCoderV2 - if not compat_version else coders.DeterministicFastPrimitivesCoder) - - self.check_coder(deterministic_coder, *values) - - @parameterized.expand([ - param(compat_version=None), - param(compat_version="2.66.0"), - ]) - def test_deterministic_listlike_coder_is_update_compatible( - self, compat_version): - _ = [MyTypedNamedTuple(1, 'a'), MyTypedNamedTuple(2, 'a')] - - coder = coders.ListLikeCoder(coders.FastPrimitivesCoder()) - - deterministic_coder = coder.as_deterministic_coder( - step_label="step", update_compatibility_version=compat_version) - - assert isinstance( - deterministic_coder._elem_coder, - coders.DeterministicFastPrimitivesCoderV2 - if not compat_version else coders.DeterministicFastPrimitivesCoder) - - # This check fails for both compat options with - # AssertionError: MyTypedNamedTuple(f1=1, f2='a') != [1, 'a'] - # self.check_coder(deterministic_coder, *values) - def test_dill_coder(self): cell_value = (lambda x: lambda: x)(0).__closure__[0] self.check_coder(coders.DillCoder(), 'a', 1, cell_value) @@ -736,8 +655,8 @@ def test_param_windowed_value_coder(self): PaneInfo(True, False, 1, 2, 3)))) @parameterized.expand([ + param(compat_version=None), param(compat_version="2.67.0"), - param(compat_version="2.66.0"), ]) def test_cross_process_encoding_of_special_types_is_deterministic( self, compat_version): @@ -745,7 +664,7 @@ def test_cross_process_encoding_of_special_types_is_deterministic( if sys.executable is None: self.skipTest('No Python interpreter found') - + typecoders.registry.update_compatibility_version = compat_version # pylint: disable=line-too-long script = textwrap.dedent( @@ -757,6 +676,7 @@ def test_cross_process_encoding_of_special_types_is_deterministic( import logging from apache_beam.coders import coders + from apache_beam.coders import typecoders from apache_beam.coders.coders_test_common import MyNamedTuple from apache_beam.coders.coders_test_common import MyTypedNamedTuple from apache_beam.coders.coders_test_common import MyEnum @@ -803,9 +723,10 @@ def test_cross_process_encoding_of_special_types_is_deterministic( ("frozen_dataclass_list", [FrozenDataClass(1, 2), FrozenDataClass(3, 4)]), ]) - compat_version = "{compat_version}" + compat_version = {'"'+ compat_version +'"' if compat_version else None} + typecoders.registry.update_compatibility_version = compat_version coder = coders.FastPrimitivesCoder() - deterministic_coder = coder.as_deterministic_coder("step", update_compatibility_version=compat_version) + deterministic_coder = coder.as_deterministic_coder("step") results = dict() for test_name, value in test_cases: @@ -835,8 +756,7 @@ def run_subprocess(): results2 = run_subprocess() coder = coders.FastPrimitivesCoder() - deterministic_coder = coder.as_deterministic_coder( - "step", update_compatibility_version=compat_version) + deterministic_coder = coder.as_deterministic_coder("step") for test_name in results1: data1 = results1[test_name] diff --git a/sdks/python/apache_beam/coders/row_coder.py b/sdks/python/apache_beam/coders/row_coder.py index 6ffd183f4e84..1becf408cfbf 100644 --- a/sdks/python/apache_beam/coders/row_coder.py +++ b/sdks/python/apache_beam/coders/row_coder.py @@ -52,11 +52,7 @@ class RowCoder(FastCoder): Implements the beam:coder:row:v1 standard coder spec. """ - def __init__( - self, - schema, - force_deterministic=False, - update_compatibility_version=None): + def __init__(self, schema, force_deterministic=False): """Initializes a :class:`RowCoder`. Args: @@ -75,10 +71,7 @@ def __init__( ] if force_deterministic: self.components = [ - c.as_deterministic_coder( - force_deterministic, - update_compatibility_version=update_compatibility_version) - for c in self.components + c.as_deterministic_coder(force_deterministic) for c in self.components ] self.forced_deterministic = bool(force_deterministic) @@ -88,15 +81,11 @@ def _create_impl(self): def is_deterministic(self): return all(c.is_deterministic() for c in self.components) - def as_deterministic_coder( - self, step_label, error_message=None, update_compatibility_version=None): + def as_deterministic_coder(self, step_label, error_message=None): if self.is_deterministic(): return self else: - return RowCoder( - self.schema, - error_message or step_label, - update_compatibility_version=update_compatibility_version) + return RowCoder(self.schema, error_message or step_label) def to_type_hint(self): return self._type_hint diff --git a/sdks/python/apache_beam/coders/typecoders.py b/sdks/python/apache_beam/coders/typecoders.py index 19300c675596..779c65dc772c 100644 --- a/sdks/python/apache_beam/coders/typecoders.py +++ b/sdks/python/apache_beam/coders/typecoders.py @@ -84,6 +84,7 @@ def __init__(self, fallback_coder=None): self._coders: Dict[Any, Type[coders.Coder]] = {} self.custom_types: List[Any] = [] self.register_standard_coders(fallback_coder) + self.update_compatibility_version = None def register_standard_coders(self, fallback_coder): """Register coders for all basic and composite types.""" diff --git a/sdks/python/apache_beam/pipeline.py b/sdks/python/apache_beam/pipeline.py index f5a7d86288fc..0ed5a435e788 100644 --- a/sdks/python/apache_beam/pipeline.py +++ b/sdks/python/apache_beam/pipeline.py @@ -76,6 +76,7 @@ from google.protobuf import message from apache_beam import pvalue +from apache_beam.coders import typecoders from apache_beam.internal import pickler from apache_beam.io.filesystems import FileSystems from apache_beam.options.pipeline_options import CrossLanguageOptions @@ -83,6 +84,7 @@ from apache_beam.options.pipeline_options import PipelineOptions from apache_beam.options.pipeline_options import SetupOptions from apache_beam.options.pipeline_options import StandardOptions +from apache_beam.options.pipeline_options import StreamingOptions from apache_beam.options.pipeline_options import TypeOptions from apache_beam.options.pipeline_options_validator import PipelineOptionsValidator from apache_beam.portability import common_urns @@ -229,6 +231,9 @@ def __init__( raise ValueError( 'Pipeline has validations errors: \n' + '\n'.join(errors)) + typecoders.registry.update_compatibility_version = self._options.view_as( + StreamingOptions).update_compatibility_version + # set default experiments for portable runners # (needs to occur prior to pipeline construction) if runner.is_fnapi_compatible(): @@ -989,8 +994,7 @@ def to_runner_api( context = pipeline_context.PipelineContext( use_fake_coders=use_fake_coders, component_id_map=self.component_id_map, - default_environment=default_environment, - pipeline_options=self._options) + default_environment=default_environment) elif default_environment is not None: raise ValueError( 'Only one of context or default_environment may be specified.') @@ -1093,9 +1097,7 @@ def from_runner_api( }) from apache_beam.runners import pipeline_context context = pipeline_context.PipelineContext( - proto.components, - requirements=proto.requirements, - pipeline_options=options) + proto.components, requirements=proto.requirements) if proto.root_transform_ids: root_transform_id, = proto.root_transform_ids p.transforms_stack = [context.transforms.get_by_id(root_transform_id)] diff --git a/sdks/python/apache_beam/runners/pipeline_context.py b/sdks/python/apache_beam/runners/pipeline_context.py index 56267d7c57e9..132a1aedca33 100644 --- a/sdks/python/apache_beam/runners/pipeline_context.py +++ b/sdks/python/apache_beam/runners/pipeline_context.py @@ -43,7 +43,6 @@ from apache_beam.coders.coder_impl import IterableStateReader from apache_beam.coders.coder_impl import IterableStateWriter from apache_beam.internal import pickler -from apache_beam.options import pipeline_options from apache_beam.pipeline import ComponentIdMap from apache_beam.portability.api import beam_fn_api_pb2 from apache_beam.portability.api import beam_runner_api_pb2 @@ -178,9 +177,7 @@ def __init__( iterable_state_write: Optional[IterableStateWriter] = None, namespace: str = 'ref', requirements: Iterable[str] = (), - pipeline_options: Optional[pipeline_options.PipelineOptions] = None ) -> None: - self.pipeline_options = pipeline_options if isinstance(proto, beam_fn_api_pb2.ProcessBundleDescriptor): proto = beam_runner_api_pb2.Components( coders=dict(proto.coders.items()), @@ -225,7 +222,7 @@ def __init__( default_environment, label='default_environment') self.use_fake_coders = use_fake_coders - self.deterministic_coder_map: Dict[coders.Coder, coders.Coder] = {} + self.deterministic_coder_map: Mapping[coders.Coder, coders.Coder] = {} self.iterable_state_read = iterable_state_read self.iterable_state_write = iterable_state_write self._requirements = set(requirements) @@ -260,11 +257,7 @@ def coder_id_from_element_type( def deterministic_coder(self, coder: coders.Coder, msg: str) -> coders.Coder: if coder not in self.deterministic_coder_map: - update_compatibility_version = self.pipeline_options.view_as( - pipeline_options.StreamingOptions - ).update_compatibility_version if self.pipeline_options else None - self.deterministic_coder_map[coder] = coder.as_deterministic_coder( - msg, update_compatibility_version=update_compatibility_version) + self.deterministic_coder_map[coder] = coder.as_deterministic_coder(msg) # type: ignore return self.deterministic_coder_map[coder] def element_type_from_coder_id(self, coder_id: str) -> Any: diff --git a/sdks/python/apache_beam/transforms/util.py b/sdks/python/apache_beam/transforms/util.py index bbd2815b84a2..2df66aadcc64 100644 --- a/sdks/python/apache_beam/transforms/util.py +++ b/sdks/python/apache_beam/transforms/util.py @@ -932,16 +932,13 @@ def get_window_coder(self): return self._window_coder -def is_v1_prior_to_v2(v1, v2): +def is_v1_prior_to_v2(*, v1, v2): if v1 is None: return False - v1 = tuple(map(int, v1.split('.')[0:3])) - v2 = tuple(map(int, v2.split('.')[0:3])) - for i in range(min(len(v1), len(v2))): - if v1[i] < v2[i]: - return True - return False + v1_parts = (v1.split('.') + ['0', '0', '0'])[:3] + v2_parts = (v2.split('.') + ['0', '0', '0'])[:3] + return tuple(map(int, v1_parts)) < tuple(map(int, v2_parts)) def is_compat_version_prior_to(options, breaking_change_version): diff --git a/sdks/python/apache_beam/transforms/util_test.py b/sdks/python/apache_beam/transforms/util_test.py index ad185ac6a6d1..b365d9b22090 100644 --- a/sdks/python/apache_beam/transforms/util_test.py +++ b/sdks/python/apache_beam/transforms/util_test.py @@ -2193,6 +2193,68 @@ def record(tag): label='result') +class CompatCheckTest(unittest.TestCase): + def test_is_v1_prior_to_v2(self): + test_cases = [ + # Basic comparison cases + ("1.0.0", "2.0.0", True), # v1 < v2 in major + ("2.0.0", "1.0.0", False), # v1 > v2 in major + ("1.1.0", "1.2.0", True), # v1 < v2 in minor + ("1.2.0", "1.1.0", False), # v1 > v2 in minor + ("1.0.1", "1.0.2", True), # v1 < v2 in patch + ("1.0.2", "1.0.1", False), # v1 > v2 in patch + + # Equal versions + ("1.0.0", "1.0.0", False), # Identical + ("0.0.0", "0.0.0", False), # Both zero + + # Different lengths - shorter vs longer + ("1.0", "1.0.0", False), # Should be equal (1.0 = 1.0.0) + ("1.0", "1.0.1", True), # 1.0.0 < 1.0.1 + ("1.2", "1.2.0", False), # Should be equal (1.2 = 1.2.0) + ("1.2", "1.2.3", True), # 1.2.0 < 1.2.3 + ("2", "2.0.0", False), # Should be equal (2 = 2.0.0) + ("2", "2.0.1", True), # 2.0.0 < 2.0.1 + ("1", "2.0", True), # 1.0.0 < 2.0.0 + + # Different lengths - longer vs shorter + ("1.0.0", "1.0", False), # Should be equal + ("1.0.1", "1.0", False), # 1.0.1 > 1.0.0 + ("1.2.0", "1.2", False), # Should be equal + ("1.2.3", "1.2", False), # 1.2.3 > 1.2.0 + ("2.0.0", "2", False), # Should be equal + ("2.0.1", "2", False), # 2.0.1 > 2.0.0 + ("2.0", "1", False), # 2.0.0 > 1.0.0 + + # Mixed length comparisons + ("1.0", "2.0.0", True), # 1.0.0 < 2.0.0 + ("2.0", "1.0.0", False), # 2.0.0 > 1.0.0 + ("1", "1.0.1", True), # 1.0.0 < 1.0.1 + ("1.1", "1.0.9", False), # 1.1.0 > 1.0.9 + + # Large numbers + ("1.9.9", "2.0.0", True), # 1.9.9 < 2.0.0 + ("10.0.0", "9.9.9", False), # 10.0.0 > 9.9.9 + ("1.10.0", "1.9.0", False), # 1.10.0 > 1.9.0 + ("1.2.10", "1.2.9", False), # 1.2.10 > 1.2.9 + + # Sequential versions + ("1.0.0", "1.0.1", True), + ("1.0.1", "1.0.2", True), + ("1.0.9", "1.1.0", True), + ("1.9.9", "2.0.0", True), + + # Null/None cases + (None, "1.0.0", False), # v1 is None + ] + + for v1, v2, expected in test_cases: + self.assertEqual( + util.is_v1_prior_to_v2(v1=v1, v2=v2), + expected, + msg=f"Failed {v1} < {v2} == {expected}") + + if __name__ == '__main__': logging.getLogger().setLevel(logging.INFO) unittest.main() From 13ab3c021586122e22752255e0c64592bc194e7f Mon Sep 17 00:00:00 2001 From: Claude Date: Fri, 29 Aug 2025 17:22:37 -0400 Subject: [PATCH 12/15] Rebase and comments. --- sdks/python/apache_beam/coders/coder_impl.py | 6 +++--- sdks/python/apache_beam/coders/coders.py | 9 ++++++--- sdks/python/apache_beam/transforms/ptransform_test.py | 6 +----- 3 files changed, 10 insertions(+), 11 deletions(-) diff --git a/sdks/python/apache_beam/coders/coder_impl.py b/sdks/python/apache_beam/coders/coder_impl.py index 88b1a8b75155..50a3ef43af70 100644 --- a/sdks/python/apache_beam/coders/coder_impl.py +++ b/sdks/python/apache_beam/coders/coder_impl.py @@ -361,9 +361,9 @@ def decode(self, value): def _verify_dill_compat(): base_error = ( - "This pipeline runs with the update_compatibility_version=2.67.0 flag. " - "When running with this flag on SDKs 2.68.0 or higher, you must " - "ensure dill==0.3.1.1 is installed.") + "This pipeline runs with the update_compatibility_version=2.67.0 " + "pipeline option. When running with this option on SDKs 2.68.0 or " + "higher, you must ensure dill==0.3.1.1 is installed.") if not dill: raise RuntimeError(base_error + ". Dill is not installed.") if dill.__version__ != "0.3.1.1": diff --git a/sdks/python/apache_beam/coders/coders.py b/sdks/python/apache_beam/coders/coders.py index cdd93aff08fd..53c92e2523df 100644 --- a/sdks/python/apache_beam/coders/coders.py +++ b/sdks/python/apache_beam/coders/coders.py @@ -982,16 +982,19 @@ def to_type_hint(self): def _should_force_use_dill(): from apache_beam.coders import typecoders from apache_beam.transforms.util import is_v1_prior_to_v2 + update_compat_version = typecoders.registry.update_compatibility_version - if not is_v1_prior_to_v2(v1=typecoders.registry.update_compatibility_version, - v2="2.68.0"): + if not update_compat_version: + return False + + if not is_v1_prior_to_v2(v1=update_compat_version, v2="2.68.0"): return False try: import dill assert dill.__version__ == "0.3.1.1" except Exception as e: - raise RuntimeError("This pipeline runs with the " \ + raise RuntimeError("This pipeline runs with the pipeline option " \ "--update_compatibility_version=2.67.0 or earlier. When running with " \ "this option on SDKs 2.68.0 or higher, you must ensure dill==0.3.1.1 " \ f"is installed. Error {e}") diff --git a/sdks/python/apache_beam/transforms/ptransform_test.py b/sdks/python/apache_beam/transforms/ptransform_test.py index c044e0d4a562..533fd113ca47 100644 --- a/sdks/python/apache_beam/transforms/ptransform_test.py +++ b/sdks/python/apache_beam/transforms/ptransform_test.py @@ -47,12 +47,8 @@ from apache_beam.metrics import Metrics from apache_beam.metrics.metric import MetricsFilter from apache_beam.options.pipeline_options import PipelineOptions -<<<<<<< HEAD -======= -from apache_beam.options.pipeline_options import StandardOptions -from apache_beam.options.pipeline_options import StreamingOptions ->>>>>>> Trigger tests. from apache_beam.options.pipeline_options import TypeOptions +from apache_beam.options.pipeline_options import StreamingOptions from apache_beam.portability import common_urns from apache_beam.testing.test_pipeline import TestPipeline from apache_beam.testing.test_stream import TestStream From 7cb81fc27dab89959b8afd791a88bce4689716e6 Mon Sep 17 00:00:00 2001 From: Claude Date: Fri, 29 Aug 2025 18:56:55 -0400 Subject: [PATCH 13/15] Lint fix. --- sdks/python/apache_beam/transforms/ptransform_test.py | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/sdks/python/apache_beam/transforms/ptransform_test.py b/sdks/python/apache_beam/transforms/ptransform_test.py index 533fd113ca47..39d216c4b3b4 100644 --- a/sdks/python/apache_beam/transforms/ptransform_test.py +++ b/sdks/python/apache_beam/transforms/ptransform_test.py @@ -47,8 +47,8 @@ from apache_beam.metrics import Metrics from apache_beam.metrics.metric import MetricsFilter from apache_beam.options.pipeline_options import PipelineOptions -from apache_beam.options.pipeline_options import TypeOptions from apache_beam.options.pipeline_options import StreamingOptions +from apache_beam.options.pipeline_options import TypeOptions from apache_beam.portability import common_urns from apache_beam.testing.test_pipeline import TestPipeline from apache_beam.testing.test_stream import TestStream From 7e5ef598fece1ea2c293c0fd7568943cfa018654 Mon Sep 17 00:00:00 2001 From: tvalentyn Date: Sat, 30 Aug 2025 09:07:01 +0200 Subject: [PATCH 14/15] Update the base message to include possible lower values for the flag. --- sdks/python/apache_beam/coders/coder_impl.py | 7 ++++--- 1 file changed, 4 insertions(+), 3 deletions(-) diff --git a/sdks/python/apache_beam/coders/coder_impl.py b/sdks/python/apache_beam/coders/coder_impl.py index 50a3ef43af70..4f28fb3c916b 100644 --- a/sdks/python/apache_beam/coders/coder_impl.py +++ b/sdks/python/apache_beam/coders/coder_impl.py @@ -361,9 +361,10 @@ def decode(self, value): def _verify_dill_compat(): base_error = ( - "This pipeline runs with the update_compatibility_version=2.67.0 " - "pipeline option. When running with this option on SDKs 2.68.0 or " - "higher, you must ensure dill==0.3.1.1 is installed.") + "This pipeline runs with the pipeline option " + "--update_compatibility_version=2.67.0 or earlier. " + "When running with this option on SDKs 2.68.0 or " + "later, you must ensure dill==0.3.1.1 is installed.") if not dill: raise RuntimeError(base_error + ". Dill is not installed.") if dill.__version__ != "0.3.1.1": From da634633447d7abf9c0ba1106a5aef4d0e5f64d4 Mon Sep 17 00:00:00 2001 From: tvalentyn Date: Sat, 30 Aug 2025 09:09:03 +0200 Subject: [PATCH 15/15] Update sdks/python/apache_beam/coders/coders.py --- sdks/python/apache_beam/coders/coders.py | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/sdks/python/apache_beam/coders/coders.py b/sdks/python/apache_beam/coders/coders.py index 53c92e2523df..e527185bd571 100644 --- a/sdks/python/apache_beam/coders/coders.py +++ b/sdks/python/apache_beam/coders/coders.py @@ -996,7 +996,7 @@ def _should_force_use_dill(): except Exception as e: raise RuntimeError("This pipeline runs with the pipeline option " \ "--update_compatibility_version=2.67.0 or earlier. When running with " \ - "this option on SDKs 2.68.0 or higher, you must ensure dill==0.3.1.1 " \ + "this option on SDKs 2.68.0 or later, you must ensure dill==0.3.1.1 " \ f"is installed. Error {e}") return True