@@ -3147,33 +3147,40 @@ def process(self, element):
3147
3147
yield pvalue .TaggedOutput ('hot' , ((self ._nonce % fanout , key ), value ))
3148
3148
3149
3149
class PreCombineFn (CombineFn ):
3150
+ def __init__ (self ):
3151
+ # Deepcopy of the combine_fn to avoid sharing state between lifted
3152
+ # stages when using cloudpickle.
3153
+ self ._combine_fn_copy = copy .deepcopy (combine_fn )
3154
+ self .setup = self ._combine_fn_copy .setup
3155
+ self .create_accumulator = self ._combine_fn_copy .create_accumulator
3156
+ self .add_input = self ._combine_fn_copy .add_input
3157
+ self .merge_accumulators = self ._combine_fn_copy .merge_accumulators
3158
+ self .compact = self ._combine_fn_copy .compact
3159
+ self .teardown = self ._combine_fn_copy .teardown
3160
+
3150
3161
@staticmethod
3151
3162
def extract_output (accumulator ):
3152
3163
# Boolean indicates this is an accumulator.
3153
3164
return (True , accumulator )
3154
3165
3155
- setup = combine_fn .setup
3156
- create_accumulator = combine_fn .create_accumulator
3157
- add_input = combine_fn .add_input
3158
- merge_accumulators = combine_fn .merge_accumulators
3159
- compact = combine_fn .compact
3160
- teardown = combine_fn .teardown
3161
-
3162
3166
class PostCombineFn (CombineFn ):
3163
- @staticmethod
3164
- def add_input (accumulator , element ):
3167
+ def __init__ (self ):
3168
+ # Deepcopy of the combine_fn to avoid sharing state between lifted
3169
+ # stages when using cloudpickle.
3170
+ self ._combine_fn_copy = copy .deepcopy (combine_fn )
3171
+ self .setup = self ._combine_fn_copy .setup
3172
+ self .create_accumulator = self ._combine_fn_copy .create_accumulator
3173
+ self .merge_accumulators = self ._combine_fn_copy .merge_accumulators
3174
+ self .compact = self ._combine_fn_copy .compact
3175
+ self .extract_output = self ._combine_fn_copy .extract_output
3176
+ self .teardown = self ._combine_fn_copy .teardown
3177
+
3178
+ def add_input (self , accumulator , element ):
3165
3179
is_accumulator , value = element
3166
3180
if is_accumulator :
3167
- return combine_fn .merge_accumulators ([accumulator , value ])
3181
+ return self . _combine_fn_copy .merge_accumulators ([accumulator , value ])
3168
3182
else :
3169
- return combine_fn .add_input (accumulator , value )
3170
-
3171
- setup = combine_fn .setup
3172
- create_accumulator = combine_fn .create_accumulator
3173
- merge_accumulators = combine_fn .merge_accumulators
3174
- compact = combine_fn .compact
3175
- extract_output = combine_fn .extract_output
3176
- teardown = combine_fn .teardown
3183
+ return self ._combine_fn_copy .add_input (accumulator , value )
3177
3184
3178
3185
def StripNonce (nonce_key_value ):
3179
3186
(_ , key ), value = nonce_key_value
0 commit comments