MAPREDUCE-2908. Fix all findbugs warnings. Contributed by Vinod K V.

git-svn-id: https://svn.apache.org/repos/asf/hadoop/common/trunk@1166838 13f79535-47bb-0310-9956-ffa450edef68
This commit is contained in:
Arun Murthy 2011-09-08 18:26:20 +00:00
parent ae5e8e0104
commit 1f46b991da
25 changed files with 162 additions and 159 deletions

View File

@ -1225,6 +1225,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

View File

@ -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" />
@ -166,6 +172,14 @@
</Or>
<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" />

View File

@ -157,6 +157,7 @@ public static TaskAttemptStateUI taskAttemptState(String attemptStateStr) {
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 static void setInitialClasspath(
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 static void setInitialClasspath(
if (classpathFileStream != null) {
classpathFileStream.close();
}
if (reader != null) {
reader.close();
}
}
// TODO: Remove duplicates.
}

View File

@ -868,15 +868,6 @@ protected long getCounter(org.apache.hadoop.mapreduce.Counters cntrs,
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.

View File

@ -97,7 +97,7 @@ public abstract class ResourceCalculatorPlugin extends Configured {
@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;

View File

@ -149,8 +149,7 @@ private MRClientProtocol getProxy() throws YarnRemoteException {
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 @@ private MRClientProtocol getProxy() throws YarnRemoteException {
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 @@ JobStatus getJobStatus(JobID oldJobID) throws YarnRemoteException {
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 =

View File

@ -263,10 +263,6 @@ public synchronized ByteBuffer getMeta() {
}
}
Shuffle createShuffle() {
return new Shuffle(getConfig());
}
class HttpPipelineFactory implements ChannelPipelineFactory {
final Shuffle SHUFFLE;
@ -296,10 +292,12 @@ class Shuffle extends SimpleChannelUpstreamHandler {
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 void messageReceived(ChannelHandlerContext ctx, MessageEvent evt)
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);

View File

@ -65,6 +65,11 @@
<Class name="~org\.apache\.hadoop\.yarn\.server\.resourcemanager\.rmnode\.RMNodeImpl.*" />
<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" />

View File

@ -97,7 +97,7 @@ public abstract class ResourceCalculatorPlugin extends Configured {
@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;

View File

@ -67,7 +67,6 @@ public static class Builder<T> {
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 Builder<T> with(Configuration conf) {
return this;
}
public Builder<T> with(Module... modules) {
this.modules = modules; // OK
return this;
}
public Builder<T> inDevMode() {
devMode = true;
return this;

View File

@ -311,7 +311,14 @@ public StopContainerResponse stopContainer(StopContainerRequest request)
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);

View File

@ -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 static void logAppSummary(RMApp app) {
}
}
protected void setCompletedAppsMax(int max) {
protected synchronized void setCompletedAppsMax(int max) {
this.completedAppsMax = max;
}

View File

@ -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;
}

View File

@ -123,10 +123,9 @@ private NodeReportPBImpl createNodeManagerInfo(RMNode rmNode) {
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 @@ private void load() throws IOException {
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 */

View File

@ -278,10 +278,7 @@ public synchronized RMContainer getRMContainer(ContainerId id) {
}
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 @@ synchronized public int getSchedulingOpportunities(Priority priority) {
}
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) {

View File

@ -35,18 +35,18 @@
import org.apache.hadoop.yarn.server.resourcemanager.scheduler.SchedulerNode;
/**
* Queue represents a node in the tree of
* <code>CSQueue</code> represents a node in the tree of
* hierarchical queues in the {@link CapacityScheduler}.
*/
@Stable
@Private
public interface Queue
public interface CSQueue
extends org.apache.hadoop.yarn.server.resourcemanager.scheduler.Queue {
/**
* Get the parent <code>Queue</code>.
* @return the parent queue
*/
public Queue getParent();
public CSQueue getParent();
/**
* Get the queue name.
@ -122,7 +122,7 @@ public interface Queue
* Get child queues
* @return child queues
*/
public List<Queue> getChildQueues();
public List<CSQueue> getChildQueues();
/**
* Check if the <code>user</code> has permission to perform the operation
@ -183,7 +183,7 @@ public void completedContainer(Resource clusterResource,
* @param queue new queue to re-initalize from
* @param clusterResource resources in the cluster
*/
public void reinitialize(Queue queue, Resource clusterResource)
public void reinitialize(CSQueue queue, Resource clusterResource)
throws IOException;
/**

View File

@ -80,14 +80,14 @@ public class CapacityScheduler
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 @@ public int compare(SchedulerApp a1, SchedulerApp a2) {
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 @@ public int compare(SchedulerApp a1, SchedulerApp a2) {
private boolean initialized = false;
public Queue getRootQueue() {
public CSQueue getRootQueue() {
return root;
}
@ -207,7 +207,7 @@ public synchronized void reinitialize(Configuration conf,
CapacitySchedulerConfiguration.PREFIX + ROOT;
static class QueueHook {
public Queue hook(Queue queue) {
public CSQueue hook(CSQueue queue) {
return queue;
}
}
@ -225,8 +225,8 @@ private void initializeQueues(CapacitySchedulerConfiguration conf) {
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 @@ private void reinitializeQueues(CapacitySchedulerConfiguration conf)
*/
@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 @@ private void validateExistingQueues(
*/
@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 @@ private void addNewQueues(
}
@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 @@ static Queue parseQueue(
// 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 @@ static Queue parseQueue(
return queue;
}
synchronized Queue getQueue(String queueName) {
synchronized CSQueue getQueue(String queueName) {
return queues.get(queueName);
}
@ -331,7 +331,7 @@ synchronized Queue getQueue(String queueName) {
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 @@ private synchronized void doneApplication(
// 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 @@ public Allocation allocate(ApplicationAttemptId applicationAttemptId,
public QueueInfo getQueueInfo(String queueName,
boolean includeChildQueues, boolean recursive)
throws IOException {
Queue queue = null;
CSQueue queue = null;
synchronized (this) {
queue = this.queues.get(queueName);

View File

@ -64,11 +64,11 @@
@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 @@ private int computeMaxActiveApplications(Resource clusterResource,
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 synchronized float getAbsoluteMaximumCapacity() {
}
@Override
public Queue getParent() {
public CSQueue getParent() {
return parent;
}
@ -313,15 +313,15 @@ public int getMaxApplications() {
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 synchronized float getUtilization() {
}
@Override
public List<Queue> getChildQueues() {
public List<CSQueue> getChildQueues() {
return null;
}
@ -381,7 +381,7 @@ synchronized void setUserLimitFactor(int userLimitFactor) {
this.userLimitFactor = userLimitFactor;
}
synchronized void setParentQueue(Queue parent) {
synchronized void setParentQueue(CSQueue parent) {
this.parent = parent;
}
@ -423,12 +423,12 @@ public synchronized QueueState getState() {
}
@Private
public int getUserLimit() {
public synchronized int getUserLimit() {
return userLimit;
}
@Private
public float getUserLimitFactor() {
public synchronized float getUserLimitFactor() {
return userLimitFactor;
}
@ -480,7 +480,7 @@ private synchronized User getUser(String userName) {
}
@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 synchronized void reinitialize(Queue queue, Resource clusterResource)
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 @@ boolean needContainers(SchedulerApp application, Priority priority, Resource req
// 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())))
);

View File

@ -53,11 +53,11 @@
@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 Queue {
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 Queue {
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 ParentQueue(CapacitySchedulerContext cs,
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 @@ private synchronized void setupQueueConfigs(
}
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 @@ void setChildQueues(Collection<Queue> childQueues) {
}
@Override
public Queue getParent() {
public CSQueue getParent() {
return parent;
}
@ -251,8 +251,8 @@ public synchronized float getUtilization() {
}
@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 synchronized QueueInfo getQueueInfo(
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 synchronized List<QueueUserACLInfo> getQueueUserAclInfo(
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 String toString() {
}
@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 synchronized void reinitialize(Queue queue, Resource clusterResource)
// 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 synchronized void reinitialize(Queue queue, Resource clusterResource)
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 @@ synchronized Resource assignContainersToChildQueues(Resource cluster,
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 @@ synchronized Resource assignContainersToChildQueues(Resource cluster,
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 @@ synchronized void releaseResource(Resource clusterResource,
@Override
public synchronized void updateClusterResource(Resource clusterResource) {
// Update all children
for (Queue childQueue : childQueues) {
for (CSQueue childQueue : childQueues) {
childQueue.updateClusterResource(clusterResource);
}
}

View File

@ -24,7 +24,7 @@
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 RmView {
@RequestScoped
static class Parent {
Queue queue;
CSQueue queue;
}
public static class QueueBlock extends HtmlBlock {
@ -56,8 +56,8 @@ public static class QueueBlock extends HtmlBlock {
@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 @@ public void render(Block html) {
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();

View File

@ -38,8 +38,8 @@ public void setUp() {
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 void testLimitsComputation() throws Exception {
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,

View File

@ -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 Container answer(InvocationOnMock invocation)
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));

View File

@ -81,7 +81,7 @@ private void setupSingleLevelQueues(CapacitySchedulerConfiguration conf) {
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 Resource answer(InvocationOnMock invocation) throws Throwable {
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 void testSingleLevelQueues() throws Exception {
// 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 void testMultiLevelQueues() throws Exception {
// 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 void testMultiLevelQueues() throws Exception {
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);
CSQueue a = queues.get(A);
CSQueue b = queues.get(B);
CSQueue c = queues.get(C);
CSQueue d = queues.get(D);
Queue a1 = queues.get(A1);
Queue a2 = queues.get(A2);
CSQueue a1 = queues.get(A1);
CSQueue a2 = queues.get(A2);
Queue b1 = queues.get(B1);
Queue b2 = queues.get(B2);
Queue b3 = queues.get(B3);
CSQueue b1 = queues.get(B1);
CSQueue b2 = queues.get(B2);
CSQueue b3 = queues.get(B3);
final float delta = 0.0001f;

View File

@ -85,7 +85,7 @@ public EventHandler getEventHandler() {
*/
static class SpyHook extends CapacityScheduler.QueueHook {
@Override
public Queue hook(Queue queue) {
public CSQueue hook(CSQueue queue) {
return spy(queue);
}
}

View File

@ -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>