Teng Qiu created BEAM-12514:
-------------------------------

             Summary: BigQueryIO - ReadFromBigQuery can not get table reference 
from RuntimeValueProvider
                 Key: BEAM-12514
                 URL: https://issues.apache.org/jira/browse/BEAM-12514
             Project: Beam
          Issue Type: Bug
          Components: io-py-gcp
    Affects Versions: 2.30.0, 2.29.0, 2.28.0, 2.27.0, 2.26.0
            Reporter: Teng Qiu


After this 
[change|https://github.com/apache/beam/commit/1a08c01ab1cadec16cc13866d5e6c64f6b447b03#diff-0c77ebbb0792b80a76f4ec1f2b89b9d319f5e33f1ee0857ed6fe414f75c48cbbR746],
 the table reference couldn't be parsed correctly when {{ReadFromBigQuery}} is 
taking a {{RuntimeValueProvider}} as value of table.



This commit is included since v2.26.0

 

Code to reproduce:
{code:python}
import apache_beam as beam
from apache_beam.options.pipeline_options import PipelineOptions


class MyOptions(PipelineOptions):
    @classmethod
    def _add_argparse_args(cls, parser):
        parser.add_value_provider_argument(
            "--input",
            type=str,
            default='test-project:test_dataset.test_table',
        )


pipeline_options = PipelineOptions()
with beam.Pipeline(options=pipeline_options) as p:
    my_options = pipeline_options.view_as(MyOptions)
    lines = p | "read" >> beam.io.ReadFromBigQuery(table=my_options.input)
    row_count = lines | "count" >> beam.combiners.Count.Globally()
    row_count | beam.Map(print)

    p.run().wait_until_finish()
{code}
Error message will be:
{noformat}
Traceback (most recent call last):
  ..
  ..
  File 
"....../venv/lib/python3.8/site-packages/apache_beam/io/gcp/bigquery.py", line 
722, in estimate_size
    if not table_ref.projectId:
AttributeError: 'RuntimeValueProvider' object has no attribute 'projectId' 
[while running 
'read/Read/SDFBoundedSourceReader/ParDo(SDFBoundedSourceDoFn)/SplitAndSizeRestriction']
{noformat}



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

Reply via email to