Sfoglia il codice sorgente

Merged r1295016 and r1295017 from trunk for HADOOP-7994 and HDFS-3030

git-svn-id: https://svn.apache.org/repos/asf/hadoop/common/branches/branch-0.23@1295422 13f79535-47bb-0310-9956-ffa450edef68
Jitendra Nath Pandey 13 anni fa
parent
commit
d0de078933
43 ha cambiato i file con 81 aggiunte e 800 eliminazioni
  1. 3 0
      hadoop-common-project/hadoop-common/CHANGES.txt
  2. 1 1
      hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/security/RefreshUserMappingsProtocol.java
  3. 1 1
      hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/security/authorize/RefreshAuthorizationPolicyProtocol.java
  4. 1 1
      hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/tools/GetUserMappingsProtocol.java
  5. 3 0
      hadoop-hdfs-project/hadoop-hdfs/CHANGES.txt
  6. 1 1
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/protocol/ClientDatanodeProtocol.java
  7. 1 2
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/protocol/ClientProtocol.java
  8. 1 13
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/protocolPB/ClientDatanodeProtocolPB.java
  9. 0 50
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/protocolPB/ClientDatanodeProtocolServerSideTranslatorPB.java
  10. 0 13
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/protocolPB/ClientDatanodeProtocolTranslatorPB.java
  11. 1 13
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/protocolPB/ClientNamenodeProtocolPB.java
  12. 0 54
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/protocolPB/ClientNamenodeProtocolServerSideTranslatorPB.java
  13. 27 43
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/protocolPB/ClientNamenodeProtocolTranslatorPB.java
  14. 0 13
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/protocolPB/DatanodeProtocolClientSideTranslatorPB.java
  15. 1 13
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/protocolPB/DatanodeProtocolPB.java
  16. 0 39
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/protocolPB/DatanodeProtocolServerSideTranslatorPB.java
  17. 0 13
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/protocolPB/GetUserMappingsProtocolClientSideTranslatorPB.java
  18. 1 13
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/protocolPB/GetUserMappingsProtocolPB.java
  19. 0 39
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/protocolPB/GetUserMappingsProtocolServerSideTranslatorPB.java
  20. 1 13
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/protocolPB/InterDatanodeProtocolPB.java
  21. 0 52
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/protocolPB/InterDatanodeProtocolServerSideTranslatorPB.java
  22. 0 15
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/protocolPB/InterDatanodeProtocolTranslatorPB.java
  23. 3 14
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/protocolPB/JournalProtocolPB.java
  24. 0 51
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/protocolPB/JournalProtocolServerSideTranslatorPB.java
  25. 0 13
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/protocolPB/JournalProtocolTranslatorPB.java
  26. 1 12
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/protocolPB/NamenodeProtocolPB.java
  27. 0 47
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/protocolPB/NamenodeProtocolServerSideTranslatorPB.java
  28. 0 13
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/protocolPB/NamenodeProtocolTranslatorPB.java
  29. 0 13
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/protocolPB/RefreshAuthorizationPolicyProtocolClientSideTranslatorPB.java
  30. 1 13
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/protocolPB/RefreshAuthorizationPolicyProtocolPB.java
  31. 0 39
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/protocolPB/RefreshAuthorizationPolicyProtocolServerSideTranslatorPB.java
  32. 0 13
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/protocolPB/RefreshUserMappingsProtocolClientSideTranslatorPB.java
  33. 1 13
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/protocolPB/RefreshUserMappingsProtocolPB.java
  34. 0 39
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/protocolPB/RefreshUserMappingsProtocolServerSideTranslatorPB.java
  35. 6 22
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/DataNode.java
  36. 0 10
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/BackupNode.java
  37. 12 44
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/NameNodeRpcServer.java
  38. 1 2
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/protocol/DatanodeProtocol.java
  39. 1 1
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/protocol/InterDatanodeProtocol.java
  40. 1 1
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/protocol/JournalProtocol.java
  41. 1 1
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/protocol/NamenodeProtocol.java
  42. 9 23
      hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/security/TestClientProtocolWithDelegationToken.java
  43. 1 14
      hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/security/token/block/TestBlockToken.java

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

@@ -62,6 +62,9 @@ Release 0.23.3 - UNRELEASED
     HADOOP-8118.  In metrics2.util.MBeans, change log level to trace for the
     stack trace of InstanceAlreadyExistsException.  (szetszwo)
 
+    HADOOP-7994. Remove getProtocolVersion and getProtocolSignature from the
+    client side translator and server side implementation. (jitendra)
+
   OPTIMIZATIONS
 
   BUG FIXES

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

@@ -33,7 +33,7 @@ import org.apache.hadoop.security.KerberosInfo;
     serverPrincipal=CommonConfigurationKeys.HADOOP_SECURITY_SERVICE_USER_NAME_KEY)
 @InterfaceAudience.LimitedPrivate({"HDFS", "MapReduce"})
 @InterfaceStability.Evolving
