[ 
https://issues.apache.org/jira/browse/BEAM-12725?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel
 ]

Christopher Cornwell updated BEAM-12725:
----------------------------------------
    Description: 
While running a job on Dataflow that writes to BigQuery using the `FILE_LOADS` 
write method I notice the following error in the `MultiPartitionsWriteTables` 
step:

 
{code:java}
{"errorResult":\{"message":"Schema update options should only be specified with 
WRITE_APPEND disposition, or with WRITE_TRUNCATE disposition on a table 
partition.","reason":"invalid"},"errors":[\{"message":"Schema update options 
should only be specified with WRITE_APPEND disposition, or with WRITE_TRUNCATE 
disposition on a table partition.","reason":"invalid"}],"state":"DONE"}
{code}
 


 Here's the write configuration that I'm using:
{code:java}
BigQueryIO
 .write()
 .to(...)
 .withSchema(...)
 .withFormatFunction(...)
 .withCreateDisposition(CREATE_IF_NEEDED)
 .withWriteDisposition(WRITE_APPEND)
.withSchemaUpdateOptions(Collections.singleton(SchemaUpdateOption.ALLOW_FIELD_ADDITION))
 .withTimePartitioning(new 
TimePartitioning().setType("DAY").setRequirePartitionFilter(false).setField("ts"))
 .withMethod(Method.FILE_LOADS)
 .withTriggeringFrequency(Minutes.minutes(5).toStandardDuration)
 .withAutoSharding()
 .optimizedWrites(){code}
 

I believe it is due to the fact that the schema update options are being passed 
to the `WriteTables` constructor for the temp tables 
[here|[https://github.com/apache/beam/blob/ecedd3e654352f1b51ab2caae0fd4665403bd0eb/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/bigquery/BatchLoads.java#L610]].
 It might be okay to just pass `null` there instead since I don't think we need 
the schema update options if we're always generating those temp tables from 
scratch, but I'm not sure if that will have other consequences.

This is preventing any of the load jobs from completing, causing none of the 
data to ever make it to the BigQuery table.

  was:
While running a job on Dataflow that writes to BigQuery using the `FILE_LOADS` 
write method I notice the following error in the `MultiPartitionsWriteTables` 
step:
```
{"errorResult":\{"message":"Schema update options should only be specified with 
WRITE_APPEND disposition, or with WRITE_TRUNCATE disposition on a table 
partition.","reason":"invalid"},"errors":[\{"message":"Schema update options 
should only be specified with WRITE_APPEND disposition, or with WRITE_TRUNCATE 
disposition on a table partition.","reason":"invalid"}],"state":"DONE"}

```
Here's the write configuration that I'm using:
```

BigQueryIO
 .write()
 .to(...)
 .withSchema(...)
 .withFormatFunction(...)
 .withCreateDisposition(CREATE_IF_NEEDED)
 .withWriteDisposition(WRITE_APPEND)
.withSchemaUpdateOptions(Collections.singleton(SchemaUpdateOption.ALLOW_FIELD_ADDITION))
 .withTimePartitioning(new 
TimePartitioning().setType("DAY").setRequirePartitionFilter(false).setField("ts"))
 .withMethod(Method.FILE_LOADS)
 .withTriggeringFrequency(Minutes.minutes(5).toStandardDuration)
 .withAutoSharding()
 .optimizedWrites()

```

I believe it is due to the fact that the schema update options are being passed 
to the `WriteTables` constructor for the temp tables 
[here](https://github.com/apache/beam/blob/ecedd3e654352f1b51ab2caae0fd4665403bd0eb/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/bigquery/BatchLoads.java#L610).
 It might be okay to just pass `null` there instead since I don't think we need 
the schema update options if we're always generating those temp tables from 
scratch, but I'm not sure if that will have other consequences.

This is preventing any of the load jobs from completing, causing none of the 
data to ever make it to the BigQuery table.


> BigQuery FILE_LOADS fails with ALLOW_FIELD_ADDITION set
> -------------------------------------------------------
>
>                 Key: BEAM-12725
>                 URL: https://issues.apache.org/jira/browse/BEAM-12725
>             Project: Beam
>          Issue Type: Bug
>          Components: io-java-gcp
>    Affects Versions: 2.29.0
>            Reporter: Christopher Cornwell
>            Priority: P2
>
> While running a job on Dataflow that writes to BigQuery using the 
> `FILE_LOADS` write method I notice the following error in the 
> `MultiPartitionsWriteTables` step:
>  
> {code:java}
> {"errorResult":\{"message":"Schema update options should only be specified 
> with WRITE_APPEND disposition, or with WRITE_TRUNCATE disposition on a table 
> partition.","reason":"invalid"},"errors":[\{"message":"Schema update options 
> should only be specified with WRITE_APPEND disposition, or with 
> WRITE_TRUNCATE disposition on a table 
> partition.","reason":"invalid"}],"state":"DONE"}
> {code}
>  
>  Here's the write configuration that I'm using:
> {code:java}
> BigQueryIO
>  .write()
>  .to(...)
>  .withSchema(...)
>  .withFormatFunction(...)
>  .withCreateDisposition(CREATE_IF_NEEDED)
>  .withWriteDisposition(WRITE_APPEND)
> .withSchemaUpdateOptions(Collections.singleton(SchemaUpdateOption.ALLOW_FIELD_ADDITION))
>  .withTimePartitioning(new 
> TimePartitioning().setType("DAY").setRequirePartitionFilter(false).setField("ts"))
>  .withMethod(Method.FILE_LOADS)
>  .withTriggeringFrequency(Minutes.minutes(5).toStandardDuration)
>  .withAutoSharding()
>  .optimizedWrites(){code}
>  
> I believe it is due to the fact that the schema update options are being 
> passed to the `WriteTables` constructor for the temp tables 
> [here|[https://github.com/apache/beam/blob/ecedd3e654352f1b51ab2caae0fd4665403bd0eb/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/bigquery/BatchLoads.java#L610]].
>  It might be okay to just pass `null` there instead since I don't think we 
> need the schema update options if we're always generating those temp tables 
> from scratch, but I'm not sure if that will have other consequences.
> This is preventing any of the load jobs from completing, causing none of the 
> data to ever make it to the BigQuery table.



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

Reply via email to