bastewart opened a new issue, #25227:
URL: https://github.com/apache/beam/issues/25227

   ### What happened?
   
   Writing to BigQuery with `BigQueryIO` in `STORAGE_WRITE_API` mode leads to 
loss of rows if any individual datapoint in it is invalid, even when using 
`ignoreUnknownValues`. The rows are piped instead to the failed output 
collection and dropped entirely.
   
   This is related to another issue which together mean it's quite common (at 
least in my use-case!) for rows to be lost.
   
   As an example I get the following error for an optional field even with 
`ignoreUnknownValues` set:
   
   ```
   Exception: 
org.apache.beam.sdk.io.gcp.bigquery.TableRowToStorageApiProto$SchemaDoesntMatchException:
 Unexpected value :36183.0, type: class java.lang.Double. Table field name: 
<snip>, type: INT64'}
   ```
   
   I think this could be a relatively easy fix; instead of throwing a 
`SchemaDoesNotMatch` exception at the end of `singularFieldToProtoValue` we 
could return `null` if `FieldDescriptor.isOptional` is true.
   
   Similarly some over `throw`s would need to change to check if the field is 
optional. e.g.: 
https://github.com/apache/beam/blob/634b0453469b66ee4c135aca48b02d2425916f36/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/bigquery/TableRowToStorageApiProto.java#L692-L697
   
   Apologies if this is not a bug, but I personally classify it as "unexpected 
behaviour" having moved from the Streaming API where this behaviour is 
supported.
   
   I'd be happy to open a PR here, but have never contributed here and Java 
isn't my primary language, so it may be a little slow...
   
   ### Issue Priority
   
   Priority: 2 (default / most bugs should be filed as P2)
   
   ### Issue Components
   
   - [ ] Component: Python SDK
   - [X] Component: Java SDK
   - [ ] Component: Go SDK
   - [ ] Component: Typescript SDK
   - [X] Component: IO connector
   - [ ] Component: Beam examples
   - [ ] Component: Beam playground
   - [ ] Component: Beam katas
   - [ ] Component: Website
   - [ ] Component: Spark Runner
   - [ ] Component: Flink Runner
   - [ ] Component: Samza Runner
   - [ ] Component: Twister2 Runner
   - [ ] Component: Hazelcast Jet Runner
   - [ ] Component: Google Cloud Dataflow Runner


-- 
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