-public interface RefreshUserMappingsProtocol extends VersionedProtocol {
+public interface RefreshUserMappingsProtocol {
   
   /**
    * Version 1: Initial version.

+ 1 - 1
hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/security/authorize/RefreshAuthorizationPolicyProtocol.java

@@ -32,7 +32,7 @@ import org.apache.hadoop.security.KerberosInfo;
     serverPrincipal=CommonConfigurationKeys.HADOOP_SECURITY_SERVICE_USER_NAME_KEY)
 @InterfaceAudience.LimitedPrivate({"HDFS", "MapReduce"})
 @InterfaceStability.Evolving
-public interface RefreshAuthorizationPolicyProtocol extends VersionedProtocol {
+public interface RefreshAuthorizationPolicyProtocol {
   
   /**
    * Version 1: Initial version

+ 1 - 1
hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/tools/GetUserMappingsProtocol.java

@@ -29,7 +29,7 @@ import org.apache.hadoop.ipc.VersionedProtocol;
  */
 @InterfaceAudience.LimitedPrivate({"HDFS", "MapReduce"})
 @InterfaceStability.Evolving
-public interface GetUserMappingsProtocol extends VersionedProtocol {
+public interface GetUserMappingsProtocol {
   
   /**
    * Version 1: Initial version.

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

@@ -93,6 +93,9 @@ Release 0.23.3 - UNRELEASED
     HDFS-2992. Edit log failure trace should include transaction ID of
     error.  (Colin Patrick McCabe via eli)
 
+    HDFS-3030. Remove getProtocolVersion and getProtocolSignature from 
+    translators. (jitendra)
+
   OPTIMIZATIONS
 
   BUG FIXES

+ 1 - 1
hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/protocol/ClientDatanodeProtocol.java

@@ -38,7 +38,7 @@ import org.apache.hadoop.security.token.TokenInfo;
 @KerberosInfo(
     serverPrincipal = DFSConfigKeys.DFS_DATANODE_USER_NAME_KEY)
 @TokenInfo(BlockTokenSelector.class)
-public interface ClientDatanodeProtocol extends VersionedProtocol {
+public interface ClientDatanodeProtocol {
   /**
    * Until version 9, this class ClientDatanodeProtocol served as both
    * the client interface to the DN AND the RPC protocol used to 

+ 1 - 2
hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/protocol/ClientProtocol.java

@@ -39,7 +39,6 @@ import org.apache.hadoop.hdfs.server.namenode.NotReplicatedYetException;
 import org.apache.hadoop.hdfs.server.namenode.SafeModeException;
 import org.apache.hadoop.io.EnumSetWritable;
 import org.apache.hadoop.io.Text;
-import org.apache.hadoop.ipc.VersionedProtocol;
 import org.apache.hadoop.security.AccessControlException;
 import org.apache.hadoop.security.KerberosInfo;
 import org.apache.hadoop.security.token.Token;
@@ -59,7 +58,7 @@ import org.apache.hadoop.hdfs.security.token.delegation.DelegationTokenSelector;
 @KerberosInfo(
     serverPrincipal = DFSConfigKeys.DFS_NAMENODE_USER_NAME_KEY)
 @TokenInfo(DelegationTokenSelector.class)
-public interface ClientProtocol extends VersionedProtocol {
+public interface ClientProtocol {
 
   /**
    * Until version 69, this class ClientProtocol served as both

+ 1 - 13
hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/protocolPB/ClientDatanodeProtocolPB.java

@@ -17,15 +17,11 @@
  */
 package org.apache.hadoop.hdfs.protocolPB;
 
-import java.io.IOException;
-
 import org.apache.hadoop.classification.InterfaceAudience;
 import org.apache.hadoop.hdfs.DFSConfigKeys;
 import org.apache.hadoop.hdfs.protocol.proto.ClientDatanodeProtocolProtos.ClientDatanodeProtocolService;
-import org.apache.hadoop.hdfs.protocolR23Compatible.ProtocolSignatureWritable;
 import org.apache.hadoop.hdfs.security.token.block.BlockTokenSelector;
 import org.apache.hadoop.ipc.ProtocolInfo;
-import org.apache.hadoop.ipc.VersionedProtocol;
 import org.apache.hadoop.security.KerberosInfo;
 import org.apache.hadoop.security.token.TokenInfo;
 
@@ -37,13 +33,5 @@ import org.apache.hadoop.security.token.TokenInfo;
     protocolVersion = 1)
 @InterfaceAudience.Private
 public interface ClientDatanodeProtocolPB extends
-    ClientDatanodeProtocolService.BlockingInterface, VersionedProtocol {
-  
-  /**
-   * This method is defined to get the protocol signature using 
-   * ProtocolSignatureWritable - suffix of 2 to the method name
-   * avoids conflict.
-   */
-  public ProtocolSignatureWritable getProtocolSignature2(String protocol,
-      long clientVersion, int clientMethodsHash) throws IOException;
+    ClientDatanodeProtocolService.BlockingInterface {
 }

+ 0 - 50
hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/protocolPB/ClientDatanodeProtocolServerSideTranslatorPB.java

@@ -30,10 +30,6 @@ import org.apache.hadoop.hdfs.protocol.proto.ClientDatanodeProtocolProtos.GetRep
 import org.apache.hadoop.hdfs.protocol.proto.ClientDatanodeProtocolProtos.GetReplicaVisibleLengthResponseProto;
 import org.apache.hadoop.hdfs.protocol.proto.ClientDatanodeProtocolProtos.RefreshNamenodesRequestProto;
 import org.apache.hadoop.hdfs.protocol.proto.ClientDatanodeProtocolProtos.RefreshNamenodesResponseProto;
-import org.apache.hadoop.hdfs.protocolR23Compatible.ProtocolSignatureWritable;
-import org.apache.hadoop.ipc.ProtocolSignature;
-import org.apache.hadoop.ipc.RPC;
-import org.apache.hadoop.ipc.VersionedProtocol;
 
 import com.google.protobuf.RpcController;
 import com.google.protobuf.ServiceException;
@@ -110,50 +106,4 @@ public class ClientDatanodeProtocolServerSideTranslatorPB implements
         .setLocalPath(resp.getBlockPath()).setLocalMetaPath(resp.getMetaPath())
         .build();
   }
-
-  @Override
-  public long getProtocolVersion(String protocol, long clientVersion)
-      throws IOException {
-    return RPC.getProtocolVersion(ClientDatanodeProtocolPB.class);
-  }
-
-  /**
-   * The client side will redirect getProtocolSignature to
-   * getProtocolSignature2.
-   * 
-   * However the RPC layer below on the Server side will call getProtocolVersion
-   * and possibly in the future getProtocolSignature. Hence we still implement
-   * it even though the end client will never call this method.
-   * 
-   * @see VersionedProtocol#getProtocolVersion
-   */
-  @Override
-  public ProtocolSignature getProtocolSignature(String protocol,
-      long clientVersion, int clientMethodsHash) throws IOException {
-    /**
-     * Don't forward this to the server. The protocol version and signature is
-     * that of {@link ClientDatanodeProtocol}
-     */
-    if (!protocol.equals(RPC.getProtocolName(ClientDatanodeProtocol.class))) {
-      throw new IOException("Namenode Serverside implements " +
-          RPC.getProtocolName(ClientDatanodeProtocol.class) +
-          ". The following requested protocol is unknown: " + protocol);
-    }
-
-    return ProtocolSignature.getProtocolSignature(clientMethodsHash,
-        RPC.getProtocolVersion(ClientDatanodeProtocolPB.class),
-        ClientDatanodeProtocolPB.class);
-  }
-
-
-  @Override
-  public ProtocolSignatureWritable getProtocolSignature2(String protocol,
-      long clientVersion, int clientMethodsHash) throws IOException {
-    /**
-     * Don't forward this to the server. The protocol version and signature is
-     * that of {@link ClientDatanodeProtocol}
-     */
-    return ProtocolSignatureWritable.convert(
-        this.getProtocolSignature(protocol, clientVersion, clientMethodsHash));
-  }
 }

+ 0 - 13
hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/protocolPB/ClientDatanodeProtocolTranslatorPB.java

@@ -144,19 +144,6 @@ public class ClientDatanodeProtocolTranslatorPB implements
     RPC.stopProxy(rpcProxy);
   }
 
-  @Override
-  public long getProtocolVersion(String protocolName, long clientVersion)
-      throws IOException {
-    return rpcProxy.getProtocolVersion(protocolName, clientVersion);
-  }
-
-  @Override
-  public ProtocolSignature getProtocolSignature(String protocol,
-      long clientVersion, int clientMethodsHash) throws IOException {
-    return ProtocolSignatureWritable.convert(rpcProxy.getProtocolSignature2(
-        protocol, clientVersion, clientMethodsHash));
-  }
-
   @Override
   public long getReplicaVisibleLength(ExtendedBlock b) throws IOException {
     GetReplicaVisibleLengthRequestProto req = GetReplicaVisibleLengthRequestProto

+ 1 - 13
hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/protocolPB/ClientNamenodeProtocolPB.java

@@ -17,17 +17,13 @@
  */
 package org.apache.hadoop.hdfs.protocolPB;
 
-import java.io.IOException;
-
 import org.apache.hadoop.classification.InterfaceAudience;
 import org.apache.hadoop.classification.InterfaceStability;
 import org.apache.hadoop.hdfs.DFSConfigKeys;
 import org.apache.hadoop.hdfs.protocol.HdfsConstants;
 import org.apache.hadoop.hdfs.protocol.proto.ClientNamenodeProtocolProtos.ClientNamenodeProtocol;
-import org.apache.hadoop.hdfs.protocolR23Compatible.ProtocolSignatureWritable;
 import org.apache.hadoop.hdfs.security.token.delegation.DelegationTokenSelector;
 import org.apache.hadoop.ipc.ProtocolInfo;
-import org.apache.hadoop.ipc.VersionedProtocol;
 import org.apache.hadoop.security.KerberosInfo;
 import org.apache.hadoop.security.token.TokenInfo;
 
@@ -46,13 +42,5 @@ import org.apache.hadoop.security.token.TokenInfo;
  * add annotations required for security.
  */
 public interface ClientNamenodeProtocolPB extends 
-  ClientNamenodeProtocol.BlockingInterface,  VersionedProtocol {
-  
-  /**
-   * This method is defined to get the protocol signature using 
-   * the R23 protocol - hence we have added the suffix of 2 the method name
-   * to avoid conflict.
-   */
-  public ProtocolSignatureWritable getProtocolSignature2(String protocol,
-      long clientVersion, int clientMethodsHash) throws IOException;
+  ClientNamenodeProtocol.BlockingInterface {
 }

+ 0 - 54
hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/protocolPB/ClientNamenodeProtocolServerSideTranslatorPB.java

@@ -124,17 +124,11 @@ import org.apache.hadoop.hdfs.protocol.proto.ClientNamenodeProtocolProtos.Update
 import org.apache.hadoop.hdfs.protocol.proto.ClientNamenodeProtocolProtos.UpdatePipelineRequestProto;
 import org.apache.hadoop.hdfs.protocol.proto.ClientNamenodeProtocolProtos.UpdatePipelineResponseProto;
 import org.apache.hadoop.hdfs.protocol.proto.HdfsProtos.BlockTokenIdentifierProto;
-import org.apache.hadoop.hdfs.protocol.proto.HdfsProtos.ContentSummaryProto;
-import org.apache.hadoop.hdfs.protocol.proto.HdfsProtos.CorruptFileBlocksProto;
 import org.apache.hadoop.hdfs.protocol.proto.HdfsProtos.DatanodeIDProto;
 import org.apache.hadoop.hdfs.protocol.proto.HdfsProtos.DatanodeInfoProto;
-import org.apache.hadoop.hdfs.protocol.proto.HdfsProtos.DirectoryListingProto;
 import org.apache.hadoop.hdfs.protocol.proto.HdfsProtos.LocatedBlockProto;
-import org.apache.hadoop.hdfs.protocolR23Compatible.ProtocolSignatureWritable;
 import org.apache.hadoop.hdfs.server.common.UpgradeStatusReport;
 import org.apache.hadoop.io.Text;
-import org.apache.hadoop.ipc.ProtocolSignature;
-import org.apache.hadoop.ipc.RPC;
 
 import com.google.protobuf.RpcController;
 import com.google.protobuf.ServiceException;
@@ -163,54 +157,6 @@ public class ClientNamenodeProtocolServerSideTranslatorPB implements
     this.server = server;
   }
 
-  /**
-   * The client side will redirect getProtocolSignature to
-   * getProtocolSignature2.
-   * 
-   * However the RPC layer below on the Server side will call getProtocolVersion
-   * and possibly in the future getProtocolSignature. Hence we still implement
-   * it even though the end client's call will never reach here.
-   */
-  @Override
-  public ProtocolSignature getProtocolSignature(String protocol,
-      long clientVersion, int clientMethodsHash) throws IOException {
-    /**
-     * Don't forward this to the server. The protocol version and signature is
-     * that of {@link ClientNamenodeProtocol}
-     * 
-     */
-    if (!protocol.equals(RPC.getProtocolName(
-        ClientNamenodeProtocolPB.class))) {
-      throw new IOException("Namenode Serverside implements " +
-          RPC.getProtocolName(ClientNamenodeProtocolPB.class) +
-          ". The following requested protocol is unknown: " + protocol);
-    }
-
-    return ProtocolSignature.getProtocolSignature(clientMethodsHash,
-        RPC.getProtocolVersion(ClientNamenodeProtocolPB.class),
-        ClientNamenodeProtocolPB.class);
-  }
-
-  @Override
-  public ProtocolSignatureWritable 
-          getProtocolSignature2(
-      String protocol, long clientVersion, int clientMethodsHash)
-      throws IOException {
-    /**
-     * Don't forward this to the server. The protocol version and signature is
-     * that of {@link ClientNamenodeProtocol}
-     * 
-     */
-    return ProtocolSignatureWritable.convert(
-        this.getProtocolSignature(protocol, clientVersion, clientMethodsHash));
-  }
-
-  @Override
-  public long getProtocolVersion(String protocol, long clientVersion)
-      throws IOException {
-    return RPC.getProtocolVersion(InterDatanodeProtocolPB.class);
-  }
-
   @Override
   public GetBlockLocationsResponseProto getBlockLocations(
       RpcController controller, GetBlockLocationsRequestProto req)

+ 27 - 43
hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/protocolPB/ClientNamenodeProtocolTranslatorPB.java

@@ -33,49 +33,26 @@ import org.apache.hadoop.fs.ContentSummary;
 import org.apache.hadoop.fs.CreateFlag;
 import org.apache.hadoop.fs.FileAlreadyExistsException;
 import org.apache.hadoop.fs.FsServerDefaults;
+import org.apache.hadoop.fs.Options.Rename;
 import org.apache.hadoop.fs.ParentNotDirectoryException;
 import org.apache.hadoop.fs.UnresolvedLinkException;
-import org.apache.hadoop.fs.Options.Rename;
 import org.apache.hadoop.fs.permission.FsPermission;
 import org.apache.hadoop.hdfs.protocol.AlreadyBeingCreatedException;
 import org.apache.hadoop.hdfs.protocol.ClientProtocol;
-import org.apache.hadoop.hdfs.protocol.LocatedBlock;
-import org.apache.hadoop.hdfs.protocol.LocatedBlocks;
 import org.apache.hadoop.hdfs.protocol.CorruptFileBlocks;
 import org.apache.hadoop.hdfs.protocol.DSQuotaExceededException;
 import org.apache.hadoop.hdfs.protocol.DatanodeID;
 import org.apache.hadoop.hdfs.protocol.DatanodeInfo;
 import org.apache.hadoop.hdfs.protocol.DirectoryListing;
 import org.apache.hadoop.hdfs.protocol.ExtendedBlock;
-import org.apache.hadoop.hdfs.protocol.HdfsFileStatus;
-import org.apache.hadoop.hdfs.protocol.NSQuotaExceededException;
+import org.apache.hadoop.hdfs.protocol.HdfsConstants;
 import org.apache.hadoop.hdfs.protocol.HdfsConstants.DatanodeReportType;
 import org.apache.hadoop.hdfs.protocol.HdfsConstants.SafeModeAction;
 import org.apache.hadoop.hdfs.protocol.HdfsConstants.UpgradeAction;
-import org.apache.hadoop.hdfs.protocolR23Compatible.ProtocolSignatureWritable;
-import org.apache.hadoop.hdfs.security.token.delegation.DelegationTokenIdentifier;
-import org.apache.hadoop.hdfs.server.common.UpgradeStatusReport;
-import org.apache.hadoop.hdfs.server.namenode.NameNode;
-import org.apache.hadoop.hdfs.server.namenode.NotReplicatedYetException;
-import org.apache.hadoop.hdfs.server.namenode.SafeModeException;
-import org.apache.hadoop.io.EnumSetWritable;
-import org.apache.hadoop.io.Text;
-import org.apache.hadoop.io.retry.RetryPolicies;
-import org.apache.hadoop.io.retry.RetryPolicy;
-import org.apache.hadoop.io.retry.RetryProxy;
-import org.apache.hadoop.ipc.ProtobufHelper;
-import org.apache.hadoop.ipc.ProtobufRpcEngine;
-import org.apache.hadoop.ipc.ProtocolMetaInterface;
-import org.apache.hadoop.ipc.ProtocolSignature;
-import org.apache.hadoop.ipc.RPC;
-import org.apache.hadoop.ipc.RemoteException;
-import org.apache.hadoop.ipc.RpcClientUtil;
-import org.apache.hadoop.ipc.RpcPayloadHeader.RpcKind;
-import org.apache.hadoop.net.NetUtils;
-import org.apache.hadoop.security.AccessControlException;
-import org.apache.hadoop.security.UserGroupInformation;
-import org.apache.hadoop.security.token.Token;
-import org.apache.hadoop.hdfs.protocol.HdfsConstants;
+import org.apache.hadoop.hdfs.protocol.HdfsFileStatus;
+import org.apache.hadoop.hdfs.protocol.LocatedBlock;
+import org.apache.hadoop.hdfs.protocol.LocatedBlocks;
+import org.apache.hadoop.hdfs.protocol.NSQuotaExceededException;
 import org.apache.hadoop.hdfs.protocol.proto.ClientNamenodeProtocolProtos.AbandonBlockRequestProto;
 import org.apache.hadoop.hdfs.protocol.proto.ClientNamenodeProtocolProtos.AddBlockRequestProto;
 import org.apache.hadoop.hdfs.protocol.proto.ClientNamenodeProtocolProtos.AppendRequestProto;
@@ -127,6 +104,27 @@ import org.apache.hadoop.hdfs.protocol.proto.ClientNamenodeProtocolProtos.SetSaf
 import org.apache.hadoop.hdfs.protocol.proto.ClientNamenodeProtocolProtos.SetTimesRequestProto;
 import org.apache.hadoop.hdfs.protocol.proto.ClientNamenodeProtocolProtos.UpdateBlockForPipelineRequestProto;
 import org.apache.hadoop.hdfs.protocol.proto.ClientNamenodeProtocolProtos.UpdatePipelineRequestProto;
+import org.apache.hadoop.hdfs.security.token.delegation.DelegationTokenIdentifier;
+import org.apache.hadoop.hdfs.server.common.UpgradeStatusReport;
+import org.apache.hadoop.hdfs.server.namenode.NameNode;
+import org.apache.hadoop.hdfs.server.namenode.NotReplicatedYetException;
+import org.apache.hadoop.hdfs.server.namenode.SafeModeException;
+import org.apache.hadoop.io.EnumSetWritable;
+import org.apache.hadoop.io.Text;
+import org.apache.hadoop.io.retry.RetryPolicies;
+import org.apache.hadoop.io.retry.RetryPolicy;
+import org.apache.hadoop.io.retry.RetryProxy;
+import org.apache.hadoop.ipc.ProtobufHelper;
+import org.apache.hadoop.ipc.ProtobufRpcEngine;
+import org.apache.hadoop.ipc.ProtocolMetaInterface;
+import org.apache.hadoop.ipc.RPC;
+import org.apache.hadoop.ipc.RemoteException;
+import org.apache.hadoop.ipc.RpcClientUtil;
+import org.apache.hadoop.ipc.RpcPayloadHeader.RpcKind;
+import org.apache.hadoop.net.NetUtils;
+import org.apache.hadoop.security.AccessControlException;
+import org.apache.hadoop.security.UserGroupInformation;
+import org.apache.hadoop.security.token.Token;
 
 import com.google.protobuf.ByteString;
 import com.google.protobuf.ServiceException;
@@ -189,20 +187,6 @@ public class ClientNamenodeProtocolTranslatorPB implements
     RPC.stopProxy(rpcProxy);
   }
 
-  @Override
-  public ProtocolSignature getProtocolSignature(String protocolName,
-      long clientVersion, int clientMethodHash)
-      throws IOException {
-    return ProtocolSignatureWritable.convert(rpcProxy.getProtocolSignature2(
-        protocolName, clientVersion, clientMethodHash));
-  }
-
-  @Override
-  public long getProtocolVersion(String protocolName, long clientVersion) 
-      throws IOException {
-    return rpcProxy.getProtocolVersion(protocolName, clientVersion);
-  }
-
   @Override
   public LocatedBlocks getBlockLocations(String src, long offset, long length)
       throws AccessControlException, FileNotFoundException,

+ 0 - 13
hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/protocolPB/DatanodeProtocolClientSideTranslatorPB.java

@@ -136,19 +136,6 @@ public class DatanodeProtocolClientSideTranslatorPB implements
     return (DatanodeProtocolPB) RetryProxy.create(DatanodeProtocolPB.class,
         rpcNamenode, methodNameToPolicyMap);
   }
-  
-  @Override
-  public long getProtocolVersion(String protocol, long clientVersion)
-      throws IOException {
-    return rpcProxy.getProtocolVersion(protocol, clientVersion);
-  }
-
-  @Override
-  public ProtocolSignature getProtocolSignature(String protocolName,
-      long clientVersion, int clientMethodsHash) throws IOException {
-    return ProtocolSignatureWritable.convert(rpcProxy.getProtocolSignature2(
-        protocolName, clientVersion, clientMethodsHash));
-  }
 
   @Override
   public void close() throws IOException {

+ 1 - 13
hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/protocolPB/DatanodeProtocolPB.java

@@ -18,14 +18,10 @@
 
 package org.apache.hadoop.hdfs.protocolPB;
 
-import java.io.IOException;
-
 import org.apache.hadoop.classification.InterfaceAudience;
 import org.apache.hadoop.hdfs.DFSConfigKeys;
 import org.apache.hadoop.hdfs.protocol.proto.DatanodeProtocolProtos.DatanodeProtocolService;
-import org.apache.hadoop.hdfs.protocolR23Compatible.ProtocolSignatureWritable;
 import org.apache.hadoop.ipc.ProtocolInfo;
-import org.apache.hadoop.ipc.VersionedProtocol;
 import org.apache.hadoop.security.KerberosInfo;
 
 @KerberosInfo(
@@ -36,13 +32,5 @@ import org.apache.hadoop.security.KerberosInfo;
     protocolVersion = 1)
 @InterfaceAudience.Private
 public interface DatanodeProtocolPB extends
-    DatanodeProtocolService.BlockingInterface, VersionedProtocol {
-  
-  /**
-   * This method is defined to get the protocol signature using 
-   * the R23 protocol - hence we have added the suffix of 2 the method name
-   * to avoid conflict.
-   */
-  public ProtocolSignatureWritable getProtocolSignature2(String protocol,
-      long clientVersion, int clientMethodsHash) throws IOException;
+    DatanodeProtocolService.BlockingInterface {
 }

+ 0 - 39
hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/protocolPB/DatanodeProtocolServerSideTranslatorPB.java

@@ -47,7 +47,6 @@ import org.apache.hadoop.hdfs.protocol.proto.HdfsProtos.DatanodeIDProto;
 import org.apache.hadoop.hdfs.protocol.proto.HdfsProtos.LocatedBlockProto;
 import org.apache.hadoop.hdfs.protocol.proto.HdfsProtos.VersionRequestProto;
 import org.apache.hadoop.hdfs.protocol.proto.HdfsProtos.VersionResponseProto;
-import org.apache.hadoop.hdfs.protocolR23Compatible.ProtocolSignatureWritable;
 import org.apache.hadoop.hdfs.server.protocol.DatanodeCommand;
 import org.apache.hadoop.hdfs.server.protocol.DatanodeProtocol;
 import org.apache.hadoop.hdfs.server.protocol.DatanodeRegistration;
@@ -58,8 +57,6 @@ import org.apache.hadoop.hdfs.server.protocol.StorageBlockReport;
 import org.apache.hadoop.hdfs.server.protocol.StorageReceivedDeletedBlocks;
 import org.apache.hadoop.hdfs.server.protocol.StorageReport;
 import org.apache.hadoop.hdfs.server.protocol.UpgradeCommand;
-import org.apache.hadoop.ipc.ProtocolSignature;
-import org.apache.hadoop.ipc.RPC;
 
 import com.google.protobuf.RpcController;
 import com.google.protobuf.ServiceException;
@@ -268,40 +265,4 @@ public class DatanodeProtocolServerSideTranslatorPB implements
     }
     return COMMIT_BLOCK_SYNCHRONIZATION_RESPONSE_PROTO;
   }
-
-  @Override
-  public long getProtocolVersion(String protocol, long clientVersion)
-      throws IOException {
-    return RPC.getProtocolVersion(DatanodeProtocolPB.class);
-  }
-
-  @Override
-  public ProtocolSignature getProtocolSignature(String protocol,
-      long clientVersion, int clientMethodsHash) throws IOException {
-    /**
-     * Don't forward this to the server. The protocol version and signature is
-     * that of {@link DatanodeProtocol}
-     */
-    if (!protocol.equals(RPC.getProtocolName(DatanodeProtocolPB.class))) {
-      throw new IOException("Namenode Serverside implements " +
-          RPC.getProtocolName(DatanodeProtocolPB.class) +
-          ". The following requested protocol is unknown: " + protocol);
-    }
-
-    return ProtocolSignature.getProtocolSignature(clientMethodsHash,
-        RPC.getProtocolVersion(DatanodeProtocolPB.class),
-        DatanodeProtocolPB.class);
-  }
-
-  @Override
-  public ProtocolSignatureWritable getProtocolSignature2(String protocol,
-      long clientVersion, int clientMethodsHash) throws IOException {
-    /**
-     * Don't forward this to the server. The protocol version and signature is
-     * that of {@link DatanodeProtocolPB}
-     */
-    return ProtocolSignatureWritable.convert(
-        this.getProtocolSignature(protocol, clientVersion, clientMethodsHash));
-  }
-
 }

+ 0 - 13
hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/protocolPB/GetUserMappingsProtocolClientSideTranslatorPB.java

@@ -59,19 +59,6 @@ public class GetUserMappingsProtocolClientSideTranslatorPB implements
         NetUtils.getSocketFactory(conf, GetUserMappingsProtocol.class));
   }
 
-  @Override
-  public long getProtocolVersion(String protocol, long clientVersion)
-      throws IOException {
-    return rpcProxy.getProtocolVersion(protocol, clientVersion);
-  }
-
-  @Override
-  public ProtocolSignature getProtocolSignature(String protocol,
-      long clientVersion, int clientMethodsHash) throws IOException {
-    return ProtocolSignatureWritable.convert(rpcProxy.getProtocolSignature2(
-        protocol, clientVersion, clientMethodsHash));
-  }
-
   @Override
   public void close() throws IOException {
     RPC.stopProxy(rpcProxy);

+ 1 - 13
hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/protocolPB/GetUserMappingsProtocolPB.java

@@ -18,14 +18,10 @@
 
 package org.apache.hadoop.hdfs.protocolPB;
 
-import java.io.IOException;
-
 import org.apache.hadoop.classification.InterfaceAudience;
 import org.apache.hadoop.classification.InterfaceStability;
 import org.apache.hadoop.hdfs.protocol.proto.GetUserMappingsProtocolProtos.GetUserMappingsProtocolService;
-import org.apache.hadoop.hdfs.protocolR23Compatible.ProtocolSignatureWritable;
 import org.apache.hadoop.ipc.ProtocolInfo;
-import org.apache.hadoop.ipc.VersionedProtocol;
 
 @ProtocolInfo(
     protocolName = "org.apache.hadoop.tools.GetUserMappingsProtocol", 
@@ -33,13 +29,5 @@ import org.apache.hadoop.ipc.VersionedProtocol;
 @InterfaceAudience.LimitedPrivate({"HDFS", "MapReduce"})
 @InterfaceStability.Evolving
 public interface GetUserMappingsProtocolPB extends
-  GetUserMappingsProtocolService.BlockingInterface, VersionedProtocol {
-  
-  /**
-   * This method is defined to get the protocol signature using 
-   * the R23 protocol - hence we have added the suffix of 2 the method name
-   * to avoid conflict.
-   */
-  public ProtocolSignatureWritable getProtocolSignature2(String protocol,
-      long clientVersion, int clientMethodsHash) throws IOException;
+  GetUserMappingsProtocolService.BlockingInterface {
 }

+ 0 - 39
hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/protocolPB/GetUserMappingsProtocolServerSideTranslatorPB.java

@@ -22,9 +22,6 @@ import java.io.IOException;
 
 import org.apache.hadoop.hdfs.protocol.proto.GetUserMappingsProtocolProtos.GetGroupsForUserRequestProto;
 import org.apache.hadoop.hdfs.protocol.proto.GetUserMappingsProtocolProtos.GetGroupsForUserResponseProto;
-import org.apache.hadoop.hdfs.protocolR23Compatible.ProtocolSignatureWritable;
-import org.apache.hadoop.ipc.ProtocolSignature;
-import org.apache.hadoop.ipc.RPC;
 import org.apache.hadoop.tools.GetUserMappingsProtocol;
 
 import com.google.protobuf.RpcController;
@@ -40,42 +37,6 @@ public class GetUserMappingsProtocolServerSideTranslatorPB implements
     this.impl = impl;
   }
 
-  @Override
-  public long getProtocolVersion(String protocol, long clientVersion)
-      throws IOException {
-    return RPC.getProtocolVersion(GetUserMappingsProtocolPB.class);
-  }
-
-  @Override
-  public ProtocolSignature getProtocolSignature(String protocol,
-      long clientVersion, int clientMethodsHash) throws IOException {
-    /**
-     * Don't forward this to the server. The protocol version and signature is
-     * that of {@link GetUserMappingsProtocol}
-     */
-    if (!protocol.equals(RPC
-        .getProtocolName(GetUserMappingsProtocolPB.class))) {
-      throw new IOException("Namenode Serverside implements "
-          + RPC.getProtocolName(GetUserMappingsProtocolPB.class)
-          + ". The following requested protocol is unknown: " + protocol);
-    }
-
-    return ProtocolSignature.getProtocolSignature(clientMethodsHash,
-        RPC.getProtocolVersion(GetUserMappingsProtocolPB.class),
-        GetUserMappingsProtocolPB.class);
-  }
-
-  @Override
-  public ProtocolSignatureWritable getProtocolSignature2(String protocol,
-      long clientVersion, int clientMethodsHash) throws IOException {
-    /**
-     * Don't forward this to the server. The protocol version and signature is
-     * that of {@link GetUserMappingsProtocolPB}
-     */
-    return ProtocolSignatureWritable.convert(this.getProtocolSignature(
-        protocol, clientVersion, clientMethodsHash));
-  }
-
   @Override
   public GetGroupsForUserResponseProto getGroupsForUser(
       RpcController controller, GetGroupsForUserRequestProto request)

+ 1 - 13
hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/protocolPB/InterDatanodeProtocolPB.java

@@ -17,14 +17,10 @@
  */
 package org.apache.hadoop.hdfs.protocolPB;
 
-import java.io.IOException;
-
 import org.apache.hadoop.classification.InterfaceAudience;
 import org.apache.hadoop.hdfs.DFSConfigKeys;
 import org.apache.hadoop.hdfs.protocol.proto.InterDatanodeProtocolProtos.InterDatanodeProtocolService;
-import org.apache.hadoop.hdfs.protocolR23Compatible.ProtocolSignatureWritable;
 import org.apache.hadoop.ipc.ProtocolInfo;
-import org.apache.hadoop.ipc.VersionedProtocol;
 import org.apache.hadoop.security.KerberosInfo;
 
 @KerberosInfo(
@@ -35,13 +31,5 @@ import org.apache.hadoop.security.KerberosInfo;
     protocolVersion = 1)
 @InterfaceAudience.Private
 public interface InterDatanodeProtocolPB extends
-    InterDatanodeProtocolService.BlockingInterface, VersionedProtocol {
-
-  /**
-   * This method is defined to get the protocol signature using 
-   * the R23 protocol - hence we have added the suffix of 2 the method name
-   * to avoid conflict.
-   */
-  public ProtocolSignatureWritable getProtocolSignature2(String protocol,
-      long clientVersion, int clientMethodsHash) throws IOException;
+    InterDatanodeProtocolService.BlockingInterface {
 }

+ 0 - 52
hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/protocolPB/InterDatanodeProtocolServerSideTranslatorPB.java

@@ -25,14 +25,9 @@ import org.apache.hadoop.hdfs.protocol.proto.InterDatanodeProtocolProtos.InitRep
 import org.apache.hadoop.hdfs.protocol.proto.InterDatanodeProtocolProtos.InitReplicaRecoveryResponseProto;
 import org.apache.hadoop.hdfs.protocol.proto.InterDatanodeProtocolProtos.UpdateReplicaUnderRecoveryRequestProto;
 import org.apache.hadoop.hdfs.protocol.proto.InterDatanodeProtocolProtos.UpdateReplicaUnderRecoveryResponseProto;
-import org.apache.hadoop.hdfs.protocolR23Compatible.ProtocolSignatureWritable;
 import org.apache.hadoop.hdfs.server.protocol.BlockRecoveryCommand.RecoveringBlock;
 import org.apache.hadoop.hdfs.server.protocol.InterDatanodeProtocol;
-import org.apache.hadoop.hdfs.server.protocol.JournalProtocol;
 import org.apache.hadoop.hdfs.server.protocol.ReplicaRecoveryInfo;
-import org.apache.hadoop.ipc.ProtocolSignature;
-import org.apache.hadoop.ipc.RPC;
-import org.apache.hadoop.ipc.VersionedProtocol;
 
 import com.google.protobuf.RpcController;
 import com.google.protobuf.ServiceException;
@@ -81,51 +76,4 @@ public class InterDatanodeProtocolServerSideTranslatorPB implements
     return UpdateReplicaUnderRecoveryResponseProto.newBuilder()
         .setBlock(PBHelper.convert(b)).build();
   }
-
-  /** @see VersionedProtocol#getProtocolVersion */
-  @Override
-  public long getProtocolVersion(String protocol, long clientVersion)
-      throws IOException {
-    return RPC.getProtocolVersion(InterDatanodeProtocolPB.class);
-  }
-  
-  /**
-   * The client side will redirect getProtocolSignature to
-   * getProtocolSignature2.
-   * 
-   * However the RPC layer below on the Server side will call getProtocolVersion
-   * and possibly in the future getProtocolSignature. Hence we still implement
-   * it even though the end client will never call this method.
-   * 
-   * @see VersionedProtocol#getProtocolVersion
-   */
-  @Override
-  public ProtocolSignature getProtocolSignature(String protocol,
-      long clientVersion, int clientMethodsHash) throws IOException {
-    /**
-     * Don't forward this to the server. The protocol version and signature is
-     * that of {@link InterDatanodeProtocol}
-     */
-    if (!protocol.equals(RPC.getProtocolName(InterDatanodeProtocol.class))) {
-      throw new IOException("Namenode Serverside implements " +
-          RPC.getProtocolName(InterDatanodeProtocol.class) +
-          ". The following requested protocol is unknown: " + protocol);
-    }
-
-    return ProtocolSignature.getProtocolSignature(clientMethodsHash,
-        RPC.getProtocolVersion(InterDatanodeProtocolPB.class),
-        InterDatanodeProtocolPB.class);
-  }
-
-
-  @Override
-  public ProtocolSignatureWritable getProtocolSignature2(String protocol,
-      long clientVersion, int clientMethodsHash) throws IOException {
-    /**
-     * Don't forward this to the server. The protocol version and signature is
-     * that of {@link InterDatanodeProtocol}
-     */
-    return ProtocolSignatureWritable.convert(
-        this.getProtocolSignature(protocol, clientVersion, clientMethodsHash));
-  }
 }

+ 0 - 15
hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/protocolPB/InterDatanodeProtocolTranslatorPB.java

@@ -31,14 +31,12 @@ import org.apache.hadoop.hdfs.protocol.proto.HdfsProtos.BlockProto;
 import org.apache.hadoop.hdfs.protocol.proto.InterDatanodeProtocolProtos.InitReplicaRecoveryRequestProto;
 import org.apache.hadoop.hdfs.protocol.proto.InterDatanodeProtocolProtos.InitReplicaRecoveryResponseProto;
 import org.apache.hadoop.hdfs.protocol.proto.InterDatanodeProtocolProtos.UpdateReplicaUnderRecoveryRequestProto;
-import org.apache.hadoop.hdfs.protocolR23Compatible.ProtocolSignatureWritable;
 import org.apache.hadoop.hdfs.server.protocol.BlockRecoveryCommand.RecoveringBlock;
 import org.apache.hadoop.hdfs.server.protocol.InterDatanodeProtocol;
 import org.apache.hadoop.hdfs.server.protocol.ReplicaRecoveryInfo;
 import org.apache.hadoop.ipc.ProtobufHelper;
 import org.apache.hadoop.ipc.ProtobufRpcEngine;
 import org.apache.hadoop.ipc.ProtocolMetaInterface;
-import org.apache.hadoop.ipc.ProtocolSignature;
 import org.apache.hadoop.ipc.RPC;
 import org.apache.hadoop.ipc.RpcClientUtil;
 import org.apache.hadoop.ipc.RpcPayloadHeader.RpcKind;
@@ -76,19 +74,6 @@ public class InterDatanodeProtocolTranslatorPB implements
     RPC.stopProxy(rpcProxy);
   }
 
-  @Override
-  public long getProtocolVersion(String protocolName, long clientVersion)
-      throws IOException {
-    return rpcProxy.getProtocolVersion(protocolName, clientVersion);
-  }
-
-  @Override
-  public ProtocolSignature getProtocolSignature(String protocol,
-      long clientVersion, int clientMethodsHash) throws IOException {
-    return ProtocolSignatureWritable.convert(rpcProxy.getProtocolSignature2(
-        protocol, clientVersion, clientMethodsHash));
-  }
-
   @Override
   public ReplicaRecoveryInfo initReplicaRecovery(RecoveringBlock rBlock)
       throws IOException {

+ 3 - 14
hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/protocolPB/JournalProtocolPB.java

@@ -17,15 +17,11 @@
  */
 package org.apache.hadoop.hdfs.protocolPB;
 
-import java.io.IOException;
-
-import org.apache.hadoop.hdfs.protocol.proto.JournalProtocolProtos.JournalProtocolService;
-import org.apache.hadoop.hdfs.protocolR23Compatible.ProtocolSignatureWritable;
-import org.apache.hadoop.security.KerberosInfo;
 import org.apache.hadoop.classification.InterfaceAudience;
 import org.apache.hadoop.hdfs.DFSConfigKeys;
+import org.apache.hadoop.hdfs.protocol.proto.JournalProtocolProtos.JournalProtocolService;
 import org.apache.hadoop.ipc.ProtocolInfo;
-import org.apache.hadoop.ipc.VersionedProtocol;
+import org.apache.hadoop.security.KerberosInfo;
 
 /**
  * Protocol used to journal edits to a remote node. Currently,
@@ -42,12 +38,5 @@ import org.apache.hadoop.ipc.VersionedProtocol;
     protocolVersion = 1)
 @InterfaceAudience.Private
 public interface JournalProtocolPB extends
-    JournalProtocolService.BlockingInterface, VersionedProtocol {
-  /**
-   * This method is defined to get the protocol signature using 
-   * the R23 protocol - hence we have added the suffix of 2 the method name
-   * to avoid conflict.
-   */
-  public ProtocolSignatureWritable getProtocolSignature2(String protocol,
-      long clientVersion, int clientMethodsHash) throws IOException;
+    JournalProtocolService.BlockingInterface {
 }

+ 0 - 51
hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/protocolPB/JournalProtocolServerSideTranslatorPB.java

@@ -24,11 +24,7 @@ import org.apache.hadoop.hdfs.protocol.proto.JournalProtocolProtos.JournalReques
 import org.apache.hadoop.hdfs.protocol.proto.JournalProtocolProtos.JournalResponseProto;
 import org.apache.hadoop.hdfs.protocol.proto.JournalProtocolProtos.StartLogSegmentRequestProto;
 import org.apache.hadoop.hdfs.protocol.proto.JournalProtocolProtos.StartLogSegmentResponseProto;
-import org.apache.hadoop.hdfs.protocolR23Compatible.ProtocolSignatureWritable;
 import org.apache.hadoop.hdfs.server.protocol.JournalProtocol;
-import org.apache.hadoop.ipc.ProtocolSignature;
-import org.apache.hadoop.ipc.RPC;
-import org.apache.hadoop.ipc.VersionedProtocol;
 
 import com.google.protobuf.RpcController;
 import com.google.protobuf.ServiceException;
@@ -73,51 +69,4 @@ public class JournalProtocolServerSideTranslatorPB implements JournalProtocolPB
     }
     return StartLogSegmentResponseProto.newBuilder().build();
   }
-
-  /** @see VersionedProtocol#getProtocolVersion */
-  @Override
-  public long getProtocolVersion(String protocol, long clientVersion)
-      throws IOException {
-    return RPC.getProtocolVersion(JournalProtocolPB.class);
-  }
-
-  /**
-   * The client side will redirect getProtocolSignature to
-   * getProtocolSignature2.
-   * 
-   * However the RPC layer below on the Server side will call getProtocolVersion
-   * and possibly in the future getProtocolSignature. Hence we still implement
-   * it even though the end client will never call this method.
-   * 
-   * @see VersionedProtocol#getProtocolSignature(String, long, int)
-   */
-  @Override
-  public ProtocolSignature getProtocolSignature(String protocol,
-      long clientVersion, int clientMethodsHash) throws IOException {
-    /**
-     * Don't forward this to the server. The protocol version and signature is
-     * that of {@link JournalProtocol}
-     */
-    if (!protocol.equals(RPC.getProtocolName(JournalProtocolPB.class))) {
-      throw new IOException("Namenode Serverside implements " +
-          RPC.getProtocolName(JournalProtocolPB.class) +
-          ". The following requested protocol is unknown: " + protocol);
-    }
-
-    return ProtocolSignature.getProtocolSignature(clientMethodsHash,
-        RPC.getProtocolVersion(JournalProtocolPB.class),
-        JournalProtocolPB.class);
-  }
-
-
-  @Override
-  public ProtocolSignatureWritable getProtocolSignature2(String protocol,
-      long clientVersion, int clientMethodsHash) throws IOException {
-    /**
-     * Don't forward this to the server. The protocol version and signature is
-     * that of {@link JournalPBProtocol}
-     */
-    return ProtocolSignatureWritable.convert(
-        this.getProtocolSignature(protocol, clientVersion, clientMethodsHash));
-  }
 }

+ 0 - 13
hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/protocolPB/JournalProtocolTranslatorPB.java

@@ -65,19 +65,6 @@ public class JournalProtocolTranslatorPB implements ProtocolMetaInterface,
     RPC.stopProxy(rpcProxy);
   }
 
-  @Override
-  public long getProtocolVersion(String protocolName, long clientVersion)
-      throws IOException {
-    return rpcProxy.getProtocolVersion(protocolName, clientVersion);
-  }
-
-  @Override
-  public ProtocolSignature getProtocolSignature(String protocol,
-      long clientVersion, int clientMethodsHash) throws IOException {
-    return ProtocolSignatureWritable.convert(rpcProxy.getProtocolSignature2(
-        protocol, clientVersion, clientMethodsHash));
-  }
-
   @Override
   public void journal(NamenodeRegistration reg, long firstTxnId,
       int numTxns, byte[] records) throws IOException {

+ 1 - 12
hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/protocolPB/NamenodeProtocolPB.java

@@ -18,14 +18,10 @@
 
 package org.apache.hadoop.hdfs.protocolPB;
 
-import java.io.IOException;
-
 import org.apache.hadoop.classification.InterfaceAudience;
 import org.apache.hadoop.hdfs.DFSConfigKeys;
 import org.apache.hadoop.hdfs.protocol.proto.NamenodeProtocolProtos.NamenodeProtocolService;
-import org.apache.hadoop.hdfs.protocolR23Compatible.ProtocolSignatureWritable;
 import org.apache.hadoop.ipc.ProtocolInfo;
-import org.apache.hadoop.ipc.VersionedProtocol;
 import org.apache.hadoop.security.KerberosInfo;
 
 /**
@@ -43,12 +39,5 @@ import org.apache.hadoop.security.KerberosInfo;
     protocolVersion = 1)
 @InterfaceAudience.Private
 public interface NamenodeProtocolPB extends
-    NamenodeProtocolService.BlockingInterface, VersionedProtocol {
-  /**
-   * This method is defined to get the protocol signature using 
-   * the R23 protocol - hence we have added the suffix of 2 the method name
-   * to avoid conflict.
-   */
-  public ProtocolSignatureWritable getProtocolSignature2(String protocol,
-      long clientVersion, int clientMethodsHash) throws IOException;
+    NamenodeProtocolService.BlockingInterface {
 }

+ 0 - 47
hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/protocolPB/NamenodeProtocolServerSideTranslatorPB.java

@@ -40,7 +40,6 @@ import org.apache.hadoop.hdfs.protocol.proto.NamenodeProtocolProtos.RollEditLogR
 import org.apache.hadoop.hdfs.protocol.proto.NamenodeProtocolProtos.RollEditLogResponseProto;
 import org.apache.hadoop.hdfs.protocol.proto.NamenodeProtocolProtos.StartCheckpointRequestProto;
 import org.apache.hadoop.hdfs.protocol.proto.NamenodeProtocolProtos.StartCheckpointResponseProto;
-import org.apache.hadoop.hdfs.protocolR23Compatible.ProtocolSignatureWritable;
 import org.apache.hadoop.hdfs.security.token.block.ExportedBlockKeys;
 import org.apache.hadoop.hdfs.server.namenode.CheckpointSignature;
 import org.apache.hadoop.hdfs.server.protocol.BlocksWithLocations;
@@ -49,8 +48,6 @@ import org.apache.hadoop.hdfs.server.protocol.NamenodeProtocol;
 import org.apache.hadoop.hdfs.server.protocol.NamenodeRegistration;
 import org.apache.hadoop.hdfs.server.protocol.NamespaceInfo;
 import org.apache.hadoop.hdfs.server.protocol.RemoteEditLogManifest;
-import org.apache.hadoop.ipc.ProtocolSignature;
-import org.apache.hadoop.ipc.RPC;
 
 import com.google.protobuf.RpcController;
 import com.google.protobuf.ServiceException;
@@ -184,50 +181,6 @@ public class NamenodeProtocolServerSideTranslatorPB implements
     return GetEditLogManifestResponseProto.newBuilder()
         .setManifest(PBHelper.convert(manifest)).build();
   }
-  
-  @Override
-  public long getProtocolVersion(String protocol, long clientVersion)
-      throws IOException {
-    return RPC.getProtocolVersion(NamenodeProtocolPB.class);
-  }
-
-  /**
-   * The client side will redirect getProtocolSignature to
-   * getProtocolSignature2.
-   * 
-   * However the RPC layer below on the Server side will call getProtocolVersion
-   * and possibly in the future getProtocolSignature. Hence we still implement
-   * it even though the end client will never call this method.
-   */
-  @Override
-  public ProtocolSignature getProtocolSignature(String protocol,
-      long clientVersion, int clientMethodsHash) throws IOException {
-    /**
-     * Don't forward this to the server. The protocol version and signature is
-     * that of {@link NamenodeProtocol}
-     */
-    if (!protocol.equals(RPC.getProtocolName(NamenodeProtocolPB.class))) {
-      throw new IOException("Namenode Serverside implements " +
-          RPC.getProtocolName(NamenodeProtocolPB.class) +
-          ". The following requested protocol is unknown: " + protocol);
-    }
-
-    return ProtocolSignature.getProtocolSignature(clientMethodsHash,
-        RPC.getProtocolVersion(NamenodeProtocolPB.class),
-        NamenodeProtocolPB.class);
-  }
-
-
-  @Override
-  public ProtocolSignatureWritable getProtocolSignature2(String protocol,
-      long clientVersion, int clientMethodsHash) throws IOException {
-    /**
-     * Don't forward this to the server. The protocol version and signature is
-     * that of {@link NamenodePBProtocol}
-     */
-    return ProtocolSignatureWritable.convert(
-        this.getProtocolSignature(protocol, clientVersion, clientMethodsHash));
-  }
 
   @Override
   public VersionResponseProto versionRequest(RpcController controller,

+ 0 - 13
hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/protocolPB/NamenodeProtocolTranslatorPB.java

@@ -102,19 +102,6 @@ public class NamenodeProtocolTranslatorPB implements NamenodeProtocol,
     RPC.stopProxy(rpcProxy);
   }
 
