-
Notifications
You must be signed in to change notification settings - Fork 4.3k
/
coders.py
1664 lines (1241 loc) · 47.2 KB
/
coders.py
1
2
3
4
5
6
7
8
9
10
11
12
13
14
15
16
17
18
19
20
21
22
23
24
25
26
27
28
29
30
31
32
33
34
35
36
37
38
39
40
41
42
43
44
45
46
47
48
49
50
51
52
53
54
55
56
57
58
59
60
61
62
63
64
65
66
67
68
69
70
71
72
73
74
75
76
77
78
79
80
81
82
83
84
85
86
87
88
89
90
91
92
93
94
95
96
97
98
99
100
101
102
103
104
105
106
107
108
109
110
111
112
113
114
115
116
117
118
119
120
121
122
123
124
125
126
127
128
129
130
131
132
133
134
135
136
137
138
139
140
141
142
143
144
145
146
147
148
149
150
151
152
153
154
155
156
157
158
159
160
161
162
163
164
165
166
167
168
169
170
171
172
173
174
175
176
177
178
179
180
181
182
183
184
185
186
187
188
189
190
191
192
193
194
195
196
197
198
199
200
201
202
203
204
205
206
207
208
209
210
211
212
213
214
215
216
217
218
219
220
221
222
223
224
225
226
227
228
229
230
231
232
233
234
235
236
237
238
239
240
241
242
243
244
245
246
247
248
249
250
251
252
253
254
255
256
257
258
259
260
261
262
263
264
265
266
267
268
269
270
271
272
273
274
275
276
277
278
279
280
281
282
283
284
285
286
287
288
289
290
291
292
293
294
295
296
297
298
299
300
301
302
303
304
305
306
307
308
309
310
311
312
313
314
315
316
317
318
319
320
321
322
323
324
325
326
327
328
329
330
331
332
333
334
335
336
337
338
339
340
341
342
343
344
345
346
347
348
349
350
351
352
353
354
355
356
357
358
359
360
361
362
363
364
365
366
367
368
369
370
371
372
373
374
375
376
377
378
379
380
381
382
383
384
385
386
387
388
389
390
391
392
393
394
395
396
397
398
399
400
401
402
403
404
405
406
407
408
409
410
411
412
413
414
415
416
417
418
419
420
421
422
423
424
425
426
427
428
429
430
431
432
433
434
435
436
437
438
439
440
441
442
443
444
445
446
447
448
449
450
451
452
453
454
455
456
457
458
459
460
461
462
463
464
465
466
467
468
469
470
471
472
473
474
475
476
477
478
479
480
481
482
483
484
485
486
487
488
489
490
491
492
493
494
495
496
497
498
499
500
501
502
503
504
505
506
507
508
509
510
511
512
513
514
515
516
517
518
519
520
521
522
523
524
525
526
527
528
529
530
531
532
533
534
535
536
537
538
539
540
541
542
543
544
545
546
547
548
549
550
551
552
553
554
555
556
557
558
559
560
561
562
563
564
565
566
567
568
569
570
571
572
573
574
575
576
577
578
579
580
581
582
583
584
585
586
587
588
589
590
591
592
593
594
595
596
597
598
599
600
601
602
603
604
605
606
607
608
609
610
611
612
613
614
615
616
617
618
619
620
621
622
623
624
625
626
627
628
629
630
631
632
633
634
635
636
637
638
639
640
641
642
643
644
645
646
647
648
649
650
651
652
653
654
655
656
657
658
659
660
661
662
663
664
665
666
667
668
669
670
671
672
673
674
675
676
677
678
679
680
681
682
683
684
685
686
687
688
689
690
691
692
693
694
695
696
697
698
699
700
701
702
703
704
705
706
707
708
709
710
711
712
713
714
715
716
717
718
719
720
721
722
723
724
725
726
727
728
729
730
731
732
733
734
735
736
737
738
739
740
741
742
743
744
745
746
747
748
749
750
751
752
753
754
755
756
757
758
759
760
761
762
763
764
765
766
767
768
769
770
771
772
773
774
775
776
777
778
779
780
781
782
783
784
785
786
787
788
789
790
791
792
793
794
795
796
797
798
799
800
801
802
803
804
805
806
807
808
809
810
811
812
813
814
815
816
817
818
819
820
821
822
823
824
825
826
827
828
829
830
831
832
833
834
835
836
837
838
839
840
841
842
843
844
845
846
847
848
849
850
851
852
853
854
855
856
857
858
859
860
861
862
863
864
865
866
867
868
869
870
871
872
873
874
875
876
877
878
879
880
881
882
883
884
885
886
887
888
889
890
891
892
893
894
895
896
897
898
899
900
901
902
903
904
905
906
907
908
909
910
911
912
913
914
915
916
917
918
919
920
921
922
923
924
925
926
927
928
929
930
931
932
933
934
935
936
937
938
939
940
941
942
943
944
945
946
947
948
949
950
951
952
953
954
955
956
957
958
959
960
961
962
963
964
965
966
967
968
969
970
971
972
973
974
975
976
977
978
979
980
981
982
983
984
985
986
987
988
989
990
991
992
993
994
995
996
997
998
999
1000
#
# Licensed to the Apache Software Foundation (ASF) under one or more
# contributor license agreements. See the NOTICE file distributed with
# this work for additional information regarding copyright ownership.
# The ASF licenses this file to You under the Apache License, Version 2.0
# (the "License"); you may not use this file except in compliance with
# the License. You may obtain a copy of the License at
#
# http://www.apache.org/licenses/LICENSE-2.0
#
# Unless required by applicable law or agreed to in writing, software
# distributed under the License is distributed on an "AS IS" BASIS,
# WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
# See the License for the specific language governing permissions and
# limitations under the License.
#
"""Collection of useful coders.
Only those coders listed in __all__ are part of the public API of this module.
## On usage of `pickle`, `dill` and `pickler` in Beam
In Beam, we generally we use `pickle` for pipeline elements and `dill` for
more complex types, like user functions.
`pickler` is Beam's own wrapping of dill + compression + error handling.
It serves also as an API to mask the actual encoding layer (so we can
change it from `dill` if necessary).
We created `_MemoizingPickleCoder` to improve performance when serializing
complex user types for the execution of SDF. Specifically to address
BEAM-12781, where many identical `BoundedSource` instances are being
encoded.
"""
# pytype: skip-file
import base64
import decimal
import pickle
from functools import lru_cache
from typing import TYPE_CHECKING
from typing import Any
from typing import Callable
from typing import Dict
from typing import Iterable
from typing import List
from typing import Optional
from typing import Sequence
from typing import Tuple
from typing import Type
from typing import TypeVar
from typing import overload
import google.protobuf.wrappers_pb2
import proto
from apache_beam.coders import coder_impl
from apache_beam.coders.avro_record import AvroRecord
from apache_beam.portability import common_urns
from apache_beam.portability import python_urns
from apache_beam.portability.api import beam_runner_api_pb2
from apache_beam.typehints import typehints
from apache_beam.utils import proto_utils
if TYPE_CHECKING:
from google.protobuf import message # pylint: disable=ungrouped-imports
from apache_beam.coders.typecoders import CoderRegistry
from apache_beam.runners.pipeline_context import PipelineContext
# pylint: disable=wrong-import-order, wrong-import-position, ungrouped-imports
try:
from .stream import get_varint_size
except ImportError:
from .slow_stream import get_varint_size
# pylint: enable=wrong-import-order, wrong-import-position, ungrouped-imports
# pylint: disable=wrong-import-order, wrong-import-position
# Avoid dependencies on the full SDK.
try:
# Import dill from the pickler module to make sure our monkey-patching of dill
# occurs.
from apache_beam.internal.dill_pickler import dill
except ImportError:
# We fall back to using the stock dill library in tests that don't use the
# full Python SDK.
import dill
__all__ = [
'Coder',
'AvroGenericCoder',
'BooleanCoder',
'BytesCoder',
'DillCoder',
'FastPrimitivesCoder',
'FloatCoder',
'IterableCoder',
'ListCoder',
'MapCoder',
'NullableCoder',
'PickleCoder',
'ProtoCoder',
'ProtoPlusCoder',
'ShardedKeyCoder',
'SinglePrecisionFloatCoder',
'SingletonCoder',
'StrUtf8Coder',
'TimestampCoder',
'TupleCoder',
'TupleSequenceCoder',
'VarIntCoder',
'WindowedValueCoder',
'ParamWindowedValueCoder',
'BigIntegerCoder',
'DecimalCoder'
]
T = TypeVar('T')
CoderT = TypeVar('CoderT', bound='Coder')
ProtoCoderT = TypeVar('ProtoCoderT', bound='ProtoCoder')
ConstructorFn = Callable[[Optional[Any], List['Coder'], 'PipelineContext'], Any]
def serialize_coder(coder):
from apache_beam.internal import pickler
return b'%s$%s' % (
coder.__class__.__name__.encode('utf-8'),
pickler.dumps(coder, use_zlib=True))
def deserialize_coder(serialized):
from apache_beam.internal import pickler
return pickler.loads(serialized.split(b'$', 1)[1], use_zlib=True)
# pylint: enable=wrong-import-order, wrong-import-position
class Coder(object):
"""Base class for coders."""
def encode(self, value):
# type: (Any) -> bytes
"""Encodes the given object into a byte string."""
raise NotImplementedError('Encode not implemented: %s.' % self)
def decode(self, encoded):
"""Decodes the given byte string into the corresponding object."""
raise NotImplementedError('Decode not implemented: %s.' % self)
def encode_nested(self, value):
"""Uses the underlying implementation to encode in nested format."""
return self.get_impl().encode_nested(value)
def decode_nested(self, encoded):
"""Uses the underlying implementation to decode in nested format."""
return self.get_impl().decode_nested(encoded)
def is_deterministic(self):
# type: () -> bool
"""Whether this coder is guaranteed to encode values deterministically.
A deterministic coder is required for key coders in GroupByKey operations
to produce consistent results.
For example, note that the default coder, the PickleCoder, is not
deterministic: the ordering of picked entries in maps may vary across
executions since there is no defined order, and such a coder is not in
general suitable for usage as a key coder in GroupByKey operations, since
each instance of the same key may be encoded differently.
Returns:
Whether coder is deterministic.
"""
return False
def as_deterministic_coder(self, step_label, error_message=None):
"""Returns a deterministic version of self, if possible.
Otherwise raises a value error.
"""
if self.is_deterministic():
return self
else:
raise ValueError(
error_message or
"%s cannot be made deterministic for '%s'." % (self, step_label))
def estimate_size(self, value):
"""Estimates the encoded size of the given value, in bytes.
Dataflow estimates the encoded size of a PCollection processed in a pipeline
step by using the estimated size of a random sample of elements in that
PCollection.
The default implementation encodes the given value and returns its byte
size. If a coder can provide a fast estimate of the encoded size of a value
(e.g., if the encoding has a fixed size), it can provide its estimate here
to improve performance.
Arguments:
value: the value whose encoded size is to be estimated.
Returns:
The estimated encoded size of the given value.
"""
return len(self.encode(value))
# ===========================================================================
# Methods below are internal SDK details that don't need to be modified for
# user-defined coders.
# ===========================================================================
def _create_impl(self):
# type: () -> coder_impl.CoderImpl
"""Creates a CoderImpl to do the actual encoding and decoding.
"""
return coder_impl.CallbackCoderImpl(
self.encode, self.decode, self.estimate_size)
def get_impl(self):
"""For internal use only; no backwards-compatibility guarantees.
Returns the CoderImpl backing this Coder.
"""
if not hasattr(self, '_impl'):
self._impl = self._create_impl()
assert isinstance(self._impl, coder_impl.CoderImpl)
return self._impl
def __getstate__(self):
return self._dict_without_impl()
def _dict_without_impl(self):
if hasattr(self, '_impl'):
d = dict(self.__dict__)
del d['_impl']
return d
return self.__dict__
def to_type_hint(self):
raise NotImplementedError(
'https://github.com/apache/beam/issues/18490: %s' %
self.__class__.__name__)
@classmethod
def from_type_hint(cls, unused_typehint, unused_registry):
# type: (Type[CoderT], Any, CoderRegistry) -> CoderT
# If not overridden, just construct the coder without arguments.
return cls()
def is_kv_coder(self):
# type: () -> bool
return False
def key_coder(self):
# type: () -> Coder
if self.is_kv_coder():
raise NotImplementedError('key_coder: %s' % self)
else:
raise ValueError('Not a KV coder: %s.' % self)
def value_coder(self):
# type: () -> Coder
if self.is_kv_coder():
raise NotImplementedError('value_coder: %s' % self)
else:
raise ValueError('Not a KV coder: %s.' % self)
def _get_component_coders(self):
# type: () -> Sequence[Coder]
"""For internal use only; no backwards-compatibility guarantees.
Returns the internal component coders of this coder."""
# This is an internal detail of the Coder API and does not need to be
# refined in user-defined Coders.
return []
def __repr__(self):
return self.__class__.__name__
# pylint: disable=protected-access
def __eq__(self, other):
return (
self.__class__ == other.__class__ and
self._dict_without_impl() == other._dict_without_impl())
# pylint: enable=protected-access
def __hash__(self):
return hash(type(self))
_known_urns = {} # type: Dict[str, Tuple[type, ConstructorFn]]
@classmethod
@overload
def register_urn(
cls,
urn, # type: str
parameter_type, # type: Optional[Type[T]]
):
# type: (...) -> Callable[[Callable[[T, List[Coder], PipelineContext], Any]], Callable[[T, List[Coder], PipelineContext], Any]]
pass
@classmethod
@overload
def register_urn(
cls,
urn, # type: str
parameter_type, # type: Optional[Type[T]]
fn # type: Callable[[T, List[Coder], PipelineContext], Any]
):
# type: (...) -> None
pass
@classmethod
def register_urn(cls, urn, parameter_type, fn=None):
"""Registers a urn with a constructor.
For example, if 'beam:fn:foo' had parameter type FooPayload, one could
write `RunnerApiFn.register_urn('bean:fn:foo', FooPayload, foo_from_proto)`
where foo_from_proto took as arguments a FooPayload and a PipelineContext.
This function can also be used as a decorator rather than passing the
callable in as the final parameter.
A corresponding to_runner_api_parameter method would be expected that
returns the tuple ('beam:fn:foo', FooPayload)
"""
def register(fn):
cls._known_urns[urn] = parameter_type, fn
return fn
if fn:
# Used as a statement.
register(fn)
else:
# Used as a decorator.
return register
def to_runner_api(self, context):
# type: (PipelineContext) -> beam_runner_api_pb2.Coder
urn, typed_param, components = self.to_runner_api_parameter(context)
return beam_runner_api_pb2.Coder(
spec=beam_runner_api_pb2.FunctionSpec(
urn=urn,
payload=typed_param if isinstance(typed_param, (bytes, type(None)))
else typed_param.SerializeToString()),
component_coder_ids=[context.coders.get_id(c) for c in components])
@classmethod
def from_runner_api(cls, coder_proto, context):
# type: (Type[CoderT], beam_runner_api_pb2.Coder, PipelineContext) -> CoderT
"""Converts from an FunctionSpec to a Fn object.
Prefer registering a urn with its parameter type and constructor.
"""
parameter_type, constructor = cls._known_urns[coder_proto.spec.urn]
return constructor(
proto_utils.parse_Bytes(coder_proto.spec.payload, parameter_type),
[context.coders.get_by_id(c) for c in coder_proto.component_coder_ids],
context)
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)),
())
@staticmethod
def register_structured_urn(urn, cls):
# type: (str, Type[Coder]) -> None
"""Register a coder that's completely defined by its urn and its
component(s), if any, which are passed to construct the instance.
"""
setattr(
cls,
'to_runner_api_parameter',
lambda self,
unused_context: (urn, None, self._get_component_coders()))
# pylint: disable=unused-variable
@Coder.register_urn(urn, None)
def from_runner_api_parameter(unused_payload, components, unused_context):
if components:
return cls(*components)
else:
return cls()
@Coder.register_urn(
python_urns.PICKLED_CODER, google.protobuf.wrappers_pb2.BytesValue)
def _pickle_from_runner_api_parameter(payload, components, context):
return deserialize_coder(payload.value)
class StrUtf8Coder(Coder):
"""A coder used for reading and writing strings as UTF-8."""
def encode(self, value):
return value.encode('utf-8')
def decode(self, value):
return value.decode('utf-8')
def is_deterministic(self):
# type: () -> bool
return True
def to_type_hint(self):
return str
Coder.register_structured_urn(common_urns.coders.STRING_UTF8.urn, StrUtf8Coder)
class ToBytesCoder(Coder):
"""A default string coder used if no sink coder is specified."""
def encode(self, value):
return value if isinstance(value, bytes) else str(value).encode('utf-8')
def decode(self, _):
raise NotImplementedError('ToBytesCoder cannot be used for decoding.')
def is_deterministic(self):
# type: () -> bool
return True
# alias to the old class name for a courtesy to users who reference it
ToStringCoder = ToBytesCoder
class FastCoder(Coder):
"""Coder subclass used when a (faster) CoderImpl is supplied directly.
The Coder class defines _create_impl in terms of encode() and decode();
this class inverts that by defining encode() and decode() in terms of
_create_impl().
"""
def encode(self, value):
"""Encodes the given object into a byte string."""
return self.get_impl().encode(value)
def decode(self, encoded):
"""Decodes the given byte string into the corresponding object."""
return self.get_impl().decode(encoded)
def estimate_size(self, value):
return self.get_impl().estimate_size(value)
def _create_impl(self):
raise NotImplementedError
class BytesCoder(FastCoder):
"""Byte string coder."""
def _create_impl(self):
return coder_impl.BytesCoderImpl()
def is_deterministic(self):
# type: () -> bool
return True
def to_type_hint(self):
return bytes
def __eq__(self, other):
return type(self) == type(other)
def __hash__(self):
return hash(type(self))
Coder.register_structured_urn(common_urns.coders.BYTES.urn, BytesCoder)
class BooleanCoder(FastCoder):
def _create_impl(self):
return coder_impl.BooleanCoderImpl()
def is_deterministic(self):
# type: () -> bool
return True
def to_type_hint(self):
return bool
def __eq__(self, other):
return type(self) == type(other)
def __hash__(self):
return hash(type(self))
Coder.register_structured_urn(common_urns.coders.BOOL.urn, BooleanCoder)
class MapCoder(FastCoder):
def __init__(self, key_coder, value_coder):
# type: (Coder, Coder) -> None
self._key_coder = key_coder
self._value_coder = value_coder
def _create_impl(self):
return coder_impl.MapCoderImpl(
self._key_coder.get_impl(), self._value_coder.get_impl())
@classmethod
def from_type_hint(cls, typehint, registry):
# type: (typehints.DictConstraint, CoderRegistry) -> MapCoder
return cls(
registry.get_coder(typehint.key_type),
registry.get_coder(typehint.value_type))
def to_type_hint(self):
return typehints.Dict[self._key_coder.to_type_hint(),
self._value_coder.to_type_hint()]
def is_deterministic(self):
# type: () -> bool
# Map ordering is non-deterministic
return False
def as_deterministic_coder(self, step_label, error_message=None):
return DeterministicMapCoder(
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 (
type(self) == type(other) and self._key_coder == other._key_coder and
self._value_coder == other._value_coder)
def __hash__(self):
return hash(type(self)) + hash(self._key_coder) + hash(self._value_coder)
def __repr__(self):
return 'MapCoder[%s, %s]' % (self._key_coder, self._value_coder)
# This is a separate class from MapCoder as the former is a standard coder with
# no way to carry the is_deterministic bit.
class DeterministicMapCoder(FastCoder):
def __init__(self, key_coder, value_coder):
# type: (Coder, Coder) -> None
assert key_coder.is_deterministic()
assert value_coder.is_deterministic()
self._key_coder = key_coder
self._value_coder = value_coder
def _create_impl(self):
return coder_impl.MapCoderImpl(
self._key_coder.get_impl(), self._value_coder.get_impl(), True)
def is_deterministic(self):
return True
def __eq__(self, other):
return (
type(self) == type(other) and self._key_coder == other._key_coder and
self._value_coder == other._value_coder)
def __hash__(self):
return hash(type(self)) + hash(self._key_coder) + hash(self._value_coder)
def __repr__(self):
return 'DeterministicMapCoder[%s, %s]' % (
self._key_coder, self._value_coder)
class NullableCoder(FastCoder):
def __init__(self, value_coder):
# type: (Coder) -> None
self._value_coder = value_coder
def _create_impl(self):
return coder_impl.NullableCoderImpl(self._value_coder.get_impl())
def to_type_hint(self):
return typehints.Optional[self._value_coder.to_type_hint()]
def _get_component_coders(self):
# type: () -> List[Coder]
return [self._value_coder]
@classmethod
def from_type_hint(cls, typehint, registry):
if typehints.is_nullable(typehint):
return cls(
registry.get_coder(
typehints.get_concrete_type_from_nullable(typehint)))
else:
raise TypeError(
'Typehint is not of nullable type, '
'and cannot be converted to a NullableCoder',
typehint)
def is_deterministic(self):
# type: () -> bool
return self._value_coder.is_deterministic()
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)
return NullableCoder(deterministic_value_coder)
def __eq__(self, other):
return (
type(self) == type(other) and self._value_coder == other._value_coder)
def __hash__(self):
return hash(type(self)) + hash(self._value_coder)
def __repr__(self):
return 'NullableCoder[%s]' % self._value_coder
Coder.register_structured_urn(common_urns.coders.NULLABLE.urn, NullableCoder)
class VarIntCoder(FastCoder):
"""Variable-length integer coder."""
def _create_impl(self):
return coder_impl.VarIntCoderImpl()
def is_deterministic(self):
# type: () -> bool
return True
def to_type_hint(self):
return int
def __eq__(self, other):
return type(self) == type(other)
def __hash__(self):
return hash(type(self))
Coder.register_structured_urn(common_urns.coders.VARINT.urn, VarIntCoder)
class BigEndianShortCoder(FastCoder):
"""A coder used for big-endian int16 values."""
def _create_impl(self):
return coder_impl.BigEndianShortCoderImpl()
def is_deterministic(self):
# type: () -> bool
return True
def to_type_hint(self):
return int
def __eq__(self, other):
return type(self) == type(other)
def __hash__(self):
return hash(type(self))
class SinglePrecisionFloatCoder(FastCoder):
"""A coder used for single-precision floating-point values."""
def _create_impl(self):
return coder_impl.SinglePrecisionFloatCoderImpl()
def is_deterministic(self):
# type: () -> bool
return True
def to_type_hint(self):
return float
def __eq__(self, other):
return type(self) == type(other)
def __hash__(self):
return hash(type(self))
class FloatCoder(FastCoder):
"""A coder used for **double-precision** floating-point values.
Note that the name "FloatCoder" is in reference to Python's ``float`` built-in
which is generally implemented using C doubles. See
:class:`SinglePrecisionFloatCoder` for a single-precision version of this
coder.
"""
def _create_impl(self):
return coder_impl.FloatCoderImpl()
def is_deterministic(self):
# type: () -> bool
return True
def to_type_hint(self):
return float
def __eq__(self, other):
return type(self) == type(other)
def __hash__(self):
return hash(type(self))
Coder.register_structured_urn(common_urns.coders.DOUBLE.urn, FloatCoder)
class TimestampCoder(FastCoder):
"""A coder used for timeutil.Timestamp values."""
def _create_impl(self):
return coder_impl.TimestampCoderImpl()
def is_deterministic(self):
# type: () -> bool
return True
def __eq__(self, other):
return type(self) == type(other)
def __hash__(self):
return hash(type(self))
class _TimerCoder(FastCoder):
"""A coder used for timer values.
For internal use."""
def __init__(self, key_coder, window_coder):
# type: (Coder, Coder) -> None
self._key_coder = key_coder
self._window_coder = window_coder
def _get_component_coders(self):
# type: () -> List[Coder]
return [self._key_coder, self._window_coder]
def _create_impl(self):
return coder_impl.TimerCoderImpl(
self._key_coder.get_impl(), self._window_coder.get_impl())
def is_deterministic(self):
# type: () -> bool
return (
self._key_coder.is_deterministic() and
self._window_coder.is_deterministic())
def __eq__(self, other):
return (
type(self) == type(other) and self._key_coder == other._key_coder and
self._window_coder == other._window_coder)
def __hash__(self):
return hash(type(self)) + hash(self._key_coder) + hash(self._window_coder)
Coder.register_structured_urn(common_urns.coders.TIMER.urn, _TimerCoder)
class SingletonCoder(FastCoder):
"""A coder that always encodes exactly one value."""
def __init__(self, value):
self._value = value
def _create_impl(self):
return coder_impl.SingletonCoderImpl(self._value)
def is_deterministic(self):
# type: () -> bool
return True
def __eq__(self, other):
return type(self) == type(other) and self._value == other._value
def __hash__(self):
return hash(self._value)
def maybe_dill_dumps(o):
"""Pickle using cPickle or the Dill pickler as a fallback."""
# We need to use the dill pickler for objects of certain custom classes,
# including, for example, ones that contain lambdas.
try:
return pickle.dumps(o, pickle.HIGHEST_PROTOCOL)
except Exception: # pylint: disable=broad-except
return dill.dumps(o)
def maybe_dill_loads(o):
"""Unpickle using cPickle or the Dill pickler as a fallback."""
try:
return pickle.loads(o)
except Exception: # pylint: disable=broad-except
return dill.loads(o)
class _PickleCoderBase(FastCoder):
"""Base class for pickling coders."""
def is_deterministic(self):
# type: () -> bool
# Note that the default coder, the PickleCoder, is not deterministic (for
# example, the ordering of picked entries in maps may vary across
# executions), and so is not in general suitable for usage as a key coder in
# GroupByKey operations.
return False
# We allow .key_coder() and .value_coder() to be called on PickleCoder since
# we can't always infer the return values of lambdas in ParDo operations, the
# result of which may be used in a GroupBykey.
def is_kv_coder(self):
# type: () -> bool
return True
def key_coder(self):
return self
def value_coder(self):
return self
def __eq__(self, other):
return type(self) == type(other)
def __hash__(self):
return hash(type(self))
class _MemoizingPickleCoder(_PickleCoderBase):
"""Coder using Python's pickle functionality with memoization."""
def __init__(self, cache_size=16):
super().__init__()
self.cache_size = cache_size
def _create_impl(self):
from apache_beam.internal import pickler
dumps = pickler.dumps
mdumps = lru_cache(maxsize=self.cache_size, typed=True)(dumps)
def _nonhashable_dumps(x):
try:
return mdumps(x)
except TypeError:
return dumps(x)
return coder_impl.CallbackCoderImpl(_nonhashable_dumps, pickler.loads)
def as_deterministic_coder(self, step_label, error_message=None):
return FastPrimitivesCoder(self, requires_deterministic=step_label)
def to_type_hint(self):
return Any
class PickleCoder(_PickleCoderBase):
"""Coder using Python's pickle functionality."""
def _create_impl(self):
dumps = pickle.dumps
protocol = pickle.HIGHEST_PROTOCOL
return coder_impl.CallbackCoderImpl(
lambda x: dumps(x, protocol), pickle.loads)
def as_deterministic_coder(self, step_label, error_message=None):
return FastPrimitivesCoder(self, requires_deterministic=step_label)
def to_type_hint(self):
return Any
class DillCoder(_PickleCoderBase):
"""Coder using dill's pickle functionality."""
def _create_impl(self):
return coder_impl.CallbackCoderImpl(maybe_dill_dumps, maybe_dill_loads)
class DeterministicFastPrimitivesCoder(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)
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
class FastPrimitivesCoder(FastCoder):
"""Encodes simple primitives (e.g. str, int) efficiently.
For unknown types, falls back to another coder (e.g. PickleCoder).
"""
def __init__(self, fallback_coder=PickleCoder()):
# type: (Coder) -> None
self._fallback_coder = fallback_coder
def _create_impl(self):
return coder_impl.FastPrimitivesCoderImpl(self._fallback_coder.get_impl())
def is_deterministic(self):
# type: () -> bool
return self._fallback_coder.is_deterministic()
def as_deterministic_coder(self, step_label, error_message=None):
if self.is_deterministic():
return self
else:
return DeterministicFastPrimitivesCoder(self, step_label)
def to_type_hint(self):
return Any
# We allow .key_coder() and .value_coder() to be called on FastPrimitivesCoder
# since we can't always infer the return values of lambdas in ParDo
# operations, the result of which may be used in a GroupBykey.
def is_kv_coder(self):
# type: () -> bool
return True
def key_coder(self):
return self
def value_coder(self):
return self
def __eq__(self, other):
return type(self) == type(other)
def __hash__(self):
return hash(type(self))
class FakeDeterministicFastPrimitivesCoder(FastPrimitivesCoder):
"""A FastPrimitivesCoder that claims to be deterministic.
This can be registered as a fallback coder to go back to the behavior before
deterministic encoding was enforced (BEAM-11719).
"""
def is_deterministic(self):
return True
class Base64PickleCoder(Coder):
"""Coder of objects by Python pickle, then base64 encoding."""
# TODO(robertwb): Do base64 encoding where it's needed (e.g. in json) rather
# than via a special Coder.
def encode(self, value):
return base64.b64encode(pickle.dumps(value, pickle.HIGHEST_PROTOCOL))
def decode(self, encoded):
return pickle.loads(base64.b64decode(encoded))
def is_deterministic(self):
# type: () -> bool
# Note that the Base64PickleCoder is not deterministic. See the
# corresponding comments for PickleCoder above.
return False
# We allow .key_coder() and .value_coder() to be called on Base64PickleCoder
# since we can't always infer the return values of lambdas in ParDo
# operations, the result of which may be used in a GroupBykey.
#
# TODO(ccy): this is currently only used for KV values from Create transforms.
# Investigate a way to unify this with PickleCoder.
def is_kv_coder(self):
# type: () -> bool
return True
def key_coder(self):
return self
def value_coder(self):
return self