Author: tucu
Date: Thu Oct 11 21:53:25 2012
New Revision: 1397330

URL: http://svn.apache.org/viewvc?rev=1397330&view=rev
Log:
MAPREDUCE-4451. fairscheduler fail to init job with kerberos authentication 
configured. (erik.fang via tucu)

Modified:
    hadoop/common/branches/branch-1/CHANGES.txt
    
hadoop/common/branches/branch-1/src/contrib/fairscheduler/src/java/org/apache/hadoop/mapred/FairScheduler.java

Modified: hadoop/common/branches/branch-1/CHANGES.txt
URL: 
http://svn.apache.org/viewvc/hadoop/common/branches/branch-1/CHANGES.txt?rev=1397330&r1=1397329&r2=1397330&view=diff
==============================================================================
--- hadoop/common/branches/branch-1/CHANGES.txt (original)
+++ hadoop/common/branches/branch-1/CHANGES.txt Thu Oct 11 21:53:25 2012
@@ -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

Modified: 
hadoop/common/branches/branch-1/src/contrib/fairscheduler/src/java/org/apache/hadoop/mapred/FairScheduler.java
URL: 
http://svn.apache.org/viewvc/hadoop/common/branches/branch-1/src/contrib/fairscheduler/src/java/org/apache/hadoop/mapred/FairScheduler.java?rev=1397330&r1=1397329&r2=1397330&view=diff
==============================================================================
--- 
hadoop/common/branches/branch-1/src/contrib/fairscheduler/src/java/org/apache/hadoop/mapred/FairScheduler.java
 (original)
+++ 
hadoop/common/branches/branch-1/src/contrib/fairscheduler/src/java/org/apache/hadoop/mapred/FairScheduler.java
 Thu Oct 11 21:53:25 2012
@@ -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 TaskS
 
   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) {


Reply via email to