Просмотр исходного кода

MAPREDUCE-5732. Report proper queue when job has been automatically placed (Sandy Ryza)

git-svn-id: https://svn.apache.org/repos/asf/hadoop/common/trunk@1562641 13f79535-47bb-0310-9956-ffa450edef68
Sanford Ryza 11 лет назад
Родитель
Сommit
db80705719
17 измененных файлов с 140 добавлено и 12 удалено
  1. 3 0
      hadoop-mapreduce-project/CHANGES.txt
  2. 6 0
      hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-app/src/main/java/org/apache/hadoop/mapreduce/jobhistory/JobHistoryEventHandler.java
  3. 3 1
      hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-app/src/main/java/org/apache/hadoop/mapreduce/v2/app/job/Job.java
  4. 9 1
      hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-app/src/main/java/org/apache/hadoop/mapreduce/v2/app/job/impl/JobImpl.java
  5. 5 2
      hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-app/src/main/java/org/apache/hadoop/mapreduce/v2/app/rm/RMCommunicator.java
  6. 9 0
      hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-app/src/test/java/org/apache/hadoop/mapreduce/jobhistory/TestEvents.java
  7. 22 8
      hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-app/src/test/java/org/apache/hadoop/mapreduce/v2/app/MRApp.java
  8. 1 0
      hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-app/src/test/java/org/apache/hadoop/mapreduce/v2/app/MockAppContext.java
  9. 5 0
      hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-app/src/test/java/org/apache/hadoop/mapreduce/v2/app/MockJobs.java
  10. 5 0
      hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-app/src/test/java/org/apache/hadoop/mapreduce/v2/app/TestRuntimeEstimators.java
  11. 9 0
      hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/avro/Events.avpr
  12. 2 0
      hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/java/org/apache/hadoop/mapreduce/jobhistory/EventReader.java
  13. 7 0
      hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/java/org/apache/hadoop/mapreduce/jobhistory/JobHistoryParser.java
  14. 5 0
      hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-hs/src/main/java/org/apache/hadoop/mapreduce/v2/hs/CompletedJob.java
  15. 5 0
      hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-hs/src/main/java/org/apache/hadoop/mapreduce/v2/hs/PartialJob.java
  16. 40 0
      hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-hs/src/test/java/org/apache/hadoop/mapreduce/v2/hs/TestJobHistoryEvents.java
  17. 4 0
      hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-hs/src/test/java/org/apache/hadoop/mapreduce/v2/hs/webapp/TestHsWebServicesAcls.java

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

@@ -209,6 +209,9 @@ Release 2.4.0 - UNRELEASED
     MAPREDUCE-5464. Add analogs of the SLOTS_MILLIS counters that jive with the
     MAPREDUCE-5464. Add analogs of the SLOTS_MILLIS counters that jive with the
     YARN resource model (Sandy Ryza)
     YARN resource model (Sandy Ryza)
 
 
+    MAPREDUCE-5732. Report proper queue when job has been automatically placed
+    (Sandy Ryza)
+
   OPTIMIZATIONS
   OPTIMIZATIONS
 
 
     MAPREDUCE-5484. YarnChild unnecessarily loads job conf twice (Sandy Ryza)
     MAPREDUCE-5484. YarnChild unnecessarily loads job conf twice (Sandy Ryza)

+ 6 - 0
hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-app/src/main/java/org/apache/hadoop/mapreduce/jobhistory/JobHistoryEventHandler.java

@@ -525,6 +525,12 @@ public class JobHistoryEventHandler extends AbstractService
         JobInitedEvent jie = (JobInitedEvent) event.getHistoryEvent();
         JobInitedEvent jie = (JobInitedEvent) event.getHistoryEvent();
         mi.getJobIndexInfo().setJobStartTime(jie.getLaunchTime());
         mi.getJobIndexInfo().setJobStartTime(jie.getLaunchTime());
       }
       }
