You are viewing a plain text version of this content. The canonical link for it is here.
Posted to issues@beam.apache.org by "Udi Meiri (Jira)" <ji...@apache.org> on 2020/01/15 21:39:00 UTC

[jira] [Created] (BEAM-9127) postcommit: suites:portable:py2:crossLanguagePortableWordCount failing

Udi Meiri created BEAM-9127:
-------------------------------

             Summary: postcommit: suites:portable:py2:crossLanguagePortableWordCount failing
                 Key: BEAM-9127
                 URL: https://issues.apache.org/jira/browse/BEAM-9127
             Project: Beam
          Issue Type: Bug
          Components: test-failures
            Reporter: Udi Meiri
            Assignee: Robert Bradshaw


Log: https://builds.apache.org/job/beam_PostCommit_Python2/1445/timestamps/?time=HH:mm:ss&timeZone=GMT-8&appendLog&locale=en_US
{code}
11:32:07  RuntimeError: Pipeline BeamApp-root-0114193141-d68e793b_21a90e10-d9a1-41e8-bdba-154dddc532f3 failed in state FAILED: java.lang.RuntimeException: Error received from SDK harness for instruction 16: Traceback (most recent call last):
11:32:07    File "/usr/local/lib/python2.7/site-packages/apache_beam/runners/worker/sdk_worker.py", line 165, in _execute
11:32:07      response = task()
11:32:07    File "/usr/local/lib/python2.7/site-packages/apache_beam/runners/worker/sdk_worker.py", line 221, in <lambda>
11:32:07      lambda: self.create_worker().do_instruction(request), request)
11:32:07    File "/usr/local/lib/python2.7/site-packages/apache_beam/runners/worker/sdk_worker.py", line 350, in do_instruction
11:32:07      request.instruction_id)
11:32:07    File "/usr/local/lib/python2.7/site-packages/apache_beam/runners/worker/sdk_worker.py", line 384, in process_bundle
11:32:07      bundle_processor.process_bundle(instruction_id))
11:32:07    File "/usr/local/lib/python2.7/site-packages/apache_beam/runners/worker/bundle_processor.py", line 811, in process_bundle
11:32:07      data.transform_id].process_encoded(data.data)
11:32:07    File "/usr/local/lib/python2.7/site-packages/apache_beam/runners/worker/bundle_processor.py", line 206, in process_encoded
11:32:07      self.output(decoded_value)
11:32:07    File "apache_beam/runners/worker/operations.py", line 302, in apache_beam.runners.worker.operations.Operation.output
11:32:07      def output(self, windowed_value, output_index=0):
11:32:07    File "apache_beam/runners/worker/operations.py", line 304, in apache_beam.runners.worker.operations.Operation.output
11:32:07      cython.cast(Receiver, self.receivers[output_index]).receive(windowed_value)
11:32:07    File "apache_beam/runners/worker/operations.py", line 178, in apache_beam.runners.worker.operations.SingletonConsumerSet.receive
11:32:07      self.consumer.process(windowed_value)
11:32:07    File "apache_beam/runners/worker/operations.py", line 751, in apache_beam.runners.worker.operations.SdfProcessSizedElements.process
11:32:07      with self.scoped_process_state:
11:32:07    File "apache_beam/runners/worker/operations.py", line 759, in apache_beam.runners.worker.operations.SdfProcessSizedElements.process
11:32:07      delayed_application = self.dofn_runner.process_with_sized_restriction(o)
11:32:07    File "apache_beam/runners/common.py", line 888, in apache_beam.runners.common.DoFnRunner.process_with_sized_restriction
11:32:07      return self.do_fn_invoker.invoke_process(
11:32:07    File "apache_beam/runners/common.py", line 657, in apache_beam.runners.common.PerWindowInvoker.invoke_process
11:32:07      return self._invoke_process_per_window(
11:32:07    File "apache_beam/runners/common.py", line 744, in apache_beam.runners.common.PerWindowInvoker._invoke_process_per_window
11:32:07      output_processor.process_outputs(
11:32:07    File "apache_beam/runners/common.py", line 984, in apache_beam.runners.common._OutputProcessor.process_outputs
11:32:07      def process_outputs(self, windowed_input_element, results):
11:32:07    File "apache_beam/runners/common.py", line 1024, in apache_beam.runners.common._OutputProcessor.process_outputs
11:32:07      self.main_receivers.receive(windowed_value)
11:32:07    File "apache_beam/runners/worker/operations.py", line 178, in apache_beam.runners.worker.operations.SingletonConsumerSet.receive
11:32:07      self.consumer.process(windowed_value)
11:32:07    File "apache_beam/runners/worker/operations.py", line 657, in apache_beam.runners.worker.operations.DoOperation.process
11:32:07      with self.scoped_process_state:
11:32:07    File "apache_beam/runners/worker/operations.py", line 658, in apache_beam.runners.worker.operations.DoOperation.process
11:32:07      delayed_application = self.dofn_receiver.receive(o)
11:32:07    File "apache_beam/runners/common.py", line 876, in apache_beam.runners.common.DoFnRunner.receive
11:32:07      self.process(windowed_value)
11:32:07    File "apache_beam/runners/common.py", line 883, in apache_beam.runners.common.DoFnRunner.process
11:32:07      self._reraise_augmented(exn)
11:32:07    File "apache_beam/runners/common.py", line 952, in apache_beam.runners.common.DoFnRunner._reraise_augmented
11:32:07      raise_with_traceback(new_exn)
11:32:07    File "apache_beam/runners/common.py", line 881, in apache_beam.runners.common.DoFnRunner.process
11:32:07      return self.do_fn_invoker.invoke_process(windowed_value)
11:32:07    File "apache_beam/runners/common.py", line 496, in apache_beam.runners.common.SimpleInvoker.invoke_process
11:32:07      output_processor.process_outputs(
11:32:07    File "apache_beam/runners/common.py", line 1024, in apache_beam.runners.common._OutputProcessor.process_outputs
11:32:07      self.main_receivers.receive(windowed_value)
11:32:07    File "apache_beam/runners/worker/operations.py", line 177, in apache_beam.runners.worker.operations.SingletonConsumerSet.receive
11:32:07      self.update_counters_start(windowed_value)
11:32:07    File "apache_beam/runners/worker/operations.py", line 155, in apache_beam.runners.worker.operations.ConsumerSet.update_counters_start
11:32:07      self.opcounter.update_from(windowed_value)
11:32:07    File "apache_beam/runners/worker/opcounters.py", line 214, in apache_beam.runners.worker.opcounters.OperationCounters.update_from
11:32:07      self.do_sample(windowed_value)
11:32:07    File "apache_beam/runners/worker/opcounters.py", line 233, in apache_beam.runners.worker.opcounters.OperationCounters.do_sample
11:32:07      self.coder_impl.get_estimated_size_and_observables(windowed_value))
11:32:07    File "apache_beam/coders/coder_impl.py", line 1161, in apache_beam.coders.coder_impl.WindowedValueCoderImpl.get_estimated_size_and_observables
11:32:07      def get_estimated_size_and_observables(self, value, nested=False):
11:32:07    File "apache_beam/coders/coder_impl.py", line 1171, in apache_beam.coders.coder_impl.WindowedValueCoderImpl.get_estimated_size_and_observables
11:32:07      self._value_coder.get_estimated_size_and_observables(
11:32:07    File "apache_beam/coders/coder_impl.py", line 187, in apache_beam.coders.coder_impl.CoderImpl.get_estimated_size_and_observables
11:32:07      return self.estimate_size(value, nested), []
11:32:07    File "apache_beam/coders/coder_impl.py", line 161, in apache_beam.coders.coder_impl.CoderImpl.estimate_size
11:32:07      self.encode_to_stream(value, out, nested)
11:32:07    File "apache_beam/coders/coder_impl.py", line 489, in apache_beam.coders.coder_impl.BytesCoderImpl.encode_to_stream
11:32:07      out.write(value, nested)
11:32:07  TypeError: Expected bytes, got unicode [while running 'split']
{code}

Culprit seems to be: https://github.com/apache/beam/pull/10558




--
This message was sent by Atlassian Jira
(v8.3.4#803005)