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

Jason Lowe commented on MAPREDUCE-4062:
---------------------------------------

We've seen an issue where using both an RPC-level timeout (in this case the 
ping timeout) and the timer task can cause the AM to lose track of a container 
and hang the job.  Here's the relevant part of the AM log:

{noformat}
2012-03-29 07:32:17,794 ERROR [ContainerLauncher #199] 
org.apache.hadoop.security.UserGroupInformation: PriviledgedActionException 
as:container_1333003059741_0010_01_003408 (auth:SIMPLE) 
cause:java.net.SocketTimeoutException: 60000 millis timeout while waiting for 
channel to be ready for read. ch : java.nio.channels.SocketChannel[connected 
local=/x.x.x.x:x remote=x.x.x.x.x/x.x.x.x:x]
2012-03-29 07:32:17,794 WARN [ContainerLauncher #199] 
org.apache.hadoop.ipc.Client: Exception encountered while connecting to the 
server : java.net.SocketTimeoutException: 60000 millis timeout while waiting 
for channel to be ready for read. ch : 
java.nio.channels.SocketChannel[connected local=/x.x.x.x:x 
remote=x.x.x.x.x/x.x.x.x:x]
2012-03-29 07:32:17,794 ERROR [ContainerLauncher #199] 
org.apache.hadoop.security.UserGroupInformation: PriviledgedActionException 
as:container_1333003059741_0010_01_003408 (auth:SIMPLE) 
cause:java.io.IOException: java.net.SocketTimeoutException: 60000 millis 
timeout while waiting for channel to be ready for read. ch : 
java.nio.channels.SocketChannel[connected local=/x.x.x.x:x 
remote=x.x.x.x.x/x.x.x.x:x]
2012-03-29 07:32:17,795 WARN [Timer-1] 
org.apache.hadoop.mapreduce.v2.app.launcher.ContainerLauncherImpl: Couldn't 
complete CONTAINER_REMOTE_CLEANUP on 
container_1333003059741_0010_01_003408/attempt_1333003059741_0010_m_003097_0. 
Interrupting and returning
2012-03-29 07:32:17,798 INFO [Timer-1] 
org.apache.hadoop.mapreduce.v2.app.launcher.ContainerLauncherImpl: Stack trace 
of the command-thread: 

        at java.util.Arrays.copyOf(Arrays.java:2882)
        at 
java.lang.AbstractStringBuilder.expandCapacity(AbstractStringBuilder.java:100)
        at 
java.lang.AbstractStringBuilder.append(AbstractStringBuilder.java:390)
        at java.lang.StringBuilder.append(StringBuilder.java:119)
        at java.lang.StackTraceElement.toString(StackTraceElement.java:157)
        at java.lang.String.valueOf(String.java:2826)
        at java.lang.StringBuilder.append(StringBuilder.java:115)
        at java.lang.Throwable.printStackTrace(Throwable.java:512)
        at 
org.apache.hadoop.util.StringUtils.stringifyException(StringUtils.java:64)
        at 
org.apache.hadoop.mapreduce.v2.app.launcher.ContainerLauncherImpl$Container.kill(ContainerLauncherImpl.java:260)
        at 
org.apache.hadoop.mapreduce.v2.app.launcher.ContainerLauncherImpl$EventProcessor.run(ContainerLauncherImpl.java:479)
        at 
java.util.concurrent.ThreadPoolExecutor$Worker.runTask(ThreadPoolExecutor.java:886)
        at 
java.util.concurrent.ThreadPoolExecutor$Worker.run(ThreadPoolExecutor.java:908)
        at java.lang.Thread.run(Thread.java:619)
2012-03-29 07:32:17,800 WARN [ContainerLauncher #199] 
org.apache.hadoop.yarn.event.AsyncDispatcher: AsyncDispatcher thread interrupted
java.lang.InterruptedException
        at 
java.util.concurrent.locks.AbstractQueuedSynchronizer.acquireInterruptibly(AbstractQueuedSynchronizer.java:1199)
        at 
java.util.concurrent.locks.ReentrantLock.lockInterruptibly(ReentrantLock.java:312)
        at 
java.util.concurrent.LinkedBlockingQueue.put(LinkedBlockingQueue.java:294)
        at 
org.apache.hadoop.yarn.event.AsyncDispatcher$GenericEventHandler.handle(AsyncDispatcher.java:179)
        at 
org.apache.hadoop.mapreduce.v2.app.launcher.ContainerLauncherImpl$Container.kill(ContainerLauncherImpl.java:263)
        at 
org.apache.hadoop.mapreduce.v2.app.launcher.ContainerLauncherImpl$EventProcessor.run(ContainerLauncherImpl.java:479)
        at 
java.util.concurrent.ThreadPoolExecutor$Worker.runTask(ThreadPoolExecutor.java:886)
        at 
java.util.concurrent.ThreadPoolExecutor$Worker.run(ThreadPoolExecutor.java:908)
        at java.lang.Thread.run(Thread.java:619)
{noformat}

Looks like the socket timeout and the timer task timeout occurred almost 
simultaneously.  The socket exception was caught first, and during the catch 
clause we fielded the interrupted exception.  That broke us out of the handling 
of the socket exception and we never marked the container status properly 
before leaving.

So I'm +1 on removing the timer task and relying on the rpcTimeout, unless 
there are other cases besides RPC where we'd expect the container launcher to 
get stuck.
                
> AM Launcher thread can hang forever
> -----------------------------------
>
>                 Key: MAPREDUCE-4062
>                 URL: https://issues.apache.org/jira/browse/MAPREDUCE-4062
>             Project: Hadoop Map/Reduce
>          Issue Type: Bug
>          Components: mrv2
>    Affects Versions: 0.23.2
>            Reporter: Thomas Graves
>            Assignee: Thomas Graves
>
> We saw an instance where the RM stopped launch Application masters.  We found 
> that the launcher thread was hung because something weird/bad happened to the 
> NM node. Currently there is only 1 launcher thread (jira 4061 to fix that). 
> We need this to not happen.  Even once we increase the number of threads  to 
> > 1 if that many nodes go bad the RM would be stuck.  Note that this was 
> stuck like this for approximately 9 hours.
> Stack trace on hung AM launcher:
> "pool-1-thread-1" prio=10 tid=0x000000004343e800 nid=0x3a4c in Object.wait()
> [0x000000004fad2000]
>    java.lang.Thread.State: WAITING (on object monitor)
>     at java.lang.Object.wait(Native Method)
>     at java.lang.Object.wait(Object.java:485)
>     at org.apache.hadoop.ipc.Client.call(Client.java:1076)
>     - locked <0x00002aab05a4f3f0> (a org.apache.hadoop.ipc.Client$Call)
>     at
> org.apache.hadoop.yarn.ipc.ProtoOverHadoopRpcEngine$Invoker.invoke(ProtoOverHadoopRpcEngine.java:135)
>     at $Proxy76.startContainer(Unknown Source)
>     at
> org.apache.hadoop.yarn.api.impl.pb.client.ContainerManagerPBClientImpl.startContainer(ContainerManagerPBClientImpl.java:87)
>     at
> org.apache.hadoop.yarn.server.resourcemanager.amlauncher.AMLauncher.launch(AMLauncher.java:118)
>     at
> org.apache.hadoop.yarn.server.resourcemanager.amlauncher.AMLauncher.run(AMLauncher.java:265)
>     at
> java.util.concurrent.ThreadPoolExecutor$Worker.runTask(ThreadPoolExecutor.java:886)
>     at
> java.util.concurrent.ThreadPoolExecutor$Worker.run(ThreadPoolExecutor.java:908)
>     at java.lang.Thread.run(Thread.java:619)

--
This message is automatically generated by JIRA.
If you think it was sent incorrectly, please contact your JIRA administrators: 
https://issues.apache.org/jira/secure/ContactAdministrators!default.jspa
For more information on JIRA, see: http://www.atlassian.com/software/jira

        

Reply via email to