瀏覽代碼

MAPREDUCE-4451. fairscheduler fail to init job with kerberos authentication configured. (erik.fang via tucu)

git-svn-id: https://svn.apache.org/repos/asf/hadoop/common/branches/branch-1@1397330 13f79535-47bb-0310-9956-ffa450edef68
Alejandro Abdelnur 12 年之前
父節點
當前提交
a20e68161e
共有 2 個文件被更改,包括 14 次插入4 次删除
  1. 3 0
      CHANGES.txt
  2. 11 4
      src/contrib/fairscheduler/src/java/org/apache/hadoop/mapred/FairScheduler.java

+ 3 - 0
CHANGES.txt

@@ -243,6 +243,9 @@ Release 1.2.0 - unreleased
     webhdfs filesystem and fsck to fail when security is on.
     (Arpit Gupta via suresh)
 
+    MAPREDUCE-4451. fairscheduler fail to init job with kerberos authentication 
+    configured. (erik.fang via tucu)
+
 Release 1.1.0 - unreleased
 
   INCOMPATIBLE CHANGES

+ 11 - 4
src/contrib/fairscheduler/src/java/org/apache/hadoop/mapred/FairScheduler.java

@@ -28,8 +28,8 @@ import java.util.HashSet;
 import java.util.List;
 import java.util.Map;
 import java.util.Set;
-import java.util.concurrent.ExecutorService;
-import java.util.concurrent.Executors;
+import java.util.concurrent.LinkedBlockingQueue;
+import java.util.concurrent.ThreadPoolExecutor;
 import java.util.concurrent.TimeUnit;
 
 import org.apache.commons.logging.Log;
@@ -275,12 +275,19 @@ public class FairScheduler extends TaskScheduler {
 
   private class JobInitializer {
     private final int DEFAULT_NUM_THREADS = 1;
-    private ExecutorService threadPool;
+    private ThreadPoolExecutor threadPool;
     private TaskTrackerManager ttm;
     public JobInitializer(Configuration conf, TaskTrackerManager ttm) {
       int numThreads = conf.getInt("mapred.jobinit.threads",
           DEFAULT_NUM_THREADS);
-      threadPool = Executors.newFixedThreadPool(numThreads);
+      threadPool = new ThreadPoolExecutor(numThreads, numThreads, 0L,
+					TimeUnit.MILLISECONDS, new LinkedBlockingQueue<Runnable>());
+      // Pre-starting all threads to ensure the threads are executed as JobTracker
+      // instead of the user submitted the job, otherwise job initialization fails
+      // when security is enabled
+      if (threadPool.prestartAllCoreThreads() != numThreads) {
+          throw new RuntimeException("Failed to pre-start threads in JobInitializer");
+      }
       this.ttm = ttm;
     }
     public void initJob(JobInfo jobInfo, JobInProgress job) {