You are viewing a plain text version of this content. The canonical link for it is here.
Posted to github@beam.apache.org by "liferoad (via GitHub)" <gi...@apache.org> on 2023/05/03 13:16:45 UTC

[GitHub] [beam] liferoad opened a new issue, #26515: [Bug]: Potential data loss when using STORAGE_WRITE_API with BigQuery IO as a sink for Dataflow batch jobs

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

   ### What happened?
   
   ### Bug details
   When users use the STORAGE_WRITE_API method with BigQuery IO as a sink to launch Dataflow batch jobs, a potential data loss could occur. The Dataflow job logs may contain errors like “OffsetAlreadyExists: ALREADY_EXISTS: The offset is within the stream, expected offset..”. This issue is reported here: https://lists.apache.org/thread/rpy8qbpkx4jtrxjfoor39skjp71gh0sx.
   
   ### Code to reproduce the bug
   Below code can reproduce the bug:
   ```java
   public static void main(String[] args) {
    BigQueryOptions options = PipelineOptionsFactory.fromArgs(args).create().as(BigQueryOptions.class);
    options.setStorageApiAppendThresholdRecordCount(5);
    Pipeline p= Pipeline.create(options);
   
    p.apply("ReadLines", TextIO.read().from("gs://apache-beam-samples/shakespeare/kinglear.txt"))
        .apply("Save Events To BigQuery", BigQueryIO.<String>write()
            .to("google.com:clouddfe:reprodataset.reprotable")
            .withFormatFunction(s -> new TableRow().set("words", s))
            .withMethod(Write.Method.STORAGE_WRITE_API)
            .withCreateDisposition(CreateDisposition.CREATE_NEVER)
            .withWriteDisposition(BigQueryIO.Write.WriteDisposition.WRITE_APPEND));
   
    p.run();
   }
   
   ```
   ### Workaround
   Using [FILE_LOADS](https://beam.apache.org/releases/javadoc/current/org/apache/beam/sdk/io/gcp/bigquery/BigQueryIO.Write.Method.html#FILE_LOADS) works without any data loss issue for BigQuery IO as a sink for Dataflow batch jobs.
   
   ### Fix
   #26503 fix this bug. But the next Beam release (2.47.0) will not pick up this fix.
   
   
   
   
   
   ### Issue Priority
   
   Priority: 1 (data loss / total loss of function)
   
   ### Issue Components
   
   - [ ] Component: Python SDK
   - [X] Component: Java SDK
   - [ ] Component: Go SDK
   - [ ] Component: Typescript SDK
   - [ ] 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: github-unsubscribe@beam.apache.org.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [beam] johnjcasey commented on issue #26515: [Bug]: Potential data loss when using STORAGE_WRITE_API with BigQuery IO as a sink for Dataflow batch jobs

Posted by "johnjcasey (via GitHub)" <gi...@apache.org>.
johnjcasey commented on issue #26515:
URL: https://github.com/apache/beam/issues/26515#issuecomment-1533091709

   For context, the bug that causes dataloss for most/all pipelines was introduced in 2.44.
   
   The potential for dataloss appears to have always impacted the storage write api in batch mode due to incorrect error handling when receiving an error from BQ.


-- 
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: github-unsubscribe@beam.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [beam] johnjcasey commented on issue #26515: [Bug]: Potential data loss when using STORAGE_WRITE_API with BigQuery IO as a sink for Dataflow batch jobs

Posted by "johnjcasey (via GitHub)" <gi...@apache.org>.
johnjcasey commented on issue #26515:
URL: https://github.com/apache/beam/issues/26515#issuecomment-1533255066

   I'm going to close this bug in favor of #26521 and #26520, to separate the issues.


-- 
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: github-unsubscribe@beam.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [beam] johnjcasey closed issue #26515: [Bug]: Potential data loss when using STORAGE_WRITE_API with BigQuery IO as a sink for Dataflow batch jobs

Posted by "johnjcasey (via GitHub)" <gi...@apache.org>.
johnjcasey closed issue #26515: [Bug]: Potential data loss when using STORAGE_WRITE_API with BigQuery IO as a sink for Dataflow batch jobs
URL: https://github.com/apache/beam/issues/26515


-- 
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: github-unsubscribe@beam.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [beam] johnjcasey commented on issue #26515: [Bug]: Potential data loss when using STORAGE_WRITE_API with BigQuery IO as a sink for Dataflow batch jobs

Posted by "johnjcasey (via GitHub)" <gi...@apache.org>.
johnjcasey commented on issue #26515:
URL: https://github.com/apache/beam/issues/26515#issuecomment-1533068016

   This impacts versions 2.44, 2.45, 2.46


-- 
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: github-unsubscribe@beam.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [beam] johnjcasey commented on issue #26515: [Bug]: Potential data loss when using STORAGE_WRITE_API with BigQuery IO as a sink for Dataflow batch jobs

Posted by "johnjcasey (via GitHub)" <gi...@apache.org>.
johnjcasey commented on issue #26515:
URL: https://github.com/apache/beam/issues/26515#issuecomment-1533105013

   Addendum, this is since we started caching the write stream, which started in 2.35.0


-- 
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: github-unsubscribe@beam.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org