carlpayne commented on issue #23291:
URL: https://github.com/apache/beam/issues/23291#issuecomment-1311452312

   @ahmedabu98 thanks for replying. You're totally correct that 
`withFailedInsertRetryPolicy()` doesn't apply to STORAGE_WRITE_API and these 
settings are ignored, however we _are_ still seeing cases where 
InvalidArgumentExceptions are unnecessarily retried and take several hours to 
appear via `getFailedStorageApiInserts()`, for example:
   
   1. BigQueryIO receives a TableRow which does not contain a required field in 
BigQuery (in the below example the required column is called `required_field` 
for testing purposes). As an aside, it would be fantastic if BigQueryIO could 
automatically downgrade required columns to nullable, via 
`SchemaUpdateOptions`, but this is a separate feature request I've raised under 
https://github.com/apache/beam/issues/24063 and not directly relevant here.
   2. The insert to BigQuery fails, returning `io.grpc.StatusRuntimeException: 
INVALID_ARGUMENT: The required BigQuery field .required_field is missing in the 
proto message`
   3. Unfortunately the error handling you 
[mentioned](https://github.com/apache/beam/blob/master/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/bigquery/StorageApiWriteUnshardedRecords.java#L448)
 does not apply in this case, because the error is not an instance of 
`Exceptions.AppendSerializtionError` (the code would need to check 
`failedContext.getError() instanceof InvalidArgumentException` for this to work)
   4. BigQueryIO skips over the error handling and falls back to logging the 
failure 
[here](https://github.com/apache/beam/blob/master/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/bigquery/StorageApiWriteUnshardedRecords.java#L494).
 Note that this returns `RetryType.RETRY_ALL_OPERATIONS`, meaning that the 
failure will be retried 1000 times (with backoff) as per the [RetryManager 
defaults.](https://github.com/apache/beam/blob/master/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/bigquery/StorageApiWriteUnshardedRecords.java#L577)
   
   Hope this helps to clarify? I'm not sure if it makes sense to re-open this 
issue with an updated description, or create a new issue to capture this?


-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: [email protected]

For queries about this service, please contact Infrastructure at:
[email protected]

Reply via email to