+      
+      if (event.getHistoryEvent().getEventType() == EventType.JOB_QUEUE_CHANGED) {
+        JobQueueChangeEvent jQueueEvent =
+            (JobQueueChangeEvent) event.getHistoryEvent();
+        mi.getJobIndexInfo().setQueueName(jQueueEvent.getJobQueueName());
+      }
 
 
       // If this is JobFinishedEvent, close the writer and setup the job-index
       // If this is JobFinishedEvent, close the writer and setup the job-index
       if (event.getHistoryEvent().getEventType() == EventType.JOB_FINISHED) {
       if (event.getHistoryEvent().getEventType() == EventType.JOB_FINISHED) {

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

@@ -39,7 +39,7 @@ import org.apache.hadoop.security.authorize.AccessControlList;
 
 
 
 
 /**
 /**
- * Main interface to interact with the job. Provides only getters. 
+ * Main interface to interact with the job.
  */
  */
 public interface Job {
 public interface Job {
 
 
@@ -98,4 +98,6 @@ public interface Job {
   List<AMInfo> getAMInfos();
   List<AMInfo> getAMInfos();
   
   
   boolean checkAccess(UserGroupInformation callerUGI, JobACL jobOperation);
   boolean checkAccess(UserGroupInformation callerUGI, JobACL jobOperation);
+  
+  public void setQueueName(String queueName);
 }
 }

+ 9 - 1
hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-app/src/main/java/org/apache/hadoop/mapreduce/v2/app/job/impl/JobImpl.java

@@ -59,6 +59,7 @@ import org.apache.hadoop.mapreduce.jobhistory.JobHistoryEvent;
 import org.apache.hadoop.mapreduce.jobhistory.JobHistoryParser.TaskInfo;
 import org.apache.hadoop.mapreduce.jobhistory.JobHistoryParser.TaskInfo;
 import org.apache.hadoop.mapreduce.jobhistory.JobInfoChangeEvent;
 import org.apache.hadoop.mapreduce.jobhistory.JobInfoChangeEvent;
 import org.apache.hadoop.mapreduce.jobhistory.JobInitedEvent;
 import org.apache.hadoop.mapreduce.jobhistory.JobInitedEvent;
+import org.apache.hadoop.mapreduce.jobhistory.JobQueueChangeEvent;
 import org.apache.hadoop.mapreduce.jobhistory.JobSubmittedEvent;
 import org.apache.hadoop.mapreduce.jobhistory.JobSubmittedEvent;
 import org.apache.hadoop.mapreduce.jobhistory.JobUnsuccessfulCompletionEvent;
 import org.apache.hadoop.mapreduce.jobhistory.JobUnsuccessfulCompletionEvent;
 import org.apache.hadoop.mapreduce.lib.chain.ChainMapper;
 import org.apache.hadoop.mapreduce.lib.chain.ChainMapper;
@@ -181,7 +182,7 @@ public class JobImpl implements org.apache.hadoop.mapreduce.v2.app.job.Job,
   private final EventHandler eventHandler;
   private final EventHandler eventHandler;
   private final MRAppMetrics metrics;
   private final MRAppMetrics metrics;
   private final String userName;
   private final String userName;
-  private final String queueName;
+  private String queueName;
   private final long appSubmitTime;
   private final long appSubmitTime;
   private final AppContext appContext;
   private final AppContext appContext;
 
 
@@ -1123,6 +1124,13 @@ public class JobImpl implements org.apache.hadoop.mapreduce.v2.app.job.Job,
     return queueName;
     return queueName;
   }
   }
   
   
+  @Override
+  public void setQueueName(String queueName) {
+    this.queueName = queueName;
+    JobQueueChangeEvent jqce = new JobQueueChangeEvent(oldJobId, queueName);
+    eventHandler.handle(new JobHistoryEvent(jobId, jqce));
+  }
+  
   /*
   /*
    * (non-Javadoc)
    * (non-Javadoc)
    * @see org.apache.hadoop.mapreduce.v2.app.job.Job#getConfFile()
    * @see org.apache.hadoop.mapreduce.v2.app.job.Job#getConfFile()

+ 5 - 2
hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-app/src/main/java/org/apache/hadoop/mapreduce/v2/app/rm/RMCommunicator.java

@@ -109,11 +109,11 @@ public abstract class RMCommunicator extends AbstractService
   @Override
   @Override
   protected void serviceStart() throws Exception {
   protected void serviceStart() throws Exception {
     scheduler= createSchedulerProxy();
     scheduler= createSchedulerProxy();
-    register();
-    startAllocatorThread();
     JobID id = TypeConverter.fromYarn(this.applicationId);
     JobID id = TypeConverter.fromYarn(this.applicationId);
     JobId jobId = TypeConverter.toYarn(id);
     JobId jobId = TypeConverter.toYarn(id);
     job = context.getJob(jobId);
     job = context.getJob(jobId);
+    register();
+    startAllocatorThread();
     super.serviceStart();
     super.serviceStart();
   }
   }
 
 
@@ -161,6 +161,9 @@ public abstract class RMCommunicator extends AbstractService
       }
       }
       this.applicationACLs = response.getApplicationACLs();
       this.applicationACLs = response.getApplicationACLs();
       LOG.info("maxContainerCapability: " + maxContainerCapability.getMemory());
       LOG.info("maxContainerCapability: " + maxContainerCapability.getMemory());
+      String queue = response.getQueue();
+      LOG.info("queue: " + queue);
+      job.setQueueName(queue);
     } catch (Exception are) {
     } catch (Exception are) {
       LOG.error("Exception while registering", are);
       LOG.error("Exception while registering", are);
       throw new YarnRuntimeException(are);
       throw new YarnRuntimeException(are);

+ 9 - 0
hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-app/src/test/java/org/apache/hadoop/mapreduce/jobhistory/TestEvents.java

@@ -81,6 +81,15 @@ public class TestEvents {
     assertEquals(test.getPriority(), JobPriority.LOW);
     assertEquals(test.getPriority(), JobPriority.LOW);
 
 
   }
   }
+  
+  @Test(timeout = 10000)
+  public void testJobQueueChange() throws Exception {
+    org.apache.hadoop.mapreduce.JobID jid = new JobID("001", 1);
+    JobQueueChangeEvent test = new JobQueueChangeEvent(jid,
+        "newqueue");
+    assertEquals(test.getJobId().toString(), jid.toString());
+    assertEquals(test.getJobQueueName(), "newqueue");
+  }
 
 
   /**
   /**
    * simple test TaskUpdatedEvent and TaskUpdated
    * simple test TaskUpdatedEvent and TaskUpdated

+ 22 - 8
hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-app/src/test/java/org/apache/hadoop/mapreduce/v2/app/MRApp.java

@@ -117,6 +117,9 @@ public class MRApp extends MRAppMaster {
   private File testWorkDir;
   private File testWorkDir;
   private Path testAbsPath;
   private Path testAbsPath;
   private ClusterInfo clusterInfo;
   private ClusterInfo clusterInfo;
+  
+  // Queue to pretend the RM assigned us
+  private String assignedQueue;
 
 
   public static String NM_HOST = "localhost";
   public static String NM_HOST = "localhost";
   public static int NM_PORT = 1234;
   public static int NM_PORT = 1234;
@@ -133,7 +136,7 @@ public class MRApp extends MRAppMaster {
 
 
   public MRApp(int maps, int reduces, boolean autoComplete, String testName,
   public MRApp(int maps, int reduces, boolean autoComplete, String testName,
       boolean cleanOnStart, Clock clock) {
       boolean cleanOnStart, Clock clock) {
-    this(maps, reduces, autoComplete, testName, cleanOnStart, 1, clock);
+    this(maps, reduces, autoComplete, testName, cleanOnStart, 1, clock, null);
   }
   }
 
 
   public MRApp(int maps, int reduces, boolean autoComplete, String testName,
   public MRApp(int maps, int reduces, boolean autoComplete, String testName,
@@ -146,6 +149,12 @@ public class MRApp extends MRAppMaster {
       boolean cleanOnStart) {
       boolean cleanOnStart) {
     this(maps, reduces, autoComplete, testName, cleanOnStart, 1);
     this(maps, reduces, autoComplete, testName, cleanOnStart, 1);
   }
   }
+  
+  public MRApp(int maps, int reduces, boolean autoComplete, String testName,
+      boolean cleanOnStart, String assignedQueue) {
+    this(maps, reduces, autoComplete, testName, cleanOnStart, 1,
+        new SystemClock(), assignedQueue);
+  }
 
 
   public MRApp(int maps, int reduces, boolean autoComplete, String testName,
   public MRApp(int maps, int reduces, boolean autoComplete, String testName,
       boolean cleanOnStart, boolean unregistered) {
       boolean cleanOnStart, boolean unregistered) {
@@ -178,7 +187,7 @@ public class MRApp extends MRAppMaster {
   public MRApp(int maps, int reduces, boolean autoComplete, String testName,
   public MRApp(int maps, int reduces, boolean autoComplete, String testName,
       boolean cleanOnStart, int startCount) {
       boolean cleanOnStart, int startCount) {
     this(maps, reduces, autoComplete, testName, cleanOnStart, startCount,
     this(maps, reduces, autoComplete, testName, cleanOnStart, startCount,
-        new SystemClock());
+        new SystemClock(), null);
   }
   }
 
 
   public MRApp(int maps, int reduces, boolean autoComplete, String testName,
   public MRApp(int maps, int reduces, boolean autoComplete, String testName,
@@ -191,33 +200,34 @@ public class MRApp extends MRAppMaster {
       boolean cleanOnStart, int startCount, Clock clock, boolean unregistered) {
       boolean cleanOnStart, int startCount, Clock clock, boolean unregistered) {
     this(getApplicationAttemptId(applicationId, startCount), getContainerId(
     this(getApplicationAttemptId(applicationId, startCount), getContainerId(
       applicationId, startCount), maps, reduces, autoComplete, testName,
       applicationId, startCount), maps, reduces, autoComplete, testName,
-      cleanOnStart, startCount, clock, unregistered);
+      cleanOnStart, startCount, clock, unregistered, null);
   }
   }
 
 
   public MRApp(int maps, int reduces, boolean autoComplete, String testName,
   public MRApp(int maps, int reduces, boolean autoComplete, String testName,
-      boolean cleanOnStart, int startCount, Clock clock) {
+      boolean cleanOnStart, int startCount, Clock clock, String assignedQueue) {
     this(getApplicationAttemptId(applicationId, startCount), getContainerId(
     this(getApplicationAttemptId(applicationId, startCount), getContainerId(
       applicationId, startCount), maps, reduces, autoComplete, testName,
       applicationId, startCount), maps, reduces, autoComplete, testName,
-      cleanOnStart, startCount, clock, true);
+      cleanOnStart, startCount, clock, true, assignedQueue);
   }
   }
 
 
   public MRApp(ApplicationAttemptId appAttemptId, ContainerId amContainerId,
   public MRApp(ApplicationAttemptId appAttemptId, ContainerId amContainerId,
       int maps, int reduces, boolean autoComplete, String testName,
       int maps, int reduces, boolean autoComplete, String testName,
       boolean cleanOnStart, int startCount, boolean unregistered) {
       boolean cleanOnStart, int startCount, boolean unregistered) {
     this(appAttemptId, amContainerId, maps, reduces, autoComplete, testName,
     this(appAttemptId, amContainerId, maps, reduces, autoComplete, testName,
-        cleanOnStart, startCount, new SystemClock(), unregistered);
+        cleanOnStart, startCount, new SystemClock(), unregistered, null);
   }
   }
 
 
   public MRApp(ApplicationAttemptId appAttemptId, ContainerId amContainerId,
   public MRApp(ApplicationAttemptId appAttemptId, ContainerId amContainerId,
       int maps, int reduces, boolean autoComplete, String testName,
       int maps, int reduces, boolean autoComplete, String testName,
       boolean cleanOnStart, int startCount) {
       boolean cleanOnStart, int startCount) {
     this(appAttemptId, amContainerId, maps, reduces, autoComplete, testName,
     this(appAttemptId, amContainerId, maps, reduces, autoComplete, testName,
-        cleanOnStart, startCount, new SystemClock(), true);
+        cleanOnStart, startCount, new SystemClock(), true, null);
   }
   }
 
 
   public MRApp(ApplicationAttemptId appAttemptId, ContainerId amContainerId,
   public MRApp(ApplicationAttemptId appAttemptId, ContainerId amContainerId,
       int maps, int reduces, boolean autoComplete, String testName,
       int maps, int reduces, boolean autoComplete, String testName,
-      boolean cleanOnStart, int startCount, Clock clock, boolean unregistered) {
+      boolean cleanOnStart, int startCount, Clock clock, boolean unregistered,
+      String assignedQueue) {
     super(appAttemptId, amContainerId, NM_HOST, NM_PORT, NM_HTTP_PORT, clock, System
     super(appAttemptId, amContainerId, NM_HOST, NM_PORT, NM_HTTP_PORT, clock, System
         .currentTimeMillis(), MRJobConfig.DEFAULT_MR_AM_MAX_ATTEMPTS);
         .currentTimeMillis(), MRJobConfig.DEFAULT_MR_AM_MAX_ATTEMPTS);
     this.testWorkDir = new File("target", testName);
     this.testWorkDir = new File("target", testName);
@@ -239,6 +249,7 @@ public class MRApp extends MRAppMaster {
     // If safeToReportTerminationToUser is set to true, we can verify whether
     // If safeToReportTerminationToUser is set to true, we can verify whether
     // the job can reaches the final state when MRAppMaster shuts down.
     // the job can reaches the final state when MRAppMaster shuts down.
     this.successfullyUnregistered.set(unregistered);
     this.successfullyUnregistered.set(unregistered);
+    this.assignedQueue = assignedQueue;
   }
   }
 
 
   @Override
   @Override
@@ -285,6 +296,9 @@ public class MRApp extends MRAppMaster {
     start();
     start();
     DefaultMetricsSystem.shutdown();
     DefaultMetricsSystem.shutdown();
     Job job = getContext().getAllJobs().values().iterator().next();
     Job job = getContext().getAllJobs().values().iterator().next();
+    if (assignedQueue != null) {
+      job.setQueueName(assignedQueue);
+    }
 
 
     // Write job.xml
     // Write job.xml
     String jobFile = MRApps.getJobFile(conf, user,
     String jobFile = MRApps.getJobFile(conf, user,

+ 1 - 0
hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-app/src/test/java/org/apache/hadoop/mapreduce/v2/app/MockAppContext.java

@@ -39,6 +39,7 @@ public class MockAppContext implements AppContext {
   final Map<JobId, Job> jobs;
   final Map<JobId, Job> jobs;
   final long startTime = System.currentTimeMillis();
   final long startTime = System.currentTimeMillis();
   Set<String> blacklistedNodes;
   Set<String> blacklistedNodes;
+  String queue;
   
   
   public MockAppContext(int appid) {
   public MockAppContext(int appid) {
     appID = MockJobs.newAppID(appid);
     appID = MockJobs.newAppID(appid);

+ 5 - 0
hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-app/src/test/java/org/apache/hadoop/mapreduce/v2/app/MockJobs.java

@@ -629,6 +629,11 @@ public class MockJobs extends MockApps {
         jobConf.addResource(fc.open(configFile), configFile.toString());
         jobConf.addResource(fc.open(configFile), configFile.toString());
         return jobConf;
         return jobConf;
       }
       }
+
+      @Override
+      public void setQueueName(String queueName) {
+        // do nothing
+      }
     };
     };
   }
   }
 
 

+ 5 - 0
hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-app/src/test/java/org/apache/hadoop/mapreduce/v2/app/TestRuntimeEstimators.java

@@ -505,6 +505,11 @@ public class TestRuntimeEstimators {
     public Configuration loadConfFile() {
     public Configuration loadConfFile() {
       throw new UnsupportedOperationException();
       throw new UnsupportedOperationException();
     }
     }
+
+    @Override
+    public void setQueueName(String queueName) {
+      // do nothing
+    }
   }
   }
 
 
   /*
   /*

+ 9 - 0
hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/avro/Events.avpr

@@ -122,6 +122,13 @@
       ]
       ]
      },
      },
 
 
+     {"type": "record", "name": "JobQueueChange",
+      "fields": [
+          {"name": "jobid", "type": "string"},
+          {"name": "jobQueueName", "type": "string"}
+      ]
+     },
+
      {"type": "record", "name": "JobUnsuccessfulCompletion",
      {"type": "record", "name": "JobUnsuccessfulCompletion",
       "fields": [
       "fields": [
           {"name": "jobid", "type": "string"},
           {"name": "jobid", "type": "string"},
@@ -267,6 +274,7 @@
           "JOB_FINISHED",
           "JOB_FINISHED",
           "JOB_PRIORITY_CHANGED",
           "JOB_PRIORITY_CHANGED",
           "JOB_STATUS_CHANGED",
           "JOB_STATUS_CHANGED",
+          "JOB_QUEUE_CHANGED",
           "JOB_FAILED",
           "JOB_FAILED",
           "JOB_KILLED",
           "JOB_KILLED",
           "JOB_ERROR",
           "JOB_ERROR",
@@ -306,6 +314,7 @@
                "JobInited",
                "JobInited",
                "AMStarted",
                "AMStarted",
                "JobPriorityChange",
                "JobPriorityChange",
+               "JobQueueChange",
                "JobStatusChanged",
                "JobStatusChanged",
                "JobSubmitted",
                "JobSubmitted",
                "JobUnsuccessfulCompletion",
                "JobUnsuccessfulCompletion",

+ 2 - 0
hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/java/org/apache/hadoop/mapreduce/jobhistory/EventReader.java

@@ -98,6 +98,8 @@ public class EventReader implements Closeable {
       result = new JobFinishedEvent(); break;
       result = new JobFinishedEvent(); break;
     case JOB_PRIORITY_CHANGED:
     case JOB_PRIORITY_CHANGED:
       result = new JobPriorityChangeEvent(); break;
       result = new JobPriorityChangeEvent(); break;
+    case JOB_QUEUE_CHANGED:
+      result = new JobQueueChangeEvent(); break;
     case JOB_STATUS_CHANGED:
     case JOB_STATUS_CHANGED:
       result = new JobStatusChangedEvent(); break;
       result = new JobStatusChangedEvent(); break;
     case JOB_FAILED:
     case JOB_FAILED:

+ 7 - 0
hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/java/org/apache/hadoop/mapreduce/jobhistory/JobHistoryParser.java

@@ -183,6 +183,9 @@ public class JobHistoryParser implements HistoryEventHandler {
     case JOB_PRIORITY_CHANGED:
     case JOB_PRIORITY_CHANGED:
       handleJobPriorityChangeEvent((JobPriorityChangeEvent) event);
       handleJobPriorityChangeEvent((JobPriorityChangeEvent) event);
       break;
       break;
+    case JOB_QUEUE_CHANGED:
+      handleJobQueueChangeEvent((JobQueueChangeEvent) event);
+      break;
     case JOB_FAILED:
     case JOB_FAILED:
     case JOB_KILLED:
     case JOB_KILLED:
     case JOB_ERROR:
     case JOB_ERROR:
@@ -385,6 +388,10 @@ public class JobHistoryParser implements HistoryEventHandler {
   private void handleJobPriorityChangeEvent(JobPriorityChangeEvent event) {
   private void handleJobPriorityChangeEvent(JobPriorityChangeEvent event) {
     info.priority = event.getPriority();
     info.priority = event.getPriority();
   }
   }
+  
+  private void handleJobQueueChangeEvent(JobQueueChangeEvent event) {
+    info.jobQueueName = event.getJobQueueName();
+  }
 
 
   private void handleJobInitedEvent(JobInitedEvent event) {
   private void handleJobInitedEvent(JobInitedEvent event) {
     info.launchTime = event.getLaunchTime();
     info.launchTime = event.getLaunchTime();

+ 5 - 0
hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-hs/src/main/java/org/apache/hadoop/mapreduce/v2/hs/CompletedJob.java

@@ -453,4 +453,9 @@ public class CompletedJob implements org.apache.hadoop.mapreduce.v2.app.job.Job
     }
     }
     return amInfos;
     return amInfos;
   }
   }
+
+  @Override
+  public void setQueueName(String queueName) {
+    throw new UnsupportedOperationException("Can't set job's queue name in history");
+  }
 }
 }

+ 5 - 0
hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-hs/src/main/java/org/apache/hadoop/mapreduce/v2/hs/PartialJob.java

@@ -190,5 +190,10 @@ public class PartialJob implements org.apache.hadoop.mapreduce.v2.app.job.Job {
   public List<AMInfo> getAMInfos() {
   public List<AMInfo> getAMInfos() {
     return null;
     return null;
   }
   }
+  
+  @Override
+  public void setQueueName(String queueName) {
+    throw new UnsupportedOperationException("Can't set job's queue name in history");
+  }
 
 
 }
 }

+ 40 - 0
hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-hs/src/test/java/org/apache/hadoop/mapreduce/v2/hs/TestJobHistoryEvents.java

@@ -155,6 +155,41 @@ public class TestJobHistoryEvents {
     Assert.assertEquals("JobHistoryEventHandler",
     Assert.assertEquals("JobHistoryEventHandler",
         services[services.length - 1].getName());
         services[services.length - 1].getName());
   }
   }
+  
+  @Test
+  public void testAssignedQueue() throws Exception {
+    Configuration conf = new Configuration();
+    MRApp app = new MRAppWithHistory(2, 1, true, this.getClass().getName(),
+        true, "assignedQueue");
+    app.submit(conf);
+    Job job = app.getContext().getAllJobs().values().iterator().next();
+    JobId jobId = job.getID();
+    LOG.info("JOBID is " + TypeConverter.fromYarn(jobId).toString());
+    app.waitForState(job, JobState.SUCCEEDED);
+    
+    //make sure all events are flushed 
+    app.waitForState(Service.STATE.STOPPED);
+    /*
+     * Use HistoryContext to read logged events and verify the number of 
+     * completed maps 
+    */
+    HistoryContext context = new JobHistory();
+    // test start and stop states
+    ((JobHistory)context).init(conf);
+    ((JobHistory)context).start();
+    Assert.assertTrue( context.getStartTime()>0);
+    Assert.assertEquals(((JobHistory)context).getServiceState(),Service.STATE.STARTED);
+
+    // get job before stopping JobHistory
+    Job parsedJob = context.getJob(jobId);
+
+    // stop JobHistory
+    ((JobHistory)context).stop();
+    Assert.assertEquals(((JobHistory)context).getServiceState(),Service.STATE.STOPPED);
+
+    Assert.assertEquals("QueueName not correct", "assignedQueue",
+        parsedJob.getQueueName());
+  }
 
 
   private void verifyTask(Task task) {
   private void verifyTask(Task task) {
     Assert.assertEquals("Task state not currect", TaskState.SUCCEEDED,
     Assert.assertEquals("Task state not currect", TaskState.SUCCEEDED,
@@ -184,6 +219,11 @@ public class TestJobHistoryEvents {
       super(maps, reduces, autoComplete, testName, cleanOnStart);
       super(maps, reduces, autoComplete, testName, cleanOnStart);
     }
     }
 
 
+    public MRAppWithHistory(int maps, int reduces, boolean autoComplete,
+        String testName, boolean cleanOnStart, String assignedQueue) {
+      super(maps, reduces, autoComplete, testName, cleanOnStart, assignedQueue);
+    }
+
     @Override
     @Override
     protected EventHandler<JobHistoryEvent> createJobHistoryHandler(
     protected EventHandler<JobHistoryEvent> createJobHistoryHandler(
         AppContext context) {
         AppContext context) {

+ 4 - 0
hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-hs/src/test/java/org/apache/hadoop/mapreduce/v2/hs/webapp/TestHsWebServicesAcls.java

@@ -415,5 +415,9 @@ public class TestHsWebServicesAcls {
       return aclsMgr.checkAccess(callerUGI, jobOperation,
       return aclsMgr.checkAccess(callerUGI, jobOperation,
           this.getUserName(), jobAcls.get(jobOperation));
           this.getUserName(), jobAcls.get(jobOperation));
     }
     }
+
+    @Override
+    public void setQueueName(String queueName) {
+    }
   }
   }
 }
 }