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
We are enabling BigQuery row-level security and as an example using the TRUE filter (but other filters have the same result)
We can write to the table fine with this enabled, however, when reading from the table we get this error below:
/main.py:57: BeamDeprecationWarning: BigQuerySource is deprecated since 2.25.0. Use ReadFromBigQuery instead.
beam.io.BigQuerySource(query=tokens_for_resubmit.format(env=tokens_pipeline_options.env, start_date=start_date),
/venv/lib/python3.8/site-packages/apache_beam/io/gcp/bigquery.py:1881: BeamDeprecationWarning: options is deprecated since First stable release. References to .options will not be supported
temp_location = pcoll.pipeline.options.view_as(
WARNING:apache_beam.options.pipeline_options:Discarding unparseable args: ['/main.py', '-m', 'cleanup_pipeline']
Traceback (most recent call last):
File "apache_beam/runners/common.py", line 1239, in apache_beam.runners.common.DoFnRunner.process
File "apache_beam/runners/common.py", line 587, in apache_beam.runners.common.SimpleInvoker.invoke_process
File "apache_beam/runners/common.py", line 1374, in apache_beam.runners.common._OutputProcessor.process_outputs
File "/venv/lib/python3.8/site-packages/apache_beam/runners/worker/bundle_processor.py", line 1446, in process
for part, size in self.restriction_provider.split_and_size(
File "/venv/lib/python3.8/site-packages/apache_beam/transforms/core.py", line 333, in split_and_size
for part in self.split(element, restriction):
File "/venv/lib/python3.8/site-packages/apache_beam/io/iobase.py", line 1559, in split
estimated_size = restriction.source().estimate_size()
File "/venv/lib/python3.8/site-packages/apache_beam/io/gcp/bigquery.py", line 753, in estimate_size
size = int(job.statistics.totalBytesProcessed)
TypeError: int() argument must be a string, a bytes-like object or a number, not 'NoneType'
During handling of the above exception, another exception occurred:
Traceback (most recent call last):
File "/venv/lib/python3.8/site-packages/apache_beam/pipeline.py", line 580, in __exit__
self.result = self.run()
File "/venv/lib/python3.8/site-packages/apache_beam/pipeline.py", line 559, in run
return self.runner.run_pipeline(self, self._options)
File "/venv/lib/python3.8/site-packages/apache_beam/runners/direct/direct_runner.py", line 133, in run_pipeline
return runner.run_pipeline(pipeline, options)
File "/venv/lib/python3.8/site-packages/apache_beam/runners/portability/fn_api_runner/fn_runner.py", line 182, in run_pipeline
self._latest_run_result = self.run_via_runner_api(
File "/venv/lib/python3.8/site-packages/apache_beam/runners/portability/fn_api_runner/fn_runner.py", line 193, in run_via_runner_api
return self.run_stages(stage_context, stages)
File "/venv/lib/python3.8/site-packages/apache_beam/runners/portability/fn_api_runner/fn_runner.py", line 357, in run_stages
stage_results = self._run_stage(
File "/venv/lib/python3.8/site-packages/apache_beam/runners/portability/fn_api_runner/fn_runner.py", line 548, in _run_stage
last_result, deferred_inputs, fired_timers = self._run_bundle(
File "/venv/lib/python3.8/site-packages/apache_beam/runners/portability/fn_api_runner/fn_runner.py", line 594, in _run_bundle
result, splits = bundle_manager.process_bundle(
File "/venv/lib/python3.8/site-packages/apache_beam/runners/portability/fn_api_runner/fn_runner.py", line 896, in process_bundle
result_future = self._worker_handler.control_conn.push(process_bundle_req)
File "/venv/lib/python3.8/site-packages/apache_beam/runners/portability/fn_api_runner/worker_handlers.py", line 380, in push
response = self.worker.do_instruction(request)
File "/venv/lib/python3.8/site-packages/apache_beam/runners/worker/sdk_worker.py", line 606, in do_instruction
return getattr(self, request_type)(
File "/venv/lib/python3.8/site-packages/apache_beam/runners/worker/sdk_worker.py", line 644, in process_bundle
bundle_processor.process_bundle(instruction_id))
File "/venv/lib/python3.8/site-packages/apache_beam/runners/worker/bundle_processor.py", line 999, in process_bundle
input_op_by_transform_id[element.transform_id].process_encoded(
File "/venv/lib/python3.8/site-packages/apache_beam/runners/worker/bundle_processor.py", line 228, in process_encoded
self.output(decoded_value)
File "apache_beam/runners/worker/operations.py", line 357, in apache_beam.runners.worker.operations.Operation.output
File "apache_beam/runners/worker/operations.py", line 359, in apache_beam.runners.worker.operations.Operation.output
File "apache_beam/runners/worker/operations.py", line 221, in apache_beam.runners.worker.operations.SingletonConsumerSet.receive
File "apache_beam/runners/worker/operations.py", line 718, in apache_beam.runners.worker.operations.DoOperation.process
File "apache_beam/runners/worker/operations.py", line 719, in apache_beam.runners.worker.operations.DoOperation.process
File "apache_beam/runners/common.py", line 1241, in apache_beam.runners.common.DoFnRunner.process
File "apache_beam/runners/common.py", line 1306, in apache_beam.runners.common.DoFnRunner._reraise_augmented
File "apache_beam/runners/common.py", line 1239, in apache_beam.runners.common.DoFnRunner.process
File "apache_beam/runners/common.py", line 587, in apache_beam.runners.common.SimpleInvoker.invoke_process
File "apache_beam/runners/common.py", line 1401, in apache_beam.runners.common._OutputProcessor.process_outputs
File "apache_beam/runners/worker/operations.py", line 221, in apache_beam.runners.worker.operations.SingletonConsumerSet.receive
File "apache_beam/runners/worker/operations.py", line 718, in apache_beam.runners.worker.operations.DoOperation.process
File "apache_beam/runners/worker/operations.py", line 719, in apache_beam.runners.worker.operations.DoOperation.process
File "apache_beam/runners/common.py", line 1241, in apache_beam.runners.common.DoFnRunner.process
File "apache_beam/runners/common.py", line 1306, in apache_beam.runners.common.DoFnRunner._reraise_augmented
File "apache_beam/runners/common.py", line 1239, in apache_beam.runners.common.DoFnRunner.process
File "apache_beam/runners/common.py", line 587, in apache_beam.runners.common.SimpleInvoker.invoke_process
File "apache_beam/runners/common.py", line 1401, in apache_beam.runners.common._OutputProcessor.process_outputs
File "apache_beam/runners/worker/operations.py", line 221, in apache_beam.runners.worker.operations.SingletonConsumerSet.receive
File "apache_beam/runners/worker/operations.py", line 718, in apache_beam.runners.worker.operations.DoOperation.process
File "apache_beam/runners/worker/operations.py", line 719, in apache_beam.runners.worker.operations.DoOperation.process
File "apache_beam/runners/common.py", line 1241, in apache_beam.runners.common.DoFnRunner.process
File "apache_beam/runners/common.py", line 1321, in apache_beam.runners.common.DoFnRunner._reraise_augmented
File "/venv/lib/python3.8/site-packages/future/utils/__init__.py", line 446, in raise_with_traceback
raise exc.with_traceback(traceback)
File "apache_beam/runners/common.py", line 1239, in apache_beam.runners.common.DoFnRunner.process
File "apache_beam/runners/common.py", line 587, in apache_beam.runners.common.SimpleInvoker.invoke_process
File "apache_beam/runners/common.py", line 1374, in apache_beam.runners.common._OutputProcessor.process_outputs
File "/venv/lib/python3.8/site-packages/apache_beam/runners/worker/bundle_processor.py", line 1446, in process
for part, size in self.restriction_provider.split_and_size(
File "/venv/lib/python3.8/site-packages/apache_beam/transforms/core.py", line 333, in split_and_size
for part in self.split(element, restriction):
File "/venv/lib/python3.8/site-packages/apache_beam/io/iobase.py", line 1559, in split
estimated_size = restriction.source().estimate_size()
File "/venv/lib/python3.8/site-packages/apache_beam/io/gcp/bigquery.py", line 753, in estimate_size
size = int(job.statistics.totalBytesProcessed)
TypeError: int() argument must be a string, a bytes-like object or a number, not 'NoneType' [while running 'Read Tokens from BigQuery/ReadFromBigQuery/Read/SDFBoundedSourceReader/ParDo(SDFBoundedSourceDoFn)/SplitAndSizeRestriction']
Process finished with exit code 1
From what I can tell BigQuery is not sending the totalBytesProcessed back from the Job statistics and that code may need to be skipped to support row-level security within BigQuery.
Also to note, running the same query from Google's BigQuery Console on the web browser works just fine while having row-level security enabled.
Issue Priority
Priority: 2
Issue Component
Component: io-py-gcp
The text was updated successfully, but these errors were encountered:
What happened?
This may be a feature request because BigQuery row-level security may not be supported by Apache Beam SDKs.
We are enabling BigQuery row-level security and as an example using the TRUE filter (but other filters have the same result)
We can write to the table fine with this enabled, however, when reading from the table we get this error below:
From what I can tell BigQuery is not sending the totalBytesProcessed back from the Job statistics and that code may need to be skipped to support row-level security within BigQuery.
Also to note, running the same query from Google's BigQuery Console on the web browser works just fine while having row-level security enabled.
Issue Priority
Priority: 2
Issue Component
Component: io-py-gcp
The text was updated successfully, but these errors were encountered: