YARn-4844. Add getMemoryLong/getVirtualCoreLong to o.a.h.y.api.records.Resource. Contributed by Wangda Tan.
This commit is contained in:
parent
e5b1fb2a91
commit
457884737f
|
@ -689,9 +689,9 @@ public class JobHistoryEventHandler extends AbstractService
|
||||||
NormalizedResourceEvent normalizedResourceEvent =
|
NormalizedResourceEvent normalizedResourceEvent =
|
||||||
(NormalizedResourceEvent) event;
|
(NormalizedResourceEvent) event;
|
||||||
if (normalizedResourceEvent.getTaskType() == TaskType.MAP) {
|
if (normalizedResourceEvent.getTaskType() == TaskType.MAP) {
|
||||||
summary.setResourcesPerMap(normalizedResourceEvent.getMemory());
|
summary.setResourcesPerMap((int) normalizedResourceEvent.getMemory());
|
||||||
} else if (normalizedResourceEvent.getTaskType() == TaskType.REDUCE) {
|
} else if (normalizedResourceEvent.getTaskType() == TaskType.REDUCE) {
|
||||||
summary.setResourcesPerReduce(normalizedResourceEvent.getMemory());
|
summary.setResourcesPerReduce((int) normalizedResourceEvent.getMemory());
|
||||||
}
|
}
|
||||||
break;
|
break;
|
||||||
case JOB_INITED:
|
case JOB_INITED:
|
||||||
|
|
|
@ -1436,6 +1436,7 @@ public abstract class TaskAttemptImpl implements
|
||||||
TaskAttemptImpl taskAttempt) {
|
TaskAttemptImpl taskAttempt) {
|
||||||
TaskType taskType = taskAttempt.getID().getTaskId().getTaskType();
|
TaskType taskType = taskAttempt.getID().getTaskId().getTaskType();
|
||||||
long duration = (taskAttempt.getFinishTime() - taskAttempt.getLaunchTime());
|
long duration = (taskAttempt.getFinishTime() - taskAttempt.getLaunchTime());
|
||||||
|
|
||||||
int mbRequired =
|
int mbRequired =
|
||||||
taskAttempt.getMemoryRequired(taskAttempt.conf, taskType);
|
taskAttempt.getMemoryRequired(taskAttempt.conf, taskType);
|
||||||
int vcoresRequired = taskAttempt.getCpuRequired(taskAttempt.conf, taskType);
|
int vcoresRequired = taskAttempt.getCpuRequired(taskAttempt.conf, taskType);
|
||||||
|
|
|
@ -357,10 +357,10 @@ public class RMContainerAllocator extends RMContainerRequestor
|
||||||
eventHandler.handle(new JobHistoryEvent(jobId,
|
eventHandler.handle(new JobHistoryEvent(jobId,
|
||||||
new NormalizedResourceEvent(
|
new NormalizedResourceEvent(
|
||||||
org.apache.hadoop.mapreduce.TaskType.MAP, mapResourceRequest
|
org.apache.hadoop.mapreduce.TaskType.MAP, mapResourceRequest
|
||||||
.getMemory())));
|
.getMemorySize())));
|
||||||
LOG.info("mapResourceRequest:" + mapResourceRequest);
|
LOG.info("mapResourceRequest:" + mapResourceRequest);
|
||||||
if (mapResourceRequest.getMemory() > supportedMaxContainerCapability
|
if (mapResourceRequest.getMemorySize() > supportedMaxContainerCapability
|
||||||
.getMemory()
|
.getMemorySize()
|
||||||
|| mapResourceRequest.getVirtualCores() > supportedMaxContainerCapability
|
|| mapResourceRequest.getVirtualCores() > supportedMaxContainerCapability
|
||||||
.getVirtualCores()) {
|
.getVirtualCores()) {
|
||||||
String diagMsg =
|
String diagMsg =
|
||||||
|
@ -374,7 +374,7 @@ public class RMContainerAllocator extends RMContainerRequestor
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
// set the resources
|
// set the resources
|
||||||
reqEvent.getCapability().setMemory(mapResourceRequest.getMemory());
|
reqEvent.getCapability().setMemory(mapResourceRequest.getMemorySize());
|
||||||
reqEvent.getCapability().setVirtualCores(
|
reqEvent.getCapability().setVirtualCores(
|
||||||
mapResourceRequest.getVirtualCores());
|
mapResourceRequest.getVirtualCores());
|
||||||
scheduledRequests.addMap(reqEvent);//maps are immediately scheduled
|
scheduledRequests.addMap(reqEvent);//maps are immediately scheduled
|
||||||
|
@ -384,10 +384,10 @@ public class RMContainerAllocator extends RMContainerRequestor
|
||||||
eventHandler.handle(new JobHistoryEvent(jobId,
|
eventHandler.handle(new JobHistoryEvent(jobId,
|
||||||
new NormalizedResourceEvent(
|
new NormalizedResourceEvent(
|
||||||
org.apache.hadoop.mapreduce.TaskType.REDUCE,
|
org.apache.hadoop.mapreduce.TaskType.REDUCE,
|
||||||
reduceResourceRequest.getMemory())));
|
reduceResourceRequest.getMemorySize())));
|
||||||
LOG.info("reduceResourceRequest:" + reduceResourceRequest);
|
LOG.info("reduceResourceRequest:" + reduceResourceRequest);
|
||||||
if (reduceResourceRequest.getMemory() > supportedMaxContainerCapability
|
if (reduceResourceRequest.getMemorySize() > supportedMaxContainerCapability
|
||||||
.getMemory()
|
.getMemorySize()
|
||||||
|| reduceResourceRequest.getVirtualCores() > supportedMaxContainerCapability
|
|| reduceResourceRequest.getVirtualCores() > supportedMaxContainerCapability
|
||||||
.getVirtualCores()) {
|
.getVirtualCores()) {
|
||||||
String diagMsg =
|
String diagMsg =
|
||||||
|
@ -402,7 +402,7 @@ public class RMContainerAllocator extends RMContainerRequestor
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
// set the resources
|
// set the resources
|
||||||
reqEvent.getCapability().setMemory(reduceResourceRequest.getMemory());
|
reqEvent.getCapability().setMemory(reduceResourceRequest.getMemorySize());
|
||||||
reqEvent.getCapability().setVirtualCores(
|
reqEvent.getCapability().setVirtualCores(
|
||||||
reduceResourceRequest.getVirtualCores());
|
reduceResourceRequest.getVirtualCores());
|
||||||
if (reqEvent.getEarlierAttemptFailed()) {
|
if (reqEvent.getEarlierAttemptFailed()) {
|
||||||
|
|
|
@ -20,43 +20,42 @@ package org.apache.hadoop.mapreduce.v2.app.rm;
|
||||||
|
|
||||||
import org.apache.hadoop.yarn.api.records.Resource;
|
import org.apache.hadoop.yarn.api.records.Resource;
|
||||||
import org.apache.hadoop.yarn.proto.YarnServiceProtos.SchedulerResourceTypes;
|
import org.apache.hadoop.yarn.proto.YarnServiceProtos.SchedulerResourceTypes;
|
||||||
import org.apache.hadoop.yarn.util.Records;
|
|
||||||
|
|
||||||
import java.util.EnumSet;
|
import java.util.EnumSet;
|
||||||
|
|
||||||
public class ResourceCalculatorUtils {
|
public class ResourceCalculatorUtils {
|
||||||
public static int divideAndCeil(int a, int b) {
|
public static int divideAndCeil(long a, long b) {
|
||||||
if (b == 0) {
|
if (b == 0) {
|
||||||
return 0;
|
return 0;
|
||||||
}
|
}
|
||||||
return (a + (b - 1)) / b;
|
return (int) ((a + (b - 1)) / b);
|
||||||
}
|
}
|
||||||
|
|
||||||
public static int computeAvailableContainers(Resource available,
|
public static int computeAvailableContainers(Resource available,
|
||||||
Resource required, EnumSet<SchedulerResourceTypes> resourceTypes) {
|
Resource required, EnumSet<SchedulerResourceTypes> resourceTypes) {
|
||||||
if (resourceTypes.contains(SchedulerResourceTypes.CPU)) {
|
if (resourceTypes.contains(SchedulerResourceTypes.CPU)) {
|
||||||
return Math.min(
|
return Math.min(
|
||||||
calculateRatioOrMaxValue(available.getMemory(), required.getMemory()),
|
calculateRatioOrMaxValue(available.getMemorySize(), required.getMemorySize()),
|
||||||
calculateRatioOrMaxValue(available.getVirtualCores(), required
|
calculateRatioOrMaxValue(available.getVirtualCores(), required
|
||||||
.getVirtualCores()));
|
.getVirtualCores()));
|
||||||
}
|
}
|
||||||
return calculateRatioOrMaxValue(
|
return calculateRatioOrMaxValue(
|
||||||
available.getMemory(), required.getMemory());
|
available.getMemorySize(), required.getMemorySize());
|
||||||
}
|
}
|
||||||
|
|
||||||
public static int divideAndCeilContainers(Resource required, Resource factor,
|
public static int divideAndCeilContainers(Resource required, Resource factor,
|
||||||
EnumSet<SchedulerResourceTypes> resourceTypes) {
|
EnumSet<SchedulerResourceTypes> resourceTypes) {
|
||||||
if (resourceTypes.contains(SchedulerResourceTypes.CPU)) {
|
if (resourceTypes.contains(SchedulerResourceTypes.CPU)) {
|
||||||
return Math.max(divideAndCeil(required.getMemory(), factor.getMemory()),
|
return Math.max(divideAndCeil(required.getMemorySize(), factor.getMemorySize()),
|
||||||
divideAndCeil(required.getVirtualCores(), factor.getVirtualCores()));
|
divideAndCeil(required.getVirtualCores(), factor.getVirtualCores()));
|
||||||
}
|
}
|
||||||
return divideAndCeil(required.getMemory(), factor.getMemory());
|
return divideAndCeil(required.getMemorySize(), factor.getMemorySize());
|
||||||
}
|
}
|
||||||
|
|
||||||
private static int calculateRatioOrMaxValue(int numerator, int denominator) {
|
private static int calculateRatioOrMaxValue(long numerator, long denominator) {
|
||||||
if (denominator == 0) {
|
if (denominator == 0) {
|
||||||
return Integer.MAX_VALUE;
|
return Integer.MAX_VALUE;
|
||||||
}
|
}
|
||||||
return numerator / denominator;
|
return (int) (numerator / denominator);
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
|
|
|
@ -1789,7 +1789,7 @@ public class TestRecovery {
|
||||||
int appAttemptId = 3;
|
int appAttemptId = 3;
|
||||||
MRAppMetrics metrics = mock(MRAppMetrics.class);
|
MRAppMetrics metrics = mock(MRAppMetrics.class);
|
||||||
Resource minContainerRequirements = mock(Resource.class);
|
Resource minContainerRequirements = mock(Resource.class);
|
||||||
when(minContainerRequirements.getMemory()).thenReturn(1000);
|
when(minContainerRequirements.getMemorySize()).thenReturn(1000L);
|
||||||
|
|
||||||
ClusterInfo clusterInfo = mock(ClusterInfo.class);
|
ClusterInfo clusterInfo = mock(ClusterInfo.class);
|
||||||
AppContext appContext = mock(AppContext.class);
|
AppContext appContext = mock(AppContext.class);
|
||||||
|
|
|
@ -305,6 +305,7 @@ public class TestTaskAttempt{
|
||||||
Assert.assertEquals(rta.getLaunchTime(), 10);
|
Assert.assertEquals(rta.getLaunchTime(), 10);
|
||||||
Counters counters = job.getAllCounters();
|
Counters counters = job.getAllCounters();
|
||||||
Assert.assertEquals((int) Math.ceil((float) mapMemMb / minContainerSize),
|
Assert.assertEquals((int) Math.ceil((float) mapMemMb / minContainerSize),
|
||||||
|
|
||||||
counters.findCounter(JobCounter.SLOTS_MILLIS_MAPS).getValue());
|
counters.findCounter(JobCounter.SLOTS_MILLIS_MAPS).getValue());
|
||||||
Assert.assertEquals((int) Math.ceil((float) reduceMemMb / minContainerSize),
|
Assert.assertEquals((int) Math.ceil((float) reduceMemMb / minContainerSize),
|
||||||
counters.findCounter(JobCounter.SLOTS_MILLIS_REDUCES).getValue());
|
counters.findCounter(JobCounter.SLOTS_MILLIS_REDUCES).getValue());
|
||||||
|
@ -566,7 +567,7 @@ public class TestTaskAttempt{
|
||||||
ClusterInfo clusterInfo = mock(ClusterInfo.class);
|
ClusterInfo clusterInfo = mock(ClusterInfo.class);
|
||||||
Resource resource = mock(Resource.class);
|
Resource resource = mock(Resource.class);
|
||||||
when(appCtx.getClusterInfo()).thenReturn(clusterInfo);
|
when(appCtx.getClusterInfo()).thenReturn(clusterInfo);
|
||||||
when(resource.getMemory()).thenReturn(1024);
|
when(resource.getMemorySize()).thenReturn(1024L);
|
||||||
setupTaskAttemptFinishingMonitor(eventHandler, jobConf, appCtx);
|
setupTaskAttemptFinishingMonitor(eventHandler, jobConf, appCtx);
|
||||||
|
|
||||||
TaskAttemptImpl taImpl =
|
TaskAttemptImpl taImpl =
|
||||||
|
@ -624,7 +625,7 @@ public class TestTaskAttempt{
|
||||||
ClusterInfo clusterInfo = mock(ClusterInfo.class);
|
ClusterInfo clusterInfo = mock(ClusterInfo.class);
|
||||||
Resource resource = mock(Resource.class);
|
Resource resource = mock(Resource.class);
|
||||||
when(appCtx.getClusterInfo()).thenReturn(clusterInfo);
|
when(appCtx.getClusterInfo()).thenReturn(clusterInfo);
|
||||||
when(resource.getMemory()).thenReturn(1024);
|
when(resource.getMemorySize()).thenReturn(1024L);
|
||||||
setupTaskAttemptFinishingMonitor(eventHandler, jobConf, appCtx);
|
setupTaskAttemptFinishingMonitor(eventHandler, jobConf, appCtx);
|
||||||
|
|
||||||
TaskAttemptImpl taImpl =
|
TaskAttemptImpl taImpl =
|
||||||
|
@ -688,7 +689,7 @@ public class TestTaskAttempt{
|
||||||
ClusterInfo clusterInfo = mock(ClusterInfo.class);
|
ClusterInfo clusterInfo = mock(ClusterInfo.class);
|
||||||
Resource resource = mock(Resource.class);
|
Resource resource = mock(Resource.class);
|
||||||
when(appCtx.getClusterInfo()).thenReturn(clusterInfo);
|
when(appCtx.getClusterInfo()).thenReturn(clusterInfo);
|
||||||
when(resource.getMemory()).thenReturn(1024);
|
when(resource.getMemorySize()).thenReturn(1024L);
|
||||||
setupTaskAttemptFinishingMonitor(eventHandler, jobConf, appCtx);
|
setupTaskAttemptFinishingMonitor(eventHandler, jobConf, appCtx);
|
||||||
|
|
||||||
TaskAttemptImpl taImpl =
|
TaskAttemptImpl taImpl =
|
||||||
|
@ -758,7 +759,7 @@ public class TestTaskAttempt{
|
||||||
ClusterInfo clusterInfo = mock(ClusterInfo.class);
|
ClusterInfo clusterInfo = mock(ClusterInfo.class);
|
||||||
Resource resource = mock(Resource.class);
|
Resource resource = mock(Resource.class);
|
||||||
when(appCtx.getClusterInfo()).thenReturn(clusterInfo);
|
when(appCtx.getClusterInfo()).thenReturn(clusterInfo);
|
||||||
when(resource.getMemory()).thenReturn(1024);
|
when(resource.getMemorySize()).thenReturn(1024L);
|
||||||
setupTaskAttemptFinishingMonitor(eventHandler, jobConf, appCtx);
|
setupTaskAttemptFinishingMonitor(eventHandler, jobConf, appCtx);
|
||||||
|
|
||||||
TaskAttemptImpl taImpl = new MapTaskAttemptImpl(taskId, 1, eventHandler,
|
TaskAttemptImpl taImpl = new MapTaskAttemptImpl(taskId, 1, eventHandler,
|
||||||
|
@ -815,7 +816,7 @@ public class TestTaskAttempt{
|
||||||
ClusterInfo clusterInfo = mock(ClusterInfo.class);
|
ClusterInfo clusterInfo = mock(ClusterInfo.class);
|
||||||
Resource resource = mock(Resource.class);
|
Resource resource = mock(Resource.class);
|
||||||
when(appCtx.getClusterInfo()).thenReturn(clusterInfo);
|
when(appCtx.getClusterInfo()).thenReturn(clusterInfo);
|
||||||
when(resource.getMemory()).thenReturn(1024);
|
when(resource.getMemorySize()).thenReturn(1024L);
|
||||||
setupTaskAttemptFinishingMonitor(eventHandler, jobConf, appCtx);
|
setupTaskAttemptFinishingMonitor(eventHandler, jobConf, appCtx);
|
||||||
|
|
||||||
TaskAttemptImpl taImpl =
|
TaskAttemptImpl taImpl =
|
||||||
|
@ -883,7 +884,7 @@ public class TestTaskAttempt{
|
||||||
ClusterInfo clusterInfo = mock(ClusterInfo.class);
|
ClusterInfo clusterInfo = mock(ClusterInfo.class);
|
||||||
Resource resource = mock(Resource.class);
|
Resource resource = mock(Resource.class);
|
||||||
when(appCtx.getClusterInfo()).thenReturn(clusterInfo);
|
when(appCtx.getClusterInfo()).thenReturn(clusterInfo);
|
||||||
when(resource.getMemory()).thenReturn(1024);
|
when(resource.getMemorySize()).thenReturn(1024L);
|
||||||
setupTaskAttemptFinishingMonitor(eventHandler, jobConf, appCtx);
|
setupTaskAttemptFinishingMonitor(eventHandler, jobConf, appCtx);
|
||||||
|
|
||||||
TaskAttemptImpl taImpl = new MapTaskAttemptImpl(taskId, 1, eventHandler,
|
TaskAttemptImpl taImpl = new MapTaskAttemptImpl(taskId, 1, eventHandler,
|
||||||
|
@ -1043,7 +1044,7 @@ public class TestTaskAttempt{
|
||||||
ClusterInfo clusterInfo = mock(ClusterInfo.class);
|
ClusterInfo clusterInfo = mock(ClusterInfo.class);
|
||||||
Resource resource = mock(Resource.class);
|
Resource resource = mock(Resource.class);
|
||||||
when(appCtx.getClusterInfo()).thenReturn(clusterInfo);
|
when(appCtx.getClusterInfo()).thenReturn(clusterInfo);
|
||||||
when(resource.getMemory()).thenReturn(1024);
|
when(resource.getMemorySize()).thenReturn(1024L);
|
||||||
|
|
||||||
TaskAttemptImpl taImpl = new MapTaskAttemptImpl(taskId, 1, eventHandler,
|
TaskAttemptImpl taImpl = new MapTaskAttemptImpl(taskId, 1, eventHandler,
|
||||||
jobFile, 1, splits, jobConf, taListener, new Token(),
|
jobFile, 1, splits, jobConf, taListener, new Token(),
|
||||||
|
@ -1097,7 +1098,7 @@ public class TestTaskAttempt{
|
||||||
ClusterInfo clusterInfo = mock(ClusterInfo.class);
|
ClusterInfo clusterInfo = mock(ClusterInfo.class);
|
||||||
Resource resource = mock(Resource.class);
|
Resource resource = mock(Resource.class);
|
||||||
when(appCtx.getClusterInfo()).thenReturn(clusterInfo);
|
when(appCtx.getClusterInfo()).thenReturn(clusterInfo);
|
||||||
when(resource.getMemory()).thenReturn(1024);
|
when(resource.getMemorySize()).thenReturn(1024L);
|
||||||
|
|
||||||
TaskAttemptImpl taImpl = new MapTaskAttemptImpl(taskId, 1, eventHandler,
|
TaskAttemptImpl taImpl = new MapTaskAttemptImpl(taskId, 1, eventHandler,
|
||||||
jobFile, 1, splits, jobConf, taListener, new Token(),
|
jobFile, 1, splits, jobConf, taListener, new Token(),
|
||||||
|
@ -1154,7 +1155,7 @@ public class TestTaskAttempt{
|
||||||
ClusterInfo clusterInfo = mock(ClusterInfo.class);
|
ClusterInfo clusterInfo = mock(ClusterInfo.class);
|
||||||
Resource resource = mock(Resource.class);
|
Resource resource = mock(Resource.class);
|
||||||
when(appCtx.getClusterInfo()).thenReturn(clusterInfo);
|
when(appCtx.getClusterInfo()).thenReturn(clusterInfo);
|
||||||
when(resource.getMemory()).thenReturn(1024);
|
when(resource.getMemorySize()).thenReturn(1024L);
|
||||||
|
|
||||||
TaskAttemptImpl taImpl = new MapTaskAttemptImpl(taskId, 1, eventHandler,
|
TaskAttemptImpl taImpl = new MapTaskAttemptImpl(taskId, 1, eventHandler,
|
||||||
jobFile, 1, splits, jobConf, taListener, new Token(),
|
jobFile, 1, splits, jobConf, taListener, new Token(),
|
||||||
|
|
|
@ -1771,7 +1771,7 @@ public class TestRMContainerAllocator {
|
||||||
when(excessC.getId()).thenReturn(containerId);
|
when(excessC.getId()).thenReturn(containerId);
|
||||||
when(excessC.getPriority()).thenReturn(RMContainerAllocator.PRIORITY_REDUCE);
|
when(excessC.getPriority()).thenReturn(RMContainerAllocator.PRIORITY_REDUCE);
|
||||||
Resource mockR = mock(Resource.class);
|
Resource mockR = mock(Resource.class);
|
||||||
when(mockR.getMemory()).thenReturn(2048);
|
when(mockR.getMemorySize()).thenReturn(2048L);
|
||||||
when(excessC.getResource()).thenReturn(mockR);
|
when(excessC.getResource()).thenReturn(mockR);
|
||||||
NodeId nId = mock(NodeId.class);
|
NodeId nId = mock(NodeId.class);
|
||||||
when(nId.getHost()).thenReturn("local");
|
when(nId.getHost()).thenReturn("local");
|
||||||
|
|
|
@ -47,8 +47,8 @@ public class TestResourceCalculatorUtils {
|
||||||
Integer.MAX_VALUE,
|
Integer.MAX_VALUE,
|
||||||
expectedNumberOfContainersForCPU);
|
expectedNumberOfContainersForCPU);
|
||||||
|
|
||||||
Resource zeroCpuResource = Resource.newInstance(nonZeroResource.getMemory(),
|
Resource zeroCpuResource = Resource.newInstance(
|
||||||
0);
|
nonZeroResource.getMemorySize(), 0);
|
||||||
|
|
||||||
verifyDifferentResourceTypes(clusterAvailableResources, zeroCpuResource,
|
verifyDifferentResourceTypes(clusterAvailableResources, zeroCpuResource,
|
||||||
expectedNumberOfContainersForMemory,
|
expectedNumberOfContainersForMemory,
|
||||||
|
|
|
@ -520,13 +520,13 @@ public class TypeConverter {
|
||||||
application.getApplicationResourceUsageReport();
|
application.getApplicationResourceUsageReport();
|
||||||
if (resourceUsageReport != null) {
|
if (resourceUsageReport != null) {
|
||||||
jobStatus.setNeededMem(
|
jobStatus.setNeededMem(
|
||||||
resourceUsageReport.getNeededResources().getMemory());
|
resourceUsageReport.getNeededResources().getMemorySize());
|
||||||
jobStatus.setNumReservedSlots(
|
jobStatus.setNumReservedSlots(
|
||||||
resourceUsageReport.getNumReservedContainers());
|
resourceUsageReport.getNumReservedContainers());
|
||||||
jobStatus.setNumUsedSlots(resourceUsageReport.getNumUsedContainers());
|
jobStatus.setNumUsedSlots(resourceUsageReport.getNumUsedContainers());
|
||||||
jobStatus.setReservedMem(
|
jobStatus.setReservedMem(
|
||||||
resourceUsageReport.getReservedResources().getMemory());
|
resourceUsageReport.getReservedResources().getMemorySize());
|
||||||
jobStatus.setUsedMem(resourceUsageReport.getUsedResources().getMemory());
|
jobStatus.setUsedMem(resourceUsageReport.getUsedResources().getMemorySize());
|
||||||
}
|
}
|
||||||
return jobStatus;
|
return jobStatus;
|
||||||
}
|
}
|
||||||
|
|
|
@ -95,9 +95,9 @@ public class JobStatus implements Writable, Cloneable {
|
||||||
private String trackingUrl ="";
|
private String trackingUrl ="";
|
||||||
private int numUsedSlots;
|
private int numUsedSlots;
|
||||||
private int numReservedSlots;
|
private int numReservedSlots;
|
||||||
private int usedMem;
|
private long usedMem;
|
||||||
private int reservedMem;
|
private long reservedMem;
|
||||||
private int neededMem;
|
private long neededMem;
|
||||||
private boolean isUber;
|
private boolean isUber;
|
||||||
|
|
||||||
/**
|
/**
|
||||||
|
@ -552,42 +552,42 @@ public class JobStatus implements Writable, Cloneable {
|
||||||
/**
|
/**
|
||||||
* @return the used memory
|
* @return the used memory
|
||||||
*/
|
*/
|
||||||
public int getUsedMem() {
|
public long getUsedMem() {
|
||||||
return usedMem;
|
return usedMem;
|
||||||
}
|
}
|
||||||
|
|
||||||
/**
|
/**
|
||||||
* @param m the used memory
|
* @param m the used memory
|
||||||
*/
|
*/
|
||||||
public void setUsedMem(int m) {
|
public void setUsedMem(long m) {
|
||||||
this.usedMem = m;
|
this.usedMem = m;
|
||||||
}
|
}
|
||||||
|
|
||||||
/**
|
/**
|
||||||
* @return the reserved memory
|
* @return the reserved memory
|
||||||
*/
|
*/
|
||||||
public int getReservedMem() {
|
public long getReservedMem() {
|
||||||
return reservedMem;
|
return reservedMem;
|
||||||
}
|
}
|
||||||
|
|
||||||
/**
|
/**
|
||||||
* @param r the reserved memory
|
* @param r the reserved memory
|
||||||
*/
|
*/
|
||||||
public void setReservedMem(int r) {
|
public void setReservedMem(long r) {
|
||||||
this.reservedMem = r;
|
this.reservedMem = r;
|
||||||
}
|
}
|
||||||
|
|
||||||
/**
|
/**
|
||||||
* @return the needed memory
|
* @return the needed memory
|
||||||
*/
|
*/
|
||||||
public int getNeededMem() {
|
public long getNeededMem() {
|
||||||
return neededMem;
|
return neededMem;
|
||||||
}
|
}
|
||||||
|
|
||||||
/**
|
/**
|
||||||
* @param n the needed memory
|
* @param n the needed memory
|
||||||
*/
|
*/
|
||||||
public void setNeededMem(int n) {
|
public void setNeededMem(long n) {
|
||||||
this.neededMem = n;
|
this.neededMem = n;
|
||||||
}
|
}
|
||||||
|
|
||||||
|
|
|
@ -28,7 +28,7 @@ import org.apache.hadoop.mapreduce.TaskType;
|
||||||
@InterfaceAudience.Private
|
@InterfaceAudience.Private
|
||||||
@InterfaceStability.Unstable
|
@InterfaceStability.Unstable
|
||||||
public class NormalizedResourceEvent implements HistoryEvent {
|
public class NormalizedResourceEvent implements HistoryEvent {
|
||||||
private int memory;
|
private long memory;
|
||||||
private TaskType taskType;
|
private TaskType taskType;
|
||||||
|
|
||||||
/**
|
/**
|
||||||
|
@ -36,7 +36,7 @@ public class NormalizedResourceEvent implements HistoryEvent {
|
||||||
* @param taskType the tasktype of the request.
|
* @param taskType the tasktype of the request.
|
||||||
* @param memory the normalized memory requirements.
|
* @param memory the normalized memory requirements.
|
||||||
*/
|
*/
|
||||||
public NormalizedResourceEvent(TaskType taskType, int memory) {
|
public NormalizedResourceEvent(TaskType taskType, long memory) {
|
||||||
this.memory = memory;
|
this.memory = memory;
|
||||||
this.taskType = taskType;
|
this.taskType = taskType;
|
||||||
}
|
}
|
||||||
|
@ -53,7 +53,7 @@ public class NormalizedResourceEvent implements HistoryEvent {
|
||||||
* the normalized memory
|
* the normalized memory
|
||||||
* @return the normalized memory
|
* @return the normalized memory
|
||||||
*/
|
*/
|
||||||
public int getMemory() {
|
public long getMemory() {
|
||||||
return this.memory;
|
return this.memory;
|
||||||
}
|
}
|
||||||
|
|
||||||
|
|
|
@ -666,9 +666,10 @@ public class CLI extends Configured implements Tool {
|
||||||
for (JobStatus job : jobs) {
|
for (JobStatus job : jobs) {
|
||||||
int numUsedSlots = job.getNumUsedSlots();
|
int numUsedSlots = job.getNumUsedSlots();
|
||||||
int numReservedSlots = job.getNumReservedSlots();
|
int numReservedSlots = job.getNumReservedSlots();
|
||||||
int usedMem = job.getUsedMem();
|
|
||||||
int rsvdMem = job.getReservedMem();
|
long usedMem = job.getUsedMem();
|
||||||
int neededMem = job.getNeededMem();
|
long rsvdMem = job.getReservedMem();
|
||||||
|
long neededMem = job.getNeededMem();
|
||||||
writer.printf(dataPattern,
|
writer.printf(dataPattern,
|
||||||
job.getJobID().toString(), job.getState(), job.getStartTime(),
|
job.getJobID().toString(), job.getState(), job.getStartTime(),
|
||||||
job.getUsername(), job.getQueue(),
|
job.getUsername(), job.getQueue(),
|
||||||
|
|
|
@ -170,9 +170,9 @@ public class JobClientUnitTest {
|
||||||
when(mockJobStatus.getPriority()).thenReturn(JobPriority.NORMAL);
|
when(mockJobStatus.getPriority()).thenReturn(JobPriority.NORMAL);
|
||||||
when(mockJobStatus.getNumUsedSlots()).thenReturn(1);
|
when(mockJobStatus.getNumUsedSlots()).thenReturn(1);
|
||||||
when(mockJobStatus.getNumReservedSlots()).thenReturn(1);
|
when(mockJobStatus.getNumReservedSlots()).thenReturn(1);
|
||||||
when(mockJobStatus.getUsedMem()).thenReturn(1024);
|
when(mockJobStatus.getUsedMem()).thenReturn(1024L);
|
||||||
when(mockJobStatus.getReservedMem()).thenReturn(512);
|
when(mockJobStatus.getReservedMem()).thenReturn(512L);
|
||||||
when(mockJobStatus.getNeededMem()).thenReturn(2048);
|
when(mockJobStatus.getNeededMem()).thenReturn(2048L);
|
||||||
when(mockJobStatus.getSchedulingInfo()).thenReturn("NA");
|
when(mockJobStatus.getSchedulingInfo()).thenReturn("NA");
|
||||||
|
|
||||||
Job mockJob = mock(Job.class);
|
Job mockJob = mock(Job.class);
|
||||||
|
|
|
@ -67,10 +67,10 @@ public class FairSchedulerMetrics extends SchedulerMetrics {
|
||||||
FairScheduler fair = (FairScheduler) scheduler;
|
FairScheduler fair = (FairScheduler) scheduler;
|
||||||
final FSAppAttempt app = fair.getSchedulerApp(appAttemptId);
|
final FSAppAttempt app = fair.getSchedulerApp(appAttemptId);
|
||||||
metrics.register("variable.app." + oldAppId + ".demand.memory",
|
metrics.register("variable.app." + oldAppId + ".demand.memory",
|
||||||
new Gauge<Integer>() {
|
new Gauge<Long>() {
|
||||||
@Override
|
@Override
|
||||||
public Integer getValue() {
|
public Long getValue() {
|
||||||
return app.getDemand().getMemory();
|
return app.getDemand().getMemorySize();
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
);
|
);
|
||||||
|
@ -83,10 +83,10 @@ public class FairSchedulerMetrics extends SchedulerMetrics {
|
||||||
}
|
}
|
||||||
);
|
);
|
||||||
metrics.register("variable.app." + oldAppId + ".usage.memory",
|
metrics.register("variable.app." + oldAppId + ".usage.memory",
|
||||||
new Gauge<Integer>() {
|
new Gauge<Long>() {
|
||||||
@Override
|
@Override
|
||||||
public Integer getValue() {
|
public Long getValue() {
|
||||||
return app.getResourceUsage().getMemory();
|
return app.getResourceUsage().getMemorySize();
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
);
|
);
|
||||||
|
@ -99,26 +99,26 @@ public class FairSchedulerMetrics extends SchedulerMetrics {
|
||||||
}
|
}
|
||||||
);
|
);
|
||||||
metrics.register("variable.app." + oldAppId + ".minshare.memory",
|
metrics.register("variable.app." + oldAppId + ".minshare.memory",
|
||||||
new Gauge<Integer>() {
|
new Gauge<Long>() {
|
||||||
@Override
|
@Override
|
||||||
public Integer getValue() {
|
public Long getValue() {
|
||||||
return app.getMinShare().getMemory();
|
return app.getMinShare().getMemorySize();
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
);
|
);
|
||||||
metrics.register("variable.app." + oldAppId + ".minshare.vcores",
|
metrics.register("variable.app." + oldAppId + ".minshare.vcores",
|
||||||
new Gauge<Integer>() {
|
new Gauge<Long>() {
|
||||||
@Override
|
@Override
|
||||||
public Integer getValue() {
|
public Long getValue() {
|
||||||
return app.getMinShare().getMemory();
|
return app.getMinShare().getMemorySize();
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
);
|
);
|
||||||
metrics.register("variable.app." + oldAppId + ".maxshare.memory",
|
metrics.register("variable.app." + oldAppId + ".maxshare.memory",
|
||||||
new Gauge<Integer>() {
|
new Gauge<Long>() {
|
||||||
@Override
|
@Override
|
||||||
public Integer getValue() {
|
public Long getValue() {
|
||||||
return Math.min(app.getMaxShare().getMemory(), totalMemoryMB);
|
return Math.min(app.getMaxShare().getMemorySize(), totalMemoryMB);
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
);
|
);
|
||||||
|
@ -154,10 +154,10 @@ public class FairSchedulerMetrics extends SchedulerMetrics {
|
||||||
FairScheduler fair = (FairScheduler) scheduler;
|
FairScheduler fair = (FairScheduler) scheduler;
|
||||||
final FSQueue queue = fair.getQueueManager().getQueue(queueName);
|
final FSQueue queue = fair.getQueueManager().getQueue(queueName);
|
||||||
metrics.register("variable.queue." + queueName + ".demand.memory",
|
metrics.register("variable.queue." + queueName + ".demand.memory",
|
||||||
new Gauge<Integer>() {
|
new Gauge<Long>() {
|
||||||
@Override
|
@Override
|
||||||
public Integer getValue() {
|
public Long getValue() {
|
||||||
return queue.getDemand().getMemory();
|
return queue.getDemand().getMemorySize();
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
);
|
);
|
||||||
|
@ -170,10 +170,10 @@ public class FairSchedulerMetrics extends SchedulerMetrics {
|
||||||
}
|
}
|
||||||
);
|
);
|
||||||
metrics.register("variable.queue." + queueName + ".usage.memory",
|
metrics.register("variable.queue." + queueName + ".usage.memory",
|
||||||
new Gauge<Integer>() {
|
new Gauge<Long>() {
|
||||||
@Override
|
@Override
|
||||||
public Integer getValue() {
|
public Long getValue() {
|
||||||
return queue.getResourceUsage().getMemory();
|
return queue.getResourceUsage().getMemorySize();
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
);
|
);
|
||||||
|
@ -186,10 +186,10 @@ public class FairSchedulerMetrics extends SchedulerMetrics {
|
||||||
}
|
}
|
||||||
);
|
);
|
||||||
metrics.register("variable.queue." + queueName + ".minshare.memory",
|
metrics.register("variable.queue." + queueName + ".minshare.memory",
|
||||||
new Gauge<Integer>() {
|
new Gauge<Long>() {
|
||||||
@Override
|
@Override
|
||||||
public Integer getValue() {
|
public Long getValue() {
|
||||||
return queue.getMinShare().getMemory();
|
return queue.getMinShare().getMemorySize();
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
);
|
);
|
||||||
|
@ -202,9 +202,9 @@ public class FairSchedulerMetrics extends SchedulerMetrics {
|
||||||
}
|
}
|
||||||
);
|
);
|
||||||
metrics.register("variable.queue." + queueName + ".maxshare.memory",
|
metrics.register("variable.queue." + queueName + ".maxshare.memory",
|
||||||
new Gauge<Integer>() {
|
new Gauge<Long>() {
|
||||||
@Override
|
@Override
|
||||||
public Integer getValue() {
|
public Long getValue() {
|
||||||
if (! maxReset &&
|
if (! maxReset &&
|
||||||
SLSRunner.simulateInfoMap.containsKey("Number of nodes") &&
|
SLSRunner.simulateInfoMap.containsKey("Number of nodes") &&
|
||||||
SLSRunner.simulateInfoMap.containsKey("Node memory (MB)") &&
|
SLSRunner.simulateInfoMap.containsKey("Node memory (MB)") &&
|
||||||
|
@ -221,7 +221,7 @@ public class FairSchedulerMetrics extends SchedulerMetrics {
|
||||||
maxReset = false;
|
maxReset = false;
|
||||||
}
|
}
|
||||||
|
|
||||||
return Math.min(queue.getMaxShare().getMemory(), totalMemoryMB);
|
return Math.min(queue.getMaxShare().getMemorySize(), totalMemoryMB);
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
);
|
);
|
||||||
|
@ -234,10 +234,10 @@ public class FairSchedulerMetrics extends SchedulerMetrics {
|
||||||
}
|
}
|
||||||
);
|
);
|
||||||
metrics.register("variable.queue." + queueName + ".fairshare.memory",
|
metrics.register("variable.queue." + queueName + ".fairshare.memory",
|
||||||
new Gauge<Integer>() {
|
new Gauge<Long>() {
|
||||||
@Override
|
@Override
|
||||||
public Integer getValue() {
|
public Long getValue() {
|
||||||
return queue.getFairShare().getMemory();
|
return queue.getFairShare().getMemorySize();
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
);
|
);
|
||||||
|
|
|
@ -267,7 +267,7 @@ public class ResourceSchedulerWrapper
|
||||||
// should have one container which is AM container
|
// should have one container which is AM container
|
||||||
RMContainer rmc = app.getLiveContainers().iterator().next();
|
RMContainer rmc = app.getLiveContainers().iterator().next();
|
||||||
updateQueueMetrics(queue,
|
updateQueueMetrics(queue,
|
||||||
rmc.getContainer().getResource().getMemory(),
|
rmc.getContainer().getResource().getMemorySize(),
|
||||||
rmc.getContainer().getResource().getVirtualCores());
|
rmc.getContainer().getResource().getVirtualCores());
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
|
@ -323,7 +323,7 @@ public class ResourceSchedulerWrapper
|
||||||
if (status.getExitStatus() == ContainerExitStatus.SUCCESS) {
|
if (status.getExitStatus() == ContainerExitStatus.SUCCESS) {
|
||||||
for (RMContainer rmc : app.getLiveContainers()) {
|
for (RMContainer rmc : app.getLiveContainers()) {
|
||||||
if (rmc.getContainerId() == containerId) {
|
if (rmc.getContainerId() == containerId) {
|
||||||
releasedMemory += rmc.getContainer().getResource().getMemory();
|
releasedMemory += rmc.getContainer().getResource().getMemorySize();
|
||||||
releasedVCores += rmc.getContainer()
|
releasedVCores += rmc.getContainer()
|
||||||
.getResource().getVirtualCores();
|
.getResource().getVirtualCores();
|
||||||
break;
|
break;
|
||||||
|
@ -332,7 +332,7 @@ public class ResourceSchedulerWrapper
|
||||||
} else if (status.getExitStatus() == ContainerExitStatus.ABORTED) {
|
} else if (status.getExitStatus() == ContainerExitStatus.ABORTED) {
|
||||||
if (preemptionContainerMap.containsKey(containerId)) {
|
if (preemptionContainerMap.containsKey(containerId)) {
|
||||||
Resource preResource = preemptionContainerMap.get(containerId);
|
Resource preResource = preemptionContainerMap.get(containerId);
|
||||||
releasedMemory += preResource.getMemory();
|
releasedMemory += preResource.getMemorySize();
|
||||||
releasedVCores += preResource.getVirtualCores();
|
releasedVCores += preResource.getVirtualCores();
|
||||||
preemptionContainerMap.remove(containerId);
|
preemptionContainerMap.remove(containerId);
|
||||||
}
|
}
|
||||||
|
@ -423,9 +423,9 @@ public class ResourceSchedulerWrapper
|
||||||
"counter.queue." + queueName + ".pending.cores",
|
"counter.queue." + queueName + ".pending.cores",
|
||||||
"counter.queue." + queueName + ".allocated.memory",
|
"counter.queue." + queueName + ".allocated.memory",
|
||||||
"counter.queue." + queueName + ".allocated.cores"};
|
"counter.queue." + queueName + ".allocated.cores"};
|
||||||
int values[] = new int[]{pendingResource.getMemory(),
|
long values[] = new long[]{pendingResource.getMemorySize(),
|
||||||
pendingResource.getVirtualCores(),
|
pendingResource.getVirtualCores(),
|
||||||
allocatedResource.getMemory(), allocatedResource.getVirtualCores()};
|
allocatedResource.getMemorySize(), allocatedResource.getVirtualCores()};
|
||||||
for (int i = names.length - 1; i >= 0; i --) {
|
for (int i = names.length - 1; i >= 0; i --) {
|
||||||
if (! counterMap.containsKey(names[i])) {
|
if (! counterMap.containsKey(names[i])) {
|
||||||
metrics.counter(names[i]);
|
metrics.counter(names[i]);
|
||||||
|
@ -531,11 +531,11 @@ public class ResourceSchedulerWrapper
|
||||||
|
|
||||||
private void registerClusterResourceMetrics() {
|
private void registerClusterResourceMetrics() {
|
||||||
metrics.register("variable.cluster.allocated.memory",
|
metrics.register("variable.cluster.allocated.memory",
|
||||||
new Gauge<Integer>() {
|
new Gauge<Long>() {
|
||||||
@Override
|
@Override
|
||||||
public Integer getValue() {
|
public Long getValue() {
|
||||||
if(scheduler == null || scheduler.getRootQueueMetrics() == null) {
|
if(scheduler == null || scheduler.getRootQueueMetrics() == null) {
|
||||||
return 0;
|
return 0L;
|
||||||
} else {
|
} else {
|
||||||
return scheduler.getRootQueueMetrics().getAllocatedMB();
|
return scheduler.getRootQueueMetrics().getAllocatedMB();
|
||||||
}
|
}
|
||||||
|
@ -543,11 +543,11 @@ public class ResourceSchedulerWrapper
|
||||||
}
|
}
|
||||||
);
|
);
|
||||||
metrics.register("variable.cluster.allocated.vcores",
|
metrics.register("variable.cluster.allocated.vcores",
|
||||||
new Gauge<Integer>() {
|
new Gauge<Long>() {
|
||||||
@Override
|
@Override
|
||||||
public Integer getValue() {
|
public Long getValue() {
|
||||||
if(scheduler == null || scheduler.getRootQueueMetrics() == null) {
|
if(scheduler == null || scheduler.getRootQueueMetrics() == null) {
|
||||||
return 0;
|
return 0L;
|
||||||
} else {
|
} else {
|
||||||
return scheduler.getRootQueueMetrics().getAllocatedVirtualCores();
|
return scheduler.getRootQueueMetrics().getAllocatedVirtualCores();
|
||||||
}
|
}
|
||||||
|
@ -555,11 +555,11 @@ public class ResourceSchedulerWrapper
|
||||||
}
|
}
|
||||||
);
|
);
|
||||||
metrics.register("variable.cluster.available.memory",
|
metrics.register("variable.cluster.available.memory",
|
||||||
new Gauge<Integer>() {
|
new Gauge<Long>() {
|
||||||
@Override
|
@Override
|
||||||
public Integer getValue() {
|
public Long getValue() {
|
||||||
if(scheduler == null || scheduler.getRootQueueMetrics() == null) {
|
if(scheduler == null || scheduler.getRootQueueMetrics() == null) {
|
||||||
return 0;
|
return 0L;
|
||||||
} else {
|
} else {
|
||||||
return scheduler.getRootQueueMetrics().getAvailableMB();
|
return scheduler.getRootQueueMetrics().getAvailableMB();
|
||||||
}
|
}
|
||||||
|
@ -567,11 +567,11 @@ public class ResourceSchedulerWrapper
|
||||||
}
|
}
|
||||||
);
|
);
|
||||||
metrics.register("variable.cluster.available.vcores",
|
metrics.register("variable.cluster.available.vcores",
|
||||||
new Gauge<Integer>() {
|
new Gauge<Long>() {
|
||||||
@Override
|
@Override
|
||||||
public Integer getValue() {
|
public Long getValue() {
|
||||||
if(scheduler == null || scheduler.getRootQueueMetrics() == null) {
|
if(scheduler == null || scheduler.getRootQueueMetrics() == null) {
|
||||||
return 0;
|
return 0L;
|
||||||
} else {
|
} else {
|
||||||
return scheduler.getRootQueueMetrics().getAvailableVirtualCores();
|
return scheduler.getRootQueueMetrics().getAvailableVirtualCores();
|
||||||
}
|
}
|
||||||
|
@ -749,7 +749,7 @@ public class ResourceSchedulerWrapper
|
||||||
}
|
}
|
||||||
|
|
||||||
private void updateQueueMetrics(String queue,
|
private void updateQueueMetrics(String queue,
|
||||||
int releasedMemory, int releasedVCores) {
|
long releasedMemory, int releasedVCores) {
|
||||||
// update queue counters
|
// update queue counters
|
||||||
SortedMap<String, Counter> counterMap = metrics.getCounters();
|
SortedMap<String, Counter> counterMap = metrics.getCounters();
|
||||||
if (releasedMemory != 0) {
|
if (releasedMemory != 0) {
|
||||||
|
|
|
@ -18,7 +18,9 @@
|
||||||
|
|
||||||
package org.apache.hadoop.yarn.api.records;
|
package org.apache.hadoop.yarn.api.records;
|
||||||
|
|
||||||
|
import org.apache.hadoop.classification.InterfaceAudience.Private;
|
||||||
import org.apache.hadoop.classification.InterfaceAudience.Public;
|
import org.apache.hadoop.classification.InterfaceAudience.Public;
|
||||||
|
import org.apache.hadoop.classification.InterfaceStability.Unstable;
|
||||||
import org.apache.hadoop.classification.InterfaceStability.Evolving;
|
import org.apache.hadoop.classification.InterfaceStability.Evolving;
|
||||||
import org.apache.hadoop.classification.InterfaceStability.Stable;
|
import org.apache.hadoop.classification.InterfaceStability.Stable;
|
||||||
import org.apache.hadoop.yarn.api.ApplicationMasterProtocol;
|
import org.apache.hadoop.yarn.api.ApplicationMasterProtocol;
|
||||||
|
@ -53,7 +55,7 @@ public abstract class Resource implements Comparable<Resource> {
|
||||||
|
|
||||||
@Public
|
@Public
|
||||||
@Stable
|
@Stable
|
||||||
public static Resource newInstance(int memory, int vCores) {
|
public static Resource newInstance(long memory, long vCores) {
|
||||||
Resource resource = Records.newRecord(Resource.class);
|
Resource resource = Records.newRecord(Resource.class);
|
||||||
resource.setMemory(memory);
|
resource.setMemory(memory);
|
||||||
resource.setVirtualCores(vCores);
|
resource.setVirtualCores(vCores);
|
||||||
|
@ -61,12 +63,23 @@ public abstract class Resource implements Comparable<Resource> {
|
||||||
}
|
}
|
||||||
|
|
||||||
/**
|
/**
|
||||||
|
* This method is DEPRECATED:
|
||||||
|
* Use {@link Resource#getMemorySize()} instead
|
||||||
|
*
|
||||||
* Get <em>memory</em> of the resource.
|
* Get <em>memory</em> of the resource.
|
||||||
* @return <em>memory</em> of the resource
|
* @return <em>memory</em> of the resource
|
||||||
*/
|
*/
|
||||||
@Public
|
@Public
|
||||||
@Stable
|
@Deprecated
|
||||||
public abstract int getMemory();
|
public abstract int getMemory();
|
||||||
|
|
||||||
|
/**
|
||||||
|
* Get <em>memory</em> of the resource.
|
||||||
|
* @return <em>memory</em> of the resource
|
||||||
|
*/
|
||||||
|
@Private
|
||||||
|
@Unstable
|
||||||
|
public abstract long getMemorySize();
|
||||||
|
|
||||||
/**
|
/**
|
||||||
* Set <em>memory</em> of the resource.
|
* Set <em>memory</em> of the resource.
|
||||||
|
@ -74,7 +87,7 @@ public abstract class Resource implements Comparable<Resource> {
|
||||||
*/
|
*/
|
||||||
@Public
|
@Public
|
||||||
@Stable
|
@Stable
|
||||||
public abstract void setMemory(int memory);
|
public abstract void setMemory(long memory);
|
||||||
|
|
||||||
|
|
||||||
/**
|
/**
|
||||||
|
@ -90,6 +103,10 @@ public abstract class Resource implements Comparable<Resource> {
|
||||||
@Public
|
@Public
|
||||||
@Evolving
|
@Evolving
|
||||||
public abstract int getVirtualCores();
|
public abstract int getVirtualCores();
|
||||||
|
|
||||||
|
@Public
|
||||||
|
@Unstable
|
||||||
|
public abstract long getVirtualCoresSize();
|
||||||
|
|
||||||
/**
|
/**
|
||||||
* Set <em>number of virtual cpu cores</em> of the resource.
|
* Set <em>number of virtual cpu cores</em> of the resource.
|
||||||
|
@ -103,13 +120,14 @@ public abstract class Resource implements Comparable<Resource> {
|
||||||
*/
|
*/
|
||||||
@Public
|
@Public
|
||||||
@Evolving
|
@Evolving
|
||||||
public abstract void setVirtualCores(int vCores);
|
public abstract void setVirtualCores(long vCores);
|
||||||
|
|
||||||
@Override
|
@Override
|
||||||
public int hashCode() {
|
public int hashCode() {
|
||||||
final int prime = 263167;
|
final int prime = 263167;
|
||||||
int result = 3571;
|
|
||||||
result = 939769357 + getMemory(); // prime * result = 939769357 initially
|
int result = (int) (939769357
|
||||||
|
+ getMemorySize()); // prime * result = 939769357 initially
|
||||||
result = prime * result + getVirtualCores();
|
result = prime * result + getVirtualCores();
|
||||||
return result;
|
return result;
|
||||||
}
|
}
|
||||||
|
@ -123,7 +141,7 @@ public abstract class Resource implements Comparable<Resource> {
|
||||||
if (!(obj instanceof Resource))
|
if (!(obj instanceof Resource))
|
||||||
return false;
|
return false;
|
||||||
Resource other = (Resource) obj;
|
Resource other = (Resource) obj;
|
||||||
if (getMemory() != other.getMemory() ||
|
if (getMemorySize() != other.getMemorySize() ||
|
||||||
getVirtualCores() != other.getVirtualCores()) {
|
getVirtualCores() != other.getVirtualCores()) {
|
||||||
return false;
|
return false;
|
||||||
}
|
}
|
||||||
|
@ -132,6 +150,6 @@ public abstract class Resource implements Comparable<Resource> {
|
||||||
|
|
||||||
@Override
|
@Override
|
||||||
public String toString() {
|
public String toString() {
|
||||||
return "<memory:" + getMemory() + ", vCores:" + getVirtualCores() + ">";
|
return "<memory:" + getMemorySize() + ", vCores:" + getVirtualCores() + ">";
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
|
|
|
@ -54,8 +54,8 @@ message ContainerIdProto {
|
||||||
}
|
}
|
||||||
|
|
||||||
message ResourceProto {
|
message ResourceProto {
|
||||||
optional int32 memory = 1;
|
optional int64 memory = 1;
|
||||||
optional int32 virtual_cores = 2;
|
optional int64 virtual_cores = 2;
|
||||||
}
|
}
|
||||||
|
|
||||||
message ResourceUtilizationProto {
|
message ResourceUtilizationProto {
|
||||||
|
|
|
@ -221,7 +221,7 @@ public class ApplicationMaster {
|
||||||
@VisibleForTesting
|
@VisibleForTesting
|
||||||
protected int numTotalContainers = 1;
|
protected int numTotalContainers = 1;
|
||||||
// Memory to request for the container on which the shell command will run
|
// Memory to request for the container on which the shell command will run
|
||||||
private int containerMemory = 10;
|
private long containerMemory = 10;
|
||||||
// VirtualCores to request for the container on which the shell command will run
|
// VirtualCores to request for the container on which the shell command will run
|
||||||
private int containerVirtualCores = 1;
|
private int containerVirtualCores = 1;
|
||||||
// Priority of the request
|
// Priority of the request
|
||||||
|
@ -593,7 +593,7 @@ public class ApplicationMaster {
|
||||||
appMasterTrackingUrl);
|
appMasterTrackingUrl);
|
||||||
// Dump out information about cluster capability as seen by the
|
// Dump out information about cluster capability as seen by the
|
||||||
// resource manager
|
// resource manager
|
||||||
int maxMem = response.getMaximumResourceCapability().getMemory();
|
long maxMem = response.getMaximumResourceCapability().getMemorySize();
|
||||||
LOG.info("Max mem capability of resources in this cluster " + maxMem);
|
LOG.info("Max mem capability of resources in this cluster " + maxMem);
|
||||||
|
|
||||||
int maxVCores = response.getMaximumResourceCapability().getVirtualCores();
|
int maxVCores = response.getMaximumResourceCapability().getVirtualCores();
|
||||||
|
@ -823,7 +823,7 @@ public class ApplicationMaster {
|
||||||
+ ":" + allocatedContainer.getNodeId().getPort()
|
+ ":" + allocatedContainer.getNodeId().getPort()
|
||||||
+ ", containerNodeURI=" + allocatedContainer.getNodeHttpAddress()
|
+ ", containerNodeURI=" + allocatedContainer.getNodeHttpAddress()
|
||||||
+ ", containerResourceMemory"
|
+ ", containerResourceMemory"
|
||||||
+ allocatedContainer.getResource().getMemory()
|
+ allocatedContainer.getResource().getMemorySize()
|
||||||
+ ", containerResourceVirtualCores"
|
+ ", containerResourceVirtualCores"
|
||||||
+ allocatedContainer.getResource().getVirtualCores());
|
+ allocatedContainer.getResource().getVirtualCores());
|
||||||
// + ", containerToken"
|
// + ", containerToken"
|
||||||
|
|
|
@ -126,7 +126,7 @@ public class Client {
|
||||||
// Queue for App master
|
// Queue for App master
|
||||||
private String amQueue = "";
|
private String amQueue = "";
|
||||||
// Amt. of memory resource to request for to run the App Master
|
// Amt. of memory resource to request for to run the App Master
|
||||||
private int amMemory = 10;
|
private long amMemory = 10;
|
||||||
// Amt. of virtual core resource to request for to run the App Master
|
// Amt. of virtual core resource to request for to run the App Master
|
||||||
private int amVCores = 1;
|
private int amVCores = 1;
|
||||||
|
|
||||||
|
@ -487,7 +487,7 @@ public class Client {
|
||||||
// the required resources from the RM for the app master
|
// the required resources from the RM for the app master
|
||||||
// Memory ask has to be a multiple of min and less than max.
|
// Memory ask has to be a multiple of min and less than max.
|
||||||
// Dump out information about cluster capability as seen by the resource manager
|
// Dump out information about cluster capability as seen by the resource manager
|
||||||
int maxMem = appResponse.getMaximumResourceCapability().getMemory();
|
long maxMem = appResponse.getMaximumResourceCapability().getMemorySize();
|
||||||
LOG.info("Max mem capability of resources in this cluster " + maxMem);
|
LOG.info("Max mem capability of resources in this cluster " + maxMem);
|
||||||
|
|
||||||
// A resource ask cannot exceed the max.
|
// A resource ask cannot exceed the max.
|
||||||
|
|
|
@ -122,10 +122,10 @@ public class AMRMClientImpl<T extends ContainerRequest> extends AMRMClient<T> {
|
||||||
class ResourceReverseMemoryThenCpuComparator implements Comparator<Resource> {
|
class ResourceReverseMemoryThenCpuComparator implements Comparator<Resource> {
|
||||||
@Override
|
@Override
|
||||||
public int compare(Resource arg0, Resource arg1) {
|
public int compare(Resource arg0, Resource arg1) {
|
||||||
int mem0 = arg0.getMemory();
|
long mem0 = arg0.getMemorySize();
|
||||||
int mem1 = arg1.getMemory();
|
long mem1 = arg1.getMemorySize();
|
||||||
int cpu0 = arg0.getVirtualCores();
|
long cpu0 = arg0.getVirtualCores();
|
||||||
int cpu1 = arg1.getVirtualCores();
|
long cpu1 = arg1.getVirtualCores();
|
||||||
if(mem0 == mem1) {
|
if(mem0 == mem1) {
|
||||||
if(cpu0 == cpu1) {
|
if(cpu0 == cpu1) {
|
||||||
return 0;
|
return 0;
|
||||||
|
@ -143,10 +143,10 @@ public class AMRMClientImpl<T extends ContainerRequest> extends AMRMClient<T> {
|
||||||
}
|
}
|
||||||
|
|
||||||
static boolean canFit(Resource arg0, Resource arg1) {
|
static boolean canFit(Resource arg0, Resource arg1) {
|
||||||
int mem0 = arg0.getMemory();
|
long mem0 = arg0.getMemorySize();
|
||||||
int mem1 = arg1.getMemory();
|
long mem1 = arg1.getMemorySize();
|
||||||
int cpu0 = arg0.getVirtualCores();
|
long cpu0 = arg0.getVirtualCores();
|
||||||
int cpu1 = arg1.getVirtualCores();
|
long cpu1 = arg1.getVirtualCores();
|
||||||
|
|
||||||
return (mem0 <= mem1 && cpu0 <= cpu1);
|
return (mem0 <= mem1 && cpu0 <= cpu1);
|
||||||
}
|
}
|
||||||
|
|
|
@ -275,9 +275,9 @@ public class NodeCLI extends YarnCLI {
|
||||||
nodeReportStr.println(nodeReport.getNumContainers());
|
nodeReportStr.println(nodeReport.getNumContainers());
|
||||||
nodeReportStr.print("\tMemory-Used : ");
|
nodeReportStr.print("\tMemory-Used : ");
|
||||||
nodeReportStr.println((nodeReport.getUsed() == null) ? "0MB"
|
nodeReportStr.println((nodeReport.getUsed() == null) ? "0MB"
|
||||||
: (nodeReport.getUsed().getMemory() + "MB"));
|
: (nodeReport.getUsed().getMemorySize() + "MB"));
|
||||||
nodeReportStr.print("\tMemory-Capacity : ");
|
nodeReportStr.print("\tMemory-Capacity : ");
|
||||||
nodeReportStr.println(nodeReport.getCapability().getMemory() + "MB");
|
nodeReportStr.println(nodeReport.getCapability().getMemorySize() + "MB");
|
||||||
nodeReportStr.print("\tCPU-Used : ");
|
nodeReportStr.print("\tCPU-Used : ");
|
||||||
nodeReportStr.println((nodeReport.getUsed() == null) ? "0 vcores"
|
nodeReportStr.println((nodeReport.getUsed() == null) ? "0 vcores"
|
||||||
: (nodeReport.getUsed().getVirtualCores() + " vcores"));
|
: (nodeReport.getUsed().getVirtualCores() + " vcores"));
|
||||||
|
|
|
@ -155,7 +155,7 @@ public class TopCLI extends YarnCLI {
|
||||||
displayStringsMap.put(Columns.VCORES, String.valueOf(usedVirtualCores));
|
displayStringsMap.put(Columns.VCORES, String.valueOf(usedVirtualCores));
|
||||||
usedMemory =
|
usedMemory =
|
||||||
appReport.getApplicationResourceUsageReport().getUsedResources()
|
appReport.getApplicationResourceUsageReport().getUsedResources()
|
||||||
.getMemory() / 1024;
|
.getMemorySize() / 1024;
|
||||||
displayStringsMap.put(Columns.MEM, String.valueOf(usedMemory) + "G");
|
displayStringsMap.put(Columns.MEM, String.valueOf(usedMemory) + "G");
|
||||||
reservedVirtualCores =
|
reservedVirtualCores =
|
||||||
appReport.getApplicationResourceUsageReport().getReservedResources()
|
appReport.getApplicationResourceUsageReport().getReservedResources()
|
||||||
|
@ -164,7 +164,7 @@ public class TopCLI extends YarnCLI {
|
||||||
String.valueOf(reservedVirtualCores));
|
String.valueOf(reservedVirtualCores));
|
||||||
reservedMemory =
|
reservedMemory =
|
||||||
appReport.getApplicationResourceUsageReport().getReservedResources()
|
appReport.getApplicationResourceUsageReport().getReservedResources()
|
||||||
.getMemory() / 1024;
|
.getMemorySize() / 1024;
|
||||||
displayStringsMap.put(Columns.RMEM, String.valueOf(reservedMemory) + "G");
|
displayStringsMap.put(Columns.RMEM, String.valueOf(reservedMemory) + "G");
|
||||||
attempts = appReport.getCurrentApplicationAttemptId().getAttemptId();
|
attempts = appReport.getCurrentApplicationAttemptId().getAttemptId();
|
||||||
nodes = 0;
|
nodes = 0;
|
||||||
|
|
|
@ -1206,7 +1206,7 @@ public class TestYarnClient {
|
||||||
for (attempts = 10; attempts > 0; attempts--) {
|
for (attempts = 10; attempts > 0; attempts--) {
|
||||||
if (cluster.getResourceManager().getRMContext().getReservationSystem()
|
if (cluster.getResourceManager().getRMContext().getReservationSystem()
|
||||||
.getPlan(ReservationSystemTestUtil.reservationQ).getTotalCapacity()
|
.getPlan(ReservationSystemTestUtil.reservationQ).getTotalCapacity()
|
||||||
.getMemory() > 6000) {
|
.getMemorySize() > 6000) {
|
||||||
break;
|
break;
|
||||||
}
|
}
|
||||||
try {
|
try {
|
||||||
|
|
|
@ -53,39 +53,49 @@ public class ResourcePBImpl extends Resource {
|
||||||
}
|
}
|
||||||
viaProto = false;
|
viaProto = false;
|
||||||
}
|
}
|
||||||
|
|
||||||
|
|
||||||
@Override
|
@Override
|
||||||
|
@SuppressWarnings("deprecation")
|
||||||
public int getMemory() {
|
public int getMemory() {
|
||||||
ResourceProtoOrBuilder p = viaProto ? proto : builder;
|
return (int) getMemorySize();
|
||||||
return (p.getMemory());
|
|
||||||
}
|
}
|
||||||
|
|
||||||
@Override
|
@Override
|
||||||
public void setMemory(int memory) {
|
public long getMemorySize() {
|
||||||
|
ResourceProtoOrBuilder p = viaProto ? proto : builder;
|
||||||
|
return p.getMemory();
|
||||||
|
}
|
||||||
|
|
||||||
|
@Override
|
||||||
|
public void setMemory(long memory) {
|
||||||
maybeInitBuilder();
|
maybeInitBuilder();
|
||||||
builder.setMemory((memory));
|
builder.setMemory(memory);
|
||||||
}
|
}
|
||||||
|
|
||||||
@Override
|
@Override
|
||||||
public int getVirtualCores() {
|
public int getVirtualCores() {
|
||||||
ResourceProtoOrBuilder p = viaProto ? proto : builder;
|
return (int) getVirtualCoresSize();
|
||||||
return (p.getVirtualCores());
|
|
||||||
}
|
}
|
||||||
|
|
||||||
@Override
|
@Override
|
||||||
public void setVirtualCores(int vCores) {
|
public long getVirtualCoresSize() {
|
||||||
|
ResourceProtoOrBuilder p = viaProto ? proto : builder;
|
||||||
|
return p.getVirtualCores();
|
||||||
|
}
|
||||||
|
|
||||||
|
@Override
|
||||||
|
public void setVirtualCores(long vCores) {
|
||||||
maybeInitBuilder();
|
maybeInitBuilder();
|
||||||
builder.setVirtualCores((vCores));
|
builder.setVirtualCores(vCores);
|
||||||
}
|
}
|
||||||
|
|
||||||
@Override
|
@Override
|
||||||
public int compareTo(Resource other) {
|
public int compareTo(Resource other) {
|
||||||
int diff = this.getMemory() - other.getMemory();
|
long diff = this.getMemorySize() - other.getMemorySize();
|
||||||
if (diff == 0) {
|
if (diff == 0) {
|
||||||
diff = this.getVirtualCores() - other.getVirtualCores();
|
diff = this.getVirtualCores() - other.getVirtualCores();
|
||||||
}
|
}
|
||||||
return diff;
|
return diff == 0 ? 0 : (diff > 0 ? 1 : -1);
|
||||||
}
|
}
|
||||||
|
|
||||||
|
|
||||||
|
|
|
@ -28,13 +28,13 @@ public class DefaultResourceCalculator extends ResourceCalculator {
|
||||||
@Override
|
@Override
|
||||||
public int compare(Resource unused, Resource lhs, Resource rhs) {
|
public int compare(Resource unused, Resource lhs, Resource rhs) {
|
||||||
// Only consider memory
|
// Only consider memory
|
||||||
return lhs.getMemory() - rhs.getMemory();
|
return Long.compare(lhs.getMemorySize(), rhs.getMemorySize());
|
||||||
}
|
}
|
||||||
|
|
||||||
@Override
|
@Override
|
||||||
public int computeAvailableContainers(Resource available, Resource required) {
|
public long computeAvailableContainers(Resource available, Resource required) {
|
||||||
// Only consider memory
|
// Only consider memory
|
||||||
return available.getMemory() / required.getMemory();
|
return available.getMemorySize() / required.getMemorySize();
|
||||||
}
|
}
|
||||||
|
|
||||||
@Override
|
@Override
|
||||||
|
@ -44,7 +44,7 @@ public class DefaultResourceCalculator extends ResourceCalculator {
|
||||||
}
|
}
|
||||||
|
|
||||||
public boolean isInvalidDivisor(Resource r) {
|
public boolean isInvalidDivisor(Resource r) {
|
||||||
if (r.getMemory() == 0.0f) {
|
if (r.getMemorySize() == 0.0f) {
|
||||||
return true;
|
return true;
|
||||||
}
|
}
|
||||||
return false;
|
return false;
|
||||||
|
@ -52,23 +52,23 @@ public class DefaultResourceCalculator extends ResourceCalculator {
|
||||||
|
|
||||||
@Override
|
@Override
|
||||||
public float ratio(Resource a, Resource b) {
|
public float ratio(Resource a, Resource b) {
|
||||||
return (float)a.getMemory() / b.getMemory();
|
return (float)a.getMemorySize() / b.getMemorySize();
|
||||||
}
|
}
|
||||||
|
|
||||||
@Override
|
@Override
|
||||||
public Resource divideAndCeil(Resource numerator, int denominator) {
|
public Resource divideAndCeil(Resource numerator, long denominator) {
|
||||||
return Resources.createResource(
|
return Resources.createResource(
|
||||||
divideAndCeil(numerator.getMemory(), denominator));
|
divideAndCeil(numerator.getMemorySize(), denominator));
|
||||||
}
|
}
|
||||||
|
|
||||||
@Override
|
@Override
|
||||||
public Resource normalize(Resource r, Resource minimumResource,
|
public Resource normalize(Resource r, Resource minimumResource,
|
||||||
Resource maximumResource, Resource stepFactor) {
|
Resource maximumResource, Resource stepFactor) {
|
||||||
int normalizedMemory = Math.min(
|
long normalizedMemory = Math.min(
|
||||||
roundUp(
|
roundUp(
|
||||||
Math.max(r.getMemory(), minimumResource.getMemory()),
|
Math.max(r.getMemorySize(), minimumResource.getMemorySize()),
|
||||||
stepFactor.getMemory()),
|
stepFactor.getMemorySize()),
|
||||||
maximumResource.getMemory());
|
maximumResource.getMemorySize());
|
||||||
return Resources.createResource(normalizedMemory);
|
return Resources.createResource(normalizedMemory);
|
||||||
}
|
}
|
||||||
|
|
||||||
|
@ -81,22 +81,22 @@ public class DefaultResourceCalculator extends ResourceCalculator {
|
||||||
@Override
|
@Override
|
||||||
public Resource roundUp(Resource r, Resource stepFactor) {
|
public Resource roundUp(Resource r, Resource stepFactor) {
|
||||||
return Resources.createResource(
|
return Resources.createResource(
|
||||||
roundUp(r.getMemory(), stepFactor.getMemory())
|
roundUp(r.getMemorySize(), stepFactor.getMemorySize())
|
||||||
);
|
);
|
||||||
}
|
}
|
||||||
|
|
||||||
@Override
|
@Override
|
||||||
public Resource roundDown(Resource r, Resource stepFactor) {
|
public Resource roundDown(Resource r, Resource stepFactor) {
|
||||||
return Resources.createResource(
|
return Resources.createResource(
|
||||||
roundDown(r.getMemory(), stepFactor.getMemory()));
|
roundDown(r.getMemorySize(), stepFactor.getMemorySize()));
|
||||||
}
|
}
|
||||||
|
|
||||||
@Override
|
@Override
|
||||||
public Resource multiplyAndNormalizeUp(Resource r, double by,
|
public Resource multiplyAndNormalizeUp(Resource r, double by,
|
||||||
Resource stepFactor) {
|
Resource stepFactor) {
|
||||||
return Resources.createResource(
|
return Resources.createResource(
|
||||||
roundUp((int)(r.getMemory() * by + 0.5), stepFactor.getMemory())
|
roundUp((long) (r.getMemorySize() * by + 0.5),
|
||||||
);
|
stepFactor.getMemorySize()));
|
||||||
}
|
}
|
||||||
|
|
||||||
@Override
|
@Override
|
||||||
|
@ -104,8 +104,8 @@ public class DefaultResourceCalculator extends ResourceCalculator {
|
||||||
Resource stepFactor) {
|
Resource stepFactor) {
|
||||||
return Resources.createResource(
|
return Resources.createResource(
|
||||||
roundDown(
|
roundDown(
|
||||||
(int)(r.getMemory() * by),
|
(long)(r.getMemorySize() * by),
|
||||||
stepFactor.getMemory()
|
stepFactor.getMemorySize()
|
||||||
)
|
)
|
||||||
);
|
);
|
||||||
}
|
}
|
||||||
|
@ -113,6 +113,6 @@ public class DefaultResourceCalculator extends ResourceCalculator {
|
||||||
@Override
|
@Override
|
||||||
public boolean fitsIn(Resource cluster,
|
public boolean fitsIn(Resource cluster,
|
||||||
Resource smaller, Resource bigger) {
|
Resource smaller, Resource bigger) {
|
||||||
return smaller.getMemory() <= bigger.getMemory();
|
return smaller.getMemorySize() <= bigger.getMemorySize();
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
|
|
|
@ -54,15 +54,15 @@ public class DominantResourceCalculator extends ResourceCalculator {
|
||||||
}
|
}
|
||||||
|
|
||||||
if (isInvalidDivisor(clusterResource)) {
|
if (isInvalidDivisor(clusterResource)) {
|
||||||
if ((lhs.getMemory() < rhs.getMemory() && lhs.getVirtualCores() > rhs
|
if ((lhs.getMemorySize() < rhs.getMemorySize() && lhs.getVirtualCores() > rhs
|
||||||
.getVirtualCores())
|
.getVirtualCores())
|
||||||
|| (lhs.getMemory() > rhs.getMemory() && lhs.getVirtualCores() < rhs
|
|| (lhs.getMemorySize() > rhs.getMemorySize() && lhs.getVirtualCores() < rhs
|
||||||
.getVirtualCores())) {
|
.getVirtualCores())) {
|
||||||
return 0;
|
return 0;
|
||||||
} else if (lhs.getMemory() > rhs.getMemory()
|
} else if (lhs.getMemorySize() > rhs.getMemorySize()
|
||||||
|| lhs.getVirtualCores() > rhs.getVirtualCores()) {
|
|| lhs.getVirtualCores() > rhs.getVirtualCores()) {
|
||||||
return 1;
|
return 1;
|
||||||
} else if (lhs.getMemory() < rhs.getMemory()
|
} else if (lhs.getMemorySize() < rhs.getMemorySize()
|
||||||
|| lhs.getVirtualCores() < rhs.getVirtualCores()) {
|
|| lhs.getVirtualCores() < rhs.getVirtualCores()) {
|
||||||
return -1;
|
return -1;
|
||||||
}
|
}
|
||||||
|
@ -100,20 +100,20 @@ public class DominantResourceCalculator extends ResourceCalculator {
|
||||||
// Just use 'dominant' resource
|
// Just use 'dominant' resource
|
||||||
return (dominant) ?
|
return (dominant) ?
|
||||||
Math.max(
|
Math.max(
|
||||||
(float)resource.getMemory() / clusterResource.getMemory(),
|
(float)resource.getMemorySize() / clusterResource.getMemorySize(),
|
||||||
(float)resource.getVirtualCores() / clusterResource.getVirtualCores()
|
(float)resource.getVirtualCores() / clusterResource.getVirtualCores()
|
||||||
)
|
)
|
||||||
:
|
:
|
||||||
Math.min(
|
Math.min(
|
||||||
(float)resource.getMemory() / clusterResource.getMemory(),
|
(float)resource.getMemorySize() / clusterResource.getMemorySize(),
|
||||||
(float)resource.getVirtualCores() / clusterResource.getVirtualCores()
|
(float)resource.getVirtualCores() / clusterResource.getVirtualCores()
|
||||||
);
|
);
|
||||||
}
|
}
|
||||||
|
|
||||||
@Override
|
@Override
|
||||||
public int computeAvailableContainers(Resource available, Resource required) {
|
public long computeAvailableContainers(Resource available, Resource required) {
|
||||||
return Math.min(
|
return Math.min(
|
||||||
available.getMemory() / required.getMemory(),
|
available.getMemorySize() / required.getMemorySize(),
|
||||||
available.getVirtualCores() / required.getVirtualCores());
|
available.getVirtualCores() / required.getVirtualCores());
|
||||||
}
|
}
|
||||||
|
|
||||||
|
@ -127,7 +127,7 @@ public class DominantResourceCalculator extends ResourceCalculator {
|
||||||
|
|
||||||
@Override
|
@Override
|
||||||
public boolean isInvalidDivisor(Resource r) {
|
public boolean isInvalidDivisor(Resource r) {
|
||||||
if (r.getMemory() == 0.0f || r.getVirtualCores() == 0.0f) {
|
if (r.getMemorySize() == 0.0f || r.getVirtualCores() == 0.0f) {
|
||||||
return true;
|
return true;
|
||||||
}
|
}
|
||||||
return false;
|
return false;
|
||||||
|
@ -136,15 +136,15 @@ public class DominantResourceCalculator extends ResourceCalculator {
|
||||||
@Override
|
@Override
|
||||||
public float ratio(Resource a, Resource b) {
|
public float ratio(Resource a, Resource b) {
|
||||||
return Math.max(
|
return Math.max(
|
||||||
(float)a.getMemory()/b.getMemory(),
|
(float)a.getMemorySize()/b.getMemorySize(),
|
||||||
(float)a.getVirtualCores()/b.getVirtualCores()
|
(float)a.getVirtualCores()/b.getVirtualCores()
|
||||||
);
|
);
|
||||||
}
|
}
|
||||||
|
|
||||||
@Override
|
@Override
|
||||||
public Resource divideAndCeil(Resource numerator, int denominator) {
|
public Resource divideAndCeil(Resource numerator, long denominator) {
|
||||||
return Resources.createResource(
|
return Resources.createResource(
|
||||||
divideAndCeil(numerator.getMemory(), denominator),
|
divideAndCeil(numerator.getMemorySize(), denominator),
|
||||||
divideAndCeil(numerator.getVirtualCores(), denominator)
|
divideAndCeil(numerator.getVirtualCores(), denominator)
|
||||||
);
|
);
|
||||||
}
|
}
|
||||||
|
@ -152,12 +152,12 @@ public class DominantResourceCalculator extends ResourceCalculator {
|
||||||
@Override
|
@Override
|
||||||
public Resource normalize(Resource r, Resource minimumResource,
|
public Resource normalize(Resource r, Resource minimumResource,
|
||||||
Resource maximumResource, Resource stepFactor) {
|
Resource maximumResource, Resource stepFactor) {
|
||||||
int normalizedMemory = Math.min(
|
long normalizedMemory = Math.min(
|
||||||
roundUp(
|
roundUp(
|
||||||
Math.max(r.getMemory(), minimumResource.getMemory()),
|
Math.max(r.getMemorySize(), minimumResource.getMemorySize()),
|
||||||
stepFactor.getMemory()),
|
stepFactor.getMemorySize()),
|
||||||
maximumResource.getMemory());
|
maximumResource.getMemorySize());
|
||||||
int normalizedCores = Math.min(
|
long normalizedCores = Math.min(
|
||||||
roundUp(
|
roundUp(
|
||||||
Math.max(r.getVirtualCores(), minimumResource.getVirtualCores()),
|
Math.max(r.getVirtualCores(), minimumResource.getVirtualCores()),
|
||||||
stepFactor.getVirtualCores()),
|
stepFactor.getVirtualCores()),
|
||||||
|
@ -169,7 +169,7 @@ public class DominantResourceCalculator extends ResourceCalculator {
|
||||||
@Override
|
@Override
|
||||||
public Resource roundUp(Resource r, Resource stepFactor) {
|
public Resource roundUp(Resource r, Resource stepFactor) {
|
||||||
return Resources.createResource(
|
return Resources.createResource(
|
||||||
roundUp(r.getMemory(), stepFactor.getMemory()),
|
roundUp(r.getMemorySize(), stepFactor.getMemorySize()),
|
||||||
roundUp(r.getVirtualCores(), stepFactor.getVirtualCores())
|
roundUp(r.getVirtualCores(), stepFactor.getVirtualCores())
|
||||||
);
|
);
|
||||||
}
|
}
|
||||||
|
@ -177,7 +177,7 @@ public class DominantResourceCalculator extends ResourceCalculator {
|
||||||
@Override
|
@Override
|
||||||
public Resource roundDown(Resource r, Resource stepFactor) {
|
public Resource roundDown(Resource r, Resource stepFactor) {
|
||||||
return Resources.createResource(
|
return Resources.createResource(
|
||||||
roundDown(r.getMemory(), stepFactor.getMemory()),
|
roundDown(r.getMemorySize(), stepFactor.getMemorySize()),
|
||||||
roundDown(r.getVirtualCores(), stepFactor.getVirtualCores())
|
roundDown(r.getVirtualCores(), stepFactor.getVirtualCores())
|
||||||
);
|
);
|
||||||
}
|
}
|
||||||
|
@ -187,7 +187,7 @@ public class DominantResourceCalculator extends ResourceCalculator {
|
||||||
Resource stepFactor) {
|
Resource stepFactor) {
|
||||||
return Resources.createResource(
|
return Resources.createResource(
|
||||||
roundUp(
|
roundUp(
|
||||||
(int)Math.ceil(r.getMemory() * by), stepFactor.getMemory()),
|
(int)Math.ceil(r.getMemorySize() * by), stepFactor.getMemorySize()),
|
||||||
roundUp(
|
roundUp(
|
||||||
(int)Math.ceil(r.getVirtualCores() * by),
|
(int)Math.ceil(r.getVirtualCores() * by),
|
||||||
stepFactor.getVirtualCores())
|
stepFactor.getVirtualCores())
|
||||||
|
@ -199,8 +199,8 @@ public class DominantResourceCalculator extends ResourceCalculator {
|
||||||
Resource stepFactor) {
|
Resource stepFactor) {
|
||||||
return Resources.createResource(
|
return Resources.createResource(
|
||||||
roundDown(
|
roundDown(
|
||||||
(int)(r.getMemory() * by),
|
(int)(r.getMemorySize() * by),
|
||||||
stepFactor.getMemory()
|
stepFactor.getMemorySize()
|
||||||
),
|
),
|
||||||
roundDown(
|
roundDown(
|
||||||
(int)(r.getVirtualCores() * by),
|
(int)(r.getVirtualCores() * by),
|
||||||
|
@ -212,7 +212,7 @@ public class DominantResourceCalculator extends ResourceCalculator {
|
||||||
@Override
|
@Override
|
||||||
public boolean fitsIn(Resource cluster,
|
public boolean fitsIn(Resource cluster,
|
||||||
Resource smaller, Resource bigger) {
|
Resource smaller, Resource bigger) {
|
||||||
return smaller.getMemory() <= bigger.getMemory()
|
return smaller.getMemorySize() <= bigger.getMemorySize()
|
||||||
&& smaller.getVirtualCores() <= bigger.getVirtualCores();
|
&& smaller.getVirtualCores() <= bigger.getVirtualCores();
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
|
|
|
@ -31,18 +31,18 @@ public abstract class ResourceCalculator {
|
||||||
public abstract int
|
public abstract int
|
||||||
compare(Resource clusterResource, Resource lhs, Resource rhs);
|
compare(Resource clusterResource, Resource lhs, Resource rhs);
|
||||||
|
|
||||||
public static int divideAndCeil(int a, int b) {
|
public static long divideAndCeil(long a, long b) {
|
||||||
if (b == 0) {
|
if (b == 0) {
|
||||||
return 0;
|
return 0;
|
||||||
}
|
}
|
||||||
return (a + (b - 1)) / b;
|
return (a + (b - 1)) / b;
|
||||||
}
|
}
|
||||||
|
|
||||||
public static int roundUp(int a, int b) {
|
public static long roundUp(long a, long b) {
|
||||||
return divideAndCeil(a, b) * b;
|
return divideAndCeil(a, b) * b;
|
||||||
}
|
}
|
||||||
|
|
||||||
public static int roundDown(int a, int b) {
|
public static long roundDown(long a, long b) {
|
||||||
return (a / b) * b;
|
return (a / b) * b;
|
||||||
}
|
}
|
||||||
|
|
||||||
|
@ -54,7 +54,7 @@ public abstract class ResourceCalculator {
|
||||||
* @param required required resources
|
* @param required required resources
|
||||||
* @return number of containers which can be allocated
|
* @return number of containers which can be allocated
|
||||||
*/
|
*/
|
||||||
public abstract int computeAvailableContainers(
|
public abstract long computeAvailableContainers(
|
||||||
Resource available, Resource required);
|
Resource available, Resource required);
|
||||||
|
|
||||||
/**
|
/**
|
||||||
|
@ -169,7 +169,7 @@ public abstract class ResourceCalculator {
|
||||||
* @param denominator denominator
|
* @param denominator denominator
|
||||||
* @return resultant resource
|
* @return resultant resource
|
||||||
*/
|
*/
|
||||||
public abstract Resource divideAndCeil(Resource numerator, int denominator);
|
public abstract Resource divideAndCeil(Resource numerator, long denominator);
|
||||||
|
|
||||||
/**
|
/**
|
||||||
* Check if a smaller resource can be contained by bigger resource.
|
* Check if a smaller resource can be contained by bigger resource.
|
||||||
|
|
|
@ -31,12 +31,18 @@ public class Resources {
|
||||||
private static final Resource NONE = new Resource() {
|
private static final Resource NONE = new Resource() {
|
||||||
|
|
||||||
@Override
|
@Override
|
||||||
|
@SuppressWarnings("deprecation")
|
||||||
public int getMemory() {
|
public int getMemory() {
|
||||||
return 0;
|
return 0;
|
||||||
}
|
}
|
||||||
|
|
||||||
@Override
|
@Override
|
||||||
public void setMemory(int memory) {
|
public long getMemorySize() {
|
||||||
|
return 0;
|
||||||
|
}
|
||||||
|
|
||||||
|
@Override
|
||||||
|
public void setMemory(long memory) {
|
||||||
throw new RuntimeException("NONE cannot be modified!");
|
throw new RuntimeException("NONE cannot be modified!");
|
||||||
}
|
}
|
||||||
|
|
||||||
|
@ -46,17 +52,22 @@ public class Resources {
|
||||||
}
|
}
|
||||||
|
|
||||||
@Override
|
@Override
|
||||||
public void setVirtualCores(int cores) {
|
public long getVirtualCoresSize() {
|
||||||
|
return 0;
|
||||||
|
}
|
||||||
|
|
||||||
|
@Override
|
||||||
|
public void setVirtualCores(long cores) {
|
||||||
throw new RuntimeException("NONE cannot be modified!");
|
throw new RuntimeException("NONE cannot be modified!");
|
||||||
}
|
}
|
||||||
|
|
||||||
@Override
|
@Override
|
||||||
public int compareTo(Resource o) {
|
public int compareTo(Resource o) {
|
||||||
int diff = 0 - o.getMemory();
|
long diff = 0 - o.getMemorySize();
|
||||||
if (diff == 0) {
|
if (diff == 0) {
|
||||||
diff = 0 - o.getVirtualCores();
|
diff = 0 - o.getVirtualCores();
|
||||||
}
|
}
|
||||||
return diff;
|
return Long.signum(diff);
|
||||||
}
|
}
|
||||||
|
|
||||||
};
|
};
|
||||||
|
@ -64,12 +75,18 @@ public class Resources {
|
||||||
private static final Resource UNBOUNDED = new Resource() {
|
private static final Resource UNBOUNDED = new Resource() {
|
||||||
|
|
||||||
@Override
|
@Override
|
||||||
|
@SuppressWarnings("deprecation")
|
||||||
public int getMemory() {
|
public int getMemory() {
|
||||||
return Integer.MAX_VALUE;
|
return Integer.MAX_VALUE;
|
||||||
}
|
}
|
||||||
|
|
||||||
@Override
|
@Override
|
||||||
public void setMemory(int memory) {
|
public long getMemorySize() {
|
||||||
|
return Long.MAX_VALUE;
|
||||||
|
}
|
||||||
|
|
||||||
|
@Override
|
||||||
|
public void setMemory(long memory) {
|
||||||
throw new RuntimeException("UNBOUNDED cannot be modified!");
|
throw new RuntimeException("UNBOUNDED cannot be modified!");
|
||||||
}
|
}
|
||||||
|
|
||||||
|
@ -79,26 +96,31 @@ public class Resources {
|
||||||
}
|
}
|
||||||
|
|
||||||
@Override
|
@Override
|
||||||
public void setVirtualCores(int cores) {
|
public long getVirtualCoresSize() {
|
||||||
|
return Long.MAX_VALUE;
|
||||||
|
}
|
||||||
|
|
||||||
|
@Override
|
||||||
|
public void setVirtualCores(long cores) {
|
||||||
throw new RuntimeException("UNBOUNDED cannot be modified!");
|
throw new RuntimeException("UNBOUNDED cannot be modified!");
|
||||||
}
|
}
|
||||||
|
|
||||||
@Override
|
@Override
|
||||||
public int compareTo(Resource o) {
|
public int compareTo(Resource o) {
|
||||||
int diff = Integer.MAX_VALUE - o.getMemory();
|
long diff = Long.MAX_VALUE - o.getMemorySize();
|
||||||
if (diff == 0) {
|
if (diff == 0) {
|
||||||
diff = Integer.MAX_VALUE - o.getVirtualCores();
|
diff = Long.MAX_VALUE - o.getVirtualCoresSize();
|
||||||
}
|
}
|
||||||
return diff;
|
return Long.signum(diff);
|
||||||
}
|
}
|
||||||
|
|
||||||
};
|
};
|
||||||
|
|
||||||
public static Resource createResource(int memory) {
|
public static Resource createResource(long memory) {
|
||||||
return createResource(memory, (memory > 0) ? 1 : 0);
|
return createResource(memory, (memory > 0) ? 1 : 0);
|
||||||
}
|
}
|
||||||
|
|
||||||
public static Resource createResource(int memory, int cores) {
|
public static Resource createResource(long memory, long cores) {
|
||||||
Resource resource = Records.newRecord(Resource.class);
|
Resource resource = Records.newRecord(Resource.class);
|
||||||
resource.setMemory(memory);
|
resource.setMemory(memory);
|
||||||
resource.setVirtualCores(cores);
|
resource.setVirtualCores(cores);
|
||||||
|
@ -114,11 +136,11 @@ public class Resources {
|
||||||
}
|
}
|
||||||
|
|
||||||
public static Resource clone(Resource res) {
|
public static Resource clone(Resource res) {
|
||||||
return createResource(res.getMemory(), res.getVirtualCores());
|
return createResource(res.getMemorySize(), res.getVirtualCores());
|
||||||
}
|
}
|
||||||
|
|
||||||
public static Resource addTo(Resource lhs, Resource rhs) {
|
public static Resource addTo(Resource lhs, Resource rhs) {
|
||||||
lhs.setMemory(lhs.getMemory() + rhs.getMemory());
|
lhs.setMemory(lhs.getMemorySize() + rhs.getMemorySize());
|
||||||
lhs.setVirtualCores(lhs.getVirtualCores() + rhs.getVirtualCores());
|
lhs.setVirtualCores(lhs.getVirtualCores() + rhs.getVirtualCores());
|
||||||
return lhs;
|
return lhs;
|
||||||
}
|
}
|
||||||
|
@ -128,7 +150,7 @@ public class Resources {
|
||||||
}
|
}
|
||||||
|
|
||||||
public static Resource subtractFrom(Resource lhs, Resource rhs) {
|
public static Resource subtractFrom(Resource lhs, Resource rhs) {
|
||||||
lhs.setMemory(lhs.getMemory() - rhs.getMemory());
|
lhs.setMemory(lhs.getMemorySize() - rhs.getMemorySize());
|
||||||
lhs.setVirtualCores(lhs.getVirtualCores() - rhs.getVirtualCores());
|
lhs.setVirtualCores(lhs.getVirtualCores() - rhs.getVirtualCores());
|
||||||
return lhs;
|
return lhs;
|
||||||
}
|
}
|
||||||
|
@ -142,7 +164,7 @@ public class Resources {
|
||||||
}
|
}
|
||||||
|
|
||||||
public static Resource multiplyTo(Resource lhs, double by) {
|
public static Resource multiplyTo(Resource lhs, double by) {
|
||||||
lhs.setMemory((int)(lhs.getMemory() * by));
|
lhs.setMemory((int)(lhs.getMemorySize() * by));
|
||||||
lhs.setVirtualCores((int)(lhs.getVirtualCores() * by));
|
lhs.setVirtualCores((int)(lhs.getVirtualCores() * by));
|
||||||
return lhs;
|
return lhs;
|
||||||
}
|
}
|
||||||
|
@ -157,7 +179,7 @@ public class Resources {
|
||||||
*/
|
*/
|
||||||
public static Resource multiplyAndAddTo(
|
public static Resource multiplyAndAddTo(
|
||||||
Resource lhs, Resource rhs, double by) {
|
Resource lhs, Resource rhs, double by) {
|
||||||
lhs.setMemory(lhs.getMemory() + (int)(rhs.getMemory() * by));
|
lhs.setMemory(lhs.getMemorySize() + (int)(rhs.getMemorySize() * by));
|
||||||
lhs.setVirtualCores(lhs.getVirtualCores()
|
lhs.setVirtualCores(lhs.getVirtualCores()
|
||||||
+ (int)(rhs.getVirtualCores() * by));
|
+ (int)(rhs.getVirtualCores() * by));
|
||||||
return lhs;
|
return lhs;
|
||||||
|
@ -175,7 +197,7 @@ public class Resources {
|
||||||
|
|
||||||
public static Resource multiplyAndRoundDown(Resource lhs, double by) {
|
public static Resource multiplyAndRoundDown(Resource lhs, double by) {
|
||||||
Resource out = clone(lhs);
|
Resource out = clone(lhs);
|
||||||
out.setMemory((int)(lhs.getMemory() * by));
|
out.setMemory((int)(lhs.getMemorySize() * by));
|
||||||
out.setVirtualCores((int)(lhs.getVirtualCores() * by));
|
out.setVirtualCores((int)(lhs.getVirtualCores() * by));
|
||||||
return out;
|
return out;
|
||||||
}
|
}
|
||||||
|
@ -264,7 +286,7 @@ public class Resources {
|
||||||
}
|
}
|
||||||
|
|
||||||
public static boolean fitsIn(Resource smaller, Resource bigger) {
|
public static boolean fitsIn(Resource smaller, Resource bigger) {
|
||||||
return smaller.getMemory() <= bigger.getMemory() &&
|
return smaller.getMemorySize() <= bigger.getMemorySize() &&
|
||||||
smaller.getVirtualCores() <= bigger.getVirtualCores();
|
smaller.getVirtualCores() <= bigger.getVirtualCores();
|
||||||
}
|
}
|
||||||
|
|
||||||
|
@ -274,12 +296,12 @@ public class Resources {
|
||||||
}
|
}
|
||||||
|
|
||||||
public static Resource componentwiseMin(Resource lhs, Resource rhs) {
|
public static Resource componentwiseMin(Resource lhs, Resource rhs) {
|
||||||
return createResource(Math.min(lhs.getMemory(), rhs.getMemory()),
|
return createResource(Math.min(lhs.getMemorySize(), rhs.getMemorySize()),
|
||||||
Math.min(lhs.getVirtualCores(), rhs.getVirtualCores()));
|
Math.min(lhs.getVirtualCores(), rhs.getVirtualCores()));
|
||||||
}
|
}
|
||||||
|
|
||||||
public static Resource componentwiseMax(Resource lhs, Resource rhs) {
|
public static Resource componentwiseMax(Resource lhs, Resource rhs) {
|
||||||
return createResource(Math.max(lhs.getMemory(), rhs.getMemory()),
|
return createResource(Math.max(lhs.getMemorySize(), rhs.getMemorySize()),
|
||||||
Math.max(lhs.getVirtualCores(), rhs.getVirtualCores()));
|
Math.max(lhs.getVirtualCores(), rhs.getVirtualCores()));
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
|
|
|
@ -24,18 +24,18 @@ import static org.junit.Assert.assertTrue;
|
||||||
|
|
||||||
public class TestResources {
|
public class TestResources {
|
||||||
|
|
||||||
public Resource createResource(int memory, int vCores) {
|
public Resource createResource(long memory, long vCores) {
|
||||||
return Resource.newInstance(memory, vCores);
|
return Resource.newInstance(memory, vCores);
|
||||||
}
|
}
|
||||||
|
|
||||||
@Test(timeout=1000)
|
@Test(timeout=1000)
|
||||||
public void testCompareToWithUnboundedResource() {
|
public void testCompareToWithUnboundedResource() {
|
||||||
assertTrue(Resources.unbounded().compareTo(
|
assertTrue(Resources.unbounded().compareTo(
|
||||||
createResource(Integer.MAX_VALUE, Integer.MAX_VALUE)) == 0);
|
createResource(Long.MAX_VALUE, Long.MAX_VALUE)) == 0);
|
||||||
assertTrue(Resources.unbounded().compareTo(
|
assertTrue(Resources.unbounded().compareTo(
|
||||||
createResource(Integer.MAX_VALUE, 0)) > 0);
|
createResource(Long.MAX_VALUE, 0)) > 0);
|
||||||
assertTrue(Resources.unbounded().compareTo(
|
assertTrue(Resources.unbounded().compareTo(
|
||||||
createResource(0, Integer.MAX_VALUE)) > 0);
|
createResource(0, Long.MAX_VALUE)) > 0);
|
||||||
}
|
}
|
||||||
|
|
||||||
@Test(timeout=1000)
|
@Test(timeout=1000)
|
||||||
|
|
|
@ -425,7 +425,7 @@ public class BuilderUtils {
|
||||||
return report;
|
return report;
|
||||||
}
|
}
|
||||||
|
|
||||||
public static Resource newResource(int memory, int vCores) {
|
public static Resource newResource(long memory, long vCores) {
|
||||||
Resource resource = recordFactory.newRecordInstance(Resource.class);
|
Resource resource = recordFactory.newRecordInstance(Resource.class);
|
||||||
resource.setMemory(memory);
|
resource.setMemory(memory);
|
||||||
resource.setVirtualCores(vCores);
|
resource.setVirtualCores(vCores);
|
||||||
|
|
|
@ -59,8 +59,8 @@ public class AppInfo {
|
||||||
protected long elapsedTime;
|
protected long elapsedTime;
|
||||||
protected String applicationTags;
|
protected String applicationTags;
|
||||||
protected int priority;
|
protected int priority;
|
||||||
private int allocatedCpuVcores;
|
private long allocatedCpuVcores;
|
||||||
private int allocatedMemoryMB;
|
private long allocatedMemoryMB;
|
||||||
protected boolean unmanagedApplication;
|
protected boolean unmanagedApplication;
|
||||||
private String appNodeLabelExpression;
|
private String appNodeLabelExpression;
|
||||||
private String amNodeLabelExpression;
|
private String amNodeLabelExpression;
|
||||||
|
@ -100,7 +100,7 @@ public class AppInfo {
|
||||||
allocatedCpuVcores = app.getApplicationResourceUsageReport()
|
allocatedCpuVcores = app.getApplicationResourceUsageReport()
|
||||||
.getUsedResources().getVirtualCores();
|
.getUsedResources().getVirtualCores();
|
||||||
allocatedMemoryMB = app.getApplicationResourceUsageReport()
|
allocatedMemoryMB = app.getApplicationResourceUsageReport()
|
||||||
.getUsedResources().getMemory();
|
.getUsedResources().getMemorySize();
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
progress = app.getProgress() * 100; // in percent
|
progress = app.getProgress() * 100; // in percent
|
||||||
|
@ -152,11 +152,11 @@ public class AppInfo {
|
||||||
return runningContainers;
|
return runningContainers;
|
||||||
}
|
}
|
||||||
|
|
||||||
public int getAllocatedCpuVcores() {
|
public long getAllocatedCpuVcores() {
|
||||||
return allocatedCpuVcores;
|
return allocatedCpuVcores;
|
||||||
}
|
}
|
||||||
|
|
||||||
public int getAllocatedMemoryMB() {
|
public long getAllocatedMemoryMB() {
|
||||||
return allocatedMemoryMB;
|
return allocatedMemoryMB;
|
||||||
}
|
}
|
||||||
|
|
||||||
|
|
|
@ -36,8 +36,8 @@ import org.apache.hadoop.yarn.util.Times;
|
||||||
public class ContainerInfo {
|
public class ContainerInfo {
|
||||||
|
|
||||||
protected String containerId;
|
protected String containerId;
|
||||||
protected int allocatedMB;
|
protected long allocatedMB;
|
||||||
protected int allocatedVCores;
|
protected long allocatedVCores;
|
||||||
protected String assignedNodeId;
|
protected String assignedNodeId;
|
||||||
protected int priority;
|
protected int priority;
|
||||||
protected long startedTime;
|
protected long startedTime;
|
||||||
|
@ -57,7 +57,7 @@ public class ContainerInfo {
|
||||||
public ContainerInfo(ContainerReport container) {
|
public ContainerInfo(ContainerReport container) {
|
||||||
containerId = container.getContainerId().toString();
|
containerId = container.getContainerId().toString();
|
||||||
if (container.getAllocatedResource() != null) {
|
if (container.getAllocatedResource() != null) {
|
||||||
allocatedMB = container.getAllocatedResource().getMemory();
|
allocatedMB = container.getAllocatedResource().getMemorySize();
|
||||||
allocatedVCores = container.getAllocatedResource().getVirtualCores();
|
allocatedVCores = container.getAllocatedResource().getVirtualCores();
|
||||||
}
|
}
|
||||||
if (container.getAssignedNode() != null) {
|
if (container.getAssignedNode() != null) {
|
||||||
|
@ -79,11 +79,11 @@ public class ContainerInfo {
|
||||||
return containerId;
|
return containerId;
|
||||||
}
|
}
|
||||||
|
|
||||||
public int getAllocatedMB() {
|
public long getAllocatedMB() {
|
||||||
return allocatedMB;
|
return allocatedMB;
|
||||||
}
|
}
|
||||||
|
|
||||||
public int getAllocatedVCores() {
|
public long getAllocatedVCores() {
|
||||||
return allocatedVCores;
|
return allocatedVCores;
|
||||||
}
|
}
|
||||||
|
|
||||||
|
|
|
@ -180,7 +180,7 @@ public class TestYarnServerApiClasses {
|
||||||
assertEquals(1, copy.getContainersToDecrease().get(0)
|
assertEquals(1, copy.getContainersToDecrease().get(0)
|
||||||
.getId().getContainerId());
|
.getId().getContainerId());
|
||||||
assertEquals(1024, copy.getContainersToDecrease().get(1)
|
assertEquals(1024, copy.getContainersToDecrease().get(1)
|
||||||
.getResource().getMemory());
|
.getResource().getMemorySize());
|
||||||
}
|
}
|
||||||
|
|
||||||
/**
|
/**
|
||||||
|
@ -201,7 +201,7 @@ public class TestYarnServerApiClasses {
|
||||||
|
|
||||||
assertEquals(8080, copy.getHttpPort());
|
assertEquals(8080, copy.getHttpPort());
|
||||||
assertEquals(9090, copy.getNodeId().getPort());
|
assertEquals(9090, copy.getNodeId().getPort());
|
||||||
assertEquals(10000, copy.getResource().getMemory());
|
assertEquals(10000, copy.getResource().getMemorySize());
|
||||||
assertEquals(2, copy.getResource().getVirtualCores());
|
assertEquals(2, copy.getResource().getVirtualCores());
|
||||||
|
|
||||||
}
|
}
|
||||||
|
@ -273,7 +273,7 @@ public class TestYarnServerApiClasses {
|
||||||
assertEquals(1, copy.getIncreasedContainers().get(0)
|
assertEquals(1, copy.getIncreasedContainers().get(0)
|
||||||
.getId().getContainerId());
|
.getId().getContainerId());
|
||||||
assertEquals(4096, copy.getIncreasedContainers().get(1)
|
assertEquals(4096, copy.getIncreasedContainers().get(1)
|
||||||
.getResource().getMemory());
|
.getResource().getMemorySize());
|
||||||
}
|
}
|
||||||
|
|
||||||
@Test
|
@Test
|
||||||
|
|
|
@ -404,7 +404,7 @@ public abstract class ContainerExecutor implements Configurable {
|
||||||
.getBoolean(
|
.getBoolean(
|
||||||
YarnConfiguration.NM_WINDOWS_CONTAINER_MEMORY_LIMIT_ENABLED,
|
YarnConfiguration.NM_WINDOWS_CONTAINER_MEMORY_LIMIT_ENABLED,
|
||||||
YarnConfiguration.DEFAULT_NM_WINDOWS_CONTAINER_MEMORY_LIMIT_ENABLED)) {
|
YarnConfiguration.DEFAULT_NM_WINDOWS_CONTAINER_MEMORY_LIMIT_ENABLED)) {
|
||||||
memory = resource.getMemory();
|
memory = (int) resource.getMemorySize();
|
||||||
}
|
}
|
||||||
|
|
||||||
if (conf.getBoolean(
|
if (conf.getBoolean(
|
||||||
|
|
|
@ -168,7 +168,7 @@ public class ContainerImpl implements Container {
|
||||||
if (recoveredCapability != null
|
if (recoveredCapability != null
|
||||||
&& !this.resource.equals(recoveredCapability)) {
|
&& !this.resource.equals(recoveredCapability)) {
|
||||||
// resource capability had been updated before NM was down
|
// resource capability had been updated before NM was down
|
||||||
this.resource = Resource.newInstance(recoveredCapability.getMemory(),
|
this.resource = Resource.newInstance(recoveredCapability.getMemorySize(),
|
||||||
recoveredCapability.getVirtualCores());
|
recoveredCapability.getVirtualCores());
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
|
@ -516,7 +516,7 @@ public class ContainerImpl implements Container {
|
||||||
long launchDuration = clock.getTime() - containerLaunchStartTime;
|
long launchDuration = clock.getTime() - containerLaunchStartTime;
|
||||||
metrics.addContainerLaunchDuration(launchDuration);
|
metrics.addContainerLaunchDuration(launchDuration);
|
||||||
|
|
||||||
long pmemBytes = getResource().getMemory() * 1024 * 1024L;
|
long pmemBytes = getResource().getMemorySize() * 1024 * 1024L;
|
||||||
float pmemRatio = daemonConf.getFloat(
|
float pmemRatio = daemonConf.getFloat(
|
||||||
YarnConfiguration.NM_VMEM_PMEM_RATIO,
|
YarnConfiguration.NM_VMEM_PMEM_RATIO,
|
||||||
YarnConfiguration.DEFAULT_NM_VMEM_PMEM_RATIO);
|
YarnConfiguration.DEFAULT_NM_VMEM_PMEM_RATIO);
|
||||||
|
|
|
@ -642,7 +642,7 @@ public class ContainersMonitorImpl extends AbstractService implements
|
||||||
ChangeMonitoringContainerResourceEvent changeEvent =
|
ChangeMonitoringContainerResourceEvent changeEvent =
|
||||||
(ChangeMonitoringContainerResourceEvent) monitoringEvent;
|
(ChangeMonitoringContainerResourceEvent) monitoringEvent;
|
||||||
Resource resource = changeEvent.getResource();
|
Resource resource = changeEvent.getResource();
|
||||||
pmemLimitMBs = resource.getMemory();
|
pmemLimitMBs = (int) resource.getMemorySize();
|
||||||
vmemLimitMBs = (int) (pmemLimitMBs * vmemRatio);
|
vmemLimitMBs = (int) (pmemLimitMBs * vmemRatio);
|
||||||
cpuVcores = resource.getVirtualCores();
|
cpuVcores = resource.getVirtualCores();
|
||||||
usageMetrics.recordResourceLimit(
|
usageMetrics.recordResourceLimit(
|
||||||
|
@ -740,7 +740,7 @@ public class ContainersMonitorImpl extends AbstractService implements
|
||||||
}
|
}
|
||||||
LOG.info("Changing resource-monitoring for " + containerId);
|
LOG.info("Changing resource-monitoring for " + containerId);
|
||||||
updateContainerMetrics(monitoringEvent);
|
updateContainerMetrics(monitoringEvent);
|
||||||
long pmemLimit = changeEvent.getResource().getMemory() * 1024L * 1024L;
|
long pmemLimit = changeEvent.getResource().getMemorySize() * 1024L * 1024L;
|
||||||
long vmemLimit = (long) (pmemLimit * vmemRatio);
|
long vmemLimit = (long) (pmemLimit * vmemRatio);
|
||||||
int cpuVcores = changeEvent.getResource().getVirtualCores();
|
int cpuVcores = changeEvent.getResource().getVirtualCores();
|
||||||
processTreeInfo.setResourceLimit(pmemLimit, vmemLimit, cpuVcores);
|
processTreeInfo.setResourceLimit(pmemLimit, vmemLimit, cpuVcores);
|
||||||
|
|
|
@ -115,9 +115,9 @@ public class NodeManagerMetrics {
|
||||||
|
|
||||||
public void allocateContainer(Resource res) {
|
public void allocateContainer(Resource res) {
|
||||||
allocatedContainers.incr();
|
allocatedContainers.incr();
|
||||||
allocatedMB = allocatedMB + res.getMemory();
|
allocatedMB = allocatedMB + res.getMemorySize();
|
||||||
allocatedGB.set((int)Math.ceil(allocatedMB/1024d));
|
allocatedGB.set((int)Math.ceil(allocatedMB/1024d));
|
||||||
availableMB = availableMB - res.getMemory();
|
availableMB = availableMB - res.getMemorySize();
|
||||||
availableGB.set((int)Math.floor(availableMB/1024d));
|
availableGB.set((int)Math.floor(availableMB/1024d));
|
||||||
allocatedVCores.incr(res.getVirtualCores());
|
allocatedVCores.incr(res.getVirtualCores());
|
||||||
availableVCores.decr(res.getVirtualCores());
|
availableVCores.decr(res.getVirtualCores());
|
||||||
|
@ -125,16 +125,16 @@ public class NodeManagerMetrics {
|
||||||
|
|
||||||
public void releaseContainer(Resource res) {
|
public void releaseContainer(Resource res) {
|
||||||
allocatedContainers.decr();
|
allocatedContainers.decr();
|
||||||
allocatedMB = allocatedMB - res.getMemory();
|
allocatedMB = allocatedMB - res.getMemorySize();
|
||||||
allocatedGB.set((int)Math.ceil(allocatedMB/1024d));
|
allocatedGB.set((int)Math.ceil(allocatedMB/1024d));
|
||||||
availableMB = availableMB + res.getMemory();
|
availableMB = availableMB + res.getMemorySize();
|
||||||
availableGB.set((int)Math.floor(availableMB/1024d));
|
availableGB.set((int)Math.floor(availableMB/1024d));
|
||||||
allocatedVCores.decr(res.getVirtualCores());
|
allocatedVCores.decr(res.getVirtualCores());
|
||||||
availableVCores.incr(res.getVirtualCores());
|
availableVCores.incr(res.getVirtualCores());
|
||||||
}
|
}
|
||||||
|
|
||||||
public void changeContainer(Resource before, Resource now) {
|
public void changeContainer(Resource before, Resource now) {
|
||||||
int deltaMB = now.getMemory() - before.getMemory();
|
long deltaMB = now.getMemorySize() - before.getMemorySize();
|
||||||
int deltaVCores = now.getVirtualCores() - before.getVirtualCores();
|
int deltaVCores = now.getVirtualCores() - before.getVirtualCores();
|
||||||
allocatedMB = allocatedMB + deltaMB;
|
allocatedMB = allocatedMB + deltaMB;
|
||||||
allocatedGB.set((int)Math.ceil(allocatedMB/1024d));
|
allocatedGB.set((int)Math.ceil(allocatedMB/1024d));
|
||||||
|
@ -145,7 +145,7 @@ public class NodeManagerMetrics {
|
||||||
}
|
}
|
||||||
|
|
||||||
public void addResource(Resource res) {
|
public void addResource(Resource res) {
|
||||||
availableMB = availableMB + res.getMemory();
|
availableMB = availableMB + res.getMemorySize();
|
||||||
availableGB.incr((int)Math.floor(availableMB/1024d));
|
availableGB.incr((int)Math.floor(availableMB/1024d));
|
||||||
availableVCores.incr(res.getVirtualCores());
|
availableVCores.incr(res.getVirtualCores());
|
||||||
}
|
}
|
||||||
|
|
|
@ -81,7 +81,7 @@ public class ContainerInfo {
|
||||||
this.user = container.getUser();
|
this.user = container.getUser();
|
||||||
Resource res = container.getResource();
|
Resource res = container.getResource();
|
||||||
if (res != null) {
|
if (res != null) {
|
||||||
this.totalMemoryNeededMB = res.getMemory();
|
this.totalMemoryNeededMB = res.getMemorySize();
|
||||||
this.totalVCoresNeeded = res.getVirtualCores();
|
this.totalVCoresNeeded = res.getVirtualCores();
|
||||||
}
|
}
|
||||||
this.containerLogsShortLink = ujoin("containerlogs", this.id,
|
this.containerLogsShortLink = ujoin("containerlogs", this.id,
|
||||||
|
|
|
@ -188,7 +188,7 @@ public class TestNodeStatusUpdater {
|
||||||
InetSocketAddress expected = NetUtils.getConnectAddress(
|
InetSocketAddress expected = NetUtils.getConnectAddress(
|
||||||
conf.getSocketAddr(YarnConfiguration.NM_ADDRESS, null, -1));
|
conf.getSocketAddr(YarnConfiguration.NM_ADDRESS, null, -1));
|
||||||
Assert.assertEquals(NetUtils.getHostPortString(expected), nodeId.toString());
|
Assert.assertEquals(NetUtils.getHostPortString(expected), nodeId.toString());
|
||||||
Assert.assertEquals(5 * 1024, resource.getMemory());
|
Assert.assertEquals(5 * 1024, resource.getMemorySize());
|
||||||
registeredNodes.add(nodeId);
|
registeredNodes.add(nodeId);
|
||||||
|
|
||||||
RegisterNodeManagerResponse response = recordFactory
|
RegisterNodeManagerResponse response = recordFactory
|
||||||
|
@ -914,7 +914,7 @@ public class TestNodeStatusUpdater {
|
||||||
conf.getSocketAddr(YarnConfiguration.NM_ADDRESS, null, -1));
|
conf.getSocketAddr(YarnConfiguration.NM_ADDRESS, null, -1));
|
||||||
Assert.assertEquals(NetUtils.getHostPortString(expected),
|
Assert.assertEquals(NetUtils.getHostPortString(expected),
|
||||||
nodeId.toString());
|
nodeId.toString());
|
||||||
Assert.assertEquals(5 * 1024, resource.getMemory());
|
Assert.assertEquals(5 * 1024, resource.getMemorySize());
|
||||||
registeredNodes.add(nodeId);
|
registeredNodes.add(nodeId);
|
||||||
|
|
||||||
RegisterNodeManagerResponse response = recordFactory
|
RegisterNodeManagerResponse response = recordFactory
|
||||||
|
|
|
@ -228,7 +228,7 @@ public class TestContainersMonitor extends BaseContainerManagerTest {
|
||||||
commands.add("/bin/bash");
|
commands.add("/bin/bash");
|
||||||
commands.add(scriptFile.getAbsolutePath());
|
commands.add(scriptFile.getAbsolutePath());
|
||||||
containerLaunchContext.setCommands(commands);
|
containerLaunchContext.setCommands(commands);
|
||||||
Resource r = BuilderUtils.newResource(8 * 1024 * 1024, 1);
|
Resource r = BuilderUtils.newResource(0, 0);
|
||||||
ContainerTokenIdentifier containerIdentifier =
|
ContainerTokenIdentifier containerIdentifier =
|
||||||
new ContainerTokenIdentifier(cId, context.getNodeId().toString(), user,
|
new ContainerTokenIdentifier(cId, context.getNodeId().toString(), user,
|
||||||
r, System.currentTimeMillis() + 120000, 123, DUMMY_RM_IDENTIFIER,
|
r, System.currentTimeMillis() + 120000, 123, DUMMY_RM_IDENTIFIER,
|
||||||
|
|
|
@ -94,9 +94,9 @@ public class RMNMInfo implements RMNMInfoBeans {
|
||||||
ni.getNodeManagerVersion());
|
ni.getNodeManagerVersion());
|
||||||
if(report != null) {
|
if(report != null) {
|
||||||
info.put("NumContainers", report.getNumContainers());
|
info.put("NumContainers", report.getNumContainers());
|
||||||
info.put("UsedMemoryMB", report.getUsedResource().getMemory());
|
info.put("UsedMemoryMB", report.getUsedResource().getMemorySize());
|
||||||
info.put("AvailableMemoryMB",
|
info.put("AvailableMemoryMB",
|
||||||
report.getAvailableResource().getMemory());
|
report.getAvailableResource().getMemorySize());
|
||||||
}
|
}
|
||||||
|
|
||||||
nodesInfo.add(info);
|
nodesInfo.add(info);
|
||||||
|
|
|
@ -142,7 +142,7 @@ public class RMServerUtils {
|
||||||
// example, you cannot request target resource of a <10G, 10> container to
|
// example, you cannot request target resource of a <10G, 10> container to
|
||||||
// <20G, 8>
|
// <20G, 8>
|
||||||
if (increase) {
|
if (increase) {
|
||||||
if (originalResource.getMemory() > targetResource.getMemory()
|
if (originalResource.getMemorySize() > targetResource.getMemorySize()
|
||||||
|| originalResource.getVirtualCores() > targetResource
|
|| originalResource.getVirtualCores() > targetResource
|
||||||
.getVirtualCores()) {
|
.getVirtualCores()) {
|
||||||
String msg =
|
String msg =
|
||||||
|
@ -153,7 +153,7 @@ public class RMServerUtils {
|
||||||
throw new InvalidResourceRequestException(msg);
|
throw new InvalidResourceRequestException(msg);
|
||||||
}
|
}
|
||||||
} else {
|
} else {
|
||||||
if (originalResource.getMemory() < targetResource.getMemory()
|
if (originalResource.getMemorySize() < targetResource.getMemorySize()
|
||||||
|| originalResource.getVirtualCores() < targetResource
|
|| originalResource.getVirtualCores() < targetResource
|
||||||
.getVirtualCores()) {
|
.getVirtualCores()) {
|
||||||
String msg =
|
String msg =
|
||||||
|
@ -243,15 +243,15 @@ public class RMServerUtils {
|
||||||
return;
|
return;
|
||||||
}
|
}
|
||||||
for (ContainerResourceChangeRequest request : requests) {
|
for (ContainerResourceChangeRequest request : requests) {
|
||||||
if (request.getCapability().getMemory() < 0
|
if (request.getCapability().getMemorySize() < 0
|
||||||
|| request.getCapability().getMemory() > maximumAllocation
|
|| request.getCapability().getMemorySize() > maximumAllocation
|
||||||
.getMemory()) {
|
.getMemorySize()) {
|
||||||
throw new InvalidResourceRequestException("Invalid "
|
throw new InvalidResourceRequestException("Invalid "
|
||||||
+ (increase ? "increase" : "decrease") + " request"
|
+ (increase ? "increase" : "decrease") + " request"
|
||||||
+ ", requested memory < 0"
|
+ ", requested memory < 0"
|
||||||
+ ", or requested memory > max configured" + ", requestedMemory="
|
+ ", or requested memory > max configured" + ", requestedMemory="
|
||||||
+ request.getCapability().getMemory() + ", maxMemory="
|
+ request.getCapability().getMemorySize() + ", maxMemory="
|
||||||
+ maximumAllocation.getMemory());
|
+ maximumAllocation.getMemorySize());
|
||||||
}
|
}
|
||||||
if (request.getCapability().getVirtualCores() < 0
|
if (request.getCapability().getVirtualCores() < 0
|
||||||
|| request.getCapability().getVirtualCores() > maximumAllocation
|
|| request.getCapability().getVirtualCores() > maximumAllocation
|
||||||
|
|
|
@ -339,7 +339,7 @@ public class ResourceTrackerService extends AbstractService implements
|
||||||
}
|
}
|
||||||
|
|
||||||
// Check if this node has minimum allocations
|
// Check if this node has minimum allocations
|
||||||
if (capability.getMemory() < minAllocMb
|
if (capability.getMemorySize() < minAllocMb
|
||||||
|| capability.getVirtualCores() < minAllocVcores) {
|
|| capability.getVirtualCores() < minAllocVcores) {
|
||||||
String message =
|
String message =
|
||||||
"NodeManager from " + host
|
"NodeManager from " + host
|
||||||
|
|
|
@ -480,7 +480,7 @@ public class SystemMetricsPublisher extends CompositeService {
|
||||||
TimelineEntity entity = createContainerEntity(event.getContainerId());
|
TimelineEntity entity = createContainerEntity(event.getContainerId());
|
||||||
Map<String, Object> entityInfo = new HashMap<String, Object>();
|
Map<String, Object> entityInfo = new HashMap<String, Object>();
|
||||||
entityInfo.put(ContainerMetricsConstants.ALLOCATED_MEMORY_ENTITY_INFO,
|
entityInfo.put(ContainerMetricsConstants.ALLOCATED_MEMORY_ENTITY_INFO,
|
||||||
event.getAllocatedResource().getMemory());
|
event.getAllocatedResource().getMemorySize());
|
||||||
entityInfo.put(ContainerMetricsConstants.ALLOCATED_VCORE_ENTITY_INFO,
|
entityInfo.put(ContainerMetricsConstants.ALLOCATED_VCORE_ENTITY_INFO,
|
||||||
event.getAllocatedResource().getVirtualCores());
|
event.getAllocatedResource().getVirtualCores());
|
||||||
entityInfo.put(ContainerMetricsConstants.ALLOCATED_HOST_ENTITY_INFO,
|
entityInfo.put(ContainerMetricsConstants.ALLOCATED_HOST_ENTITY_INFO,
|
||||||
|
|
|
@ -1083,17 +1083,17 @@ public class ProportionalCapacityPreemptionPolicy implements SchedulingEditPolic
|
||||||
|
|
||||||
void appendLogString(StringBuilder sb) {
|
void appendLogString(StringBuilder sb) {
|
||||||
sb.append(queueName).append(", ")
|
sb.append(queueName).append(", ")
|
||||||
.append(current.getMemory()).append(", ")
|
.append(current.getMemorySize()).append(", ")
|
||||||
.append(current.getVirtualCores()).append(", ")
|
.append(current.getVirtualCores()).append(", ")
|
||||||
.append(pending.getMemory()).append(", ")
|
.append(pending.getMemorySize()).append(", ")
|
||||||
.append(pending.getVirtualCores()).append(", ")
|
.append(pending.getVirtualCores()).append(", ")
|
||||||
.append(guaranteed.getMemory()).append(", ")
|
.append(guaranteed.getMemorySize()).append(", ")
|
||||||
.append(guaranteed.getVirtualCores()).append(", ")
|
.append(guaranteed.getVirtualCores()).append(", ")
|
||||||
.append(idealAssigned.getMemory()).append(", ")
|
.append(idealAssigned.getMemorySize()).append(", ")
|
||||||
.append(idealAssigned.getVirtualCores()).append(", ")
|
.append(idealAssigned.getVirtualCores()).append(", ")
|
||||||
.append(toBePreempted.getMemory()).append(", ")
|
.append(toBePreempted.getMemorySize()).append(", ")
|
||||||
.append(toBePreempted.getVirtualCores() ).append(", ")
|
.append(toBePreempted.getVirtualCores() ).append(", ")
|
||||||
.append(actuallyPreempted.getMemory()).append(", ")
|
.append(actuallyPreempted.getMemorySize()).append(", ")
|
||||||
.append(actuallyPreempted.getVirtualCores());
|
.append(actuallyPreempted.getVirtualCores());
|
||||||
}
|
}
|
||||||
|
|
||||||
|
|
|
@ -160,7 +160,7 @@ public abstract class AbstractSchedulerPlanFollower implements PlanFollower {
|
||||||
}
|
}
|
||||||
Resource capToAssign = res.getResourcesAtTime(now);
|
Resource capToAssign = res.getResourcesAtTime(now);
|
||||||
float targetCapacity = 0f;
|
float targetCapacity = 0f;
|
||||||
if (planResources.getMemory() > 0
|
if (planResources.getMemorySize() > 0
|
||||||
&& planResources.getVirtualCores() > 0) {
|
&& planResources.getVirtualCores() > 0) {
|
||||||
if (shouldResize) {
|
if (shouldResize) {
|
||||||
capToAssign =
|
capToAssign =
|
||||||
|
|
|
@ -246,7 +246,7 @@ public class CapacityOverTimePolicy implements SharingPolicy {
|
||||||
long vcores;
|
long vcores;
|
||||||
|
|
||||||
public IntegralResource(Resource resource) {
|
public IntegralResource(Resource resource) {
|
||||||
this.memory = resource.getMemory();
|
this.memory = resource.getMemorySize();
|
||||||
this.vcores = resource.getVirtualCores();
|
this.vcores = resource.getVirtualCores();
|
||||||
}
|
}
|
||||||
|
|
||||||
|
@ -256,12 +256,12 @@ public class CapacityOverTimePolicy implements SharingPolicy {
|
||||||
}
|
}
|
||||||
|
|
||||||
public void add(Resource r) {
|
public void add(Resource r) {
|
||||||
memory += r.getMemory();
|
memory += r.getMemorySize();
|
||||||
vcores += r.getVirtualCores();
|
vcores += r.getVirtualCores();
|
||||||
}
|
}
|
||||||
|
|
||||||
public void subtract(Resource r) {
|
public void subtract(Resource r) {
|
||||||
memory -= r.getMemory();
|
memory -= r.getMemorySize();
|
||||||
vcores -= r.getVirtualCores();
|
vcores -= r.getVirtualCores();
|
||||||
}
|
}
|
||||||
|
|
||||||
|
|
|
@ -106,7 +106,7 @@ public final class ReservationSystemUtil {
|
||||||
|
|
||||||
public static ResourceProto convertToProtoFormat(Resource e) {
|
public static ResourceProto convertToProtoFormat(Resource e) {
|
||||||
return YarnProtos.ResourceProto.newBuilder()
|
return YarnProtos.ResourceProto.newBuilder()
|
||||||
.setMemory(e.getMemory())
|
.setMemory(e.getMemorySize())
|
||||||
.setVirtualCores(e.getVirtualCores())
|
.setVirtualCores(e.getVirtualCores())
|
||||||
.build();
|
.build();
|
||||||
}
|
}
|
||||||
|
|
|
@ -88,7 +88,7 @@ public class StageEarliestStartByDemand implements StageEarliestStart {
|
||||||
|
|
||||||
// Weight = total memory consumption of stage
|
// Weight = total memory consumption of stage
|
||||||
protected double calcWeight(ReservationRequest stage) {
|
protected double calcWeight(ReservationRequest stage) {
|
||||||
return (stage.getDuration() * stage.getCapability().getMemory())
|
return (stage.getDuration() * stage.getCapability().getMemorySize())
|
||||||
* (stage.getNumContainers());
|
* (stage.getNumContainers());
|
||||||
}
|
}
|
||||||
|
|
||||||
|
|
|
@ -708,7 +708,7 @@ public class RMContainerImpl implements RMContainer, Comparable<RMContainer> {
|
||||||
}
|
}
|
||||||
|
|
||||||
long usedMillis = container.finishTime - container.creationTime;
|
long usedMillis = container.finishTime - container.creationTime;
|
||||||
long memorySeconds = resource.getMemory()
|
long memorySeconds = resource.getMemorySize()
|
||||||
* usedMillis / DateUtils.MILLIS_PER_SECOND;
|
* usedMillis / DateUtils.MILLIS_PER_SECOND;
|
||||||
long vcoreSeconds = resource.getVirtualCores()
|
long vcoreSeconds = resource.getVirtualCores()
|
||||||
* usedMillis / DateUtils.MILLIS_PER_SECOND;
|
* usedMillis / DateUtils.MILLIS_PER_SECOND;
|
||||||
|
|
|
@ -711,11 +711,11 @@ public abstract class AbstractYarnScheduler
|
||||||
maxAllocWriteLock.lock();
|
maxAllocWriteLock.lock();
|
||||||
try {
|
try {
|
||||||
if (add) { // added node
|
if (add) { // added node
|
||||||
int nodeMemory = totalResource.getMemory();
|
int nodeMemory = (int)totalResource.getMemorySize();
|
||||||
if (nodeMemory > maxNodeMemory) {
|
if (nodeMemory > maxNodeMemory) {
|
||||||
maxNodeMemory = nodeMemory;
|
maxNodeMemory = nodeMemory;
|
||||||
maximumAllocation.setMemory(Math.min(
|
maximumAllocation.setMemory(Math.min(
|
||||||
configuredMaximumAllocation.getMemory(), maxNodeMemory));
|
configuredMaximumAllocation.getMemorySize(), maxNodeMemory));
|
||||||
}
|
}
|
||||||
int nodeVCores = totalResource.getVirtualCores();
|
int nodeVCores = totalResource.getVirtualCores();
|
||||||
if (nodeVCores > maxNodeVCores) {
|
if (nodeVCores > maxNodeVCores) {
|
||||||
|
@ -724,7 +724,7 @@ public abstract class AbstractYarnScheduler
|
||||||
configuredMaximumAllocation.getVirtualCores(), maxNodeVCores));
|
configuredMaximumAllocation.getVirtualCores(), maxNodeVCores));
|
||||||
}
|
}
|
||||||
} else { // removed node
|
} else { // removed node
|
||||||
if (maxNodeMemory == totalResource.getMemory()) {
|
if (maxNodeMemory == totalResource.getMemorySize()) {
|
||||||
maxNodeMemory = -1;
|
maxNodeMemory = -1;
|
||||||
}
|
}
|
||||||
if (maxNodeVCores == totalResource.getVirtualCores()) {
|
if (maxNodeVCores == totalResource.getVirtualCores()) {
|
||||||
|
@ -735,7 +735,7 @@ public abstract class AbstractYarnScheduler
|
||||||
if (maxNodeMemory == -1 || maxNodeVCores == -1) {
|
if (maxNodeMemory == -1 || maxNodeVCores == -1) {
|
||||||
for (Map.Entry<NodeId, N> nodeEntry : nodes.entrySet()) {
|
for (Map.Entry<NodeId, N> nodeEntry : nodes.entrySet()) {
|
||||||
int nodeMemory =
|
int nodeMemory =
|
||||||
nodeEntry.getValue().getTotalResource().getMemory();
|
(int)nodeEntry.getValue().getTotalResource().getMemorySize();
|
||||||
if (nodeMemory > maxNodeMemory) {
|
if (nodeMemory > maxNodeMemory) {
|
||||||
maxNodeMemory = nodeMemory;
|
maxNodeMemory = nodeMemory;
|
||||||
}
|
}
|
||||||
|
@ -746,10 +746,10 @@ public abstract class AbstractYarnScheduler
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
if (maxNodeMemory == -1) { // no nodes
|
if (maxNodeMemory == -1) { // no nodes
|
||||||
maximumAllocation.setMemory(configuredMaximumAllocation.getMemory());
|
maximumAllocation.setMemory(configuredMaximumAllocation.getMemorySize());
|
||||||
} else {
|
} else {
|
||||||
maximumAllocation.setMemory(
|
maximumAllocation.setMemory(
|
||||||
Math.min(configuredMaximumAllocation.getMemory(), maxNodeMemory));
|
Math.min(configuredMaximumAllocation.getMemorySize(), maxNodeMemory));
|
||||||
}
|
}
|
||||||
if (maxNodeVCores == -1) { // no nodes
|
if (maxNodeVCores == -1) { // no nodes
|
||||||
maximumAllocation.setVirtualCores(configuredMaximumAllocation.getVirtualCores());
|
maximumAllocation.setVirtualCores(configuredMaximumAllocation.getVirtualCores());
|
||||||
|
@ -768,7 +768,7 @@ public abstract class AbstractYarnScheduler
|
||||||
maxAllocWriteLock.lock();
|
maxAllocWriteLock.lock();
|
||||||
try {
|
try {
|
||||||
configuredMaximumAllocation = Resources.clone(newMaxAlloc);
|
configuredMaximumAllocation = Resources.clone(newMaxAlloc);
|
||||||
int maxMemory = newMaxAlloc.getMemory();
|
long maxMemory = newMaxAlloc.getMemorySize();
|
||||||
if (maxNodeMemory != -1) {
|
if (maxNodeMemory != -1) {
|
||||||
maxMemory = Math.min(maxMemory, maxNodeMemory);
|
maxMemory = Math.min(maxMemory, maxNodeMemory);
|
||||||
}
|
}
|
||||||
|
|
|
@ -38,6 +38,7 @@ import org.apache.hadoop.metrics2.lib.MetricsRegistry;
|
||||||
import org.apache.hadoop.metrics2.lib.MutableCounterInt;
|
import org.apache.hadoop.metrics2.lib.MutableCounterInt;
|
||||||
import org.apache.hadoop.metrics2.lib.MutableCounterLong;
|
import org.apache.hadoop.metrics2.lib.MutableCounterLong;
|
||||||
import org.apache.hadoop.metrics2.lib.MutableGaugeInt;
|
import org.apache.hadoop.metrics2.lib.MutableGaugeInt;
|
||||||
|
import org.apache.hadoop.metrics2.lib.MutableGaugeLong;
|
||||||
import org.apache.hadoop.metrics2.lib.MutableRate;
|
import org.apache.hadoop.metrics2.lib.MutableRate;
|
||||||
import org.apache.hadoop.yarn.api.records.ApplicationId;
|
import org.apache.hadoop.yarn.api.records.ApplicationId;
|
||||||
import org.apache.hadoop.yarn.api.records.Resource;
|
import org.apache.hadoop.yarn.api.records.Resource;
|
||||||
|
@ -59,8 +60,8 @@ public class QueueMetrics implements MetricsSource {
|
||||||
@Metric("# of apps killed") MutableCounterInt appsKilled;
|
@Metric("# of apps killed") MutableCounterInt appsKilled;
|
||||||
@Metric("# of apps failed") MutableCounterInt appsFailed;
|
@Metric("# of apps failed") MutableCounterInt appsFailed;
|
||||||
|
|
||||||
@Metric("Allocated memory in MB") MutableGaugeInt allocatedMB;
|
@Metric("Allocated memory in MB") MutableGaugeLong allocatedMB;
|
||||||
@Metric("Allocated CPU in virtual cores") MutableGaugeInt allocatedVCores;
|
@Metric("Allocated CPU in virtual cores") MutableGaugeLong allocatedVCores;
|
||||||
@Metric("# of allocated containers") MutableGaugeInt allocatedContainers;
|
@Metric("# of allocated containers") MutableGaugeInt allocatedContainers;
|
||||||
@Metric("Aggregate # of allocated containers") MutableCounterLong aggregateContainersAllocated;
|
@Metric("Aggregate # of allocated containers") MutableCounterLong aggregateContainersAllocated;
|
||||||
@Metric("Aggregate # of allocated node-local containers")
|
@Metric("Aggregate # of allocated node-local containers")
|
||||||
|
@ -70,13 +71,13 @@ public class QueueMetrics implements MetricsSource {
|
||||||
@Metric("Aggregate # of allocated off-switch containers")
|
@Metric("Aggregate # of allocated off-switch containers")
|
||||||
MutableCounterLong aggregateOffSwitchContainersAllocated;
|
MutableCounterLong aggregateOffSwitchContainersAllocated;
|
||||||
@Metric("Aggregate # of released containers") MutableCounterLong aggregateContainersReleased;
|
@Metric("Aggregate # of released containers") MutableCounterLong aggregateContainersReleased;
|
||||||
@Metric("Available memory in MB") MutableGaugeInt availableMB;
|
@Metric("Available memory in MB") MutableGaugeLong availableMB;
|
||||||
@Metric("Available CPU in virtual cores") MutableGaugeInt availableVCores;
|
@Metric("Available CPU in virtual cores") MutableGaugeLong availableVCores;
|
||||||
@Metric("Pending memory allocation in MB") MutableGaugeInt pendingMB;
|
@Metric("Pending memory allocation in MB") MutableGaugeLong pendingMB;
|
||||||
@Metric("Pending CPU allocation in virtual cores") MutableGaugeInt pendingVCores;
|
@Metric("Pending CPU allocation in virtual cores") MutableGaugeLong pendingVCores;
|
||||||
@Metric("# of pending containers") MutableGaugeInt pendingContainers;
|
@Metric("# of pending containers") MutableGaugeInt pendingContainers;
|
||||||
@Metric("# of reserved memory in MB") MutableGaugeInt reservedMB;
|
@Metric("# of reserved memory in MB") MutableGaugeLong reservedMB;
|
||||||
@Metric("Reserved CPU in virtual cores") MutableGaugeInt reservedVCores;
|
@Metric("Reserved CPU in virtual cores") MutableGaugeLong reservedVCores;
|
||||||
@Metric("# of reserved containers") MutableGaugeInt reservedContainers;
|
@Metric("# of reserved containers") MutableGaugeInt reservedContainers;
|
||||||
@Metric("# of active users") MutableGaugeInt activeUsers;
|
@Metric("# of active users") MutableGaugeInt activeUsers;
|
||||||
@Metric("# of active applications") MutableGaugeInt activeApplications;
|
@Metric("# of active applications") MutableGaugeInt activeApplications;
|
||||||
|
@ -325,7 +326,7 @@ public class QueueMetrics implements MetricsSource {
|
||||||
* @param limit resource limit
|
* @param limit resource limit
|
||||||
*/
|
*/
|
||||||
public void setAvailableResourcesToQueue(Resource limit) {
|
public void setAvailableResourcesToQueue(Resource limit) {
|
||||||
availableMB.set(limit.getMemory());
|
availableMB.set(limit.getMemorySize());
|
||||||
availableVCores.set(limit.getVirtualCores());
|
availableVCores.set(limit.getVirtualCores());
|
||||||
}
|
}
|
||||||
|
|
||||||
|
@ -362,8 +363,8 @@ public class QueueMetrics implements MetricsSource {
|
||||||
|
|
||||||
private void _incrPendingResources(int containers, Resource res) {
|
private void _incrPendingResources(int containers, Resource res) {
|
||||||
pendingContainers.incr(containers);
|
pendingContainers.incr(containers);
|
||||||
pendingMB.incr(res.getMemory() * containers);
|
pendingMB.incr(res.getMemorySize() * containers);
|
||||||
pendingVCores.incr(res.getVirtualCores() * containers);
|
pendingVCores.incr(res.getVirtualCoresSize() * containers);
|
||||||
}
|
}
|
||||||
|
|
||||||
public void decrPendingResources(String user, int containers, Resource res) {
|
public void decrPendingResources(String user, int containers, Resource res) {
|
||||||
|
@ -379,8 +380,8 @@ public class QueueMetrics implements MetricsSource {
|
||||||
|
|
||||||
private void _decrPendingResources(int containers, Resource res) {
|
private void _decrPendingResources(int containers, Resource res) {
|
||||||
pendingContainers.decr(containers);
|
pendingContainers.decr(containers);
|
||||||
pendingMB.decr(res.getMemory() * containers);
|
pendingMB.decr(res.getMemorySize() * containers);
|
||||||
pendingVCores.decr(res.getVirtualCores() * containers);
|
pendingVCores.decr(res.getVirtualCoresSize() * containers);
|
||||||
}
|
}
|
||||||
|
|
||||||
public void incrNodeTypeAggregations(String user, NodeType type) {
|
public void incrNodeTypeAggregations(String user, NodeType type) {
|
||||||
|
@ -407,8 +408,8 @@ public class QueueMetrics implements MetricsSource {
|
||||||
allocatedContainers.incr(containers);
|
allocatedContainers.incr(containers);
|
||||||
aggregateContainersAllocated.incr(containers);
|
aggregateContainersAllocated.incr(containers);
|
||||||
|
|
||||||
allocatedMB.incr(res.getMemory() * containers);
|
allocatedMB.incr(res.getMemorySize() * containers);
|
||||||
allocatedVCores.incr(res.getVirtualCores() * containers);
|
allocatedVCores.incr(res.getVirtualCoresSize() * containers);
|
||||||
if (decrPending) {
|
if (decrPending) {
|
||||||
_decrPendingResources(containers, res);
|
_decrPendingResources(containers, res);
|
||||||
}
|
}
|
||||||
|
@ -428,10 +429,10 @@ public class QueueMetrics implements MetricsSource {
|
||||||
* @param res
|
* @param res
|
||||||
*/
|
*/
|
||||||
public void allocateResources(String user, Resource res) {
|
public void allocateResources(String user, Resource res) {
|
||||||
allocatedMB.incr(res.getMemory());
|
allocatedMB.incr(res.getMemorySize());
|
||||||
allocatedVCores.incr(res.getVirtualCores());
|
allocatedVCores.incr(res.getVirtualCores());
|
||||||
|
|
||||||
pendingMB.decr(res.getMemory());
|
pendingMB.decr(res.getMemorySize());
|
||||||
pendingVCores.decr(res.getVirtualCores());
|
pendingVCores.decr(res.getVirtualCores());
|
||||||
|
|
||||||
QueueMetrics userMetrics = getUserMetrics(user);
|
QueueMetrics userMetrics = getUserMetrics(user);
|
||||||
|
@ -446,8 +447,8 @@ public class QueueMetrics implements MetricsSource {
|
||||||
public void releaseResources(String user, int containers, Resource res) {
|
public void releaseResources(String user, int containers, Resource res) {
|
||||||
allocatedContainers.decr(containers);
|
allocatedContainers.decr(containers);
|
||||||
aggregateContainersReleased.incr(containers);
|
aggregateContainersReleased.incr(containers);
|
||||||
allocatedMB.decr(res.getMemory() * containers);
|
allocatedMB.decr(res.getMemorySize() * containers);
|
||||||
allocatedVCores.decr(res.getVirtualCores() * containers);
|
allocatedVCores.decr(res.getVirtualCoresSize() * containers);
|
||||||
QueueMetrics userMetrics = getUserMetrics(user);
|
QueueMetrics userMetrics = getUserMetrics(user);
|
||||||
if (userMetrics != null) {
|
if (userMetrics != null) {
|
||||||
userMetrics.releaseResources(user, containers, res);
|
userMetrics.releaseResources(user, containers, res);
|
||||||
|
@ -464,7 +465,7 @@ public class QueueMetrics implements MetricsSource {
|
||||||
* @param res
|
* @param res
|
||||||
*/
|
*/
|
||||||
public void releaseResources(String user, Resource res) {
|
public void releaseResources(String user, Resource res) {
|
||||||
allocatedMB.decr(res.getMemory());
|
allocatedMB.decr(res.getMemorySize());
|
||||||
allocatedVCores.decr(res.getVirtualCores());
|
allocatedVCores.decr(res.getVirtualCores());
|
||||||
QueueMetrics userMetrics = getUserMetrics(user);
|
QueueMetrics userMetrics = getUserMetrics(user);
|
||||||
if (userMetrics != null) {
|
if (userMetrics != null) {
|
||||||
|
@ -477,7 +478,7 @@ public class QueueMetrics implements MetricsSource {
|
||||||
|
|
||||||
public void reserveResource(String user, Resource res) {
|
public void reserveResource(String user, Resource res) {
|
||||||
reservedContainers.incr();
|
reservedContainers.incr();
|
||||||
reservedMB.incr(res.getMemory());
|
reservedMB.incr(res.getMemorySize());
|
||||||
reservedVCores.incr(res.getVirtualCores());
|
reservedVCores.incr(res.getVirtualCores());
|
||||||
QueueMetrics userMetrics = getUserMetrics(user);
|
QueueMetrics userMetrics = getUserMetrics(user);
|
||||||
if (userMetrics != null) {
|
if (userMetrics != null) {
|
||||||
|
@ -490,7 +491,7 @@ public class QueueMetrics implements MetricsSource {
|
||||||
|
|
||||||
public void unreserveResource(String user, Resource res) {
|
public void unreserveResource(String user, Resource res) {
|
||||||
reservedContainers.decr();
|
reservedContainers.decr();
|
||||||
reservedMB.decr(res.getMemory());
|
reservedMB.decr(res.getMemorySize());
|
||||||
reservedVCores.decr(res.getVirtualCores());
|
reservedVCores.decr(res.getVirtualCores());
|
||||||
QueueMetrics userMetrics = getUserMetrics(user);
|
QueueMetrics userMetrics = getUserMetrics(user);
|
||||||
if (userMetrics != null) {
|
if (userMetrics != null) {
|
||||||
|
@ -563,11 +564,11 @@ public class QueueMetrics implements MetricsSource {
|
||||||
return BuilderUtils.newResource(allocatedMB.value(), allocatedVCores.value());
|
return BuilderUtils.newResource(allocatedMB.value(), allocatedVCores.value());
|
||||||
}
|
}
|
||||||
|
|
||||||
public int getAllocatedMB() {
|
public long getAllocatedMB() {
|
||||||
return allocatedMB.value();
|
return allocatedMB.value();
|
||||||
}
|
}
|
||||||
|
|
||||||
public int getAllocatedVirtualCores() {
|
public long getAllocatedVirtualCores() {
|
||||||
return allocatedVCores.value();
|
return allocatedVCores.value();
|
||||||
}
|
}
|
||||||
|
|
||||||
|
@ -575,19 +576,19 @@ public class QueueMetrics implements MetricsSource {
|
||||||
return allocatedContainers.value();
|
return allocatedContainers.value();
|
||||||
}
|
}
|
||||||
|
|
||||||
public int getAvailableMB() {
|
public long getAvailableMB() {
|
||||||
return availableMB.value();
|
return availableMB.value();
|
||||||
}
|
}
|
||||||
|
|
||||||
public int getAvailableVirtualCores() {
|
public long getAvailableVirtualCores() {
|
||||||
return availableVCores.value();
|
return availableVCores.value();
|
||||||
}
|
}
|
||||||
|
|
||||||
public int getPendingMB() {
|
public long getPendingMB() {
|
||||||
return pendingMB.value();
|
return pendingMB.value();
|
||||||
}
|
}
|
||||||
|
|
||||||
public int getPendingVirtualCores() {
|
public long getPendingVirtualCores() {
|
||||||
return pendingVCores.value();
|
return pendingVCores.value();
|
||||||
}
|
}
|
||||||
|
|
||||||
|
@ -595,11 +596,11 @@ public class QueueMetrics implements MetricsSource {
|
||||||
return pendingContainers.value();
|
return pendingContainers.value();
|
||||||
}
|
}
|
||||||
|
|
||||||
public int getReservedMB() {
|
public long getReservedMB() {
|
||||||
return reservedMB.value();
|
return reservedMB.value();
|
||||||
}
|
}
|
||||||
|
|
||||||
public int getReservedVirtualCores() {
|
public long getReservedVirtualCores() {
|
||||||
return reservedVCores.value();
|
return reservedVCores.value();
|
||||||
}
|
}
|
||||||
|
|
||||||
|
|
|
@ -442,7 +442,7 @@ public class SchedulerApplicationAttempt implements SchedulableEntity {
|
||||||
*/
|
*/
|
||||||
public synchronized Resource getHeadroom() {
|
public synchronized Resource getHeadroom() {
|
||||||
// Corner case to deal with applications being slightly over-limit
|
// Corner case to deal with applications being slightly over-limit
|
||||||
if (resourceLimit.getMemory() < 0) {
|
if (resourceLimit.getMemorySize() < 0) {
|
||||||
resourceLimit.setMemory(0);
|
resourceLimit.setMemory(0);
|
||||||
}
|
}
|
||||||
|
|
||||||
|
@ -478,7 +478,7 @@ public class SchedulerApplicationAttempt implements SchedulableEntity {
|
||||||
if (requests != null) {
|
if (requests != null) {
|
||||||
LOG.debug("showRequests:" + " application=" + getApplicationId()
|
LOG.debug("showRequests:" + " application=" + getApplicationId()
|
||||||
+ " headRoom=" + getHeadroom() + " currentConsumption="
|
+ " headRoom=" + getHeadroom() + " currentConsumption="
|
||||||
+ attemptResourceUsage.getUsed().getMemory());
|
+ attemptResourceUsage.getUsed().getMemorySize());
|
||||||
for (ResourceRequest request : requests.values()) {
|
for (ResourceRequest request : requests.values()) {
|
||||||
LOG.debug("showRequests:" + " application=" + getApplicationId()
|
LOG.debug("showRequests:" + " application=" + getApplicationId()
|
||||||
+ " request=" + request);
|
+ " request=" + request);
|
||||||
|
@ -680,7 +680,7 @@ public class SchedulerApplicationAttempt implements SchedulableEntity {
|
||||||
for (RMContainer rmContainer : this.liveContainers.values()) {
|
for (RMContainer rmContainer : this.liveContainers.values()) {
|
||||||
long usedMillis = currentTimeMillis - rmContainer.getCreationTime();
|
long usedMillis = currentTimeMillis - rmContainer.getCreationTime();
|
||||||
Resource resource = rmContainer.getContainer().getResource();
|
Resource resource = rmContainer.getContainer().getResource();
|
||||||
memorySeconds += resource.getMemory() * usedMillis /
|
memorySeconds += resource.getMemorySize() * usedMillis /
|
||||||
DateUtils.MILLIS_PER_SECOND;
|
DateUtils.MILLIS_PER_SECOND;
|
||||||
vcoreSeconds += resource.getVirtualCores() * usedMillis
|
vcoreSeconds += resource.getVirtualCores() * usedMillis
|
||||||
/ DateUtils.MILLIS_PER_SECOND;
|
/ DateUtils.MILLIS_PER_SECOND;
|
||||||
|
|
|
@ -274,13 +274,13 @@ public class SchedulerUtils {
|
||||||
private static void validateResourceRequest(ResourceRequest resReq,
|
private static void validateResourceRequest(ResourceRequest resReq,
|
||||||
Resource maximumResource, QueueInfo queueInfo, RMContext rmContext)
|
Resource maximumResource, QueueInfo queueInfo, RMContext rmContext)
|
||||||
throws InvalidResourceRequestException {
|
throws InvalidResourceRequestException {
|
||||||
if (resReq.getCapability().getMemory() < 0 ||
|
if (resReq.getCapability().getMemorySize() < 0 ||
|
||||||
resReq.getCapability().getMemory() > maximumResource.getMemory()) {
|
resReq.getCapability().getMemorySize() > maximumResource.getMemorySize()) {
|
||||||
throw new InvalidResourceRequestException("Invalid resource request"
|
throw new InvalidResourceRequestException("Invalid resource request"
|
||||||
+ ", requested memory < 0"
|
+ ", requested memory < 0"
|
||||||
+ ", or requested memory > max configured"
|
+ ", or requested memory > max configured"
|
||||||
+ ", requestedMemory=" + resReq.getCapability().getMemory()
|
+ ", requestedMemory=" + resReq.getCapability().getMemorySize()
|
||||||
+ ", maxMemory=" + maximumResource.getMemory());
|
+ ", maxMemory=" + maximumResource.getMemorySize());
|
||||||
}
|
}
|
||||||
if (resReq.getCapability().getVirtualCores() < 0 ||
|
if (resReq.getCapability().getVirtualCores() < 0 ||
|
||||||
resReq.getCapability().getVirtualCores() >
|
resReq.getCapability().getVirtualCores() >
|
||||||
|
|
|
@ -23,7 +23,7 @@ import org.apache.hadoop.metrics2.MetricsSystem;
|
||||||
import org.apache.hadoop.metrics2.annotation.Metric;
|
import org.apache.hadoop.metrics2.annotation.Metric;
|
||||||
import org.apache.hadoop.metrics2.annotation.Metrics;
|
import org.apache.hadoop.metrics2.annotation.Metrics;
|
||||||
import org.apache.hadoop.metrics2.lib.DefaultMetricsSystem;
|
import org.apache.hadoop.metrics2.lib.DefaultMetricsSystem;
|
||||||
import org.apache.hadoop.metrics2.lib.MutableGaugeInt;
|
import org.apache.hadoop.metrics2.lib.MutableGaugeLong;
|
||||||
import org.apache.hadoop.yarn.api.records.Resource;
|
import org.apache.hadoop.yarn.api.records.Resource;
|
||||||
import org.apache.hadoop.yarn.server.resourcemanager.scheduler.Queue;
|
import org.apache.hadoop.yarn.server.resourcemanager.scheduler.Queue;
|
||||||
import org.apache.hadoop.yarn.server.resourcemanager.scheduler.QueueMetrics;
|
import org.apache.hadoop.yarn.server.resourcemanager.scheduler.QueueMetrics;
|
||||||
|
@ -32,37 +32,37 @@ import org.apache.hadoop.yarn.server.resourcemanager.scheduler.QueueMetrics;
|
||||||
public class CSQueueMetrics extends QueueMetrics {
|
public class CSQueueMetrics extends QueueMetrics {
|
||||||
|
|
||||||
@Metric("AM memory limit in MB")
|
@Metric("AM memory limit in MB")
|
||||||
MutableGaugeInt AMResourceLimitMB;
|
MutableGaugeLong AMResourceLimitMB;
|
||||||
@Metric("AM CPU limit in virtual cores")
|
@Metric("AM CPU limit in virtual cores")
|
||||||
MutableGaugeInt AMResourceLimitVCores;
|
MutableGaugeLong AMResourceLimitVCores;
|
||||||
@Metric("Used AM memory limit in MB")
|
@Metric("Used AM memory limit in MB")
|
||||||
MutableGaugeInt usedAMResourceMB;
|
MutableGaugeLong usedAMResourceMB;
|
||||||
@Metric("Used AM CPU limit in virtual cores")
|
@Metric("Used AM CPU limit in virtual cores")
|
||||||
MutableGaugeInt usedAMResourceVCores;
|
MutableGaugeLong usedAMResourceVCores;
|
||||||
|
|
||||||
CSQueueMetrics(MetricsSystem ms, String queueName, Queue parent,
|
CSQueueMetrics(MetricsSystem ms, String queueName, Queue parent,
|
||||||
boolean enableUserMetrics, Configuration conf) {
|
boolean enableUserMetrics, Configuration conf) {
|
||||||
super(ms, queueName, parent, enableUserMetrics, conf);
|
super(ms, queueName, parent, enableUserMetrics, conf);
|
||||||
}
|
}
|
||||||
|
|
||||||
public int getAMResourceLimitMB() {
|
public long getAMResourceLimitMB() {
|
||||||
return AMResourceLimitMB.value();
|
return AMResourceLimitMB.value();
|
||||||
}
|
}
|
||||||
|
|
||||||
public int getAMResourceLimitVCores() {
|
public long getAMResourceLimitVCores() {
|
||||||
return AMResourceLimitVCores.value();
|
return AMResourceLimitVCores.value();
|
||||||
}
|
}
|
||||||
|
|
||||||
public int getUsedAMResourceMB() {
|
public long getUsedAMResourceMB() {
|
||||||
return usedAMResourceMB.value();
|
return usedAMResourceMB.value();
|
||||||
}
|
}
|
||||||
|
|
||||||
public int getUsedAMResourceVCores() {
|
public long getUsedAMResourceVCores() {
|
||||||
return usedAMResourceVCores.value();
|
return usedAMResourceVCores.value();
|
||||||
}
|
}
|
||||||
|
|
||||||
public void setAMResouceLimit(Resource res) {
|
public void setAMResouceLimit(Resource res) {
|
||||||
AMResourceLimitMB.set(res.getMemory());
|
AMResourceLimitMB.set(res.getMemorySize());
|
||||||
AMResourceLimitVCores.set(res.getVirtualCores());
|
AMResourceLimitVCores.set(res.getVirtualCores());
|
||||||
}
|
}
|
||||||
|
|
||||||
|
@ -74,7 +74,7 @@ public class CSQueueMetrics extends QueueMetrics {
|
||||||
}
|
}
|
||||||
|
|
||||||
public void incAMUsed(String user, Resource res) {
|
public void incAMUsed(String user, Resource res) {
|
||||||
usedAMResourceMB.incr(res.getMemory());
|
usedAMResourceMB.incr(res.getMemorySize());
|
||||||
usedAMResourceVCores.incr(res.getVirtualCores());
|
usedAMResourceVCores.incr(res.getVirtualCores());
|
||||||
CSQueueMetrics userMetrics = (CSQueueMetrics) getUserMetrics(user);
|
CSQueueMetrics userMetrics = (CSQueueMetrics) getUserMetrics(user);
|
||||||
if (userMetrics != null) {
|
if (userMetrics != null) {
|
||||||
|
@ -83,7 +83,7 @@ public class CSQueueMetrics extends QueueMetrics {
|
||||||
}
|
}
|
||||||
|
|
||||||
public void decAMUsed(String user, Resource res) {
|
public void decAMUsed(String user, Resource res) {
|
||||||
usedAMResourceMB.decr(res.getMemory());
|
usedAMResourceMB.decr(res.getMemorySize());
|
||||||
usedAMResourceVCores.decr(res.getVirtualCores());
|
usedAMResourceVCores.decr(res.getVirtualCores());
|
||||||
CSQueueMetrics userMetrics = (CSQueueMetrics) getUserMetrics(user);
|
CSQueueMetrics userMetrics = (CSQueueMetrics) getUserMetrics(user);
|
||||||
if (userMetrics != null) {
|
if (userMetrics != null) {
|
||||||
|
|
|
@ -65,7 +65,7 @@ public class CapacityHeadroomProvider {
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
// Corner case to deal with applications being slightly over-limit
|
// Corner case to deal with applications being slightly over-limit
|
||||||
if (headroom.getMemory() < 0) {
|
if (headroom.getMemorySize() < 0) {
|
||||||
headroom.setMemory(0);
|
headroom.setMemory(0);
|
||||||
}
|
}
|
||||||
return headroom;
|
return headroom;
|
||||||
|
|
|
@ -671,7 +671,7 @@ public class CapacitySchedulerConfiguration extends ReservationSchedulerConfigur
|
||||||
*/
|
*/
|
||||||
public Resource getMaximumAllocationPerQueue(String queue) {
|
public Resource getMaximumAllocationPerQueue(String queue) {
|
||||||
String queuePrefix = getQueuePrefix(queue);
|
String queuePrefix = getQueuePrefix(queue);
|
||||||
int maxAllocationMbPerQueue = getInt(queuePrefix + MAXIMUM_ALLOCATION_MB,
|
long maxAllocationMbPerQueue = getInt(queuePrefix + MAXIMUM_ALLOCATION_MB,
|
||||||
(int)UNDEFINED);
|
(int)UNDEFINED);
|
||||||
int maxAllocationVcoresPerQueue = getInt(
|
int maxAllocationVcoresPerQueue = getInt(
|
||||||
queuePrefix + MAXIMUM_ALLOCATION_VCORES, (int)UNDEFINED);
|
queuePrefix + MAXIMUM_ALLOCATION_VCORES, (int)UNDEFINED);
|
||||||
|
@ -684,7 +684,7 @@ public class CapacitySchedulerConfiguration extends ReservationSchedulerConfigur
|
||||||
Resource clusterMax = getMaximumAllocation();
|
Resource clusterMax = getMaximumAllocation();
|
||||||
if (maxAllocationMbPerQueue == (int)UNDEFINED) {
|
if (maxAllocationMbPerQueue == (int)UNDEFINED) {
|
||||||
LOG.info("max alloc mb per queue for " + queue + " is undefined");
|
LOG.info("max alloc mb per queue for " + queue + " is undefined");
|
||||||
maxAllocationMbPerQueue = clusterMax.getMemory();
|
maxAllocationMbPerQueue = clusterMax.getMemorySize();
|
||||||
}
|
}
|
||||||
if (maxAllocationVcoresPerQueue == (int)UNDEFINED) {
|
if (maxAllocationVcoresPerQueue == (int)UNDEFINED) {
|
||||||
LOG.info("max alloc vcore per queue for " + queue + " is undefined");
|
LOG.info("max alloc vcore per queue for " + queue + " is undefined");
|
||||||
|
@ -692,7 +692,7 @@ public class CapacitySchedulerConfiguration extends ReservationSchedulerConfigur
|
||||||
}
|
}
|
||||||
Resource result = Resources.createResource(maxAllocationMbPerQueue,
|
Resource result = Resources.createResource(maxAllocationMbPerQueue,
|
||||||
maxAllocationVcoresPerQueue);
|
maxAllocationVcoresPerQueue);
|
||||||
if (maxAllocationMbPerQueue > clusterMax.getMemory()
|
if (maxAllocationMbPerQueue > clusterMax.getMemorySize()
|
||||||
|| maxAllocationVcoresPerQueue > clusterMax.getVirtualCores()) {
|
|| maxAllocationVcoresPerQueue > clusterMax.getVirtualCores()) {
|
||||||
throw new IllegalArgumentException(
|
throw new IllegalArgumentException(
|
||||||
"Queue maximum allocation cannot be larger than the cluster setting"
|
"Queue maximum allocation cannot be larger than the cluster setting"
|
||||||
|
|
|
@ -445,7 +445,7 @@ public class LeafQueue extends AbstractCSQueue {
|
||||||
// since we have already told running AM's the size
|
// since we have already told running AM's the size
|
||||||
Resource oldMax = getMaximumAllocation();
|
Resource oldMax = getMaximumAllocation();
|
||||||
Resource newMax = newlyParsedLeafQueue.getMaximumAllocation();
|
Resource newMax = newlyParsedLeafQueue.getMaximumAllocation();
|
||||||
if (newMax.getMemory() < oldMax.getMemory()
|
if (newMax.getMemorySize() < oldMax.getMemorySize()
|
||||||
|| newMax.getVirtualCores() < oldMax.getVirtualCores()) {
|
|| newMax.getVirtualCores() < oldMax.getVirtualCores()) {
|
||||||
throw new IOException(
|
throw new IOException(
|
||||||
"Trying to reinitialize "
|
"Trying to reinitialize "
|
||||||
|
|
|
@ -442,7 +442,7 @@ public class RegularContainerAllocator extends AbstractContainerAllocator {
|
||||||
priority, capability);
|
priority, capability);
|
||||||
|
|
||||||
// Can we allocate a container on this node?
|
// Can we allocate a container on this node?
|
||||||
int availableContainers =
|
long availableContainers =
|
||||||
rc.computeAvailableContainers(available, capability);
|
rc.computeAvailableContainers(available, capability);
|
||||||
|
|
||||||
// How much need to unreserve equals to:
|
// How much need to unreserve equals to:
|
||||||
|
|
|
@ -128,7 +128,7 @@ public class FSParentQueue extends FSQueue {
|
||||||
public Resource getDemand() {
|
public Resource getDemand() {
|
||||||
readLock.lock();
|
readLock.lock();
|
||||||
try {
|
try {
|
||||||
return Resource.newInstance(demand.getMemory(), demand.getVirtualCores());
|
return Resource.newInstance(demand.getMemorySize(), demand.getVirtualCores());
|
||||||
} finally {
|
} finally {
|
||||||
readLock.unlock();
|
readLock.unlock();
|
||||||
}
|
}
|
||||||
|
|
|
@ -130,18 +130,18 @@ public abstract class FSQueue implements Queue, Schedulable {
|
||||||
QueueInfo queueInfo = recordFactory.newRecordInstance(QueueInfo.class);
|
QueueInfo queueInfo = recordFactory.newRecordInstance(QueueInfo.class);
|
||||||
queueInfo.setQueueName(getQueueName());
|
queueInfo.setQueueName(getQueueName());
|
||||||
|
|
||||||
if (scheduler.getClusterResource().getMemory() == 0) {
|
if (scheduler.getClusterResource().getMemorySize() == 0) {
|
||||||
queueInfo.setCapacity(0.0f);
|
queueInfo.setCapacity(0.0f);
|
||||||
} else {
|
} else {
|
||||||
queueInfo.setCapacity((float) getFairShare().getMemory() /
|
queueInfo.setCapacity((float) getFairShare().getMemorySize() /
|
||||||
scheduler.getClusterResource().getMemory());
|
scheduler.getClusterResource().getMemorySize());
|
||||||
}
|
}
|
||||||
|
|
||||||
if (getFairShare().getMemory() == 0) {
|
if (getFairShare().getMemorySize() == 0) {
|
||||||
queueInfo.setCurrentCapacity(0.0f);
|
queueInfo.setCurrentCapacity(0.0f);
|
||||||
} else {
|
} else {
|
||||||
queueInfo.setCurrentCapacity((float) getResourceUsage().getMemory() /
|
queueInfo.setCurrentCapacity((float) getResourceUsage().getMemorySize() /
|
||||||
getFairShare().getMemory());
|
getFairShare().getMemorySize());
|
||||||
}
|
}
|
||||||
|
|
||||||
ArrayList<QueueInfo> childQueueInfos = new ArrayList<QueueInfo>();
|
ArrayList<QueueInfo> childQueueInfos = new ArrayList<QueueInfo>();
|
||||||
|
|
|
@ -24,6 +24,7 @@ import org.apache.hadoop.metrics2.annotation.Metric;
|
||||||
import org.apache.hadoop.metrics2.annotation.Metrics;
|
import org.apache.hadoop.metrics2.annotation.Metrics;
|
||||||
import org.apache.hadoop.metrics2.lib.DefaultMetricsSystem;
|
import org.apache.hadoop.metrics2.lib.DefaultMetricsSystem;
|
||||||
import org.apache.hadoop.metrics2.lib.MutableGaugeInt;
|
import org.apache.hadoop.metrics2.lib.MutableGaugeInt;
|
||||||
|
import org.apache.hadoop.metrics2.lib.MutableGaugeLong;
|
||||||
import org.apache.hadoop.yarn.api.records.Resource;
|
import org.apache.hadoop.yarn.api.records.Resource;
|
||||||
import org.apache.hadoop.yarn.server.resourcemanager.scheduler.Queue;
|
import org.apache.hadoop.yarn.server.resourcemanager.scheduler.Queue;
|
||||||
import org.apache.hadoop.yarn.server.resourcemanager.scheduler.QueueMetrics;
|
import org.apache.hadoop.yarn.server.resourcemanager.scheduler.QueueMetrics;
|
||||||
|
@ -31,69 +32,69 @@ import org.apache.hadoop.yarn.server.resourcemanager.scheduler.QueueMetrics;
|
||||||
@Metrics(context="yarn")
|
@Metrics(context="yarn")
|
||||||
public class FSQueueMetrics extends QueueMetrics {
|
public class FSQueueMetrics extends QueueMetrics {
|
||||||
|
|
||||||
@Metric("Fair share of memory in MB") MutableGaugeInt fairShareMB;
|
@Metric("Fair share of memory in MB") MutableGaugeLong fairShareMB;
|
||||||
@Metric("Fair share of CPU in vcores") MutableGaugeInt fairShareVCores;
|
@Metric("Fair share of CPU in vcores") MutableGaugeLong fairShareVCores;
|
||||||
@Metric("Steady fair share of memory in MB") MutableGaugeInt steadyFairShareMB;
|
@Metric("Steady fair share of memory in MB") MutableGaugeLong steadyFairShareMB;
|
||||||
@Metric("Steady fair share of CPU in vcores") MutableGaugeInt steadyFairShareVCores;
|
@Metric("Steady fair share of CPU in vcores") MutableGaugeLong steadyFairShareVCores;
|
||||||
@Metric("Minimum share of memory in MB") MutableGaugeInt minShareMB;
|
@Metric("Minimum share of memory in MB") MutableGaugeLong minShareMB;
|
||||||
@Metric("Minimum share of CPU in vcores") MutableGaugeInt minShareVCores;
|
@Metric("Minimum share of CPU in vcores") MutableGaugeLong minShareVCores;
|
||||||
@Metric("Maximum share of memory in MB") MutableGaugeInt maxShareMB;
|
@Metric("Maximum share of memory in MB") MutableGaugeLong maxShareMB;
|
||||||
@Metric("Maximum share of CPU in vcores") MutableGaugeInt maxShareVCores;
|
@Metric("Maximum share of CPU in vcores") MutableGaugeLong maxShareVCores;
|
||||||
|
|
||||||
FSQueueMetrics(MetricsSystem ms, String queueName, Queue parent,
|
FSQueueMetrics(MetricsSystem ms, String queueName, Queue parent,
|
||||||
boolean enableUserMetrics, Configuration conf) {
|
boolean enableUserMetrics, Configuration conf) {
|
||||||
super(ms, queueName, parent, enableUserMetrics, conf);
|
super(ms, queueName, parent, enableUserMetrics, conf);
|
||||||
}
|
}
|
||||||
|
|
||||||
public void setFairShare(Resource resource) {
|
public void setFairShare(Resource resource) {
|
||||||
fairShareMB.set(resource.getMemory());
|
fairShareMB.set(resource.getMemorySize());
|
||||||
fairShareVCores.set(resource.getVirtualCores());
|
fairShareVCores.set(resource.getVirtualCores());
|
||||||
}
|
}
|
||||||
|
|
||||||
public int getFairShareMB() {
|
public long getFairShareMB() {
|
||||||
return fairShareMB.value();
|
return fairShareMB.value();
|
||||||
}
|
}
|
||||||
|
|
||||||
public int getFairShareVirtualCores() {
|
public long getFairShareVirtualCores() {
|
||||||
return fairShareVCores.value();
|
return fairShareVCores.value();
|
||||||
}
|
}
|
||||||
|
|
||||||
public void setSteadyFairShare(Resource resource) {
|
public void setSteadyFairShare(Resource resource) {
|
||||||
steadyFairShareMB.set(resource.getMemory());
|
steadyFairShareMB.set(resource.getMemorySize());
|
||||||
steadyFairShareVCores.set(resource.getVirtualCores());
|
steadyFairShareVCores.set(resource.getVirtualCores());
|
||||||
}
|
}
|
||||||
|
|
||||||
public int getSteadyFairShareMB() {
|
public long getSteadyFairShareMB() {
|
||||||
return steadyFairShareMB.value();
|
return steadyFairShareMB.value();
|
||||||
}
|
}
|
||||||
|
|
||||||
public int getSteadyFairShareVCores() {
|
public long getSteadyFairShareVCores() {
|
||||||
return steadyFairShareVCores.value();
|
return steadyFairShareVCores.value();
|
||||||
}
|
}
|
||||||
|
|
||||||
public void setMinShare(Resource resource) {
|
public void setMinShare(Resource resource) {
|
||||||
minShareMB.set(resource.getMemory());
|
minShareMB.set(resource.getMemorySize());
|
||||||
minShareVCores.set(resource.getVirtualCores());
|
minShareVCores.set(resource.getVirtualCores());
|
||||||
}
|
}
|
||||||
|
|
||||||
public int getMinShareMB() {
|
public long getMinShareMB() {
|
||||||
return minShareMB.value();
|
return minShareMB.value();
|
||||||
}
|
}
|
||||||
|
|
||||||
public int getMinShareVirtualCores() {
|
public long getMinShareVirtualCores() {
|
||||||
return minShareVCores.value();
|
return minShareVCores.value();
|
||||||
}
|
}
|
||||||
|
|
||||||
public void setMaxShare(Resource resource) {
|
public void setMaxShare(Resource resource) {
|
||||||
maxShareMB.set(resource.getMemory());
|
maxShareMB.set(resource.getMemorySize());
|
||||||
maxShareVCores.set(resource.getVirtualCores());
|
maxShareVCores.set(resource.getVirtualCores());
|
||||||
}
|
}
|
||||||
|
|
||||||
public int getMaxShareMB() {
|
public long getMaxShareMB() {
|
||||||
return maxShareMB.value();
|
return maxShareMB.value();
|
||||||
}
|
}
|
||||||
|
|
||||||
public int getMaxShareVirtualCores() {
|
public long getMaxShareVirtualCores() {
|
||||||
return maxShareVCores.value();
|
return maxShareVCores.value();
|
||||||
}
|
}
|
||||||
|
|
||||||
|
|
|
@ -478,7 +478,7 @@ public class FairScheduler extends
|
||||||
}
|
}
|
||||||
|
|
||||||
private boolean isResourceGreaterThanNone(Resource toPreempt) {
|
private boolean isResourceGreaterThanNone(Resource toPreempt) {
|
||||||
return (toPreempt.getMemory() > 0) || (toPreempt.getVirtualCores() > 0);
|
return (toPreempt.getMemorySize() > 0) || (toPreempt.getVirtualCores() > 0);
|
||||||
}
|
}
|
||||||
|
|
||||||
protected void warnOrKillContainer(RMContainer container) {
|
protected void warnOrKillContainer(RMContainer container) {
|
||||||
|
@ -562,7 +562,7 @@ public class FairScheduler extends
|
||||||
double weight = 1.0;
|
double weight = 1.0;
|
||||||
if (sizeBasedWeight) {
|
if (sizeBasedWeight) {
|
||||||
// Set weight based on current memory demand
|
// Set weight based on current memory demand
|
||||||
weight = Math.log1p(app.getDemand().getMemory()) / Math.log(2);
|
weight = Math.log1p(app.getDemand().getMemorySize()) / Math.log(2);
|
||||||
}
|
}
|
||||||
weight *= app.getPriority().getPriority();
|
weight *= app.getPriority().getPriority();
|
||||||
if (weightAdjuster != null) {
|
if (weightAdjuster != null) {
|
||||||
|
@ -1238,7 +1238,7 @@ public class FairScheduler extends
|
||||||
private boolean shouldAttemptPreemption() {
|
private boolean shouldAttemptPreemption() {
|
||||||
if (preemptionEnabled) {
|
if (preemptionEnabled) {
|
||||||
return (preemptionUtilizationThreshold < Math.max(
|
return (preemptionUtilizationThreshold < Math.max(
|
||||||
(float) rootMetrics.getAllocatedMB() / clusterResource.getMemory(),
|
(float) rootMetrics.getAllocatedMB() / clusterResource.getMemorySize(),
|
||||||
(float) rootMetrics.getAllocatedVirtualCores() /
|
(float) rootMetrics.getAllocatedVirtualCores() /
|
||||||
clusterResource.getVirtualCores()));
|
clusterResource.getVirtualCores()));
|
||||||
}
|
}
|
||||||
|
|
|
@ -124,15 +124,15 @@ public class ComputeFairShares {
|
||||||
// have met all Schedulables' max shares.
|
// have met all Schedulables' max shares.
|
||||||
int totalMaxShare = 0;
|
int totalMaxShare = 0;
|
||||||
for (Schedulable sched : schedulables) {
|
for (Schedulable sched : schedulables) {
|
||||||
int maxShare = getResourceValue(sched.getMaxShare(), type);
|
long maxShare = getResourceValue(sched.getMaxShare(), type);
|
||||||
totalMaxShare = (int) Math.min((long)maxShare + (long)totalMaxShare,
|
totalMaxShare = (int) Math.min(maxShare + (long)totalMaxShare,
|
||||||
Integer.MAX_VALUE);
|
Integer.MAX_VALUE);
|
||||||
if (totalMaxShare == Integer.MAX_VALUE) {
|
if (totalMaxShare == Integer.MAX_VALUE) {
|
||||||
break;
|
break;
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
|
|
||||||
int totalResource = Math.max((getResourceValue(totalResources, type) -
|
long totalResource = Math.max((getResourceValue(totalResources, type) -
|
||||||
takenResources), 0);
|
takenResources), 0);
|
||||||
totalResource = Math.min(totalMaxShare, totalResource);
|
totalResource = Math.min(totalMaxShare, totalResource);
|
||||||
|
|
||||||
|
@ -207,7 +207,7 @@ public class ComputeFairShares {
|
||||||
int totalResource = 0;
|
int totalResource = 0;
|
||||||
|
|
||||||
for (Schedulable sched : schedulables) {
|
for (Schedulable sched : schedulables) {
|
||||||
int fixedShare = getFairShareIfFixed(sched, isSteadyShare, type);
|
long fixedShare = getFairShareIfFixed(sched, isSteadyShare, type);
|
||||||
if (fixedShare < 0) {
|
if (fixedShare < 0) {
|
||||||
nonFixedSchedulables.add(sched);
|
nonFixedSchedulables.add(sched);
|
||||||
} else {
|
} else {
|
||||||
|
@ -229,7 +229,7 @@ public class ComputeFairShares {
|
||||||
* The fairshare is fixed if either the maxShare is 0, weight is 0,
|
* The fairshare is fixed if either the maxShare is 0, weight is 0,
|
||||||
* or the Schedulable is not active for instantaneous fairshare.
|
* or the Schedulable is not active for instantaneous fairshare.
|
||||||
*/
|
*/
|
||||||
private static int getFairShareIfFixed(Schedulable sched,
|
private static long getFairShareIfFixed(Schedulable sched,
|
||||||
boolean isSteadyShare, ResourceType type) {
|
boolean isSteadyShare, ResourceType type) {
|
||||||
|
|
||||||
// Check if maxShare is 0
|
// Check if maxShare is 0
|
||||||
|
@ -245,17 +245,17 @@ public class ComputeFairShares {
|
||||||
|
|
||||||
// Check if weight is 0
|
// Check if weight is 0
|
||||||
if (sched.getWeights().getWeight(type) <= 0) {
|
if (sched.getWeights().getWeight(type) <= 0) {
|
||||||
int minShare = getResourceValue(sched.getMinShare(), type);
|
long minShare = getResourceValue(sched.getMinShare(), type);
|
||||||
return (minShare <= 0) ? 0 : minShare;
|
return (minShare <= 0) ? 0 : minShare;
|
||||||
}
|
}
|
||||||
|
|
||||||
return -1;
|
return -1;
|
||||||
}
|
}
|
||||||
|
|
||||||
private static int getResourceValue(Resource resource, ResourceType type) {
|
private static long getResourceValue(Resource resource, ResourceType type) {
|
||||||
switch (type) {
|
switch (type) {
|
||||||
case MEMORY:
|
case MEMORY:
|
||||||
return resource.getMemory();
|
return resource.getMemorySize();
|
||||||
case CPU:
|
case CPU:
|
||||||
return resource.getVirtualCores();
|
return resource.getVirtualCores();
|
||||||
default:
|
default:
|
||||||
|
@ -263,7 +263,7 @@ public class ComputeFairShares {
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
|
|
||||||
private static void setResourceValue(int val, Resource resource, ResourceType type) {
|
private static void setResourceValue(long val, Resource resource, ResourceType type) {
|
||||||
switch (type) {
|
switch (type) {
|
||||||
case MEMORY:
|
case MEMORY:
|
||||||
resource.setMemory(val);
|
resource.setMemory(val);
|
||||||
|
|
|
@ -101,13 +101,13 @@ public class DominantResourceFairnessPolicy extends SchedulingPolicy {
|
||||||
@Override
|
@Override
|
||||||
public Resource getHeadroom(Resource queueFairShare, Resource queueUsage,
|
public Resource getHeadroom(Resource queueFairShare, Resource queueUsage,
|
||||||
Resource maxAvailable) {
|
Resource maxAvailable) {
|
||||||
int queueAvailableMemory =
|
long queueAvailableMemory =
|
||||||
Math.max(queueFairShare.getMemory() - queueUsage.getMemory(), 0);
|
Math.max(queueFairShare.getMemorySize() - queueUsage.getMemorySize(), 0);
|
||||||
int queueAvailableCPU =
|
int queueAvailableCPU =
|
||||||
Math.max(queueFairShare.getVirtualCores() - queueUsage
|
Math.max(queueFairShare.getVirtualCores() - queueUsage
|
||||||
.getVirtualCores(), 0);
|
.getVirtualCores(), 0);
|
||||||
Resource headroom = Resources.createResource(
|
Resource headroom = Resources.createResource(
|
||||||
Math.min(maxAvailable.getMemory(), queueAvailableMemory),
|
Math.min(maxAvailable.getMemorySize(), queueAvailableMemory),
|
||||||
Math.min(maxAvailable.getVirtualCores(),
|
Math.min(maxAvailable.getVirtualCores(),
|
||||||
queueAvailableCPU));
|
queueAvailableCPU));
|
||||||
return headroom;
|
return headroom;
|
||||||
|
@ -180,8 +180,8 @@ public class DominantResourceFairnessPolicy extends SchedulingPolicy {
|
||||||
*/
|
*/
|
||||||
void calculateShares(Resource resource, Resource pool,
|
void calculateShares(Resource resource, Resource pool,
|
||||||
ResourceWeights shares, ResourceType[] resourceOrder, ResourceWeights weights) {
|
ResourceWeights shares, ResourceType[] resourceOrder, ResourceWeights weights) {
|
||||||
shares.setWeight(MEMORY, (float)resource.getMemory() /
|
shares.setWeight(MEMORY, (float)resource.getMemorySize() /
|
||||||
(pool.getMemory() * weights.getWeight(MEMORY)));
|
(pool.getMemorySize() * weights.getWeight(MEMORY)));
|
||||||
shares.setWeight(CPU, (float)resource.getVirtualCores() /
|
shares.setWeight(CPU, (float)resource.getVirtualCores() /
|
||||||
(pool.getVirtualCores() * weights.getWeight(CPU)));
|
(pool.getVirtualCores() * weights.getWeight(CPU)));
|
||||||
// sort order vector by resource share
|
// sort order vector by resource share
|
||||||
|
|
|
@ -82,13 +82,13 @@ public class FairSharePolicy extends SchedulingPolicy {
|
||||||
s1.getResourceUsage(), minShare1);
|
s1.getResourceUsage(), minShare1);
|
||||||
boolean s2Needy = Resources.lessThan(RESOURCE_CALCULATOR, null,
|
boolean s2Needy = Resources.lessThan(RESOURCE_CALCULATOR, null,
|
||||||
s2.getResourceUsage(), minShare2);
|
s2.getResourceUsage(), minShare2);
|
||||||
minShareRatio1 = (double) s1.getResourceUsage().getMemory()
|
minShareRatio1 = (double) s1.getResourceUsage().getMemorySize()
|
||||||
/ Resources.max(RESOURCE_CALCULATOR, null, minShare1, ONE).getMemory();
|
/ Resources.max(RESOURCE_CALCULATOR, null, minShare1, ONE).getMemorySize();
|
||||||
minShareRatio2 = (double) s2.getResourceUsage().getMemory()
|
minShareRatio2 = (double) s2.getResourceUsage().getMemorySize()
|
||||||
/ Resources.max(RESOURCE_CALCULATOR, null, minShare2, ONE).getMemory();
|
/ Resources.max(RESOURCE_CALCULATOR, null, minShare2, ONE).getMemorySize();
|
||||||
useToWeightRatio1 = s1.getResourceUsage().getMemory() /
|
useToWeightRatio1 = s1.getResourceUsage().getMemorySize() /
|
||||||
s1.getWeights().getWeight(ResourceType.MEMORY);
|
s1.getWeights().getWeight(ResourceType.MEMORY);
|
||||||
useToWeightRatio2 = s2.getResourceUsage().getMemory() /
|
useToWeightRatio2 = s2.getResourceUsage().getMemorySize() /
|
||||||
s2.getWeights().getWeight(ResourceType.MEMORY);
|
s2.getWeights().getWeight(ResourceType.MEMORY);
|
||||||
int res = 0;
|
int res = 0;
|
||||||
if (s1Needy && !s2Needy)
|
if (s1Needy && !s2Needy)
|
||||||
|
@ -124,10 +124,10 @@ public class FairSharePolicy extends SchedulingPolicy {
|
||||||
@Override
|
@Override
|
||||||
public Resource getHeadroom(Resource queueFairShare,
|
public Resource getHeadroom(Resource queueFairShare,
|
||||||
Resource queueUsage, Resource maxAvailable) {
|
Resource queueUsage, Resource maxAvailable) {
|
||||||
int queueAvailableMemory = Math.max(
|
long queueAvailableMemory = Math.max(
|
||||||
queueFairShare.getMemory() - queueUsage.getMemory(), 0);
|
queueFairShare.getMemorySize() - queueUsage.getMemorySize(), 0);
|
||||||
Resource headroom = Resources.createResource(
|
Resource headroom = Resources.createResource(
|
||||||
Math.min(maxAvailable.getMemory(), queueAvailableMemory),
|
Math.min(maxAvailable.getMemorySize(), queueAvailableMemory),
|
||||||
maxAvailable.getVirtualCores());
|
maxAvailable.getVirtualCores());
|
||||||
return headroom;
|
return headroom;
|
||||||
}
|
}
|
||||||
|
@ -152,7 +152,7 @@ public class FairSharePolicy extends SchedulingPolicy {
|
||||||
|
|
||||||
@Override
|
@Override
|
||||||
public boolean checkIfAMResourceUsageOverLimit(Resource usage, Resource maxAMResource) {
|
public boolean checkIfAMResourceUsageOverLimit(Resource usage, Resource maxAMResource) {
|
||||||
return usage.getMemory() > maxAMResource.getMemory();
|
return usage.getMemorySize() > maxAMResource.getMemorySize();
|
||||||
}
|
}
|
||||||
|
|
||||||
@Override
|
@Override
|
||||||
|
|
|
@ -115,16 +115,16 @@ public class FifoPolicy extends SchedulingPolicy {
|
||||||
|
|
||||||
@Override
|
@Override
|
||||||
public boolean checkIfAMResourceUsageOverLimit(Resource usage, Resource maxAMResource) {
|
public boolean checkIfAMResourceUsageOverLimit(Resource usage, Resource maxAMResource) {
|
||||||
return usage.getMemory() > maxAMResource.getMemory();
|
return usage.getMemorySize() > maxAMResource.getMemorySize();
|
||||||
}
|
}
|
||||||
|
|
||||||
@Override
|
@Override
|
||||||
public Resource getHeadroom(Resource queueFairShare,
|
public Resource getHeadroom(Resource queueFairShare,
|
||||||
Resource queueUsage, Resource maxAvailable) {
|
Resource queueUsage, Resource maxAvailable) {
|
||||||
int queueAvailableMemory = Math.max(
|
long queueAvailableMemory = Math.max(
|
||||||
queueFairShare.getMemory() - queueUsage.getMemory(), 0);
|
queueFairShare.getMemorySize() - queueUsage.getMemorySize(), 0);
|
||||||
Resource headroom = Resources.createResource(
|
Resource headroom = Resources.createResource(
|
||||||
Math.min(maxAvailable.getMemory(), queueAvailableMemory),
|
Math.min(maxAvailable.getMemorySize(), queueAvailableMemory),
|
||||||
maxAvailable.getVirtualCores());
|
maxAvailable.getVirtualCores());
|
||||||
return headroom;
|
return headroom;
|
||||||
}
|
}
|
||||||
|
|
|
@ -142,11 +142,11 @@ public class FifoScheduler extends
|
||||||
QueueInfo queueInfo = recordFactory.newRecordInstance(QueueInfo.class);
|
QueueInfo queueInfo = recordFactory.newRecordInstance(QueueInfo.class);
|
||||||
queueInfo.setQueueName(DEFAULT_QUEUE.getQueueName());
|
queueInfo.setQueueName(DEFAULT_QUEUE.getQueueName());
|
||||||
queueInfo.setCapacity(1.0f);
|
queueInfo.setCapacity(1.0f);
|
||||||
if (clusterResource.getMemory() == 0) {
|
if (clusterResource.getMemorySize() == 0) {
|
||||||
queueInfo.setCurrentCapacity(0.0f);
|
queueInfo.setCurrentCapacity(0.0f);
|
||||||
} else {
|
} else {
|
||||||
queueInfo.setCurrentCapacity((float) usedResource.getMemory()
|
queueInfo.setCurrentCapacity((float) usedResource.getMemorySize()
|
||||||
/ clusterResource.getMemory());
|
/ clusterResource.getMemorySize());
|
||||||
}
|
}
|
||||||
queueInfo.setMaximumCapacity(1.0f);
|
queueInfo.setMaximumCapacity(1.0f);
|
||||||
queueInfo.setChildQueues(new ArrayList<QueueInfo>());
|
queueInfo.setChildQueues(new ArrayList<QueueInfo>());
|
||||||
|
@ -671,7 +671,7 @@ public class FifoScheduler extends
|
||||||
return assignedContainers;
|
return assignedContainers;
|
||||||
}
|
}
|
||||||
|
|
||||||
private int assignContainer(FiCaSchedulerNode node, FiCaSchedulerApp application,
|
private int assignContainer(FiCaSchedulerNode node, FiCaSchedulerApp application,
|
||||||
Priority priority, int assignableContainers,
|
Priority priority, int assignableContainers,
|
||||||
ResourceRequest request, NodeType type) {
|
ResourceRequest request, NodeType type) {
|
||||||
LOG.debug("assignContainers:" +
|
LOG.debug("assignContainers:" +
|
||||||
|
@ -682,14 +682,11 @@ public class FifoScheduler extends
|
||||||
" request=" + request + " type=" + type);
|
" request=" + request + " type=" + type);
|
||||||
Resource capability = request.getCapability();
|
Resource capability = request.getCapability();
|
||||||
|
|
||||||
int availableContainers =
|
int availableContainers =
|
||||||
node.getAvailableResource().getMemory() / capability.getMemory(); // TODO: A buggy
|
(int) (node.getAvailableResource().getMemorySize() / capability
|
||||||
// application
|
.getMemorySize());
|
||||||
// with this
|
// TODO: A buggy application with this zero would crash the scheduler.
|
||||||
// zero would
|
int assignedContainers =
|
||||||
// crash the
|
|
||||||
// scheduler.
|
|
||||||
int assignedContainers =
|
|
||||||
Math.min(assignableContainers, availableContainers);
|
Math.min(assignableContainers, availableContainers);
|
||||||
|
|
||||||
if (assignedContainers > 0) {
|
if (assignedContainers > 0) {
|
||||||
|
|
|
@ -23,7 +23,6 @@ import java.util.*;
|
||||||
import com.google.common.annotations.VisibleForTesting;
|
import com.google.common.annotations.VisibleForTesting;
|
||||||
|
|
||||||
import org.apache.hadoop.yarn.server.resourcemanager.rmcontainer.RMContainer;
|
import org.apache.hadoop.yarn.server.resourcemanager.rmcontainer.RMContainer;
|
||||||
import org.apache.hadoop.yarn.server.resourcemanager.scheduler.*;
|
|
||||||
import org.apache.hadoop.yarn.nodelabels.CommonNodeLabelsManager;
|
import org.apache.hadoop.yarn.nodelabels.CommonNodeLabelsManager;
|
||||||
|
|
||||||
/**
|
/**
|
||||||
|
@ -67,10 +66,10 @@ public class FairOrderingPolicy<S extends SchedulableEntity> extends AbstractCom
|
||||||
|
|
||||||
private double getMagnitude(SchedulableEntity r) {
|
private double getMagnitude(SchedulableEntity r) {
|
||||||
double mag = r.getSchedulingResourceUsage().getCachedUsed(
|
double mag = r.getSchedulingResourceUsage().getCachedUsed(
|
||||||
CommonNodeLabelsManager.ANY).getMemory();
|
CommonNodeLabelsManager.ANY).getMemorySize();
|
||||||
if (sizeBasedWeight) {
|
if (sizeBasedWeight) {
|
||||||
double weight = Math.log1p(r.getSchedulingResourceUsage().getCachedDemand(
|
double weight = Math.log1p(r.getSchedulingResourceUsage().getCachedDemand(
|
||||||
CommonNodeLabelsManager.ANY).getMemory()) / Math.log(2);
|
CommonNodeLabelsManager.ANY).getMemorySize()) / Math.log(2);
|
||||||
mag = mag / weight;
|
mag = mag / weight;
|
||||||
}
|
}
|
||||||
return mag;
|
return mag;
|
||||||
|
|
|
@ -54,8 +54,8 @@ class DefaultSchedulerPage extends RmView {
|
||||||
@Override public void render(Block html) {
|
@Override public void render(Block html) {
|
||||||
info("\'" + sinfo.getQueueName() + "\' Queue Status").
|
info("\'" + sinfo.getQueueName() + "\' Queue Status").
|
||||||
_("Queue State:" , sinfo.getState()).
|
_("Queue State:" , sinfo.getState()).
|
||||||
_("Minimum Queue Memory Capacity:" , Integer.toString(sinfo.getMinQueueMemoryCapacity())).
|
_("Minimum Queue Memory Capacity:" , Long.toString(sinfo.getMinQueueMemoryCapacity())).
|
||||||
_("Maximum Queue Memory Capacity:" , Integer.toString(sinfo.getMaxQueueMemoryCapacity())).
|
_("Maximum Queue Memory Capacity:" , Long.toString(sinfo.getMaxQueueMemoryCapacity())).
|
||||||
_("Number of Nodes:" , Integer.toString(sinfo.getNumNodes())).
|
_("Number of Nodes:" , Integer.toString(sinfo.getNumNodes())).
|
||||||
_("Used Node Capacity:" , Integer.toString(sinfo.getUsedNodeCapacity())).
|
_("Used Node Capacity:" , Integer.toString(sinfo.getUsedNodeCapacity())).
|
||||||
_("Available Node Capacity:" , Integer.toString(sinfo.getAvailNodeCapacity())).
|
_("Available Node Capacity:" , Integer.toString(sinfo.getAvailNodeCapacity())).
|
||||||
|
|
|
@ -115,7 +115,7 @@ public class FairSchedulerAppsBlock extends HtmlBlock {
|
||||||
AppInfo appInfo = new AppInfo(rm, app, true, WebAppUtils.getHttpSchemePrefix(conf));
|
AppInfo appInfo = new AppInfo(rm, app, true, WebAppUtils.getHttpSchemePrefix(conf));
|
||||||
String percent = StringUtils.format("%.1f", appInfo.getProgress());
|
String percent = StringUtils.format("%.1f", appInfo.getProgress());
|
||||||
ApplicationAttemptId attemptId = app.getCurrentAppAttempt().getAppAttemptId();
|
ApplicationAttemptId attemptId = app.getCurrentAppAttempt().getAppAttemptId();
|
||||||
int fairShare = fsinfo.getAppFairShare(attemptId);
|
long fairShare = fsinfo.getAppFairShare(attemptId);
|
||||||
if (fairShare == FairSchedulerInfo.INVALID_FAIR_SHARE) {
|
if (fairShare == FairSchedulerInfo.INVALID_FAIR_SHARE) {
|
||||||
// FairScheduler#applications don't have the entry. Skip it.
|
// FairScheduler#applications don't have the entry. Skip it.
|
||||||
continue;
|
continue;
|
||||||
|
|
|
@ -1491,14 +1491,14 @@ public class RMWebServices {
|
||||||
String msg = "Requested more cores than configured max";
|
String msg = "Requested more cores than configured max";
|
||||||
throw new BadRequestException(msg);
|
throw new BadRequestException(msg);
|
||||||
}
|
}
|
||||||
if (newApp.getResource().getMemory() > rm.getConfig().getInt(
|
if (newApp.getResource().getMemorySize() > rm.getConfig().getInt(
|
||||||
YarnConfiguration.RM_SCHEDULER_MAXIMUM_ALLOCATION_MB,
|
YarnConfiguration.RM_SCHEDULER_MAXIMUM_ALLOCATION_MB,
|
||||||
YarnConfiguration.DEFAULT_RM_SCHEDULER_MAXIMUM_ALLOCATION_MB)) {
|
YarnConfiguration.DEFAULT_RM_SCHEDULER_MAXIMUM_ALLOCATION_MB)) {
|
||||||
String msg = "Requested more memory than configured max";
|
String msg = "Requested more memory than configured max";
|
||||||
throw new BadRequestException(msg);
|
throw new BadRequestException(msg);
|
||||||
}
|
}
|
||||||
Resource r =
|
Resource r =
|
||||||
Resource.newInstance(newApp.getResource().getMemory(), newApp
|
Resource.newInstance(newApp.getResource().getMemorySize(), newApp
|
||||||
.getResource().getvCores());
|
.getResource().getvCores());
|
||||||
return r;
|
return r;
|
||||||
}
|
}
|
||||||
|
@ -1971,7 +1971,7 @@ public class RMWebServices {
|
||||||
.getReservationRequest()) {
|
.getReservationRequest()) {
|
||||||
ResourceInfo rInfo = resReqInfo.getCapability();
|
ResourceInfo rInfo = resReqInfo.getCapability();
|
||||||
Resource capability =
|
Resource capability =
|
||||||
Resource.newInstance(rInfo.getMemory(), rInfo.getvCores());
|
Resource.newInstance(rInfo.getMemorySize(), rInfo.getvCores());
|
||||||
int numContainers = resReqInfo.getNumContainers();
|
int numContainers = resReqInfo.getNumContainers();
|
||||||
int minConcurrency = resReqInfo.getMinConcurrency();
|
int minConcurrency = resReqInfo.getMinConcurrency();
|
||||||
long duration = resReqInfo.getDuration();
|
long duration = resReqInfo.getDuration();
|
||||||
|
@ -2084,7 +2084,7 @@ public class RMWebServices {
|
||||||
.getReservationRequest()) {
|
.getReservationRequest()) {
|
||||||
ResourceInfo rInfo = resReqInfo.getCapability();
|
ResourceInfo rInfo = resReqInfo.getCapability();
|
||||||
Resource capability =
|
Resource capability =
|
||||||
Resource.newInstance(rInfo.getMemory(), rInfo.getvCores());
|
Resource.newInstance(rInfo.getMemorySize(), rInfo.getvCores());
|
||||||
int numContainers = resReqInfo.getNumContainers();
|
int numContainers = resReqInfo.getNumContainers();
|
||||||
int minConcurrency = resReqInfo.getMinConcurrency();
|
int minConcurrency = resReqInfo.getMinConcurrency();
|
||||||
long duration = resReqInfo.getDuration();
|
long duration = resReqInfo.getDuration();
|
||||||
|
|
|
@ -82,8 +82,8 @@ public class AppInfo {
|
||||||
protected long elapsedTime;
|
protected long elapsedTime;
|
||||||
protected String amContainerLogs;
|
protected String amContainerLogs;
|
||||||
protected String amHostHttpAddress;
|
protected String amHostHttpAddress;
|
||||||
protected int allocatedMB;
|
protected long allocatedMB;
|
||||||
protected int allocatedVCores;
|
protected long allocatedVCores;
|
||||||
protected int runningContainers;
|
protected int runningContainers;
|
||||||
protected long memorySeconds;
|
protected long memorySeconds;
|
||||||
protected long vcoreSeconds;
|
protected long vcoreSeconds;
|
||||||
|
@ -91,8 +91,8 @@ public class AppInfo {
|
||||||
protected float clusterUsagePercentage;
|
protected float clusterUsagePercentage;
|
||||||
|
|
||||||
// preemption info fields
|
// preemption info fields
|
||||||
protected int preemptedResourceMB;
|
protected long preemptedResourceMB;
|
||||||
protected int preemptedResourceVCores;
|
protected long preemptedResourceVCores;
|
||||||
protected int numNonAMContainerPreempted;
|
protected int numNonAMContainerPreempted;
|
||||||
protected int numAMContainerPreempted;
|
protected int numAMContainerPreempted;
|
||||||
|
|
||||||
|
@ -174,7 +174,7 @@ public class AppInfo {
|
||||||
.getApplicationResourceUsageReport();
|
.getApplicationResourceUsageReport();
|
||||||
if (resourceReport != null) {
|
if (resourceReport != null) {
|
||||||
Resource usedResources = resourceReport.getUsedResources();
|
Resource usedResources = resourceReport.getUsedResources();
|
||||||
allocatedMB = usedResources.getMemory();
|
allocatedMB = usedResources.getMemorySize();
|
||||||
allocatedVCores = usedResources.getVirtualCores();
|
allocatedVCores = usedResources.getVirtualCores();
|
||||||
runningContainers = resourceReport.getNumUsedContainers();
|
runningContainers = resourceReport.getNumUsedContainers();
|
||||||
queueUsagePercentage = resourceReport.getQueueUsagePercentage();
|
queueUsagePercentage = resourceReport.getQueueUsagePercentage();
|
||||||
|
@ -190,7 +190,7 @@ public class AppInfo {
|
||||||
numAMContainerPreempted =
|
numAMContainerPreempted =
|
||||||
appMetrics.getNumAMContainersPreempted();
|
appMetrics.getNumAMContainersPreempted();
|
||||||
preemptedResourceMB =
|
preemptedResourceMB =
|
||||||
appMetrics.getResourcePreempted().getMemory();
|
appMetrics.getResourcePreempted().getMemorySize();
|
||||||
numNonAMContainerPreempted =
|
numNonAMContainerPreempted =
|
||||||
appMetrics.getNumNonAMContainersPreempted();
|
appMetrics.getNumNonAMContainersPreempted();
|
||||||
preemptedResourceVCores =
|
preemptedResourceVCores =
|
||||||
|
@ -302,19 +302,19 @@ public class AppInfo {
|
||||||
return this.runningContainers;
|
return this.runningContainers;
|
||||||
}
|
}
|
||||||
|
|
||||||
public int getAllocatedMB() {
|
public long getAllocatedMB() {
|
||||||
return this.allocatedMB;
|
return this.allocatedMB;
|
||||||
}
|
}
|
||||||
|
|
||||||
public int getAllocatedVCores() {
|
public long getAllocatedVCores() {
|
||||||
return this.allocatedVCores;
|
return this.allocatedVCores;
|
||||||
}
|
}
|
||||||
|
|
||||||
public int getPreemptedMB() {
|
public long getPreemptedMB() {
|
||||||
return preemptedResourceMB;
|
return preemptedResourceMB;
|
||||||
}
|
}
|
||||||
|
|
||||||
public int getPreemptedVCores() {
|
public long getPreemptedVCores() {
|
||||||
return preemptedResourceVCores;
|
return preemptedResourceVCores;
|
||||||
}
|
}
|
||||||
|
|
||||||
|
|
|
@ -54,10 +54,10 @@ public class FairSchedulerInfo extends SchedulerInfo {
|
||||||
* <code>FairSchedulerInfo#INVALID_FAIR_SHARE</code> if the scheduler does
|
* <code>FairSchedulerInfo#INVALID_FAIR_SHARE</code> if the scheduler does
|
||||||
* not know about this application attempt.
|
* not know about this application attempt.
|
||||||
*/
|
*/
|
||||||
public int getAppFairShare(ApplicationAttemptId appAttemptId) {
|
public long getAppFairShare(ApplicationAttemptId appAttemptId) {
|
||||||
FSAppAttempt fsAppAttempt = scheduler.getSchedulerApp(appAttemptId);
|
FSAppAttempt fsAppAttempt = scheduler.getSchedulerApp(appAttemptId);
|
||||||
return fsAppAttempt == null ?
|
return fsAppAttempt == null ?
|
||||||
INVALID_FAIR_SHARE : fsAppAttempt.getFairShare().getMemory();
|
INVALID_FAIR_SHARE : fsAppAttempt.getFairShare().getMemorySize();
|
||||||
}
|
}
|
||||||
|
|
||||||
public FairSchedulerQueueInfo getRootQueueInfo() {
|
public FairSchedulerQueueInfo getRootQueueInfo() {
|
||||||
|
|
|
@ -81,8 +81,8 @@ public class FairSchedulerQueueInfo {
|
||||||
|
|
||||||
usedResources = new ResourceInfo(queue.getResourceUsage());
|
usedResources = new ResourceInfo(queue.getResourceUsage());
|
||||||
demandResources = new ResourceInfo(queue.getDemand());
|
demandResources = new ResourceInfo(queue.getDemand());
|
||||||
fractionMemUsed = (float)usedResources.getMemory() /
|
fractionMemUsed = (float)usedResources.getMemorySize() /
|
||||||
clusterResources.getMemory();
|
clusterResources.getMemorySize();
|
||||||
|
|
||||||
steadyFairResources = new ResourceInfo(queue.getSteadyFairShare());
|
steadyFairResources = new ResourceInfo(queue.getSteadyFairShare());
|
||||||
fairResources = new ResourceInfo(queue.getFairShare());
|
fairResources = new ResourceInfo(queue.getFairShare());
|
||||||
|
@ -93,11 +93,11 @@ public class FairSchedulerQueueInfo {
|
||||||
scheduler.getClusterResource()));
|
scheduler.getClusterResource()));
|
||||||
|
|
||||||
fractionMemSteadyFairShare =
|
fractionMemSteadyFairShare =
|
||||||
(float)steadyFairResources.getMemory() / clusterResources.getMemory();
|
(float)steadyFairResources.getMemorySize() / clusterResources.getMemorySize();
|
||||||
fractionMemFairShare = (float) fairResources.getMemory()
|
fractionMemFairShare = (float) fairResources.getMemorySize()
|
||||||
/ clusterResources.getMemory();
|
/ clusterResources.getMemorySize();
|
||||||
fractionMemMinShare = (float)minResources.getMemory() / clusterResources.getMemory();
|
fractionMemMinShare = (float)minResources.getMemorySize() / clusterResources.getMemorySize();
|
||||||
fractionMemMaxShare = (float)maxResources.getMemory() / clusterResources.getMemory();
|
fractionMemMaxShare = (float)maxResources.getMemorySize() / clusterResources.getMemorySize();
|
||||||
|
|
||||||
maxApps = allocConf.getQueueMaxApps(queueName);
|
maxApps = allocConf.getQueueMaxApps(queueName);
|
||||||
|
|
||||||
|
|
|
@ -40,8 +40,8 @@ public class FifoSchedulerInfo extends SchedulerInfo {
|
||||||
protected float capacity;
|
protected float capacity;
|
||||||
protected float usedCapacity;
|
protected float usedCapacity;
|
||||||
protected QueueState qstate;
|
protected QueueState qstate;
|
||||||
protected int minQueueMemoryCapacity;
|
protected long minQueueMemoryCapacity;
|
||||||
protected int maxQueueMemoryCapacity;
|
protected long maxQueueMemoryCapacity;
|
||||||
protected int numNodes;
|
protected int numNodes;
|
||||||
protected int usedNodeCapacity;
|
protected int usedNodeCapacity;
|
||||||
protected int availNodeCapacity;
|
protected int availNodeCapacity;
|
||||||
|
@ -67,8 +67,8 @@ public class FifoSchedulerInfo extends SchedulerInfo {
|
||||||
|
|
||||||
this.usedCapacity = qInfo.getCurrentCapacity();
|
this.usedCapacity = qInfo.getCurrentCapacity();
|
||||||
this.capacity = qInfo.getCapacity();
|
this.capacity = qInfo.getCapacity();
|
||||||
this.minQueueMemoryCapacity = fs.getMinimumResourceCapability().getMemory();
|
this.minQueueMemoryCapacity = fs.getMinimumResourceCapability().getMemorySize();
|
||||||
this.maxQueueMemoryCapacity = fs.getMaximumResourceCapability().getMemory();
|
this.maxQueueMemoryCapacity = fs.getMaximumResourceCapability().getMemorySize();
|
||||||
this.qstate = qInfo.getQueueState();
|
this.qstate = qInfo.getQueueState();
|
||||||
|
|
||||||
this.numNodes = rmContext.getRMNodes().size();
|
this.numNodes = rmContext.getRMNodes().size();
|
||||||
|
@ -79,9 +79,9 @@ public class FifoSchedulerInfo extends SchedulerInfo {
|
||||||
|
|
||||||
for (RMNode ni : rmContext.getRMNodes().values()) {
|
for (RMNode ni : rmContext.getRMNodes().values()) {
|
||||||
SchedulerNodeReport report = fs.getNodeReport(ni.getNodeID());
|
SchedulerNodeReport report = fs.getNodeReport(ni.getNodeID());
|
||||||
this.usedNodeCapacity += report.getUsedResource().getMemory();
|
this.usedNodeCapacity += report.getUsedResource().getMemorySize();
|
||||||
this.availNodeCapacity += report.getAvailableResource().getMemory();
|
this.availNodeCapacity += report.getAvailableResource().getMemorySize();
|
||||||
this.totalNodeCapacity += ni.getTotalCapability().getMemory();
|
this.totalNodeCapacity += ni.getTotalCapability().getMemorySize();
|
||||||
this.numContainers += fs.getNodeReport(ni.getNodeID()).getNumContainers();
|
this.numContainers += fs.getNodeReport(ni.getNodeID()).getNumContainers();
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
|
@ -114,11 +114,11 @@ public class FifoSchedulerInfo extends SchedulerInfo {
|
||||||
return this.qName;
|
return this.qName;
|
||||||
}
|
}
|
||||||
|
|
||||||
public int getMinQueueMemoryCapacity() {
|
public long getMinQueueMemoryCapacity() {
|
||||||
return this.minQueueMemoryCapacity;
|
return this.minQueueMemoryCapacity;
|
||||||
}
|
}
|
||||||
|
|
||||||
public int getMaxQueueMemoryCapacity() {
|
public long getMaxQueueMemoryCapacity() {
|
||||||
return this.maxQueueMemoryCapacity;
|
return this.maxQueueMemoryCapacity;
|
||||||
}
|
}
|
||||||
|
|
||||||
|
|
|
@ -63,8 +63,8 @@ public class NodeInfo {
|
||||||
this.availMemoryMB = 0;
|
this.availMemoryMB = 0;
|
||||||
if (report != null) {
|
if (report != null) {
|
||||||
this.numContainers = report.getNumContainers();
|
this.numContainers = report.getNumContainers();
|
||||||
this.usedMemoryMB = report.getUsedResource().getMemory();
|
this.usedMemoryMB = report.getUsedResource().getMemorySize();
|
||||||
this.availMemoryMB = report.getAvailableResource().getMemory();
|
this.availMemoryMB = report.getAvailableResource().getMemorySize();
|
||||||
this.usedVirtualCores = report.getUsedResource().getVirtualCores();
|
this.usedVirtualCores = report.getUsedResource().getVirtualCores();
|
||||||
this.availableVirtualCores = report.getAvailableResource().getVirtualCores();
|
this.availableVirtualCores = report.getAvailableResource().getVirtualCores();
|
||||||
}
|
}
|
||||||
|
|
|
@ -27,22 +27,22 @@ import org.apache.hadoop.yarn.api.records.Resource;
|
||||||
@XmlRootElement
|
@XmlRootElement
|
||||||
@XmlAccessorType(XmlAccessType.FIELD)
|
@XmlAccessorType(XmlAccessType.FIELD)
|
||||||
public class ResourceInfo {
|
public class ResourceInfo {
|
||||||
int memory;
|
long memory;
|
||||||
int vCores;
|
long vCores;
|
||||||
|
|
||||||
public ResourceInfo() {
|
public ResourceInfo() {
|
||||||
}
|
}
|
||||||
|
|
||||||
public ResourceInfo(Resource res) {
|
public ResourceInfo(Resource res) {
|
||||||
memory = res.getMemory();
|
memory = res.getMemorySize();
|
||||||
vCores = res.getVirtualCores();
|
vCores = res.getVirtualCores();
|
||||||
}
|
}
|
||||||
|
|
||||||
public int getMemory() {
|
public long getMemorySize() {
|
||||||
return memory;
|
return memory;
|
||||||
}
|
}
|
||||||
|
|
||||||
public int getvCores() {
|
public long getvCores() {
|
||||||
return vCores;
|
return vCores;
|
||||||
}
|
}
|
||||||
|
|
||||||
|
|
|
@ -194,7 +194,7 @@ public class Application {
|
||||||
Resource currentSpec = requestSpec.put(priority, capability);
|
Resource currentSpec = requestSpec.put(priority, capability);
|
||||||
if (currentSpec != null) {
|
if (currentSpec != null) {
|
||||||
throw new IllegalStateException("Resource spec already exists for " +
|
throw new IllegalStateException("Resource spec already exists for " +
|
||||||
"priority " + priority.getPriority() + " - " + currentSpec.getMemory());
|
"priority " + priority.getPriority() + " - " + currentSpec.getMemorySize());
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
|
|
||||||
|
|
|
@ -50,8 +50,10 @@ public class MockNM {
|
||||||
|
|
||||||
private int responseId;
|
private int responseId;
|
||||||
private NodeId nodeId;
|
private NodeId nodeId;
|
||||||
private final int memory;
|
|
||||||
private final int vCores;
|
private long memory;
|
||||||
|
private int vCores;
|
||||||
|
|
||||||
private ResourceTrackerService resourceTracker;
|
private ResourceTrackerService resourceTracker;
|
||||||
private int httpPort = 2;
|
private int httpPort = 2;
|
||||||
private MasterKey currentContainerTokenMasterKey;
|
private MasterKey currentContainerTokenMasterKey;
|
||||||
|
@ -142,7 +144,7 @@ public class MockNM {
|
||||||
this.currentContainerTokenMasterKey =
|
this.currentContainerTokenMasterKey =
|
||||||
registrationResponse.getContainerTokenMasterKey();
|
registrationResponse.getContainerTokenMasterKey();
|
||||||
this.currentNMTokenMasterKey = registrationResponse.getNMTokenMasterKey();
|
this.currentNMTokenMasterKey = registrationResponse.getNMTokenMasterKey();
|
||||||
return registrationResponse;
|
return registrationResponse;
|
||||||
}
|
}
|
||||||
|
|
||||||
public NodeHeartbeatResponse nodeHeartbeat(boolean isHealthy) throws Exception {
|
public NodeHeartbeatResponse nodeHeartbeat(boolean isHealthy) throws Exception {
|
||||||
|
@ -211,11 +213,11 @@ public class MockNM {
|
||||||
.getKeyId()) {
|
.getKeyId()) {
|
||||||
this.currentNMTokenMasterKey = masterKeyFromRM;
|
this.currentNMTokenMasterKey = masterKeyFromRM;
|
||||||
}
|
}
|
||||||
|
|
||||||
return heartbeatResponse;
|
return heartbeatResponse;
|
||||||
}
|
}
|
||||||
|
|
||||||
public int getMemory() {
|
public long getMemory() {
|
||||||
return memory;
|
return memory;
|
||||||
}
|
}
|
||||||
|
|
||||||
|
|
|
@ -89,13 +89,13 @@ public class MockNodes {
|
||||||
|
|
||||||
public static Resource newUsedResource(Resource total) {
|
public static Resource newUsedResource(Resource total) {
|
||||||
Resource rs = recordFactory.newRecordInstance(Resource.class);
|
Resource rs = recordFactory.newRecordInstance(Resource.class);
|
||||||
rs.setMemory((int)(Math.random() * total.getMemory()));
|
rs.setMemory((int)(Math.random() * total.getMemorySize()));
|
||||||
return rs;
|
return rs;
|
||||||
}
|
}
|
||||||
|
|
||||||
public static Resource newAvailResource(Resource total, Resource used) {
|
public static Resource newAvailResource(Resource total, Resource used) {
|
||||||
Resource rs = recordFactory.newRecordInstance(Resource.class);
|
Resource rs = recordFactory.newRecordInstance(Resource.class);
|
||||||
rs.setMemory(total.getMemory() - used.getMemory());
|
rs.setMemory(total.getMemorySize() - used.getMemorySize());
|
||||||
return rs;
|
return rs;
|
||||||
}
|
}
|
||||||
|
|
||||||
|
|
|
@ -213,12 +213,12 @@ public class NodeManager implements ContainerManagementProtocol {
|
||||||
|
|
||||||
synchronized public void checkResourceUsage() {
|
synchronized public void checkResourceUsage() {
|
||||||
LOG.info("Checking resource usage for " + containerManagerAddress);
|
LOG.info("Checking resource usage for " + containerManagerAddress);
|
||||||
Assert.assertEquals(available.getMemory(),
|
Assert.assertEquals(available.getMemorySize(),
|
||||||
resourceManager.getResourceScheduler().getNodeReport(
|
resourceManager.getResourceScheduler().getNodeReport(
|
||||||
this.nodeId).getAvailableResource().getMemory());
|
this.nodeId).getAvailableResource().getMemorySize());
|
||||||
Assert.assertEquals(used.getMemory(),
|
Assert.assertEquals(used.getMemorySize(),
|
||||||
resourceManager.getResourceScheduler().getNodeReport(
|
resourceManager.getResourceScheduler().getNodeReport(
|
||||||
this.nodeId).getUsedResource().getMemory());
|
this.nodeId).getUsedResource().getMemorySize());
|
||||||
}
|
}
|
||||||
|
|
||||||
@Override
|
@Override
|
||||||
|
|
|
@ -488,7 +488,7 @@ public class ReservationACLsTestBase extends ACLsTestBase {
|
||||||
|
|
||||||
private boolean checkCapacity(Collection<Plan> plans) {
|
private boolean checkCapacity(Collection<Plan> plans) {
|
||||||
for (Plan plan : plans) {
|
for (Plan plan : plans) {
|
||||||
if (plan.getTotalCapacity().getMemory() > 0) {
|
if (plan.getTotalCapacity().getMemorySize() > 0) {
|
||||||
return true;
|
return true;
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
|
|
|
@ -383,11 +383,11 @@ public class TestApplicationACLs {
|
||||||
Assert.assertEquals("Enemy should not see app reserved containers",
|
Assert.assertEquals("Enemy should not see app reserved containers",
|
||||||
-1, usageReport.getNumReservedContainers());
|
-1, usageReport.getNumReservedContainers());
|
||||||
Assert.assertEquals("Enemy should not see app used resources",
|
Assert.assertEquals("Enemy should not see app used resources",
|
||||||
-1, usageReport.getUsedResources().getMemory());
|
-1, usageReport.getUsedResources().getMemorySize());
|
||||||
Assert.assertEquals("Enemy should not see app reserved resources",
|
Assert.assertEquals("Enemy should not see app reserved resources",
|
||||||
-1, usageReport.getReservedResources().getMemory());
|
-1, usageReport.getReservedResources().getMemorySize());
|
||||||
Assert.assertEquals("Enemy should not see app needed resources",
|
Assert.assertEquals("Enemy should not see app needed resources",
|
||||||
-1, usageReport.getNeededResources().getMemory());
|
-1, usageReport.getNeededResources().getMemorySize());
|
||||||
}
|
}
|
||||||
|
|
||||||
private void verifyAdministerQueueUserAccess() throws Exception {
|
private void verifyAdministerQueueUserAccess() throws Exception {
|
||||||
|
|
|
@ -531,7 +531,7 @@ public class TestApplicationCleanup {
|
||||||
// 4. Verify Memory Usage by cluster, it should be 3072. AM memory +
|
// 4. Verify Memory Usage by cluster, it should be 3072. AM memory +
|
||||||
// requested memory. 1024 + 2048=3072
|
// requested memory. 1024 + 2048=3072
|
||||||
ResourceScheduler rs = rm1.getRMContext().getScheduler();
|
ResourceScheduler rs = rm1.getRMContext().getScheduler();
|
||||||
int allocatedMB = rs.getRootQueueMetrics().getAllocatedMB();
|
long allocatedMB = rs.getRootQueueMetrics().getAllocatedMB();
|
||||||
Assert.assertEquals(amMemory + containerMemory, allocatedMB);
|
Assert.assertEquals(amMemory + containerMemory, allocatedMB);
|
||||||
|
|
||||||
// 5. Re-register NM by sending completed container status
|
// 5. Re-register NM by sending completed container status
|
||||||
|
|
|
@ -404,7 +404,7 @@ public class TestContainerResourceUsage {
|
||||||
Resource resource = rmContainer.getContainer().getResource();
|
Resource resource = rmContainer.getContainer().getResource();
|
||||||
long usedMillis =
|
long usedMillis =
|
||||||
rmContainer.getFinishTime() - rmContainer.getCreationTime();
|
rmContainer.getFinishTime() - rmContainer.getCreationTime();
|
||||||
long memorySeconds = resource.getMemory()
|
long memorySeconds = resource.getMemorySize()
|
||||||
* usedMillis / DateUtils.MILLIS_PER_SECOND;
|
* usedMillis / DateUtils.MILLIS_PER_SECOND;
|
||||||
long vcoreSeconds = resource.getVirtualCores()
|
long vcoreSeconds = resource.getVirtualCores()
|
||||||
* usedMillis / DateUtils.MILLIS_PER_SECOND;
|
* usedMillis / DateUtils.MILLIS_PER_SECOND;
|
||||||
|
|
|
@ -21,7 +21,6 @@ package org.apache.hadoop.yarn.server.resourcemanager;
|
||||||
import org.junit.Before;
|
import org.junit.Before;
|
||||||
import static org.mockito.Matchers.argThat;
|
import static org.mockito.Matchers.argThat;
|
||||||
import static org.mockito.Mockito.doNothing;
|
import static org.mockito.Mockito.doNothing;
|
||||||
import static org.mockito.Mockito.doAnswer;
|
|
||||||
import static org.mockito.Mockito.spy;
|
import static org.mockito.Mockito.spy;
|
||||||
|
|
||||||
import java.util.ArrayList;
|
import java.util.ArrayList;
|
||||||
|
@ -45,7 +44,6 @@ import org.apache.hadoop.yarn.api.protocolrecords.GetApplicationsResponse;
|
||||||
import org.apache.hadoop.yarn.api.protocolrecords.GetNewApplicationResponse;
|
import org.apache.hadoop.yarn.api.protocolrecords.GetNewApplicationResponse;
|
||||||
import org.apache.hadoop.yarn.api.protocolrecords.KillApplicationRequest;
|
import org.apache.hadoop.yarn.api.protocolrecords.KillApplicationRequest;
|
||||||
import org.apache.hadoop.yarn.api.records.ApplicationAttemptId;
|
import org.apache.hadoop.yarn.api.records.ApplicationAttemptId;
|
||||||
import org.apache.hadoop.yarn.api.records.ApplicationId;
|
|
||||||
import org.apache.hadoop.yarn.api.records.ApplicationReport;
|
import org.apache.hadoop.yarn.api.records.ApplicationReport;
|
||||||
import org.apache.hadoop.yarn.api.records.Container;
|
import org.apache.hadoop.yarn.api.records.Container;
|
||||||
import org.apache.hadoop.yarn.api.records.ContainerId;
|
import org.apache.hadoop.yarn.api.records.ContainerId;
|
||||||
|
@ -61,8 +59,6 @@ import org.apache.hadoop.yarn.event.AbstractEvent;
|
||||||
import org.apache.hadoop.yarn.event.AsyncDispatcher;
|
import org.apache.hadoop.yarn.event.AsyncDispatcher;
|
||||||
import org.apache.hadoop.yarn.event.Dispatcher;
|
import org.apache.hadoop.yarn.event.Dispatcher;
|
||||||
import org.apache.hadoop.yarn.event.EventHandler;
|
import org.apache.hadoop.yarn.event.EventHandler;
|
||||||
import org.apache.hadoop.yarn.server.resourcemanager.TestRMRestart.TestSecurityMockRM;
|
|
||||||
import org.apache.hadoop.yarn.server.resourcemanager.recovery.MemoryRMStateStore;
|
|
||||||
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.RMAppEvent;
|
||||||
import org.apache.hadoop.yarn.server.resourcemanager.rmapp.RMAppEventType;
|
import org.apache.hadoop.yarn.server.resourcemanager.rmapp.RMAppEventType;
|
||||||
|
@ -79,8 +75,6 @@ import org.apache.log4j.LogManager;
|
||||||
import org.apache.log4j.Logger;
|
import org.apache.log4j.Logger;
|
||||||
import org.junit.Test;
|
import org.junit.Test;
|
||||||
import org.mockito.ArgumentMatcher;
|
import org.mockito.ArgumentMatcher;
|
||||||
import org.mockito.invocation.InvocationOnMock;
|
|
||||||
import org.mockito.stubbing.Answer;
|
|
||||||
|
|
||||||
@SuppressWarnings({"unchecked", "rawtypes"})
|
@SuppressWarnings({"unchecked", "rawtypes"})
|
||||||
public class TestRM extends ParameterizedSchedulerTestBase {
|
public class TestRM extends ParameterizedSchedulerTestBase {
|
||||||
|
@ -116,7 +110,7 @@ public class TestRM extends ParameterizedSchedulerTestBase {
|
||||||
|
|
||||||
GetNewApplicationResponse resp = rm.getNewAppId();
|
GetNewApplicationResponse resp = rm.getNewAppId();
|
||||||
assert (resp.getApplicationId().getId() != 0);
|
assert (resp.getApplicationId().getId() != 0);
|
||||||
assert (resp.getMaximumResourceCapability().getMemory() > 0);
|
assert (resp.getMaximumResourceCapability().getMemorySize() > 0);
|
||||||
rm.stop();
|
rm.stop();
|
||||||
}
|
}
|
||||||
|
|
||||||
|
|
|
@ -695,7 +695,7 @@ public class TestRMHA {
|
||||||
}
|
}
|
||||||
|
|
||||||
private void verifyClusterMetrics(int activeNodes, int appsSubmitted,
|
private void verifyClusterMetrics(int activeNodes, int appsSubmitted,
|
||||||
int appsPending, int containersPending, int availableMB,
|
int appsPending, int containersPending, long availableMB,
|
||||||
int activeApplications) throws Exception {
|
int activeApplications) throws Exception {
|
||||||
int timeoutSecs = 0;
|
int timeoutSecs = 0;
|
||||||
QueueMetrics metrics = rm.getResourceScheduler().getRootQueueMetrics();
|
QueueMetrics metrics = rm.getResourceScheduler().getRootQueueMetrics();
|
||||||
|
@ -726,7 +726,7 @@ public class TestRMHA {
|
||||||
assertTrue(message, isAllMetricAssertionDone);
|
assertTrue(message, isAllMetricAssertionDone);
|
||||||
}
|
}
|
||||||
|
|
||||||
private void assertMetric(String metricName, int expected, int actual) {
|
private void assertMetric(String metricName, long expected, long actual) {
|
||||||
assertEquals("Incorrect value for metric " + metricName, expected, actual);
|
assertEquals("Incorrect value for metric " + metricName, expected, actual);
|
||||||
}
|
}
|
||||||
|
|
||||||
|
|
|
@ -31,7 +31,6 @@ import java.util.Collections;
|
||||||
import java.util.List;
|
import java.util.List;
|
||||||
import java.util.Random;
|
import java.util.Random;
|
||||||
|
|
||||||
import org.apache.hadoop.net.Node;
|
|
||||||
import org.apache.hadoop.util.HostsFileReader;
|
import org.apache.hadoop.util.HostsFileReader;
|
||||||
import org.apache.hadoop.yarn.api.records.ApplicationAttemptId;
|
import org.apache.hadoop.yarn.api.records.ApplicationAttemptId;
|
||||||
import org.apache.hadoop.yarn.api.records.ApplicationId;
|
import org.apache.hadoop.yarn.api.records.ApplicationId;
|
||||||
|
@ -50,7 +49,6 @@ import org.apache.hadoop.yarn.server.api.records.NodeStatus;
|
||||||
import org.apache.hadoop.yarn.server.resourcemanager.rmcontainer
|
import org.apache.hadoop.yarn.server.resourcemanager.rmcontainer
|
||||||
.AllocationExpirationInfo;
|
.AllocationExpirationInfo;
|
||||||
import org.apache.hadoop.yarn.server.resourcemanager.rmcontainer.ContainerAllocationExpirer;
|
import org.apache.hadoop.yarn.server.resourcemanager.rmcontainer.ContainerAllocationExpirer;
|
||||||
import org.apache.hadoop.yarn.server.resourcemanager.rmnode.RMNode;
|
|
||||||
import org.apache.hadoop.yarn.server.resourcemanager.rmnode.RMNodeCleanAppEvent;
|
import org.apache.hadoop.yarn.server.resourcemanager.rmnode.RMNodeCleanAppEvent;
|
||||||
import org.apache.hadoop.yarn.server.resourcemanager.rmnode.RMNodeCleanContainerEvent;
|
import org.apache.hadoop.yarn.server.resourcemanager.rmnode.RMNodeCleanContainerEvent;
|
||||||
import org.apache.hadoop.yarn.server.resourcemanager.rmnode.RMNodeEvent;
|
import org.apache.hadoop.yarn.server.resourcemanager.rmnode.RMNodeEvent;
|
||||||
|
@ -869,13 +867,13 @@ public class TestRMNodeTransitions {
|
||||||
public void testResourceUpdateOnRunningNode() {
|
public void testResourceUpdateOnRunningNode() {
|
||||||
RMNodeImpl node = getRunningNode();
|
RMNodeImpl node = getRunningNode();
|
||||||
Resource oldCapacity = node.getTotalCapability();
|
Resource oldCapacity = node.getTotalCapability();
|
||||||
assertEquals("Memory resource is not match.", oldCapacity.getMemory(), 4096);
|
assertEquals("Memory resource is not match.", oldCapacity.getMemorySize(), 4096);
|
||||||
assertEquals("CPU resource is not match.", oldCapacity.getVirtualCores(), 4);
|
assertEquals("CPU resource is not match.", oldCapacity.getVirtualCores(), 4);
|
||||||
node.handle(new RMNodeResourceUpdateEvent(node.getNodeID(),
|
node.handle(new RMNodeResourceUpdateEvent(node.getNodeID(),
|
||||||
ResourceOption.newInstance(Resource.newInstance(2048, 2),
|
ResourceOption.newInstance(Resource.newInstance(2048, 2),
|
||||||
ResourceOption.OVER_COMMIT_TIMEOUT_MILLIS_DEFAULT)));
|
ResourceOption.OVER_COMMIT_TIMEOUT_MILLIS_DEFAULT)));
|
||||||
Resource newCapacity = node.getTotalCapability();
|
Resource newCapacity = node.getTotalCapability();
|
||||||
assertEquals("Memory resource is not match.", newCapacity.getMemory(), 2048);
|
assertEquals("Memory resource is not match.", newCapacity.getMemorySize(), 2048);
|
||||||
assertEquals("CPU resource is not match.", newCapacity.getVirtualCores(), 2);
|
assertEquals("CPU resource is not match.", newCapacity.getVirtualCores(), 2);
|
||||||
|
|
||||||
Assert.assertEquals(NodeState.RUNNING, node.getState());
|
Assert.assertEquals(NodeState.RUNNING, node.getState());
|
||||||
|
@ -893,13 +891,13 @@ public class TestRMNodeTransitions {
|
||||||
public void testResourceUpdateOnNewNode() {
|
public void testResourceUpdateOnNewNode() {
|
||||||
RMNodeImpl node = getNewNode(Resource.newInstance(4096, 4));
|
RMNodeImpl node = getNewNode(Resource.newInstance(4096, 4));
|
||||||
Resource oldCapacity = node.getTotalCapability();
|
Resource oldCapacity = node.getTotalCapability();
|
||||||
assertEquals("Memory resource is not match.", oldCapacity.getMemory(), 4096);
|
assertEquals("Memory resource is not match.", oldCapacity.getMemorySize(), 4096);
|
||||||
assertEquals("CPU resource is not match.", oldCapacity.getVirtualCores(), 4);
|
assertEquals("CPU resource is not match.", oldCapacity.getVirtualCores(), 4);
|
||||||
node.handle(new RMNodeResourceUpdateEvent(node.getNodeID(),
|
node.handle(new RMNodeResourceUpdateEvent(node.getNodeID(),
|
||||||
ResourceOption.newInstance(Resource.newInstance(2048, 2),
|
ResourceOption.newInstance(Resource.newInstance(2048, 2),
|
||||||
ResourceOption.OVER_COMMIT_TIMEOUT_MILLIS_DEFAULT)));
|
ResourceOption.OVER_COMMIT_TIMEOUT_MILLIS_DEFAULT)));
|
||||||
Resource newCapacity = node.getTotalCapability();
|
Resource newCapacity = node.getTotalCapability();
|
||||||
assertEquals("Memory resource is not match.", newCapacity.getMemory(), 2048);
|
assertEquals("Memory resource is not match.", newCapacity.getMemorySize(), 2048);
|
||||||
assertEquals("CPU resource is not match.", newCapacity.getVirtualCores(), 2);
|
assertEquals("CPU resource is not match.", newCapacity.getVirtualCores(), 2);
|
||||||
|
|
||||||
Assert.assertEquals(NodeState.NEW, node.getState());
|
Assert.assertEquals(NodeState.NEW, node.getState());
|
||||||
|
@ -913,13 +911,13 @@ public class TestRMNodeTransitions {
|
||||||
int initialUnHealthy = cm.getUnhealthyNMs();
|
int initialUnHealthy = cm.getUnhealthyNMs();
|
||||||
int initialDecommissioning = cm.getNumDecommissioningNMs();
|
int initialDecommissioning = cm.getNumDecommissioningNMs();
|
||||||
Resource oldCapacity = node.getTotalCapability();
|
Resource oldCapacity = node.getTotalCapability();
|
||||||
assertEquals("Memory resource is not match.", oldCapacity.getMemory(), 4096);
|
assertEquals("Memory resource is not match.", oldCapacity.getMemorySize(), 4096);
|
||||||
assertEquals("CPU resource is not match.", oldCapacity.getVirtualCores(), 4);
|
assertEquals("CPU resource is not match.", oldCapacity.getVirtualCores(), 4);
|
||||||
node.handle(new RMNodeResourceUpdateEvent(node.getNodeID(), ResourceOption
|
node.handle(new RMNodeResourceUpdateEvent(node.getNodeID(), ResourceOption
|
||||||
.newInstance(Resource.newInstance(2048, 2),
|
.newInstance(Resource.newInstance(2048, 2),
|
||||||
ResourceOption.OVER_COMMIT_TIMEOUT_MILLIS_DEFAULT)));
|
ResourceOption.OVER_COMMIT_TIMEOUT_MILLIS_DEFAULT)));
|
||||||
Resource newCapacity = node.getTotalCapability();
|
Resource newCapacity = node.getTotalCapability();
|
||||||
assertEquals("Memory resource is not match.", newCapacity.getMemory(), 2048);
|
assertEquals("Memory resource is not match.", newCapacity.getMemorySize(), 2048);
|
||||||
assertEquals("CPU resource is not match.", newCapacity.getVirtualCores(), 2);
|
assertEquals("CPU resource is not match.", newCapacity.getVirtualCores(), 2);
|
||||||
|
|
||||||
Assert.assertEquals(NodeState.REBOOTED, node.getState());
|
Assert.assertEquals(NodeState.REBOOTED, node.getState());
|
||||||
|
@ -994,16 +992,16 @@ public class TestRMNodeTransitions {
|
||||||
public void testResourceUpdateOnDecommissioningNode() {
|
public void testResourceUpdateOnDecommissioningNode() {
|
||||||
RMNodeImpl node = getDecommissioningNode();
|
RMNodeImpl node = getDecommissioningNode();
|
||||||
Resource oldCapacity = node.getTotalCapability();
|
Resource oldCapacity = node.getTotalCapability();
|
||||||
assertEquals("Memory resource is not match.", oldCapacity.getMemory(), 4096);
|
assertEquals("Memory resource is not match.", oldCapacity.getMemorySize(), 4096);
|
||||||
assertEquals("CPU resource is not match.", oldCapacity.getVirtualCores(), 4);
|
assertEquals("CPU resource is not match.", oldCapacity.getVirtualCores(), 4);
|
||||||
node.handle(new RMNodeResourceUpdateEvent(node.getNodeID(),
|
node.handle(new RMNodeResourceUpdateEvent(node.getNodeID(),
|
||||||
ResourceOption.newInstance(Resource.newInstance(2048, 2),
|
ResourceOption.newInstance(Resource.newInstance(2048, 2),
|
||||||
ResourceOption.OVER_COMMIT_TIMEOUT_MILLIS_DEFAULT)));
|
ResourceOption.OVER_COMMIT_TIMEOUT_MILLIS_DEFAULT)));
|
||||||
Resource originalCapacity = node.getOriginalTotalCapability();
|
Resource originalCapacity = node.getOriginalTotalCapability();
|
||||||
assertEquals("Memory resource is not match.", originalCapacity.getMemory(), oldCapacity.getMemory());
|
assertEquals("Memory resource is not match.", originalCapacity.getMemorySize(), oldCapacity.getMemorySize());
|
||||||
assertEquals("CPU resource is not match.", originalCapacity.getVirtualCores(), oldCapacity.getVirtualCores());
|
assertEquals("CPU resource is not match.", originalCapacity.getVirtualCores(), oldCapacity.getVirtualCores());
|
||||||
Resource newCapacity = node.getTotalCapability();
|
Resource newCapacity = node.getTotalCapability();
|
||||||
assertEquals("Memory resource is not match.", newCapacity.getMemory(), 2048);
|
assertEquals("Memory resource is not match.", newCapacity.getMemorySize(), 2048);
|
||||||
assertEquals("CPU resource is not match.", newCapacity.getVirtualCores(), 2);
|
assertEquals("CPU resource is not match.", newCapacity.getVirtualCores(), 2);
|
||||||
|
|
||||||
Assert.assertEquals(NodeState.DECOMMISSIONING, node.getState());
|
Assert.assertEquals(NodeState.DECOMMISSIONING, node.getState());
|
||||||
|
@ -1016,7 +1014,7 @@ public class TestRMNodeTransitions {
|
||||||
public void testResourceUpdateOnRecommissioningNode() {
|
public void testResourceUpdateOnRecommissioningNode() {
|
||||||
RMNodeImpl node = getDecommissioningNode();
|
RMNodeImpl node = getDecommissioningNode();
|
||||||
Resource oldCapacity = node.getTotalCapability();
|
Resource oldCapacity = node.getTotalCapability();
|
||||||
assertEquals("Memory resource is not match.", oldCapacity.getMemory(), 4096);
|
assertEquals("Memory resource is not match.", oldCapacity.getMemorySize(), 4096);
|
||||||
assertEquals("CPU resource is not match.", oldCapacity.getVirtualCores(), 4);
|
assertEquals("CPU resource is not match.", oldCapacity.getVirtualCores(), 4);
|
||||||
node.handle(new RMNodeEvent(node.getNodeID(),
|
node.handle(new RMNodeEvent(node.getNodeID(),
|
||||||
RMNodeEventType.RECOMMISSION));
|
RMNodeEventType.RECOMMISSION));
|
||||||
|
|
|
@ -193,7 +193,7 @@ public class TestReservationSystemWithRMHA extends RMHATestBase {
|
||||||
.synchronizePlan(ReservationSystemTestUtil.reservationQ, false);
|
.synchronizePlan(ReservationSystemTestUtil.reservationQ, false);
|
||||||
if (rm.getRMContext().getReservationSystem()
|
if (rm.getRMContext().getReservationSystem()
|
||||||
.getPlan(ReservationSystemTestUtil.reservationQ).getTotalCapacity()
|
.getPlan(ReservationSystemTestUtil.reservationQ).getTotalCapacity()
|
||||||
.getMemory() > 0) {
|
.getMemorySize() > 0) {
|
||||||
break;
|
break;
|
||||||
}
|
}
|
||||||
LOG.info("Waiting for node capacity to be added to plan");
|
LOG.info("Waiting for node capacity to be added to plan");
|
||||||
|
|
|
@ -1056,7 +1056,7 @@ public class TestResourceTrackerService extends NodeLabelTestBase {
|
||||||
rm.drainEvents();
|
rm.drainEvents();
|
||||||
RMNode rmNode = rm.getRMContext().getRMNodes().get(nm1.getNodeId());
|
RMNode rmNode = rm.getRMContext().getRMNodes().get(nm1.getNodeId());
|
||||||
Assert.assertEquals(3, rmNode.getHttpPort());
|
Assert.assertEquals(3, rmNode.getHttpPort());
|
||||||
Assert.assertEquals(5120, rmNode.getTotalCapability().getMemory());
|
Assert.assertEquals(5120, rmNode.getTotalCapability().getMemorySize());
|
||||||
Assert.assertEquals(5120 + 15360, metrics.getAvailableMB());
|
Assert.assertEquals(5120 + 15360, metrics.getAvailableMB());
|
||||||
|
|
||||||
}
|
}
|
||||||
|
|
|
@ -104,7 +104,6 @@ import org.junit.runner.RunWith;
|
||||||
import org.junit.runners.Parameterized;
|
import org.junit.runners.Parameterized;
|
||||||
import com.google.common.base.Supplier;
|
import com.google.common.base.Supplier;
|
||||||
|
|
||||||
|
|
||||||
@SuppressWarnings({"rawtypes", "unchecked"})
|
@SuppressWarnings({"rawtypes", "unchecked"})
|
||||||
@RunWith(value = Parameterized.class)
|
@RunWith(value = Parameterized.class)
|
||||||
public class TestWorkPreservingRMRestart extends ParameterizedSchedulerTestBase {
|
public class TestWorkPreservingRMRestart extends ParameterizedSchedulerTestBase {
|
||||||
|
@ -427,15 +426,15 @@ public class TestWorkPreservingRMRestart extends ParameterizedSchedulerTestBase
|
||||||
|
|
||||||
// ************* check Queue metrics ************
|
// ************* check Queue metrics ************
|
||||||
QueueMetrics queueMetrics = queue.getMetrics();
|
QueueMetrics queueMetrics = queue.getMetrics();
|
||||||
assertMetrics(queueMetrics, 1, 0, 1, 0, 2, availableResources.getMemory(),
|
assertMetrics(queueMetrics, 1, 0, 1, 0, 2, availableResources.getMemorySize(),
|
||||||
availableResources.getVirtualCores(), usedResource.getMemory(),
|
availableResources.getVirtualCores(), usedResource.getMemorySize(),
|
||||||
usedResource.getVirtualCores());
|
usedResource.getVirtualCores());
|
||||||
|
|
||||||
// ************ check user metrics ***********
|
// ************ check user metrics ***********
|
||||||
QueueMetrics userMetrics =
|
QueueMetrics userMetrics =
|
||||||
queueMetrics.getUserMetrics(app.getUser());
|
queueMetrics.getUserMetrics(app.getUser());
|
||||||
assertMetrics(userMetrics, 1, 0, 1, 0, 2, availableResources.getMemory(),
|
assertMetrics(userMetrics, 1, 0, 1, 0, 2, availableResources.getMemorySize(),
|
||||||
availableResources.getVirtualCores(), usedResource.getMemory(),
|
availableResources.getVirtualCores(), usedResource.getMemorySize(),
|
||||||
usedResource.getVirtualCores());
|
usedResource.getVirtualCores());
|
||||||
}
|
}
|
||||||
|
|
||||||
|
@ -495,8 +494,8 @@ public class TestWorkPreservingRMRestart extends ParameterizedSchedulerTestBase
|
||||||
|
|
||||||
// ************ check queue metrics ****************
|
// ************ check queue metrics ****************
|
||||||
QueueMetrics queueMetrics = scheduler.getRootQueueMetrics();
|
QueueMetrics queueMetrics = scheduler.getRootQueueMetrics();
|
||||||
assertMetrics(queueMetrics, 1, 0, 1, 0, 2, availableResources.getMemory(),
|
assertMetrics(queueMetrics, 1, 0, 1, 0, 2, availableResources.getMemorySize(),
|
||||||
availableResources.getVirtualCores(), usedResources.getMemory(),
|
availableResources.getVirtualCores(), usedResources.getMemorySize(),
|
||||||
usedResources.getVirtualCores());
|
usedResources.getVirtualCores());
|
||||||
}
|
}
|
||||||
|
|
||||||
|
@ -670,8 +669,8 @@ public class TestWorkPreservingRMRestart extends ParameterizedSchedulerTestBase
|
||||||
q1UsedResource, 4);
|
q1UsedResource, 4);
|
||||||
QueueMetrics queue1Metrics = schedulerApp1_1.getQueue().getMetrics();
|
QueueMetrics queue1Metrics = schedulerApp1_1.getQueue().getMetrics();
|
||||||
assertMetrics(queue1Metrics, 2, 0, 2, 0, 4,
|
assertMetrics(queue1Metrics, 2, 0, 2, 0, 4,
|
||||||
q1availableResources.getMemory(),
|
q1availableResources.getMemorySize(),
|
||||||
q1availableResources.getVirtualCores(), q1UsedResource.getMemory(),
|
q1availableResources.getVirtualCores(), q1UsedResource.getMemorySize(),
|
||||||
q1UsedResource.getVirtualCores());
|
q1UsedResource.getVirtualCores());
|
||||||
|
|
||||||
// assert queue B state.
|
// assert queue B state.
|
||||||
|
@ -681,8 +680,8 @@ public class TestWorkPreservingRMRestart extends ParameterizedSchedulerTestBase
|
||||||
q2UsedResource, 2);
|
q2UsedResource, 2);
|
||||||
QueueMetrics queue2Metrics = schedulerApp2.getQueue().getMetrics();
|
QueueMetrics queue2Metrics = schedulerApp2.getQueue().getMetrics();
|
||||||
assertMetrics(queue2Metrics, 1, 0, 1, 0, 2,
|
assertMetrics(queue2Metrics, 1, 0, 1, 0, 2,
|
||||||
q2availableResources.getMemory(),
|
q2availableResources.getMemorySize(),
|
||||||
q2availableResources.getVirtualCores(), q2UsedResource.getMemory(),
|
q2availableResources.getVirtualCores(), q2UsedResource.getMemorySize(),
|
||||||
q2UsedResource.getVirtualCores());
|
q2UsedResource.getVirtualCores());
|
||||||
|
|
||||||
// assert parent queue state.
|
// assert parent queue state.
|
||||||
|
@ -691,8 +690,8 @@ public class TestWorkPreservingRMRestart extends ParameterizedSchedulerTestBase
|
||||||
checkParentQueue(parentQueue, 6, totalUsedResource, (float) 6 / 16,
|
checkParentQueue(parentQueue, 6, totalUsedResource, (float) 6 / 16,
|
||||||
(float) 6 / 16);
|
(float) 6 / 16);
|
||||||
assertMetrics(parentQueue.getMetrics(), 3, 0, 3, 0, 6,
|
assertMetrics(parentQueue.getMetrics(), 3, 0, 3, 0, 6,
|
||||||
totalAvailableResource.getMemory(),
|
totalAvailableResource.getMemorySize(),
|
||||||
totalAvailableResource.getVirtualCores(), totalUsedResource.getMemory(),
|
totalAvailableResource.getVirtualCores(), totalUsedResource.getMemorySize(),
|
||||||
totalUsedResource.getVirtualCores());
|
totalUsedResource.getVirtualCores());
|
||||||
}
|
}
|
||||||
|
|
||||||
|
@ -1110,8 +1109,8 @@ public class TestWorkPreservingRMRestart extends ParameterizedSchedulerTestBase
|
||||||
|
|
||||||
private void assertMetrics(QueueMetrics qm, int appsSubmitted,
|
private void assertMetrics(QueueMetrics qm, int appsSubmitted,
|
||||||
int appsPending, int appsRunning, int appsCompleted,
|
int appsPending, int appsRunning, int appsCompleted,
|
||||||
int allocatedContainers, int availableMB, int availableVirtualCores,
|
int allocatedContainers, long availableMB, long availableVirtualCores,
|
||||||
int allocatedMB, int allocatedVirtualCores) {
|
long allocatedMB, long allocatedVirtualCores) {
|
||||||
assertEquals(appsSubmitted, qm.getAppsSubmitted());
|
assertEquals(appsSubmitted, qm.getAppsSubmitted());
|
||||||
assertEquals(appsPending, qm.getAppsPending());
|
assertEquals(appsPending, qm.getAppsPending());
|
||||||
assertEquals(appsRunning, qm.getAppsRunning());
|
assertEquals(appsRunning, qm.getAppsRunning());
|
||||||
|
|
|
@ -424,10 +424,12 @@ public class TestSystemMetricsPublisher {
|
||||||
container.getAllocatedNode().getPort(),
|
container.getAllocatedNode().getPort(),
|
||||||
entity.getOtherInfo().get(
|
entity.getOtherInfo().get(
|
||||||
ContainerMetricsConstants.ALLOCATED_PORT_ENTITY_INFO));
|
ContainerMetricsConstants.ALLOCATED_PORT_ENTITY_INFO));
|
||||||
Assert.assertEquals(
|
Assert.assertEquals(container.getAllocatedResource().getMemorySize(),
|
||||||
container.getAllocatedResource().getMemory(),
|
// KeyValueBasedTimelineStore could cast long to integer, need make sure
|
||||||
entity.getOtherInfo().get(
|
// variables for compare have same type.
|
||||||
ContainerMetricsConstants.ALLOCATED_MEMORY_ENTITY_INFO));
|
((Integer) entity.getOtherInfo().get(
|
||||||
|
ContainerMetricsConstants.ALLOCATED_MEMORY_ENTITY_INFO))
|
||||||
|
.longValue());
|
||||||
Assert.assertEquals(
|
Assert.assertEquals(
|
||||||
container.getAllocatedResource().getVirtualCores(),
|
container.getAllocatedResource().getVirtualCores(),
|
||||||
entity.getOtherInfo().get(
|
entity.getOtherInfo().get(
|
||||||
|
|
|
@ -921,13 +921,19 @@ public class TestProportionalCapacityPreemptionPolicy {
|
||||||
// which is likely triggered since we use small numbers for readability
|
// which is likely triggered since we use small numbers for readability
|
||||||
//run with Logger.getRootLogger().setLevel(Level.DEBUG);
|
//run with Logger.getRootLogger().setLevel(Level.DEBUG);
|
||||||
verify(mDisp, times(9)).handle(argThat(new IsPreemptionRequestFor(appC)));
|
verify(mDisp, times(9)).handle(argThat(new IsPreemptionRequestFor(appC)));
|
||||||
assertEquals(10, policy.getQueuePartitions().get("queueE").get("").preemptableExtra.getMemory());
|
assertEquals(10, policy.getQueuePartitions().get("queueE")
|
||||||
|
.get("").preemptableExtra.getMemorySize());
|
||||||
//2nd level child(E) preempts 10, but parent A has only 9 extra
|
//2nd level child(E) preempts 10, but parent A has only 9 extra
|
||||||
//check the parent can prempt only the extra from > 2 level child
|
//check the parent can prempt only the extra from > 2 level child
|
||||||
TempQueuePerPartition tempQueueAPartition = policy.getQueuePartitions().get("queueA").get("");
|
|
||||||
assertEquals(0, tempQueueAPartition.untouchableExtra.getMemory());
|
TempQueuePerPartition tempQueueAPartition = policy.getQueuePartitions().get(
|
||||||
int extraForQueueA = tempQueueAPartition.current.getMemory()- tempQueueAPartition.guaranteed.getMemory();
|
"queueA").get("");
|
||||||
assertEquals(extraForQueueA,tempQueueAPartition.preemptableExtra.getMemory());
|
assertEquals(0, tempQueueAPartition.untouchableExtra.getMemorySize());
|
||||||
|
long extraForQueueA =
|
||||||
|
tempQueueAPartition.current.getMemorySize() - tempQueueAPartition.guaranteed
|
||||||
|
.getMemorySize();
|
||||||
|
assertEquals(extraForQueueA,
|
||||||
|
tempQueueAPartition.preemptableExtra.getMemorySize());
|
||||||
}
|
}
|
||||||
|
|
||||||
static class IsPreemptionRequestFor
|
static class IsPreemptionRequestFor
|
||||||
|
@ -1061,12 +1067,12 @@ public class TestProportionalCapacityPreemptionPolicy {
|
||||||
when(root.getAbsoluteCapacity()).thenReturn(
|
when(root.getAbsoluteCapacity()).thenReturn(
|
||||||
Resources.divide(rc, tot, abs[0], tot));
|
Resources.divide(rc, tot, abs[0], tot));
|
||||||
when(root.getAbsoluteMaximumCapacity()).thenReturn(
|
when(root.getAbsoluteMaximumCapacity()).thenReturn(
|
||||||
maxCap[0] / (float) tot.getMemory());
|
maxCap[0] / (float) tot.getMemorySize());
|
||||||
when(root.getQueueResourceUsage()).thenReturn(resUsage);
|
when(root.getQueueResourceUsage()).thenReturn(resUsage);
|
||||||
QueueCapacities rootQc = new QueueCapacities(true);
|
QueueCapacities rootQc = new QueueCapacities(true);
|
||||||
rootQc.setAbsoluteUsedCapacity(Resources.divide(rc, tot, used[0], tot));
|
rootQc.setAbsoluteUsedCapacity(Resources.divide(rc, tot, used[0], tot));
|
||||||
rootQc.setAbsoluteCapacity(Resources.divide(rc, tot, abs[0], tot));
|
rootQc.setAbsoluteCapacity(Resources.divide(rc, tot, abs[0], tot));
|
||||||
rootQc.setAbsoluteMaximumCapacity(maxCap[0] / (float) tot.getMemory());
|
rootQc.setAbsoluteMaximumCapacity(maxCap[0] / (float) tot.getMemorySize());
|
||||||
when(root.getQueueCapacities()).thenReturn(rootQc);
|
when(root.getQueueCapacities()).thenReturn(rootQc);
|
||||||
when(root.getQueuePath()).thenReturn(CapacitySchedulerConfiguration.ROOT);
|
when(root.getQueuePath()).thenReturn(CapacitySchedulerConfiguration.ROOT);
|
||||||
boolean preemptionDisabled = mockPreemptionStatus("root");
|
boolean preemptionDisabled = mockPreemptionStatus("root");
|
||||||
|
@ -1091,13 +1097,13 @@ public class TestProportionalCapacityPreemptionPolicy {
|
||||||
when(q.getAbsoluteCapacity()).thenReturn(
|
when(q.getAbsoluteCapacity()).thenReturn(
|
||||||
Resources.divide(rc, tot, abs[i], tot));
|
Resources.divide(rc, tot, abs[i], tot));
|
||||||
when(q.getAbsoluteMaximumCapacity()).thenReturn(
|
when(q.getAbsoluteMaximumCapacity()).thenReturn(
|
||||||
maxCap[i] / (float) tot.getMemory());
|
maxCap[i] / (float) tot.getMemorySize());
|
||||||
|
|
||||||
// We need to make these fields to QueueCapacities
|
// We need to make these fields to QueueCapacities
|
||||||
QueueCapacities qc = new QueueCapacities(false);
|
QueueCapacities qc = new QueueCapacities(false);
|
||||||
qc.setAbsoluteUsedCapacity(Resources.divide(rc, tot, used[i], tot));
|
qc.setAbsoluteUsedCapacity(Resources.divide(rc, tot, used[i], tot));
|
||||||
qc.setAbsoluteCapacity(Resources.divide(rc, tot, abs[i], tot));
|
qc.setAbsoluteCapacity(Resources.divide(rc, tot, abs[i], tot));
|
||||||
qc.setAbsoluteMaximumCapacity(maxCap[i] / (float) tot.getMemory());
|
qc.setAbsoluteMaximumCapacity(maxCap[i] / (float) tot.getMemorySize());
|
||||||
when(q.getQueueCapacities()).thenReturn(qc);
|
when(q.getQueueCapacities()).thenReturn(qc);
|
||||||
|
|
||||||
String parentPathName = p.getQueuePath();
|
String parentPathName = p.getQueuePath();
|
||||||
|
|
|
@ -873,7 +873,7 @@ public class TestProportionalCapacityPreemptionPolicyForNodePartitions {
|
||||||
|
|
||||||
private void checkPendingResource(CSQueue queue, String partition, int pending) {
|
private void checkPendingResource(CSQueue queue, String partition, int pending) {
|
||||||
ResourceUsage ru = queue.getQueueResourceUsage();
|
ResourceUsage ru = queue.getQueueResourceUsage();
|
||||||
Assert.assertEquals(pending, ru.getPending(partition).getMemory());
|
Assert.assertEquals(pending, ru.getPending(partition).getMemorySize());
|
||||||
}
|
}
|
||||||
|
|
||||||
private void buildEnv(String labelsConfig, String nodesConfig,
|
private void buildEnv(String labelsConfig, String nodesConfig,
|
||||||
|
|
|
@ -580,7 +580,7 @@ public class TestRMNodeLabelsManager extends NodeLabelTestBase {
|
||||||
for (RMNodeLabel info : infos) {
|
for (RMNodeLabel info : infos) {
|
||||||
if (info.getLabelName().equals(labelName)) {
|
if (info.getLabelName().equals(labelName)) {
|
||||||
Assert.assertEquals(activeNMs, info.getNumActiveNMs());
|
Assert.assertEquals(activeNMs, info.getNumActiveNMs());
|
||||||
Assert.assertEquals(memory, info.getResource().getMemory());
|
Assert.assertEquals(memory, info.getResource().getMemorySize());
|
||||||
return;
|
return;
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
|
|
|
@ -90,7 +90,7 @@ public class ReservationSystemTestUtil {
|
||||||
Assert.assertNotNull(plan);
|
Assert.assertNotNull(plan);
|
||||||
Assert.assertTrue(plan instanceof InMemoryPlan);
|
Assert.assertTrue(plan instanceof InMemoryPlan);
|
||||||
Assert.assertEquals(planQName, plan.getQueueName());
|
Assert.assertEquals(planQName, plan.getQueueName());
|
||||||
Assert.assertEquals(8192, plan.getTotalCapacity().getMemory());
|
Assert.assertEquals(8192, plan.getTotalCapacity().getMemorySize());
|
||||||
Assert.assertTrue(
|
Assert.assertTrue(
|
||||||
plan.getReservationAgent() instanceof AlignedPlannerWithGreedy);
|
plan.getReservationAgent() instanceof AlignedPlannerWithGreedy);
|
||||||
Assert
|
Assert
|
||||||
|
|
|
@ -349,24 +349,24 @@ public class TestRLESparseResourceAllocation {
|
||||||
// does not handle removal of "partial"
|
// does not handle removal of "partial"
|
||||||
// allocations correctly.
|
// allocations correctly.
|
||||||
Assert.assertEquals(102400, rleSparseVector.getCapacityAtTime(10)
|
Assert.assertEquals(102400, rleSparseVector.getCapacityAtTime(10)
|
||||||
.getMemory());
|
.getMemorySize());
|
||||||
Assert.assertEquals(0, rleSparseVector.getCapacityAtTime(13).getMemory());
|
Assert.assertEquals(0, rleSparseVector.getCapacityAtTime(13).getMemorySize());
|
||||||
Assert.assertEquals(0, rleSparseVector.getCapacityAtTime(19).getMemory());
|
Assert.assertEquals(0, rleSparseVector.getCapacityAtTime(19).getMemorySize());
|
||||||
Assert.assertEquals(102400, rleSparseVector.getCapacityAtTime(21)
|
Assert.assertEquals(102400, rleSparseVector.getCapacityAtTime(21)
|
||||||
.getMemory());
|
.getMemorySize());
|
||||||
Assert.assertEquals(2 * 102400, rleSparseVector.getCapacityAtTime(26)
|
Assert.assertEquals(2 * 102400, rleSparseVector.getCapacityAtTime(26)
|
||||||
.getMemory());
|
.getMemorySize());
|
||||||
|
|
||||||
ReservationInterval riRemove2 = new ReservationInterval(9, 13);
|
ReservationInterval riRemove2 = new ReservationInterval(9, 13);
|
||||||
rleSparseVector.removeInterval(riRemove2, rr);
|
rleSparseVector.removeInterval(riRemove2, rr);
|
||||||
LOG.info(rleSparseVector.toString());
|
LOG.info(rleSparseVector.toString());
|
||||||
|
|
||||||
Assert.assertEquals(0, rleSparseVector.getCapacityAtTime(11).getMemory());
|
Assert.assertEquals(0, rleSparseVector.getCapacityAtTime(11).getMemorySize());
|
||||||
Assert.assertEquals(-102400, rleSparseVector.getCapacityAtTime(9)
|
Assert.assertEquals(-102400, rleSparseVector.getCapacityAtTime(9)
|
||||||
.getMemory());
|
.getMemorySize());
|
||||||
Assert.assertEquals(0, rleSparseVector.getCapacityAtTime(13).getMemory());
|
Assert.assertEquals(0, rleSparseVector.getCapacityAtTime(13).getMemorySize());
|
||||||
Assert.assertEquals(102400, rleSparseVector.getCapacityAtTime(20)
|
Assert.assertEquals(102400, rleSparseVector.getCapacityAtTime(20)
|
||||||
.getMemory());
|
.getMemorySize());
|
||||||
|
|
||||||
}
|
}
|
||||||
|
|
||||||
|
|
|
@ -153,9 +153,9 @@ public class TestSimpleCapacityReplanner {
|
||||||
|
|
||||||
// check resources at each moment in time no more exceed capacity
|
// check resources at each moment in time no more exceed capacity
|
||||||
for (int i = 0; i < 20; i++) {
|
for (int i = 0; i < 20; i++) {
|
||||||
int tot = 0;
|
long tot = 0;
|
||||||
for (ReservationAllocation r : plan.getReservationsAtTime(i)) {
|
for (ReservationAllocation r : plan.getReservationsAtTime(i)) {
|
||||||
tot = r.getResourcesAtTime(i).getMemory();
|
tot = r.getResourcesAtTime(i).getMemorySize();
|
||||||
}
|
}
|
||||||
assertTrue(tot <= 70 * 1024);
|
assertTrue(tot <= 70 * 1024);
|
||||||
}
|
}
|
||||||
|
|
|
@ -137,38 +137,38 @@ public class TestAbstractYarnScheduler extends ParameterizedSchedulerTestBase {
|
||||||
Assert.assertEquals(6, expectedMaxMemory.length);
|
Assert.assertEquals(6, expectedMaxMemory.length);
|
||||||
|
|
||||||
Assert.assertEquals(0, scheduler.getNumClusterNodes());
|
Assert.assertEquals(0, scheduler.getNumClusterNodes());
|
||||||
int maxMemory = scheduler.getMaximumResourceCapability().getMemory();
|
long maxMemory = scheduler.getMaximumResourceCapability().getMemorySize();
|
||||||
Assert.assertEquals(expectedMaxMemory[0], maxMemory);
|
Assert.assertEquals(expectedMaxMemory[0], maxMemory);
|
||||||
|
|
||||||
RMNode node1 = MockNodes.newNodeInfo(
|
RMNode node1 = MockNodes.newNodeInfo(
|
||||||
0, Resources.createResource(node1MaxMemory), 1, "127.0.0.2");
|
0, Resources.createResource(node1MaxMemory), 1, "127.0.0.2");
|
||||||
scheduler.handle(new NodeAddedSchedulerEvent(node1));
|
scheduler.handle(new NodeAddedSchedulerEvent(node1));
|
||||||
Assert.assertEquals(1, scheduler.getNumClusterNodes());
|
Assert.assertEquals(1, scheduler.getNumClusterNodes());
|
||||||
maxMemory = scheduler.getMaximumResourceCapability().getMemory();
|
maxMemory = scheduler.getMaximumResourceCapability().getMemorySize();
|
||||||
Assert.assertEquals(expectedMaxMemory[1], maxMemory);
|
Assert.assertEquals(expectedMaxMemory[1], maxMemory);
|
||||||
|
|
||||||
scheduler.handle(new NodeRemovedSchedulerEvent(node1));
|
scheduler.handle(new NodeRemovedSchedulerEvent(node1));
|
||||||
Assert.assertEquals(0, scheduler.getNumClusterNodes());
|
Assert.assertEquals(0, scheduler.getNumClusterNodes());
|
||||||
maxMemory = scheduler.getMaximumResourceCapability().getMemory();
|
maxMemory = scheduler.getMaximumResourceCapability().getMemorySize();
|
||||||
Assert.assertEquals(expectedMaxMemory[2], maxMemory);
|
Assert.assertEquals(expectedMaxMemory[2], maxMemory);
|
||||||
|
|
||||||
RMNode node2 = MockNodes.newNodeInfo(
|
RMNode node2 = MockNodes.newNodeInfo(
|
||||||
0, Resources.createResource(node2MaxMemory), 2, "127.0.0.3");
|
0, Resources.createResource(node2MaxMemory), 2, "127.0.0.3");
|
||||||
scheduler.handle(new NodeAddedSchedulerEvent(node2));
|
scheduler.handle(new NodeAddedSchedulerEvent(node2));
|
||||||
Assert.assertEquals(1, scheduler.getNumClusterNodes());
|
Assert.assertEquals(1, scheduler.getNumClusterNodes());
|
||||||
maxMemory = scheduler.getMaximumResourceCapability().getMemory();
|
maxMemory = scheduler.getMaximumResourceCapability().getMemorySize();
|
||||||
Assert.assertEquals(expectedMaxMemory[3], maxMemory);
|
Assert.assertEquals(expectedMaxMemory[3], maxMemory);
|
||||||
|
|
||||||
RMNode node3 = MockNodes.newNodeInfo(
|
RMNode node3 = MockNodes.newNodeInfo(
|
||||||
0, Resources.createResource(node3MaxMemory), 3, "127.0.0.4");
|
0, Resources.createResource(node3MaxMemory), 3, "127.0.0.4");
|
||||||
scheduler.handle(new NodeAddedSchedulerEvent(node3));
|
scheduler.handle(new NodeAddedSchedulerEvent(node3));
|
||||||
Assert.assertEquals(2, scheduler.getNumClusterNodes());
|
Assert.assertEquals(2, scheduler.getNumClusterNodes());
|
||||||
maxMemory = scheduler.getMaximumResourceCapability().getMemory();
|
maxMemory = scheduler.getMaximumResourceCapability().getMemorySize();
|
||||||
Assert.assertEquals(expectedMaxMemory[4], maxMemory);
|
Assert.assertEquals(expectedMaxMemory[4], maxMemory);
|
||||||
|
|
||||||
scheduler.handle(new NodeRemovedSchedulerEvent(node3));
|
scheduler.handle(new NodeRemovedSchedulerEvent(node3));
|
||||||
Assert.assertEquals(1, scheduler.getNumClusterNodes());
|
Assert.assertEquals(1, scheduler.getNumClusterNodes());
|
||||||
maxMemory = scheduler.getMaximumResourceCapability().getMemory();
|
maxMemory = scheduler.getMaximumResourceCapability().getMemorySize();
|
||||||
Assert.assertEquals(expectedMaxMemory[5], maxMemory);
|
Assert.assertEquals(expectedMaxMemory[5], maxMemory);
|
||||||
|
|
||||||
scheduler.handle(new NodeRemovedSchedulerEvent(node2));
|
scheduler.handle(new NodeRemovedSchedulerEvent(node2));
|
||||||
|
@ -632,8 +632,8 @@ public class TestAbstractYarnScheduler extends ParameterizedSchedulerTestBase {
|
||||||
|
|
||||||
final Resource schedulerMaximumResourceCapability = scheduler
|
final Resource schedulerMaximumResourceCapability = scheduler
|
||||||
.getMaximumResourceCapability();
|
.getMaximumResourceCapability();
|
||||||
Assert.assertEquals(expectedMaximumResource.getMemory(),
|
Assert.assertEquals(expectedMaximumResource.getMemorySize(),
|
||||||
schedulerMaximumResourceCapability.getMemory());
|
schedulerMaximumResourceCapability.getMemorySize());
|
||||||
Assert.assertEquals(expectedMaximumResource.getVirtualCores(),
|
Assert.assertEquals(expectedMaximumResource.getVirtualCores(),
|
||||||
schedulerMaximumResourceCapability.getVirtualCores());
|
schedulerMaximumResourceCapability.getVirtualCores());
|
||||||
}
|
}
|
||||||
|
|
Some files were not shown because too many files have changed in this diff Show More
Loading…
Reference in New Issue