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 use Apache-beam 2.53 version, python sdk, and dataflow runner. After upgrading to 2.53 my pipeline fails on steps when it reads data from BigQuery with this error:
Error message from worker: Traceback (most recent call last):
File "apache_beam/runners/common.py", line 1435, in apache_beam.runners.common.DoFnRunner.process
File "apache_beam/runners/common.py", line 851, in apache_beam.runners.common.PerWindowInvoker.invoke_process
File "apache_beam/runners/common.py", line 997, in apache_beam.runners.common.PerWindowInvoker._invoke_process_per_window
File "/opt/conda/envs/python39/lib/python3.9/site-packages/apache_beam/io/gcp/bigquery_read_internal.py", line 121, in process
FileSystems.delete(list(gcs_locations))
File "/usr/local/lib/python3.9/site-packages/apache_beam/io/filesystems.py", line 343, in delete
return filesystem.delete(paths)
File "/usr/local/lib/python3.9/site-packages/apache_beam/io/gcp/gcsfilesystem.py", line 364, in delete
statuses = self._gcsIO().delete_batch(
File "/usr/local/lib/python3.9/site-packages/apache_beam/io/gcp/gcsio.py", line 220, in delete_batch
current_batch = self.client.batch(raise_exception=False)
TypeError: batch() got an unexpected keyword argument 'raise_exception'
During handling of the above exception, another exception occurred:
Traceback (most recent call last):
File "/usr/local/lib/python3.9/site-packages/apache_beam/runners/worker/sdk_worker.py", line 300, in _execute
response = task()
File "/usr/local/lib/python3.9/site-packages/apache_beam/runners/worker/sdk_worker.py", line 375, 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 639, in do_instruction
return getattr(self, request_type)(
File "/usr/local/lib/python3.9/site-packages/apache_beam/runners/worker/sdk_worker.py", line 677, 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 1113, 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 237, in process_encoded
self.output(decoded_value)
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 572, in apache_beam.runners.worker.operations.Operation.output
File "apache_beam/runners/worker/operations.py", line 263, in apache_beam.runners.worker.operations.SingletonElementConsumerSet.receive
File "apache_beam/runners/worker/operations.py", line 266, in apache_beam.runners.worker.operations.SingletonElementConsumerSet.receive
File "apache_beam/runners/worker/operations.py", line 953, in apache_beam.runners.worker.operations.DoOperation.process
File "apache_beam/runners/worker/operations.py", line 954, in apache_beam.runners.worker.operations.DoOperation.process
File "apache_beam/runners/common.py", line 1437, in apache_beam.runners.common.DoFnRunner.process
File "apache_beam/runners/common.py", line 1526, in apache_beam.runners.common.DoFnRunner._reraise_augmented
File "apache_beam/runners/common.py", line 1435, in apache_beam.runners.common.DoFnRunner.process
File "apache_beam/runners/common.py", line 636, in apache_beam.runners.common.SimpleInvoker.invoke_process
File "apache_beam/runners/common.py", line 1621, in apache_beam.runners.common._OutputHandler.handle_process_outputs
File "apache_beam/runners/common.py", line 1734, in apache_beam.runners.common._OutputHandler._write_value_to_tag
File "apache_beam/runners/worker/operations.py", line 266, in apache_beam.runners.worker.operations.SingletonElementConsumerSet.receive
File "apache_beam/runners/worker/operations.py", line 953, in apache_beam.runners.worker.operations.DoOperation.process
File "apache_beam/runners/worker/operations.py", line 954, in apache_beam.runners.worker.operations.DoOperation.process
File "apache_beam/runners/common.py", line 1437, in apache_beam.runners.common.DoFnRunner.process
File "apache_beam/runners/common.py", line 1526, in apache_beam.runners.common.DoFnRunner._reraise_augmented
File "apache_beam/runners/common.py", line 1435, in apache_beam.runners.common.DoFnRunner.process
File "apache_beam/runners/common.py", line 636, in apache_beam.runners.common.SimpleInvoker.invoke_process
File "apache_beam/runners/common.py", line 1621, in apache_beam.runners.common._OutputHandler.handle_process_outputs
File "apache_beam/runners/common.py", line 1734, in apache_beam.runners.common._OutputHandler._write_value_to_tag
File "apache_beam/runners/worker/operations.py", line 266, in apache_beam.runners.worker.operations.SingletonElementConsumerSet.receive
File "apache_beam/runners/worker/operations.py", line 953, in apache_beam.runners.worker.operations.DoOperation.process
File "apache_beam/runners/worker/operations.py", line 954, in apache_beam.runners.worker.operations.DoOperation.process
File "apache_beam/runners/common.py", line 1437, in apache_beam.runners.common.DoFnRunner.process
File "apache_beam/runners/common.py", line 1547, in apache_beam.runners.common.DoFnRunner._reraise_augmented
File "apache_beam/runners/common.py", line 1435, in apache_beam.runners.common.DoFnRunner.process
File "apache_beam/runners/common.py", line 851, in apache_beam.runners.common.PerWindowInvoker.invoke_process
File "apache_beam/runners/common.py", line 997, in apache_beam.runners.common.PerWindowInvoker._invoke_process_per_window
File "/opt/conda/envs/python39/lib/python3.9/site-packages/apache_beam/io/gcp/bigquery_read_internal.py", line 121, in process
FileSystems.delete(list(gcs_locations))
File "/usr/local/lib/python3.9/site-packages/apache_beam/io/filesystems.py", line 343, in delete
return filesystem.delete(paths)
File "/usr/local/lib/python3.9/site-packages/apache_beam/io/gcp/gcsfilesystem.py", line 364, in delete
statuses = self._gcsIO().delete_batch(
File "/usr/local/lib/python3.9/site-packages/apache_beam/io/gcp/gcsio.py", line 220, in delete_batch
current_batch = self.client.batch(raise_exception=False)
TypeError: batch() got an unexpected keyword argument 'raise_exception' [while running 'read_data_from_bq/_PassThroughThenCleanup/ParDo(RemoveExtractedFiles)-ptransform-27']
Code that reads data is standard, like
p
| 'read_data_from_bq' >> beam.io.ReadFromBigQuery(query='some query',
use_standard_sql=True)
Issue Priority
Priority: 2 (default / most bugs should be filed as P2)
Issue Components
Component: Python SDK
Component: Java SDK
Component: Go SDK
Component: Typescript SDK
Component: IO connector
Component: Beam YAML
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:
What happened?
I use Apache-beam 2.53 version, python sdk, and dataflow runner. After upgrading to 2.53 my pipeline fails on steps when it reads data from BigQuery with this error:
Code that reads data is standard, like
Issue Priority
Priority: 2 (default / most bugs should be filed as P2)
Issue Components
The text was updated successfully, but these errors were encountered: