[
https://issues.apache.org/jira/browse/BEAM-7195?focusedWorklogId=492652&page=com.atlassian.jira.plugin.system.issuetabpanels:worklog-tabpanel#worklog-492652
]
ASF GitHub Bot logged work on BEAM-7195:
----------------------------------------
Author: ASF GitHub Bot
Created on: 29/Sep/20 20:17
Start Date: 29/Sep/20 20:17
Worklog Time Spent: 10m
Work Description: guangstick opened a new pull request #12968:
URL: https://github.com/apache/beam/pull/12968
In WriteTables and WriteRename, the `createDisposition` is set to
`CREATE_NEVER` when the pane index is great than 0.
https://github.com/apache/beam/blob/ecedd3e654352f1b51ab2caae0fd4665403bd0eb/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/bigquery/WriteTables.java#L205-L210
https://github.com/apache/beam/blob/ecedd3e654352f1b51ab2caae0fd4665403bd0eb/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/bigquery/WriteRename.java#L145-L148
Before actually triggering BQ load, BatchLoads groups all temporary files by
a singleton key.
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#L308-L313
This can be problematic when using
[DynamicDestinations](https://github.com/apache/beam/blob/ecedd3e654352f1b51ab2caae0fd4665403bd0eb/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/bigquery/DynamicDestinations.java)
in a streaming pipeline with triggers. Sometimes users would like to create
tables in the middle of the pipeline based on the data or time. Because of the
`GroupOntoSingleton`, different BQ destinations are grouped under the same key
(always `Void`), therefore the pane index is always incremental and new tables
won't be created even though users have specified `CreateDisposition =
CREATE_IF_NEEDED`
Proposed Solution: Instead of grouping into a singleton, grouping by
Bigquery destinations.
------------------------
Thank you for your contribution! Follow this checklist to help us
incorporate your contribution quickly and easily:
- [ ] [**Choose
reviewer(s)**](https://beam.apache.org/contribute/#make-your-change) and
mention them in a comment (`R: @username`).
- [ ] Format the pull request title like `[BEAM-XXX] Fixes bug in
ApproximateQuantiles`, where you replace `BEAM-XXX` with the appropriate JIRA
issue, if applicable. This will automatically link the pull request to the
issue.
- [ ] Update `CHANGES.md` with noteworthy changes.
- [ ] If this contribution is large, please file an Apache [Individual
Contributor License Agreement](https://www.apache.org/licenses/icla.pdf).
See the [Contributor Guide](https://beam.apache.org/contribute) for more
tips on [how to make review process
smoother](https://beam.apache.org/contribute/#make-reviewers-job-easier).
Post-Commit Tests Status (on master branch)
------------------------------------------------------------------------------------------------
Lang | SDK | Dataflow | Flink | Samza | Spark | Twister2
--- | --- | --- | --- | --- | --- | ---
Go | [](https://ci-beam.apache.org/job/beam_PostCommit_Go/lastCompletedBuild/)
| --- | [](https://ci-beam.apache.org/job/beam_PostCommit_Go_VR_Flink/lastCompletedBuild/)
| --- | [](https://ci-beam.apache.org/job/beam_PostCommit_Go_VR_Spark/lastCompletedBuild/)
| ---
Java | [](https://ci-beam.apache.org/job/beam_PostCommit_Java/lastCompletedBuild/)
| [](https://ci-beam.apache.org/job/beam_PostCommit_Java_ValidatesRunner_Dataflow/lastCompletedBuild/)<br>[](https://ci-beam.apache.org/job/beam_PostCommit_Java_ValidatesRunner_Dataflow_Java11/lastCompletedBuild/)
| [](https://ci-beam.apache.org/job/beam_PostCommit_Java_ValidatesRunner_Flink/lastCompletedBuild/)<br>[](https://ci-beam.apache.org/job/beam_PostCommit_Java_ValidatesRunner_Flink_Java11/lastCompletedBuild/)<br>[](https://ci-beam.apache.org/job/beam_PostCommit_Java_PVR_Flink_Batch/lastCompletedBuild/)<br>[](https://ci-beam.apache.org/job/beam_PostCommit_Java_PVR_Flink_Streaming/lastCompletedBuild/)
| [](https://ci-beam.apache.org/job/beam_PostCommit_Java_ValidatesRunner_Samza/lastCompletedBuild/)
| [](https://ci-beam.apache.org/job/beam_PostCommit_Java_ValidatesRunner_Spark/lastCompletedBuild/)<br>[](https://ci-beam.apache.org/job/beam_PostCommit_Java_PVR_Spark_Batch/lastCompletedBuild/)<br>[](https://ci-beam.apache.org/job/beam_PostCommit_Java_ValidatesRunner_SparkStructuredStreaming/lastCompletedBuild/)
| [](https://ci-beam.apache.org/job/beam_PostCommit_Java_ValidatesRunner_Twister2/lastCompletedBuild/)
Python | [](https://ci-beam.apache.org/job/beam_PostCommit_Python36/lastCompletedBuild/)<br>[](https://ci-beam.apache.org/job/beam_PostCommit_Python37/lastCompletedBuild/)<br>[](https://ci-beam.apache.org/job/beam_PostCommit_Python38/lastCompletedBuild/)
| [](https://ci-beam.apache.org/job/beam_PostCommit_Py_VR_Dataflow/lastCompletedBuild/)<br>[](https://ci-beam.apache.org/job/beam_PostCommit_Py_VR_Dataflow_V2/lastCompletedBuild/)<br>[](https://ci-beam.apache.org/job/beam_PostCommit_Py_ValCont/lastCompletedBuild/)
| [](https://ci-beam.apache.org/job/beam_PreCommit_Python_PVR_Flink_Cron/lastCompletedBuild/)<br>[](https://ci-beam.apache.org/job/beam_PostCommit_Python_VR_Flink/lastCompletedBuild/)
| --- | [](https://ci-beam.apache.org/job/beam_PostCommit_Python_VR_Spark/lastCompletedBuild/)
| ---
XLang | [](https://ci-beam.apache.org/job/beam_PostCommit_XVR_Direct/lastCompletedBuild/)
| --- | [](https://ci-beam.apache.org/job/beam_PostCommit_XVR_Flink/lastCompletedBuild/)
| --- | [](https://ci-beam.apache.org/job/beam_PostCommit_XVR_Spark/lastCompletedBuild/)
| ---
Pre-Commit Tests Status (on master branch)
------------------------------------------------------------------------------------------------
--- |Java | Python | Go | Website | Whitespace | Typescript
--- | --- | --- | --- | --- | --- | ---
Non-portable | [](https://ci-beam.apache.org/job/beam_PreCommit_Java_Cron/lastCompletedBuild/)
| [](https://ci-beam.apache.org/job/beam_PreCommit_Python_Cron/lastCompletedBuild/)<br>[](https://ci-beam.apache.org/job/beam_PreCommit_PythonLint_Cron/lastCompletedBuild/)<br>[](https://ci-beam.apache.org/job/beam_PreCommit_PythonDocker_Cron/lastCompletedBuild/)
<br>[](https://ci-beam.apache.org/job/beam_PreCommit_PythonDocs_Cron/lastCompletedBuild/)
| [](https://ci-beam.apache.org/job/beam_PreCommit_Go_Cron/lastCompletedBuild/)
| [](https://ci-beam.apache.org/job/beam_PreCommit_Website_Cron/lastCompletedBuild/)
| [](https://ci-beam.apache.org/job/beam_PreCommit_Whitespace_Cron/lastCompletedBuild/)
| [](https://ci-beam.apache.org/job/beam_PreCommit_Typescript_Cron/lastCompletedBuild/)
Portable | --- | [](https://ci-beam.apache.org/job/beam_PreCommit_Portable_Python_Cron/lastCompletedBuild/)
| --- | --- | --- | ---
See
[.test-infra/jenkins/README](https://github.com/apache/beam/blob/master/.test-infra/jenkins/README.md)
for trigger phrase, status and link of all Jenkins jobs.
GitHub Actions Tests Status (on master branch)
------------------------------------------------------------------------------------------------
[](https://github.com/apache/beam/actions?query=workflow%3A%22Build+python+source+distribution+and+wheels%22+branch%3Amaster+event%3Aschedule)
[](https://github.com/apache/beam/actions?query=workflow%3A%22Python+Tests%22+branch%3Amaster+event%3Aschedule)
[](https://github.com/apache/beam/actions?query=workflow%3A%22Java+Tests%22+branch%3Amaster+event%3Aschedule)
See [CI.md](https://github.com/apache/beam/blob/master/CI.md) for more
information about GitHub Actions CI.
----------------------------------------------------------------
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.
For queries about this service, please contact Infrastructure at:
[email protected]
Issue Time Tracking
-------------------
Worklog Id: (was: 492652)
Remaining Estimate: 0h
Time Spent: 10m
> BigQuery - 404 errors for 'table not found' when using dynamic destinations -
> sometimes, new table fails to get created
> -----------------------------------------------------------------------------------------------------------------------
>
> Key: BEAM-7195
> URL: https://issues.apache.org/jira/browse/BEAM-7195
> Project: Beam
> Issue Type: Bug
> Components: io-java-gcp
> Affects Versions: 2.5.0
> Environment: Windows
> Reporter: Chris
> Assignee: Guangyu Chen
> Priority: P1
> Time Spent: 10m
> Remaining Estimate: 0h
>
> See the following StackOverflow question, which describes the details:
>
> [https://stackoverflow.com/questions/55932291/apache-beam-for-google-cloud-dataflow-404-errors-when-using-bigqueryio-write-c]
--
This message was sent by Atlassian Jira
(v8.3.4#803005)