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

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_r206059383
 
 

 ##########
 File path: 
sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/bigquery/WriteTables.java
 ##########
 @@ -261,16 +265,33 @@ private void load(
     Job lastFailedLoadJob = null;
     String bqLocation =
         BigQueryHelpers.getDatasetLocation(datasetService, ref.getProjectId(), 
ref.getDatasetId());
-    for (int i = 0; i < BatchLoads.MAX_RETRY_JOBS; ++i) {
-      String jobId = jobIdPrefix + "-" + i;
-
+    // First attempt is always jobIdPrefix-0.
+    String jobId = jobIdPrefix + "-0";
+    for (int i = 0; i < maxRetryJobs; ++i) {
       JobReference jobRef =
           new 
JobReference().setProjectId(projectId).setJobId(jobId).setLocation(bqLocation);
 
       LOG.info("Loading {} files into {} using job {}, attempt {}", 
gcsUris.size(), ref, jobRef, i);
-      jobService.startLoadJob(jobRef, loadConfig);
+      try {
+        jobService.startLoadJob(jobRef, loadConfig);
+      } catch (IOException e) {
+        LOG.warn("Load job {} failed with {}", jobRef, e);
+        // It's possible that the job actually made it to BQ even though we 
got a failure here.
+        // For example, the response from BQ may have timed out returning. 
getRetryJobId will
+        // return the correct job id to use on retry, or a job id to continue 
polling (if it turns
+        // out the the job has not actually failed yet).
+        RetryJobIdResult result =
+            BigQueryHelpers.getRetryJobId(jobId, projectId, bqLocation, 
jobService);
+        jobId = result.jobId;
+        if (result.shouldRetry) {
+          // Try the load again with the new job id.
+          continue;
 
 Review comment:
   Use exponential backoff ?

----------------------------------------------------------------
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: 128820)
    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