You are viewing a plain text version of this content. The canonical link for it is here.
Posted to dev@tez.apache.org by "Jonathan Turner Eagles (Jira)" <ji...@apache.org> on 2019/12/13 20:03:00 UTC

[jira] [Created] (TEZ-4108) NullPointerException during speculative execution race condition

Jonathan Turner Eagles created TEZ-4108:
-------------------------------------------

             Summary: NullPointerException during speculative execution race condition
                 Key: TEZ-4108
                 URL: https://issues.apache.org/jira/browse/TEZ-4108
             Project: Apache Tez
          Issue Type: Bug
            Reporter: Jonathan Turner Eagles
            Assignee: Jonathan Turner Eagles


What is happening is a race condition between speculative execution task launch and failure. Such that when the speculative attempt begins to launch, the earlier attempt is no longer running. Somehow the containerNodeId is becoming null. From the log line we can see that at least when the original task attempt starts, the container node id is not null.

This points to a bug in Tez.

Posting AM log context and full stack trace.
{noformat}
2019-11-18 12:58:13,261 [INFO] [Dispatcher thread {Central}] |HistoryEventHandler.criticalEvents|: [HISTORY][DAG:dag_1571959303156_328042_1][Event:TASK_ATTEMPT_STARTED]: vertexName=Map 1, taskAttemptId=attempt_1571959303156_328042_1_00_091618_0, startTime=1574081893261, containerId=container_e20_1571959303156_328042_01_006583, nodeId=node1.example.com:8041
2019-11-18 12:58:25,001 [INFO] [Dispatcher thread {Central}] |HistoryEventHandler.criticalEvents|: [HISTORY][DAG:dag_1571959303156_328042_1][Event:TASK_STARTED]: vertexName=Map 1, taskId=task_1571959303156_328042_1_00_091618, scheduledTime=1574081308038, launchTime=1574081893261
2019-11-18 12:58:26,916 [INFO] [Dispatcher thread {Central}] |HistoryEventHandler.criticalEvents|: [HISTORY][DAG:dag_1571959303156_328042_1][Event:TASK_ATTEMPT_FINISHED]: vertexName=Map 1, taskAttemptId=attempt_1571959303156_328042_1_00_091618_0, creationTime=1574081308038, allocationTime=1574081880179, startTime=1574081893261, finishTime=1574081906915, timeTaken=13654, status=KILLED, errorEnum=EXTERNAL_PREEMPTION, diagnostics=Container container_e20_1571959303156_328042_01_006583 finished with diagnostics set to [Container preempted externally. Container Preempted due to node overcommit
2019-11-18 12:58:27,264 [INFO] [Dispatcher thread {Speculator}] |legacy.LegacySpeculator|: DefaultSpeculator.addSpeculativeAttempt -- we are speculating task_1571959303156_328042_1_00_091618
2019-11-18 12:58:34,633 [INFO] [TaskSchedulerEventHandlerThread] |rm.TaskSchedulerManager|: Task: attempt_1571959303156_328042_1_00_091618_0 has no container assignment in the scheduler
2019-11-18 12:58:34,636 [ERROR] [TaskSchedulerEventHandlerThread] |rm.TaskSchedulerManager|: No container allocated to task: attempt_1571959303156_328042_1_00_091618_0 according to scheduler. Task reported container id: container_e20_1571959303156_328042_01_006583
2019-11-18 12:58:34,866 [INFO] [Dispatcher thread {Central}] |impl.TaskImpl|: Scheduling a redundant attempt for task task_1571959303156_328042_1_00_091618
2019-11-18 12:58:34,866 [ERROR] [Dispatcher thread {Central}] |impl.TaskImpl|: Uncaught exception when trying handle event T_ADD_SPEC_ATTEMPT at current state RUNNING for task task_1571959303156_328042_1_00_091618
java.lang.NullPointerException
        at java.util.concurrent.ConcurrentHashMap.putVal(ConcurrentHashMap.java:1011)
        at java.util.concurrent.ConcurrentHashMap.put(ConcurrentHashMap.java:1006)
        at java.util.Collections$SetFromMap.add(Collections.java:5461)
        at org.apache.tez.dag.app.dag.impl.TaskImpl$RedundantScheduleTransition.transition(TaskImpl.java:1044)
        at org.apache.tez.dag.app.dag.impl.TaskImpl$RedundantScheduleTransition.transition(TaskImpl.java:1032)
        at org.apache.hadoop.yarn.state.StateMachineFactory$SingleInternalArc.doTransition(StateMachineFactory.java:362)
        at org.apache.hadoop.yarn.state.StateMachineFactory.doTransition(StateMachineFactory.java:302)
        at org.apache.hadoop.yarn.state.StateMachineFactory.access$300(StateMachineFactory.java:46)
        at org.apache.hadoop.yarn.state.StateMachineFactory$InternalStateMachine.doTransition(StateMachineFactory.java:448)
        at org.apache.tez.state.StateMachineTez.doTransition(StateMachineTez.java:59)
        at org.apache.tez.dag.app.dag.impl.TaskImpl.handle(TaskImpl.java:841)
        at org.apache.tez.dag.app.dag.impl.TaskImpl.handle(TaskImpl.java:115)
        at org.apache.tez.dag.app.DAGAppMaster$TaskEventDispatcher.handle(DAGAppMaster.java:2311)
        at org.apache.tez.dag.app.DAGAppMaster$TaskEventDispatcher.handle(DAGAppMaster.java:2298)
        at org.apache.tez.common.AsyncDispatcher.dispatch(AsyncDispatcher.java:180)
        at org.apache.tez.common.AsyncDispatcher$1.run(AsyncDispatcher.java:115)
        at java.lang.Thread.run(Thread.java:748)
{noformat}

{code:title=TaskImpl.java}
1032   private static class RedundantScheduleTransition
1033     implements SingleArcTransition<TaskImpl, TaskEvent> {
1034
1035     @Override
1036     public void transition(TaskImpl task, TaskEvent event) {
1037       LOG.info("Scheduling a redundant attempt for task " + task.taskId);
1038       task.counters.findCounter(TaskCounter.NUM_SPECULATIONS).increment(1);
1039       TaskAttempt earliestUnfinishedAttempt = null;
1040       for (TaskAttempt ta : task.attempts.values()) {
1041         // find the oldest running attempt
1042         if (!ta.isFinished()) {
1043           earliestUnfinishedAttempt = ta;
1044           task.nodesWithRunningAttempts.add(ta.getNodeId());
1045         } else {
1046           if (TaskAttemptState.SUCCEEDED.equals(ta.getState())) {
1047             LOG.info("Ignore speculation scheduling for task {} since it has succeeded with attempt {}.",
1048                 task.getTaskId(), ta.getID());
1049             return;
1050           }
1051         }
1052       }
1053       if (earliestUnfinishedAttempt == null) {
1054         // no running (or SUCCEEDED) task attempt at this moment, no need to schedule speculative attempt either
1055         LOG.info("Ignore speculation scheduling since there is no running attempt on task {}.", task.getTaskId());
1056         return;
1057       }
1058       if (task.commitAttempt != null) {
1059         LOG.info("Ignore speculation scheduling for task {} since commit has started with commitAttempt {}.",
1060             task.getTaskId(), task.commitAttempt);
1061         return;
1062       }
1063       task.addAndScheduleAttempt(earliestUnfinishedAttempt.getID());
1064     }
1065   }
{code}



--
This message was sent by Atlassian Jira
(v8.3.4#803005)