[jira] [Commented] (MAPREDUCE-5388) There are two TaskLaunchers for map after the TaskTracker reinit

2013-07-14 Thread Hua xu (JIRA)

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

Hua xu commented on MAPREDUCE-5388:
---

We fix that:
{code}
private static class PathCleanupThread extends Thread {

// cleanup queue which deletes files/directories of the paths queued up.
private LinkedBlockingQueue queue =
  new LinkedBlockingQueue();

public PathCleanupThread() {
  setName("Directory/File cleanup thread");
  setDaemon(true);
  start();
}

void addToQueue(PathDeletionContext[] contexts) {
+ boolean interrupted = false;
  for (PathDeletionContext context : contexts) {
try {
  queue.put(context);
} catch(InterruptedException ie) {
+ interrupted = true;
}
  }

+ if(interrupted){
+ Thread.currentThread().interrupt();
+ }
}

public void run() {
  if (LOG.isDebugEnabled()) {
LOG.debug(getName() + " started.");
  }
  PathDeletionContext context = null;
  while (true) {
try {
  context = queue.take();
  // delete the path.
  if (!deletePath(context)) {
LOG.warn("CleanupThread:Unable to delete path " + context.fullPath);
  }
  else if (LOG.isDebugEnabled()) {
LOG.debug("DELETED " + context.fullPath);
  }
} catch (InterruptedException t) {
  LOG.warn("Interrupted deletion of " + context.fullPath);
  return;
} catch (Exception e) {
  LOG.warn("Error deleting path " + context.fullPath + ": " + e);
}
  }
}
  }
{code}

> There are two TaskLaunchers for map after the TaskTracker reinit
> 
>
> Key: MAPREDUCE-5388
> URL: https://issues.apache.org/jira/browse/MAPREDUCE-5388
> Project: Hadoop Map/Reduce
>  Issue Type: Bug
>  Components: tasktracker
>Affects Versions: 0.21.0, 0.22.0, 1.0.3
>Reporter: Hua xu
>
> The TaskTracker was asked to reinit as the full gc,then the process contained 
> of two TaskLauncher instance for map:
> {code}
> 2013-06-22 08:32:59,457 INFO org.apache.hadoop.mapred.TaskTracker: Process 
> Thread Dump: lost task
> 36 active threads
> ..
> Thread 43 (TaskLauncher for REDUCE tasks):
>   State: WAITING
>   Blocked count: 73824
>   Waited count: 72848
>   Waiting on java.util.LinkedList@98a9557
>   Stack:
> java.lang.Object.wait(Native Method)
> java.lang.Object.wait(Object.java:485)
> 
> org.apache.hadoop.mapred.TaskTracker$TaskLauncher.run(TaskTracker.java:2157)
> Thread 42 (TaskLauncher for MAP tasks):
>   State: WAITING
>   Blocked count: 358814
>   Waited count: 356105
>   Waiting on org.apache.hadoop.ipc.Client$Call@6f764bc1
>   Stack:
> java.lang.Object.wait(Native Method)
> java.lang.Object.wait(Object.java:485)
> org.apache.hadoop.ipc.Client.call(Client.java:891)
> 
> org.apache.hadoop.ipc.WritableRpcEngine$Invoker.invoke(WritableRpcEngine.java:198)
> $Proxy5.getFileInfo(Unknown Source)
> sun.reflect.GeneratedMethodAccessor4.invoke(Unknown Source)
> 
> sun.reflect.DelegatingMethodAccessorImpl.invoke(DelegatingMethodAccessorImpl.java:25)
> java.lang.reflect.Method.invoke(Method.java:597)
> 
> org.apache.hadoop.io.retry.RetryInvocationHandler.invokeMethod(RetryInvocationHandler.java:82)
> 
> org.apache.hadoop.io.retry.RetryInvocationHandler.invoke(RetryInvocationHandler.java:59)
> $Proxy5.getFileInfo(Unknown Source)
> org.apache.hadoop.hdfs.DFSClient.getFileInfo(DFSClient.java:850)
> 
> org.apache.hadoop.hdfs.DistributedFileSystem.getFileStatus(DistributedFileSystem.java:620)
> org.apache.hadoop.fs.FileUtil.copy(FileUtil.java:215)
> org.apache.hadoop.fs.FileUtil.copy(FileUtil.java:167)
> org.apache.hadoop.fs.FileSystem.copyToLocalFile(FileSystem.java:1543)
> org.apache.hadoop.fs.FileSystem.copyToLocalFile(FileSystem.java:1524)
> 
> org.apache.hadoop.mapred.TaskTracker.localizeJobTokenFile(TaskTracker.java:3995)
> 
> org.apache.hadoop.mapred.TaskTracker.localizeJobFiles(TaskTracker.java:1036)
> org.apache.hadoop.mapred.TaskTracker.localizeJob(TaskTracker.java:977)
> ..
> Thread 1 (main):
>   State: RUNNABLE
>   Stack:
> sun.management.ThreadImpl.getThreadInfo1(Native Method)
> sun.management.ThreadImpl.getThreadInfo(ThreadImpl.java:156)
> sun.management.ThreadImpl.getThreadInfo(ThreadImpl.java:121)
> 
> org.apache.hadoop.util.ReflectionUtils.printThreadInfo(ReflectionUtils.java:157)
> 
> org.apache.hadoop.util.ReflectionUtils.logThreadInfo(ReflectionUtils.java:211)
> 
> org.apache.hadoop.mapred.TaskTracker.markUnresponsiveTasks(TaskTracker.java:1792)
> org.apache.hadoop.mapred.TaskTracker.offerService(TaskTracker.java:1468)
>

