You are viewing a plain text version of this content. The canonical link for it is here.
Posted to dev@gobblin.apache.org by "Matthew Ho (Jira)" <ji...@apache.org> on 2023/02/15 18:59:00 UTC

[jira] [Commented] (GOBBLIN-318) Gobblin Helix Jobs Hang Indefinitely

    [ https://issues.apache.org/jira/browse/GOBBLIN-318?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=17689308#comment-17689308 ] 

Matthew Ho commented on GOBBLIN-318:
------------------------------------

[https://github.com/apache/gobblin/pull/3580]

 

Issue is addressed by using delete workflow instead of stop and wait. For gobblin use-case, delete works much better and is confirmed to be safer because the job is immediately dropped and task runners are unable to commit data anymore

> Gobblin Helix Jobs Hang Indefinitely 
> -------------------------------------
>
>                 Key: GOBBLIN-318
>                 URL: https://issues.apache.org/jira/browse/GOBBLIN-318
>             Project: Apache Gobblin
>          Issue Type: Bug
>            Reporter: Joel Baranick
>            Priority: Critical
>
> In some cases, gobblin helix jobs can hang indefinitely.  When coupled with job locks, this can result in a job becoming stuck and not progressing.  The only solution currently is to restart the master node.
> Assume the following is for a {{job_myjob_1510884004834}} and which hung at 2017-11-17 02:09:00 UTC and was still hung at 2017-11-17 09:12:00 UTC. {{GobblinHelixJobLauncher.waitForJobCompletion()}} is never detecting the job as completed. This results in the {{TaskStateCollectorService}} indefinitely searching for more task states, even though it has processed all the task states that are ever going to be produced.  There is no reference to the hung job in Zookeeper at {{/mycluster/CONFIGS/RESOURCE}}.  In the Helix Web Admin, the hung job doesn't exist at {{/clusters/mycluster/jobQueues/jobname}}. There is no record of the job in Zookeeper at {{/mycluster/PROPERTYSTORE/TaskRebalancer/jobname/Context}}.  This means that the {{GobblinHelixJobLauncher.waitForJobCompletion()}} code fails.
> {code:java}
> private void waitForJobCompletion() throws InterruptedException {
>     while (true) {
>       WorkflowContext workflowContext = TaskDriver.getWorkflowContext(this.helixManager, this.helixQueueName);
>       if (workflowContext != null) {
>         org.apache.helix.task.TaskState helixJobState = workflowContext.getJobState(this.jobResourceName);
>         if (helixJobState == org.apache.helix.task.TaskState.COMPLETED ||
>             helixJobState == org.apache.helix.task.TaskState.FAILED ||
>             helixJobState == org.apache.helix.task.TaskState.STOPPED) {
>           return;
>         }
>       }
>       Thread.sleep(1000);
>     }
>   }
> {code}
> The code gets the job state from Zookeeper:
> {code:javascript}
> {
>   "id": "WorkflowContext",
>   "simpleFields": {
>     "START_TIME": "1505159715449",
>     "STATE": "IN_PROGRESS"
>   },
>   "listFields": {},
>   "mapFields": {
>     "JOB_STATES": {
>       "jobname_job_jobname_1507415700001": "COMPLETED",
>       "jobname_job_jobname_1507756800000": "COMPLETED",
>       "jobname_job_jobname_1507959300001": "COMPLETED",
>       "jobname_job_jobname_1509857102910": "COMPLETED",
>       "jobname_job_jobname_1510253708033": "COMPLETED",
>       "jobname_job_jobname_1510271102898": "COMPLETED",
>       "jobname_job_jobname_1510852210668": "COMPLETED",
>       "jobname_job_jobname_1510853133675": "COMPLETED"
>     }
>   }
> }
> {code}
> But there is no information contained in the job state for the hung job.
> Also, it is really strange that the job states contained in that json blob are so old.  The oldest one is from 2017-10-7 10:35:00 PM UTC, more than a month ago.
> I'm not sure how the system got in this state, but this isn't the first time we have seen this.  While it would be good to prevent this from happening, it would also be good to allow the system to recover if this state is entered.



--
This message was sent by Atlassian Jira
(v8.20.10#820010)