hadoop-common-user mailing list archives

Site index · List index
Message view « Date » · « Thread »
Top « Date » · « Thread »
From Matt K <matvey1...@gmail.com>
Subject Re: tasks stuck in UNASSIGNED state
Date Tue, 16 Jun 2015 16:18:40 GMT
I made a little bit of progress with investigation.

TaskTracker has 2 consumer threads to kick off tasks - one for mappers and
another for reducers. The mapper thread got stuck, so the consumer isn't
picking up any more tasks, and they are timing out after 10 minutes.

Looks like it got stuck here:
https://github.com/matvey14/cdh5.0.2/blob/master/src/hadoop-mapreduce1-project/src/mapred/org/apache/hadoop/mapred/TaskTracker.java#L2754

And here's the stack-trace. From that it looks like it got stuck listening
on a socket.

Thread 9335: (state = IN_NATIVE)

 - sun.nio.ch.EPollArrayWrapper.epollWait(long, int, long, int) @bci=0
(Compiled frame; information may be imprecise)

 - sun.nio.ch.EPollArrayWrapper.poll(long) @bci=18, line=269 (Compiled
frame)

 - sun.nio.ch.EPollSelectorImpl.doSelect(long) @bci=28, line=79
(Interpreted frame)

 - sun.nio.ch.SelectorImpl.lockAndDoSelect(long) @bci=37, line=87
(Interpreted frame)

 - sun.nio.ch.SelectorImpl.select(long) @bci=30, line=98 (Compiled frame)

 -
org.apache.hadoop.net.SocketIOWithTimeout$SelectorPool.select(java.nio.channels.SelectableChannel,
int, long) @bci=46, line=335 (Interpreted frame)

 - org.apache.hadoop.net.SocketIOWithTimeout.doIO(java.nio.ByteBuffer, int)
@bci=80, line=157 (Interpreted frame)

 - org.apache.hadoop.net.SocketInputStream.read(java.nio.ByteBuffer)
@bci=6, line=161 (Interpreted frame)

 -
org.apache.hadoop.hdfs.protocol.datatransfer.PacketReceiver.readChannelFully(java.nio.channels.ReadableByteChannel,
java.nio.ByteBuffer) @bci=9, line=258 (Interpreted frame)

 -
org.apache.hadoop.hdfs.protocol.datatransfer.PacketReceiver.doReadFully(java.nio.channels.ReadableByteChannel,
java.io.InputStream, java.nio.ByteBuffer) @bci=6, line=209 (Interpreted
frame)

 -
org.apache.hadoop.hdfs.protocol.datatransfer.PacketReceiver.doRead(java.nio.channels.ReadableByteChannel,
java.io.InputStream) @bci=293, line=171 (Interpreted frame)

 -
org.apache.hadoop.hdfs.protocol.datatransfer.PacketReceiver.receiveNextPacket(java.nio.channels.ReadableByteChannel)
@bci=3, line=102 (Interpreted frame)

 - org.apache.hadoop.hdfs.RemoteBlockReader2.read(byte[], int, int)
@bci=27, line=138 (Interpreted frame)

 -
org.apache.hadoop.hdfs.DFSInputStream$ByteArrayStrategy.doRead(org.apache.hadoop.hdfs.BlockReader,
int, int, org.apache.hadoop.hdfs.DFSInputStream$ReadStatistics) @bci=7,
line=686 (Interpreted frame)

 -
org.apache.hadoop.hdfs.DFSInputStream.readBuffer(org.apache.hadoop.hdfs.DFSInputStream$ReaderStrategy,
int, int, java.util.Map) @bci=14, line=742 (Interpreted frame)

 -
org.apache.hadoop.hdfs.DFSInputStream.readWithStrategy(org.apache.hadoop.hdfs.DFSInputStream$ReaderStrategy,
int, int) @bci=141, line=799 (Interpreted frame)

 - org.apache.hadoop.hdfs.DFSInputStream.read(byte[], int, int) @bci=15,