[jira] [Commented] (MAPREDUCE-5388) There are two TaskLaunchers for map after the TaskTracker reinit

2013-07-14 Thread Hua xu (JIRA)

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

Hua xu commented on MAPREDUCE-5388:
---

from the logs the TaskTracker produced when it war reinitializing,  the stack 
of method is that:
{code}
+org.apache.hadoop.mapred.TaskTracker$TaskLauncher.run
  -org.apache.hadoop.mapred.TaskTracker.startNewTask
   -org.apache.hadoop.mapred.TaskTracker.localizeJob   
-org.apache.hadoop.mapred.TaskTracker.launchTaskForJob
   -org.apache.hadoop.mapred.TaskTracker.TaskInProgress.kill
   -org.apache.hadoop.mapred.TaskTracker.TaskInProgress.cleanup
-org.apache.hadoop.mapred.TaskTracker.TaskInProgress.removeTaskFiles
 -org.apache.hadoop.mapred.CleanupQueue.addToQueue
  -org.apache.hadoop.mapred.CleanupQueue$PathCleanupThread.addToQueue
{code} 
the source code is:
{code}
private static class PathCleanupThread extends Thread {

// cleanup queue which deletes files/directories of the paths queued up.
private LinkedBlockingQueue queue =
  new LinkedBlockingQueue();

public PathCleanupThread() {
  setName("Directory/File cleanup thread");
  setDaemon(true);
  start();
}

void addToQueue(PathDeletionContext[] contexts) {
  for (PathDeletionContext context : contexts) {
try {
  queue.put(context);
} catch(InterruptedException ie) {}
  }
}

public void run() {
  if (LOG.isDebugEnabled()) {
LOG.debug(getName() + " started.");
  }
  PathDeletionContext context = null;
  while (true) {
try {
  context = queue.take();
  context.deletePath();
  // delete the path.
  if (LOG.isDebugEnabled()) {
LOG.debug("DELETED " + context);
  }
} catch (InterruptedException t) {
  LOG.warn("Interrupted deletion of " + context);
  return;
} catch (Throwable e) {
  LOG.warn("Error deleting path " + context, e);
} 
  }
}
  }
{code}
Maybe, the map TaskLauncher thread was interrupted when it was invoking the 
method-addToQueue, so that the interruption was processed and the TaskLauncher 
could keep running.

