Details
-
Sub-task
-
Status: Closed
-
Major
-
Resolution: Fixed
-
None
-
None
-
None
Description
No idea if this is LLAP bug, Tez bug, Hadoop IPC bug (due to patch on the cluster), or all 3.
So for now I will just dump all I have here.
TPCH Q1 started running for a long time for me on large number of runs today (didn't happen yesterday). It would always be one Map task timing out.
Example attempt (logs from am):
2015-04-24 11:11:01,073 INFO [TaskCommunicator # 0] tezplugins.LlapTaskCommunicator: Successfully launched task: attempt_1429683757595_0321_9_00_000928_0 2015-04-24 11:16:25,498 INFO [Dispatcher thread: Central] history.HistoryEventHandler: [HISTORY][DAG:dag_1429683757595_0321_9][Event:TASK_ATTEMPT_FINISHED]: vertexName=Map 1, taskAttemptId=attempt_1429683757595_0321_9_00_000928_0, startTime=1429899061071, finishTime=1429899385498, timeTaken=324427, status=FAILED, errorEnum=TASK_HEARTBEAT_ERROR, diagnostics=AttemptID:attempt_1429683757595_0321_9_00_000928_0 Timed out after 300 secs, counters=Counters: 1, org.apache.tez.common.counters.DAGCounter, RACK_LOCAL_TASKS=1
No other lines for this attempt in between.
However there's this:
2015-04-24 11:11:01,074 WARN [Socket Reader #1 for port 59446] ipc.Server: Unable to read call parameters for client 172.19.128.56on connection protocol org.apache.hadoop.hive.llap.protocol.LlapTaskUmbilicalProtocol for rpcKind RPC_WRITABLE java.lang.ArrayIndexOutOfBoundsException 2015-04-24 11:11:01,075 INFO [Socket Reader #1 for port 59446] ipc.Server: Socket Reader #1 for port 59446: readAndProcess from client 172.19.128.56 threw exception [org.apache.hadoop.ipc.RpcServerException: IPC server unable to read call parameters: null]
On LLAP, the following is logged
2015-04-24 11:11:01,142 [TaskHeartbeatThread()] ERROR org.apache.tez.runtime.task.TezTaskRunner: TaskReporter reported error org.apache.hadoop.ipc.RemoteException(org.apache.hadoop.ipc.RpcServerException): IPC server unable to read call parameters: null at org.apache.hadoop.ipc.Client.call(Client.java:1492) at org.apache.hadoop.ipc.Client.call(Client.java:1423) at org.apache.hadoop.ipc.WritableRpcEngine$Invoker.invoke(WritableRpcEngine.java:242) at com.sun.proxy.$Proxy19.heartbeat(Unknown Source) at org.apache.hadoop.hive.llap.daemon.impl.LlapTaskReporter$HeartbeatCallable.heartbeat(LlapTaskReporter.java:258) at org.apache.hadoop.hive.llap.daemon.impl.LlapTaskReporter$HeartbeatCallable.call(LlapTaskReporter.java:186) at org.apache.hadoop.hive.llap.daemon.impl.LlapTaskReporter$HeartbeatCallable.call(LlapTaskReporter.java:128) at java.util.concurrent.FutureTask.run(FutureTask.java:266) at java.util.concurrent.ThreadPoolExecutor.runWorker(ThreadPoolExecutor.java:1142) at java.util.concurrent.ThreadPoolExecutor$Worker.run(ThreadPoolExecutor.java:617) at java.lang.Thread.run(Thread.java:745)
The attempt starts but is then interrupted (not clear by whom)
2015-04-24 11:11:01,144 [Initializer 0(container_222212222_0321_01_008943_sershe_20150424110948_86ce1f6f-7cd2-4a40-b9a6-4a6854f010f6:9_Map 1_928_0)] INFO org.apache.tez.runtime.LogicalIOProcessorRuntimeTask: Initialized Input with src edge: lineitem 2015-04-24 11:11:01,145 [TezTaskRunner_attempt_1429683757595_0321_9_00_000928_0(container_222212222_0321_01_008943_sershe_20150424110948_86ce1f6f-7cd2-4a40-b9a6-4a6854f010f6:9_Map 1_928_0)] INFO org.apache.tez.runtime.task.TezTaskRunner: Encounted an error while executing task: attempt_1429683757595_0321_9_00_000928_0 java.lang.InterruptedException at java.util.concurrent.locks.AbstractQueuedSynchronizer.acquireInterruptibly(AbstractQueuedSynchronizer.java:1220) at java.util.concurrent.locks.ReentrantLock.lockInterruptibly(ReentrantLock.java:335) at java.util.concurrent.LinkedBlockingQueue.take(LinkedBlockingQueue.java:439) at java.util.concurrent.ExecutorCompletionService.take(ExecutorCompletionService.java:193) at org.apache.tez.runtime.LogicalIOProcessorRuntimeTask.initialize(LogicalIOProcessorRuntimeTask.java:218) at org.apache.tez.runtime.task.TezTaskRunner$TaskRunnerCallable$1.run(TezTaskRunner.java:177) at org.apache.tez.runtime.task.TezTaskRunner$TaskRunnerCallable$1.run(TezTaskRunner.java:172)
There's a heartbeat error that may or may not be spurious:
2015-04-24 11:11:01,146 [TezTaskRunner_attempt_1429683757595_0321_9_00_000928_0(container_222212222_0321_01_008943_sershe_20150424110948_86ce1f6f-7cd2-4a40-b9a6-4a6854f010f6:9_Map 1_928_0)] INFO org.apache.tez.runtime.task.TezTaskRunner: Ignoring the following exception since a previous exception is already registered java.lang.NullPointerException at org.apache.hadoop.hive.llap.daemon.impl.LlapTaskReporter$HeartbeatCallable.access$300(LlapTaskReporter.java:129) at org.apache.hadoop.hive.llap.daemon.impl.LlapTaskReporter.taskFailed(LlapTaskReporter.java:394) at org.apache.tez.runtime.task.TezTaskRunner.sendFailure(TezTaskRunner.java:261) at org.apache.tez.runtime.task.TezTaskRunner.access$600(TezTaskRunner.java:52) at org.apache.tez.runtime.task.TezTaskRunner$TaskRunnerCallable$1.run(TezTaskRunner.java:228) at org.apache.tez.runtime.task.TezTaskRunner$TaskRunnerCallable$1.run(TezTaskRunner.java:172) at java.security.AccessController.doPrivileged(Native Method) at javax.security.auth.Subject.doAs(Subject.java:422) at org.apache.hadoop.security.UserGroupInformation.doAs(UserGroupInformation.java:1657) at org.apache.tez.runtime.task.TezTaskRunner$TaskRunnerCallable.callInternal(TezTaskRunner.java:172)
And nothing more for this task.
Task succeeds on rerun. Other tasks also succeed on this node meanwhile.
1) Is it Hadoop IPC bug due to new patch?
2) Even if so (assuming IPC is not totally broken) I wonder if the Heartbeat NPE (which we have seen before, without the IPC patch too IIRC) is actually a real issue that prevents the task from being sent to AM?
3) Who interrupts the task and why? AM doesn't have any logs about that and it happens after RPC error, not before.
Btw, there's another flavor or RPC error:
2015-04-24 10:36:30,183 INFO [Socket Reader #1 for port 59446] ipc.Server: Socket Reader #1 for port 59446: readAndProcess from client 172.19.128.56 threw exception [org.apache.hadoop.ipc.RpcServerException: IPC server unable to read call parameters: 1382376565]
application_1429683757595_0320, application_1429683757595_0321