line=840 (Interpreted frame)

 - java.io.DataInputStream.read(byte[]) @bci=8, line=100 (Interpreted frame)

 - org.apache.hadoop.io.IOUtils.copyBytes(java.io.InputStream,
java.io.OutputStream, int) @bci=67, line=84 (Interpreted frame)

 - org.apache.hadoop.io.IOUtils.copyBytes(java.io.InputStream,
java.io.OutputStream, int, boolean) @bci=3, line=52 (Interpreted frame)

 - org.apache.hadoop.io.IOUtils.copyBytes(java.io.InputStream,
java.io.OutputStream, org.apache.hadoop.conf.Configuration, boolean)
@bci=12, line=112 (Interpreted frame)

 - org.apache.hadoop.fs.FileUtil.copy(org.apache.hadoop.fs.FileSystem,
org.apache.hadoop.fs.FileStatus, org.apache.hadoop.fs.FileSystem,
org.apache.hadoop.fs.Path, boolean, boolean,
org.apache.hadoop.conf.Configuration) @bci=138, line=366 (Interpreted frame)

 - org.apache.hadoop.fs.FileUtil.copy(org.apache.hadoop.fs.FileSystem,
org.apache.hadoop.fs.FileStatus, org.apache.hadoop.fs.FileSystem,
org.apache.hadoop.fs.Path, boolean, boolean,
org.apache.hadoop.conf.Configuration) @bci=95, line=356 (Interpreted frame)

 - org.apache.hadoop.fs.FileUtil.copy(org.apache.hadoop.fs.FileSystem,
org.apache.hadoop.fs.FileStatus, org.apache.hadoop.fs.FileSystem,
org.apache.hadoop.fs.Path, boolean, boolean,
org.apache.hadoop.conf.Configuration) @bci=95, line=356 (Interpreted frame)

 - org.apache.hadoop.fs.FileUtil.copy(org.apache.hadoop.fs.FileSystem,
org.apache.hadoop.fs.Path, org.apache.hadoop.fs.FileSystem,
org.apache.hadoop.fs.Path, boolean, boolean,
org.apache.hadoop.conf.Configuration) @bci=18, line=338 (Interpreted frame)

 - org.apache.hadoop.fs.FileUtil.copy(org.apache.hadoop.fs.FileSystem,
org.apache.hadoop.fs.Path, org.apache.hadoop.fs.FileSystem,
org.apache.hadoop.fs.Path, boolean, org.apache.hadoop.conf.Configuration)
@bci=9, line=289 (Interpreted frame)

 - org.apache.hadoop.fs.FileSystem.copyToLocalFile(boolean,
org.apache.hadoop.fs.Path, org.apache.hadoop.fs.Path, boolean) @bci=42,
line=1968 (Interpreted frame)

 - org.apache.hadoop.fs.FileSystem.copyToLocalFile(boolean,
org.apache.hadoop.fs.Path, org.apache.hadoop.fs.Path) @bci=5, line=1937
(Interpreted frame)

 -
org.apache.hadoop.fs.FileSystem.copyToLocalFile(org.apache.hadoop.fs.Path,
org.apache.hadoop.fs.Path) @bci=4, line=1913 (Interpreted frame)

 -
org.apache.hadoop.filecache.TrackerDistributedCacheManager.downloadCacheObject(org.apache.hadoop.conf.Configuration,
java.net.URI, org.apache.hadoop.fs.Path, long, boolean,
org.apache.hadoop.fs.permission.FsPermission) @bci=288, line=445
(Interpreted frame)

 -
org.apache.hadoop.mapred.JobLocalizer.downloadPrivateCacheObjects(org.apache.hadoop.conf.Configuration,
java.net.URI[], org.apache.hadoop.fs.Path[], long[], boolean[], boolean)
@bci=150, line=328 (Interpreted frame)

 -
org.apache.hadoop.mapred.JobLocalizer.downloadPrivateCache(org.apache.hadoop.conf.Configuration)
@bci=18, line=346 (Interpreted frame)

 -
org.apache.hadoop.mapred.JobLocalizer.localizeJobFiles(org.apache.hadoop.mapreduce.JobID,
org.apache.hadoop.mapred.JobConf, org.apache.hadoop.fs.Path,
org.apache.hadoop.fs.Path, org.apache.hadoop.mapred.TaskUmbilicalProtocol)
@bci=51, line=391 (Interpreted frame)

 -
org.apache.hadoop.mapred.JobLocalizer.localizeJobFiles(org.apache.hadoop.mapreduce.JobID,
org.apache.hadoop.mapred.JobConf, org.apache.hadoop.fs.Path,
org.apache.hadoop.mapred.TaskUmbilicalProtocol) @bci=21, line=370
(Interpreted frame)

 -
org.apache.hadoop.mapred.DefaultTaskController.initializeJob(java.lang.String,
java.lang.String, org.apache.hadoop.fs.Path, org.apache.hadoop.fs.Path,
org.apache.hadoop.mapred.TaskUmbilicalProtocol, java.net.InetSocketAddress)
@bci=115, line=231 (Interpreted frame)

 - org.apache.hadoop.mapred.TaskTracker$4.run() @bci=163, line=1472
(Interpreted frame)

 -
java.security.AccessController.doPrivileged(java.security.PrivilegedExceptionAction,
java.security.AccessControlContext) @bci=0 (Interpreted frame)

 - javax.security.auth.Subject.doAs(javax.security.auth.Subject,
java.security.PrivilegedExceptionAction) @bci=42, line=415 (Interpreted
frame)

 -
org.apache.hadoop.security.UserGroupInformation.doAs(java.security.PrivilegedExceptionAction)
@bci=14, line=1548 (Interpreted frame)

 -
org.apache.hadoop.mapred.TaskTracker.initializeJob(org.apache.hadoop.mapred.Task,
org.apache.hadoop.mapred.TaskTracker$RunningJob,
java.net.InetSocketAddress) @bci=215, line=1447 (Interpreted frame)

 -
org.apache.hadoop.mapred.TaskTracker.localizeJob(org.apache.hadoop.mapred.TaskTracker$TaskInProgress)
@bci=96, line=1362 (Interpreted frame)

 -
