Skip to content
New issue

Have a question about this project? Sign up for a free GitHub account to open an issue and contact its maintainers and the community.

By clicking “Sign up for GitHub”, you agree to our terms of service and privacy statement. We’ll occasionally send you account related emails.

Already on GitHub? Sign in to your account

Try deepcopy combine_fn and fallback to pickling if TypeError. #32645

Draft
wants to merge 1 commit into
base: master
Choose a base branch
from
Draft
Show file tree
Hide file tree
Changes from all commits
Commits
File filter

Filter by extension

Filter by extension

Conversations
Failed to load comments.
Loading
Jump to
Jump to file
Failed to load files.
Loading
Diff view
Diff view
11 changes: 7 additions & 4 deletions sdks/python/apache_beam/transforms/combinefn_lifecycle_test.py
Original file line number Diff line number Diff line change
Expand Up @@ -53,15 +53,18 @@ def test_combining_value_state(self):


@parameterized_class([
{'runner': direct_runner.BundleBasedDirectRunner},
{'runner': fn_api_runner.FnApiRunner},
]) # yapf: disable
{'runner': direct_runner.BundleBasedDirectRunner, 'pickler': 'dill'},
{'runner': direct_runner.BundleBasedDirectRunner, 'pickler': 'cloudpickle'},
{'runner': fn_api_runner.FnApiRunner, 'pickler': 'dill'},
{'runner': fn_api_runner.FnApiRunner, 'pickler': 'cloudpickle'},
]) # yapf: disable
class LocalCombineFnLifecycleTest(unittest.TestCase):
def tearDown(self):
CallSequenceEnforcingCombineFn.instances.clear()

def test_combine(self):
run_combine(TestPipeline(runner=self.runner()))
test_options = PipelineOptions(flags=[f"--pickle_library={self.pickler}"])
run_combine(TestPipeline(runner=self.runner(), options=test_options))
self._assert_teardown_called()

def test_non_liftable_combine(self):
Expand Down
59 changes: 41 additions & 18 deletions sdks/python/apache_beam/transforms/core.py
Original file line number Diff line number Diff line change
Expand Up @@ -3158,33 +3158,56 @@ def process(self, element):
yield pvalue.TaggedOutput('hot', ((self._nonce % fanout, key), value))

class PreCombineFn(CombineFn):
def __init__(self):
# Deepcopy of the combine_fn to avoid sharing state between lifted
# stages when using cloudpickle.
try:
self._combine_fn_copy = copy.deepcopy(combine_fn)
except TypeError as e:
logging.warning(
'Failed to copy combine function. Ensure python dependencies are'
' properly set up: %s',
e)
self._combine_fn_copy = pickler.loads(pickler.dumps(combine_fn))

self.setup = self._combine_fn_copy.setup
self.create_accumulator = self._combine_fn_copy.create_accumulator
self.add_input = self._combine_fn_copy.add_input
self.merge_accumulators = self._combine_fn_copy.merge_accumulators
self.compact = self._combine_fn_copy.compact
self.teardown = self._combine_fn_copy.teardown

@staticmethod
def extract_output(accumulator):
# Boolean indicates this is an accumulator.
return (True, accumulator)

setup = combine_fn.setup
create_accumulator = combine_fn.create_accumulator
add_input = combine_fn.add_input
merge_accumulators = combine_fn.merge_accumulators
compact = combine_fn.compact
teardown = combine_fn.teardown

class PostCombineFn(CombineFn):
@staticmethod
def add_input(accumulator, element):
def __init__(self):
# Deepcopy of the combine_fn to avoid sharing state between lifted
# stages when using cloudpickle.
try:
self._combine_fn_copy = copy.deepcopy(combine_fn)
except TypeError as e:
logging.warning(
'Failed to copy combine function. Ensure python dependencies are'
' properly set up: %s',
e)
self._combine_fn_copy = pickler.loads(pickler.dumps(combine_fn))

self.setup = self._combine_fn_copy.setup
self.create_accumulator = self._combine_fn_copy.create_accumulator
self.merge_accumulators = self._combine_fn_copy.merge_accumulators
self.compact = self._combine_fn_copy.compact
self.extract_output = self._combine_fn_copy.extract_output
self.teardown = self._combine_fn_copy.teardown

def add_input(self, accumulator, element):
is_accumulator, value = element
if is_accumulator:
return combine_fn.merge_accumulators([accumulator, value])
return self._combine_fn_copy.merge_accumulators([accumulator, value])
else:
return combine_fn.add_input(accumulator, value)

setup = combine_fn.setup
create_accumulator = combine_fn.create_accumulator
merge_accumulators = combine_fn.merge_accumulators
compact = combine_fn.compact
extract_output = combine_fn.extract_output
teardown = combine_fn.teardown
return self._combine_fn_copy.add_input(accumulator, value)

def StripNonce(nonce_key_value):
(_, key), value = nonce_key_value
Expand Down
Loading