forked from apache/beam
-
Notifications
You must be signed in to change notification settings - Fork 0
Expand file tree
/
Copy pathcloudpickle_pickler.py
More file actions
245 lines (191 loc) · 7.21 KB
/
cloudpickle_pickler.py
File metadata and controls
245 lines (191 loc) · 7.21 KB
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
#
# 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.
#
"""Pickler for values, functions, and classes.
For internal use only. No backwards compatibility guarantees.
Uses the cloudpickle library to pickle data, functions, lambdas
and classes.
dump_session and load_session are no-ops.
"""
# pytype: skip-file
import base64
import bz2
import io
import logging
import sys
import threading
import zlib
from apache_beam.internal import code_object_pickler
from apache_beam.internal.cloudpickle import cloudpickle
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)
STABLE_CODE_IDENTIFIER_CONFIG = cloudpickle.CloudPickleConfig(
skip_reset_dynamic_type_state=True,
get_code_object_params=cloudpickle.GetCodeObjectParams(
get_code_object_identifier=code_object_pickler.
get_code_object_identifier,
get_code_from_identifier=code_object_pickler.get_code_from_identifier))
try:
from absl import flags
except (ImportError, ModuleNotFoundError):
pass
def _get_proto_enum_descriptor_class():
try:
from google.protobuf.internal import api_implementation
except ImportError:
return None
implementation_type = api_implementation.Type()
if implementation_type == 'upb':
try:
from google._upb._message import EnumDescriptor
return EnumDescriptor
except ImportError:
pass
elif implementation_type == 'cpp':
try:
from google.protobuf.pyext._message import EnumDescriptor
return EnumDescriptor
except ImportError:
pass
elif implementation_type == 'python':
try:
from google.protobuf.internal.python_message import EnumDescriptor
return EnumDescriptor
except ImportError:
pass
return None
EnumDescriptor = _get_proto_enum_descriptor_class()
# Pickling, especially unpickling, causes broken module imports on Python 3
# if executed concurrently, see: BEAM-8651, http://bugs.python.org/issue38884.
_pickle_lock = threading.RLock()
RLOCK_TYPE = type(_pickle_lock)
LOCK_TYPE = type(threading.Lock())
_LOGGER = logging.getLogger(__name__)
def _reconstruct_enum_descriptor(full_name):
for _, module in list(sys.modules.items()):
if not hasattr(module, 'DESCRIPTOR'):
continue
if hasattr(module.DESCRIPTOR, 'enum_types_by_name'):
for (_, enum_desc) in module.DESCRIPTOR.enum_types_by_name.items():
if enum_desc.full_name == full_name:
return enum_desc
for _, attr_value in vars(module).items():
if not hasattr(attr_value, 'DESCRIPTOR'):
continue
if hasattr(attr_value.DESCRIPTOR, 'enum_types_by_name'):
for (_, enum_desc) in attr_value.DESCRIPTOR.enum_types_by_name.items():
if enum_desc.full_name == full_name:
return enum_desc
raise ImportError(f'Could not find enum descriptor: {full_name}')
def _pickle_enum_descriptor(obj):
full_name = obj.full_name
return _reconstruct_enum_descriptor, (full_name, )
def dumps(
o,
enable_trace=True,
use_zlib=False,
enable_best_effort_determinism=False,
enable_stable_code_identifier_pickling=False,
config: cloudpickle.CloudPickleConfig = DEFAULT_CONFIG) -> bytes:
"""For internal use only; no backwards-compatibility guarantees."""
s = _dumps(
o,
enable_best_effort_determinism,
enable_stable_code_identifier_pickling,
config)
# Compress as compactly as possible (compresslevel=9) to decrease peak memory
# usage (of multiple in-memory copies) and to avoid hitting protocol buffer
# limits.
# WARNING: Be cautious about compressor change since it can lead to pipeline
# representation change, and can break streaming job update compatibility on
# runners such as Dataflow.
if use_zlib:
c = zlib.compress(s, 9)
else:
c = bz2.compress(s, compresslevel=9)
del s # Free up some possibly large and no-longer-needed memory.
return base64.b64encode(c)
def _dumps(
o,
enable_best_effort_determinism=False,
enable_stable_code_identifier_pickling=False,
config: cloudpickle.CloudPickleConfig = DEFAULT_CONFIG) -> bytes:
if enable_stable_code_identifier_pickling:
config = STABLE_CODE_IDENTIFIER_CONFIG
config_kwargs = config.__dict__.copy()
if enable_best_effort_determinism:
# TODO: Add support once https://github.com/cloudpipe/cloudpickle/pull/563
# is merged in.
config_kwargs['filepath_interceptor'] = cloudpickle.get_relative_path
_LOGGER.info(
'Option not fully supported:'
'enable_best_effort_determinism is True: Applying file path '
'normalization for pickling.'
'This has been fully implemented for dill.')
final_config = cloudpickle.CloudPickleConfig(**config_kwargs)
with _pickle_lock:
with io.BytesIO() as file:
pickler = cloudpickle.CloudPickler(file, config=final_config)
try:
pickler.dispatch_table[type(flags.FLAGS)] = _pickle_absl_flags
except NameError:
pass
try:
pickler.dispatch_table[RLOCK_TYPE] = _pickle_rlock
except NameError:
pass
try:
pickler.dispatch_table[LOCK_TYPE] = _lock_reducer
except NameError:
pass
if EnumDescriptor is not None:
pickler.dispatch_table[EnumDescriptor] = _pickle_enum_descriptor
pickler.dump(o)
return file.getvalue()
def loads(encoded, enable_trace=True, use_zlib=False):
"""For internal use only; no backwards-compatibility guarantees."""
c = base64.b64decode(encoded)
if use_zlib:
s = zlib.decompress(c)
else:
s = bz2.decompress(c)
del c # Free up some possibly large and no-longer-needed memory.
return _loads(s)
def _loads(s):
with _pickle_lock:
unpickled = cloudpickle.loads(s)
return unpickled
def roundtrip(o):
"""Internal utility for testing round-trip pickle serialization."""
return _loads(_dumps(o))
def _pickle_absl_flags(obj):
return _create_absl_flags, tuple([])
def _create_absl_flags():
return flags.FLAGS
def _pickle_rlock(obj):
return RLOCK_TYPE, tuple([])
def _lock_reducer(obj):
return threading.Lock, tuple([])
def dump_session(file_path):
# It is possible to dump session with cloudpickle. However, since references
# are saved it should not be necessary. See https://s.apache.org/beam-picklers
pass
def load_session(file_path):
# It is possible to load_session with cloudpickle. However, since references
# are saved it should not be necessary. See https://s.apache.org/beam-picklers
pass