瀏覽代碼

HDFS-4866. Merging change r1493300 from trunk to branch-2.

git-svn-id: https://svn.apache.org/repos/asf/hadoop/common/branches/branch-2@1493301 13f79535-47bb-0310-9956-ffa450edef68
Chris Nauroth 12 年之前
父節點
當前提交
2d8999dc35

+ 3 - 0
hadoop-hdfs-project/hadoop-hdfs/CHANGES.txt

@@ -38,6 +38,9 @@ Release 2.1.0-beta - UNRELEASED
     HDFS-2802. Add HDFS Snapshot feature.  (See breakdown of tasks below for
     subtasks and contributors)
 
+    HDFS-4866. Protocol buffer support cannot compile under C. (Arpit Agarwal via
+    cnauroth)
+
   NEW FEATURES
 
     HDFS-1804. Add a new block-volume device choosing policy that looks at

+ 5 - 3
hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/protocolPB/NamenodeProtocolServerSideTranslatorPB.java

@@ -157,11 +157,13 @@ public class NamenodeProtocolServerSideTranslatorPB implements
   }
 
   @Override
-  public RegisterResponseProto register(RpcController unused,
-      RegisterRequestProto request) throws ServiceException {
+  public RegisterResponseProto registerSubordinateNamenode(
+      RpcController unused, RegisterRequestProto request)
+      throws ServiceException {
     NamenodeRegistration reg;
     try {
-      reg = impl.register(PBHelper.convert(request.getRegistration()));
+      reg = impl.registerSubordinateNamenode(
+          PBHelper.convert(request.getRegistration()));
     } catch (IOException e) {
       throw new ServiceException(e);
     }

+ 4 - 3
hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/protocolPB/NamenodeProtocolTranslatorPB.java

@@ -167,12 +167,13 @@ public class NamenodeProtocolTranslatorPB implements NamenodeProtocol,
   }
 
   @Override
-  public NamenodeRegistration register(NamenodeRegistration registration)
-      throws IOException {
+  public NamenodeRegistration registerSubordinateNamenode(
+      NamenodeRegistration registration) throws IOException {
     RegisterRequestProto req = RegisterRequestProto.newBuilder()
         .setRegistration(PBHelper.convert(registration)).build();
     try {
-      return PBHelper.convert(rpcProxy.register(NULL_CONTROLLER, req)
+      return PBHelper.convert(
+          rpcProxy.registerSubordinateNamenode(NULL_CONTROLLER, req)
           .getRegistration());
     } catch (ServiceException e) {
       throw ProtobufHelper.getRemoteException(e);

+ 1 - 1
hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/BackupNode.java

@@ -361,7 +361,7 @@ public class BackupNode extends NameNode {
     NamenodeRegistration nnReg = null;
     while(!isStopRequested()) {
       try {
-        nnReg = namenode.register(getRegistration());
+        nnReg = namenode.registerSubordinateNamenode(getRegistration());
         break;
       } catch(SocketTimeoutException e) {  // name-node is busy
         LOG.info("Problem connecting to name-node: " + nnRpcAddress);

+ 2 - 2
hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/NameNodeRpcServer.java

@@ -374,8 +374,8 @@ class NameNodeRpcServer implements NamenodeProtocols {
   }
 
   @Override // NamenodeProtocol
-  public NamenodeRegistration register(NamenodeRegistration registration)
-  throws IOException {
+  public NamenodeRegistration registerSubordinateNamenode(
+      NamenodeRegistration registration) throws IOException {
     namesystem.checkSuperuserPrivilege();
     verifyLayoutVersion(registration.getVersion());
     NamenodeRegistration myRegistration = nn.setRegistration();

+ 2 - 2
hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/protocol/NamenodeProtocol.java

@@ -134,8 +134,8 @@ public interface NamenodeProtocol {
    * @return  {@link NamenodeRegistration} of the node,
    *          which this node has just registered with.
    */
-  public NamenodeRegistration register(NamenodeRegistration registration)
-  throws IOException;
+  public NamenodeRegistration registerSubordinateNamenode(
+      NamenodeRegistration registration) throws IOException;
 
   /**
    * A request to the active name-node to start a checkpoint.

+ 1 - 1
hadoop-hdfs-project/hadoop-hdfs/src/main/proto/NamenodeProtocol.proto

@@ -225,7 +225,7 @@ service NamenodeProtocolService {
   /**
    * Request to register a sub-ordinate namenode
    */
-  rpc register(RegisterRequestProto) returns(RegisterResponseProto);
+  rpc registerSubordinateNamenode(RegisterRequestProto) returns(RegisterResponseProto);
 
   /**
    * Request to start a checkpoint.