ソースを参照

YARN-5162. Fix Exceptions thrown during in registerAM call when Distributed Scheduling is Enabled (Hitesh Sharma via asuresh)

(cherry picked from commit 5b41b288d01b0124664ddf6a3d6b545058bcfe6f)
Arun Suresh 9 年 前
コミット
44cbf5b7f5

+ 1 - 1
hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/security/SchedulerSecurityInfo.java

@@ -41,7 +41,7 @@ public class SchedulerSecurityInfo extends SecurityInfo {
 
   @Override
   public TokenInfo getTokenInfo(Class<?> protocol, Configuration conf) {
-    if (!protocol.equals(ApplicationMasterProtocolPB.class)) {
+    if (!ApplicationMasterProtocolPB.class.isAssignableFrom(protocol)) {
       return null;
     }
     return new TokenInfo() {

+ 5 - 5
hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/src/main/java/org/apache/hadoop/yarn/server/api/DistributedSchedulerProtocolPB.java

@@ -21,16 +21,16 @@ package org.apache.hadoop.yarn.server.api;
 import org.apache.hadoop.classification.InterfaceAudience.Private;
 import org.apache.hadoop.classification.InterfaceStability.Unstable;
 import org.apache.hadoop.ipc.ProtocolInfo;
+import org.apache.hadoop.yarn.api.ApplicationMasterProtocolPB;
 import org.apache.hadoop.yarn.proto.ApplicationMasterProtocol.ApplicationMasterProtocolService;
-
-
-import org.apache.hadoop.yarn.proto.DistributedSchedulerProtocol;
+import org.apache.hadoop.yarn.proto.DistributedSchedulerProtocol.DistributedSchedulerProtocolService;
 
 @Private
 @Unstable
 @ProtocolInfo(protocolName = "org.apache.hadoop.yarn.server.api.DistributedSchedulerProtocolPB",
     protocolVersion = 1)
 public interface DistributedSchedulerProtocolPB extends
-    DistributedSchedulerProtocol.DistributedSchedulerProtocolService.BlockingInterface,
-    ApplicationMasterProtocolService.BlockingInterface {
+    DistributedSchedulerProtocolService.BlockingInterface,
+    ApplicationMasterProtocolService.BlockingInterface,
+    ApplicationMasterProtocolPB {
 }

+ 5 - 1
hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/src/main/java/org/apache/hadoop/yarn/server/api/protocolrecords/impl/pb/DistSchedRegisterResponsePBImpl.java

@@ -94,9 +94,13 @@ public class DistSchedRegisterResponsePBImpl extends DistSchedRegisterResponse {
           ProtoUtils.convertToProtoFormat(this.maxAllocatableCapability));
     }
     if (this.minAllocatableCapability != null) {
-      builder.setMaxAllocCapability(
+      builder.setMinAllocCapability(
           ProtoUtils.convertToProtoFormat(this.minAllocatableCapability));
     }
+    if (this.incrAllocatableCapability != null) {
+      builder.setIncrAllocCapability(
+          ProtoUtils.convertToProtoFormat(this.incrAllocatableCapability));
+    }
     if (this.registerApplicationMasterResponse != null) {
       builder.setRegisterResponse(
           ((RegisterApplicationMasterResponsePBImpl)

+ 1 - 0
hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/src/main/proto/distributed_scheduler_protocol.proto

@@ -34,5 +34,6 @@ import "yarn_server_common_service_protos.proto";
 
 service DistributedSchedulerProtocolService {
   rpc registerApplicationMasterForDistributedScheduling (RegisterApplicationMasterRequestProto) returns (DistSchedRegisterResponseProto);
+  rpc finishApplicationMaster (FinishApplicationMasterRequestProto) returns (FinishApplicationMasterResponseProto);
   rpc allocateForDistributedScheduling (AllocateRequestProto) returns (DistSchedAllocateResponseProto);
 }

+ 20 - 0
hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/TestDistributedSchedulingService.java

@@ -35,6 +35,7 @@ import org.apache.hadoop.yarn.api.protocolrecords.impl.pb
     .RegisterApplicationMasterRequestPBImpl;
 import org.apache.hadoop.yarn.api.protocolrecords.impl.pb
     .RegisterApplicationMasterResponsePBImpl;
+import org.apache.hadoop.yarn.api.records.Resource;
 import org.apache.hadoop.yarn.server.api.DistributedSchedulerProtocolPB;
 import org.apache.hadoop.yarn.api.protocolrecords.AllocateRequest;
 import org.apache.hadoop.yarn.api.protocolrecords.AllocateResponse;
@@ -132,6 +133,9 @@ public class TestDistributedSchedulingService {
             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;
           }
 
@@ -194,6 +198,13 @@ public class TestDistributedSchedulingService {
                     .newRecordInstance(RegisterApplicationMasterRequest.class))
                     .getProto()));
     Assert.assertEquals(54321l, dsRegResp.getContainerIdStart());
+    Assert.assertEquals(4,
+        dsRegResp.getMaxAllocatableCapabilty().getVirtualCores());
+    Assert.assertEquals(1024,
+        dsRegResp.getMinAllocatableCapabilty().getMemory());
+    Assert.assertEquals(2,
+        dsRegResp.getIncrAllocatableCapabilty().getVirtualCores());
+
     DistSchedAllocateResponse dsAllocResp =
         new DistSchedAllocateResponsePBImpl(
             dsProxy.allocateForDistributedScheduling(null,
@@ -201,5 +212,14 @@ public class TestDistributedSchedulingService {
                     .newRecordInstance(AllocateRequest.class)).getProto()));
     Assert.assertEquals(
         "h1", dsAllocResp.getNodesForScheduling().get(0).getHost());
+
+    FinishApplicationMasterResponse dsfinishResp =
+        new FinishApplicationMasterResponsePBImpl(
+            dsProxy.finishApplicationMaster(null,
+                ((FinishApplicationMasterRequestPBImpl) factory
+                    .newRecordInstance(FinishApplicationMasterRequest.class))
+                    .getProto()));
+    Assert.assertEquals(
+        false, dsfinishResp.getIsUnregistered());
   }
 }