-  @Override
-  public ProtocolSignature getProtocolSignature(String protocolName,
-      long clientVersion, int clientMethodHash) throws IOException {
-    return ProtocolSignatureWritable.convert(rpcProxy.getProtocolSignature2(
-        protocolName, clientVersion, clientMethodHash));
-  }
-
-  @Override
-  public long getProtocolVersion(String protocolName, long clientVersion)
-      throws IOException {
-    return rpcProxy.getProtocolVersion(protocolName, clientVersion);
-  }
-
   @Override
   public BlocksWithLocations getBlocks(DatanodeInfo datanode, long size)
       throws IOException {

+ 0 - 13
hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/protocolPB/RefreshAuthorizationPolicyProtocolClientSideTranslatorPB.java

@@ -58,19 +58,6 @@ public class RefreshAuthorizationPolicyProtocolClientSideTranslatorPB implements
         NetUtils.getSocketFactory(conf, RefreshAuthorizationPolicyProtocol.class));
   }
 
-  @Override
-  public long getProtocolVersion(String protocol, long clientVersion)
-      throws IOException {
-    return rpcProxy.getProtocolVersion(protocol, clientVersion);
-  }
-
-  @Override
-  public ProtocolSignature getProtocolSignature(String protocol,
-      long clientVersion, int clientMethodsHash) throws IOException {
-    return ProtocolSignatureWritable.convert(rpcProxy.getProtocolSignature2(
-        protocol, clientVersion, clientMethodsHash));
-  }
-
   @Override
   public void close() throws IOException {
     RPC.stopProxy(rpcProxy);

+ 1 - 13
hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/protocolPB/RefreshAuthorizationPolicyProtocolPB.java

@@ -18,15 +18,11 @@
 
 package org.apache.hadoop.hdfs.protocolPB;
 
-import java.io.IOException;
-
 import org.apache.hadoop.classification.InterfaceAudience;
 import org.apache.hadoop.classification.InterfaceStability;
 import org.apache.hadoop.fs.CommonConfigurationKeys;
 import org.apache.hadoop.hdfs.protocol.proto.RefreshAuthorizationPolicyProtocolProtos.RefreshAuthorizationPolicyProtocolService;
-import org.apache.hadoop.hdfs.protocolR23Compatible.ProtocolSignatureWritable;
 import org.apache.hadoop.ipc.ProtocolInfo;
-import org.apache.hadoop.ipc.VersionedProtocol;
 import org.apache.hadoop.security.KerberosInfo;
 
 @KerberosInfo(
@@ -37,13 +33,5 @@ import org.apache.hadoop.security.KerberosInfo;
 @InterfaceAudience.LimitedPrivate({"HDFS", "MapReduce"})
 @InterfaceStability.Evolving
 public interface RefreshAuthorizationPolicyProtocolPB extends
-  RefreshAuthorizationPolicyProtocolService.BlockingInterface, VersionedProtocol {
-  
-  /**
-   * This method is defined to get the protocol signature using 
-   * the R23 protocol - hence we have added the suffix of 2 the method name
-   * to avoid conflict.
-   */
-  public ProtocolSignatureWritable getProtocolSignature2(String protocol,
-      long clientVersion, int clientMethodsHash) throws IOException;
+  RefreshAuthorizationPolicyProtocolService.BlockingInterface {
 }

+ 0 - 39
hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/protocolPB/RefreshAuthorizationPolicyProtocolServerSideTranslatorPB.java

@@ -22,9 +22,6 @@ import java.io.IOException;
 
 import org.apache.hadoop.hdfs.protocol.proto.RefreshAuthorizationPolicyProtocolProtos.RefreshServiceAclRequestProto;
 import org.apache.hadoop.hdfs.protocol.proto.RefreshAuthorizationPolicyProtocolProtos.RefreshServiceAclResponseProto;
-import org.apache.hadoop.hdfs.protocolR23Compatible.ProtocolSignatureWritable;
-import org.apache.hadoop.ipc.ProtocolSignature;
-import org.apache.hadoop.ipc.RPC;
 import org.apache.hadoop.security.authorize.RefreshAuthorizationPolicyProtocol;
 
 import com.google.protobuf.RpcController;
@@ -40,42 +37,6 @@ public class RefreshAuthorizationPolicyProtocolServerSideTranslatorPB implements
     this.impl = impl;
   }
 
-  @Override
-  public long getProtocolVersion(String protocol, long clientVersion)
-      throws IOException {
-    return RPC.getProtocolVersion(RefreshAuthorizationPolicyProtocolPB.class);
-  }
-
-  @Override
-  public ProtocolSignature getProtocolSignature(String protocol,
-      long clientVersion, int clientMethodsHash) throws IOException {
-    /**
-     * Don't forward this to the server. The protocol version and signature is
-     * that of {@link RefreshAuthorizationPolicyProtocol}
-     */
-    if (!protocol.equals(RPC
-        .getProtocolName(RefreshAuthorizationPolicyProtocolPB.class))) {
-      throw new IOException("Namenode Serverside implements "
-          + RPC.getProtocolName(RefreshAuthorizationPolicyProtocolPB.class)
-          + ". The following requested protocol is unknown: " + protocol);
-    }
-
-    return ProtocolSignature.getProtocolSignature(clientMethodsHash,
-        RPC.getProtocolVersion(RefreshAuthorizationPolicyProtocolPB.class),
-        RefreshAuthorizationPolicyProtocolPB.class);
-  }
-
-  @Override
-  public ProtocolSignatureWritable getProtocolSignature2(String protocol,
-      long clientVersion, int clientMethodsHash) throws IOException {
-    /**
-     * Don't forward this to the server. The protocol version and signature is
-     * that of {@link RefreshAuthorizationPolicyProtocolPB}
-     */
-    return ProtocolSignatureWritable.convert(this.getProtocolSignature(
-        protocol, clientVersion, clientMethodsHash));
-  }
-
   @Override
   public RefreshServiceAclResponseProto refreshServiceAcl(
       RpcController controller, RefreshServiceAclRequestProto request)

+ 0 - 13
hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/protocolPB/RefreshUserMappingsProtocolClientSideTranslatorPB.java

@@ -59,19 +59,6 @@ public class RefreshUserMappingsProtocolClientSideTranslatorPB implements
         NetUtils.getSocketFactory(conf, RefreshUserMappingsProtocol.class));
   }
 
-  @Override
-  public long getProtocolVersion(String protocol, long clientVersion)
-      throws IOException {
-    return rpcProxy.getProtocolVersion(protocol, clientVersion);
-  }
-
-  @Override
-  public ProtocolSignature getProtocolSignature(String protocol,
-      long clientVersion, int clientMethodsHash) throws IOException {
-    return ProtocolSignatureWritable.convert(rpcProxy.getProtocolSignature2(
-        protocol, clientVersion, clientMethodsHash));
-  }
-
   @Override
   public void close() throws IOException {
     RPC.stopProxy(rpcProxy);

+ 1 - 13
hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/protocolPB/RefreshUserMappingsProtocolPB.java

@@ -18,15 +18,11 @@
 
 package org.apache.hadoop.hdfs.protocolPB;
 
-import java.io.IOException;
-
 import org.apache.hadoop.classification.InterfaceAudience;
 import org.apache.hadoop.classification.InterfaceStability;
 import org.apache.hadoop.fs.CommonConfigurationKeys;
 import org.apache.hadoop.hdfs.protocol.proto.RefreshUserMappingsProtocolProtos.RefreshUserMappingsProtocolService;
-import org.apache.hadoop.hdfs.protocolR23Compatible.ProtocolSignatureWritable;
 import org.apache.hadoop.ipc.ProtocolInfo;
-import org.apache.hadoop.ipc.VersionedProtocol;
 import org.apache.hadoop.security.KerberosInfo;
 
 @KerberosInfo(
@@ -37,13 +33,5 @@ import org.apache.hadoop.security.KerberosInfo;
 @InterfaceAudience.LimitedPrivate({"HDFS", "MapReduce"})
 @InterfaceStability.Evolving
 public interface RefreshUserMappingsProtocolPB extends
-    RefreshUserMappingsProtocolService.BlockingInterface, VersionedProtocol {
-  
-  /**
-   * This method is defined to get the protocol signature using 
-   * the R23 protocol - hence we have added the suffix of 2 the method name
-   * to avoid conflict.
-   */
-  public ProtocolSignatureWritable getProtocolSignature2(String protocol,
-      long clientVersion, int clientMethodsHash) throws IOException;
+    RefreshUserMappingsProtocolService.BlockingInterface {
 }

+ 0 - 39
hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/protocolPB/RefreshUserMappingsProtocolServerSideTranslatorPB.java

@@ -24,9 +24,6 @@ import org.apache.hadoop.hdfs.protocol.proto.RefreshUserMappingsProtocolProtos.R
 import org.apache.hadoop.hdfs.protocol.proto.RefreshUserMappingsProtocolProtos.RefreshSuperUserGroupsConfigurationResponseProto;
 import org.apache.hadoop.hdfs.protocol.proto.RefreshUserMappingsProtocolProtos.RefreshUserToGroupsMappingsRequestProto;
 import org.apache.hadoop.hdfs.protocol.proto.RefreshUserMappingsProtocolProtos.RefreshUserToGroupsMappingsResponseProto;
-import org.apache.hadoop.hdfs.protocolR23Compatible.ProtocolSignatureWritable;
-import org.apache.hadoop.ipc.ProtocolSignature;
-import org.apache.hadoop.ipc.RPC;
 import org.apache.hadoop.security.RefreshUserMappingsProtocol;
 
 import com.google.protobuf.RpcController;
@@ -66,40 +63,4 @@ public class RefreshUserMappingsProtocolServerSideTranslatorPB implements Refres
     return RefreshSuperUserGroupsConfigurationResponseProto.newBuilder()
         .build();
   }
-
-  @Override
-  public long getProtocolVersion(String protocol, long clientVersion)
-      throws IOException {
-    return RPC.getProtocolVersion(RefreshUserMappingsProtocolPB.class);
-  }
-
-  @Override
-  public ProtocolSignature getProtocolSignature(String protocol,
-      long clientVersion, int clientMethodsHash) throws IOException {
-    /**
-     * Don't forward this to the server. The protocol version and signature is
-     * that of {@link RefreshUserMappingsProtocol}
-     */
-    if (!protocol.equals(RPC
-        .getProtocolName(RefreshUserMappingsProtocolPB.class))) {
-      throw new IOException("Namenode Serverside implements "
-          + RPC.getProtocolName(RefreshUserMappingsProtocolPB.class)
-          + ". The following requested protocol is unknown: " + protocol);
-    }
-
-    return ProtocolSignature.getProtocolSignature(clientMethodsHash,
-        RPC.getProtocolVersion(RefreshUserMappingsProtocolPB.class),
-        RefreshUserMappingsProtocolPB.class);
-  }
-
-  @Override
-  public ProtocolSignatureWritable getProtocolSignature2(String protocol,
-      long clientVersion, int clientMethodsHash) throws IOException {
-    /**
-     * Don't forward this to the server. The protocol version and signature is
-     * that of {@link RefreshUserMappingsProtocolPB}
-     */
-    return ProtocolSignatureWritable.convert(this.getProtocolSignature(
-        protocol, clientVersion, clientMethodsHash));
-  }
 }

+ 6 - 22
hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/DataNode.java

@@ -704,9 +704,12 @@ public class DataNode extends Configured
                     // DatanodeProtocol namenode,
                      SecureResources resources
                      ) throws IOException {
-    if(UserGroupInformation.isSecurityEnabled() && resources == null)
-      throw new RuntimeException("Cannot start secure cluster without " +
-      "privileged resources.");
+    if(UserGroupInformation.isSecurityEnabled() && resources == null) {
+      if (!conf.getBoolean("ignore.secure.ports.for.testing", false)) {
+        throw new RuntimeException("Cannot start secure cluster without "
+            + "privileged resources.");
+      }
+    }
 
     // settings global for all BPs in the Data Node
     this.secureResources = resources;
@@ -1855,25 +1858,6 @@ public class DataNode extends Configured
     return new ExtendedBlock(oldBlock.getBlockPoolId(), r);
   }
 
