|
59 | 59 |
|
60 | 60 | from apache_beam.coders import coder_impl |
61 | 61 | from apache_beam.coders.avro_record import AvroRecord |
| 62 | +from apache_beam.internal import cloudpickle_pickler |
| 63 | +from apache_beam.internal import pickler |
62 | 64 | from apache_beam.portability import common_urns |
63 | 65 | from apache_beam.portability import python_urns |
64 | 66 | from apache_beam.portability.api import beam_runner_api_pb2 |
|
77 | 79 | # pylint: enable=wrong-import-order, wrong-import-position, ungrouped-imports |
78 | 80 |
|
79 | 81 | # pylint: disable=wrong-import-order, wrong-import-position |
80 | | -# Avoid dependencies on the full SDK. |
81 | 82 | 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 |
| 83 | + from apache_beam.internal import dill_pickler |
85 | 84 | 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 |
| 85 | + dill_pickler = None # type: ignore |
89 | 86 |
|
90 | 87 | __all__ = [ |
91 | 88 | 'Coder', |
92 | 89 | 'AvroGenericCoder', |
93 | 90 | 'BooleanCoder', |
94 | 91 | 'BytesCoder', |
| 92 | + 'CloudpickleCoder', |
95 | 93 | 'DillCoder', |
96 | 94 | 'FastPrimitivesCoder', |
97 | 95 | 'FloatCoder', |
|
123 | 121 |
|
124 | 122 |
|
125 | 123 | def serialize_coder(coder): |
126 | | - from apache_beam.internal import pickler |
127 | 124 | return b'%s$%s' % ( |
128 | 125 | coder.__class__.__name__.encode('utf-8'), |
129 | 126 | pickler.dumps(coder, use_zlib=True)) |
130 | 127 |
|
131 | 128 |
|
132 | 129 | def deserialize_coder(serialized): |
133 | | - from apache_beam.internal import pickler |
134 | 130 | return pickler.loads(serialized.split(b'$', 1)[1], use_zlib=True) |
135 | 131 |
|
136 | 132 |
|
@@ -812,15 +808,15 @@ def maybe_dill_dumps(o): |
812 | 808 | try: |
813 | 809 | return pickle.dumps(o, pickle.HIGHEST_PROTOCOL) |
814 | 810 | except Exception: # pylint: disable=broad-except |
815 | | - return dill.dumps(o) |
| 811 | + return dill_pickler.dumps(o) |
816 | 812 |
|
817 | 813 |
|
818 | 814 | def maybe_dill_loads(o): |
819 | 815 | """Unpickle using cPickle or the Dill pickler as a fallback.""" |
820 | 816 | try: |
821 | 817 | return pickle.loads(o) |
822 | 818 | except Exception: # pylint: disable=broad-except |
823 | | - return dill.loads(o) |
| 819 | + return dill_pickler.loads(o) |
824 | 820 |
|
825 | 821 |
|
826 | 822 | class _PickleCoderBase(FastCoder): |
@@ -860,7 +856,6 @@ def __init__(self, cache_size=16): |
860 | 856 | self.cache_size = cache_size |
861 | 857 |
|
862 | 858 | def _create_impl(self): |
863 | | - from apache_beam.internal import pickler |
864 | 859 | dumps = pickler.dumps |
865 | 860 |
|
866 | 861 | mdumps = lru_cache(maxsize=self.cache_size, typed=True)(dumps) |
@@ -896,11 +891,24 @@ def to_type_hint(self): |
896 | 891 |
|
897 | 892 |
|
898 | 893 | class DillCoder(_PickleCoderBase): |
899 | | - """Coder using dill's pickle functionality.""" |
| 894 | + def __init__(self): |
| 895 | + """Coder using dill's pickle functionality.""" |
| 896 | + if dill_pickler is None: |
| 897 | + raise ImportError( |
| 898 | + "Dill is not installed. To use DillCoder, please install " |
| 899 | + "apache-beam[dill] or install dill separately.") |
| 900 | + |
900 | 901 | def _create_impl(self): |
901 | 902 | return coder_impl.CallbackCoderImpl(maybe_dill_dumps, maybe_dill_loads) |
902 | 903 |
|
903 | 904 |
|
| 905 | +class CloudpickleCoder(_PickleCoderBase): |
| 906 | + """Coder using Apache Beam's vendored Cloudpickle pickler.""" |
| 907 | + def _create_impl(self): |
| 908 | + return coder_impl.CallbackCoderImpl( |
| 909 | + cloudpickle_pickler.dumps, cloudpickle_pickler.loads) |
| 910 | + |
| 911 | + |
904 | 912 | class DeterministicFastPrimitivesCoder(FastCoder): |
905 | 913 | """Throws runtime errors when encoding non-deterministic values.""" |
906 | 914 | def __init__(self, coder, step_label): |
|
0 commit comments