You are viewing a plain text version of this content. The canonical link for it is here.
Posted to mapreduce-commits@hadoop.apache.org by ac...@apache.org on 2011/09/08 20:28:16 UTC

svn commit: r1166840 - in /hadoop/common/branches/branch-0.23/hadoop-mapreduce-project: ./ dev-support/ hadoop-mapreduce-client/hadoop-mapreduce-client-common/src/main/java/org/apache/hadoop/mapreduce/v2/util/ hadoop-mapreduce-client/hadoop-mapreduce-c...

Author: acmurthy
Date: Thu Sep  8 18:28:14 2011
New Revision: 1166840

URL: http://svn.apache.org/viewvc?rev=1166840&view=rev
Log:
Merge -r 1166837:1166838 from trunk to branch-0.23 to fix MAPREDUCE-2908.

Added:
    hadoop/common/branches/branch-0.23/hadoop-mapreduce-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/capacity/CSQueue.java
      - copied unchanged from r1166838, hadoop/common/trunk/hadoop-mapreduce-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/capacity/CSQueue.java
Modified:
    hadoop/common/branches/branch-0.23/hadoop-mapreduce-project/CHANGES.txt
    hadoop/common/branches/branch-0.23/hadoop-mapreduce-project/dev-support/findbugs-exclude.xml
    hadoop/common/branches/branch-0.23/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-common/src/main/java/org/apache/hadoop/mapreduce/v2/util/MRApps.java
    hadoop/common/branches/branch-0.23/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/java/org/apache/hadoop/mapred/JobClient.java
    hadoop/common/branches/branch-0.23/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/java/org/apache/hadoop/mapreduce/util/ResourceCalculatorPlugin.java
    hadoop/common/branches/branch-0.23/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/main/java/org/apache/hadoop/mapred/ClientServiceDelegate.java
    hadoop/common/branches/branch-0.23/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-shuffle/src/main/java/org/apache/hadoop/mapred/ShuffleHandler.java
    hadoop/common/branches/branch-0.23/hadoop-mapreduce-project/hadoop-yarn/dev-support/findbugs-exclude.xml
    hadoop/common/branches/branch-0.23/hadoop-mapreduce-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/util/ResourceCalculatorPlugin.java
    hadoop/common/branches/branch-0.23/hadoop-mapreduce-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/webapp/WebApps.java
    hadoop/common/branches/branch-0.23/hadoop-mapreduce-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/containermanager/ContainerManagerImpl.java
    hadoop/common/branches/branch-0.23/hadoop-mapreduce-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/RMAppManager.java
    hadoop/common/branches/branch-0.23/hadoop-mapreduce-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/RMConfig.java
    hadoop/common/branches/branch-0.23/hadoop-mapreduce-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/recovery/ZKStore.java
    hadoop/common/branches/branch-0.23/hadoop-mapreduce-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/SchedulerApp.java
    hadoop/common/branches/branch-0.23/hadoop-mapreduce-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/capacity/CapacityScheduler.java
    hadoop/common/branches/branch-0.23/hadoop-mapreduce-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/capacity/LeafQueue.java
    hadoop/common/branches/branch-0.23/hadoop-mapreduce-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/capacity/ParentQueue.java
    hadoop/common/branches/branch-0.23/hadoop-mapreduce-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/capacity/Queue.java
    hadoop/common/branches/branch-0.23/hadoop-mapreduce-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/webapp/CapacitySchedulerPage.java
    hadoop/common/branches/branch-0.23/hadoop-mapreduce-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/capacity/TestApplicationLimits.java
    hadoop/common/branches/branch-0.23/hadoop-mapreduce-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/capacity/TestLeafQueue.java
    hadoop/common/branches/branch-0.23/hadoop-mapreduce-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/capacity/TestParentQueue.java
    hadoop/common/branches/branch-0.23/hadoop-mapreduce-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/capacity/TestUtils.java
    hadoop/common/branches/branch-0.23/hadoop-mapreduce-project/src/test/findbugsExcludeFile.xml

Modified: hadoop/common/branches/branch-0.23/hadoop-mapreduce-project/CHANGES.txt
URL: http://svn.apache.org/viewvc/hadoop/common/branches/branch-0.23/hadoop-mapreduce-project/CHANGES.txt?rev=1166840&r1=1166839&r2=1166840&view=diff
==============================================================================
--- hadoop/common/branches/branch-0.23/hadoop-mapreduce-project/CHANGES.txt (original)
+++ hadoop/common/branches/branch-0.23/hadoop-mapreduce-project/CHANGES.txt Thu Sep  8 18:28:14 2011
@@ -1215,6 +1215,8 @@ Release 0.23.0 - Unreleased
 
    MAPREDUCE-2948. Hadoop streaming test failure, post MR-2767 (mahadev)
 
+   MAPREDUCE-2908. Fix all findbugs warnings. (vinodkv via acmurthy) 
+
 Release 0.22.0 - Unreleased
 
   INCOMPATIBLE CHANGES

Modified: hadoop/common/branches/branch-0.23/hadoop-mapreduce-project/dev-support/findbugs-exclude.xml
URL: http://svn.apache.org/viewvc/hadoop/common/branches/branch-0.23/hadoop-mapreduce-project/dev-support/findbugs-exclude.xml?rev=1166840&r1=1166839&r2=1166840&view=diff
==============================================================================
--- hadoop/common/branches/branch-0.23/hadoop-mapreduce-project/dev-support/findbugs-exclude.xml (original)
+++ hadoop/common/branches/branch-0.23/hadoop-mapreduce-project/dev-support/findbugs-exclude.xml Thu Sep  8 18:28:14 2011
@@ -138,6 +138,11 @@
        <Method name="run" />
        <Bug pattern="DM_EXIT" />
      </Match>
+     <Match>
+       <Class name="org.apache.hadoop.mapreduce.security.token.DelegationTokenRenewal$DelegationTokenCancelThread" />
+       <Method name="run" />
+       <Bug pattern="DM_EXIT" />
+    </Match>
      <!--
        We need to cast objects between old and new api objects
      -->
@@ -155,7 +160,8 @@
      </Match>
      <Match>
        <Class name="org.apache.hadoop.mapred.FileOutputCommitter" />
-       <Bug pattern="NM_WRONG_PACKAGE_INTENTIONAL" />
+       <Method name="commitJob" />
+       <Bug pattern="NM_WRONG_PACKAGE" />
      </Match>
      <Match>
        <Class name="org.apache.hadoop.mapred.OutputCommitter" />
@@ -167,6 +173,14 @@
        <Bug pattern="NM_WRONG_PACKAGE_INTENTIONAL" />
      </Match>
      <Match>
+       <Class name="org.apache.hadoop.mapred.TaskCompletionEvent" />
+       <Or>
+       <Method name="setTaskStatus" />
+       <Method name="setTaskAttemptId" />
+       </Or>
+       <Bug pattern="NM_WRONG_PACKAGE" />
+     </Match>
+     <Match>
        <Class name="org.apache.hadoop.mapred.lib.db.DBInputFormat$DBRecordReader" />
        <Method name="next" />
        <Bug pattern="NM_WRONG_PACKAGE_INTENTIONAL" />

