Details
-
Bug
-
Status: Resolved
-
P2
-
Resolution: Fixed
-
None
Description
First failure I can find is https://ci-beam.apache.org/job/beam_PreCommit_Python_Cron/5214/
self = <apache_beam.transforms.combinefn_lifecycle_test.LocalCombineFnLifecycleTest_0 testMethod=test_combine> def test_combine(self): > run_combine(TestPipeline(runner=self.runner())) apache_beam/transforms/combinefn_lifecycle_test.py:88: _ _ _ _ _ _ _ _ _ _ _ _ _ _ _ _ _ _ _ _ _ _ _ _ _ _ _ _ _ _ _ _ _ _ _ _ _ _ _ _ apache_beam/transforms/combinefn_lifecycle_pipeline.py:124: in run_combine assert_that(pcoll, equal_to([(expected_result, expected_result)])) apache_beam/pipeline.py:596: in __exit__ self.result = self.run() apache_beam/testing/test_pipeline.py:116: in run state = result.wait_until_finish() apache_beam/runners/direct/direct_runner.py:588: in wait_until_finish self._executor.await_completion() apache_beam/runners/direct/executor.py:432: in await_completion self._executor.await_completion() apache_beam/runners/direct/executor.py:480: in await_completion raise update.exception apache_beam/runners/direct/executor.py:375: in call finish_state) apache_beam/runners/direct/executor.py:404: in attempt_call evaluator.start_bundle() apache_beam/runners/direct/transform_evaluator.py:865: in start_bundle self.runner.setup() apache_beam/runners/common.py:1245: in setup self._invoke_lifecycle_method(self.do_fn_invoker.invoke_setup) apache_beam/runners/common.py:1241: in _invoke_lifecycle_method self._reraise_augmented(exn) apache_beam/runners/common.py:1281: in _reraise_augmented raise new_exn.with_traceback(tb) apache_beam/runners/common.py:1239: in _invoke_lifecycle_method lifecycle_method() apache_beam/runners/common.py:465: in invoke_setup self.signature.setup_lifecycle_method.method_value() apache_beam/runners/direct/helper_transforms.py:101: in setup self._combine_fn.setup() apache_beam/typehints/typecheck.py:205: in setup self._combinefn.setup(*args, **kwargs) apache_beam/transforms/combiners.py:824: in setup self.fn.setup(*self.args, **self.kwargs) apache_beam/transforms/combiners.py:646: in setup c.setup(*args, **kwargs) _ _ _ _ _ _ _ _ _ _ _ _ _ _ _ _ _ _ _ _ _ _ _ _ _ _ _ _ _ _ _ _ _ _ _ _ _ _ _ _ self = <apache_beam.transforms.combinefn_lifecycle_pipeline.CallSequenceEnforcingCombineFn object at 0x7fd4428e9358> args = (None,), kwargs = {} def setup(self, *args, **kwargs): > assert not self._setup_called, 'setup should not be called twice' E AssertionError: setup should not be called twice [while running 'Do/CombinePerKey/CombinePerKey(PreCombineFn)/ParDo(FinishCombine)']
Attachments
Issue Links
- relates to
-
BEAM-13769 beam_PreCommit_Python_Cron failing on test_create_uses_coder_for_pickling
- Open