You signed in with another tab or window. Reload to refresh your session.You signed out in another tab or window. Reload to refresh your session.You switched accounts on another tab or window. Reload to refresh your session.Dismiss alert
I think the recent addition #27599 is causing build failure for python 3.11
apache_beam/io/gcp/bigquery_read_it_test.py::�[1mReadUsingStorageApiTests::test_iobase_source_with_very_selective_filters�[0m - apache_beam.runners.dataflow.dataflow_runner.DataflowRuntimeException: Dataflow pipeline failed. State: FAILED, Error:
Traceback (most recent call last):
File "apache_beam/runners/common.py", line 1423, in apache_beam.runners.common.DoFnRunner.process
File "apache_beam/runners/common.py", line 839, in apache_beam.runners.common.PerWindowInvoker.invoke_process
File "apache_beam/runners/common.py", line 985, in apache_beam.runners.common.PerWindowInvoker._invoke_process_per_window
File "/home/jenkins/jenkins-slave/workspace/beam_PostCommit_Python311/src/sdks/python/apache_beam/io/gcp/bigquery_read_internal.py", line 154, in process
input.pipeline.options)
^^^^^^^^^^^^^^
AttributeError: '_InvalidUnpickledPCollection' object has no attribute 'pipeline'
During handling of the above exception, another exception occurred:
Traceback (most recent call last):
File "/opt/apache/beam-venv/beam-venv-worker-sdk-0-0/lib/python3.11/site-packages/apache_beam/runners/worker/sdk_worker.py", line 297, in _execute
response = task()
^^^^^^
File "/opt/apache/beam-venv/beam-venv-worker-sdk-0-0/lib/python3.11/site-packages/apache_beam/runners/worker/sdk_worker.py", line 372, in <lambda>
lambda: self.create_worker().do_instruction(request), request)
^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^
File "/opt/apache/beam-venv/beam-venv-worker-sdk-0-0/lib/python3.11/site-packages/apache_beam/runners/worker/sdk_worker.py", line 625, in do_instruction
return getattr(self, request_type)(
^^^^^^^^^^^^^^^^^^^^^^^^^^^^
File "/opt/apache/beam-venv/beam-venv-worker-sdk-0-0/lib/python3.11/site-packages/apache_beam/runners/worker/sdk_worker.py", line 663, in process_bundle
bundle_processor.process_bundle(instruction_id))
^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^
File "/opt/apache/beam-venv/beam-venv-worker-sdk-0-0/lib/python3.11/site-packages/apache_beam/runners/worker/bundle_processor.py", line 1040, in process_bundle
input_op_by_transform_id[element.transform_id].process_encoded(
File "/opt/apache/beam-venv/beam-venv-worker-sdk-0-0/lib/python3.11/site-packages/apache_beam/runners/worker/bundle_processor.py", line 232, in process_encoded
self.output(decoded_value)
File "apache_beam/runners/worker/operations.py", line 568, in apache_beam.runners.worker.operations.Operation.output
File "apache_beam/runners/worker/operations.py", line 570, in apache_beam.runners.worker.operations.Operation.output
File "apache_beam/runners/worker/operations.py", line 261, in apache_beam.runners.worker.operations.SingletonElementConsumerSet.receive
File "apache_beam/runners/worker/operations.py", line 264, in apache_beam.runners.worker.operations.SingletonElementConsumerSet.receive
File "apache_beam/runners/worker/operations.py", line 951, in apache_beam.runners.worker.operations.DoOperation.process
File "apache_beam/runners/worker/operations.py", line 952, in apache_beam.runners.worker.operations.DoOperation.process
File "apache_beam/runners/common.py", line 1425, in apache_beam.runners.common.DoFnRunner.process
File "apache_beam/runners/common.py", line 1513, in apache_beam.runners.common.DoFnRunner._reraise_augmented
File "apache_beam/runners/common.py", line 1423, in apache_beam.runners.common.DoFnRunner.process
File "apache_beam/runners/common.py", line 625, in apache_beam.runners.common.SimpleInvoker.invoke_process
File "apache_beam/runners/common.py", line 1607, in apache_beam.runners.common._OutputHandler.handle_process_outputs
File "apache_beam/runners/common.py", line 1720, in apache_beam.runners.common._OutputHandler._write_value_to_tag
File "apache_beam/runners/worker/operations.py", line 264, in apache_beam.runners.worker.operations.SingletonElementConsumerSet.receive
File "apache_beam/runners/worker/operations.py", line 951, in apache_beam.runners.worker.operations.DoOperation.process
File "apache_beam/runners/worker/operations.py", line 952, in apache_beam.runners.worker.operations.DoOperation.process
File "apache_beam/runners/common.py", line 1425, in apache_beam.runners.common.DoFnRunner.process
File "apache_beam/runners/common.py", line 1513, in apache_beam.runners.common.DoFnRunner._reraise_augmented
File "apache_beam/runners/common.py", line 1423, in apache_beam.runners.common.DoFnRunner.process
File "apache_beam/runners/common.py", line 625, in apache_beam.runners.common.SimpleInvoker.invoke_process
File "apache_beam/runners/common.py", line 1607, in apache_beam.runners.common._OutputHandler.handle_process_outputs
File "apache_beam/runners/common.py", line 1720, in apache_beam.runners.common._OutputHandler._write_value_to_tag
File "apache_beam/runners/worker/operations.py", line 264, in apache_beam.runners.worker.operations.SingletonElementConsumerSet.receive
File "apache_beam/runners/worker/operations.py", line 951, in apache_beam.runners.worker.operations.DoOperation.process
File "apache_beam/runners/worker/operations.py", line 952, in apache_beam.runners.worker.operations.DoOperation.process
File "apache_beam/runners/common.py", line 1425, in apache_beam.runners.common.DoFnRunner.process
File "apache_beam/runners/common.py", line 1533, in apache_beam.runners.common.DoFnRunner._reraise_augmented
File "apache_beam/runners/common.py", line 1423, in apache_beam.runners.common.DoFnRunner.process
File "apache_beam/runners/common.py", line 839, in apache_beam.runners.common.PerWindowInvoker.invoke_process
File "apache_beam/runners/common.py", line 985, in apache_beam.runners.common.PerWindowInvoker._invoke_process_per_window
File "/home/jenkins/jenkins-slave/workspace/beam_PostCommit_Python311/src/sdks/python/apache_beam/io/gcp/bigquery_read_internal.py", line 154, in process
input.pipeline.options)
^^^^^^^^^^^^^^
AttributeError: '_InvalidUnpickledPCollection' object has no attribute 'pipeline' [while running 'Read with BigQuery Storage API/_PassThroughThenCleanupTempDatasets/ParDo(CleanUpProjects)-ptransform-27']
�[31m===== �[31m�[1m12 failed�[0m, �[32m70 passed�[0m, �[33m43 skipped�[0m, �[33m670 warnings�[0m�[31m in 8431.50s (2:20:31)�[0m�[31m =====�[0m
Issue Failure
Failure: Test is continually failing
Issue Priority
Priority: 1 (unhealthy code / failing or flaky postcommit so we cannot be sure the product is healthy)
Issue Components
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
Component: Google Cloud Dataflow Runner
The text was updated successfully, but these errors were encountered:
riteshghorse
changed the title
[Failing Test]: Python 3.11 PostCommit is red with failing bigquery test
[Failing Test]: Python PostCommits are red with failing bigquery test
Jul 25, 2023
What happened?
I think the recent addition #27599 is causing build failure for python 3.11
Issue Failure
Failure: Test is continually failing
Issue Priority
Priority: 1 (unhealthy code / failing or flaky postcommit so we cannot be sure the product is healthy)
Issue Components
The text was updated successfully, but these errors were encountered: