[ 
https://issues.apache.org/jira/browse/BEAM-9390?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=17061239#comment-17061239
 ] 

Luke Cwik edited comment on BEAM-9390 at 3/17/20, 10:56 PM:
------------------------------------------------------------

The issue is that the BigQuerySource only works with initial splitting since 
initial splitting kicks off the import job.

 

This can be fixed by:

1) Restore the JavaReadViaImpulse expansion for Dataflow runner only when using 
beam_fn_api

2) Update the JRH to support doing the expansion for SplittableDoFns itself

3) Migrate BigQuerySource to a SplittableDoFn

4) Implement "concat" source over the AvroIO files.

5) Migrate to use the UW and then Dataflow will perform the SDF expansion.

 

All but #3 is throw away work but #3 would make the source incompatible with 
non-portable runners. #5 is likely the easiest to do once it becomes available.


was (Author: lcwik):
The issue is that the BigQuerySource only works with initial splitting since 
initial splitting kicks off the import job.

 

This can be fixed by:

1) Restore the JavaReadViaImpulse expansion for Dataflow runner only when using 
beam_fn_api

2) Update the JRH to support doing the expansion for SplittableDoFns itself

3) Migrate BigQuerySource to a SplittableDoFn

4) Implement "concat" source over the AvroIO files.

 

All but #3 is throw away work but #3 would make the source incompatible with 
non-portable runners. #1 is likely the easiest to do.

> [PostCommit_Java_PortabilityApi] [BigQuery related ITs] 
> UnsupportedOperationException: BigQuery source must be split before being read
> --------------------------------------------------------------------------------------------------------------------------------------
>
>                 Key: BEAM-9390
>                 URL: https://issues.apache.org/jira/browse/BEAM-9390
>             Project: Beam
>          Issue Type: Bug
>          Components: test-failures
>            Reporter: Yueyang Qiu
>            Assignee: Luke Cwik
>            Priority: Major
>              Labels: currently-failing
>
> Failed tests:
> org.apache.beam.examples.cookbook.BigQueryTornadoesIT.testE2EBigQueryTornadoesWithExport
>  
> org.apache.beam.examples.cookbook.BigQueryTornadoesIT.testE2eBigQueryTornadoesWithStorageApi
>  
> org.apache.beam.sdk.io.gcp.bigquery.BigQueryTimePartitioningClusteringIT.testE2EBigQueryClusteringTableFunction
>  
> org.apache.beam.sdk.io.gcp.bigquery.BigQueryTimePartitioningClusteringIT.testE2EBigQueryClusteringDynamicDestinations
>  
> org.apache.beam.sdk.io.gcp.bigquery.BigQueryTimePartitioningClusteringIT.testE2EBigQueryTimePartitioning
>  
> org.apache.beam.sdk.io.gcp.bigquery.BigQueryTimePartitioningClusteringIT.testE2EBigQueryClustering
>  
> org.apache.beam.sdk.io.gcp.bigquery.BigQueryToTableIT.testNewTypesQueryWithReshuffle
>  
> org.apache.beam.sdk.io.gcp.bigquery.BigQueryToTableIT.testLegacyQueryWithoutReshuffle
>  
> org.apache.beam.sdk.io.gcp.bigquery.BigQueryToTableIT.testNewTypesQueryWithoutReshuffle
>  
> org.apache.beam.sdk.io.gcp.bigquery.BigQueryToTableIT.testStandardQueryWithoutCustom
> ([https://builds.apache.org/job/beam_PostCommit_Java_PortabilityApi/4226/#showFailuresLink)]
>  
> Example failures:
> java.lang.RuntimeException: java.util.concurrent.ExecutionException: 
> java.lang.RuntimeException: Error received from SDK harness for instruction 
> -596: java.lang.UnsupportedOperationException: BigQuery source must be split 
> before being read at 
> org.apache.beam.sdk.io.gcp.bigquery.BigQuerySourceBase.createReader(BigQuerySourceBase.java:173)
>  at 
> org.apache.beam.fn.harness.BoundedSourceRunner.runReadLoop(BoundedSourceRunner.java:159)
>  at 
> org.apache.beam.fn.harness.BoundedSourceRunner.start(BoundedSourceRunner.java:146)
> ...



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

Reply via email to