org.apache.hadoop.mapred.TaskTracker.startNewTask(org.apache.hadoop.mapred.TaskTracker$TaskInProgress)
@bci=2, line=2790 (Interpreted frame)

 - org.apache.hadoop.mapred.TaskTracker$TaskLauncher.run() @bci=491,
line=2754 (Interpreted frame)
There aren't any tasks running on that node when the thread-dump was taken,
so I'm pretty sure it's stuck here. However, there's a socket timeout of 10
seconds, so I don't understand why it didn't time out.


On Tue, Jun 16, 2015 at 7:51 AM, Matt K <matvey1414@gmail.com> wrote:

> Hi Nat, I should've mentioned this before. We're running MRv1.
>
> On Tue, Jun 16, 2015 at 2:24 AM, nataraj jonnalagadda <
> nataraj.jonnalagadda@gmail.com> wrote:
>
>> Hey Matt,
>>
>> Its possibly due to your YARN config... Possibly, YARN/Mapred ACLs / YARN
>> scheduler config or Cgroups not (incase enabled) set up not correctly. We
>> could dig in more if we have the yarn-site.xml and scheduler conf files.
>>
>>
>> Thanks,
>> Nat.
>>
>>
>>
>> On Mon, Jun 15, 2015 at 10:39 PM, Matt K <matvey1414@gmail.com> wrote:
>>
>>> I see there's 2 threads - one that kicks off the mappers, and another
>>> that kicks off reducers. The one that kicks off the mappers got stuck. It's
>>> not yet clear to me where it got stuck exactly.
>>>
>>> On Tue, Jun 16, 2015 at 1:11 AM, Matt K <matvey1414@gmail.com> wrote:
>>>
>>>> Hi all,
>>>>
>>>> I'm dealing with a production issue, any help would be appreciated. I
>>>> am seeing very strange behavior in the TaskTrackers. After they pick up the
>>>> task, it never comes out of the UNASSIGNED state, and the task just gets
>>>> killed 10 minutes later.
>>>>
>>>> 2015-06-16 02:42:21,114 INFO org.apache.hadoop.mapred.TaskTracker:
>>>> LaunchTaskAction (registerTask): attempt_201506152116_0046_m_000286_0
>>>> task's state:UNASSIGNED
>>>> 2015-06-16 02:52:21,805 INFO org.apache.hadoop.mapred.TaskTracker:
>>>> attempt_201506152116_0046_m_000286_0: Task
>>>> attempt_201506152116_0046_m_000286_0 failed to report status for 600
>>>> seconds. Killing!
>>>>
>>>> Normally, I would see the following in the logs:
>>>>
>>>> 2015-06-16 04:30:32,328 INFO org.apache.hadoop.mapred.TaskTracker:
>>>> Trying to launch : attempt_201506152116_0062_r_000004_0 which needs 1 slots
>>>>
>>>> However, it doesn't get this far for these particular tasks. I am
>>>> perusing the source code here, and this doesn't seem to be possible:
>>>>
>>>> http://grepcode.com/file/repository.cloudera.com/content/repositories/releases/com.cloudera.hadoop/hadoop-core/0.20.2-737/org/apache/hadoop/mapred/TaskTracker.java#TaskTracker.TaskLauncher.0tasksToLaunch
>>>>
>>>> The code does something like this:
>>>>
>>>>     public void addToTaskQueue(LaunchTaskAction action) {
>>>>       synchronized (tasksToLaunch) {
>>>>         TaskInProgress tip = registerTask(action, this);
>>>>         tasksToLaunch.add(tip);
>>>>         tasksToLaunch.notifyAll();
>>>>       }
>>>>     }
>>>>
>>>> The following should pick it up:
>>>>
>>>>     public void run() {
>>>>       while (!Thread.interrupted()) {
>>>>         try {
>>>>           TaskInProgress tip;
>>>>           Task task;
>>>>           synchronized (tasksToLaunch) {
>>>>             while (tasksToLaunch.isEmpty()) {
>>>>               tasksToLaunch.wait();
>>>>             }
>>>>             //get the TIP
>>>>             tip = tasksToLaunch.remove(0);
>>>>             task = tip.getTask();
>>>>             LOG.info("Trying to launch : " + tip.getTask().getTaskID() +
>>>>                      " which needs " + task.getNumSlotsRequired() + " slots");
>>>>           }
>>>>
>>>> What's even stranger is that this is happening for Map tasks only. Reduce
tasks are fine.
>>>>
>>>> This is only happening on a handful of the nodes, but enough to either slow
down jobs or cause them to fail.
>>>>
>>>> We're running Hadoop 2.3.0-cdh5.0.2
>>>>
>>>> Thanks,
>>>>
>>>> -Matt
>>>>
>>>>
>>>
>>>
>>> --
>>> www.calcmachine.com - easy online calculator.
>>>
>>
>>
>
>
> --
> www.calcmachine.com - easy online calculator.
>



-- 
www.calcmachine.com - easy online calculator.

Mime
View raw message