> There are two TaskLaunchers for map after the TaskTracker reinit
> 
>
> Key: MAPREDUCE-5388
> URL: https://issues.apache.org/jira/browse/MAPREDUCE-5388
> Project: Hadoop Map/Reduce
>  Issue Type: Bug
>  Components: tasktracker
>Affects Versions: 0.21.0, 0.22.0, 1.0.3
>Reporter: Hua xu
>
> The TaskTracker was asked to reinit as the full gc,then the process contained 
> of two TaskLauncher instance for map:
> {code}
> 2013-06-22 08:32:59,457 INFO org.apache.hadoop.mapred.TaskTracker: Process 
> Thread Dump: lost task
> 36 active threads
> ..
> Thread 43 (TaskLauncher for REDUCE tasks):
>   State: WAITING
>   Blocked count: 73824
>   Waited count: 72848
>   Waiting on java.util.LinkedList@98a9557
>   Stack:
> java.lang.Object.wait(Native Method)
> java.lang.Object.wait(Object.java:485)
> 
> org.apache.hadoop.mapred.TaskTracker$TaskLauncher.run(TaskTracker.java:2157)
> Thread 42 (TaskLauncher for MAP tasks):
>   State: WAITING
>   Blocked count: 358814
>   Waited count: 356105
>   Waiting on org.apache.hadoop.ipc.Client$Call@6f764bc1
>   Stack:
> java.lang.Object.wait(Native Method)
> java.lang.Object.wait(Object.java:485)
> org.apache.hadoop.ipc.Client.call(Client.java:891)
> 
> org.apache.hadoop.ipc.WritableRpcEngine$Invoker.invoke(WritableRpcEngine.java:198)
> $Proxy5.getFileInfo(Unknown Source)
> sun.reflect.GeneratedMethodAccessor4.invoke(Unknown Source)
> 
> sun.reflect.DelegatingMethodAccessorImpl.invoke(DelegatingMethodAccessorImpl.java:25)
> java.lang.reflect.Method.invoke(Method.java:597)
> 
> org.apache.hadoop.io.retry.RetryInvocationHandler.invokeMethod(RetryInvocationHandler.java:82)
> 
> org.apache.hadoop.io.retry.RetryInvocationHandler.invoke(RetryInvocationHandler.java:59)
> $Proxy5.getFileInfo(Unknown Source)
> org.apache.hadoop.hdfs.DFSClient.getFileInfo(DFSClient.java:850)
> 
> org.apache.hadoop.hdfs.DistributedFileSystem.getFileStatus(DistributedFileSystem.java:620)
> org.apache.hadoop.fs.FileUtil.copy(FileUtil.java:215)
> org.apache.hadoop.fs.FileUtil.copy(FileUtil.java:167)
> org.apache.hadoop.fs.FileSystem.copyToLocalFile(FileSystem.java:1543)
> org.apache.hadoop.fs.FileSystem.copyToLocalFile(FileSystem.java:1524)
> 
> org.apache.hadoop.mapred.TaskTracker.localizeJobTokenFile(TaskTracker.java:3995)
> 
> org.apache.hadoop.mapred.TaskTracker.localizeJobFiles(TaskTracker.java:1036)
> org.apache.hadoop.mapred.TaskTracker.localizeJob(TaskTracker.java:977)
> ..
> Thread 1 (main):
>   State: RUNNABLE
>  

[jira] [Created] (MAPREDUCE-5388) There are two TaskLaunchers for map after the TaskTracker reinit

2013-07-14 Thread Hua xu (JIRA)
Hua xu created MAPREDUCE-5388:
-

 Summary: There are two TaskLaunchers for map after the TaskTracker 
reinit
 Key: MAPREDUCE-5388
 URL: https://issues.apache.org/jira/browse/MAPREDUCE-5388
 Project: Hadoop Map/Reduce
  Issue Type: Bug
  Components: tasktracker
Affects Versions: 1.0.3, 0.22.0, 0.21.0
Reporter: Hua xu


The TaskTracker was asked to reinit as the full gc,then the process contained 
of two TaskLauncher instance for map:
{code}


2013-06-22 08:32:59,457 INFO org.apache.hadoop.mapred.TaskTracker: Process 
Thread Dump: lost task
36 active threads

..

Thread 43 (TaskLauncher for REDUCE tasks):
  State: WAITING
  Blocked count: 73824
  Waited count: 72848
  Waiting on java.util.LinkedList@98a9557
  Stack:
java.lang.Object.wait(Native Method)
java.lang.Object.wait(Object.java:485)
org.apache.hadoop.mapred.TaskTracker$TaskLauncher.run(TaskTracker.java:2157)
Thread 42 (TaskLauncher for MAP tasks):
  State: WAITING
  Blocked count: 358814
  Waited count: 356105
  Waiting on org.apache.hadoop.ipc.Client$Call@6f764bc1
  Stack:
