EdwardCuiPeacock opened a new issue, #26716:
URL: https://github.com/apache/beam/issues/26716

   ### What happened?
   
   I subclassed `beam.transforms.stats.ApproximateQuantilesCombineFn` in the 
following way:
   
   ```python
   class ApproxQuantileCombiner(ApproximateQuantilesCombineFn):
       def __init__(
           self,
           metric_key: tfma.metrics.MetricKey,
           num_quantiles: int = 100,
           max_num_elements: int = 20_000_000,
           epsilon: float = 0.01,
           num_buffers: int = None,
           buffer_size: int = None,
           return_quantile: int = 50,
       ):
           # The create method will calculate the buffers
           dummy_instance = ApproximateQuantilesCombineFn.create(
               num_quantiles, epsilon, max_num_elements
           )
           # print(estimated_buffer)
           super(ApproxQuantileCombiner, self).__init__(
               num_quantiles=num_quantiles,
               buffer_size=buffer_size or dummy_instance._spec.buffer_size,
               num_buffers=num_buffers or dummy_instance._spec.num_buffers,
               input_batched=True,
           )
           self._metric_key = metric_key
           self.num_quantiles = num_quantiles
           self.max_num_elements = max_num_elements
           self.epsilon = epsilon
           self.return_quantile = int(return_quantile)
   
       def __reduce__(self):
           # need to match exactly to the __init__ args
           return (
               self.__class__,
               (
                   self._metric_key,
                   self.num_quantiles,
                   self.max_num_elements,
                   self.epsilon,
                   self._spec.num_buffers,
                   self._spec.buffer_size,
                   self.return_quantile,
               ),
           )
   
       def extract_output(self, accumulator):
           outputs = np.array(super().extract_output(accumulator))
           if self.return_quantile:
               return {self._metric_key: float(outputs[self.return_quantile])}
           else:
               return {self._metric_key: outputs}
   ```
   
   My preprocessor DoFn returns an list of float numbers which I want to 
compute the median on. This works on a local direct runner or in one instance 
of the Dataflow pipeline I was running. However, in another instance of the 
run, the Dataflow pipeline throws the following error:
   
   
   ```log
   Root cause: Traceback (most recent call last): File 
"/usr/local/lib/python3.9/site-packages/apache_beam/runners/worker/sdk_worker.py",
 
   line 287, in _execute response = task() File 
"/usr/local/lib/python3.9/site-packages/apache_beam/runners/worker/sdk_worker.py",
 
   line 360, in <lambda> lambda: self.create_worker().do_instruction(request), 
request) File 
"/usr/local/lib/python3.9/site-packages/apache_beam/runners/worker/sdk_worker.py",
 
   line 596, in do_instruction return getattr(self, request_type)( File 
"/usr/local/lib/python3.9/site-packages/apache_beam/runners/worker/sdk_worker.py",
 
   line 634, in process_bundle bundle_processor.process_bundle(instruction_id)) 
File 
"/usr/local/lib/python3.9/site-packages/apache_beam/runners/worker/bundle_processor.py",
 
   line 1003, in process_bundle 
input_op_by_transform_id[element.transform_id].process_encoded( File 
"/usr/local/lib/python3.9/site-packages/apache_beam/runners/worker/bundle_processor.py",
 
   line 225, in process_encoded decoded_value = 
self.windowed_coder_impl.decode_from_stream( File 
"apache_beam/coders/coder_impl.py", 
   line 1519, in 
apache_beam.coders.coder_impl.ParamWindowedValueCoderImpl.decode_from_stream 
File "apache_beam/coders/coder_impl.py", 
   line 1520, in 
apache_beam.coders.coder_impl.ParamWindowedValueCoderImpl.decode_from_stream 
File "apache_beam/coders/coder_impl.py", 
   line 1009, in 
apache_beam.coders.coder_impl.AbstractComponentCoderImpl.decode_from_stream 
File "apache_beam/coders/coder_impl.py", 
   line 1195, in 
apache_beam.coders.coder_impl.SequenceCoderImpl.decode_from_stream File 
"apache_beam/coders/coder_impl.py", 
   line 1557, in 
apache_beam.coders.coder_impl.LengthPrefixCoderImpl.decode_from_stream File 
"apache_beam/coders/coder_impl.py", 
   line 240, in apache_beam.coders.coder_impl.StreamCoderImpl.decode File 
"apache_beam/coders/coder_impl.py", line 548, in 
apache_beam.coders.coder_impl.FastPrimitivesCoderImpl.decode_from_stream File 
"apache_beam/coders/coder_impl.py", 
   line 583, in 
apache_beam.coders.coder_impl.FastPrimitivesCoderImpl.decode_from_stream File 
"apache_beam/coders/coder_impl.py", 
   line 273, in 
apache_beam.coders.coder_impl.CallbackCoderImpl.decode_from_stream File 
"stringsource", 
   line 17, in apache_beam.transforms.stats._QuantileBuffer.__setstate_cython__ 
   TypeError: Expected tuple, got dict 
   Worker ID: beamapp-root-0516134446-3-05160644-xj8p-harness-zzz3, 
   ```
   
   Also, the job does not fail until ~30 min later, instead of failing 
immediately. I am wondering what could be causing this issue as the error 
message does not seem very obvious to me what's the source of the problem.
   
   
   ### Issue Priority
   
   Priority: 2 (default / most bugs should be filed as P2)
   
   ### Issue Components
   
   - [X] Component: Python SDK
   - [ ] Component: Java SDK
   - [ ] Component: Go SDK
   - [ ] Component: Typescript SDK
   - [ ] Component: IO connector
   - [ ] Component: Beam examples
   - [ ] Component: Beam playground
   - [ ] Component: Beam katas
   - [ ] Component: Website
   - [ ] Component: Spark Runner
   - [ ] Component: Flink Runner
   - [ ] Component: Samza Runner
   - [ ] Component: Twister2 Runner
   - [ ] Component: Hazelcast Jet Runner
   - [X] Component: Google Cloud Dataflow Runner


-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: [email protected]

For queries about this service, please contact Infrastructure at:
[email protected]

Reply via email to