Selaa lähdekoodia

YARN-440. Flatten RegisterNodeManagerResponse. Contributed by Xuan Gong.

git-svn-id: https://svn.apache.org/repos/asf/hadoop/common/trunk@1461256 13f79535-47bb-0310-9956-ffa450edef68
Siddharth Seth 12 vuotta sitten
vanhempi
commit
66e90b205a
18 muutettua tiedostoa jossa 82 lisäystä ja 246 poistoa
  1. 2 0
      hadoop-yarn-project/CHANGES.txt
  2. 9 4
      hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/src/main/java/org/apache/hadoop/yarn/server/api/protocolrecords/RegisterNodeManagerResponse.java
  3. 47 25
      hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/src/main/java/org/apache/hadoop/yarn/server/api/protocolrecords/impl/pb/RegisterNodeManagerResponsePBImpl.java
  4. 0 29
      hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/src/main/java/org/apache/hadoop/yarn/server/api/records/RegistrationResponse.java
  5. 0 133
      hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/src/main/java/org/apache/hadoop/yarn/server/api/records/impl/pb/RegistrationResponsePBImpl.java
  6. 0 6
      hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/src/main/proto/yarn_server_common_protos.proto
  7. 3 1
      hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/src/main/proto/yarn_server_common_service_protos.proto
  8. 6 7
      hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/NodeStatusUpdaterImpl.java
  9. 0 3
      hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/test/java/org/apache/hadoop/yarn/server/nodemanager/LocalRMInterface.java
  10. 0 5
      hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/test/java/org/apache/hadoop/yarn/server/nodemanager/MockNodeStatusUpdater.java
  11. 2 12
      hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/test/java/org/apache/hadoop/yarn/server/nodemanager/TestNodeStatusUpdater.java
  12. 3 8
      hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/ResourceTrackerService.java
  13. 4 4
      hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/MockNM.java
  14. 1 2
      hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/NodeManager.java
  15. 1 1
      hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/TestResourceTrackerService.java
  16. 1 1
      hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/resourcetracker/TestNMExpiry.java
  17. 1 3
      hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-tests/src/test/java/org/apache/hadoop/yarn/server/MiniYARNCluster.java
  18. 2 2
      hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-tests/src/test/java/org/apache/hadoop/yarn/server/TestRMNMSecretKeys.java

+ 2 - 0
hadoop-yarn-project/CHANGES.txt

@@ -64,6 +64,8 @@ Release 2.0.5-beta - UNRELEASED
 
     YARN-439. Flatten NodeHeartbeatResponse. (Xuan Gong via sseth)
 
+    YARN-440. Flatten RegisterNodeManagerResponse. (Xuan Gong via sseth)
+
   NEW FEATURES
 
   IMPROVEMENTS

+ 9 - 4
hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/src/main/java/org/apache/hadoop/yarn/server/api/protocolrecords/RegisterNodeManagerResponse.java

@@ -18,11 +18,16 @@
 
 package org.apache.hadoop.yarn.server.api.protocolrecords;
 
-import org.apache.hadoop.yarn.server.api.records.RegistrationResponse;
+import org.apache.hadoop.yarn.server.api.records.MasterKey;
+import org.apache.hadoop.yarn.server.api.records.NodeAction;
 
 public interface RegisterNodeManagerResponse {
-  public abstract RegistrationResponse getRegistrationResponse();
-  
-  public abstract void setRegistrationResponse(RegistrationResponse registrationResponse);
+  MasterKey getMasterKey();
+
+  void setMasterKey(MasterKey secretKey);
+
+  NodeAction getNodeAction();
+
+  void setNodeAction(NodeAction nodeAction);
 
 }

+ 47 - 25
hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/src/main/java/org/apache/hadoop/yarn/server/api/protocolrecords/impl/pb/RegisterNodeManagerResponsePBImpl.java

@@ -20,12 +20,14 @@ package org.apache.hadoop.yarn.server.api.protocolrecords.impl.pb;
 
 
 import org.apache.hadoop.yarn.api.records.ProtoBase;
