|
@@ -35,6 +35,11 @@ import org.apache.hadoop.yarn.api.protocolrecords.impl.pb
|
|
.RegisterApplicationMasterRequestPBImpl;
|
|
.RegisterApplicationMasterRequestPBImpl;
|
|
import org.apache.hadoop.yarn.api.protocolrecords.impl.pb
|
|
import org.apache.hadoop.yarn.api.protocolrecords.impl.pb
|
|
.RegisterApplicationMasterResponsePBImpl;
|
|
.RegisterApplicationMasterResponsePBImpl;
|
|
|
|
+import org.apache.hadoop.yarn.api.records.ApplicationAttemptId;
|
|
|
|
+import org.apache.hadoop.yarn.api.records.ApplicationId;
|
|
|
|
+import org.apache.hadoop.yarn.api.records.Container;
|
|
|
|
+import org.apache.hadoop.yarn.api.records.ContainerId;
|
|
|
|
+import org.apache.hadoop.yarn.api.records.ExecutionType;
|
|
import org.apache.hadoop.yarn.api.records.Resource;
|
|
import org.apache.hadoop.yarn.api.records.Resource;
|
|
import org.apache.hadoop.yarn.server.api.DistributedSchedulerProtocolPB;
|
|
import org.apache.hadoop.yarn.server.api.DistributedSchedulerProtocolPB;
|
|
import org.apache.hadoop.yarn.api.protocolrecords.AllocateRequest;
|
|
import org.apache.hadoop.yarn.api.protocolrecords.AllocateRequest;
|
|
@@ -66,6 +71,7 @@ import org.junit.Test;
|
|
import java.io.IOException;
|
|
import java.io.IOException;
|
|
import java.net.InetSocketAddress;
|
|
import java.net.InetSocketAddress;
|
|
import java.util.Arrays;
|
|
import java.util.Arrays;
|
|
|
|
+import java.util.List;
|
|
|
|
|
|
public class TestDistributedSchedulingService {
|
|
public class TestDistributedSchedulingService {
|
|
|
|
|
|
@@ -92,63 +98,13 @@ public class TestDistributedSchedulingService {
|
|
return new YarnConfiguration();
|
|
return new YarnConfiguration();
|
|
}
|
|
}
|
|
};
|
|
};
|
|
- DistributedSchedulingService service =
|
|
|
|
- new DistributedSchedulingService(rmContext, null) {
|
|
|
|
- @Override
|
|
|
|
- public RegisterApplicationMasterResponse registerApplicationMaster
|
|
|
|
- (RegisterApplicationMasterRequest request) throws
|
|
|
|
- YarnException, IOException {
|
|
|
|
- RegisterApplicationMasterResponse resp = factory.newRecordInstance(
|
|
|
|
- RegisterApplicationMasterResponse.class);
|
|
|
|
- // Dummy Entry to Assert that we get this object back
|
|
|
|
- resp.setQueue("dummyQueue");
|
|
|
|
- return resp;
|
|
|
|
- }
|
|
|
|
-
|
|
|
|
- @Override
|
|
|
|
- public FinishApplicationMasterResponse finishApplicationMaster
|
|
|
|
- (FinishApplicationMasterRequest request) throws YarnException,
|
|
|
|
- IOException {
|
|
|
|
- FinishApplicationMasterResponse resp = factory.newRecordInstance(
|
|
|
|
- FinishApplicationMasterResponse.class);
|
|
|
|
- // Dummy Entry to Assert that we get this object back
|
|
|
|
- resp.setIsUnregistered(false);
|
|
|
|
- return resp;
|
|
|
|
- }
|
|
|
|
-
|
|
|
|
- @Override
|
|
|
|
- public AllocateResponse allocate(AllocateRequest request) throws
|
|
|
|
- YarnException, IOException {
|
|
|
|
- AllocateResponse response = factory.newRecordInstance
|
|
|
|
- (AllocateResponse.class);
|
|
|
|
- response.setNumClusterNodes(12345);
|
|
|
|
- return response;
|
|
|
|
- }
|
|
|
|
-
|
|
|
|
- @Override
|
|
|
|
- public DistSchedRegisterResponse
|
|
|
|
- registerApplicationMasterForDistributedScheduling
|
|
|
|
- (RegisterApplicationMasterRequest request) throws
|
|
|
|
- YarnException, IOException {
|
|
|
|
- DistSchedRegisterResponse resp = factory.newRecordInstance(
|
|
|
|
- DistSchedRegisterResponse.class);
|
|
|
|
- resp.setContainerIdStart(54321l);
|
|
|
|
- resp.setMaxAllocatableCapabilty(Resource.newInstance(4096, 4));
|
|
|
|
- resp.setMinAllocatableCapabilty(Resource.newInstance(1024, 1));
|
|
|
|
- resp.setIncrAllocatableCapabilty(Resource.newInstance(2048, 2));
|
|
|
|
- return resp;
|
|
|
|
- }
|
|
|
|
-
|
|
|
|
- @Override
|
|
|
|
- public DistSchedAllocateResponse allocateForDistributedScheduling
|
|
|
|
- (AllocateRequest request) throws YarnException, IOException {
|
|
|
|
- DistSchedAllocateResponse resp =
|
|
|
|
- factory.newRecordInstance(DistSchedAllocateResponse.class);
|
|
|
|
- resp.setNodesForScheduling(
|
|
|
|
- Arrays.asList(NodeId.newInstance("h1", 1234)));
|
|
|
|
- return resp;
|
|
|
|
- }
|
|
|
|
- };
|
|
|
|
|
|
+ Container c = factory.newRecordInstance(Container.class);
|
|
|
|
+ c.setExecutionType(ExecutionType.OPPORTUNISTIC);
|
|
|
|
+ c.setId(
|
|
|
|
+ ContainerId.newContainerId(
|
|
|
|
+ ApplicationAttemptId.newInstance(
|
|
|
|
+ ApplicationId.newInstance(12345, 1), 2), 3));
|
|
|
|
+ DistributedSchedulingService service = createService(factory, rmContext, c);
|
|
Server server = service.getServer(rpc, conf, addr, null);
|
|
Server server = service.getServer(rpc, conf, addr, null);
|
|
server.start();
|
|
server.start();
|
|
|
|
|
|
@@ -180,6 +136,10 @@ public class TestDistributedSchedulingService {
|
|
((AllocateRequestPBImpl)factory
|
|
((AllocateRequestPBImpl)factory
|
|
.newRecordInstance(AllocateRequest.class)).getProto())
|
|
.newRecordInstance(AllocateRequest.class)).getProto())
|
|
);
|
|
);
|
|
|
|
+ List<Container> allocatedContainers = allocResp.getAllocatedContainers();
|
|
|
|
+ Assert.assertEquals(1, allocatedContainers.size());
|
|
|
|
+ Assert.assertEquals(ExecutionType.OPPORTUNISTIC,
|
|
|
|
+ allocatedContainers.get(0).getExecutionType());
|
|
Assert.assertEquals(12345, allocResp.getNumClusterNodes());
|
|
Assert.assertEquals(12345, allocResp.getNumClusterNodes());
|
|
|
|
|
|
|
|
|
|
@@ -222,4 +182,65 @@ public class TestDistributedSchedulingService {
|
|
Assert.assertEquals(
|
|
Assert.assertEquals(
|
|
false, dsfinishResp.getIsUnregistered());
|
|
false, dsfinishResp.getIsUnregistered());
|
|
}
|
|
}
|
|
|
|
+
|
|
|
|
+ private DistributedSchedulingService createService(final RecordFactory
|
|
|
|
+ factory, final RMContext rmContext, final Container c) {
|
|
|
|
+ return new DistributedSchedulingService(rmContext, null) {
|
|
|
|
+ @Override
|
|
|
|
+ public RegisterApplicationMasterResponse registerApplicationMaster(
|
|
|
|
+ RegisterApplicationMasterRequest request) throws
|
|
|
|
+ YarnException, IOException {
|
|
|
|
+ RegisterApplicationMasterResponse resp = factory.newRecordInstance(
|
|
|
|
+ RegisterApplicationMasterResponse.class);
|
|
|
|
+ // Dummy Entry to Assert that we get this object back
|
|
|
|
+ resp.setQueue("dummyQueue");
|
|
|
|
+ return resp;
|
|
|
|
+ }
|
|
|
|
+
|
|
|
|
+ @Override
|
|
|
|
+ public FinishApplicationMasterResponse finishApplicationMaster(
|
|
|
|
+ FinishApplicationMasterRequest request) throws YarnException,
|
|
|
|
+ IOException {
|
|
|
|
+ FinishApplicationMasterResponse resp = factory.newRecordInstance(
|
|
|
|
+ FinishApplicationMasterResponse.class);
|
|
|
|
+ // Dummy Entry to Assert that we get this object back
|
|
|
|
+ resp.setIsUnregistered(false);
|
|
|
|
+ return resp;
|
|
|
|
+ }
|
|
|
|
+
|
|
|
|
+ @Override
|
|
|
|
+ public AllocateResponse allocate(AllocateRequest request) throws
|
|
|
|
+ YarnException, IOException {
|
|
|
|
+ AllocateResponse response = factory.newRecordInstance(
|
|
|
|
+ AllocateResponse.class);
|
|
|
|
+ response.setNumClusterNodes(12345);
|
|
|
|
+ response.setAllocatedContainers(Arrays.asList(c));
|
|
|
|
+ return response;
|
|
|
|
+ }
|
|
|
|
+
|
|
|
|
+ @Override
|
|
|
|
+ public DistSchedRegisterResponse
|
|
|
|
+ registerApplicationMasterForDistributedScheduling(
|
|
|
|
+ RegisterApplicationMasterRequest request) throws
|
|
|
|
+ YarnException, IOException {
|
|
|
|
+ DistSchedRegisterResponse resp = factory.newRecordInstance(
|
|
|
|
+ DistSchedRegisterResponse.class);
|
|
|
|
+ resp.setContainerIdStart(54321L);
|
|
|
|
+ resp.setMaxAllocatableCapabilty(Resource.newInstance(4096, 4));
|
|
|
|
+ resp.setMinAllocatableCapabilty(Resource.newInstance(1024, 1));
|
|
|
|
+ resp.setIncrAllocatableCapabilty(Resource.newInstance(2048, 2));
|
|
|
|
+ return resp;
|
|
|
|
+ }
|
|
|
|
+
|
|
|
|
+ @Override
|
|
|
|
+ public DistSchedAllocateResponse allocateForDistributedScheduling(
|
|
|
|
+ AllocateRequest request) throws YarnException, IOException {
|
|
|
|
+ DistSchedAllocateResponse resp =
|
|
|
|
+ factory.newRecordInstance(DistSchedAllocateResponse.class);
|
|
|
|
+ resp.setNodesForScheduling(
|
|
|
|
+ Arrays.asList(NodeId.newInstance("h1", 1234)));
|
|
|
|
+ return resp;
|
|
|
|
+ }
|
|
|
|
+ };
|
|
|
|
+ }
|
|
}
|
|
}
|