-  @Override
-  public long getProtocolVersion(String protocol, long clientVersion
-      ) throws IOException {
-    if (protocol.equals(InterDatanodeProtocol.class.getName())) {
-      return InterDatanodeProtocol.versionID; 
-    } else if (protocol.equals(ClientDatanodeProtocol.class.getName())) {
-      return ClientDatanodeProtocol.versionID; 
-    }
-    throw new IOException("Unknown protocol to " + getClass().getSimpleName()
-        + ": " + protocol);
-  }
-
-  @Override
-  public ProtocolSignature getProtocolSignature(String protocol,
-      long clientVersion, int clientMethodsHash) throws IOException {
-    return ProtocolSignature.getProtocolSignature(
-        this, protocol, clientVersion, clientMethodsHash);
-  }
-
   /** A convenient class used in block recovery */
   static class BlockRecord { 
     final DatanodeID id;

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

@@ -41,7 +41,6 @@ import org.apache.hadoop.hdfs.server.protocol.NamenodeProtocol;
 import org.apache.hadoop.hdfs.server.protocol.NamenodeRegistration;
 import org.apache.hadoop.hdfs.server.protocol.NamespaceInfo;
 import org.apache.hadoop.io.IOUtils;
-import org.apache.hadoop.ipc.RPC;
 import org.apache.hadoop.net.NetUtils;
 import org.apache.hadoop.security.UserGroupInformation;
 
@@ -212,15 +211,6 @@ public class BackupNode extends NameNode {
           this.clientRpcServer);
       nnRpcAddress = nn.nnRpcAddress;
     }
-
-    @Override
-    public long getProtocolVersion(String protocol, long clientVersion)
-        throws IOException {
-      if (protocol.equals(JournalProtocol.class.getName())) {
-        return JournalProtocol.versionID;
-      }
-      return super.getProtocolVersion(protocol, clientVersion);
-    }
   
     /////////////////////////////////////////////////////
     // NamenodeProtocol implementation for backup node.

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

@@ -17,6 +17,10 @@
  */
 package org.apache.hadoop.hdfs.server.namenode;
 
+import static org.apache.hadoop.hdfs.DFSConfigKeys.DFS_NAMENODE_HANDLER_COUNT_DEFAULT;
+import static org.apache.hadoop.hdfs.DFSConfigKeys.DFS_NAMENODE_HANDLER_COUNT_KEY;
+import static org.apache.hadoop.hdfs.DFSConfigKeys.DFS_NAMENODE_SERVICE_HANDLER_COUNT_DEFAULT;
+import static org.apache.hadoop.hdfs.DFSConfigKeys.DFS_NAMENODE_SERVICE_HANDLER_COUNT_KEY;
 import static org.apache.hadoop.hdfs.protocol.HdfsConstants.MAX_PATH_DEPTH;
 import static org.apache.hadoop.hdfs.protocol.HdfsConstants.MAX_PATH_LENGTH;
 
@@ -37,51 +41,48 @@ import org.apache.hadoop.fs.Path;
 import org.apache.hadoop.fs.UnresolvedLinkException;
 import org.apache.hadoop.fs.permission.FsPermission;
 import org.apache.hadoop.fs.permission.PermissionStatus;
-import static org.apache.hadoop.hdfs.DFSConfigKeys.*;
-
 import org.apache.hadoop.hdfs.DFSUtil;
 import org.apache.hadoop.hdfs.HDFSPolicyProvider;
 import org.apache.hadoop.hdfs.HdfsConfiguration;
 import org.apache.hadoop.hdfs.protocol.Block;
 import org.apache.hadoop.hdfs.protocol.BlockListAsLongs;
-import org.apache.hadoop.hdfs.protocol.ClientProtocol;
 import org.apache.hadoop.hdfs.protocol.CorruptFileBlocks;
 import org.apache.hadoop.hdfs.protocol.DatanodeID;
 import org.apache.hadoop.hdfs.protocol.DatanodeInfo;
 import org.apache.hadoop.hdfs.protocol.DirectoryListing;
 import org.apache.hadoop.hdfs.protocol.ExtendedBlock;
 import org.apache.hadoop.hdfs.protocol.HdfsConstants;
+import org.apache.hadoop.hdfs.protocol.HdfsConstants.DatanodeReportType;
+import org.apache.hadoop.hdfs.protocol.HdfsConstants.SafeModeAction;
+import org.apache.hadoop.hdfs.protocol.HdfsConstants.UpgradeAction;
 import org.apache.hadoop.hdfs.protocol.HdfsFileStatus;
 import org.apache.hadoop.hdfs.protocol.LocatedBlock;
 import org.apache.hadoop.hdfs.protocol.LocatedBlocks;
 import org.apache.hadoop.hdfs.protocol.UnregisteredNodeException;
 import org.apache.hadoop.hdfs.protocol.UnresolvedPathException;
-import org.apache.hadoop.hdfs.protocol.HdfsConstants.DatanodeReportType;
-import org.apache.hadoop.hdfs.protocol.HdfsConstants.SafeModeAction;
-import org.apache.hadoop.hdfs.protocol.HdfsConstants.UpgradeAction;
 import org.apache.hadoop.hdfs.protocol.proto.ClientNamenodeProtocolProtos.ClientNamenodeProtocol;
-import org.apache.hadoop.hdfs.protocol.proto.NamenodeProtocolProtos.NamenodeProtocolService;
 import org.apache.hadoop.hdfs.protocol.proto.DatanodeProtocolProtos.DatanodeProtocolService;
 import org.apache.hadoop.hdfs.protocol.proto.GetUserMappingsProtocolProtos.GetUserMappingsProtocolService;
+import org.apache.hadoop.hdfs.protocol.proto.NamenodeProtocolProtos.NamenodeProtocolService;
 import org.apache.hadoop.hdfs.protocol.proto.RefreshAuthorizationPolicyProtocolProtos.RefreshAuthorizationPolicyProtocolService;
 import org.apache.hadoop.hdfs.protocol.proto.RefreshUserMappingsProtocolProtos.RefreshUserMappingsProtocolService;
+import org.apache.hadoop.hdfs.protocolPB.ClientNamenodeProtocolPB;
+import org.apache.hadoop.hdfs.protocolPB.ClientNamenodeProtocolServerSideTranslatorPB;
 import org.apache.hadoop.hdfs.protocolPB.DatanodeProtocolPB;
 import org.apache.hadoop.hdfs.protocolPB.DatanodeProtocolServerSideTranslatorPB;
 import org.apache.hadoop.hdfs.protocolPB.GetUserMappingsProtocolPB;
 import org.apache.hadoop.hdfs.protocolPB.GetUserMappingsProtocolServerSideTranslatorPB;
 import org.apache.hadoop.hdfs.protocolPB.NamenodeProtocolPB;
 import org.apache.hadoop.hdfs.protocolPB.NamenodeProtocolServerSideTranslatorPB;
-import org.apache.hadoop.hdfs.protocolPB.ClientNamenodeProtocolPB;
-import org.apache.hadoop.hdfs.protocolPB.ClientNamenodeProtocolServerSideTranslatorPB;
 import org.apache.hadoop.hdfs.protocolPB.RefreshAuthorizationPolicyProtocolPB;
 import org.apache.hadoop.hdfs.protocolPB.RefreshAuthorizationPolicyProtocolServerSideTranslatorPB;
 import org.apache.hadoop.hdfs.protocolPB.RefreshUserMappingsProtocolPB;
 import org.apache.hadoop.hdfs.protocolPB.RefreshUserMappingsProtocolServerSideTranslatorPB;
 import org.apache.hadoop.hdfs.security.token.block.ExportedBlockKeys;
 import org.apache.hadoop.hdfs.security.token.delegation.DelegationTokenIdentifier;
+import org.apache.hadoop.hdfs.server.common.HdfsServerConstants.NamenodeRole;
 import org.apache.hadoop.hdfs.server.common.IncorrectVersionException;
 import org.apache.hadoop.hdfs.server.common.UpgradeStatusReport;
-import org.apache.hadoop.hdfs.server.common.HdfsServerConstants.NamenodeRole;
 import org.apache.hadoop.hdfs.server.namenode.metrics.NameNodeMetrics;
 import org.apache.hadoop.hdfs.server.namenode.web.resources.NamenodeWebHdfsMethods;
 import org.apache.hadoop.hdfs.server.protocol.BlocksWithLocations;
@@ -91,7 +92,6 @@ import org.apache.hadoop.hdfs.server.protocol.DatanodeRegistration;
 import org.apache.hadoop.hdfs.server.protocol.DatanodeStorage;
 import org.apache.hadoop.hdfs.server.protocol.FinalizeCommand;
 import org.apache.hadoop.hdfs.server.protocol.NamenodeCommand;
-import org.apache.hadoop.hdfs.server.protocol.NamenodeProtocol;
 import org.apache.hadoop.hdfs.server.protocol.NamenodeProtocols;
 import org.apache.hadoop.hdfs.server.protocol.NamenodeRegistration;
 import org.apache.hadoop.hdfs.server.protocol.NamespaceInfo;
@@ -104,21 +104,17 @@ import org.apache.hadoop.hdfs.server.protocol.UpgradeCommand;
 import org.apache.hadoop.io.EnumSetWritable;
 import org.apache.hadoop.io.Text;
 import org.apache.hadoop.ipc.ProtobufRpcEngine;
-import org.apache.hadoop.ipc.ProtocolSignature;
 import org.apache.hadoop.ipc.RPC;
 import org.apache.hadoop.ipc.Server;
 import org.apache.hadoop.ipc.WritableRpcEngine;
 import org.apache.hadoop.net.Node;
 import org.apache.hadoop.security.AccessControlException;
 import org.apache.hadoop.security.Groups;
-import org.apache.hadoop.security.RefreshUserMappingsProtocol;
 import org.apache.hadoop.security.UserGroupInformation;
 import org.apache.hadoop.security.authorize.AuthorizationException;
 import org.apache.hadoop.security.authorize.ProxyUsers;
-import org.apache.hadoop.security.authorize.RefreshAuthorizationPolicyProtocol;
-import org.apache.hadoop.security.token.Token;
 import org.apache.hadoop.security.token.SecretManager.InvalidToken;
-import org.apache.hadoop.tools.GetUserMappingsProtocol;
+import org.apache.hadoop.security.token.Token;
 
 import com.google.protobuf.BlockingService;
 