-import org.apache.hadoop.yarn.proto.YarnServerCommonProtos.RegistrationResponseProto;
+import org.apache.hadoop.yarn.proto.YarnServerCommonProtos.MasterKeyProto;
+import org.apache.hadoop.yarn.proto.YarnServerCommonProtos.NodeActionProto;
 import org.apache.hadoop.yarn.proto.YarnServerCommonServiceProtos.RegisterNodeManagerResponseProto;
 import org.apache.hadoop.yarn.proto.YarnServerCommonServiceProtos.RegisterNodeManagerResponseProtoOrBuilder;
 import org.apache.hadoop.yarn.server.api.protocolrecords.RegisterNodeManagerResponse;
-import org.apache.hadoop.yarn.server.api.records.RegistrationResponse;
-import org.apache.hadoop.yarn.server.api.records.impl.pb.RegistrationResponsePBImpl;
+import org.apache.hadoop.yarn.server.api.records.MasterKey;
+import org.apache.hadoop.yarn.server.api.records.NodeAction;
+import org.apache.hadoop.yarn.server.api.records.impl.pb.MasterKeyPBImpl;
 
 
     
@@ -34,7 +36,7 @@ public class RegisterNodeManagerResponsePBImpl extends ProtoBase<RegisterNodeMan
   RegisterNodeManagerResponseProto.Builder builder = null;
   boolean viaProto = false;
   
-  private RegistrationResponse registartionResponse = null;
+  private MasterKey masterKey = null;
   
   private boolean rebuild = false;
   
@@ -56,9 +58,8 @@ public class RegisterNodeManagerResponsePBImpl extends ProtoBase<RegisterNodeMan
   }
 
   private void mergeLocalToBuilder() {
-    if (this.registartionResponse != null) {
-      builder.setRegistrationResponse(convertToProtoFormat(this.registartionResponse));
-      this.registartionResponse = null;
+    if (this.masterKey != null) {
+      builder.setMasterKey(convertToProtoFormat(this.masterKey));
     }
   }
 
@@ -77,39 +78,60 @@ public class RegisterNodeManagerResponsePBImpl extends ProtoBase<RegisterNodeMan
     }
     viaProto = false;
   }
-    
-  
+
   @Override
