[
https://issues.apache.org/jira/browse/HADOOP-2076?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel#action_12541849
]
Christian Kunz commented on HADOOP-2076:
----------------------------------------
We changed the max number of tasks per node to 2, but currently scaling issues
with the namenode are obscuring this issue here (also, we still run with 20
jetty threads).
It would be nice if the user was not forced to reduce the number of tasks when
the node resources could handle as many simultaneous applications as CPU's, but
not the post-processing inside the infrastructure (simultaneous merging of map
tasks and remote fetching by reduce tasks). Would it be possible to optionally
let the reduce tasks hold off fetching for a while when the map tasks do the
final merge and/or serialize the final merge of map tasks, but do it only when
resource contention is detected?
> extensive map tasks failures because of SocketTimeoutException during
> statusUpdate
> ----------------------------------------------------------------------------------
>
> Key: HADOOP-2076
> URL: https://issues.apache.org/jira/browse/HADOOP-2076
> Project: Hadoop
> Issue Type: Bug
> Components: mapred
> Affects Versions: 0.16.0
> Environment: Oct 17 #718 nightly build with patches 2033 and 2048
> Reporter: Christian Kunz
> Priority: Blocker
> Fix For: 0.15.1
>
>
> A job with 3600 tasks on a cluster of 1350 nodes (up 3 tasks per node) shows
> extensive map tasks failures because of connection timeouts at the end of the
> task (c++ application using pipes interface completed successfully)
> More than 600 tasks failed, slowing down the job because of retries. Only a
> portion of the tasks fail because of the timeout issue, but they spawn other
> failures because retries and speculatively executed tasks cannot even get a
> connection and fail just after a few seconds.
> JobTracker is running with 60 handlers. We allow up to 10 attempts for maps.
> I attach the log of a task failing because of timeout (which includes a
> thread dump), and the log of one task which could not start.
> 2007-10-18 15:58:41,743 INFO org.apache.hadoop.metrics.jvm.JvmMetrics:
> Initializing JVM Metrics with processName=MAP, sessionId=
> 2007-10-18 15:58:41,827 INFO org.apache.hadoop.mapred.MapTask:
> numReduceTasks: 3600
> 2007-10-18 16:12:28,918 INFO org.apache.hadoop.util.NativeCodeLoader: Loaded
> the native-hadoop library
> 2007-10-18 16:12:28,920 INFO org.apache.hadoop.io.compress.zlib.ZlibFactory:
> Successfully loaded & initialized native-zlib library
> 2007-10-18 17:43:00,785 INFO org.apache.hadoop.mapred.TaskRunner:
> Communication exception: java.net.SocketTimeoutException: timed out waiting
> for rpc response
> at org.apache.hadoop.ipc.Client.call(Client.java:484)
> at org.apache.hadoop.ipc.RPC$Invoker.invoke(RPC.java:184)
> at org.apache.hadoop.mapred.$Proxy0.statusUpdate(Unknown Source)
> at org.apache.hadoop.mapred.Task$1.run(Task.java:293)
> at java.lang.Thread.run(Thread.java:619)
> 2007-10-18 17:44:03,833 INFO org.apache.hadoop.mapred.TaskRunner:
> Communication exception: java.net.SocketTimeoutException: timed out waiting
> for rpc response
> at org.apache.hadoop.ipc.Client.call(Client.java:484)
> at org.apache.hadoop.ipc.RPC$Invoker.invoke(RPC.java:184)
> at org.apache.hadoop.mapred.$Proxy0.statusUpdate(Unknown Source)
> at org.apache.hadoop.mapred.Task$1.run(Task.java:293)
> at java.lang.Thread.run(Thread.java:619)
> 2007-10-18 17:45:06,838 INFO org.apache.hadoop.mapred.TaskRunner:
> Communication exception: java.net.SocketTimeoutException: timed out waiting
> for rpc response
> at org.apache.hadoop.ipc.Client.call(Client.java:484)
> at org.apache.hadoop.ipc.RPC$Invoker.invoke(RPC.java:184)
> at org.apache.hadoop.mapred.$Proxy0.statusUpdate(Unknown Source)
> at org.apache.hadoop.mapred.Task$1.run(Task.java:293)
> at java.lang.Thread.run(Thread.java:619)
> 2007-10-18 17:45:40,258 INFO org.apache.hadoop.mapred.TaskRunner: Process
> Thread Dump: Communication exception
> 8 active threads
> Thread 13 (Comm thread for task_200710172336_0016_m_000071_0):
> State: RUNNABLE
> Blocked count: 0
> Waited count: 4128
> Stack:
> sun.management.ThreadImpl.getThreadInfo0(Native Method)
> sun.management.ThreadImpl.getThreadInfo(ThreadImpl.java:147)
> sun.management.ThreadImpl.getThreadInfo(ThreadImpl.java:123)
>
> org.apache.hadoop.util.ReflectionUtils.printThreadInfo(ReflectionUtils.java:114)
>
> org.apache.hadoop.util.ReflectionUtils.logThreadInfo(ReflectionUtils.java:162)
> org.apache.hadoop.mapred.Task$1.run(Task.java:315)
> java.lang.Thread.run(Thread.java:619)
> Thread 12 ([EMAIL PROTECTED]):
> State: TIMED_WAITING
> Blocked count: 0
> Waited count: 6403
> Stack:
> java.lang.Thread.sleep(Native Method)
> org.apache.hadoop.dfs.DFSClient$LeaseChecker.run(DFSClient.java:558)
> java.lang.Thread.run(Thread.java:619)
> Thread 9 (IPC Client connection to /127.0.0.1:49458):
> State: RUNNABLE
> Blocked count: 21
> Waited count: 2063
> Stack:
> java.net.SocketInputStream.socketRead0(Native Method)
> java.net.SocketInputStream.read(SocketInputStream.java:129)
> java.io.FilterInputStream.read(FilterInputStream.java:116)
> org.apache.hadoop.ipc.Client$Connection$1.read(Client.java:181)
> java.io.BufferedInputStream.fill(BufferedInputStream.java:218)
> java.io.BufferedInputStream.read(BufferedInputStream.java:237)
> java.io.DataInputStream.readInt(DataInputStream.java:370)
> org.apache.hadoop.ipc.Client$Connection.run(Client.java:258)
> Thread 8 (org.apache.hadoop.io.ObjectWritable Connection Culler):
> State: TIMED_WAITING
> Blocked count: 0
> Waited count: 6402
> Stack:
> java.lang.Thread.sleep(Native Method)
> org.apache.hadoop.ipc.Client$ConnectionCuller.run(Client.java:404)
> Thread 4 (Signal Dispatcher):
> State: RUNNABLE
> Blocked count: 0
> Waited count: 0
> Stack:
> Thread 3 (Finalizer):
> State: WAITING
> Blocked count: 398
> Waited count: 2270
> Waiting on [EMAIL PROTECTED]
> Stack:
> java.lang.Object.wait(Native Method)
> java.lang.ref.ReferenceQueue.remove(ReferenceQueue.java:116)
> java.lang.ref.ReferenceQueue.remove(ReferenceQueue.java:132)
> java.lang.ref.Finalizer$FinalizerThread.run(Finalizer.java:159)
> Thread 2 (Reference Handler):
> State: WAITING
> Blocked count: 257
> Waited count: 2269
> Waiting on [EMAIL PROTECTED]
> Stack:
> java.lang.Object.wait(Native Method)
> java.lang.Object.wait(Object.java:485)
> java.lang.ref.Reference$ReferenceHandler.run(Reference.java:116)
> Thread 1 (main):
> State: RUNNABLE
> Blocked count: 1
> Waited count: 10
> Stack:
> java.io.FileInputStream.readBytes(Native Method)
> java.io.FileInputStream.read(FileInputStream.java:199)
>
> org.apache.hadoop.fs.RawLocalFileSystem$LocalFSFileInputStream.read(RawLocalFileSystem.java:105)
> java.io.BufferedInputStream.fill(BufferedInputStream.java:218)
> java.io.BufferedInputStream.read1(BufferedInputStream.java:258)
> java.io.BufferedInputStream.read(BufferedInputStream.java:317)
> java.io.DataInputStream.read(DataInputStream.java:132)
> org.apache.hadoop.fs.FSInputChecker.readFully(FSInputChecker.java:378)
>
> org.apache.hadoop.fs.ChecksumFileSystem$ChecksumFSInputChecker.readChunk(ChecksumFileSystem.java:200)
>
> org.apache.hadoop.fs.FSInputChecker.readChecksumChunk(FSInputChecker.java:234)
> org.apache.hadoop.fs.FSInputChecker.fill(FSInputChecker.java:176)
> org.apache.hadoop.fs.FSInputChecker.read1(FSInputChecker.java:193)
> org.apache.hadoop.fs.FSInputChecker.read(FSInputChecker.java:157)
> org.apache.hadoop.fs.FSInputChecker.readFully(FSInputChecker.java:378)
> org.apache.hadoop.fs.FSInputChecker.seek(FSInputChecker.java:359)
>
> org.apache.hadoop.fs.ChecksumFileSystem$ChecksumFSInputChecker.seek(ChecksumFileSystem.java:254)
> org.apache.hadoop.fs.FSDataInputStream.seek(FSDataInputStream.java:37)
> org.apache.hadoop.io.SequenceFile$Reader.seek(SequenceFile.java:1793)
> org.apache.hadoop.io.SequenceFile$Reader.(SequenceFile.java:1217)
> org.apache.hadoop.io.SequenceFile$Reader.(SequenceFile.java:1142)
> 2007-10-18 17:45:40,258 WARN org.apache.hadoop.mapred.TaskRunner: Last retry,
> killing task_200710172336_0016_m_000071_0
> Log of task that could not start:
> 2007-10-18 17:43:55,766 INFO org.apache.hadoop.ipc.Client: Retrying connect
> to server: /127.0.0.1:53972. Already tried 1 time(s).
> 2007-10-18 17:43:56,768 INFO org.apache.hadoop.ipc.Client: Retrying connect
> to server: /127.0.0.1:53972. Already tried 2 time(s).
> 2007-10-18 17:43:57,770 INFO org.apache.hadoop.ipc.Client: Retrying connect
> to server: /127.0.0.1:53972. Already tried 3 time(s).
> 2007-10-18 17:43:58,772 INFO org.apache.hadoop.ipc.Client: Retrying connect
> to server: /127.0.0.1:53972. Already tried 4 time(s).
> 2007-10-18 17:43:59,774 INFO org.apache.hadoop.ipc.Client: Retrying connect
> to server: /127.0.0.1:53972. Already tried 5 time(s).
> 2007-10-18 17:44:00,776 INFO org.apache.hadoop.ipc.Client: Retrying connect
> to server: /127.0.0.1:53972. Already tried 6 time(s).
> 2007-10-18 17:44:01,778 INFO org.apache.hadoop.ipc.Client: Retrying connect
> to server: /127.0.0.1:53972. Already tried 7 time(s).
> 2007-10-18 17:44:02,780 INFO org.apache.hadoop.ipc.Client: Retrying connect
> to server: /127.0.0.1:53972. Already tried 8 time(s).
> 2007-10-18 17:44:03,783 INFO org.apache.hadoop.ipc.Client: Retrying connect
> to server: /127.0.0.1:53972. Already tried 9 time(s).
> 2007-10-18 17:44:04,785 INFO org.apache.hadoop.ipc.Client: Retrying connect
> to server: /127.0.0.1:53972. Already tried 10 time(s).
--
This message is automatically generated by JIRA.
-
You can reply to this email to add a comment to the issue online.