[ 
https://issues.apache.org/jira/browse/BEAM-5040?focusedWorklogId=128821&page=com.atlassian.jira.plugin.system.issuetabpanels:worklog-tabpanel#worklog-128821
 ]

ASF GitHub Bot logged work on BEAM-5040:
----------------------------------------

                Author: ASF GitHub Bot
            Created on: 30/Jul/18 17:36
            Start Date: 30/Jul/18 17:36
    Worklog Time Spent: 10m 
      Work Description: chamikaramj commented on a change in pull request 
#6080: [BEAM-5040] Fix retry bug for BigQuery jobs.
URL: https://github.com/apache/beam/pull/6080#discussion_r206257701
 
 

 ##########
 File path: 
sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/bigquery/BigQueryHelpers.java
 ##########
 @@ -55,6 +59,77 @@
           + " an earlier stage of the pipeline, this validation can be 
disabled using"
           + " #withoutValidation.";
 
+  private static final Logger LOG = 
LoggerFactory.getLogger(BigQueryHelpers.class);
+
+  // Given a potential failure and a current job-id, return the next job-id to 
be used on retry.
+  // Algorithm is as follows (given input of job_id_prefix-N)
+  //   If BigQuery has no status for job_id_prefix-n, we should retry with the 
same id.
+  //   If job-id-prefix-n is in the PENDING or successful states, no retry is 
needed.
+  //   Otherwise (job-id-prefix-n completed with errors), try again with 
job-id-prefix-(n+1)
+  //
+  // We continue to loop through these job ids until we find one that has 
either succeed, or that
+  // has not been issued yet.
+  static class RetryJobIdResult {
+    public final String jobId;
+    public final boolean shouldRetry;
+
+    public RetryJobIdResult(String jobId, boolean shouldRetry) {
+      this.jobId = jobId;
+      this.shouldRetry = shouldRetry;
+    }
+  }
+
+  static RetryJobIdResult getRetryJobId(
+      String currentJobId, String projectId, String bqLocation, JobService 
jobService)
+      throws InterruptedException {
+    // Job ids should always be of the form <job_id_prefix>-<retry_count>
 
 Review comment:
   Add a BigQueryJobId class with prefix, retry properties to make sure that 
this invariant is preserved ?

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
us...@infra.apache.org


Issue Time Tracking
-------------------

    Worklog Id:     (was: 128821)
    Time Spent: 40m  (was: 0.5h)

> BigQueryIO retries infinitely in WriteTable and WriteRename
> -----------------------------------------------------------
>
>                 Key: BEAM-5040
>                 URL: https://issues.apache.org/jira/browse/BEAM-5040
>             Project: Beam
>          Issue Type: Bug
>          Components: io-java-gcp
>    Affects Versions: 2.5.0
>            Reporter: Reuven Lax
>            Assignee: Reuven Lax
>            Priority: Major
>          Time Spent: 40m
>  Remaining Estimate: 0h
>
> BigQueryIO retries infinitely in WriteTable and WriteRename
> Several failure scenarios with the current code:
>  # It's possible for a load job to return failure even though it actually 
> succeeded (e.g. the reply might have timed out). In this case, BigQueryIO 
> will retry the job which will fail again (because the job id has already been 
> used), leading to indefinite retries. Correct behavior is to stop retrying as 
> the load job has succeeded.
>  # It's possible for a load job to be accepted by BigQuery, but then to fail 
> on the BigQuery side. In this case a retry with the same job id will fail as 
> that job id has already been used. BigQueryIO will sometimes detect this, but 
> if the worker has restarted it will instead issue a load with the old job id 
> and go into a retry loop. Correct behavior is to generate a new deterministic 
> job id and retry using that new job id.
>  # In many cases of worker restart, BigQueryIO ends up in infinite retry 
> loops.



--
This message was sent by Atlassian JIRA
(v7.6.3#76005)

Reply via email to