github.com/apache/beam/sdks/v2@v2.48.2/python/apache_beam/coders/coders.py (about) 1 # 2 # Licensed to the Apache Software Foundation (ASF) under one or more 3 # contributor license agreements. See the NOTICE file distributed with 4 # this work for additional information regarding copyright ownership. 5 # The ASF licenses this file to You under the Apache License, Version 2.0 6 # (the "License"); you may not use this file except in compliance with 7 # the License. You may obtain a copy of the License at 8 # 9 # http://www.apache.org/licenses/LICENSE-2.0 10 # 11 # Unless required by applicable law or agreed to in writing, software 12 # distributed under the License is distributed on an "AS IS" BASIS, 13 # WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. 14 # See the License for the specific language governing permissions and 15 # limitations under the License. 16 # 17 18 """Collection of useful coders. 19 20 Only those coders listed in __all__ are part of the public API of this module. 21 22 ## On usage of `pickle`, `dill` and `pickler` in Beam 23 24 In Beam, we generally we use `pickle` for pipeline elements and `dill` for 25 more complex types, like user functions. 26 27 `pickler` is Beam's own wrapping of dill + compression + error handling. 28 It serves also as an API to mask the actual encoding layer (so we can 29 change it from `dill` if necessary). 30 31 We created `_MemoizingPickleCoder` to improve performance when serializing 32 complex user types for the execution of SDF. Specifically to address 33 BEAM-12781, where many identical `BoundedSource` instances are being 34 encoded. 35 36 """ 37 # pytype: skip-file 38 39 import base64 40 import decimal 41 import pickle 42 from functools import lru_cache 43 from typing import TYPE_CHECKING 44 from typing import Any 45 from typing import Callable 46 from typing import Dict 47 from typing import Iterable 48 from typing import List 49 from typing import Optional 50 from typing import Sequence 51 from typing import Tuple 52 from typing import Type 53 from typing import TypeVar 54 from typing import overload 55 56 import google.protobuf.wrappers_pb2 57 import proto 58 59 from apache_beam.coders import coder_impl 60 from apache_beam.coders.avro_record import AvroRecord 61 from apache_beam.portability import common_urns 62 from apache_beam.portability import python_urns 63 from apache_beam.portability.api import beam_runner_api_pb2 64 from apache_beam.typehints import typehints 65 from apache_beam.utils import proto_utils 66 67 if TYPE_CHECKING: 68 from google.protobuf import message # pylint: disable=ungrouped-imports 69 from apache_beam.coders.typecoders import CoderRegistry 70 from apache_beam.runners.pipeline_context import PipelineContext 71 72 # pylint: disable=wrong-import-order, wrong-import-position, ungrouped-imports 73 try: 74 from .stream import get_varint_size 75 except ImportError: 76 from .slow_stream import get_varint_size 77 # pylint: enable=wrong-import-order, wrong-import-position, ungrouped-imports 78 79 # pylint: disable=wrong-import-order, wrong-import-position 80 # Avoid dependencies on the full SDK. 81 try: 82 # Import dill from the pickler module to make sure our monkey-patching of dill 83 # occurs. 84 from apache_beam.internal.dill_pickler import dill 85 except ImportError: 86 # We fall back to using the stock dill library in tests that don't use the 87 # full Python SDK. 88 import dill 89 90 __all__ = [ 91 'Coder', 92 'AvroGenericCoder', 93 'BooleanCoder', 94 'BytesCoder', 95 'DillCoder', 96 'FastPrimitivesCoder', 97 'FloatCoder', 98 'IterableCoder', 99 'ListCoder', 100 'MapCoder', 101 'NullableCoder', 102 'PickleCoder', 103 'ProtoCoder', 104 'ProtoPlusCoder', 105 'ShardedKeyCoder', 106 'SinglePrecisionFloatCoder', 107 'SingletonCoder', 108 'StrUtf8Coder', 109 'TimestampCoder', 110 'TupleCoder', 111 'TupleSequenceCoder', 112 'VarIntCoder', 113 'WindowedValueCoder', 114 'ParamWindowedValueCoder', 115 'BigIntegerCoder', 116 'DecimalCoder' 117 ] 118 119 T = TypeVar('T') 120 CoderT = TypeVar('CoderT', bound='Coder') 121 ProtoCoderT = TypeVar('ProtoCoderT', bound='ProtoCoder') 122 ConstructorFn = Callable[[Optional[Any], List['Coder'], 'PipelineContext'], Any] 123 124 125 def serialize_coder(coder): 126 from apache_beam.internal import pickler 127 return b'%s$%s' % ( 128 coder.__class__.__name__.encode('utf-8'), 129 pickler.dumps(coder, use_zlib=True)) 130 131 132 def deserialize_coder(serialized): 133 from apache_beam.internal import pickler 134 return pickler.loads(serialized.split(b'$', 1)[1], use_zlib=True) 135 136 137 # pylint: enable=wrong-import-order, wrong-import-position 138 139 140 class Coder(object): 141 """Base class for coders.""" 142 def encode(self, value): 143 # type: (Any) -> bytes 144 145 """Encodes the given object into a byte string.""" 146 raise NotImplementedError('Encode not implemented: %s.' % self) 147 148 def decode(self, encoded): 149 """Decodes the given byte string into the corresponding object.""" 150 raise NotImplementedError('Decode not implemented: %s.' % self) 151 152 def encode_nested(self, value): 153 """Uses the underlying implementation to encode in nested format.""" 154 return self.get_impl().encode_nested(value) 155 156 def decode_nested(self, encoded): 157 """Uses the underlying implementation to decode in nested format.""" 158 return self.get_impl().decode_nested(encoded) 159 160 def is_deterministic(self): 161 # type: () -> bool 162 163 """Whether this coder is guaranteed to encode values deterministically. 164 165 A deterministic coder is required for key coders in GroupByKey operations 166 to produce consistent results. 167 168 For example, note that the default coder, the PickleCoder, is not 169 deterministic: the ordering of picked entries in maps may vary across 170 executions since there is no defined order, and such a coder is not in 171 general suitable for usage as a key coder in GroupByKey operations, since 172 each instance of the same key may be encoded differently. 173 174 Returns: 175 Whether coder is deterministic. 176 """ 177 return False 178 179 def as_deterministic_coder(self, step_label, error_message=None): 180 """Returns a deterministic version of self, if possible. 181 182 Otherwise raises a value error. 183 """ 184 if self.is_deterministic(): 185 return self 186 else: 187 raise ValueError( 188 error_message or 189 "%s cannot be made deterministic for '%s'." % (self, step_label)) 190 191 def estimate_size(self, value): 192 """Estimates the encoded size of the given value, in bytes. 193 194 Dataflow estimates the encoded size of a PCollection processed in a pipeline 195 step by using the estimated size of a random sample of elements in that 196 PCollection. 197 198 The default implementation encodes the given value and returns its byte 199 size. If a coder can provide a fast estimate of the encoded size of a value 200 (e.g., if the encoding has a fixed size), it can provide its estimate here 201 to improve performance. 202 203 Arguments: 204 value: the value whose encoded size is to be estimated. 205 206 Returns: 207 The estimated encoded size of the given value. 208 """ 209 return len(self.encode(value)) 210 211 # =========================================================================== 212 # Methods below are internal SDK details that don't need to be modified for 213 # user-defined coders. 214 # =========================================================================== 215 216 def _create_impl(self): 217 # type: () -> coder_impl.CoderImpl 218 219 """Creates a CoderImpl to do the actual encoding and decoding. 220 """ 221 return coder_impl.CallbackCoderImpl( 222 self.encode, self.decode, self.estimate_size) 223 224 def get_impl(self): 225 """For internal use only; no backwards-compatibility guarantees. 226 227 Returns the CoderImpl backing this Coder. 228 """ 229 if not hasattr(self, '_impl'): 230 self._impl = self._create_impl() 231 assert isinstance(self._impl, coder_impl.CoderImpl) 232 return self._impl 233 234 def __getstate__(self): 235 return self._dict_without_impl() 236 237 def _dict_without_impl(self): 238 if hasattr(self, '_impl'): 239 d = dict(self.__dict__) 240 del d['_impl'] 241 return d 242 return self.__dict__ 243 244 def to_type_hint(self): 245 raise NotImplementedError( 246 'https://github.com/apache/beam/issues/18490: %s' % 247 self.__class__.__name__) 248 249 @classmethod 250 def from_type_hint(cls, unused_typehint, unused_registry): 251 # type: (Type[CoderT], Any, CoderRegistry) -> CoderT 252 # If not overridden, just construct the coder without arguments. 253 return cls() 254 255 def is_kv_coder(self): 256 # type: () -> bool 257 return False 258 259 def key_coder(self): 260 # type: () -> Coder 261 if self.is_kv_coder(): 262 raise NotImplementedError('key_coder: %s' % self) 263 else: 264 raise ValueError('Not a KV coder: %s.' % self) 265 266 def value_coder(self): 267 # type: () -> Coder 268 if self.is_kv_coder(): 269 raise NotImplementedError('value_coder: %s' % self) 270 else: 271 raise ValueError('Not a KV coder: %s.' % self) 272 273 def _get_component_coders(self): 274 # type: () -> Sequence[Coder] 275 276 """For internal use only; no backwards-compatibility guarantees. 277 278 Returns the internal component coders of this coder.""" 279 # This is an internal detail of the Coder API and does not need to be 280 # refined in user-defined Coders. 281 return [] 282 283 def as_cloud_object(self, coders_context=None): 284 """For internal use only; no backwards-compatibility guarantees. 285 286 Returns Google Cloud Dataflow API description of this coder.""" 287 # This is an internal detail of the Coder API and does not need to be 288 # refined in user-defined Coders. 289 290 value = { 291 # We pass coders in the form "<coder_name>$<pickled_data>" to make the 292 # job description JSON more readable. Data before the $ is ignored by 293 # the worker. 294 '@type': serialize_coder(self), 295 'component_encodings': [ 296 component.as_cloud_object(coders_context) 297 for component in self._get_component_coders() 298 ], 299 } 300 301 if coders_context: 302 value['pipeline_proto_coder_id'] = coders_context.get_id(self) 303 304 return value 305 306 def __repr__(self): 307 return self.__class__.__name__ 308 309 # pylint: disable=protected-access 310 def __eq__(self, other): 311 return ( 312 self.__class__ == other.__class__ and 313 self._dict_without_impl() == other._dict_without_impl()) 314 315 # pylint: enable=protected-access 316 317 def __hash__(self): 318 return hash(type(self)) 319 320 _known_urns = {} # type: Dict[str, Tuple[type, ConstructorFn]] 321 322 @classmethod 323 @overload 324 def register_urn( 325 cls, 326 urn, # type: str 327 parameter_type, # type: Optional[Type[T]] 328 ): 329 # type: (...) -> Callable[[Callable[[T, List[Coder], PipelineContext], Any]], Callable[[T, List[Coder], PipelineContext], Any]] 330 pass 331 332 @classmethod 333 @overload 334 def register_urn( 335 cls, 336 urn, # type: str 337 parameter_type, # type: Optional[Type[T]] 338 fn # type: Callable[[T, List[Coder], PipelineContext], Any] 339 ): 340 # type: (...) -> None 341 pass 342 343 @classmethod 344 def register_urn(cls, urn, parameter_type, fn=None): 345 """Registers a urn with a constructor. 346 347 For example, if 'beam:fn:foo' had parameter type FooPayload, one could 348 write `RunnerApiFn.register_urn('bean:fn:foo', FooPayload, foo_from_proto)` 349 where foo_from_proto took as arguments a FooPayload and a PipelineContext. 350 This function can also be used as a decorator rather than passing the 351 callable in as the final parameter. 352 353 A corresponding to_runner_api_parameter method would be expected that 354 returns the tuple ('beam:fn:foo', FooPayload) 355 """ 356 def register(fn): 357 cls._known_urns[urn] = parameter_type, fn 358 return fn 359 360 if fn: 361 # Used as a statement. 362 register(fn) 363 else: 364 # Used as a decorator. 365 return register 366 367 def to_runner_api(self, context): 368 # type: (PipelineContext) -> beam_runner_api_pb2.Coder 369 urn, typed_param, components = self.to_runner_api_parameter(context) 370 return beam_runner_api_pb2.Coder( 371 spec=beam_runner_api_pb2.FunctionSpec( 372 urn=urn, 373 payload=typed_param if isinstance(typed_param, (bytes, type(None))) 374 else typed_param.SerializeToString()), 375 component_coder_ids=[context.coders.get_id(c) for c in components]) 376 377 @classmethod 378 def from_runner_api(cls, coder_proto, context): 379 # type: (Type[CoderT], beam_runner_api_pb2.Coder, PipelineContext) -> CoderT 380 381 """Converts from an FunctionSpec to a Fn object. 382 383 Prefer registering a urn with its parameter type and constructor. 384 """ 385 parameter_type, constructor = cls._known_urns[coder_proto.spec.urn] 386 return constructor( 387 proto_utils.parse_Bytes(coder_proto.spec.payload, parameter_type), 388 [context.coders.get_by_id(c) for c in coder_proto.component_coder_ids], 389 context) 390 391 def to_runner_api_parameter(self, context): 392 # type: (Optional[PipelineContext]) -> Tuple[str, Any, Sequence[Coder]] 393 return ( 394 python_urns.PICKLED_CODER, 395 google.protobuf.wrappers_pb2.BytesValue(value=serialize_coder(self)), 396 ()) 397 398 @staticmethod 399 def register_structured_urn(urn, cls): 400 # type: (str, Type[Coder]) -> None 401 402 """Register a coder that's completely defined by its urn and its 403 component(s), if any, which are passed to construct the instance. 404 """ 405 setattr( 406 cls, 407 'to_runner_api_parameter', 408 lambda self, 409 unused_context: (urn, None, self._get_component_coders())) 410 411 # pylint: disable=unused-variable 412 @Coder.register_urn(urn, None) 413 def from_runner_api_parameter(unused_payload, components, unused_context): 414 if components: 415 return cls(*components) 416 else: 417 return cls() 418 419 420 @Coder.register_urn( 421 python_urns.PICKLED_CODER, google.protobuf.wrappers_pb2.BytesValue) 422 def _pickle_from_runner_api_parameter(payload, components, context): 423 return deserialize_coder(payload.value) 424 425 426 class StrUtf8Coder(Coder): 427 """A coder used for reading and writing strings as UTF-8.""" 428 def encode(self, value): 429 return value.encode('utf-8') 430 431 def decode(self, value): 432 return value.decode('utf-8') 433 434 def is_deterministic(self): 435 # type: () -> bool 436 return True 437 438 def to_type_hint(self): 439 return str 440 441 442 Coder.register_structured_urn(common_urns.coders.STRING_UTF8.urn, StrUtf8Coder) 443 444 445 class ToBytesCoder(Coder): 446 """A default string coder used if no sink coder is specified.""" 447 def encode(self, value): 448 return value if isinstance(value, bytes) else str(value).encode('utf-8') 449 450 def decode(self, _): 451 raise NotImplementedError('ToBytesCoder cannot be used for decoding.') 452 453 def is_deterministic(self): 454 # type: () -> bool 455 return True 456 457 458 # alias to the old class name for a courtesy to users who reference it 459 ToStringCoder = ToBytesCoder 460 461 462 class FastCoder(Coder): 463 """Coder subclass used when a (faster) CoderImpl is supplied directly. 464 465 The Coder class defines _create_impl in terms of encode() and decode(); 466 this class inverts that by defining encode() and decode() in terms of 467 _create_impl(). 468 """ 469 def encode(self, value): 470 """Encodes the given object into a byte string.""" 471 return self.get_impl().encode(value) 472 473 def decode(self, encoded): 474 """Decodes the given byte string into the corresponding object.""" 475 return self.get_impl().decode(encoded) 476 477 def estimate_size(self, value): 478 return self.get_impl().estimate_size(value) 479 480 def _create_impl(self): 481 raise NotImplementedError 482 483 484 class BytesCoder(FastCoder): 485 """Byte string coder.""" 486 def _create_impl(self): 487 return coder_impl.BytesCoderImpl() 488 489 def is_deterministic(self): 490 # type: () -> bool 491 return True 492 493 def to_type_hint(self): 494 return bytes 495 496 def as_cloud_object(self, coders_context=None): 497 return { 498 '@type': 'kind:bytes', 499 } 500 501 def __eq__(self, other): 502 return type(self) == type(other) 503 504 def __hash__(self): 505 return hash(type(self)) 506 507 508 Coder.register_structured_urn(common_urns.coders.BYTES.urn, BytesCoder) 509 510 511 class BooleanCoder(FastCoder): 512 def _create_impl(self): 513 return coder_impl.BooleanCoderImpl() 514 515 def is_deterministic(self): 516 # type: () -> bool 517 return True 518 519 def to_type_hint(self): 520 return bool 521 522 def __eq__(self, other): 523 return type(self) == type(other) 524 525 def __hash__(self): 526 return hash(type(self)) 527 528 529 Coder.register_structured_urn(common_urns.coders.BOOL.urn, BooleanCoder) 530 531 532 class MapCoder(FastCoder): 533 def __init__(self, key_coder, value_coder): 534 # type: (Coder, Coder) -> None 535 self._key_coder = key_coder 536 self._value_coder = value_coder 537 538 def _create_impl(self): 539 return coder_impl.MapCoderImpl( 540 self._key_coder.get_impl(), self._value_coder.get_impl()) 541 542 @classmethod 543 def from_type_hint(cls, typehint, registry): 544 # type: (typehints.DictConstraint, CoderRegistry) -> MapCoder 545 return cls( 546 registry.get_coder(typehint.key_type), 547 registry.get_coder(typehint.value_type)) 548 549 def to_type_hint(self): 550 return typehints.Dict[self._key_coder.to_type_hint(), 551 self._value_coder.to_type_hint()] 552 553 def is_deterministic(self): 554 # type: () -> bool 555 # Map ordering is non-deterministic 556 return False 557 558 def as_deterministic_coder(self, step_label, error_message=None): 559 return DeterministicMapCoder( 560 self._key_coder.as_deterministic_coder(step_label, error_message), 561 self._value_coder.as_deterministic_coder(step_label, error_message)) 562 563 def __eq__(self, other): 564 return ( 565 type(self) == type(other) and self._key_coder == other._key_coder and 566 self._value_coder == other._value_coder) 567 568 def __hash__(self): 569 return hash(type(self)) + hash(self._key_coder) + hash(self._value_coder) 570 571 def __repr__(self): 572 return 'MapCoder[%s, %s]' % (self._key_coder, self._value_coder) 573 574 575 # This is a separate class from MapCoder as the former is a standard coder with 576 # no way to carry the is_deterministic bit. 577 class DeterministicMapCoder(FastCoder): 578 def __init__(self, key_coder, value_coder): 579 # type: (Coder, Coder) -> None 580 assert key_coder.is_deterministic() 581 assert value_coder.is_deterministic() 582 self._key_coder = key_coder 583 self._value_coder = value_coder 584 585 def _create_impl(self): 586 return coder_impl.MapCoderImpl( 587 self._key_coder.get_impl(), self._value_coder.get_impl(), True) 588 589 def is_deterministic(self): 590 return True 591 592 def __eq__(self, other): 593 return ( 594 type(self) == type(other) and self._key_coder == other._key_coder and 595 self._value_coder == other._value_coder) 596 597 def __hash__(self): 598 return hash(type(self)) + hash(self._key_coder) + hash(self._value_coder) 599 600 def __repr__(self): 601 return 'DeterministicMapCoder[%s, %s]' % ( 602 self._key_coder, self._value_coder) 603 604 605 class NullableCoder(FastCoder): 606 def __init__(self, value_coder): 607 # type: (Coder) -> None 608 self._value_coder = value_coder 609 610 def _create_impl(self): 611 return coder_impl.NullableCoderImpl(self._value_coder.get_impl()) 612 613 def to_type_hint(self): 614 return typehints.Optional[self._value_coder.to_type_hint()] 615 616 def _get_component_coders(self): 617 # type: () -> List[Coder] 618 return [self._value_coder] 619 620 @classmethod 621 def from_type_hint(cls, typehint, registry): 622 if typehints.is_nullable(typehint): 623 return cls( 624 registry.get_coder( 625 typehints.get_concrete_type_from_nullable(typehint))) 626 else: 627 raise TypeError( 628 'Typehint is not of nullable type, ' 629 'and cannot be converted to a NullableCoder', 630 typehint) 631 632 def is_deterministic(self): 633 # type: () -> bool 634 return self._value_coder.is_deterministic() 635 636 def as_deterministic_coder(self, step_label, error_message=None): 637 if self.is_deterministic(): 638 return self 639 else: 640 deterministic_value_coder = self._value_coder.as_deterministic_coder( 641 step_label, error_message) 642 return NullableCoder(deterministic_value_coder) 643 644 def __eq__(self, other): 645 return ( 646 type(self) == type(other) and self._value_coder == other._value_coder) 647 648 def __hash__(self): 649 return hash(type(self)) + hash(self._value_coder) 650 651 def __repr__(self): 652 return 'NullableCoder[%s]' % self._value_coder 653 654 655 Coder.register_structured_urn(common_urns.coders.NULLABLE.urn, NullableCoder) 656 657 658 class VarIntCoder(FastCoder): 659 """Variable-length integer coder.""" 660 def _create_impl(self): 661 return coder_impl.VarIntCoderImpl() 662 663 def is_deterministic(self): 664 # type: () -> bool 665 return True 666 667 def to_type_hint(self): 668 return int 669 670 def as_cloud_object(self, coders_context=None): 671 return { 672 '@type': 'kind:varint', 673 } 674 675 def __eq__(self, other): 676 return type(self) == type(other) 677 678 def __hash__(self): 679 return hash(type(self)) 680 681 682 Coder.register_structured_urn(common_urns.coders.VARINT.urn, VarIntCoder) 683 684 685 class BigEndianShortCoder(FastCoder): 686 """A coder used for big-endian int16 values.""" 687 def _create_impl(self): 688 return coder_impl.BigEndianShortCoderImpl() 689 690 def is_deterministic(self): 691 # type: () -> bool 692 return True 693 694 def to_type_hint(self): 695 return int 696 697 def __eq__(self, other): 698 return type(self) == type(other) 699 700 def __hash__(self): 701 return hash(type(self)) 702 703 704 class SinglePrecisionFloatCoder(FastCoder): 705 """A coder used for single-precision floating-point values.""" 706 def _create_impl(self): 707 return coder_impl.SinglePrecisionFloatCoderImpl() 708 709 def is_deterministic(self): 710 # type: () -> bool 711 return True 712 713 def to_type_hint(self): 714 return float 715 716 def __eq__(self, other): 717 return type(self) == type(other) 718 719 def __hash__(self): 720 return hash(type(self)) 721 722 723 class FloatCoder(FastCoder): 724 """A coder used for **double-precision** floating-point values. 725 726 Note that the name "FloatCoder" is in reference to Python's ``float`` built-in 727 which is generally implemented using C doubles. See 728 :class:`SinglePrecisionFloatCoder` for a single-precision version of this 729 coder. 730 """ 731 def _create_impl(self): 732 return coder_impl.FloatCoderImpl() 733 734 def is_deterministic(self): 735 # type: () -> bool 736 return True 737 738 def to_type_hint(self): 739 return float 740 741 def __eq__(self, other): 742 return type(self) == type(other) 743 744 def __hash__(self): 745 return hash(type(self)) 746 747 748 Coder.register_structured_urn(common_urns.coders.DOUBLE.urn, FloatCoder) 749 750 751 class TimestampCoder(FastCoder): 752 """A coder used for timeutil.Timestamp values.""" 753 def _create_impl(self): 754 return coder_impl.TimestampCoderImpl() 755 756 def is_deterministic(self): 757 # type: () -> bool 758 return True 759 760 def __eq__(self, other): 761 return type(self) == type(other) 762 763 def __hash__(self): 764 return hash(type(self)) 765 766 767 class _TimerCoder(FastCoder): 768 """A coder used for timer values. 769 770 For internal use.""" 771 def __init__(self, key_coder, window_coder): 772 # type: (Coder, Coder) -> None 773 self._key_coder = key_coder 774 self._window_coder = window_coder 775 776 def _get_component_coders(self): 777 # type: () -> List[Coder] 778 return [self._key_coder, self._window_coder] 779 780 def _create_impl(self): 781 return coder_impl.TimerCoderImpl( 782 self._key_coder.get_impl(), self._window_coder.get_impl()) 783 784 def is_deterministic(self): 785 # type: () -> bool 786 return ( 787 self._key_coder.is_deterministic() and 788 self._window_coder.is_deterministic()) 789 790 def __eq__(self, other): 791 return ( 792 type(self) == type(other) and self._key_coder == other._key_coder and 793 self._window_coder == other._window_coder) 794 795 def __hash__(self): 796 return hash(type(self)) + hash(self._key_coder) + hash(self._window_coder) 797 798 799 Coder.register_structured_urn(common_urns.coders.TIMER.urn, _TimerCoder) 800 801 802 class SingletonCoder(FastCoder): 803 """A coder that always encodes exactly one value.""" 804 def __init__(self, value): 805 self._value = value 806 807 def _create_impl(self): 808 return coder_impl.SingletonCoderImpl(self._value) 809 810 def is_deterministic(self): 811 # type: () -> bool 812 return True 813 814 def __eq__(self, other): 815 return type(self) == type(other) and self._value == other._value 816 817 def __hash__(self): 818 return hash(self._value) 819 820 821 def maybe_dill_dumps(o): 822 """Pickle using cPickle or the Dill pickler as a fallback.""" 823 # We need to use the dill pickler for objects of certain custom classes, 824 # including, for example, ones that contain lambdas. 825 try: 826 return pickle.dumps(o, pickle.HIGHEST_PROTOCOL) 827 except Exception: # pylint: disable=broad-except 828 return dill.dumps(o) 829 830 831 def maybe_dill_loads(o): 832 """Unpickle using cPickle or the Dill pickler as a fallback.""" 833 try: 834 return pickle.loads(o) 835 except Exception: # pylint: disable=broad-except 836 return dill.loads(o) 837 838 839 class _PickleCoderBase(FastCoder): 840 """Base class for pickling coders.""" 841 def is_deterministic(self): 842 # type: () -> bool 843 # Note that the default coder, the PickleCoder, is not deterministic (for 844 # example, the ordering of picked entries in maps may vary across 845 # executions), and so is not in general suitable for usage as a key coder in 846 # GroupByKey operations. 847 return False 848 849 def as_cloud_object(self, coders_context=None, is_pair_like=True): 850 value = super().as_cloud_object(coders_context) 851 # We currently use this coder in places where we cannot infer the coder to 852 # use for the value type in a more granular way. In places where the 853 # service expects a pair, it checks for the "is_pair_like" key, in which 854 # case we would fail without the hack below. 855 if is_pair_like: 856 value['is_pair_like'] = True 857 value['component_encodings'] = [ 858 self.as_cloud_object(coders_context, is_pair_like=False), 859 self.as_cloud_object(coders_context, is_pair_like=False) 860 ] 861 862 return value 863 864 # We allow .key_coder() and .value_coder() to be called on PickleCoder since 865 # we can't always infer the return values of lambdas in ParDo operations, the 866 # result of which may be used in a GroupBykey. 867 def is_kv_coder(self): 868 # type: () -> bool 869 return True 870 871 def key_coder(self): 872 return self 873 874 def value_coder(self): 875 return self 876 877 def __eq__(self, other): 878 return type(self) == type(other) 879 880 def __hash__(self): 881 return hash(type(self)) 882 883 884 class _MemoizingPickleCoder(_PickleCoderBase): 885 """Coder using Python's pickle functionality with memoization.""" 886 def __init__(self, cache_size=16): 887 super().__init__() 888 self.cache_size = cache_size 889 890 def _create_impl(self): 891 from apache_beam.internal import pickler 892 dumps = pickler.dumps 893 894 mdumps = lru_cache(maxsize=self.cache_size, typed=True)(dumps) 895 896 def _nonhashable_dumps(x): 897 try: 898 return mdumps(x) 899 except TypeError: 900 return dumps(x) 901 902 return coder_impl.CallbackCoderImpl(_nonhashable_dumps, pickler.loads) 903 904 def as_deterministic_coder(self, step_label, error_message=None): 905 return FastPrimitivesCoder(self, requires_deterministic=step_label) 906 907 def to_type_hint(self): 908 return Any 909 910 911 class PickleCoder(_PickleCoderBase): 912 """Coder using Python's pickle functionality.""" 913 def _create_impl(self): 914 dumps = pickle.dumps 915 protocol = pickle.HIGHEST_PROTOCOL 916 return coder_impl.CallbackCoderImpl( 917 lambda x: dumps(x, protocol), pickle.loads) 918 919 def as_deterministic_coder(self, step_label, error_message=None): 920 return FastPrimitivesCoder(self, requires_deterministic=step_label) 921 922 def to_type_hint(self): 923 return Any 924 925 926 class DillCoder(_PickleCoderBase): 927 """Coder using dill's pickle functionality.""" 928 def _create_impl(self): 929 return coder_impl.CallbackCoderImpl(maybe_dill_dumps, maybe_dill_loads) 930 931 932 class DeterministicFastPrimitivesCoder(FastCoder): 933 """Throws runtime errors when encoding non-deterministic values.""" 934 def __init__(self, coder, step_label): 935 self._underlying_coder = coder 936 self._step_label = step_label 937 938 def _create_impl(self): 939 return coder_impl.FastPrimitivesCoderImpl( 940 self._underlying_coder.get_impl(), 941 requires_deterministic_step_label=self._step_label) 942 943 def is_deterministic(self): 944 # type: () -> bool 945 return True 946 947 def is_kv_coder(self): 948 # type: () -> bool 949 return True 950 951 def key_coder(self): 952 return self 953 954 def value_coder(self): 955 return self 956 957 def to_type_hint(self): 958 return Any 959 960 961 class FastPrimitivesCoder(FastCoder): 962 """Encodes simple primitives (e.g. str, int) efficiently. 963 964 For unknown types, falls back to another coder (e.g. PickleCoder). 965 """ 966 def __init__(self, fallback_coder=PickleCoder()): 967 # type: (Coder) -> None 968 self._fallback_coder = fallback_coder 969 970 def _create_impl(self): 971 return coder_impl.FastPrimitivesCoderImpl(self._fallback_coder.get_impl()) 972 973 def is_deterministic(self): 974 # type: () -> bool 975 return self._fallback_coder.is_deterministic() 976 977 def as_deterministic_coder(self, step_label, error_message=None): 978 if self.is_deterministic(): 979 return self 980 else: 981 return DeterministicFastPrimitivesCoder(self, step_label) 982 983 def to_type_hint(self): 984 return Any 985 986 def as_cloud_object(self, coders_context=None, is_pair_like=True): 987 value = super().as_cloud_object(coders_context) 988 # We currently use this coder in places where we cannot infer the coder to 989 # use for the value type in a more granular way. In places where the 990 # service expects a pair, it checks for the "is_pair_like" key, in which 991 # case we would fail without the hack below. 992 if is_pair_like: 993 value['is_pair_like'] = True 994 value['component_encodings'] = [ 995 self.as_cloud_object(coders_context, is_pair_like=False), 996 self.as_cloud_object(coders_context, is_pair_like=False) 997 ] 998 999 return value 1000 1001 # We allow .key_coder() and .value_coder() to be called on FastPrimitivesCoder 1002 # since we can't always infer the return values of lambdas in ParDo 1003 # operations, the result of which may be used in a GroupBykey. 1004 def is_kv_coder(self): 1005 # type: () -> bool 1006 return True 1007 1008 def key_coder(self): 1009 return self 1010 1011 def value_coder(self): 1012 return self 1013 1014 def __eq__(self, other): 1015 return type(self) == type(other) 1016 1017 def __hash__(self): 1018 return hash(type(self)) 1019 1020 1021 class FakeDeterministicFastPrimitivesCoder(FastPrimitivesCoder): 1022 """A FastPrimitivesCoder that claims to be deterministic. 1023 1024 This can be registered as a fallback coder to go back to the behavior before 1025 deterministic encoding was enforced (BEAM-11719). 1026 """ 1027 def is_deterministic(self): 1028 return True 1029 1030 1031 class Base64PickleCoder(Coder): 1032 """Coder of objects by Python pickle, then base64 encoding.""" 1033 1034 # TODO(robertwb): Do base64 encoding where it's needed (e.g. in json) rather 1035 # than via a special Coder. 1036 1037 def encode(self, value): 1038 return base64.b64encode(pickle.dumps(value, pickle.HIGHEST_PROTOCOL)) 1039 1040 def decode(self, encoded): 1041 return pickle.loads(base64.b64decode(encoded)) 1042 1043 def is_deterministic(self): 1044 # type: () -> bool 1045 # Note that the Base64PickleCoder is not deterministic. See the 1046 # corresponding comments for PickleCoder above. 1047 return False 1048 1049 # We allow .key_coder() and .value_coder() to be called on Base64PickleCoder 1050 # since we can't always infer the return values of lambdas in ParDo 1051 # operations, the result of which may be used in a GroupBykey. 1052 # 1053 # TODO(ccy): this is currently only used for KV values from Create transforms. 1054 # Investigate a way to unify this with PickleCoder. 1055 def is_kv_coder(self): 1056 # type: () -> bool 1057 return True 1058 1059 def key_coder(self): 1060 return self 1061 1062 def value_coder(self): 1063 return self 1064 1065 1066 class ProtoCoder(FastCoder): 1067 """A Coder for Google Protocol Buffers. 1068 1069 It supports both Protocol Buffers syntax versions 2 and 3. However, 1070 the runtime version of the python protobuf library must exactly match the 1071 version of the protoc compiler what was used to generate the protobuf 1072 messages. 1073 1074 ProtoCoder is registered in the global CoderRegistry as the default coder for 1075 any protobuf Message object. 1076 1077 """ 1078 def __init__(self, proto_message_type): 1079 # type: (Type[google.protobuf.message.Message]) -> None 1080 self.proto_message_type = proto_message_type 1081 1082 def _create_impl(self): 1083 return coder_impl.ProtoCoderImpl(self.proto_message_type) 1084 1085 def is_deterministic(self): 1086 # type: () -> bool 1087 # TODO(vikasrk): A proto message can be deterministic if it does not contain 1088 # a Map. 1089 return False 1090 1091 def as_deterministic_coder(self, step_label, error_message=None): 1092 return DeterministicProtoCoder(self.proto_message_type) 1093 1094 def __eq__(self, other): 1095 return ( 1096 type(self) == type(other) and 1097 self.proto_message_type == other.proto_message_type) 1098 1099 def __hash__(self): 1100 return hash(self.proto_message_type) 1101 1102 @classmethod 1103 def from_type_hint(cls, typehint, unused_registry): 1104 if issubclass(typehint, proto_utils.message_types): 1105 return cls(typehint) 1106 else: 1107 raise ValueError(( 1108 'Expected a subclass of google.protobuf.message.Message' 1109 ', but got a %s' % typehint)) 1110 1111 def to_type_hint(self): 1112 return self.proto_message_type 1113 1114 1115 class DeterministicProtoCoder(ProtoCoder): 1116 """A deterministic Coder for Google Protocol Buffers. 1117 1118 It supports both Protocol Buffers syntax versions 2 and 3. However, 1119 the runtime version of the python protobuf library must exactly match the 1120 version of the protoc compiler what was used to generate the protobuf 1121 messages. 1122 """ 1123 def _create_impl(self): 1124 return coder_impl.DeterministicProtoCoderImpl(self.proto_message_type) 1125 1126 def is_deterministic(self): 1127 # type: () -> bool 1128 return True 1129 1130 def as_deterministic_coder(self, step_label, error_message=None): 1131 return self 1132 1133 1134 class ProtoPlusCoder(FastCoder): 1135 """A Coder for Google Protocol Buffers wrapped using the proto-plus library. 1136 1137 ProtoPlusCoder is registered in the global CoderRegistry as the default coder 1138 for any proto.Message object. 1139 """ 1140 def __init__(self, proto_plus_message_type): 1141 # type: (Type[proto.Message]) -> None 1142 self.proto_plus_message_type = proto_plus_message_type 1143 1144 def _create_impl(self): 1145 return coder_impl.ProtoPlusCoderImpl(self.proto_plus_message_type) 1146 1147 def is_deterministic(self): 1148 return True 1149 1150 def __eq__(self, other): 1151 return ( 1152 type(self) == type(other) and 1153 self.proto_plus_message_type == other.proto_plus_message_type) 1154 1155 def __hash__(self): 1156 return hash(self.proto_plus_message_type) 1157 1158 @classmethod 1159 def from_type_hint(cls, typehint, unused_registry): 1160 if issubclass(typehint, proto.Message): 1161 return cls(typehint) 1162 else: 1163 raise ValueError( 1164 'Expected a subclass of proto.Message, but got a %s' % typehint) 1165 1166 def to_type_hint(self): 1167 return self.proto_plus_message_type 1168 1169 1170 AVRO_GENERIC_CODER_URN = "beam:coder:avro:generic:v1" 1171 1172 1173 class AvroGenericCoder(FastCoder): 1174 """A coder used for AvroRecord values.""" 1175 def __init__(self, schema): 1176 self.schema = schema 1177 1178 def _create_impl(self): 1179 return coder_impl.AvroCoderImpl(self.schema) 1180 1181 def is_deterministic(self): 1182 # TODO(https://github.com/apache/beam/issues/19628): need to confirm if 1183 # it's deterministic 1184 return False 1185 1186 def __eq__(self, other): 1187 return type(self) == type(other) and self.schema == other.schema 1188 1189 def __hash__(self): 1190 return hash(self.schema) 1191 1192 def to_type_hint(self): 1193 return AvroRecord 1194 1195 def to_runner_api_parameter(self, context): 1196 return AVRO_GENERIC_CODER_URN, self.schema.encode('utf-8'), () 1197 1198 @staticmethod 1199 @Coder.register_urn(AVRO_GENERIC_CODER_URN, bytes) 1200 def from_runner_api_parameter(payload, unused_components, unused_context): 1201 return AvroGenericCoder(payload.decode('utf-8')) 1202 1203 1204 class TupleCoder(FastCoder): 1205 """Coder of tuple objects.""" 1206 def __init__(self, components): 1207 # type: (Iterable[Coder]) -> None 1208 self._coders = tuple(components) 1209 1210 def _create_impl(self): 1211 return coder_impl.TupleCoderImpl([c.get_impl() for c in self._coders]) 1212 1213 def is_deterministic(self): 1214 # type: () -> bool 1215 return all(c.is_deterministic() for c in self._coders) 1216 1217 def as_deterministic_coder(self, step_label, error_message=None): 1218 if self.is_deterministic(): 1219 return self 1220 else: 1221 return TupleCoder([ 1222 c.as_deterministic_coder(step_label, error_message) 1223 for c in self._coders 1224 ]) 1225 1226 def to_type_hint(self): 1227 return typehints.Tuple[tuple(c.to_type_hint() for c in self._coders)] 1228 1229 @classmethod 1230 def from_type_hint(cls, typehint, registry): 1231 # type: (typehints.TupleConstraint, CoderRegistry) -> TupleCoder 1232 return cls([registry.get_coder(t) for t in typehint.tuple_types]) 1233 1234 def as_cloud_object(self, coders_context=None): 1235 if self.is_kv_coder(): 1236 return { 1237 '@type': 'kind:pair', 1238 'is_pair_like': True, 1239 'component_encodings': [ 1240 component.as_cloud_object(coders_context) 1241 for component in self._get_component_coders() 1242 ], 1243 } 1244 1245 return super().as_cloud_object(coders_context) 1246 1247 def _get_component_coders(self): 1248 # type: () -> Tuple[Coder, ...] 1249 return self.coders() 1250 1251 def coders(self): 1252 # type: () -> Tuple[Coder, ...] 1253 return self._coders 1254 1255 def is_kv_coder(self): 1256 # type: () -> bool 1257 return len(self._coders) == 2 1258 1259 def key_coder(self): 1260 # type: () -> Coder 1261 if len(self._coders) != 2: 1262 raise ValueError('TupleCoder does not have exactly 2 components.') 1263 return self._coders[0] 1264 1265 def value_coder(self): 1266 # type: () -> Coder 1267 if len(self._coders) != 2: 1268 raise ValueError('TupleCoder does not have exactly 2 components.') 1269 return self._coders[1] 1270 1271 def __repr__(self): 1272 return 'TupleCoder[%s]' % ', '.join(str(c) for c in self._coders) 1273 1274 def __eq__(self, other): 1275 return type(self) == type(other) and self._coders == other.coders() 1276 1277 def __hash__(self): 1278 return hash(self._coders) 1279 1280 def to_runner_api_parameter(self, context): 1281 if self.is_kv_coder(): 1282 return common_urns.coders.KV.urn, None, self.coders() 1283 else: 1284 return python_urns.TUPLE_CODER, None, self.coders() 1285 1286 @staticmethod 1287 @Coder.register_urn(common_urns.coders.KV.urn, None) 1288 @Coder.register_urn(python_urns.TUPLE_CODER, None) 1289 def from_runner_api_parameter(unused_payload, components, unused_context): 1290 return TupleCoder(components) 1291 1292 1293 class TupleSequenceCoder(FastCoder): 1294 """Coder of homogeneous tuple objects.""" 1295 def __init__(self, elem_coder): 1296 # type: (Coder) -> None 1297 self._elem_coder = elem_coder 1298 1299 def value_coder(self): 1300 return self._elem_coder 1301 1302 def _create_impl(self): 1303 return coder_impl.TupleSequenceCoderImpl(self._elem_coder.get_impl()) 1304 1305 def is_deterministic(self): 1306 # type: () -> bool 1307 return self._elem_coder.is_deterministic() 1308 1309 def as_deterministic_coder(self, step_label, error_message=None): 1310 if self.is_deterministic(): 1311 return self 1312 else: 1313 return TupleSequenceCoder( 1314 self._elem_coder.as_deterministic_coder(step_label, error_message)) 1315 1316 @classmethod 1317 def from_type_hint(cls, typehint, registry): 1318 # type: (Any, CoderRegistry) -> TupleSequenceCoder 1319 return cls(registry.get_coder(typehint.inner_type)) 1320 1321 def _get_component_coders(self): 1322 # type: () -> Tuple[Coder, ...] 1323 return (self._elem_coder, ) 1324 1325 def __repr__(self): 1326 return 'TupleSequenceCoder[%r]' % self._elem_coder 1327 1328 def __eq__(self, other): 1329 return ( 1330 type(self) == type(other) and self._elem_coder == other.value_coder()) 1331 1332 def __hash__(self): 1333 return hash((type(self), self._elem_coder)) 1334 1335 1336 class ListLikeCoder(FastCoder): 1337 """Coder of iterables of homogeneous objects.""" 1338 def __init__(self, elem_coder): 1339 # type: (Coder) -> None 1340 self._elem_coder = elem_coder 1341 1342 def _create_impl(self): 1343 return coder_impl.IterableCoderImpl(self._elem_coder.get_impl()) 1344 1345 def is_deterministic(self): 1346 # type: () -> bool 1347 return self._elem_coder.is_deterministic() 1348 1349 def as_deterministic_coder(self, step_label, error_message=None): 1350 if self.is_deterministic(): 1351 return self 1352 else: 1353 return type(self)( 1354 self._elem_coder.as_deterministic_coder(step_label, error_message)) 1355 1356 def as_cloud_object(self, coders_context=None): 1357 return { 1358 '@type': 'kind:stream', 1359 'is_stream_like': True, 1360 'component_encodings': [ 1361 self._elem_coder.as_cloud_object(coders_context) 1362 ], 1363 } 1364 1365 def value_coder(self): 1366 return self._elem_coder 1367 1368 @classmethod 1369 def from_type_hint(cls, typehint, registry): 1370 # type: (Any, CoderRegistry) -> ListLikeCoder 1371 return cls(registry.get_coder(typehint.inner_type)) 1372 1373 def _get_component_coders(self): 1374 # type: () -> Tuple[Coder, ...] 1375 return (self._elem_coder, ) 1376 1377 def __repr__(self): 1378 return '%s[%r]' % (self.__class__.__name__, self._elem_coder) 1379 1380 def __eq__(self, other): 1381 return ( 1382 type(self) == type(other) and self._elem_coder == other.value_coder()) 1383 1384 def __hash__(self): 1385 return hash((type(self), self._elem_coder)) 1386 1387 1388 class IterableCoder(ListLikeCoder): 1389 """Coder of iterables of homogeneous objects.""" 1390 def to_type_hint(self): 1391 return typehints.Iterable[self._elem_coder.to_type_hint()] 1392 1393 1394 Coder.register_structured_urn(common_urns.coders.ITERABLE.urn, IterableCoder) 1395 1396 1397 class ListCoder(ListLikeCoder): 1398 """Coder of Python lists.""" 1399 def to_type_hint(self): 1400 return typehints.List[self._elem_coder.to_type_hint()] 1401 1402 def _create_impl(self): 1403 return coder_impl.ListCoderImpl(self._elem_coder.get_impl()) 1404 1405 1406 class GlobalWindowCoder(SingletonCoder): 1407 """Coder for global windows.""" 1408 def __init__(self): 1409 from apache_beam.transforms import window 1410 super().__init__(window.GlobalWindow()) 1411 1412 def as_cloud_object(self, coders_context=None): 1413 return { 1414 '@type': 'kind:global_window', 1415 } 1416 1417 1418 Coder.register_structured_urn( 1419 common_urns.coders.GLOBAL_WINDOW.urn, GlobalWindowCoder) 1420 1421 1422 class IntervalWindowCoder(FastCoder): 1423 """Coder for an window defined by a start timestamp and a duration.""" 1424 def _create_impl(self): 1425 return coder_impl.IntervalWindowCoderImpl() 1426 1427 def is_deterministic(self): 1428 # type: () -> bool 1429 return True 1430 1431 def as_cloud_object(self, coders_context=None): 1432 return { 1433 '@type': 'kind:interval_window', 1434 } 1435 1436 def __eq__(self, other): 1437 return type(self) == type(other) 1438 1439 def __hash__(self): 1440 return hash(type(self)) 1441 1442 1443 Coder.register_structured_urn( 1444 common_urns.coders.INTERVAL_WINDOW.urn, IntervalWindowCoder) 1445 1446 1447 class WindowedValueCoder(FastCoder): 1448 """Coder for windowed values.""" 1449 def __init__(self, wrapped_value_coder, window_coder=None): 1450 # type: (Coder, Optional[Coder]) -> None 1451 if not window_coder: 1452 window_coder = PickleCoder() 1453 self.wrapped_value_coder = wrapped_value_coder 1454 self.timestamp_coder = TimestampCoder() 1455 self.window_coder = window_coder 1456 1457 def _create_impl(self): 1458 return coder_impl.WindowedValueCoderImpl( 1459 self.wrapped_value_coder.get_impl(), 1460 self.timestamp_coder.get_impl(), 1461 self.window_coder.get_impl()) 1462 1463 def is_deterministic(self): 1464 # type: () -> bool 1465 return all( 1466 c.is_deterministic() for c in 1467 [self.wrapped_value_coder, self.timestamp_coder, self.window_coder]) 1468 1469 def as_cloud_object(self, coders_context=None): 1470 return { 1471 '@type': 'kind:windowed_value', 1472 'is_wrapper': True, 1473 'component_encodings': [ 1474 component.as_cloud_object(coders_context) 1475 for component in self._get_component_coders() 1476 ], 1477 } 1478 1479 def _get_component_coders(self): 1480 # type: () -> List[Coder] 1481 return [self.wrapped_value_coder, self.window_coder] 1482 1483 def is_kv_coder(self): 1484 # type: () -> bool 1485 return self.wrapped_value_coder.is_kv_coder() 1486 1487 def key_coder(self): 1488 # type: () -> Coder 1489 return self.wrapped_value_coder.key_coder() 1490 1491 def value_coder(self): 1492 # type: () -> Coder 1493 return self.wrapped_value_coder.value_coder() 1494 1495 def __repr__(self): 1496 return 'WindowedValueCoder[%s]' % self.wrapped_value_coder 1497 1498 def __eq__(self, other): 1499 return ( 1500 type(self) == type(other) and 1501 self.wrapped_value_coder == other.wrapped_value_coder and 1502 self.timestamp_coder == other.timestamp_coder and 1503 self.window_coder == other.window_coder) 1504 1505 def __hash__(self): 1506 return hash( 1507 (self.wrapped_value_coder, self.timestamp_coder, self.window_coder)) 1508 1509 1510 Coder.register_structured_urn( 1511 common_urns.coders.WINDOWED_VALUE.urn, WindowedValueCoder) 1512 1513 1514 class ParamWindowedValueCoder(WindowedValueCoder): 1515 """A coder used for parameterized windowed values.""" 1516 def __init__(self, payload, components): 1517 super().__init__(components[0], components[1]) 1518 self.payload = payload 1519 1520 def _create_impl(self): 1521 return coder_impl.ParamWindowedValueCoderImpl( 1522 self.wrapped_value_coder.get_impl(), 1523 self.window_coder.get_impl(), 1524 self.payload) 1525 1526 def is_deterministic(self): 1527 # type: () -> bool 1528 return self.wrapped_value_coder.is_deterministic() 1529 1530 def as_cloud_object(self, coders_context=None): 1531 raise NotImplementedError( 1532 "as_cloud_object not supported for ParamWindowedValueCoder") 1533 1534 def __repr__(self): 1535 return 'ParamWindowedValueCoder[%s]' % self.wrapped_value_coder 1536 1537 def __eq__(self, other): 1538 return ( 1539 type(self) == type(other) and 1540 self.wrapped_value_coder == other.wrapped_value_coder and 1541 self.window_coder == other.window_coder and 1542 self.payload == other.payload) 1543 1544 def __hash__(self): 1545 return hash((self.wrapped_value_coder, self.window_coder, self.payload)) 1546 1547 @staticmethod 1548 @Coder.register_urn(common_urns.coders.PARAM_WINDOWED_VALUE.urn, bytes) 1549 def from_runner_api_parameter(payload, components, unused_context): 1550 return ParamWindowedValueCoder(payload, components) 1551 1552 def to_runner_api_parameter(self, context): 1553 return ( 1554 common_urns.coders.PARAM_WINDOWED_VALUE.urn, 1555 self.payload, (self.wrapped_value_coder, self.window_coder)) 1556 1557 1558 class LengthPrefixCoder(FastCoder): 1559 """For internal use only; no backwards-compatibility guarantees. 1560 1561 Coder which prefixes the length of the encoded object in the stream.""" 1562 def __init__(self, value_coder): 1563 # type: (Coder) -> None 1564 self._value_coder = value_coder 1565 1566 def _create_impl(self): 1567 return coder_impl.LengthPrefixCoderImpl(self._value_coder.get_impl()) 1568 1569 def is_deterministic(self): 1570 # type: () -> bool 1571 return self._value_coder.is_deterministic() 1572 1573 def estimate_size(self, value): 1574 value_size = self._value_coder.estimate_size(value) 1575 return get_varint_size(value_size) + value_size 1576 1577 def value_coder(self): 1578 return self._value_coder 1579 1580 def as_cloud_object(self, coders_context=None): 1581 return { 1582 '@type': 'kind:length_prefix', 1583 'component_encodings': [ 1584 self._value_coder.as_cloud_object(coders_context) 1585 ], 1586 } 1587 1588 def _get_component_coders(self): 1589 # type: () -> Tuple[Coder, ...] 1590 return (self._value_coder, ) 1591 1592 def __repr__(self): 1593 return 'LengthPrefixCoder[%r]' % self._value_coder 1594 1595 def __eq__(self, other): 1596 return ( 1597 type(self) == type(other) and self._value_coder == other._value_coder) 1598 1599 def __hash__(self): 1600 return hash((type(self), self._value_coder)) 1601 1602 1603 Coder.register_structured_urn( 1604 common_urns.coders.LENGTH_PREFIX.urn, LengthPrefixCoder) 1605 1606 1607 class StateBackedIterableCoder(FastCoder): 1608 DEFAULT_WRITE_THRESHOLD = 1 1609 1610 def __init__( 1611 self, 1612 element_coder, # type: Coder 1613 read_state=None, # type: Optional[coder_impl.IterableStateReader] 1614 write_state=None, # type: Optional[coder_impl.IterableStateWriter] 1615 write_state_threshold=DEFAULT_WRITE_THRESHOLD): 1616 self._element_coder = element_coder 1617 self._read_state = read_state 1618 self._write_state = write_state 1619 self._write_state_threshold = write_state_threshold 1620 1621 def _create_impl(self): 1622 return coder_impl.IterableCoderImpl( 1623 self._element_coder.get_impl(), 1624 self._read_state, 1625 self._write_state, 1626 self._write_state_threshold) 1627 1628 def is_deterministic(self): 1629 # type: () -> bool 1630 return False 1631 1632 def _get_component_coders(self): 1633 # type: () -> Tuple[Coder, ...] 1634 return (self._element_coder, ) 1635 1636 def __repr__(self): 1637 return 'StateBackedIterableCoder[%r]' % self._element_coder 1638 1639 def __eq__(self, other): 1640 return ( 1641 type(self) == type(other) and 1642 self._element_coder == other._element_coder and 1643 self._write_state_threshold == other._write_state_threshold) 1644 1645 def __hash__(self): 1646 return hash((type(self), self._element_coder, self._write_state_threshold)) 1647 1648 def to_runner_api_parameter(self, context): 1649 # type: (Optional[PipelineContext]) -> Tuple[str, Any, Sequence[Coder]] 1650 return ( 1651 common_urns.coders.STATE_BACKED_ITERABLE.urn, 1652 str(self._write_state_threshold).encode('ascii'), 1653 self._get_component_coders()) 1654 1655 @staticmethod 1656 @Coder.register_urn(common_urns.coders.STATE_BACKED_ITERABLE.urn, bytes) 1657 def from_runner_api_parameter(payload, components, context): 1658 return StateBackedIterableCoder( 1659 components[0], 1660 read_state=context.iterable_state_read, 1661 write_state=context.iterable_state_write, 1662 write_state_threshold=int(payload) 1663 if payload else StateBackedIterableCoder.DEFAULT_WRITE_THRESHOLD) 1664 1665 1666 class ShardedKeyCoder(FastCoder): 1667 """A coder for sharded key.""" 1668 def __init__(self, key_coder): 1669 # type: (Coder) -> None 1670 self._key_coder = key_coder 1671 1672 def _get_component_coders(self): 1673 # type: () -> List[Coder] 1674 return [self._key_coder] 1675 1676 def _create_impl(self): 1677 return coder_impl.ShardedKeyCoderImpl(self._key_coder.get_impl()) 1678 1679 def is_deterministic(self): 1680 # type: () -> bool 1681 return self._key_coder.is_deterministic() 1682 1683 def as_cloud_object(self, coders_context=None): 1684 return { 1685 '@type': 'kind:sharded_key', 1686 'component_encodings': [ 1687 self._key_coder.as_cloud_object(coders_context) 1688 ], 1689 } 1690 1691 def to_type_hint(self): 1692 from apache_beam.typehints import sharded_key_type 1693 return sharded_key_type.ShardedKeyTypeConstraint( 1694 self._key_coder.to_type_hint()) 1695 1696 @classmethod 1697 def from_type_hint(cls, typehint, registry): 1698 from apache_beam.typehints import sharded_key_type 1699 if isinstance(typehint, sharded_key_type.ShardedKeyTypeConstraint): 1700 return cls(registry.get_coder(typehint.key_type)) 1701 else: 1702 raise ValueError(( 1703 'Expected an instance of ShardedKeyTypeConstraint' 1704 ', but got a %s' % typehint)) 1705 1706 def __eq__(self, other): 1707 return type(self) == type(other) and self._key_coder == other._key_coder 1708 1709 def __hash__(self): 1710 return hash(type(self)) + hash(self._key_coder) 1711 1712 def __repr__(self): 1713 return 'ShardedKeyCoder[%s]' % self._key_coder 1714 1715 1716 Coder.register_structured_urn( 1717 common_urns.coders.SHARDED_KEY.urn, ShardedKeyCoder) 1718 1719 1720 class TimestampPrefixingWindowCoder(FastCoder): 1721 """For internal use only; no backwards-compatibility guarantees. 1722 1723 Coder which prefixes the max timestamp of arbitrary window to its encoded 1724 form.""" 1725 def __init__(self, window_coder: Coder) -> None: 1726 self._window_coder = window_coder 1727 1728 def _create_impl(self): 1729 return coder_impl.TimestampPrefixingWindowCoderImpl( 1730 self._window_coder.get_impl()) 1731 1732 def to_type_hint(self): 1733 return self._window_coder.to_type_hint() 1734 1735 def _get_component_coders(self) -> List[Coder]: 1736 return [self._window_coder] 1737 1738 def is_deterministic(self) -> bool: 1739 return self._window_coder.is_deterministic() 1740 1741 def as_cloud_object(self, coders_context=None): 1742 return { 1743 '@type': 'kind:custom_window', 1744 'component_encodings': [ 1745 self._window_coder.as_cloud_object(coders_context) 1746 ], 1747 } 1748 1749 def __repr__(self): 1750 return 'TimestampPrefixingWindowCoder[%r]' % self._window_coder 1751 1752 def __eq__(self, other): 1753 return ( 1754 type(self) == type(other) and self._window_coder == other._window_coder) 1755 1756 def __hash__(self): 1757 return hash((type(self), self._window_coder)) 1758 1759 1760 Coder.register_structured_urn( 1761 common_urns.coders.CUSTOM_WINDOW.urn, TimestampPrefixingWindowCoder) 1762 1763 1764 class BigIntegerCoder(FastCoder): 1765 def _create_impl(self): 1766 return coder_impl.BigIntegerCoderImpl() 1767 1768 def is_deterministic(self): 1769 # type: () -> bool 1770 return True 1771 1772 def to_type_hint(self): 1773 return int 1774 1775 def __eq__(self, other): 1776 return type(self) == type(other) 1777 1778 def __hash__(self): 1779 return hash(type(self)) 1780 1781 1782 class DecimalCoder(FastCoder): 1783 def _create_impl(self): 1784 return coder_impl.DecimalCoderImpl() 1785 1786 def is_deterministic(self): 1787 # type: () -> bool 1788 return True 1789 1790 def to_type_hint(self): 1791 return decimal.Decimal 1792 1793 def __eq__(self, other): 1794 return type(self) == type(other) 1795 1796 def __hash__(self): 1797 return hash(type(self))