@@ -281,34 +277,6 @@ class NameNodeRpcServer implements NamenodeProtocols {
   InetSocketAddress getRpcAddress() {
     return clientRpcAddress;
   }
-  
-  @Override // VersionedProtocol
-  public ProtocolSignature getProtocolSignature(String protocol,
-      long clientVersion, int clientMethodsHash) throws IOException {
-    return ProtocolSignature.getProtocolSignature(
-        this, protocol, clientVersion, clientMethodsHash);
-  }
-  
-  @Override
-  public long getProtocolVersion(String protocol, 
-                                 long clientVersion) throws IOException {
-    if (protocol.equals(ClientProtocol.class.getName())) {
-      throw new IOException("Old Namenode Client protocol is not supported:" + 
-      protocol + "Switch your clientside to " + ClientNamenodeProtocol.class); 
-    } else if (protocol.equals(DatanodeProtocol.class.getName())){
-      return DatanodeProtocol.versionID;
-    } else if (protocol.equals(NamenodeProtocol.class.getName())){
-      return NamenodeProtocol.versionID;
-    } else if (protocol.equals(RefreshAuthorizationPolicyProtocol.class.getName())){
-      return RefreshAuthorizationPolicyProtocol.versionID;
-    } else if (protocol.equals(RefreshUserMappingsProtocol.class.getName())){
-      return RefreshUserMappingsProtocol.versionID;
-    } else if (protocol.equals(GetUserMappingsProtocol.class.getName())){
-      return GetUserMappingsProtocol.versionID;
-    } else {
-      throw new IOException("Unknown protocol to name node: " + protocol);
-    }
-  }
 
   /////////////////////////////////////////////////////
   // NamenodeProtocol

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

@@ -25,7 +25,6 @@ import org.apache.hadoop.hdfs.DFSConfigKeys;
 import org.apache.hadoop.hdfs.protocol.ExtendedBlock;
 import org.apache.hadoop.hdfs.protocol.DatanodeID;
 import org.apache.hadoop.hdfs.protocol.LocatedBlock;
-import org.apache.hadoop.ipc.VersionedProtocol;
 import org.apache.hadoop.security.KerberosInfo;
 
 /**********************************************************************
@@ -40,7 +39,7 @@ import org.apache.hadoop.security.KerberosInfo;
     serverPrincipal = DFSConfigKeys.DFS_NAMENODE_USER_NAME_KEY, 
     clientPrincipal = DFSConfigKeys.DFS_DATANODE_USER_NAME_KEY)
 @InterfaceAudience.Private
-public interface DatanodeProtocol extends VersionedProtocol {
+public interface DatanodeProtocol {
   /**
    * This class is used by both the Namenode (client) and BackupNode (server) 
    * to insulate from the protocol serialization.

+ 1 - 1
hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/protocol/InterDatanodeProtocol.java

@@ -35,7 +35,7 @@ import org.apache.hadoop.security.KerberosInfo;
     serverPrincipal = DFSConfigKeys.DFS_DATANODE_USER_NAME_KEY,
     clientPrincipal = DFSConfigKeys.DFS_DATANODE_USER_NAME_KEY)
 @InterfaceAudience.Private
-public interface InterDatanodeProtocol extends VersionedProtocol {
+public interface InterDatanodeProtocol {
   public static final Log LOG = LogFactory.getLog(InterDatanodeProtocol.class);
 
   /**

+ 1 - 1
hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/protocol/JournalProtocol.java

@@ -32,7 +32,7 @@ import org.apache.hadoop.security.KerberosInfo;
     serverPrincipal = DFSConfigKeys.DFS_NAMENODE_USER_NAME_KEY,
     clientPrincipal = DFSConfigKeys.DFS_NAMENODE_USER_NAME_KEY)
 @InterfaceAudience.Private
-public interface JournalProtocol extends VersionedProtocol {
+public interface JournalProtocol {
   /**
    * 
    * This class is used by both the Namenode (client) and BackupNode (server) 

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

@@ -36,7 +36,7 @@ import org.apache.hadoop.security.KerberosInfo;
     serverPrincipal = DFSConfigKeys.DFS_NAMENODE_USER_NAME_KEY,
     clientPrincipal = DFSConfigKeys.DFS_NAMENODE_USER_NAME_KEY)
 @InterfaceAudience.Private
-public interface NamenodeProtocol extends VersionedProtocol {
+public interface NamenodeProtocol {
   /**
    * Until version 6L, this class served as both
    * the client interface to the NN AND the RPC protocol used to 

+ 9 - 23
hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/security/TestClientProtocolWithDelegationToken.java

@@ -18,39 +18,31 @@
 
 package org.apache.hadoop.hdfs.security;
 
-import static org.apache.hadoop.fs.CommonConfigurationKeys.HADOOP_SECURITY_AUTHENTICATION;
-import static org.mockito.Matchers.anyInt;
-import static org.mockito.Matchers.anyLong;
-import static org.mockito.Matchers.anyString;
+import static org.apache.hadoop.fs.CommonConfigurationKeysPublic.HADOOP_SECURITY_AUTHENTICATION;
 import static org.mockito.Mockito.mock;
-import static org.mockito.Mockito.doReturn;
-import static org.mockito.Mockito.when;
 
 import java.net.InetSocketAddress;
 import java.security.PrivilegedExceptionAction;
 
-import org.apache.commons.logging.*;
+import org.apache.commons.logging.Log;
+import org.apache.commons.logging.LogFactory;
 import org.apache.commons.logging.impl.Log4JLogger;
-
 import org.apache.hadoop.conf.Configuration;
-import org.apache.hadoop.io.Text;
-
-import org.apache.hadoop.ipc.Client;
-import org.apache.hadoop.ipc.RPC;
-import org.apache.hadoop.ipc.ProtocolSignature;
-import org.apache.hadoop.ipc.Server;
 import org.apache.hadoop.hdfs.DFSConfigKeys;
 import org.apache.hadoop.hdfs.protocol.ClientProtocol;
-import org.apache.hadoop.net.NetUtils;
-import org.apache.hadoop.security.token.Token;
 import org.apache.hadoop.hdfs.security.token.delegation.DelegationTokenIdentifier;
 import org.apache.hadoop.hdfs.security.token.delegation.DelegationTokenSecretManager;
 import org.apache.hadoop.hdfs.server.namenode.FSNamesystem;
+import org.apache.hadoop.io.Text;
+import org.apache.hadoop.ipc.Client;
+import org.apache.hadoop.ipc.RPC;
+import org.apache.hadoop.ipc.Server;
+import org.apache.hadoop.net.NetUtils;
 import org.apache.hadoop.security.SaslInputStream;
 import org.apache.hadoop.security.SaslRpcClient;
 import org.apache.hadoop.security.SaslRpcServer;
 import org.apache.hadoop.security.UserGroupInformation;
-
+import org.apache.hadoop.security.token.Token;
 import org.apache.log4j.Level;
 import org.junit.Test;
 
@@ -80,12 +72,6 @@ public class TestClientProtocolWithDelegationToken {
   public void testDelegationTokenRpc() throws Exception {
     ClientProtocol mockNN = mock(ClientProtocol.class);
     FSNamesystem mockNameSys = mock(FSNamesystem.class);
-    when(mockNN.getProtocolVersion(anyString(), anyLong())).thenReturn(
-        ClientProtocol.versionID);
-    doReturn(ProtocolSignature.getProtocolSignature(
-        mockNN, ClientProtocol.class.getName(),
-        ClientProtocol.versionID, 0))
-      .when(mockNN).getProtocolSignature(anyString(), anyLong(), anyInt());
 
     DelegationTokenSecretManager sm = new DelegationTokenSecretManager(
         DFSConfigKeys.DFS_NAMENODE_DELEGATION_KEY_UPDATE_INTERVAL_DEFAULT,

+ 1 - 14
hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/security/token/block/TestBlockToken.java

@@ -23,13 +23,8 @@ import static org.junit.Assert.assertEquals;
 import static org.junit.Assert.assertTrue;
 import static org.junit.Assert.fail;
 import static org.mockito.Matchers.any;
-import static org.mockito.Matchers.anyInt;
-import static org.mockito.Matchers.anyLong;
-import static org.mockito.Matchers.anyString;
 import static org.mockito.Mockito.doAnswer;
-import static org.mockito.Mockito.doReturn;
 import static org.mockito.Mockito.mock;
-import static org.mockito.Mockito.when;
 
 import java.io.ByteArrayInputStream;
 import java.io.DataInputStream;
@@ -51,12 +46,12 @@ import org.apache.hadoop.hdfs.DFSUtil;
 import org.apache.hadoop.hdfs.HdfsConfiguration;
 import org.apache.hadoop.hdfs.MiniDFSCluster;
 import org.apache.hadoop.hdfs.protocol.Block;
+import org.apache.hadoop.hdfs.protocol.ClientDatanodeProtocol;
 import org.apache.hadoop.hdfs.protocol.DatanodeID;
 import org.apache.hadoop.hdfs.protocol.DatanodeInfo;
 import org.apache.hadoop.hdfs.protocol.ExtendedBlock;
 import org.apache.hadoop.hdfs.protocol.LocatedBlock;
 import org.apache.hadoop.hdfs.protocol.LocatedBlocks;
-import org.apache.hadoop.hdfs.protocol.ClientDatanodeProtocol;
 import org.apache.hadoop.hdfs.protocol.proto.ClientDatanodeProtocolProtos.ClientDatanodeProtocolService;
 import org.apache.hadoop.hdfs.protocol.proto.ClientDatanodeProtocolProtos.GetReplicaVisibleLengthRequestProto;
 import org.apache.hadoop.hdfs.protocol.proto.ClientDatanodeProtocolProtos.GetReplicaVisibleLengthResponseProto;
@@ -65,7 +60,6 @@ import org.apache.hadoop.hdfs.protocolPB.PBHelper;
 import org.apache.hadoop.io.TestWritable;
 import org.apache.hadoop.ipc.Client;
 import org.apache.hadoop.ipc.ProtobufRpcEngine;
-import org.apache.hadoop.ipc.ProtocolSignature;
 import org.apache.hadoop.ipc.RPC;
 import org.apache.hadoop.ipc.Server;
 import org.apache.hadoop.net.NetUtils;
@@ -222,13 +216,6 @@ public class TestBlockToken {
   private Server createMockDatanode(BlockTokenSecretManager sm,
       Token<BlockTokenIdentifier> token) throws IOException, ServiceException {
     ClientDatanodeProtocolPB mockDN = mock(ClientDatanodeProtocolPB.class);
-    when(mockDN.getProtocolVersion(anyString(), anyLong())).thenReturn(
-        RPC.getProtocolVersion(ClientDatanodeProtocolPB.class));
-    doReturn(
-        ProtocolSignature.getProtocolSignature(mockDN,
-            ClientDatanodeProtocolPB.class.getName(),
-            RPC.getProtocolVersion(ClientDatanodeProtocolPB.class), 0)).when(
-        mockDN).getProtocolSignature(anyString(), anyLong(), anyInt());
 
     BlockTokenIdentifier id = sm.createIdentifier();
     id.readFields(new DataInputStream(new ByteArrayInputStream(token