Merge -c 1241205 from trunk to branch-0.23 to fix MAPREDUCE-3810. Performance tweaks - reduced logging in AM and defined hascode/equals for ResourceRequest & Priority.
git-svn-id: https://svn.apache.org/repos/asf/hadoop/common/branches/branch-0.23@1241208 13f79535-47bb-0310-9956-ffa450edef68
This commit is contained in:
parent
6325888402
commit
caea6d2e85
|
@ -665,6 +665,9 @@ Release 0.23.1 - Unreleased
|
|||
|
||||
MAPREDUCE-3814. Fixed MRV1 compilation. (Arun C Murthy via vinodkv)
|
||||
|
||||
MAPREDUCE-3810. Performance tweaks - reduced logging in AM and defined
|
||||
hascode/equals for ResourceRequest & Priority. (vinodkv via acmurthy)
|
||||
|
||||
Release 0.23.0 - 2011-11-01
|
||||
|
||||
INCOMPATIBLE CHANGES
|
||||
|
|
|
@ -127,7 +127,7 @@ import org.apache.hadoop.yarn.util.RackResolver;
|
|||
/**
|
||||
* Implementation of TaskAttempt interface.
|
||||
*/
|
||||
@SuppressWarnings({ "rawtypes", "deprecation" })
|
||||
@SuppressWarnings({ "rawtypes" })
|
||||
public abstract class TaskAttemptImpl implements
|
||||
org.apache.hadoop.mapreduce.v2.app.job.TaskAttempt,
|
||||
EventHandler<TaskAttemptEvent> {
|
||||
|
@ -910,8 +910,10 @@ public abstract class TaskAttemptImpl implements
|
|||
@SuppressWarnings("unchecked")
|
||||
@Override
|
||||
public void handle(TaskAttemptEvent event) {
|
||||
LOG.info("Processing " + event.getTaskAttemptID() +
|
||||
" of type " + event.getType());
|
||||
if (LOG.isDebugEnabled()) {
|
||||
LOG.debug("Processing " + event.getTaskAttemptID() + " of type "
|
||||
+ event.getType());
|
||||
}
|
||||
writeLock.lock();
|
||||
try {
|
||||
final TaskAttemptState oldState = getState();
|
||||
|
@ -1278,15 +1280,11 @@ public abstract class TaskAttemptImpl implements
|
|||
TaskAttemptEvent event) {
|
||||
//set the finish time
|
||||
taskAttempt.setFinishTime();
|
||||
String taskType =
|
||||
TypeConverter.fromYarn(taskAttempt.attemptId.getTaskId().getTaskType()).toString();
|
||||
LOG.info("In TaskAttemptImpl taskType: " + taskType);
|
||||
long slotMillis = computeSlotMillis(taskAttempt);
|
||||
JobCounterUpdateEvent jce =
|
||||
new JobCounterUpdateEvent(taskAttempt.attemptId.getTaskId()
|
||||
.getJobId());
|
||||
TaskId taskId = taskAttempt.attemptId.getTaskId();
|
||||
JobCounterUpdateEvent jce = new JobCounterUpdateEvent(taskId.getJobId());
|
||||
jce.addCounterUpdate(
|
||||
taskAttempt.attemptId.getTaskId().getTaskType() == TaskType.MAP ?
|
||||
taskId.getTaskType() == TaskType.MAP ?
|
||||
JobCounter.SLOTS_MILLIS_MAPS : JobCounter.SLOTS_MILLIS_REDUCES,
|
||||
slotMillis);
|
||||
taskAttempt.eventHandler.handle(jce);
|
||||
|
|
|
@ -81,7 +81,7 @@ import org.apache.hadoop.yarn.state.StateMachineFactory;
|
|||
/**
|
||||
* Implementation of Task interface.
|
||||
*/
|
||||
@SuppressWarnings({ "rawtypes", "unchecked", "deprecation" })
|
||||
@SuppressWarnings({ "rawtypes", "unchecked" })
|
||||
public abstract class TaskImpl implements Task, EventHandler<TaskEvent> {
|
||||
|
||||
private static final Log LOG = LogFactory.getLog(TaskImpl.class);
|
||||
|
@ -505,7 +505,9 @@ public abstract class TaskImpl implements Task, EventHandler<TaskEvent> {
|
|||
// This is always called in the Write Lock
|
||||
private void addAndScheduleAttempt() {
|
||||
TaskAttempt attempt = createAttempt();
|
||||
LOG.info("Created attempt " + attempt.getID());
|
||||
if (LOG.isDebugEnabled()) {
|
||||
LOG.debug("Created attempt " + attempt.getID());
|
||||
}
|
||||
switch (attempts.size()) {
|
||||
case 0:
|
||||
attempts = Collections.singletonMap(attempt.getID(), attempt);
|
||||
|
@ -537,7 +539,10 @@ public abstract class TaskImpl implements Task, EventHandler<TaskEvent> {
|
|||
|
||||
@Override
|
||||
public void handle(TaskEvent event) {
|
||||
LOG.debug("Processing " + event.getTaskID() + " of type " + event.getType());
|
||||
if (LOG.isDebugEnabled()) {
|
||||
LOG.debug("Processing " + event.getTaskID() + " of type "
|
||||
+ event.getType());
|
||||
}
|
||||
try {
|
||||
writeLock.lock();
|
||||
TaskState oldState = getState();
|
||||
|
|
|
@ -30,7 +30,6 @@ import org.apache.hadoop.mapreduce.JobID;
|
|||
import org.apache.hadoop.mapreduce.MRJobConfig;
|
||||
import org.apache.hadoop.mapreduce.TypeConverter;
|
||||
import org.apache.hadoop.mapreduce.v2.api.records.JobId;
|
||||
import org.apache.hadoop.mapreduce.v2.api.records.JobReport;
|
||||
import org.apache.hadoop.mapreduce.v2.api.records.JobState;
|
||||
import org.apache.hadoop.mapreduce.v2.app.AppContext;
|
||||
import org.apache.hadoop.mapreduce.v2.app.client.ClientService;
|
||||
|
@ -46,9 +45,9 @@ import org.apache.hadoop.yarn.api.ApplicationConstants;
|
|||
import org.apache.hadoop.yarn.api.protocolrecords.FinishApplicationMasterRequest;
|
||||
import org.apache.hadoop.yarn.api.protocolrecords.RegisterApplicationMasterRequest;
|
||||
import org.apache.hadoop.yarn.api.protocolrecords.RegisterApplicationMasterResponse;
|
||||
import org.apache.hadoop.yarn.api.records.ApplicationAccessType;
|
||||
import org.apache.hadoop.yarn.api.records.ApplicationAttemptId;
|
||||
import org.apache.hadoop.yarn.api.records.ApplicationId;
|
||||
import org.apache.hadoop.yarn.api.records.ApplicationAccessType;
|
||||
import org.apache.hadoop.yarn.api.records.FinalApplicationStatus;
|
||||
import org.apache.hadoop.yarn.api.records.Resource;
|
||||
import org.apache.hadoop.yarn.conf.YarnConfiguration;
|
||||
|
@ -149,7 +148,7 @@ public abstract class RMCommunicator extends AbstractService {
|
|||
LOG.info("minContainerCapability: " + minContainerCapability.getMemory());
|
||||
LOG.info("maxContainerCapability: " + maxContainerCapability.getMemory());
|
||||
} catch (Exception are) {
|
||||
LOG.info("Exception while registering", are);
|
||||
LOG.error("Exception while registering", are);
|
||||
throw new YarnException(are);
|
||||
}
|
||||
}
|
||||
|
@ -183,7 +182,7 @@ public abstract class RMCommunicator extends AbstractService {
|
|||
request.setTrackingUrl(historyUrl);
|
||||
scheduler.finishApplicationMaster(request);
|
||||
} catch(Exception are) {
|
||||
LOG.info("Exception while unregistering ", are);
|
||||
LOG.error("Exception while unregistering ", are);
|
||||
}
|
||||
}
|
||||
|
||||
|
@ -205,7 +204,7 @@ public abstract class RMCommunicator extends AbstractService {
|
|||
try {
|
||||
allocatorThread.join();
|
||||
} catch (InterruptedException ie) {
|
||||
LOG.info("InterruptedException while stopping", ie);
|
||||
LOG.warn("InterruptedException while stopping", ie);
|
||||
}
|
||||
unregister();
|
||||
super.stop();
|
||||
|
@ -228,7 +227,7 @@ public abstract class RMCommunicator extends AbstractService {
|
|||
// TODO: for other exceptions
|
||||
}
|
||||
} catch (InterruptedException e) {
|
||||
LOG.info("Allocated thread interrupted. Returning.");
|
||||
LOG.warn("Allocated thread interrupted. Returning.");
|
||||
return;
|
||||
}
|
||||
}
|
||||
|
@ -255,7 +254,9 @@ public abstract class RMCommunicator extends AbstractService {
|
|||
if (UserGroupInformation.isSecurityEnabled()) {
|
||||
String tokenURLEncodedStr = System.getenv().get(
|
||||
ApplicationConstants.APPLICATION_MASTER_TOKEN_ENV_NAME);
|
||||
LOG.debug("AppMasterToken is " + tokenURLEncodedStr);
|
||||
if (LOG.isDebugEnabled()) {
|
||||
LOG.debug("AppMasterToken is " + tokenURLEncodedStr);
|
||||
}
|
||||
Token<? extends TokenIdentifier> token = new Token<TokenIdentifier>();
|
||||
|
||||
try {
|
||||
|
|
|
@ -41,6 +41,7 @@ import org.apache.hadoop.mapreduce.JobCounter;
|
|||
import org.apache.hadoop.mapreduce.MRJobConfig;
|
||||
import org.apache.hadoop.mapreduce.jobhistory.JobHistoryEvent;
|
||||
import org.apache.hadoop.mapreduce.jobhistory.NormalizedResourceEvent;
|
||||
import org.apache.hadoop.mapreduce.v2.api.records.JobId;
|
||||
import org.apache.hadoop.mapreduce.v2.api.records.TaskAttemptId;
|
||||
import org.apache.hadoop.mapreduce.v2.api.records.TaskType;
|
||||
import org.apache.hadoop.mapreduce.v2.app.AppContext;
|
||||
|
@ -254,28 +255,30 @@ public class RMContainerAllocator extends RMContainerRequestor
|
|||
|
||||
@SuppressWarnings({ "unchecked" })
|
||||
protected synchronized void handleEvent(ContainerAllocatorEvent event) {
|
||||
LOG.info("Processing the event " + event.toString());
|
||||
recalculateReduceSchedule = true;
|
||||
if (event.getType() == ContainerAllocator.EventType.CONTAINER_REQ) {
|
||||
ContainerRequestEvent reqEvent = (ContainerRequestEvent) event;
|
||||
JobId jobId = getJob().getID();
|
||||
int supportedMaxContainerCapability =
|
||||
getMaxContainerCapability().getMemory();
|
||||
if (reqEvent.getAttemptID().getTaskId().getTaskType().equals(TaskType.MAP)) {
|
||||
if (mapResourceReqt == 0) {
|
||||
mapResourceReqt = reqEvent.getCapability().getMemory();
|
||||
int minSlotMemSize = getMinContainerCapability().getMemory();
|
||||
mapResourceReqt = (int) Math.ceil((float) mapResourceReqt/minSlotMemSize)
|
||||
* minSlotMemSize;
|
||||
eventHandler.handle(new JobHistoryEvent(getJob().getID(),
|
||||
eventHandler.handle(new JobHistoryEvent(jobId,
|
||||
new NormalizedResourceEvent(org.apache.hadoop.mapreduce.TaskType.MAP,
|
||||
mapResourceReqt)));
|
||||
LOG.info("mapResourceReqt:"+mapResourceReqt);
|
||||
if (mapResourceReqt > getMaxContainerCapability().getMemory()) {
|
||||
if (mapResourceReqt > supportedMaxContainerCapability) {
|
||||
String diagMsg = "MAP capability required is more than the supported " +
|
||||
"max container capability in the cluster. Killing the Job. mapResourceReqt: " +
|
||||
mapResourceReqt + " maxContainerCapability:" + getMaxContainerCapability().getMemory();
|
||||
mapResourceReqt + " maxContainerCapability:" + supportedMaxContainerCapability;
|
||||
LOG.info(diagMsg);
|
||||
eventHandler.handle(new JobDiagnosticsUpdateEvent(
|
||||
getJob().getID(), diagMsg));
|
||||
eventHandler.handle(new JobEvent(getJob().getID(), JobEventType.JOB_KILL));
|
||||
jobId, diagMsg));
|
||||
eventHandler.handle(new JobEvent(jobId, JobEventType.JOB_KILL));
|
||||
}
|
||||
}
|
||||
//set the rounded off memory
|
||||
|
@ -288,20 +291,20 @@ public class RMContainerAllocator extends RMContainerRequestor
|
|||
//round off on slotsize
|
||||
reduceResourceReqt = (int) Math.ceil((float)
|
||||
reduceResourceReqt/minSlotMemSize) * minSlotMemSize;
|
||||
eventHandler.handle(new JobHistoryEvent(getJob().getID(),
|
||||
eventHandler.handle(new JobHistoryEvent(jobId,
|
||||
new NormalizedResourceEvent(
|
||||
org.apache.hadoop.mapreduce.TaskType.REDUCE,
|
||||
reduceResourceReqt)));
|
||||
LOG.info("reduceResourceReqt:"+reduceResourceReqt);
|
||||
if (reduceResourceReqt > getMaxContainerCapability().getMemory()) {
|
||||
if (reduceResourceReqt > supportedMaxContainerCapability) {
|
||||
String diagMsg = "REDUCE capability required is more than the " +
|
||||
"supported max container capability in the cluster. Killing the " +
|
||||
"Job. reduceResourceReqt: " + reduceResourceReqt +
|
||||
" maxContainerCapability:" + getMaxContainerCapability().getMemory();
|
||||
" maxContainerCapability:" + supportedMaxContainerCapability;
|
||||
LOG.info(diagMsg);
|
||||
eventHandler.handle(new JobDiagnosticsUpdateEvent(
|
||||
getJob().getID(), diagMsg));
|
||||
eventHandler.handle(new JobEvent(getJob().getID(), JobEventType.JOB_KILL));
|
||||
jobId, diagMsg));
|
||||
eventHandler.handle(new JobEvent(jobId, JobEventType.JOB_KILL));
|
||||
}
|
||||
}
|
||||
//set the rounded off memory
|
||||
|
@ -317,6 +320,9 @@ public class RMContainerAllocator extends RMContainerRequestor
|
|||
|
||||
} else if (
|
||||
event.getType() == ContainerAllocator.EventType.CONTAINER_DEALLOCATE) {
|
||||
|
||||
LOG.info("Processing the event " + event.toString());
|
||||
|
||||
TaskAttemptId aId = event.getAttemptID();
|
||||
|
||||
boolean removed = scheduledRequests.remove(aId);
|
||||
|
@ -579,7 +585,7 @@ public class RMContainerAllocator extends RMContainerRequestor
|
|||
computeIgnoreBlacklisting();
|
||||
|
||||
for (ContainerStatus cont : finishedContainers) {
|
||||
LOG.info("Received completed container " + cont);
|
||||
LOG.info("Received completed container " + cont.getContainerId());
|
||||
TaskAttemptId attemptID = assignedRequests.get(cont.getContainerId());
|
||||
if (attemptID == null) {
|
||||
LOG.error("Container complete event for unknown container id "
|
||||
|
@ -664,7 +670,9 @@ public class RMContainerAllocator extends RMContainerRequestor
|
|||
mapsHostMapping.put(host, list);
|
||||
}
|
||||
list.add(event.getAttemptID());
|
||||
LOG.info("Added attempt req to host " + host);
|
||||
if (LOG.isDebugEnabled()) {
|
||||
LOG.debug("Added attempt req to host " + host);
|
||||
}
|
||||
}
|
||||
for (String rack: event.getRacks()) {
|
||||
LinkedList<TaskAttemptId> list = mapsRackMapping.get(rack);
|
||||
|
@ -673,7 +681,9 @@ public class RMContainerAllocator extends RMContainerRequestor
|
|||
mapsRackMapping.put(rack, list);
|
||||
}
|
||||
list.add(event.getAttemptID());
|
||||
LOG.info("Added attempt req to rack " + rack);
|
||||
if (LOG.isDebugEnabled()) {
|
||||
LOG.debug("Added attempt req to rack " + rack);
|
||||
}
|
||||
}
|
||||
request = new ContainerRequest(event, PRIORITY_MAP);
|
||||
}
|
||||
|
@ -694,18 +704,21 @@ public class RMContainerAllocator extends RMContainerRequestor
|
|||
containersAllocated += allocatedContainers.size();
|
||||
while (it.hasNext()) {
|
||||
Container allocated = it.next();
|
||||
LOG.info("Assigning container " + allocated.getId() +
|
||||
" with priority " + allocated.getPriority() +
|
||||
" to NM " + allocated.getNodeId());
|
||||
if (LOG.isDebugEnabled()) {
|
||||
LOG.debug("Assigning container " + allocated.getId()
|
||||
+ " with priority " + allocated.getPriority() + " to NM "
|
||||
+ allocated.getNodeId());
|
||||
}
|
||||
|
||||
// check if allocated container meets memory requirements
|
||||
// and whether we have any scheduled tasks that need
|
||||
// a container to be assigned
|
||||
boolean isAssignable = true;
|
||||
Priority priority = allocated.getPriority();
|
||||
int allocatedMemory = allocated.getResource().getMemory();
|
||||
if (PRIORITY_FAST_FAIL_MAP.equals(priority)
|
||||
|| PRIORITY_MAP.equals(priority)) {
|
||||
if (allocated.getResource().getMemory() < mapResourceReqt
|
||||
if (allocatedMemory < mapResourceReqt
|
||||
|| maps.isEmpty()) {
|
||||
LOG.info("Cannot assign container " + allocated
|
||||
+ " for a map as either "
|
||||
|
@ -716,7 +729,7 @@ public class RMContainerAllocator extends RMContainerRequestor
|
|||
}
|
||||
}
|
||||
else if (PRIORITY_REDUCE.equals(priority)) {
|
||||
if (allocated.getResource().getMemory() < reduceResourceReqt
|
||||
if (allocatedMemory < reduceResourceReqt
|
||||
|| reduces.isEmpty()) {
|
||||
LOG.info("Cannot assign container " + allocated
|
||||
+ " for a reduce as either "
|
||||
|
@ -730,15 +743,17 @@ public class RMContainerAllocator extends RMContainerRequestor
|
|||
boolean blackListed = false;
|
||||
ContainerRequest assigned = null;
|
||||
|
||||
ContainerId allocatedContainerId = allocated.getId();
|
||||
if (isAssignable) {
|
||||
// do not assign if allocated container is on a
|
||||
// blacklisted host
|
||||
blackListed = isNodeBlacklisted(allocated.getNodeId().getHost());
|
||||
String allocatedHost = allocated.getNodeId().getHost();
|
||||
blackListed = isNodeBlacklisted(allocatedHost);
|
||||
if (blackListed) {
|
||||
// we need to request for a new container
|
||||
// and release the current one
|
||||
LOG.info("Got allocated container on a blacklisted "
|
||||
+ " host "+allocated.getNodeId().getHost()
|
||||
+ " host "+allocatedHost
|
||||
+". Releasing container " + allocated);
|
||||
|
||||
// find the request matching this allocated container
|
||||
|
@ -775,11 +790,13 @@ public class RMContainerAllocator extends RMContainerRequestor
|
|||
eventHandler.handle(new TaskAttemptContainerAssignedEvent(
|
||||
assigned.attemptID, allocated, applicationACLs));
|
||||
|
||||
assignedRequests.add(allocated.getId(), assigned.attemptID);
|
||||
assignedRequests.add(allocatedContainerId, assigned.attemptID);
|
||||
|
||||
LOG.info("Assigned container (" + allocated + ") " +
|
||||
" to task " + assigned.attemptID +
|
||||
" on node " + allocated.getNodeId().toString());
|
||||
if (LOG.isDebugEnabled()) {
|
||||
LOG.info("Assigned container (" + allocated + ") "
|
||||
+ " to task " + assigned.attemptID + " on node "
|
||||
+ allocated.getNodeId().toString());
|
||||
}
|
||||
}
|
||||
else {
|
||||
//not assigned to any request, release the container
|
||||
|
@ -794,7 +811,7 @@ public class RMContainerAllocator extends RMContainerRequestor
|
|||
// or if we could not assign it
|
||||
if (blackListed || assigned == null) {
|
||||
containersReleased++;
|
||||
release(allocated.getId());
|
||||
release(allocatedContainerId);
|
||||
}
|
||||
}
|
||||
}
|
||||
|
@ -807,10 +824,14 @@ public class RMContainerAllocator extends RMContainerRequestor
|
|||
LOG.info("Assigning container " + allocated + " to fast fail map");
|
||||
assigned = assignToFailedMap(allocated);
|
||||
} else if (PRIORITY_REDUCE.equals(priority)) {
|
||||
LOG.info("Assigning container " + allocated + " to reduce");
|
||||
if (LOG.isDebugEnabled()) {
|
||||
LOG.debug("Assigning container " + allocated + " to reduce");
|
||||
}
|
||||
assigned = assignToReduce(allocated);
|
||||
} else if (PRIORITY_MAP.equals(priority)) {
|
||||
LOG.info("Assigning container " + allocated + " to map");
|
||||
if (LOG.isDebugEnabled()) {
|
||||
LOG.debug("Assigning container " + allocated + " to map");
|
||||
}
|
||||
assigned = assignToMap(allocated);
|
||||
} else {
|
||||
LOG.warn("Container allocated at unwanted priority: " + priority +
|
||||
|
@ -897,7 +918,9 @@ public class RMContainerAllocator extends RMContainerRequestor
|
|||
String host = allocated.getNodeId().getHost();
|
||||
LinkedList<TaskAttemptId> list = mapsHostMapping.get(host);
|
||||
while (list != null && list.size() > 0) {
|
||||
LOG.info("Host matched to the request list " + host);
|
||||
if (LOG.isDebugEnabled()) {
|
||||
LOG.debug("Host matched to the request list " + host);
|
||||
}
|
||||
TaskAttemptId tId = list.removeFirst();
|
||||
if (maps.containsKey(tId)) {
|
||||
assigned = maps.remove(tId);
|
||||
|
@ -906,7 +929,9 @@ public class RMContainerAllocator extends RMContainerRequestor
|
|||
jce.addCounterUpdate(JobCounter.DATA_LOCAL_MAPS, 1);
|
||||
eventHandler.handle(jce);
|
||||
hostLocalAssigned++;
|
||||
LOG.info("Assigned based on host match " + host);
|
||||
if (LOG.isDebugEnabled()) {
|
||||
LOG.debug("Assigned based on host match " + host);
|
||||
}
|
||||
break;
|
||||
}
|
||||
}
|
||||
|
@ -922,7 +947,9 @@ public class RMContainerAllocator extends RMContainerRequestor
|
|||
jce.addCounterUpdate(JobCounter.RACK_LOCAL_MAPS, 1);
|
||||
eventHandler.handle(jce);
|
||||
rackLocalAssigned++;
|
||||
LOG.info("Assigned based on rack match " + rack);
|
||||
if (LOG.isDebugEnabled()) {
|
||||
LOG.debug("Assigned based on rack match " + rack);
|
||||
}
|
||||
break;
|
||||
}
|
||||
}
|
||||
|
@ -933,7 +960,9 @@ public class RMContainerAllocator extends RMContainerRequestor
|
|||
new JobCounterUpdateEvent(assigned.attemptID.getTaskId().getJobId());
|
||||
jce.addCounterUpdate(JobCounter.OTHER_LOCAL_MAPS, 1);
|
||||
eventHandler.handle(jce);
|
||||
LOG.info("Assigned based on * match");
|
||||
if (LOG.isDebugEnabled()) {
|
||||
LOG.debug("Assigned based on * match");
|
||||
}
|
||||
break;
|
||||
}
|
||||
}
|
||||
|
@ -953,8 +982,7 @@ public class RMContainerAllocator extends RMContainerRequestor
|
|||
new HashSet<TaskAttemptId>();
|
||||
|
||||
void add(ContainerId containerId, TaskAttemptId tId) {
|
||||
LOG.info("Assigned container " + containerId.toString()
|
||||
+ " to " + tId);
|
||||
LOG.info("Assigned container " + containerId.toString() + " to " + tId);
|
||||
containerToAttemptMap.put(containerId, tId);
|
||||
if (tId.getTaskId().getTaskType().equals(TaskType.MAP)) {
|
||||
maps.put(tId, containerId);
|
||||
|
@ -963,6 +991,7 @@ public class RMContainerAllocator extends RMContainerRequestor
|
|||
}
|
||||
}
|
||||
|
||||
@SuppressWarnings("unchecked")
|
||||
void preemptReduce(int toPreempt) {
|
||||
List<TaskAttemptId> reduceList = new ArrayList<TaskAttemptId>
|
||||
(reduces.keySet());
|
||||
|
|
|
@ -155,13 +155,14 @@ public abstract class RMContainerRequestor extends RMCommunicator {
|
|||
lastClusterNmCount = clusterNmCount;
|
||||
clusterNmCount = allocateResponse.getNumClusterNodes();
|
||||
|
||||
LOG.info("getResources() for " + applicationId + ":" + " ask="
|
||||
+ ask.size() + " release= " + release.size() +
|
||||
" newContainers=" + response.getAllocatedContainers().size() +
|
||||
" finishedContainers=" +
|
||||
response.getCompletedContainersStatuses().size() +
|
||||
" resourcelimit=" + availableResources +
|
||||
" knownNMs=" + clusterNmCount);
|
||||
if (ask.size() > 0 || release.size() > 0) {
|
||||
LOG.info("getResources() for " + applicationId + ":" + " ask="
|
||||
+ ask.size() + " release= " + release.size() + " newContainers="
|
||||
+ response.getAllocatedContainers().size() + " finishedContainers="
|
||||
+ response.getCompletedContainersStatuses().size()
|
||||
+ " resourcelimit=" + availableResources + " knownNMs="
|
||||
+ clusterNmCount);
|
||||
}
|
||||
|
||||
ask.clear();
|
||||
release.clear();
|
||||
|
@ -172,6 +173,9 @@ public abstract class RMContainerRequestor extends RMCommunicator {
|
|||
// knownNodeCount is based on node managers, not hosts. blacklisting is
|
||||
// currently based on hosts.
|
||||
protected void computeIgnoreBlacklisting() {
|
||||
if (!nodeBlacklistingEnabled) {
|
||||
return;
|
||||
}
|
||||
if (blacklistDisablePercent != -1
|
||||
&& (blacklistedNodeCount != blacklistedNodes.size() ||
|
||||
clusterNmCount != lastClusterNmCount)) {
|
||||
|
@ -200,7 +204,9 @@ public abstract class RMContainerRequestor extends RMCommunicator {
|
|||
return;
|
||||
}
|
||||
if (blacklistedNodes.contains(hostName)) {
|
||||
LOG.info("Host " + hostName + " is already blacklisted.");
|
||||
if (LOG.isDebugEnabled()) {
|
||||
LOG.debug("Host " + hostName + " is already blacklisted.");
|
||||
}
|
||||
return; //already blacklisted
|
||||
}
|
||||
Integer failures = nodeFailures.remove(hostName);
|
||||
|
@ -293,7 +299,9 @@ public abstract class RMContainerRequestor extends RMCommunicator {
|
|||
if (remoteRequests == null) {
|
||||
remoteRequests = new HashMap<String, Map<Resource, ResourceRequest>>();
|
||||
this.remoteRequestsTable.put(priority, remoteRequests);
|
||||
LOG.info("Added priority=" + priority);
|
||||
if (LOG.isDebugEnabled()) {
|
||||
LOG.debug("Added priority=" + priority);
|
||||
}
|
||||
}
|
||||
Map<Resource, ResourceRequest> reqMap = remoteRequests.get(resourceName);
|
||||
if (reqMap == null) {
|
||||
|
@ -313,10 +321,12 @@ public abstract class RMContainerRequestor extends RMCommunicator {
|
|||
|
||||
// Note this down for next interaction with ResourceManager
|
||||
ask.add(remoteRequest);
|
||||
LOG.info("addResourceRequest:" + " applicationId=" + applicationId.getId()
|
||||
+ " priority=" + priority.getPriority() + " resourceName=" + resourceName
|
||||
+ " numContainers=" + remoteRequest.getNumContainers() + " #asks="
|
||||
+ ask.size());
|
||||
if (LOG.isDebugEnabled()) {
|
||||
LOG.debug("addResourceRequest:" + " applicationId="
|
||||
+ applicationId.getId() + " priority=" + priority.getPriority()
|
||||
+ " resourceName=" + resourceName + " numContainers="
|
||||
+ remoteRequest.getNumContainers() + " #asks=" + ask.size());
|
||||
}
|
||||
}
|
||||
|
||||
private void decResourceRequest(Priority priority, String resourceName,
|
||||
|
@ -328,16 +338,20 @@ public abstract class RMContainerRequestor extends RMCommunicator {
|
|||
// as we modify the resource requests by filtering out blacklisted hosts
|
||||
// when they are added, this value may be null when being
|
||||
// decremented
|
||||
LOG.debug("Not decrementing resource as " + resourceName
|
||||
+ " is not present in request table");
|
||||
if (LOG.isDebugEnabled()) {
|
||||
LOG.debug("Not decrementing resource as " + resourceName
|
||||
+ " is not present in request table");
|
||||
}
|
||||
return;
|
||||
}
|
||||
ResourceRequest remoteRequest = reqMap.get(capability);
|
||||
|
||||
LOG.info("BEFORE decResourceRequest:" + " applicationId=" + applicationId.getId()
|
||||
+ " priority=" + priority.getPriority() + " resourceName=" + resourceName
|
||||
+ " numContainers=" + remoteRequest.getNumContainers() + " #asks="
|
||||
+ ask.size());
|
||||
if (LOG.isDebugEnabled()) {
|
||||
LOG.debug("BEFORE decResourceRequest:" + " applicationId="
|
||||
+ applicationId.getId() + " priority=" + priority.getPriority()
|
||||
+ " resourceName=" + resourceName + " numContainers="
|
||||
+ remoteRequest.getNumContainers() + " #asks=" + ask.size());
|
||||
}
|
||||
|
||||
remoteRequest.setNumContainers(remoteRequest.getNumContainers() -1);
|
||||
if (remoteRequest.getNumContainers() == 0) {
|
||||
|
@ -355,10 +369,12 @@ public abstract class RMContainerRequestor extends RMCommunicator {
|
|||
//already have it.
|
||||
}
|
||||
|
||||
LOG.info("AFTER decResourceRequest:" + " applicationId="
|
||||
+ applicationId.getId() + " priority=" + priority.getPriority()
|
||||
+ " resourceName=" + resourceName + " numContainers="
|
||||
+ remoteRequest.getNumContainers() + " #asks=" + ask.size());
|
||||
if (LOG.isDebugEnabled()) {
|
||||
LOG.info("AFTER decResourceRequest:" + " applicationId="
|
||||
+ applicationId.getId() + " priority=" + priority.getPriority()
|
||||
+ " resourceName=" + resourceName + " numContainers="
|
||||
+ remoteRequest.getNumContainers() + " #asks=" + ask.size());
|
||||
}
|
||||
}
|
||||
|
||||
protected void release(ContainerId containerId) {
|
||||
|
|
|
@ -436,7 +436,7 @@ public class MRApp extends MRAppMaster {
|
|||
return new ClientService(){
|
||||
@Override
|
||||
public InetSocketAddress getBindAddress() {
|
||||
return null;
|
||||
return NetUtils.createSocketAddr("localhost:9876");
|
||||
}
|
||||
|
||||
@Override
|
||||
|
|
|
@ -18,6 +18,8 @@
|
|||
|
||||
package org.apache.hadoop.mapreduce.v2.app;
|
||||
|
||||
import java.util.ArrayList;
|
||||
import java.util.List;
|
||||
import java.util.concurrent.BlockingQueue;
|
||||
import java.util.concurrent.LinkedBlockingQueue;
|
||||
|
||||
|
@ -29,16 +31,30 @@ import org.apache.hadoop.mapreduce.v2.app.job.Job;
|
|||
import org.apache.hadoop.mapreduce.v2.app.job.event.TaskAttemptContainerAssignedEvent;
|
||||
import org.apache.hadoop.mapreduce.v2.app.rm.ContainerAllocator;
|
||||
import org.apache.hadoop.mapreduce.v2.app.rm.ContainerAllocatorEvent;
|
||||
import org.apache.hadoop.mapreduce.v2.app.rm.RMContainerAllocator;
|
||||
import org.apache.hadoop.yarn.YarnException;
|
||||
import org.apache.hadoop.yarn.api.AMRMProtocol;
|
||||
import org.apache.hadoop.yarn.api.protocolrecords.AllocateRequest;
|
||||
import org.apache.hadoop.yarn.api.protocolrecords.AllocateResponse;
|
||||
import org.apache.hadoop.yarn.api.protocolrecords.FinishApplicationMasterRequest;
|
||||
import org.apache.hadoop.yarn.api.protocolrecords.FinishApplicationMasterResponse;
|
||||
import org.apache.hadoop.yarn.api.protocolrecords.RegisterApplicationMasterRequest;
|
||||
import org.apache.hadoop.yarn.api.protocolrecords.RegisterApplicationMasterResponse;
|
||||
import org.apache.hadoop.yarn.api.records.AMResponse;
|
||||
import org.apache.hadoop.yarn.api.records.Container;
|
||||
import org.apache.hadoop.yarn.api.records.ContainerId;
|
||||
import org.apache.hadoop.yarn.api.records.NodeId;
|
||||
import org.apache.hadoop.yarn.api.records.ResourceRequest;
|
||||
import org.apache.hadoop.yarn.exceptions.YarnRemoteException;
|
||||
import org.apache.hadoop.yarn.factories.RecordFactory;
|
||||
import org.apache.hadoop.yarn.factory.providers.RecordFactoryProvider;
|
||||
import org.apache.hadoop.yarn.service.AbstractService;
|
||||
import org.apache.hadoop.yarn.util.BuilderUtils;
|
||||
import org.apache.hadoop.yarn.util.Records;
|
||||
import org.apache.log4j.Level;
|
||||
import org.apache.log4j.LogManager;
|
||||
import org.apache.log4j.Logger;
|
||||
import org.junit.Test;
|
||||
|
||||
public class MRAppBenchmark {
|
||||
|
||||
|
@ -167,17 +183,89 @@ public class MRAppBenchmark {
|
|||
}
|
||||
}
|
||||
|
||||
@Test
|
||||
public void benchmark1() throws Exception {
|
||||
int maps = 100000;
|
||||
int reduces = 100;
|
||||
int maps = 100; // Adjust for benchmarking. Start with thousands.
|
||||
int reduces = 0;
|
||||
System.out.println("Running benchmark with maps:"+maps +
|
||||
" reduces:"+reduces);
|
||||
run(new MRApp(maps, reduces, true, this.getClass().getName(), true));
|
||||
run(new MRApp(maps, reduces, true, this.getClass().getName(), true) {
|
||||
|
||||
@Override
|
||||
protected ContainerAllocator createContainerAllocator(
|
||||
ClientService clientService, AppContext context) {
|
||||
return new RMContainerAllocator(clientService, context) {
|
||||
@Override
|
||||
protected AMRMProtocol createSchedulerProxy() {
|
||||
return new AMRMProtocol() {
|
||||
|
||||
@Override
|
||||
public RegisterApplicationMasterResponse
|
||||
registerApplicationMaster(
|
||||
RegisterApplicationMasterRequest request)
|
||||
throws YarnRemoteException {
|
||||
RegisterApplicationMasterResponse response =
|
||||
Records.newRecord(RegisterApplicationMasterResponse.class);
|
||||
response.setMinimumResourceCapability(BuilderUtils
|
||||
.newResource(1024));
|
||||
response.setMaximumResourceCapability(BuilderUtils
|
||||
.newResource(10240));
|
||||
return response;
|
||||
}
|
||||
|
||||
@Override
|
||||
public FinishApplicationMasterResponse finishApplicationMaster(
|
||||
FinishApplicationMasterRequest request)
|
||||
throws YarnRemoteException {
|
||||
FinishApplicationMasterResponse response =
|
||||
Records.newRecord(FinishApplicationMasterResponse.class);
|
||||
return response;
|
||||
}
|
||||
|
||||
@Override
|
||||
public AllocateResponse allocate(AllocateRequest request)
|
||||
throws YarnRemoteException {
|
||||
|
||||
AllocateResponse response =
|
||||
Records.newRecord(AllocateResponse.class);
|
||||
List<ResourceRequest> askList = request.getAskList();
|
||||
List<Container> containers = new ArrayList<Container>();
|
||||
for (ResourceRequest req : askList) {
|
||||
if (req.getHostName() != "*") {
|
||||
continue;
|
||||
}
|
||||
int numContainers = req.getNumContainers();
|
||||
for (int i = 0; i < numContainers; i++) {
|
||||
ContainerId containerId =
|
||||
BuilderUtils.newContainerId(
|
||||
request.getApplicationAttemptId(),
|
||||
request.getResponseId() + i);
|
||||
containers.add(BuilderUtils
|
||||
.newContainer(containerId, BuilderUtils.newNodeId("host"
|
||||
+ containerId.getId(), 2345),
|
||||
"host" + containerId.getId() + ":5678", req
|
||||
.getCapability(), req.getPriority(), null));
|
||||
}
|
||||
}
|
||||
|
||||
AMResponse amResponse = Records.newRecord(AMResponse.class);
|
||||
amResponse.setAllocatedContainers(containers);
|
||||
amResponse.setResponseId(request.getResponseId() + 1);
|
||||
response.setAMResponse(amResponse);
|
||||
response.setNumClusterNodes(350);
|
||||
return response;
|
||||
}
|
||||
};
|
||||
}
|
||||
};
|
||||
}
|
||||
});
|
||||
}
|
||||
|
||||
@Test
|
||||
public void benchmark2() throws Exception {
|
||||
int maps = 4000;
|
||||
int reduces = 1000;
|
||||
int maps = 100; // Adjust for benchmarking, start with a couple of thousands
|
||||
int reduces = 50;
|
||||
int maxConcurrentRunningTasks = 500;
|
||||
|
||||
System.out.println("Running benchmark with throttled running tasks with " +
|
||||
|
|
|
@ -23,7 +23,7 @@ package org.apache.hadoop.yarn.api.records;
|
|||
* allocation
|
||||
*
|
||||
*/
|
||||
public interface Priority extends Comparable<Priority> {
|
||||
public abstract class Priority implements Comparable<Priority> {
|
||||
|
||||
/**
|
||||
* Get the assigned priority
|
||||
|
@ -37,4 +37,31 @@ public interface Priority extends Comparable<Priority> {
|
|||
*/
|
||||
public abstract void setPriority(int priority);
|
||||
|
||||
@Override
|
||||
public int hashCode() {
|
||||
final int prime = 31;
|
||||
int result = 1;
|
||||
result = prime * result + getPriority();
|
||||
return result;
|
||||
}
|
||||
|
||||
@Override
|
||||
public boolean equals(Object obj) {
|
||||
if (this == obj)
|
||||
return true;
|
||||
if (obj == null)
|
||||
return false;
|
||||
if (getClass() != obj.getClass())
|
||||
return false;
|
||||
Priority other = (Priority) obj;
|
||||
if (getPriority() != other.getPriority())
|
||||
return false;
|
||||
return true;
|
||||
}
|
||||
|
||||
@Override
|
||||
public int compareTo(Priority other) {
|
||||
return this.getPriority() - other.getPriority();
|
||||
}
|
||||
|
||||
}
|
||||
|
|
|
@ -48,7 +48,7 @@ import org.apache.hadoop.yarn.api.AMRMProtocol;
|
|||
*/
|
||||
@Public
|
||||
@Stable
|
||||
public interface ResourceRequest extends Comparable<ResourceRequest> {
|
||||
public abstract class ResourceRequest implements Comparable<ResourceRequest> {
|
||||
/**
|
||||
* Get the <code>Priority</code> of the request.
|
||||
* @return <code>Priority</code> of the request
|
||||
|
@ -121,4 +121,79 @@ public interface ResourceRequest extends Comparable<ResourceRequest> {
|
|||
@Public
|
||||
@Stable
|
||||
public abstract void setNumContainers(int numContainers);
|
||||
|
||||
@Override
|
||||
public int hashCode() {
|
||||
final int prime = 31;
|
||||
int result = 1;
|
||||
Resource capability = getCapability();
|
||||
String hostName = getHostName();
|
||||
Priority priority = getPriority();
|
||||
result =
|
||||
prime * result + ((capability == null) ? 0 : capability.hashCode());
|
||||
result = prime * result + ((hostName == null) ? 0 : hostName.hashCode());
|
||||
result = prime * result + getNumContainers();
|
||||
result = prime * result + ((priority == null) ? 0 : priority.hashCode());
|
||||
return result;
|
||||
}
|
||||
|
||||
@Override
|
||||
public boolean equals(Object obj) {
|
||||
if (this == obj)
|
||||
return true;
|
||||
if (obj == null)
|
||||
return false;
|
||||
if (getClass() != obj.getClass())
|
||||
return false;
|
||||
ResourceRequest other = (ResourceRequest) obj;
|
||||
Resource capability = getCapability();
|
||||
if (capability == null) {
|
||||
if (other.getCapability() != null)
|
||||
return false;
|
||||
} else if (!capability.equals(other.getCapability()))
|
||||
return false;
|
||||
String hostName = getHostName();
|
||||
if (hostName == null) {
|
||||
if (other.getHostName() != null)
|
||||
return false;
|
||||
} else if (!hostName.equals(other.getHostName()))
|
||||
return false;
|
||||
if (getNumContainers() != other.getNumContainers())
|
||||
return false;
|
||||
Priority priority = getPriority();
|
||||
if (priority == null) {
|
||||
if (other.getPriority() != null)
|
||||
return false;
|
||||
} else if (!priority.equals(other.getPriority()))
|
||||
return false;
|
||||
return true;
|
||||
}
|
||||
|
||||
@Override
|
||||
public int compareTo(ResourceRequest other) {
|
||||
int priorityComparison = this.getPriority().compareTo(other.getPriority());
|
||||
if (priorityComparison == 0) {
|
||||
int hostNameComparison =
|
||||
this.getHostName().compareTo(other.getHostName());
|
||||
if (hostNameComparison == 0) {
|
||||
int capabilityComparison =
|
||||
this.getCapability().compareTo(other.getCapability());
|
||||
if (capabilityComparison == 0) {
|
||||
int numContainersComparison =
|
||||
this.getNumContainers() - other.getNumContainers();
|
||||
if (numContainersComparison == 0) {
|
||||
return 0;
|
||||
} else {
|
||||
return numContainersComparison;
|
||||
}
|
||||
} else {
|
||||
return capabilityComparison;
|
||||
}
|
||||
} else {
|
||||
return hostNameComparison;
|
||||
}
|
||||
} else {
|
||||
return priorityComparison;
|
||||
}
|
||||
}
|
||||
}
|
||||
|
|
|
@ -18,15 +18,11 @@
|
|||
|
||||
package org.apache.hadoop.yarn.api.records.impl.pb;
|
||||
|
||||
|
||||
import org.apache.hadoop.yarn.api.records.Priority;
|
||||
import org.apache.hadoop.yarn.api.records.ProtoBase;
|
||||
import org.apache.hadoop.yarn.proto.YarnProtos.PriorityProto;
|
||||
import org.apache.hadoop.yarn.proto.YarnProtos.PriorityProtoOrBuilder;
|
||||
|
||||
|
||||
|
||||
public class PriorityPBImpl extends ProtoBase<PriorityProto> implements Priority {
|
||||
public class PriorityPBImpl extends Priority {
|
||||
PriorityProto proto = PriorityProto.getDefaultInstance();
|
||||
PriorityProto.Builder builder = null;
|
||||
boolean viaProto = false;
|
||||
|
@ -66,11 +62,4 @@ public class PriorityPBImpl extends ProtoBase<PriorityProto> implements Priority
|
|||
builder.setPriority((priority));
|
||||
}
|
||||
|
||||
|
||||
@Override
|
||||
public int compareTo(Priority other) {
|
||||
return this.getPriority() - other.getPriority();
|
||||
}
|
||||
|
||||
|
||||
}
|
||||
|
|
|
@ -20,19 +20,14 @@ package org.apache.hadoop.yarn.api.records.impl.pb;
|
|||
|
||||
|
||||
import org.apache.hadoop.yarn.api.records.Priority;
|
||||
import org.apache.hadoop.yarn.api.records.ProtoBase;
|
||||
import org.apache.hadoop.yarn.api.records.Resource;
|
||||
import org.apache.hadoop.yarn.api.records.ResourceRequest;
|
||||
import org.apache.hadoop.yarn.api.records.impl.pb.PriorityPBImpl;
|
||||
import org.apache.hadoop.yarn.api.records.impl.pb.ResourcePBImpl;
|
||||
import org.apache.hadoop.yarn.proto.YarnProtos.PriorityProto;
|
||||
import org.apache.hadoop.yarn.proto.YarnProtos.ResourceProto;
|
||||
import org.apache.hadoop.yarn.proto.YarnProtos.ResourceRequestProto;
|
||||
import org.apache.hadoop.yarn.proto.YarnProtos.ResourceRequestProtoOrBuilder;
|
||||
|
||||
|
||||
|
||||
public class ResourceRequestPBImpl extends ProtoBase<ResourceRequestProto> implements ResourceRequest {
|
||||
public class ResourceRequestPBImpl extends ResourceRequest {
|
||||
ResourceRequestProto proto = ResourceRequestProto.getDefaultInstance();
|
||||
ResourceRequestProto.Builder builder = null;
|
||||
boolean viaProto = false;
|
||||
|
@ -168,25 +163,4 @@ public class ResourceRequestPBImpl extends ProtoBase<ResourceRequestProto> imple
|
|||
return ((ResourcePBImpl)t).getProto();
|
||||
}
|
||||
|
||||
@Override
|
||||
public int compareTo(ResourceRequest other) {
|
||||
if (this.getPriority().compareTo(other.getPriority()) == 0) {
|
||||
if (this.getHostName().equals(other.getHostName())) {
|
||||
if (this.getCapability().equals(other.getCapability())) {
|
||||
if (this.getNumContainers() == other.getNumContainers()) {
|
||||
return 0;
|
||||
} else {
|
||||
return this.getNumContainers() - other.getNumContainers();
|
||||
}
|
||||
} else {
|
||||
return this.getCapability().compareTo(other.getCapability());
|
||||
}
|
||||
} else {
|
||||
return this.getHostName().compareTo(other.getHostName());
|
||||
}
|
||||
} else {
|
||||
return this.getPriority().compareTo(other.getPriority());
|
||||
}
|
||||
}
|
||||
|
||||
}
|
||||
|
|
Loading…
Reference in New Issue