Browse Source

MAPREDUCE-6293. Set job classloader on uber-job's LocalContainerLauncher event thread. (Sangjin Lee via gera)

Gera Shegalov 10 years ago
parent
commit
725eb52ddc

+ 3 - 0
hadoop-mapreduce-project/CHANGES.txt

@@ -337,6 +337,9 @@ Release 2.8.0 - UNRELEASED
     MAPREDUCE-6238. MR2 can't run local jobs with -libjars command options
     which is a regression from MR1 (zxu via rkanter)
 
+    MAPREDUCE-6293. Set job classloader on uber-job's LocalContainerLauncher
+    event thread. (Sangjin Lee via gera)
+
 Release 2.7.1 - UNRELEASED
 
   INCOMPATIBLE CHANGES

+ 20 - 0
hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-app/src/main/java/org/apache/hadoop/mapred/LocalContainerLauncher.java

@@ -80,6 +80,7 @@ public class LocalContainerLauncher extends AbstractService implements
   private final HashSet<File> localizedFiles;
   private final AppContext context;
   private final TaskUmbilicalProtocol umbilical;
+  private final ClassLoader jobClassLoader;
   private ExecutorService taskRunner;
   private Thread eventHandler;
   private BlockingQueue<ContainerLauncherEvent> eventQueue =
@@ -87,6 +88,12 @@ public class LocalContainerLauncher extends AbstractService implements
 
   public LocalContainerLauncher(AppContext context,
                                 TaskUmbilicalProtocol umbilical) {
+    this(context, umbilical, null);
+  }
+
+  public LocalContainerLauncher(AppContext context,
+                                TaskUmbilicalProtocol umbilical,
+                                ClassLoader jobClassLoader) {
     super(LocalContainerLauncher.class.getName());
     this.context = context;
     this.umbilical = umbilical;
@@ -94,6 +101,7 @@ public class LocalContainerLauncher extends AbstractService implements
         // (TODO/FIXME:  pointless to use RPC to talk to self; should create
         // LocalTaskAttemptListener or similar:  implement umbilical protocol
         // but skip RPC stuff)
+    this.jobClassLoader = jobClassLoader;
 
     try {
       curFC = FileContext.getFileContext(curDir.toURI());
@@ -133,6 +141,18 @@ public class LocalContainerLauncher extends AbstractService implements
             setDaemon(true).setNameFormat("uber-SubtaskRunner").build());
     // create and start an event handling thread
     eventHandler = new Thread(new EventHandler(), "uber-EventHandler");
+    // if the job classloader is specified, set it onto the event handler as the
+    // thread context classloader so that it can be used by the event handler
+    // as well as the subtask runner threads
+    if (jobClassLoader != null) {
+      LOG.info("Setting " + jobClassLoader +
+          " as the context classloader of thread " + eventHandler.getName());
+      eventHandler.setContextClassLoader(jobClassLoader);
+    } else {
+      // note the current TCCL
+      LOG.info("Context classloader of thread " + eventHandler.getName() +
+          ": " + eventHandler.getContextClassLoader());
+    }
     eventHandler.start();
     super.serviceStart();
   }

+ 1 - 1
hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-app/src/main/java/org/apache/hadoop/mapreduce/v2/app/MRAppMaster.java

@@ -889,7 +889,7 @@ public class MRAppMaster extends CompositeService {
     protected void serviceStart() throws Exception {
       if (job.isUber()) {
         this.containerLauncher = new LocalContainerLauncher(context,
-            (TaskUmbilicalProtocol) taskAttemptListener);
+            (TaskUmbilicalProtocol) taskAttemptListener, jobClassLoader);
       } else {
         this.containerLauncher = new ContainerLauncherImpl(context);
       }

+ 1 - 1
hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-common/src/main/java/org/apache/hadoop/mapreduce/v2/util/MRApps.java

@@ -378,7 +378,7 @@ public class MRApps extends Apps {
   public static void setClassLoader(ClassLoader classLoader,
       Configuration conf) {
     if (classLoader != null) {
-      LOG.info("Setting classloader " + classLoader.getClass().getName() +
+      LOG.info("Setting classloader " + classLoader +
           " on the configuration and as the thread context classloader");
       conf.setClassLoader(classLoader);
       Thread.currentThread().setContextClassLoader(classLoader);

+ 9 - 0
hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/test/java/org/apache/hadoop/mapreduce/v2/TestMRJobs.java

@@ -997,6 +997,15 @@ public class TestMRJobs {
         throws IOException, InterruptedException {
       super.setup(context);
       final Configuration conf = context.getConfiguration();
+      // check if the job classloader is enabled and verify the TCCL
+      if (conf.getBoolean(MRJobConfig.MAPREDUCE_JOB_CLASSLOADER, false)) {
+        ClassLoader tccl = Thread.currentThread().getContextClassLoader();
+        if (!(tccl instanceof ApplicationClassLoader)) {
+          throw new IOException("TCCL expected: " +
+              ApplicationClassLoader.class.getName() + ", actual: " +
+              tccl.getClass().getName());
+        }
+      }
       final String ioSortMb = conf.get(MRJobConfig.IO_SORT_MB);
       if (!TEST_IO_SORT_MB.equals(ioSortMb)) {
         throw new IOException("io.sort.mb expected: " + TEST_IO_SORT_MB