|
@@ -26,7 +26,9 @@ import static org.mockito.Matchers.anyBoolean;
|
|
|
import static org.mockito.Mockito.mock;
|
|
|
import static org.mockito.Mockito.when;
|
|
|
|
|
|
+import java.io.ByteArrayInputStream;
|
|
|
import java.io.IOException;
|
|
|
+import java.io.InputStream;
|
|
|
import java.net.InetSocketAddress;
|
|
|
import java.security.PrivilegedAction;
|
|
|
import java.util.Arrays;
|
|
@@ -35,6 +37,7 @@ import java.util.HashMap;
|
|
|
import java.util.Map;
|
|
|
import java.util.Set;
|
|
|
|
|
|
+import com.google.common.collect.ImmutableMap;
|
|
|
import org.apache.commons.logging.Log;
|
|
|
import org.apache.commons.logging.LogFactory;
|
|
|
import org.apache.hadoop.conf.Configuration;
|
|
@@ -42,6 +45,7 @@ import org.apache.hadoop.security.Credentials;
|
|
|
import org.apache.hadoop.security.UserGroupInformation;
|
|
|
import org.apache.hadoop.security.token.Token;
|
|
|
import org.apache.hadoop.security.token.TokenIdentifier;
|
|
|
+import org.apache.hadoop.yarn.LocalConfigurationProvider;
|
|
|
import org.apache.hadoop.yarn.api.ApplicationMasterProtocol;
|
|
|
import org.apache.hadoop.yarn.api.protocolrecords.AllocateRequest;
|
|
|
import org.apache.hadoop.yarn.api.protocolrecords.RegisterApplicationMasterRequest;
|
|
@@ -63,8 +67,10 @@ import org.apache.hadoop.yarn.event.EventHandler;
|
|
|
import org.apache.hadoop.yarn.exceptions.InvalidLabelResourceRequestException;
|
|
|
import org.apache.hadoop.yarn.exceptions.InvalidResourceBlacklistRequestException;
|
|
|
import org.apache.hadoop.yarn.exceptions.InvalidResourceRequestException;
|
|
|
+import org.apache.hadoop.yarn.exceptions.YarnException;
|
|
|
import org.apache.hadoop.yarn.ipc.YarnRPC;
|
|
|
import org.apache.hadoop.yarn.nodelabels.CommonNodeLabelsManager;
|
|
|
+import org.apache.hadoop.yarn.resourcetypes.ResourceTypesTestHelper;
|
|
|
import org.apache.hadoop.yarn.server.resourcemanager.MockNM;
|
|
|
import org.apache.hadoop.yarn.server.resourcemanager.RMContext;
|
|
|
import org.apache.hadoop.yarn.server.resourcemanager.TestAMAuthorization.MockRMWithAMS;
|
|
@@ -83,20 +89,79 @@ import org.apache.hadoop.yarn.util.Records;
|
|
|
import org.apache.hadoop.yarn.util.resource.DefaultResourceCalculator;
|
|
|
import org.apache.hadoop.yarn.util.resource.DominantResourceCalculator;
|
|
|
import org.apache.hadoop.yarn.util.resource.ResourceCalculator;
|
|
|
+import org.apache.hadoop.yarn.util.resource.ResourceUtils;
|
|
|
import org.apache.hadoop.yarn.util.resource.Resources;
|
|
|
import org.junit.Assert;
|
|
|
+import org.junit.Before;
|
|
|
+import org.junit.Rule;
|
|
|
import org.junit.Test;
|
|
|
|
|
|
import com.google.common.collect.ImmutableSet;
|
|
|
import com.google.common.collect.Sets;
|
|
|
+import org.junit.rules.ExpectedException;
|
|
|
|
|
|
public class TestSchedulerUtils {
|
|
|
|
|
|
private static final Log LOG = LogFactory.getLog(TestSchedulerUtils.class);
|
|
|
-
|
|
|
+ private static Resource configuredMaxAllocation;
|
|
|
+
|
|
|
+ private static class CustomResourceTypesConfigurationProvider
|
|
|
+ extends LocalConfigurationProvider {
|
|
|
+
|
|
|
+ @Override
|
|
|
+ public InputStream getConfigurationInputStream(Configuration bootstrapConf,
|
|
|
+ String name) throws YarnException, IOException {
|
|
|
+ if (YarnConfiguration.RESOURCE_TYPES_CONFIGURATION_FILE.equals(name)) {
|
|
|
+ return new ByteArrayInputStream(
|
|
|
+ ("<configuration>\n" +
|
|
|
+ " <property>\n" +
|
|
|
+ " <name>yarn.resource-types</name>\n" +
|
|
|
+ " <value>custom-resource-1," +
|
|
|
+ "custom-resource-2,custom-resource-3</value>\n" +
|
|
|
+ " </property>\n" +
|
|
|
+ " <property>\n" +
|
|
|
+ " <name>yarn.resource-types" +
|
|
|
+ ".custom-resource-1.units</name>\n" +
|
|
|
+ " <value>G</value>\n" +
|
|
|
+ " </property>\n" +
|
|
|
+ " <property>\n" +
|
|
|
+ " <name>yarn.resource-types" +
|
|
|
+ ".custom-resource-2.units</name>\n" +
|
|
|
+ " <value>G</value>\n" +
|
|
|
+ " </property>\n" +
|
|
|
+ "</configuration>\n").getBytes());
|
|
|
+ } else {
|
|
|
+ return super.getConfigurationInputStream(bootstrapConf, name);
|
|
|
+ }
|
|
|
+ }
|
|
|
+ }
|
|
|
private RMContext rmContext = getMockRMContext();
|
|
|
+
|
|
|
private static YarnConfiguration conf = new YarnConfiguration();
|
|
|
|
|
|
+ @Rule
|
|
|
+ public ExpectedException exception = ExpectedException.none();
|
|
|
+
|
|
|
+ private void initResourceTypes() {
|
|
|
+ Configuration yarnConf = new Configuration();
|
|
|
+ yarnConf.set(YarnConfiguration.RM_CONFIGURATION_PROVIDER_CLASS,
|
|
|
+ CustomResourceTypesConfigurationProvider.class.getName());
|
|
|
+ ResourceUtils.resetResourceTypes(yarnConf);
|
|
|
+ }
|
|
|
+
|
|
|
+ @Before
|
|
|
+ public void setUp() {
|
|
|
+ initResourceTypes();
|
|
|
+ //this needs to be initialized after initResourceTypes is called
|
|
|
+ configuredMaxAllocation = Resource.newInstance(8192, 4,
|
|
|
+ ImmutableMap.<String,
|
|
|
+ Long>builder()
|
|
|
+ .put("custom-resource-1", Long.MAX_VALUE)
|
|
|
+ .put("custom-resource-2", Long.MAX_VALUE)
|
|
|
+ .put("custom-resource-3", Long.MAX_VALUE)
|
|
|
+ .build());
|
|
|
+ }
|
|
|
+
|
|
|
@Test (timeout = 30000)
|
|
|
public void testNormalizeRequest() {
|
|
|
ResourceCalculator resourceCalculator = new DefaultResourceCalculator();
|
|
@@ -150,16 +215,18 @@ public class TestSchedulerUtils {
|
|
|
// multiple of minMemory > maxMemory, then reduce to maxMemory
|
|
|
SchedulerUtils.normalizeRequest(ask, resourceCalculator, minResource,
|
|
|
maxResource);
|
|
|
- assertEquals(maxResource.getMemorySize(), ask.getCapability().getMemorySize());
|
|
|
+ assertEquals(maxResource.getMemorySize(),
|
|
|
+ ask.getCapability().getMemorySize());
|
|
|
|
|
|
// ask is more than max
|
|
|
maxResource = Resources.createResource(maxMemory, 0);
|
|
|
ask.setCapability(Resources.createResource(maxMemory + 100));
|
|
|
SchedulerUtils.normalizeRequest(ask, resourceCalculator, minResource,
|
|
|
maxResource);
|
|
|
- assertEquals(maxResource.getMemorySize(), ask.getCapability().getMemorySize());
|
|
|
+ assertEquals(maxResource.getMemorySize(),
|
|
|
+ ask.getCapability().getMemorySize());
|
|
|
}
|
|
|
-
|
|
|
+
|
|
|
@Test (timeout = 30000)
|
|
|
public void testNormalizeRequestWithDominantResourceCalculator() {
|
|
|
ResourceCalculator resourceCalculator = new DominantResourceCalculator();
|
|
@@ -201,10 +268,11 @@ public class TestSchedulerUtils {
|
|
|
Set<String> queueAccessibleNodeLabels = Sets.newHashSet();
|
|
|
QueueInfo queueInfo = mock(QueueInfo.class);
|
|
|
when(queueInfo.getQueueName()).thenReturn("queue");
|
|
|
- when(queueInfo.getAccessibleNodeLabels()).thenReturn(queueAccessibleNodeLabels);
|
|
|
+ when(queueInfo.getAccessibleNodeLabels())
|
|
|
+ .thenReturn(queueAccessibleNodeLabels);
|
|
|
when(scheduler.getQueueInfo(any(String.class), anyBoolean(), anyBoolean()))
|
|
|
.thenReturn(queueInfo);
|
|
|
-
|
|
|
+
|
|
|
Resource maxResource = Resources.createResource(
|
|
|
YarnConfiguration.DEFAULT_RM_SCHEDULER_MAXIMUM_ALLOCATION_MB,
|
|
|
YarnConfiguration.DEFAULT_RM_SCHEDULER_MAXIMUM_ALLOCATION_VCORES);
|
|
@@ -363,7 +431,7 @@ public class TestSchedulerUtils {
|
|
|
rmContext.getNodeLabelManager().removeFromClusterNodeLabels(
|
|
|
Arrays.asList("x"));
|
|
|
}
|
|
|
- Assert.assertTrue("InvalidLabelResourceRequestException excpeted",
|
|
|
+ Assert.assertTrue("InvalidLabelResourceRequestException expected",
|
|
|
invalidlabelexception);
|
|
|
// queue is "*", always succeeded
|
|
|
try {
|
|
@@ -610,11 +678,9 @@ public class TestSchedulerUtils {
|
|
|
|
|
|
// more than max vcores
|
|
|
try {
|
|
|
- Resource resource =
|
|
|
- Resources
|
|
|
- .createResource(
|
|
|
- YarnConfiguration.DEFAULT_RM_SCHEDULER_MINIMUM_ALLOCATION_MB,
|
|
|
- YarnConfiguration.DEFAULT_RM_SCHEDULER_MAXIMUM_ALLOCATION_VCORES + 1);
|
|
|
+ Resource resource = Resources.createResource(
|
|
|
+ YarnConfiguration.DEFAULT_RM_SCHEDULER_MINIMUM_ALLOCATION_MB,
|
|
|
+ YarnConfiguration.DEFAULT_RM_SCHEDULER_MAXIMUM_ALLOCATION_VCORES + 1);
|
|
|
ResourceRequest resReq =
|
|
|
BuilderUtils.newResourceRequest(mock(Priority.class),
|
|
|
ResourceRequest.ANY, resource, 1);
|
|
@@ -648,10 +714,10 @@ public class TestSchedulerUtils {
|
|
|
waitForLaunchedState(attempt);
|
|
|
|
|
|
// Create a client to the RM.
|
|
|
- final Configuration conf = rm.getConfig();
|
|
|
- final YarnRPC rpc = YarnRPC.create(conf);
|
|
|
+ final Configuration yarnConf = rm.getConfig();
|
|
|
+ final YarnRPC rpc = YarnRPC.create(yarnConf);
|
|
|
|
|
|
- UserGroupInformation currentUser =
|
|
|
+ UserGroupInformation currentUser =
|
|
|
UserGroupInformation.createRemoteUser(applicationAttemptId.toString());
|
|
|
Credentials credentials = containerManager.getContainerCredentials();
|
|
|
final InetSocketAddress rmBindAddress =
|
|
@@ -665,7 +731,7 @@ public class TestSchedulerUtils {
|
|
|
@Override
|
|
|
public ApplicationMasterProtocol run() {
|
|
|
return (ApplicationMasterProtocol) rpc.getProxy(
|
|
|
- ApplicationMasterProtocol.class, rmBindAddress, conf);
|
|
|
+ ApplicationMasterProtocol.class, rmBindAddress, yarnConf);
|
|
|
}
|
|
|
});
|
|
|
|
|
@@ -775,6 +841,127 @@ public class TestSchedulerUtils {
|
|
|
}
|
|
|
}
|
|
|
|
|
|
+ @Test
|
|
|
+ public void testCustomResourceRequestedUnitIsSmallerThanAvailableUnit()
|
|
|
+ throws InvalidResourceRequestException {
|
|
|
+ Resource requestedResource =
|
|
|
+ ResourceTypesTestHelper.newResource(1, 1,
|
|
|
+ ImmutableMap.of("custom-resource-1", "11"));
|
|
|
+
|
|
|
+ Resource availableResource =
|
|
|
+ ResourceTypesTestHelper.newResource(1, 1,
|
|
|
+ ImmutableMap.of("custom-resource-1", "0G"));
|
|
|
+
|
|
|
+ exception.expect(InvalidResourceRequestException.class);
|
|
|
+ exception.expectMessage(InvalidResourceRequestExceptionMessageGenerator
|
|
|
+ .create().withRequestedResourceType("custom-resource-1")
|
|
|
+ .withRequestedResource(requestedResource)
|
|
|
+ .withAvailableAllocation(availableResource)
|
|
|
+ .withMaxAllocation(configuredMaxAllocation).build());
|
|
|
+
|
|
|
+ SchedulerUtils.checkResourceRequestAgainstAvailableResource(
|
|
|
+ requestedResource, availableResource);
|
|
|
+ }
|
|
|
+
|
|
|
+ @Test
|
|
|
+ public void testCustomResourceRequestedUnitIsSmallerThanAvailableUnit2() {
|
|
|
+ Resource requestedResource =
|
|
|
+ ResourceTypesTestHelper.newResource(1, 1,
|
|
|
+ ImmutableMap.of("custom-resource-1", "11"));
|
|
|
+
|
|
|
+ Resource availableResource =
|
|
|
+ ResourceTypesTestHelper.newResource(1, 1,
|
|
|
+ ImmutableMap.of("custom-resource-1", "1G"));
|
|
|
+
|
|
|
+ try {
|
|
|
+ SchedulerUtils.checkResourceRequestAgainstAvailableResource(
|
|
|
+ requestedResource, availableResource);
|
|
|
+ } catch (InvalidResourceRequestException e) {
|
|
|
+ fail(String.format(
|
|
|
+ "Resource request should be accepted. Requested: %s, available: %s",
|
|
|
+ requestedResource, availableResource));
|
|
|
+ }
|
|
|
+ }
|
|
|
+
|
|
|
+ @Test
|
|
|
+ public void testCustomResourceRequestedUnitIsGreaterThanAvailableUnit()
|
|
|
+ throws InvalidResourceRequestException {
|
|
|
+ Resource requestedResource =
|
|
|
+ ResourceTypesTestHelper.newResource(1, 1,
|
|
|
+ ImmutableMap.of("custom-resource-1", "1M"));
|
|
|
+
|
|
|
+ Resource availableResource = ResourceTypesTestHelper.newResource(1, 1,
|
|
|
+ ImmutableMap.<String, String> builder().put("custom-resource-1", "120k")
|
|
|
+ .build());
|
|
|
+
|
|
|
+ exception.expect(InvalidResourceRequestException.class);
|
|
|
+ exception.expectMessage(InvalidResourceRequestExceptionMessageGenerator
|
|
|
+ .create().withRequestedResourceType("custom-resource-1")
|
|
|
+ .withRequestedResource(requestedResource)
|
|
|
+ .withAvailableAllocation(availableResource)
|
|
|
+ .withMaxAllocation(configuredMaxAllocation).build());
|
|
|
+ SchedulerUtils.checkResourceRequestAgainstAvailableResource(
|
|
|
+ requestedResource, availableResource);
|
|
|
+ }
|
|
|
+
|
|
|
+ @Test
|
|
|
+ public void testCustomResourceRequestedUnitIsGreaterThanAvailableUnit2() {
|
|
|
+ Resource requestedResource = ResourceTypesTestHelper.newResource(1, 1,
|
|
|
+ ImmutableMap.<String, String> builder().put("custom-resource-1", "11M")
|
|
|
+ .build());
|
|
|
+
|
|
|
+ Resource availableResource =
|
|
|
+ ResourceTypesTestHelper.newResource(1, 1,
|
|
|
+ ImmutableMap.of("custom-resource-1", "1G"));
|
|
|
+
|
|
|
+ try {
|
|
|
+ SchedulerUtils.checkResourceRequestAgainstAvailableResource(
|
|
|
+ requestedResource, availableResource);
|
|
|
+ } catch (InvalidResourceRequestException e) {
|
|
|
+ fail(String.format(
|
|
|
+ "Resource request should be accepted. Requested: %s, available: %s",
|
|
|
+ requestedResource, availableResource));
|
|
|
+ }
|
|
|
+ }
|
|
|
+
|
|
|
+ @Test
|
|
|
+ public void testCustomResourceRequestedUnitIsSameAsAvailableUnit() {
|
|
|
+ Resource requestedResource = ResourceTypesTestHelper.newResource(1, 1,
|
|
|
+ ImmutableMap.of("custom-resource-1", "11M"));
|
|
|
+
|
|
|
+ Resource availableResource = ResourceTypesTestHelper.newResource(1, 1,
|
|
|
+ ImmutableMap.of("custom-resource-1", "100M"));
|
|
|
+
|
|
|
+ try {
|
|
|
+ SchedulerUtils.checkResourceRequestAgainstAvailableResource(
|
|
|
+ requestedResource, availableResource);
|
|
|
+ } catch (InvalidResourceRequestException e) {
|
|
|
+ fail(String.format(
|
|
|
+ "Resource request should be accepted. Requested: %s, available: %s",
|
|
|
+ requestedResource, availableResource));
|
|
|
+ }
|
|
|
+ }
|
|
|
+
|
|
|
+ @Test
|
|
|
+ public void testCustomResourceRequestedUnitIsSameAsAvailableUnit2()
|
|
|
+ throws InvalidResourceRequestException {
|
|
|
+ Resource requestedResource = ResourceTypesTestHelper.newResource(1, 1,
|
|
|
+ ImmutableMap.of("custom-resource-1", "110M"));
|
|
|
+
|
|
|
+ Resource availableResource = ResourceTypesTestHelper.newResource(1, 1,
|
|
|
+ ImmutableMap.of("custom-resource-1", "100M"));
|
|
|
+
|
|
|
+ exception.expect(InvalidResourceRequestException.class);
|
|
|
+ exception.expectMessage(InvalidResourceRequestExceptionMessageGenerator
|
|
|
+ .create().withRequestedResourceType("custom-resource-1")
|
|
|
+ .withRequestedResource(requestedResource)
|
|
|
+ .withAvailableAllocation(availableResource)
|
|
|
+ .withMaxAllocation(configuredMaxAllocation).build());
|
|
|
+
|
|
|
+ SchedulerUtils.checkResourceRequestAgainstAvailableResource(
|
|
|
+ requestedResource, availableResource);
|
|
|
+ }
|
|
|
+
|
|
|
public static void waitSchedulerApplicationAttemptStopped(
|
|
|
AbstractYarnScheduler ys,
|
|
|
ApplicationAttemptId attemptId) throws InterruptedException {
|
|
@@ -801,8 +988,7 @@ public class TestSchedulerUtils {
|
|
|
public static SchedulerApplication<SchedulerApplicationAttempt>
|
|
|
verifyAppAddedAndRemovedFromScheduler(
|
|
|
Map<ApplicationId, SchedulerApplication<SchedulerApplicationAttempt>> applications,
|
|
|
- EventHandler<SchedulerEvent> handler, String queueName)
|
|
|
- throws Exception {
|
|
|
+ EventHandler<SchedulerEvent> handler, String queueName) {
|
|
|
|
|
|
ApplicationId appId =
|
|
|
ApplicationId.newInstance(System.currentTimeMillis(), 1);
|
|
@@ -832,4 +1018,60 @@ public class TestSchedulerUtils {
|
|
|
when(rmContext.getNodeLabelManager()).thenReturn(nlm);
|
|
|
return rmContext;
|
|
|
}
|
|
|
+
|
|
|
+ private static class InvalidResourceRequestExceptionMessageGenerator {
|
|
|
+
|
|
|
+ private StringBuilder sb;
|
|
|
+ private Resource requestedResource;
|
|
|
+ private Resource availableAllocation;
|
|
|
+ private Resource configuredMaxAllowedAllocation;
|
|
|
+ private String resourceType;
|
|
|
+
|
|
|
+ InvalidResourceRequestExceptionMessageGenerator(StringBuilder sb) {
|
|
|
+ this.sb = sb;
|
|
|
+ }
|
|
|
+
|
|
|
+ public static InvalidResourceRequestExceptionMessageGenerator create() {
|
|
|
+ return new InvalidResourceRequestExceptionMessageGenerator(
|
|
|
+ new StringBuilder());
|
|
|
+ }
|
|
|
+
|
|
|
+ InvalidResourceRequestExceptionMessageGenerator withRequestedResource(
|
|
|
+ Resource r) {
|
|
|
+ this.requestedResource = r;
|
|
|
+ return this;
|
|
|
+ }
|
|
|
+
|
|
|
+ InvalidResourceRequestExceptionMessageGenerator withRequestedResourceType(
|
|
|
+ String rt) {
|
|
|
+ this.resourceType = rt;
|
|
|
+ return this;
|
|
|
+ }
|
|
|
+
|
|
|
+ InvalidResourceRequestExceptionMessageGenerator withAvailableAllocation(
|
|
|
+ Resource r) {
|
|
|
+ this.availableAllocation = r;
|
|
|
+ return this;
|
|
|
+ }
|
|
|
+
|
|
|
+ InvalidResourceRequestExceptionMessageGenerator withMaxAllocation(
|
|
|
+ Resource r) {
|
|
|
+ this.configuredMaxAllowedAllocation = r;
|
|
|
+ return this;
|
|
|
+ }
|
|
|
+
|
|
|
+ public String build() {
|
|
|
+ return sb
|
|
|
+ .append("Invalid resource request, requested resource type=[")
|
|
|
+ .append(resourceType).append("]")
|
|
|
+ .append(" < 0 or greater than maximum allowed allocation. ")
|
|
|
+ .append("Requested resource=").append(requestedResource).append(", ")
|
|
|
+ .append("maximum allowed allocation=").append(availableAllocation)
|
|
|
+ .append(", please note that maximum allowed allocation is calculated "
|
|
|
+ + "by scheduler based on maximum resource of " +
|
|
|
+ "registered NodeManagers, which might be less than " +
|
|
|
+ "configured maximum allocation=")
|
|
|
+ .append(configuredMaxAllowedAllocation).toString();
|
|
|
+ }
|
|
|
+ }
|
|
|
}
|