Modified: hadoop/common/branches/branch-0.23/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-common/src/main/java/org/apache/hadoop/mapreduce/v2/util/MRApps.java
URL: http://svn.apache.org/viewvc/hadoop/common/branches/branch-0.23/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-common/src/main/java/org/apache/hadoop/mapreduce/v2/util/MRApps.java?rev=1166840&r1=1166839&r2=1166840&view=diff
==============================================================================
--- hadoop/common/branches/branch-0.23/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-common/src/main/java/org/apache/hadoop/mapreduce/v2/util/MRApps.java (original)
+++ hadoop/common/branches/branch-0.23/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-common/src/main/java/org/apache/hadoop/mapreduce/v2/util/MRApps.java Thu Sep  8 18:28:14 2011
@@ -157,6 +157,7 @@ public class MRApps extends Apps {
   public static void setInitialClasspath(
       Map<String, String> environment) throws IOException {
     InputStream classpathFileStream = null;
+    BufferedReader reader = null;
     try {
       // Get yarn mapreduce-app classpath from generated classpath
       // Works if compile time env is same as runtime. Mainly tests.
@@ -165,8 +166,7 @@ public class MRApps extends Apps {
       String mrAppGeneratedClasspathFile = "mrapp-generated-classpath";
       classpathFileStream =
           thisClassLoader.getResourceAsStream(mrAppGeneratedClasspathFile);
-      BufferedReader reader =
-          new BufferedReader(new InputStreamReader(classpathFileStream));
+      reader = new BufferedReader(new InputStreamReader(classpathFileStream));
       String cp = reader.readLine();
       if (cp != null) {
         addToClassPath(environment, cp.trim());
@@ -198,6 +198,9 @@ public class MRApps extends Apps {
       if (classpathFileStream != null) {
         classpathFileStream.close();
       }
+      if (reader != null) {
+        reader.close();
+      }
     }
     // TODO: Remove duplicates.
   }

Modified: hadoop/common/branches/branch-0.23/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/java/org/apache/hadoop/mapred/JobClient.java
URL: http://svn.apache.org/viewvc/hadoop/common/branches/branch-0.23/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/java/org/apache/hadoop/mapred/JobClient.java?rev=1166840&r1=1166839&r2=1166840&view=diff
==============================================================================
--- hadoop/common/branches/branch-0.23/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/java/org/apache/hadoop/mapred/JobClient.java (original)
+++ hadoop/common/branches/branch-0.23/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/java/org/apache/hadoop/mapred/JobClient.java Thu Sep  8 18:28:14 2011
@@ -868,15 +868,6 @@ public class JobClient extends CLI {
     Counters counters = Counters.downgrade(cntrs);
     return counters.findCounter(counterGroupName, counterName).getValue();
   }
-  
-  void displayJobList(JobStatus[] jobs) {
-    System.out.printf("JobId\tState\tStartTime\tUserName\tQueue\tPriority\tSchedulingInfo\n");
-    for (JobStatus job : jobs) {
-      System.out.printf("%s\t%d\t%d\t%s\t%s\t%s\t%s\n", job.getJobID(), job.getRunState(),
-          job.getStartTime(), job.getUsername(), job.getQueue(), 
-          job.getJobPriority().name(), job.getSchedulingInfo());
-    }
-  }
 
   /**
    * Get status information about the max available Maps in the cluster.

Modified: hadoop/common/branches/branch-0.23/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/java/org/apache/hadoop/mapreduce/util/ResourceCalculatorPlugin.java
URL: http://svn.apache.org/viewvc/hadoop/common/branches/branch-0.23/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/java/org/apache/hadoop/mapreduce/util/ResourceCalculatorPlugin.java?rev=1166840&r1=1166839&r2=1166840&view=diff
==============================================================================
--- hadoop/common/branches/branch-0.23/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/java/org/apache/hadoop/mapreduce/util/ResourceCalculatorPlugin.java (original)
+++ hadoop/common/branches/branch-0.23/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/java/org/apache/hadoop/mapreduce/util/ResourceCalculatorPlugin.java Thu Sep  8 18:28:14 2011
@@ -97,7 +97,7 @@ public abstract class ResourceCalculator
   @InterfaceStability.Unstable
   public abstract ProcResourceValues getProcResourceValues();
 
-  public class ProcResourceValues {
+  public static class ProcResourceValues {
     private final long cumulativeCpuTime;
     private final long physicalMemorySize;
     private final long virtualMemorySize;

Modified: hadoop/common/branches/branch-0.23/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/main/java/org/apache/hadoop/mapred/ClientServiceDelegate.java
URL: http://svn.apache.org/viewvc/hadoop/common/branches/branch-0.23/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/main/java/org/apache/hadoop/mapred/ClientServiceDelegate.java?rev=1166840&r1=1166839&r2=1166840&view=diff
==============================================================================
--- hadoop/common/branches/branch-0.23/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/main/java/org/apache/hadoop/mapred/ClientServiceDelegate.java (original)
+++ hadoop/common/branches/branch-0.23/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/main/java/org/apache/hadoop/mapred/ClientServiceDelegate.java Thu Sep  8 18:28:14 2011
@@ -149,8 +149,7 @@ class ClientServiceDelegate {
         LOG.info("Connecting to " + serviceAddr);
         instantiateAMProxy(serviceAddr);
         return realProxy;
-      } catch (Exception e) {
-        //possibly
+      } catch (IOException e) {
         //possibly the AM has crashed
         //there may be some time before AM is restarted
         //keep retrying by getting the address from RM
@@ -159,8 +158,13 @@ class ClientServiceDelegate {
         try {
           Thread.sleep(2000);
         } catch (InterruptedException e1) {
+          LOG.warn("getProxy() call interruped", e1);
+          throw new YarnException(e1);
         }
         application = rm.getApplicationReport(appId);
+      } catch (InterruptedException e) {
+        LOG.warn("getProxy() call interruped", e);
+        throw new YarnException(e);
       }
     }
 
@@ -304,7 +308,6 @@ class ClientServiceDelegate {
 
   org.apache.hadoop.mapreduce.TaskReport[] getTaskReports(JobID jobID, TaskType taskType)
        throws YarnRemoteException, YarnRemoteException {
-    org.apache.hadoop.mapreduce.v2.api.records.JobId nJobID = TypeConverter.toYarn(jobID);
     GetTaskReportsRequest request = recordFactory.newRecordInstance(GetTaskReportsRequest.class);
     
     List<org.apache.hadoop.mapreduce.v2.api.records.TaskReport> taskReports = 

Modified: hadoop/common/branches/branch-0.23/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-shuffle/src/main/java/org/apache/hadoop/mapred/ShuffleHandler.java
URL: http://svn.apache.org/viewvc/hadoop/common/branches/branch-0.23/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-shuffle/src/main/java/org/apache/hadoop/mapred/ShuffleHandler.java?rev=1166840&r1=1166839&r2=1166840&view=diff
==============================================================================
--- hadoop/common/branches/branch-0.23/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-shuffle/src/main/java/org/apache/hadoop/mapred/ShuffleHandler.java (original)
+++ hadoop/common/branches/branch-0.23/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-shuffle/src/main/java/org/apache/hadoop/mapred/ShuffleHandler.java Thu Sep  8 18:28:14 2011
@@ -263,10 +263,6 @@ public class ShuffleHandler extends Abst
     }
   }
 
-  Shuffle createShuffle() {
-    return new Shuffle(getConfig());
-  }
-
   class HttpPipelineFactory implements ChannelPipelineFactory {
 
     final Shuffle SHUFFLE;
@@ -296,10 +292,12 @@ public class ShuffleHandler extends Abst
     private final IndexCache indexCache;
     private final LocalDirAllocator lDirAlloc =
       new LocalDirAllocator(NMConfig.NM_LOCAL_DIR);
+    private final int port;
 
     public Shuffle(Configuration conf) {
       this.conf = conf;
       indexCache = new IndexCache(new JobConf(conf));
+      this.port = conf.getInt(SHUFFLE_PORT_CONFIG_KEY, DEFAULT_SHUFFLE_PORT);
     }
 
     private List<String> splitMaps(List<String> mapq) {
@@ -362,7 +360,7 @@ public class ShuffleHandler extends Abst
       HttpResponse response = new DefaultHttpResponse(HTTP_1_1, OK);
       try {
         verifyRequest(jobId, ctx, request, response,
-            new URL("http", "", port, reqUri));
+            new URL("http", "", this.port, reqUri));
       } catch (IOException e) {
         LOG.warn("Shuffle failure ", e);
         sendError(ctx, e.getMessage(), UNAUTHORIZED);

Modified: hadoop/common/branches/branch-0.23/hadoop-mapreduce-project/hadoop-yarn/dev-support/findbugs-exclude.xml
URL: http://svn.apache.org/viewvc/hadoop/common/branches/branch-0.23/hadoop-mapreduce-project/hadoop-yarn/dev-support/findbugs-exclude.xml?rev=1166840&r1=1166839&r2=1166840&view=diff
==============================================================================
--- hadoop/common/branches/branch-0.23/hadoop-mapreduce-project/hadoop-yarn/dev-support/findbugs-exclude.xml (original)
+++ hadoop/common/branches/branch-0.23/hadoop-mapreduce-project/hadoop-yarn/dev-support/findbugs-exclude.xml Thu Sep  8 18:28:14 2011
@@ -66,6 +66,11 @@
     <Bug pattern="BC_UNCONFIRMED_CAST" />
   </Match>
   <Match>
+    <Class name="~org\.apache\.hadoop\.yarn\.server\.resourcemanager\.RMAppManager.*" />
+    <Method name="handle" />
+    <Bug pattern="BC_UNCONFIRMED_CAST" />
+  </Match>
+  <Match>
     <Class name="~org\.apache\.hadoop\.yarn\.server\.resourcemanager\.scheduler\.capacity\.CapacityScheduler.*" />
     <Method name="handle" />
     <Bug pattern="BC_UNCONFIRMED_CAST" />

Modified: hadoop/common/branches/branch-0.23/hadoop-mapreduce-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/util/ResourceCalculatorPlugin.java
URL: http://svn.apache.org/viewvc/hadoop/common/branches/branch-0.23/hadoop-mapreduce-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/util/ResourceCalculatorPlugin.java?rev=1166840&r1=1166839&r2=1166840&view=diff
==============================================================================
--- hadoop/common/branches/branch-0.23/hadoop-mapreduce-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/util/ResourceCalculatorPlugin.java (original)
+++ hadoop/common/branches/branch-0.23/hadoop-mapreduce-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/util/ResourceCalculatorPlugin.java Thu Sep  8 18:28:14 2011
@@ -97,7 +97,7 @@ public abstract class ResourceCalculator
   @InterfaceStability.Unstable
   public abstract ProcResourceValues getProcResourceValues();
 
-  public class ProcResourceValues {
+  public static class ProcResourceValues {
     private final long cumulativeCpuTime;
     private final long physicalMemorySize;
     private final long virtualMemorySize;

Modified: hadoop/common/branches/branch-0.23/hadoop-mapreduce-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/webapp/WebApps.java
URL: http://svn.apache.org/viewvc/hadoop/common/branches/branch-0.23/hadoop-mapreduce-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/webapp/WebApps.java?rev=1166840&r1=1166839&r2=1166840&view=diff
==============================================================================
--- hadoop/common/branches/branch-0.23/hadoop-mapreduce-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/webapp/WebApps.java (original)
+++ hadoop/common/branches/branch-0.23/hadoop-mapreduce-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/webapp/WebApps.java Thu Sep  8 18:28:14 2011
@@ -67,7 +67,6 @@ public class WebApps {
     boolean findPort = false;
     Configuration conf;
     boolean devMode = false;
-    Module[] modules;
 
     Builder(String name, Class<T> api, T application) {
       this.name = name;
@@ -99,11 +98,6 @@ public class WebApps {
       return this;
     }
 
-    public Builder<T> with(Module... modules) {
-      this.modules = modules; // OK
-      return this;
-    }
-
     public Builder<T> inDevMode() {
       devMode = true;
       return this;

Modified: hadoop/common/branches/branch-0.23/hadoop-mapreduce-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/containermanager/ContainerManagerImpl.java
URL: http://svn.apache.org/viewvc/hadoop/common/branches/branch-0.23/hadoop-mapreduce-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/containermanager/ContainerManagerImpl.java?rev=1166840&r1=1166839&r2=1166840&view=diff
==============================================================================
--- hadoop/common/branches/branch-0.23/hadoop-mapreduce-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/containermanager/ContainerManagerImpl.java (original)
+++ hadoop/common/branches/branch-0.23/hadoop-mapreduce-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/containermanager/ContainerManagerImpl.java Thu Sep  8 18:28:14 2011
@@ -311,7 +311,14 @@ public class ContainerManagerImpl extend
     Container container = this.context.getContainers().get(containerID);
     if (container == null) {
       LOG.warn("Trying to stop unknown container " + containerID);
-      NMAuditLogger.logFailure(container.getUser(),
+      String userName;
+      try {
+        userName = UserGroupInformation.getCurrentUser().getUserName();
+      } catch (IOException e) {
+        LOG.error("Error finding userName", e);
+        return response;
+      }
+      NMAuditLogger.logFailure(userName,
           AuditConstants.STOP_CONTAINER, "ContainerManagerImpl",
           "Trying to stop unknown container!",
           containerID.getAppId(), containerID);

Modified: hadoop/common/branches/branch-0.23/hadoop-mapreduce-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/RMAppManager.java
URL: http://svn.apache.org/viewvc/hadoop/common/branches/branch-0.23/hadoop-mapreduce-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/RMAppManager.java?rev=1166840&r1=1166839&r2=1166840&view=diff
==============================================================================
--- hadoop/common/branches/branch-0.23/hadoop-mapreduce-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/RMAppManager.java (original)
+++ hadoop/common/branches/branch-0.23/hadoop-mapreduce-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/RMAppManager.java Thu Sep  8 18:28:14 2011
@@ -18,32 +18,28 @@
 package org.apache.hadoop.yarn.server.resourcemanager;
 
 import java.io.IOException;
-import java.util.List;
 import java.util.LinkedList;
 
 import org.apache.commons.logging.Log;
 import org.apache.commons.logging.LogFactory;
 import org.apache.hadoop.conf.Configuration;
-import org.apache.hadoop.security.token.Token;
 import org.apache.hadoop.security.UserGroupInformation;
+import org.apache.hadoop.security.token.Token;
+import org.apache.hadoop.util.StringUtils;
 import org.apache.hadoop.yarn.api.records.ApplicationId;
 import org.apache.hadoop.yarn.api.records.ApplicationSubmissionContext;
 import org.apache.hadoop.yarn.event.EventHandler;
 import org.apache.hadoop.yarn.security.ApplicationTokenIdentifier;
-import org.apache.hadoop.yarn.security.ApplicationTokenSecretManager;
 import org.apache.hadoop.yarn.security.client.ClientToAMSecretManager;
-import org.apache.hadoop.yarn.server.resourcemanager.RMAuditLogger;
 import org.apache.hadoop.yarn.server.resourcemanager.RMAuditLogger.AuditConstants;
 import org.apache.hadoop.yarn.server.resourcemanager.recovery.ApplicationsStore.ApplicationStore;
 import org.apache.hadoop.yarn.server.resourcemanager.rmapp.RMApp;
-import org.apache.hadoop.yarn.server.resourcemanager.rmapp.RMApp;
 import org.apache.hadoop.yarn.server.resourcemanager.rmapp.RMAppEvent;
 import org.apache.hadoop.yarn.server.resourcemanager.rmapp.RMAppEventType;
 import org.apache.hadoop.yarn.server.resourcemanager.rmapp.RMAppImpl;
 import org.apache.hadoop.yarn.server.resourcemanager.rmapp.RMAppRejectedEvent;
 import org.apache.hadoop.yarn.server.resourcemanager.rmapp.attempt.RMAppAttempt;
 import org.apache.hadoop.yarn.server.resourcemanager.scheduler.YarnScheduler;
-import org.apache.hadoop.util.StringUtils;
 
 /**
  * This class manages the list of applications for the resource manager. 
@@ -154,7 +150,7 @@ public class RMAppManager implements Eve
     }
   }
 
-  protected void setCompletedAppsMax(int max) {
+  protected synchronized void setCompletedAppsMax(int max) {
     this.completedAppsMax = max;
   }
 

Modified: hadoop/common/branches/branch-0.23/hadoop-mapreduce-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/RMConfig.java
URL: http://svn.apache.org/viewvc/hadoop/common/branches/branch-0.23/hadoop-mapreduce-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/RMConfig.java?rev=1166840&r1=1166839&r2=1166840&view=diff
==============================================================================
--- hadoop/common/branches/branch-0.23/hadoop-mapreduce-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/RMConfig.java (original)
+++ hadoop/common/branches/branch-0.23/hadoop-mapreduce-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/RMConfig.java Thu Sep  8 18:28:14 2011
@@ -87,7 +87,7 @@ public class RMConfig {
   public static final String DEFAULT_RM_NODES_EXCLUDE_FILE = "";
 
   // the maximum number of completed applications RM keeps 
-  public static String EXPIRE_APPLICATIONS_COMPLETED_MAX =
+  public static final String EXPIRE_APPLICATIONS_COMPLETED_MAX =
     YarnConfiguration.RM_PREFIX + "expire.applications.completed.max";
   public static final int DEFAULT_EXPIRE_APPLICATIONS_COMPLETED_MAX = 10000;
 }

Modified: hadoop/common/branches/branch-0.23/hadoop-mapreduce-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/recovery/ZKStore.java
URL: http://svn.apache.org/viewvc/hadoop/common/branches/branch-0.23/hadoop-mapreduce-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/recovery/ZKStore.java?rev=1166840&r1=1166839&r2=1166840&view=diff
==============================================================================
--- hadoop/common/branches/branch-0.23/hadoop-mapreduce-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/recovery/ZKStore.java (original)
+++ hadoop/common/branches/branch-0.23/hadoop-mapreduce-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/recovery/ZKStore.java Thu Sep  8 18:28:14 2011
@@ -123,10 +123,9 @@ public class ZKStore implements Store {
   public synchronized void storeNode(RMNode node) throws IOException {
     /** create a storage node and store it in zk **/
     if (!doneWithRecovery) return;
-    NodeReportPBImpl nodeManagerInfo = createNodeManagerInfo(node);
-    // TODO FinBugs - will be fixed after the subsequent fixme
-    byte[] bytes = nodeManagerInfo.getProto().toByteArray();
     // TODO: FIXMEVinodkv
+//    NodeReportPBImpl nodeManagerInfo = createNodeManagerInfo(node);
+//    byte[] bytes = nodeManagerInfo.getProto().toByteArray();
 //    try {
 //      zkClient.create(NODES + Integer.toString(node.getNodeID().getId()), bytes, null,
 //          CreateMode.PERSISTENT);
@@ -476,12 +475,12 @@ public class ZKStore implements Store {
           continue;
         }
         int httpPort = Integer.valueOf(m.group(1));
-        // TODO: FindBugs Valid. Fix
-        RMNode nm = new RMNodeImpl(node.getNodeId(), null,
-            hostName, cmPort, httpPort,
-            ResourceTrackerService.resolve(node.getNodeId().getHost()), 
-            node.getCapability());
-        nodeManagers.add(nm);
+        // TODO: FindBugs warns passing null below. Commenting this for later.
+//        RMNode nm = new RMNodeImpl(node.getNodeId(), null,
+//            hostName, cmPort, httpPort,
+//            ResourceTrackerService.resolve(node.getNodeId().getHost()), 
+//            node.getCapability());
+//        nodeManagers.add(nm);
       }
       readLastNodeId();
       /* make sure we get all the applications */

Modified: hadoop/common/branches/branch-0.23/hadoop-mapreduce-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/SchedulerApp.java
URL: http://svn.apache.org/viewvc/hadoop/common/branches/branch-0.23/hadoop-mapreduce-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/SchedulerApp.java?rev=1166840&r1=1166839&r2=1166840&view=diff
==============================================================================
--- hadoop/common/branches/branch-0.23/hadoop-mapreduce-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/SchedulerApp.java (original)
+++ hadoop/common/branches/branch-0.23/hadoop-mapreduce-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/SchedulerApp.java Thu Sep  8 18:28:14 2011
@@ -278,10 +278,7 @@ public class SchedulerApp {
   }
 
   synchronized public void resetSchedulingOpportunities(Priority priority) {
-    Integer schedulingOpportunities = 
-        this.schedulingOpportunities.get(priority);
-    schedulingOpportunities = 0;
-    this.schedulingOpportunities.put(priority, schedulingOpportunities);
+    this.schedulingOpportunities.put(priority, Integer.valueOf(0));
   }
 
   synchronized public void addSchedulingOpportunity(Priority priority) {
@@ -305,9 +302,7 @@ public class SchedulerApp {
   }
 
   synchronized void resetReReservations(Priority priority) {
-    Integer reReservations = this.reReservations.get(priority);
-    reReservations = 0;
-    this.reReservations.put(priority, reReservations);
+    this.reReservations.put(priority, Integer.valueOf(0));
   }
 
   synchronized void addReReservation(Priority priority) {

Modified: hadoop/common/branches/branch-0.23/hadoop-mapreduce-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/capacity/CapacityScheduler.java
URL: http://svn.apache.org/viewvc/hadoop/common/branches/branch-0.23/hadoop-mapreduce-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/capacity/CapacityScheduler.java?rev=1166840&r1=1166839&r2=1166840&view=diff
==============================================================================
--- hadoop/common/branches/branch-0.23/hadoop-mapreduce-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/capacity/CapacityScheduler.java (original)
+++ hadoop/common/branches/branch-0.23/hadoop-mapreduce-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/capacity/CapacityScheduler.java Thu Sep  8 18:28:14 2011
@@ -80,14 +80,14 @@ implements ResourceScheduler, CapacitySc
 
   private static final Log LOG = LogFactory.getLog(CapacityScheduler.class);
 
-  private Queue root;
+  private CSQueue root;
 
   private final static List<Container> EMPTY_CONTAINER_LIST = 
     new ArrayList<Container>();
 
-  static final Comparator<Queue> queueComparator = new Comparator<Queue>() {
+  static final Comparator<CSQueue> queueComparator = new Comparator<CSQueue>() {
     @Override
-    public int compare(Queue q1, Queue q2) {
+    public int compare(CSQueue q1, CSQueue q2) {
       if (q1.getUtilization() < q2.getUtilization()) {
         return -1;
       } else if (q1.getUtilization() > q2.getUtilization()) {
@@ -110,7 +110,7 @@ implements ResourceScheduler, CapacitySc
   private ContainerTokenSecretManager containerTokenSecretManager;
   private RMContext rmContext;
 
-  private Map<String, Queue> queues = new ConcurrentHashMap<String, Queue>();
+  private Map<String, CSQueue> queues = new ConcurrentHashMap<String, CSQueue>();
 
   private Map<NodeId, SchedulerNode> nodes = 
       new ConcurrentHashMap<NodeId, SchedulerNode>();
@@ -127,7 +127,7 @@ implements ResourceScheduler, CapacitySc
 
   private boolean initialized = false;
 
-  public Queue getRootQueue() {
+  public CSQueue getRootQueue() {
     return root;
   }
 
@@ -207,7 +207,7 @@ implements ResourceScheduler, CapacitySc
     CapacitySchedulerConfiguration.PREFIX + ROOT;
 
   static class QueueHook {
-    public Queue hook(Queue queue) {
+    public CSQueue hook(CSQueue queue) {
       return queue;
     }
   }
@@ -225,8 +225,8 @@ implements ResourceScheduler, CapacitySc
   private void reinitializeQueues(CapacitySchedulerConfiguration conf) 
   throws IOException {
     // Parse new queues
-    Map<String, Queue> newQueues = new HashMap<String, Queue>();
-    Queue newRoot = 
+    Map<String, CSQueue> newQueues = new HashMap<String, CSQueue>();
+    CSQueue newRoot = 
         parseQueue(this, conf, null, ROOT, newQueues, queues, 
             queueComparator, applicationComparator, noop);
     
@@ -247,7 +247,7 @@ implements ResourceScheduler, CapacitySc
    */
   @Lock(CapacityScheduler.class)
   private void validateExistingQueues(
-      Map<String, Queue> queues, Map<String, Queue> newQueues) 
+      Map<String, CSQueue> queues, Map<String, CSQueue> newQueues) 
   throws IOException {
     for (String queue : queues.keySet()) {
       if (!newQueues.containsKey(queue)) {
@@ -264,11 +264,11 @@ implements ResourceScheduler, CapacitySc
    */
   @Lock(CapacityScheduler.class)
   private void addNewQueues(
-      Map<String, Queue> queues, Map<String, Queue> newQueues) 
+      Map<String, CSQueue> queues, Map<String, CSQueue> newQueues) 
   {
-    for (Map.Entry<String, Queue> e : newQueues.entrySet()) {
+    for (Map.Entry<String, CSQueue> e : newQueues.entrySet()) {
       String queueName = e.getKey();
-      Queue queue = e.getValue();
+      CSQueue queue = e.getValue();
       if (!queues.containsKey(queueName)) {
         queues.put(queueName, queue);
       }
@@ -276,15 +276,15 @@ implements ResourceScheduler, CapacitySc
   }
   
   @Lock(CapacityScheduler.class)
-  static Queue parseQueue(
+  static CSQueue parseQueue(
       CapacitySchedulerContext csContext, 
       CapacitySchedulerConfiguration conf, 
-      Queue parent, String queueName, Map<String, Queue> queues,
-      Map<String, Queue> oldQueues, 
-      Comparator<Queue> queueComparator,
+      CSQueue parent, String queueName, Map<String, CSQueue> queues,
+      Map<String, CSQueue> oldQueues, 
+      Comparator<CSQueue> queueComparator,
       Comparator<SchedulerApp> applicationComparator,
       QueueHook hook) {
-    Queue queue;
+    CSQueue queue;
     String[] childQueueNames = 
       conf.getQueues((parent == null) ? 
           queueName : (parent.getQueuePath()+"."+queueName));
@@ -306,9 +306,9 @@ implements ResourceScheduler, CapacitySc
       // Used only for unit tests
       queue = hook.hook(parentQueue);
       
-      List<Queue> childQueues = new ArrayList<Queue>();
+      List<CSQueue> childQueues = new ArrayList<CSQueue>();
       for (String childQueueName : childQueueNames) {
-        Queue childQueue = 
+        CSQueue childQueue = 
           parseQueue(csContext, conf, queue, childQueueName, 
               queues, oldQueues, queueComparator, applicationComparator, hook);
         childQueues.add(childQueue);
@@ -322,7 +322,7 @@ implements ResourceScheduler, CapacitySc
     return queue;
   }
 
-  synchronized Queue getQueue(String queueName) {
+  synchronized CSQueue getQueue(String queueName) {
     return queues.get(queueName);
   }
   
@@ -331,7 +331,7 @@ implements ResourceScheduler, CapacitySc
           String queueName, String user) {
 
     // Sanity checks
-    Queue queue = getQueue(queueName);
+    CSQueue queue = getQueue(queueName);
     if (queue == null) {
       String message = "Application " + applicationAttemptId + 
       " submitted by user " + user + " to unknown queue: " + queueName;
@@ -405,7 +405,7 @@ implements ResourceScheduler, CapacitySc
     
     // Inform the queue
     String queueName = application.getQueue().getQueueName();
-    Queue queue = queues.get(queueName);
+    CSQueue queue = queues.get(queueName);
     if (!(queue instanceof LeafQueue)) {
       LOG.error("Cannot finish application " + "from non-leaf queue: "
           + queueName);
@@ -479,7 +479,7 @@ implements ResourceScheduler, CapacitySc
   public QueueInfo getQueueInfo(String queueName, 
       boolean includeChildQueues, boolean recursive) 
   throws IOException {
-    Queue queue = null;
+    CSQueue queue = null;
 
     synchronized (this) {
       queue = this.queues.get(queueName); 

Modified: hadoop/common/branches/branch-0.23/hadoop-mapreduce-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/capacity/LeafQueue.java
URL: http://svn.apache.org/viewvc/hadoop/common/branches/branch-0.23/hadoop-mapreduce-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/capacity/LeafQueue.java?rev=1166840&r1=1166839&r2=1166840&view=diff
==============================================================================
--- hadoop/common/branches/branch-0.23/hadoop-mapreduce-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/capacity/LeafQueue.java (original)
+++ hadoop/common/branches/branch-0.23/hadoop-mapreduce-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/capacity/LeafQueue.java Thu Sep  8 18:28:14 2011
@@ -64,11 +64,11 @@ import org.apache.hadoop.yarn.util.Build
 
 @Private
 @Unstable
-public class LeafQueue implements Queue {
+public class LeafQueue implements CSQueue {
   private static final Log LOG = LogFactory.getLog(LeafQueue.class);
 
   private final String queueName;
-  private Queue parent;
+  private CSQueue parent;
   private float capacity;
   private float absoluteCapacity;
   private float maximumCapacity;
@@ -119,8 +119,8 @@ public class LeafQueue implements Queue 
   final static int DEFAULT_AM_RESOURCE = 2 * 1024;
   
   public LeafQueue(CapacitySchedulerContext cs, 
-      String queueName, Queue parent, 
-      Comparator<SchedulerApp> applicationComparator, Queue old) {
+      String queueName, CSQueue parent, 
+      Comparator<SchedulerApp> applicationComparator, CSQueue old) {
     this.scheduler = cs;
     this.queueName = queueName;
     this.parent = parent;
@@ -192,7 +192,7 @@ public class LeafQueue implements Queue 
       float maxAMResourcePercent, float absoluteCapacity) {
     return 
         Math.max(
-            (int)((clusterResource.getMemory() / DEFAULT_AM_RESOURCE) * 
+            (int)((clusterResource.getMemory() / (float)DEFAULT_AM_RESOURCE) * 
                    maxAMResourcePercent * absoluteCapacity), 
             1);
   }
@@ -271,7 +271,7 @@ public class LeafQueue implements Queue 
   }
 
   @Override
-  public Queue getParent() {
+  public CSQueue getParent() {
     return parent;
   }
 
@@ -313,15 +313,15 @@ public class LeafQueue implements Queue 
     return maxApplications;
   }
 
-  public int getMaxApplicationsPerUser() {
+  public synchronized int getMaxApplicationsPerUser() {
     return maxApplicationsPerUser;
   }
 
-  public int getMaximumActiveApplications() {
+  public synchronized int getMaximumActiveApplications() {
     return maxActiveApplications;
   }
 
-  public int getMaximumActiveApplicationsPerUser() {
+  public synchronized int getMaximumActiveApplicationsPerUser() {
     return maxActiveApplicationsPerUser;
   }
 
@@ -341,7 +341,7 @@ public class LeafQueue implements Queue 
   }
 
   @Override
-  public List<Queue> getChildQueues() {
+  public List<CSQueue> getChildQueues() {
     return null;
   }
 
@@ -381,7 +381,7 @@ public class LeafQueue implements Queue 
     this.userLimitFactor = userLimitFactor;
   }
 
-  synchronized void setParentQueue(Queue parent) {
+  synchronized void setParentQueue(CSQueue parent) {
     this.parent = parent;
   }
   
@@ -423,12 +423,12 @@ public class LeafQueue implements Queue 
   }
 
   @Private
-  public int getUserLimit() {
+  public synchronized int getUserLimit() {
     return userLimit;
   }
 
   @Private
-  public float getUserLimitFactor() {
+  public synchronized float getUserLimitFactor() {
     return userLimitFactor;
   }
 
@@ -480,7 +480,7 @@ public class LeafQueue implements Queue 
   }
 
   @Override
-  public synchronized void reinitialize(Queue queue, Resource clusterResource) 
+  public synchronized void reinitialize(CSQueue queue, Resource clusterResource) 
   throws IOException {
     // Sanity check
     if (!(queue instanceof LeafQueue) || 
@@ -493,9 +493,10 @@ public class LeafQueue implements Queue 
     setupQueueConfigs(leafQueue.capacity, leafQueue.absoluteCapacity, 
         leafQueue.maximumCapacity, leafQueue.absoluteMaxCapacity, 
         leafQueue.userLimit, leafQueue.userLimitFactor, 
-        leafQueue.maxApplications, leafQueue.maxApplicationsPerUser,
-        leafQueue.maxActiveApplications, 
-        leafQueue.maxActiveApplicationsPerUser,
+        leafQueue.maxApplications,
+        leafQueue.getMaxApplicationsPerUser(),
+        leafQueue.getMaximumActiveApplications(), 
+        leafQueue.getMaximumActiveApplicationsPerUser(),
         leafQueue.state, leafQueue.acls);
     
     updateResource(clusterResource);
@@ -900,7 +901,7 @@ public class LeafQueue implements Queue 
       // Protect against corner case where you need the whole node with
       // Math.min(nodeFactor, minimumAllocationFactor)
       starvation = 
-          (int)((application.getReReservations(priority) / reservedContainers) * 
+          (int)((application.getReReservations(priority) / (float)reservedContainers) * 
                 (1.0f - (Math.min(nodeFactor, getMinimumAllocationFactor())))
                );
       

Modified: hadoop/common/branches/branch-0.23/hadoop-mapreduce-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/capacity/ParentQueue.java
URL: http://svn.apache.org/viewvc/hadoop/common/branches/branch-0.23/hadoop-mapreduce-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/capacity/ParentQueue.java?rev=1166840&r1=1166839&r2=1166840&view=diff
==============================================================================
--- hadoop/common/branches/branch-0.23/hadoop-mapreduce-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/capacity/ParentQueue.java (original)
+++ hadoop/common/branches/branch-0.23/hadoop-mapreduce-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/capacity/ParentQueue.java Thu Sep  8 18:28:14 2011
@@ -53,11 +53,11 @@ import org.apache.hadoop.yarn.server.res
 
 @Private
 @Evolving
-public class ParentQueue implements Queue {
+public class ParentQueue implements CSQueue {
 
   private static final Log LOG = LogFactory.getLog(ParentQueue.class);
 
-  private final Queue parent;
+  private final CSQueue parent;
   private final String queueName;
   
   private float capacity;
@@ -68,8 +68,8 @@ public class ParentQueue implements Queu
   private float usedCapacity = 0.0f;
   private float utilization = 0.0f;
 
-  private final Set<Queue> childQueues;
-  private final Comparator<Queue> queueComparator;
+  private final Set<CSQueue> childQueues;
+  private final Comparator<CSQueue> queueComparator;
   
   private Resource usedResources = 
     Resources.createResource(0);
@@ -94,7 +94,7 @@ public class ParentQueue implements Queu
     RecordFactoryProvider.getRecordFactory(null);
 
   public ParentQueue(CapacitySchedulerContext cs, 
-      String queueName, Comparator<Queue> comparator, Queue parent, Queue old) {
+      String queueName, Comparator<CSQueue> comparator, CSQueue parent, CSQueue old) {
     minimumAllocation = cs.getMinimumResourceCapability();
     
     this.parent = parent;
@@ -140,7 +140,7 @@ public class ParentQueue implements Queu
         maximumCapacity, absoluteMaxCapacity, state, acls);
     
     this.queueComparator = comparator;
-    this.childQueues = new TreeSet<Queue>(comparator);
+    this.childQueues = new TreeSet<CSQueue>(comparator);
 
     LOG.info("Initialized parent-queue " + queueName + 
         " name=" + queueName + 
@@ -180,11 +180,11 @@ public class ParentQueue implements Queu
   }
 
   private static float PRECISION = 0.005f; // 0.05% precision
-  void setChildQueues(Collection<Queue> childQueues) {
+  void setChildQueues(Collection<CSQueue> childQueues) {
     
     // Validate
     float childCapacities = 0;
-    for (Queue queue : childQueues) {
+    for (CSQueue queue : childQueues) {
       childCapacities += queue.getCapacity();
     }
     float delta = Math.abs(1.0f - childCapacities);  // crude way to check
@@ -200,7 +200,7 @@ public class ParentQueue implements Queu
   }
   
   @Override
-  public Queue getParent() {
+  public CSQueue getParent() {
     return parent;
   }
 
@@ -251,8 +251,8 @@ public class ParentQueue implements Queu
   }
 
   @Override
-  public synchronized List<Queue> getChildQueues() {
-    return new ArrayList<Queue>(childQueues);
+  public synchronized List<CSQueue> getChildQueues() {
+    return new ArrayList<CSQueue>(childQueues);
   }
 
   public synchronized int getNumContainers() {
@@ -280,7 +280,7 @@ public class ParentQueue implements Queu
 
     List<QueueInfo> childQueuesInfo = new ArrayList<QueueInfo>();
     if (includeChildQueues) {
-      for (Queue child : childQueues) {
+      for (CSQueue child : childQueues) {
         // Get queue information recursively?
         childQueuesInfo.add(
             child.getQueueInfo(recursive, recursive));
@@ -319,7 +319,7 @@ public class ParentQueue implements Queu
     userAcls.add(getUserAclInfo(user));
     
     // Add children queue acls
-    for (Queue child : childQueues) {
+    for (CSQueue child : childQueues) {
       userAcls.addAll(child.getQueueUserAclInfo(user));
     }
     return userAcls;
@@ -333,7 +333,7 @@ public class ParentQueue implements Queu
   }
   
   @Override
-  public synchronized void reinitialize(Queue queue, Resource clusterResource)
+  public synchronized void reinitialize(CSQueue queue, Resource clusterResource)
   throws IOException {
     // Sanity check
     if (!(queue instanceof ParentQueue) ||
@@ -346,13 +346,13 @@ public class ParentQueue implements Queu
 
     // Re-configure existing child queues and add new ones
     // The CS has already checked to ensure all existing child queues are present!
-    Map<String, Queue> currentChildQueues = getQueues(childQueues);
-    Map<String, Queue> newChildQueues = getQueues(parentQueue.childQueues);
-    for (Map.Entry<String, Queue> e : newChildQueues.entrySet()) {
+    Map<String, CSQueue> currentChildQueues = getQueues(childQueues);
+    Map<String, CSQueue> newChildQueues = getQueues(parentQueue.childQueues);
+    for (Map.Entry<String, CSQueue> e : newChildQueues.entrySet()) {
       String newChildQueueName = e.getKey();
-      Queue newChildQueue = e.getValue();
+      CSQueue newChildQueue = e.getValue();
 
-      Queue childQueue = currentChildQueues.get(newChildQueueName);
+      CSQueue childQueue = currentChildQueues.get(newChildQueueName);
       if (childQueue != null){
         childQueue.reinitialize(newChildQueue, clusterResource);
         LOG.info(getQueueName() + ": re-configured queue: " + childQueue);
@@ -375,9 +375,9 @@ public class ParentQueue implements Queu
     updateResource(clusterResource);
   }
 
-  Map<String, Queue> getQueues(Set<Queue> queues) {
-    Map<String, Queue> queuesMap = new HashMap<String, Queue>();
-    for (Queue queue : queues) {
+  Map<String, CSQueue> getQueues(Set<CSQueue> queues) {
+    Map<String, CSQueue> queuesMap = new HashMap<String, CSQueue>();
+    for (CSQueue queue : queues) {
       queuesMap.put(queue.getQueueName(), queue);
     }
     return queuesMap;
@@ -568,8 +568,8 @@ public class ParentQueue implements Queu
     printChildQueues();
 
     // Try to assign to most 'under-served' sub-queue
-    for (Iterator<Queue> iter=childQueues.iterator(); iter.hasNext();) {
-      Queue childQueue = iter.next();
+    for (Iterator<CSQueue> iter=childQueues.iterator(); iter.hasNext();) {
+      CSQueue childQueue = iter.next();
       LOG.info("DEBUG --- Trying to assign to" +
       		" queue: " + childQueue.getQueuePath() + 
       		" stats: " + childQueue);
@@ -595,7 +595,7 @@ public class ParentQueue implements Queu
 
   String getChildQueuesToPrint() {
     StringBuilder sb = new StringBuilder();
-    for (Queue q : childQueues) {
+    for (CSQueue q : childQueues) {
       sb.append(q.getQueuePath() + "(" + q.getUtilization() + "), ");
     }
     return sb.toString();
@@ -648,7 +648,7 @@ public class ParentQueue implements Queu
   @Override
   public synchronized void updateClusterResource(Resource clusterResource) {
     // Update all children
-    for (Queue childQueue : childQueues) {
+    for (CSQueue childQueue : childQueues) {
       childQueue.updateClusterResource(clusterResource);
     }
   }

Modified: hadoop/common/branches/branch-0.23/hadoop-mapreduce-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/webapp/CapacitySchedulerPage.java
URL: http://svn.apache.org/viewvc/hadoop/common/branches/branch-0.23/hadoop-mapreduce-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/webapp/CapacitySchedulerPage.java?rev=1166840&r1=1166839&r2=1166840&view=diff
==============================================================================
--- hadoop/common/branches/branch-0.23/hadoop-mapreduce-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/webapp/CapacitySchedulerPage.java (original)
+++ hadoop/common/branches/branch-0.23/hadoop-mapreduce-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/webapp/CapacitySchedulerPage.java Thu Sep  8 18:28:14 2011
@@ -24,7 +24,7 @@ import com.google.inject.servlet.Request
 import org.apache.hadoop.yarn.server.resourcemanager.ResourceManager;
 import org.apache.hadoop.yarn.server.resourcemanager.scheduler.capacity.CapacityScheduler;
 import org.apache.hadoop.yarn.server.resourcemanager.scheduler.capacity.ParentQueue;
-import org.apache.hadoop.yarn.server.resourcemanager.scheduler.capacity.Queue;
+import org.apache.hadoop.yarn.server.resourcemanager.scheduler.capacity.CSQueue;
 import org.apache.hadoop.yarn.webapp.SubView;
 import org.apache.hadoop.yarn.webapp.hamlet.Hamlet;
 import org.apache.hadoop.yarn.webapp.hamlet.Hamlet.*;
@@ -43,7 +43,7 @@ class CapacitySchedulerPage extends RmVi
 
   @RequestScoped
   static class Parent {
-    Queue queue;
+    CSQueue queue;
   }
 
   public static class QueueBlock extends HtmlBlock {
@@ -56,8 +56,8 @@ class CapacitySchedulerPage extends RmVi
     @Override
     public void render(Block html) {
       UL<Hamlet> ul = html.ul();
-      Queue parentQueue = parent.queue;
-      for (Queue queue : parentQueue.getChildQueues()) {
+      CSQueue parentQueue = parent.queue;
+      for (CSQueue queue : parentQueue.getChildQueues()) {
         float used = queue.getUsedCapacity();
         float set = queue.getCapacity();
         float delta = Math.abs(set - used) + 0.001f;
@@ -109,7 +109,7 @@ class CapacitySchedulerPage extends RmVi
               span().$style(Q_END)._("100% ")._().
               span(".q", "default")._()._();
       } else {
-        Queue root = cs.getRootQueue();
+        CSQueue root = cs.getRootQueue();
         parent.queue = root;
         float used = root.getUsedCapacity();
         float set = root.getCapacity();

Modified: hadoop/common/branches/branch-0.23/hadoop-mapreduce-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/capacity/TestApplicationLimits.java
URL: http://svn.apache.org/viewvc/hadoop/common/branches/branch-0.23/hadoop-mapreduce-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/capacity/TestApplicationLimits.java?rev=1166840&r1=1166839&r2=1166840&view=diff
==============================================================================
--- hadoop/common/branches/branch-0.23/hadoop-mapreduce-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/capacity/TestApplicationLimits.java (original)
+++ hadoop/common/branches/branch-0.23/hadoop-mapreduce-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/capacity/TestApplicationLimits.java Thu Sep  8 18:28:14 2011
@@ -38,8 +38,8 @@ public class TestApplicationLimits {
     when(csContext.getMaximumResourceCapability()).thenReturn(Resources.createResource(16*GB));
     when(csContext.getClusterResources()).thenReturn(Resources.createResource(10 * 16 * GB));
     
-    Map<String, Queue> queues = new HashMap<String, Queue>();
-    Queue root = 
+    Map<String, CSQueue> queues = new HashMap<String, CSQueue>();
+    CSQueue root = 
         CapacityScheduler.parseQueue(csContext, csConf, null, "root", 
             queues, queues, 
             CapacityScheduler.queueComparator, 
@@ -108,8 +108,8 @@ public class TestApplicationLimits {
     Resource clusterResource = Resources.createResource(100 * 16 * GB);
     when(csContext.getClusterResources()).thenReturn(clusterResource);
     
-    Map<String, Queue> queues = new HashMap<String, Queue>();
-    Queue root = 
+    Map<String, CSQueue> queues = new HashMap<String, CSQueue>();
+    CSQueue root = 
         CapacityScheduler.parseQueue(csContext, csConf, null, "root", 
             queues, queues, 
             CapacityScheduler.queueComparator, 

Modified: hadoop/common/branches/branch-0.23/hadoop-mapreduce-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/capacity/TestLeafQueue.java
URL: http://svn.apache.org/viewvc/hadoop/common/branches/branch-0.23/hadoop-mapreduce-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/capacity/TestLeafQueue.java?rev=1166840&r1=1166839&r2=1166840&view=diff
==============================================================================
--- hadoop/common/branches/branch-0.23/hadoop-mapreduce-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/capacity/TestLeafQueue.java (original)
+++ hadoop/common/branches/branch-0.23/hadoop-mapreduce-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/capacity/TestLeafQueue.java Thu Sep  8 18:28:14 2011
@@ -65,8 +65,8 @@ public class TestLeafQueue {
   CapacitySchedulerConfiguration csConf;
   CapacitySchedulerContext csContext;
   
-  Queue root;
-  Map<String, Queue> queues = new HashMap<String, Queue>();
+  CSQueue root;
+  Map<String, CSQueue> queues = new HashMap<String, CSQueue>();
   
   final static int GB = 1024;
   final static String DEFAULT_RACK = "/default";
@@ -145,7 +145,7 @@ public class TestLeafQueue {
               any(Resource.class));
     
     // 2. Stub out LeafQueue.parent.completedContainer
-    Queue parent = queue.getParent();
+    CSQueue parent = queue.getParent();
     doNothing().when(parent).completedContainer(
         any(Resource.class), any(SchedulerApp.class), any(SchedulerNode.class), 
         any(RMContainer.class), any(RMContainerEventType.class));

Modified: hadoop/common/branches/branch-0.23/hadoop-mapreduce-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/capacity/TestParentQueue.java
URL: http://svn.apache.org/viewvc/hadoop/common/branches/branch-0.23/hadoop-mapreduce-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/capacity/TestParentQueue.java?rev=1166840&r1=1166839&r2=1166840&view=diff
==============================================================================
--- hadoop/common/branches/branch-0.23/hadoop-mapreduce-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/capacity/TestParentQueue.java (original)
+++ hadoop/common/branches/branch-0.23/hadoop-mapreduce-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/capacity/TestParentQueue.java Thu Sep  8 18:28:14 2011
@@ -81,7 +81,7 @@ public class TestParentQueue {
     LOG.info("Setup top-level queues a and b");
   }
 
-  private void stubQueueAllocation(final Queue queue, 
+  private void stubQueueAllocation(final CSQueue queue, 
       final Resource clusterResource, final SchedulerNode node, 
       final int allocation) {
     
@@ -121,7 +121,7 @@ public class TestParentQueue {
     when(queue).assignContainers(eq(clusterResource), eq(node));
   }
   
-  private float computeQueueUtilization(Queue queue, 
+  private float computeQueueUtilization(CSQueue queue, 
       int expectedMemory, Resource clusterResource) {
     return (expectedMemory / 
         (clusterResource.getMemory() * queue.getAbsoluteCapacity()));
@@ -132,8 +132,8 @@ public class TestParentQueue {
     // Setup queue configs
     setupSingleLevelQueues(csConf);
     
-    Map<String, Queue> queues = new HashMap<String, Queue>();
-    Queue root = 
+    Map<String, CSQueue> queues = new HashMap<String, CSQueue>();
+    CSQueue root = 
         CapacityScheduler.parseQueue(csContext, csConf, null, 
             CapacityScheduler.ROOT, queues, queues, 
             CapacityScheduler.queueComparator, 
@@ -270,8 +270,8 @@ public class TestParentQueue {
     // Setup queue configs
     setupMultiLevelQueues(csConf);
     
-    Map<String, Queue> queues = new HashMap<String, Queue>();
-    Queue root = 
+    Map<String, CSQueue> queues = new HashMap<String, CSQueue>();
+    CSQueue root = 
         CapacityScheduler.parseQueue(csContext, csConf, null, 
             CapacityScheduler.ROOT, queues, queues, 
             CapacityScheduler.queueComparator, 
@@ -294,17 +294,17 @@ public class TestParentQueue {
     when(csContext.getNumClusterNodes()).thenReturn(numNodes);
 
     // Start testing
-    Queue a = queues.get(A);
-    Queue b = queues.get(B);
-    Queue c = queues.get(C);
-    Queue d = queues.get(D);
-
-    Queue a1 = queues.get(A1);
-    Queue a2 = queues.get(A2);
-
-    Queue b1 = queues.get(B1);
-    Queue b2 = queues.get(B2);
-    Queue b3 = queues.get(B3);
+    CSQueue a = queues.get(A);
+    CSQueue b = queues.get(B);
+    CSQueue c = queues.get(C);
+    CSQueue d = queues.get(D);
+
+    CSQueue a1 = queues.get(A1);
+    CSQueue a2 = queues.get(A2);
+
+    CSQueue b1 = queues.get(B1);
+    CSQueue b2 = queues.get(B2);
+    CSQueue b3 = queues.get(B3);
 
     final float delta = 0.0001f;
     

Modified: hadoop/common/branches/branch-0.23/hadoop-mapreduce-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/capacity/TestUtils.java
URL: http://svn.apache.org/viewvc/hadoop/common/branches/branch-0.23/hadoop-mapreduce-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/capacity/TestUtils.java?rev=1166840&r1=1166839&r2=1166840&view=diff
==============================================================================
--- hadoop/common/branches/branch-0.23/hadoop-mapreduce-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/capacity/TestUtils.java (original)
+++ hadoop/common/branches/branch-0.23/hadoop-mapreduce-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/capacity/TestUtils.java Thu Sep  8 18:28:14 2011
@@ -85,7 +85,7 @@ public class TestUtils {
    */
   static class SpyHook extends CapacityScheduler.QueueHook {
     @Override
-    public Queue hook(Queue queue) {
+    public CSQueue hook(CSQueue queue) {
       return spy(queue);
     }
   }

Modified: hadoop/common/branches/branch-0.23/hadoop-mapreduce-project/src/test/findbugsExcludeFile.xml
URL: http://svn.apache.org/viewvc/hadoop/common/branches/branch-0.23/hadoop-mapreduce-project/src/test/findbugsExcludeFile.xml?rev=1166840&r1=1166839&r2=1166840&view=diff
==============================================================================
--- hadoop/common/branches/branch-0.23/hadoop-mapreduce-project/src/test/findbugsExcludeFile.xml (original)
+++ hadoop/common/branches/branch-0.23/hadoop-mapreduce-project/src/test/findbugsExcludeFile.xml Thu Sep  8 18:28:14 2011
@@ -388,9 +388,4 @@
        <Field name="started" />
        <Bug pattern="IS2_INCONSISTENT_SYNC" />
     </Match>
-    <Match>
-       <Class name="org.apache.hadoop.mapreduce.security.token.DelegationTokenRenewal$DelegationTokenCancelThread" />
-       <Method name="run" />
-       <Bug pattern="DM_EXIT" />
-    </Match>
- </FindBugsFilter>
+  </FindBugsFilter>