-  public RegistrationResponse getRegistrationResponse() {
+  public MasterKey getMasterKey() {
     RegisterNodeManagerResponseProtoOrBuilder p = viaProto ? proto : builder;
-    if (this.registartionResponse != null) {
-      return this.registartionResponse;
+    if (this.masterKey != null) {
+      return this.masterKey;
     }
-    if (!p.hasRegistrationResponse()) {
+    if (!p.hasMasterKey()) {
       return null;
     }
-    this.registartionResponse = convertFromProtoFormat(p.getRegistrationResponse());
-    rebuild = true;
-    return this.registartionResponse;
+    this.masterKey = convertFromProtoFormat(p.getMasterKey());
+    return this.masterKey;
   }
 
   @Override
-  public void setRegistrationResponse(RegistrationResponse registrationResponse) {
+  public void setMasterKey(MasterKey masterKey) {
     maybeInitBuilder();
-    if (registrationResponse == null) 
-      builder.clearRegistrationResponse();
-    this.registartionResponse = registrationResponse;
-    rebuild = true;
+    if (masterKey == null)
+      builder.clearMasterKey();
+    this.masterKey = masterKey;
   }
 
-  private RegistrationResponsePBImpl convertFromProtoFormat(RegistrationResponseProto p) {
-    return new RegistrationResponsePBImpl(p);
+  @Override
+  public NodeAction getNodeAction() {
+    RegisterNodeManagerResponseProtoOrBuilder p = viaProto ? proto : builder;
+    if(!p.hasNodeAction()) {
+      return null;
+    }
+    return convertFromProtoFormat(p.getNodeAction());
   }
 
-  private RegistrationResponseProto convertToProtoFormat(RegistrationResponse t) {
-    return ((RegistrationResponsePBImpl)t).getProto();
+  @Override
+  public void setNodeAction(NodeAction nodeAction) {
+    maybeInitBuilder();
+    if (nodeAction == null) {
+      builder.clearNodeAction();
+      return;
+    }
+    builder.setNodeAction(convertToProtoFormat(nodeAction));
   }
 
+  private NodeAction convertFromProtoFormat(NodeActionProto p) {
+    return  NodeAction.valueOf(p.name());
+  }
 
+  private NodeActionProto convertToProtoFormat(NodeAction t) {
+    return NodeActionProto.valueOf(t.name());
+  }
 
+  private MasterKeyPBImpl convertFromProtoFormat(MasterKeyProto p) {
+    return new MasterKeyPBImpl(p);
+  }
+
+  private MasterKeyProto convertToProtoFormat(MasterKey t) {
+    return ((MasterKeyPBImpl)t).getProto();
+  }
 }  

+ 0 - 29
hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/src/main/java/org/apache/hadoop/yarn/server/api/records/RegistrationResponse.java

@@ -1,29 +0,0 @@
-/**
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- *     http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-package org.apache.hadoop.yarn.server.api.records;
-
-public interface RegistrationResponse {
-
-  MasterKey getMasterKey();
-  
-  void setMasterKey(MasterKey secretKey);
-  
-  NodeAction getNodeAction();
-  
-  void setNodeAction(NodeAction nodeAction);
-}

+ 0 - 133
hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/src/main/java/org/apache/hadoop/yarn/server/api/records/impl/pb/RegistrationResponsePBImpl.java

@@ -1,133 +0,0 @@
-/**
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- *     http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-
-package org.apache.hadoop.yarn.server.api.records.impl.pb;
-
-
-import org.apache.hadoop.yarn.api.records.ProtoBase;
-import org.apache.hadoop.yarn.proto.YarnServerCommonProtos.MasterKeyProto;
-import org.apache.hadoop.yarn.proto.YarnServerCommonProtos.NodeActionProto;
-import org.apache.hadoop.yarn.proto.YarnServerCommonProtos.RegistrationResponseProto;
-import org.apache.hadoop.yarn.proto.YarnServerCommonProtos.RegistrationResponseProtoOrBuilder;
-import org.apache.hadoop.yarn.server.api.records.MasterKey;
-import org.apache.hadoop.yarn.server.api.records.NodeAction;
-import org.apache.hadoop.yarn.server.api.records.RegistrationResponse;
-
-public class RegistrationResponsePBImpl extends
-    ProtoBase<RegistrationResponseProto> implements RegistrationResponse {
-  RegistrationResponseProto proto = RegistrationResponseProto.getDefaultInstance();
-  RegistrationResponseProto.Builder builder = null;
-  boolean viaProto = false;
-  
-  private MasterKey masterKey = null;
-  
-  public RegistrationResponsePBImpl() {
-    builder = RegistrationResponseProto.newBuilder();
-  }
-
-  public RegistrationResponsePBImpl(RegistrationResponseProto proto) {
-    this.proto = proto;
-    viaProto = true;
-  }
-  
-  public RegistrationResponseProto getProto() {
-    
-      mergeLocalToProto();
-    proto = viaProto ? proto : builder.build();
-    viaProto = true;
-    return proto;
-  }
-
-  private void mergeLocalToBuilder() {
-    if (this.masterKey != null) {
-      builder.setMasterKey(convertToProtoFormat(this.masterKey));
-    }
-  }
-
-  private void mergeLocalToProto() {
-    if (viaProto) 
-      maybeInitBuilder();
-    mergeLocalToBuilder();
-    proto = builder.build();
-    
-    viaProto = true;
-  }
-
-  private void maybeInitBuilder() {
-    if (viaProto || builder == null) {
-      builder = RegistrationResponseProto.newBuilder(proto);
-    }
-    viaProto = false;
-  }
-
-  @Override
-  public MasterKey getMasterKey() {
-    RegistrationResponseProtoOrBuilder p = viaProto ? proto : builder;
-    if (this.masterKey != null) {
-      return this.masterKey;
-    }
-    if (!p.hasMasterKey()) {
-      return null;
-    }
-    this.masterKey = convertFromProtoFormat(p.getMasterKey());
-    return this.masterKey;
-  }
-
-  @Override
-  public void setMasterKey(MasterKey masterKey) {
-    maybeInitBuilder();
-    if (masterKey == null) 
-      builder.clearMasterKey();
-    this.masterKey = masterKey;
-  }
-  
-  @Override
-  public NodeAction getNodeAction() {
-    RegistrationResponseProtoOrBuilder p = viaProto ? proto : builder;
-    if(!p.hasNodeAction()) {
-      return null;
-    }
-    return convertFromProtoFormat(p.getNodeAction());
-  }
-
-  @Override
-  public void setNodeAction(NodeAction nodeAction) {
-    maybeInitBuilder();
-    if (nodeAction == null) {
-      builder.clearNodeAction();
-      return;
-    }
-    builder.setNodeAction(convertToProtoFormat(nodeAction));
-  }
-  
-  private NodeAction convertFromProtoFormat(NodeActionProto p) {
-    return  NodeAction.valueOf(p.name());
-  }
-  
-  private NodeActionProto convertToProtoFormat(NodeAction t) {
-    return NodeActionProto.valueOf(t.name());
-  }
-
-  private MasterKeyPBImpl convertFromProtoFormat(MasterKeyProto p) {
-    return new MasterKeyPBImpl(p);
-  }
-
-  private MasterKeyProto convertToProtoFormat(MasterKey t) {
-    return ((MasterKeyPBImpl)t).getProto();
-  }
-}  

+ 0 - 6
hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/src/main/proto/yarn_server_common_protos.proto

@@ -42,9 +42,3 @@ message MasterKeyProto {
   optional bytes bytes = 2;
 }
 
-message RegistrationResponseProto {
-  optional MasterKeyProto master_key = 1;
-  optional NodeActionProto nodeAction = 2;
-}
-
-

+ 3 - 1
hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/src/main/proto/yarn_server_common_service_protos.proto

@@ -29,8 +29,10 @@ message RegisterNodeManagerRequestProto {
   optional int32 http_port = 3;
   optional ResourceProto resource = 4;
 }
+
 message RegisterNodeManagerResponseProto {
-  optional RegistrationResponseProto registration_response = 1;
+  optional MasterKeyProto master_key = 1;
+  optional NodeActionProto nodeAction = 2;
 }
 
 message NodeHeartbeatRequestProto {

+ 6 - 7
hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/NodeStatusUpdaterImpl.java

@@ -52,10 +52,10 @@ import org.apache.hadoop.yarn.server.api.ResourceTracker;
 import org.apache.hadoop.yarn.server.api.protocolrecords.NodeHeartbeatRequest;
 import org.apache.hadoop.yarn.server.api.protocolrecords.NodeHeartbeatResponse;
 import org.apache.hadoop.yarn.server.api.protocolrecords.RegisterNodeManagerRequest;
+import org.apache.hadoop.yarn.server.api.protocolrecords.RegisterNodeManagerResponse;
 import org.apache.hadoop.yarn.server.api.records.MasterKey;
 import org.apache.hadoop.yarn.server.api.records.NodeAction;
 import org.apache.hadoop.yarn.server.api.records.NodeStatus;
-import org.apache.hadoop.yarn.server.api.records.RegistrationResponse;
 import org.apache.hadoop.yarn.server.nodemanager.containermanager.container.Container;
 import org.apache.hadoop.yarn.server.nodemanager.metrics.NodeManagerMetrics;
 import org.apache.hadoop.yarn.service.AbstractService;
@@ -234,7 +234,7 @@ public class NodeStatusUpdaterImpl extends AbstractService implements
     request.setHttpPort(this.httpPort);
     request.setResource(this.totalResource);
     request.setNodeId(this.nodeId);
-    RegistrationResponse regResponse;
+    RegisterNodeManagerResponse regNMResponse;
 
     while(true) {
       try {
@@ -242,9 +242,8 @@ public class NodeStatusUpdaterImpl extends AbstractService implements
         LOG.info("Connecting to ResourceManager at " + this.rmAddress
             + ". current no. of attempts is " + rmRetryCount);
         this.resourceTracker = getRMClient();
-        regResponse =
-            this.resourceTracker.registerNodeManager(request)
-                .getRegistrationResponse();
+        regNMResponse =
+            this.resourceTracker.registerNodeManager(request);
         break;
       } catch(Throwable e) {
         LOG.warn("Trying to connect to ResourceManager, " +
@@ -267,13 +266,13 @@ public class NodeStatusUpdaterImpl extends AbstractService implements
       }
     }
     // if the Resourcemanager instructs NM to shutdown.
-    if (NodeAction.SHUTDOWN.equals(regResponse.getNodeAction())) {
+    if (NodeAction.SHUTDOWN.equals(regNMResponse.getNodeAction())) {
       throw new YarnException(
           "Recieved SHUTDOWN signal from Resourcemanager ,Registration of NodeManager failed");
     }
 
     if (UserGroupInformation.isSecurityEnabled()) {
-      MasterKey masterKey = regResponse.getMasterKey();
+      MasterKey masterKey = regNMResponse.getMasterKey();
       // do this now so that its set before we start heartbeating to RM
       LOG.info("Security enabled - updating secret keys now");
       // It is expected that status updater is started by this point and

+ 0 - 3
hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/test/java/org/apache/hadoop/yarn/server/nodemanager/LocalRMInterface.java

@@ -26,7 +26,6 @@ import org.apache.hadoop.yarn.server.api.protocolrecords.NodeHeartbeatRequest;
 import org.apache.hadoop.yarn.server.api.protocolrecords.NodeHeartbeatResponse;
 import org.apache.hadoop.yarn.server.api.protocolrecords.RegisterNodeManagerRequest;
 import org.apache.hadoop.yarn.server.api.protocolrecords.RegisterNodeManagerResponse;
-import org.apache.hadoop.yarn.server.api.records.RegistrationResponse;
 
 public class LocalRMInterface implements ResourceTracker {
 
@@ -34,9 +33,7 @@ public class LocalRMInterface implements ResourceTracker {
   
   @Override
   public RegisterNodeManagerResponse registerNodeManager(RegisterNodeManagerRequest request) throws YarnRemoteException {
-    RegistrationResponse registrationResponse = recordFactory.newRecordInstance(RegistrationResponse.class);
     RegisterNodeManagerResponse response = recordFactory.newRecordInstance(RegisterNodeManagerResponse.class);
-    response.setRegistrationResponse(registrationResponse);
     return response;
   }
 

+ 0 - 5
hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/test/java/org/apache/hadoop/yarn/server/nodemanager/MockNodeStatusUpdater.java

@@ -30,7 +30,6 @@ import org.apache.hadoop.yarn.server.api.protocolrecords.NodeHeartbeatResponse;
 import org.apache.hadoop.yarn.server.api.protocolrecords.RegisterNodeManagerRequest;
 import org.apache.hadoop.yarn.server.api.protocolrecords.RegisterNodeManagerResponse;
 import org.apache.hadoop.yarn.server.api.records.NodeStatus;
-import org.apache.hadoop.yarn.server.api.records.RegistrationResponse;
 import org.apache.hadoop.yarn.server.nodemanager.metrics.NodeManagerMetrics;
 
 /**
@@ -62,12 +61,8 @@ public class MockNodeStatusUpdater extends NodeStatusUpdaterImpl {
     @Override
     public RegisterNodeManagerResponse registerNodeManager(
         RegisterNodeManagerRequest request) throws YarnRemoteException {
-      RegistrationResponse regResponse = recordFactory
-          .newRecordInstance(RegistrationResponse.class);
-
       RegisterNodeManagerResponse response = recordFactory
           .newRecordInstance(RegisterNodeManagerResponse.class);
-      response.setRegistrationResponse(regResponse);
       return response;
     }
 

+ 2 - 12
hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/test/java/org/apache/hadoop/yarn/server/nodemanager/TestNodeStatusUpdater.java

@@ -58,7 +58,6 @@ import org.apache.hadoop.yarn.server.api.protocolrecords.RegisterNodeManagerRequ
 import org.apache.hadoop.yarn.server.api.protocolrecords.RegisterNodeManagerResponse;
 import org.apache.hadoop.yarn.server.api.records.NodeAction;
 import org.apache.hadoop.yarn.server.api.records.NodeStatus;
-import org.apache.hadoop.yarn.server.api.records.RegistrationResponse;
 import org.apache.hadoop.yarn.server.nodemanager.containermanager.ContainerManagerImpl;
 import org.apache.hadoop.yarn.server.nodemanager.containermanager.application.Application;
 import org.apache.hadoop.yarn.server.nodemanager.containermanager.container.Container;
@@ -123,12 +122,9 @@ public class TestNodeStatusUpdater {
       Assert.assertEquals(NetUtils.getHostPortString(expected), nodeId.toString());
       Assert.assertEquals(5 * 1024, resource.getMemory());
       registeredNodes.add(nodeId);
-      RegistrationResponse regResponse = recordFactory
-          .newRecordInstance(RegistrationResponse.class);
 
       RegisterNodeManagerResponse response = recordFactory
           .newRecordInstance(RegisterNodeManagerResponse.class);
-      response.setRegistrationResponse(regResponse);
       return response;
     }
 
@@ -320,10 +316,7 @@ public class TestNodeStatusUpdater {
       
       RegisterNodeManagerResponse response = recordFactory
           .newRecordInstance(RegisterNodeManagerResponse.class);
-      RegistrationResponse regResponse = recordFactory
-      .newRecordInstance(RegistrationResponse.class);
-      regResponse.setNodeAction(registerNodeAction );
-      response.setRegistrationResponse(regResponse);
+      response.setNodeAction(registerNodeAction );
       return response;
     }
     @Override
@@ -358,10 +351,7 @@ public class TestNodeStatusUpdater {
 
       RegisterNodeManagerResponse response =
           recordFactory.newRecordInstance(RegisterNodeManagerResponse.class);
-      RegistrationResponse regResponse =
-          recordFactory.newRecordInstance(RegistrationResponse.class);
-      regResponse.setNodeAction(registerNodeAction);
-      response.setRegistrationResponse(regResponse);
+      response.setNodeAction(registerNodeAction);
       return response;
     }
 

+ 3 - 8
hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/ResourceTrackerService.java

@@ -42,7 +42,6 @@ import org.apache.hadoop.yarn.server.api.protocolrecords.RegisterNodeManagerResp
 import org.apache.hadoop.yarn.server.api.records.MasterKey;
 import org.apache.hadoop.yarn.server.api.records.NodeAction;
 import org.apache.hadoop.yarn.server.api.records.NodeStatus;
-import org.apache.hadoop.yarn.server.api.records.RegistrationResponse;
 import org.apache.hadoop.yarn.server.resourcemanager.recovery.RMStateStore.RMState;
 import org.apache.hadoop.yarn.server.resourcemanager.rmnode.RMNode;
 import org.apache.hadoop.yarn.server.resourcemanager.rmnode.RMNodeEvent;
@@ -150,22 +149,19 @@ public class ResourceTrackerService extends AbstractService implements
 
     RegisterNodeManagerResponse response = recordFactory
         .newRecordInstance(RegisterNodeManagerResponse.class);
-    RegistrationResponse regResponse = recordFactory
-        .newRecordInstance(RegistrationResponse.class);
 
     // Check if this node is a 'valid' node
     if (!this.nodesListManager.isValidNode(host)) {
       LOG.info("Disallowed NodeManager from  " + host
           + ", Sending SHUTDOWN signal to the NodeManager.");
-      regResponse.setNodeAction(NodeAction.SHUTDOWN);
-      response.setRegistrationResponse(regResponse);
+      response.setNodeAction(NodeAction.SHUTDOWN);
       return response;
     }
 
     if (isSecurityEnabled()) {
       MasterKey nextMasterKeyForNode =
           this.containerTokenSecretManager.getCurrentKey();
-      regResponse.setMasterKey(nextMasterKeyForNode);
+      response.setMasterKey(nextMasterKeyForNode);
     }
 
     RMNode rmNode = new RMNodeImpl(nodeId, rmContext, host, cmPort, httpPort,
@@ -188,8 +184,7 @@ public class ResourceTrackerService extends AbstractService implements
         + " httpPort: " + httpPort + ") " + "registered with capability: "
         + capability + ", assigned nodeId " + nodeId);
 
-    regResponse.setNodeAction(NodeAction.NORMAL);
-    response.setRegistrationResponse(regResponse);
+    response.setNodeAction(NodeAction.NORMAL);
     return response;
   }
 

+ 4 - 4
hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/MockNM.java

@@ -35,9 +35,9 @@ import org.apache.hadoop.yarn.api.records.Resource;
 import org.apache.hadoop.yarn.server.api.protocolrecords.NodeHeartbeatRequest;
 import org.apache.hadoop.yarn.server.api.protocolrecords.NodeHeartbeatResponse;
 import org.apache.hadoop.yarn.server.api.protocolrecords.RegisterNodeManagerRequest;
+import org.apache.hadoop.yarn.server.api.protocolrecords.RegisterNodeManagerResponse;
 import org.apache.hadoop.yarn.server.api.records.MasterKey;
 import org.apache.hadoop.yarn.server.api.records.NodeStatus;
-import org.apache.hadoop.yarn.server.api.records.RegistrationResponse;
 import org.apache.hadoop.yarn.util.BuilderUtils;
 import org.apache.hadoop.yarn.util.Records;
 
@@ -79,7 +79,7 @@ public class MockNM {
     nodeHeartbeat(conts, true);
   }
 
-  public RegistrationResponse registerNode() throws Exception {
+  public RegisterNodeManagerResponse registerNode() throws Exception {
     RegisterNodeManagerRequest req = Records.newRecord(
         RegisterNodeManagerRequest.class);
     req.setNodeId(nodeId);
@@ -87,8 +87,8 @@ public class MockNM {
     Resource resource = Records.newRecord(Resource.class);
     resource.setMemory(memory);
     req.setResource(resource);
-    RegistrationResponse registrationResponse =
-        resourceTracker.registerNodeManager(req).getRegistrationResponse();
+    RegisterNodeManagerResponse registrationResponse =
+        resourceTracker.registerNodeManager(req);
     this.currentMasterKey = registrationResponse.getMasterKey();
     return registrationResponse;
   }

+ 1 - 2
hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/NodeManager.java

@@ -96,8 +96,7 @@ public class NodeManager implements ContainerManager {
     request.setNodeId(this.nodeId);
     request.setResource(capability);
     request.setNodeId(this.nodeId);
-    resourceTrackerService.registerNodeManager(request)
-        .getRegistrationResponse();
+    resourceTrackerService.registerNodeManager(request);
     this.schedulerNode = new FiCaSchedulerNode(rmContext.getRMNodes().get(
         this.nodeId));
    

+ 1 - 1
hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/TestResourceTrackerService.java

@@ -241,7 +241,7 @@ public class TestResourceTrackerService {
     req.setHttpPort(1234);
     // trying to register a invalid node.
     RegisterNodeManagerResponse response = resourceTrackerService.registerNodeManager(req);
-    Assert.assertEquals(NodeAction.SHUTDOWN,response.getRegistrationResponse().getNodeAction());
+    Assert.assertEquals(NodeAction.SHUTDOWN,response.getNodeAction());
   }
 
   @Test

+ 1 - 1
hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/resourcetracker/TestNMExpiry.java

@@ -166,7 +166,7 @@ public class TestNMExpiry {
     request3.setHttpPort(0);
     request3.setResource(capability);
     resourceTrackerService
-        .registerNodeManager(request3).getRegistrationResponse();
+        .registerNodeManager(request3);
 
     /* test to see if hostanme 3 does not expire */
     stopT = false;

+ 1 - 3
hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-tests/src/test/java/org/apache/hadoop/yarn/server/MiniYARNCluster.java

@@ -377,9 +377,7 @@ public class MiniYARNCluster extends CompositeService {
               RegisterNodeManagerResponse response = recordFactory.
                   newRecordInstance(RegisterNodeManagerResponse.class);
               try {
-                response.setRegistrationResponse(rt
-                    .registerNodeManager(request)
-                    .getRegistrationResponse());
+                response = rt.registerNodeManager(request);
               } catch (IOException ioe) {
                 LOG.info("Exception in node registration from "
                     + request.getNodeId().toString(), ioe);

+ 2 - 2
hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-tests/src/test/java/org/apache/hadoop/yarn/server/TestRMNMSecretKeys.java

@@ -28,8 +28,8 @@ import org.apache.hadoop.yarn.conf.YarnConfiguration;
 import org.apache.hadoop.yarn.event.Dispatcher;
 import org.apache.hadoop.yarn.event.DrainDispatcher;
 import org.apache.hadoop.yarn.server.api.protocolrecords.NodeHeartbeatResponse;
+import org.apache.hadoop.yarn.server.api.protocolrecords.RegisterNodeManagerResponse;
 import org.apache.hadoop.yarn.server.api.records.MasterKey;
-import org.apache.hadoop.yarn.server.api.records.RegistrationResponse;
 import org.apache.hadoop.yarn.server.resourcemanager.MockNM;
 import org.apache.hadoop.yarn.server.resourcemanager.ResourceManager;
 import org.apache.hadoop.yarn.server.resourcemanager.security.RMContainerTokenSecretManager;
@@ -62,7 +62,7 @@ public class TestRMNMSecretKeys {
     rm.start();
 
     MockNM nm = new MockNM("host:1234", 3072, rm.getResourceTrackerService());
-    RegistrationResponse registrationResponse = nm.registerNode();
+    RegisterNodeManagerResponse registrationResponse = nm.registerNode();
     MasterKey masterKey = registrationResponse.getMasterKey();
     Assert.assertNotNull("Registration should cause a key-update!", masterKey);
     dispatcher.await();