java.lang.Object.wait(Native Method)
java.lang.Object.wait(Object.java:485)
org.apache.hadoop.ipc.Client.call(Client.java:891)

org.apache.hadoop.ipc.WritableRpcEngine$Invoker.invoke(WritableRpcEngine.java:198)
$Proxy5.getFileInfo(Unknown Source)
sun.reflect.GeneratedMethodAccessor4.invoke(Unknown Source)

sun.reflect.DelegatingMethodAccessorImpl.invoke(DelegatingMethodAccessorImpl.java:25)
java.lang.reflect.Method.invoke(Method.java:597)

org.apache.hadoop.io.retry.RetryInvocationHandler.invokeMethod(RetryInvocationHandler.java:82)

org.apache.hadoop.io.retry.RetryInvocationHandler.invoke(RetryInvocationHandler.java:59)
$Proxy5.getFileInfo(Unknown Source)
org.apache.hadoop.hdfs.DFSClient.getFileInfo(DFSClient.java:850)

org.apache.hadoop.hdfs.DistributedFileSystem.getFileStatus(DistributedFileSystem.java:620)
org.apache.hadoop.fs.FileUtil.copy(FileUtil.java:215)
org.apache.hadoop.fs.FileUtil.copy(FileUtil.java:167)
org.apache.hadoop.fs.FileSystem.copyToLocalFile(FileSystem.java:1543)
org.apache.hadoop.fs.FileSystem.copyToLocalFile(FileSystem.java:1524)

org.apache.hadoop.mapred.TaskTracker.localizeJobTokenFile(TaskTracker.java:3995)
org.apache.hadoop.mapred.TaskTracker.localizeJobFiles(TaskTracker.java:1036)
org.apache.hadoop.mapred.TaskTracker.localizeJob(TaskTracker.java:977)

..

Thread 1 (main):
  State: RUNNABLE
  Stack:
sun.management.ThreadImpl.getThreadInfo1(Native Method)
sun.management.ThreadImpl.getThreadInfo(ThreadImpl.java:156)
sun.management.ThreadImpl.getThreadInfo(ThreadImpl.java:121)

org.apache.hadoop.util.ReflectionUtils.printThreadInfo(ReflectionUtils.java:157)

org.apache.hadoop.util.ReflectionUtils.logThreadInfo(ReflectionUtils.java:211)

org.apache.hadoop.mapred.TaskTracker.markUnresponsiveTasks(TaskTracker.java:1792)
org.apache.hadoop.mapred.TaskTracker.offerService(TaskTracker.java:1468)
org.apache.hadoop.mapred.TaskTracker.run(TaskTracker.java:2329)
org.apache.hadoop.mapred.TaskTracker.main(TaskTracker.java:3461)

{code}

--
This message is automatically generated by JIRA.
If you think it was sent incorrectly, please contact your JIRA administrators
For more information on JIRA, see: http://www.atlassian.com/software/jira


[jira] [Assigned] (MAPREDUCE-4192) the TaskMemoryManager thread is not interrupt when the TaskTracker is oedered to reinit by JobTracker

2013-07-02 Thread Hua xu (JIRA)

 [ 
https://issues.apache.org/jira/browse/MAPREDUCE-4192?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel
 ]

Hua xu reassigned MAPREDUCE-4192:
-

Assignee: Hua xu

> the TaskMemoryManager thread is not interrupt when the TaskTracker is oedered 
> to reinit by JobTracker
> -
>
> Key: MAPREDUCE-4192
> URL: https://issues.apache.org/jira/browse/MAPREDUCE-4192
> Project: Hadoop Map/Reduce
>  Issue Type: Bug
>Affects Versions: 0.20.2
>Reporter: Hua xu
>Assignee: Hua xu
>
> When the TaskTracker is oedered to reinit by JobTracker, it will interrupt 
> some threads and then reinit them, but TaskTracker does not interrupt  
> TaskMemoryManager thread and create a new TaskMemoryManager thread again. I 
> use the tool--jstack to find that(I reinit TaskTracker 3 times through 
> JobTracker send TaskTrackerAction.ActionType.REINIT_TRACKER).

--
This message is automatically generated by JIRA.
If you think it was sent incorrectly, please contact your JIRA administrators
For more information on JIRA, see: http://www.atlassian.com/software/jira