You are viewing a plain text version of this content. The canonical link for it is here.
Posted to hdfs-commits@hadoop.apache.org by ji...@apache.org on 2012/02/29 09:52:47 UTC

svn commit: r1295017 [1/2] - in /hadoop/common/trunk/hadoop-hdfs-project/hadoop-hdfs: ./ src/main/java/org/apache/hadoop/hdfs/protocol/ src/main/java/org/apache/hadoop/hdfs/protocolPB/ src/main/java/org/apache/hadoop/hdfs/server/datanode/ src/main/java...

Author: jitendra
Date: Wed Feb 29 08:52:45 2012
New Revision: 1295017

URL: http://svn.apache.org/viewvc?rev=1295017&view=rev
Log:
HDFS-3030. Remove getProtocolVersion and getProtocolSignature from translators. 

Modified:
    hadoop/common/trunk/hadoop-hdfs-project/hadoop-hdfs/CHANGES.txt
    hadoop/common/trunk/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/protocol/ClientDatanodeProtocol.java
    hadoop/common/trunk/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/protocol/ClientProtocol.java
    hadoop/common/trunk/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/protocolPB/ClientDatanodeProtocolPB.java
    hadoop/common/trunk/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/protocolPB/ClientDatanodeProtocolServerSideTranslatorPB.java
    hadoop/common/trunk/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/protocolPB/ClientDatanodeProtocolTranslatorPB.java
    hadoop/common/trunk/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/protocolPB/ClientNamenodeProtocolPB.java
    hadoop/common/trunk/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/protocolPB/ClientNamenodeProtocolServerSideTranslatorPB.java
    hadoop/common/trunk/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/protocolPB/ClientNamenodeProtocolTranslatorPB.java
    hadoop/common/trunk/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/protocolPB/DatanodeProtocolClientSideTranslatorPB.java
    hadoop/common/trunk/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/protocolPB/DatanodeProtocolPB.java
    hadoop/common/trunk/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/protocolPB/DatanodeProtocolServerSideTranslatorPB.java
    hadoop/common/trunk/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/protocolPB/GetUserMappingsProtocolClientSideTranslatorPB.java
    hadoop/common/trunk/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/protocolPB/GetUserMappingsProtocolPB.java
    hadoop/common/trunk/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/protocolPB/GetUserMappingsProtocolServerSideTranslatorPB.java
    hadoop/common/trunk/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/protocolPB/InterDatanodeProtocolPB.java
    hadoop/common/trunk/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/protocolPB/InterDatanodeProtocolServerSideTranslatorPB.java
    hadoop/common/trunk/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/protocolPB/InterDatanodeProtocolTranslatorPB.java
    hadoop/common/trunk/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/protocolPB/JournalProtocolPB.java
    hadoop/common/trunk/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/protocolPB/JournalProtocolServerSideTranslatorPB.java
    hadoop/common/trunk/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/protocolPB/JournalProtocolTranslatorPB.java
    hadoop/common/trunk/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/protocolPB/NamenodeProtocolPB.java
    hadoop/common/trunk/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/protocolPB/NamenodeProtocolServerSideTranslatorPB.java
    hadoop/common/trunk/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/protocolPB/NamenodeProtocolTranslatorPB.java
    hadoop/common/trunk/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/protocolPB/RefreshAuthorizationPolicyProtocolClientSideTranslatorPB.java
    hadoop/common/trunk/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/protocolPB/RefreshAuthorizationPolicyProtocolPB.java
    hadoop/common/trunk/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/protocolPB/RefreshAuthorizationPolicyProtocolServerSideTranslatorPB.java
    hadoop/common/trunk/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/protocolPB/RefreshUserMappingsProtocolClientSideTranslatorPB.java
    hadoop/common/trunk/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/protocolPB/RefreshUserMappingsProtocolPB.java
    hadoop/common/trunk/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/protocolPB/RefreshUserMappingsProtocolServerSideTranslatorPB.java
    hadoop/common/trunk/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/DataNode.java
    hadoop/common/trunk/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/BackupNode.java
    hadoop/common/trunk/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/NameNodeRpcServer.java
    hadoop/common/trunk/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/protocol/DatanodeProtocol.java
    hadoop/common/trunk/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/protocol/InterDatanodeProtocol.java
    hadoop/common/trunk/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/protocol/JournalProtocol.java
    hadoop/common/trunk/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/protocol/NamenodeProtocol.java
    hadoop/common/trunk/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/security/TestClientProtocolWithDelegationToken.java
    hadoop/common/trunk/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/security/token/block/TestBlockToken.java

Modified: hadoop/common/trunk/hadoop-hdfs-project/hadoop-hdfs/CHANGES.txt
URL: http://svn.apache.org/viewvc/hadoop/common/trunk/hadoop-hdfs-project/hadoop-hdfs/CHANGES.txt?rev=1295017&r1=1295016&r2=1295017&view=diff
==============================================================================
--- hadoop/common/trunk/hadoop-hdfs-project/hadoop-hdfs/CHANGES.txt (original)
+++ hadoop/common/trunk/hadoop-hdfs-project/hadoop-hdfs/CHANGES.txt Wed Feb 29 08:52:45 2012
@@ -65,6 +65,9 @@ Trunk (unreleased changes)
     HDFS-3014. FSEditLogOp and its subclasses should have toString() method.
     (Sho Shimauchi via atm)
 
+    HDFS-3030. Remove getProtocolVersion and getProtocolSignature from translators.
+    (jitendra)
+
   OPTIMIZATIONS
 
     HDFS-2477. Optimize computing the diff between a block report and the

Modified: hadoop/common/trunk/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/protocol/ClientDatanodeProtocol.java
URL: http://svn.apache.org/viewvc/hadoop/common/trunk/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/protocol/ClientDatanodeProtocol.java?rev=1295017&r1=1295016&r2=1295017&view=diff
==============================================================================
--- hadoop/common/trunk/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/protocol/ClientDatanodeProtocol.java (original)
+++ hadoop/common/trunk/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/protocol/ClientDatanodeProtocol.java Wed Feb 29 08:52:45 2012
@@ -36,7 +36,7 @@ import org.apache.hadoop.security.token.
 @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 

Modified: hadoop/common/trunk/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/protocol/ClientProtocol.java
URL: http://svn.apache.org/viewvc/hadoop/common/trunk/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/protocol/ClientProtocol.java?rev=1295017&r1=1295016&r2=1295017&view=diff
==============================================================================
--- hadoop/common/trunk/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/protocol/ClientProtocol.java (original)
+++ hadoop/common/trunk/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/protocol/ClientProtocol.java Wed Feb 29 08:52:45 2012
@@ -39,7 +39,6 @@ import org.apache.hadoop.hdfs.server.nam
 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.t
 @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

Modified: hadoop/common/trunk/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/protocolPB/ClientDatanodeProtocolPB.java
URL: http://svn.apache.org/viewvc/hadoop/common/trunk/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/protocolPB/ClientDatanodeProtocolPB.java?rev=1295017&r1=1295016&r2=1295017&view=diff
==============================================================================
--- hadoop/common/trunk/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/protocolPB/ClientDatanodeProtocolPB.java (original)
+++ hadoop/common/trunk/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/protocolPB/ClientDatanodeProtocolPB.java Wed Feb 29 08:52:45 2012
@@ -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.
     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 {
 }

Modified: hadoop/common/trunk/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/protocolPB/ClientDatanodeProtocolServerSideTranslatorPB.java
URL: http://svn.apache.org/viewvc/hadoop/common/trunk/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/protocolPB/ClientDatanodeProtocolServerSideTranslatorPB.java?rev=1295017&r1=1295016&r2=1295017&view=diff
==============================================================================
--- hadoop/common/trunk/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/protocolPB/ClientDatanodeProtocolServerSideTranslatorPB.java (original)
+++ hadoop/common/trunk/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/protocolPB/ClientDatanodeProtocolServerSideTranslatorPB.java Wed Feb 29 08:52:45 2012
@@ -30,10 +30,6 @@ import org.apache.hadoop.hdfs.protocol.p
 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 ClientDatanodeProtocolServe
         .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));
-  }
 }

Modified: hadoop/common/trunk/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/protocolPB/ClientDatanodeProtocolTranslatorPB.java
URL: http://svn.apache.org/viewvc/hadoop/common/trunk/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/protocolPB/ClientDatanodeProtocolTranslatorPB.java?rev=1295017&r1=1295016&r2=1295017&view=diff
==============================================================================
--- hadoop/common/trunk/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/protocolPB/ClientDatanodeProtocolTranslatorPB.java (original)
+++ hadoop/common/trunk/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/protocolPB/ClientDatanodeProtocolTranslatorPB.java Wed Feb 29 08:52:45 2012
@@ -145,19 +145,6 @@ public class ClientDatanodeProtocolTrans
   }
 
   @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
         .newBuilder().setBlock(PBHelper.convert(b)).build();

Modified: hadoop/common/trunk/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/protocolPB/ClientNamenodeProtocolPB.java
URL: http://svn.apache.org/viewvc/hadoop/common/trunk/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/protocolPB/ClientNamenodeProtocolPB.java?rev=1295017&r1=1295016&r2=1295017&view=diff
==============================================================================
--- hadoop/common/trunk/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/protocolPB/ClientNamenodeProtocolPB.java (original)
+++ hadoop/common/trunk/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/protocolPB/ClientNamenodeProtocolPB.java Wed Feb 29 08:52:45 2012
@@ -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.
  * 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 {
 }

Modified: hadoop/common/trunk/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/protocolPB/ClientNamenodeProtocolServerSideTranslatorPB.java
URL: http://svn.apache.org/viewvc/hadoop/common/trunk/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/protocolPB/ClientNamenodeProtocolServerSideTranslatorPB.java?rev=1295017&r1=1295016&r2=1295017&view=diff
==============================================================================
--- hadoop/common/trunk/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/protocolPB/ClientNamenodeProtocolServerSideTranslatorPB.java (original)
+++ hadoop/common/trunk/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/protocolPB/ClientNamenodeProtocolServerSideTranslatorPB.java Wed Feb 29 08:52:45 2012
@@ -124,17 +124,11 @@ import org.apache.hadoop.hdfs.protocol.p
 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 ClientNamenodeProtocolServe
     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)

Modified: hadoop/common/trunk/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/protocolPB/ClientNamenodeProtocolTranslatorPB.java
URL: http://svn.apache.org/viewvc/hadoop/common/trunk/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/protocolPB/ClientNamenodeProtocolTranslatorPB.java?rev=1295017&r1=1295016&r2=1295017&view=diff
==============================================================================
--- hadoop/common/trunk/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/protocolPB/ClientNamenodeProtocolTranslatorPB.java (original)
+++ hadoop/common/trunk/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/protocolPB/ClientNamenodeProtocolTranslatorPB.java Wed Feb 29 08:52:45 2012
@@ -33,49 +33,26 @@ import org.apache.hadoop.fs.ContentSumma
 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.p
 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;
@@ -190,20 +188,6 @@ public class ClientNamenodeProtocolTrans
   }
 
   @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,
       UnresolvedLinkException, IOException {

Modified: hadoop/common/trunk/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/protocolPB/DatanodeProtocolClientSideTranslatorPB.java
URL: http://svn.apache.org/viewvc/hadoop/common/trunk/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/protocolPB/DatanodeProtocolClientSideTranslatorPB.java?rev=1295017&r1=1295016&r2=1295017&view=diff
==============================================================================
--- hadoop/common/trunk/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/protocolPB/DatanodeProtocolClientSideTranslatorPB.java (original)
+++ hadoop/common/trunk/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/protocolPB/DatanodeProtocolClientSideTranslatorPB.java Wed Feb 29 08:52:45 2012
@@ -136,19 +136,6 @@ public class DatanodeProtocolClientSideT
     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 {

Modified: hadoop/common/trunk/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/protocolPB/DatanodeProtocolPB.java
URL: http://svn.apache.org/viewvc/hadoop/common/trunk/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/protocolPB/DatanodeProtocolPB.java?rev=1295017&r1=1295016&r2=1295017&view=diff
==============================================================================
--- hadoop/common/trunk/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/protocolPB/DatanodeProtocolPB.java (original)
+++ hadoop/common/trunk/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/protocolPB/DatanodeProtocolPB.java Wed Feb 29 08:52:45 2012
@@ -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.Kerber
     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 {
 }

Modified: hadoop/common/trunk/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/protocolPB/DatanodeProtocolServerSideTranslatorPB.java
URL: http://svn.apache.org/viewvc/hadoop/common/trunk/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/protocolPB/DatanodeProtocolServerSideTranslatorPB.java?rev=1295017&r1=1295016&r2=1295017&view=diff
==============================================================================
--- hadoop/common/trunk/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/protocolPB/DatanodeProtocolServerSideTranslatorPB.java (original)
+++ hadoop/common/trunk/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/protocolPB/DatanodeProtocolServerSideTranslatorPB.java Wed Feb 29 08:52:45 2012
@@ -47,7 +47,6 @@ import org.apache.hadoop.hdfs.protocol.p
 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.pro
 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 DatanodeProtocolServerSideT
     }
     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));
-  }
-
 }

Modified: hadoop/common/trunk/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/protocolPB/GetUserMappingsProtocolClientSideTranslatorPB.java
URL: http://svn.apache.org/viewvc/hadoop/common/trunk/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/protocolPB/GetUserMappingsProtocolClientSideTranslatorPB.java?rev=1295017&r1=1295016&r2=1295017&view=diff
==============================================================================
--- hadoop/common/trunk/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/protocolPB/GetUserMappingsProtocolClientSideTranslatorPB.java (original)
+++ hadoop/common/trunk/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/protocolPB/GetUserMappingsProtocolClientSideTranslatorPB.java Wed Feb 29 08:52:45 2012
@@ -60,19 +60,6 @@ public class GetUserMappingsProtocolClie
   }
 
   @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);
   }

Modified: hadoop/common/trunk/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/protocolPB/GetUserMappingsProtocolPB.java
URL: http://svn.apache.org/viewvc/hadoop/common/trunk/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/protocolPB/GetUserMappingsProtocolPB.java?rev=1295017&r1=1295016&r2=1295017&view=diff
==============================================================================
--- hadoop/common/trunk/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/protocolPB/GetUserMappingsProtocolPB.java (original)
+++ hadoop/common/trunk/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/protocolPB/GetUserMappingsProtocolPB.java Wed Feb 29 08:52:45 2012
@@ -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.VersionedPr
 @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 {
 }

Modified: hadoop/common/trunk/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/protocolPB/GetUserMappingsProtocolServerSideTranslatorPB.java
URL: http://svn.apache.org/viewvc/hadoop/common/trunk/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/protocolPB/GetUserMappingsProtocolServerSideTranslatorPB.java?rev=1295017&r1=1295016&r2=1295017&view=diff
==============================================================================
--- hadoop/common/trunk/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/protocolPB/GetUserMappingsProtocolServerSideTranslatorPB.java (original)
+++ hadoop/common/trunk/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/protocolPB/GetUserMappingsProtocolServerSideTranslatorPB.java Wed Feb 29 08:52:45 2012
@@ -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;
@@ -41,42 +38,6 @@ public class GetUserMappingsProtocolServ
   }
 
   @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)
       throws ServiceException {

Modified: hadoop/common/trunk/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/protocolPB/InterDatanodeProtocolPB.java
URL: http://svn.apache.org/viewvc/hadoop/common/trunk/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/protocolPB/InterDatanodeProtocolPB.java?rev=1295017&r1=1295016&r2=1295017&view=diff
==============================================================================
--- hadoop/common/trunk/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/protocolPB/InterDatanodeProtocolPB.java (original)
+++ hadoop/common/trunk/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/protocolPB/InterDatanodeProtocolPB.java Wed Feb 29 08:52:45 2012
@@ -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.Kerber
     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 {
 }
\ No newline at end of file

Modified: hadoop/common/trunk/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/protocolPB/InterDatanodeProtocolServerSideTranslatorPB.java
URL: http://svn.apache.org/viewvc/hadoop/common/trunk/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/protocolPB/InterDatanodeProtocolServerSideTranslatorPB.java?rev=1295017&r1=1295016&r2=1295017&view=diff
==============================================================================
--- hadoop/common/trunk/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/protocolPB/InterDatanodeProtocolServerSideTranslatorPB.java (original)
+++ hadoop/common/trunk/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/protocolPB/InterDatanodeProtocolServerSideTranslatorPB.java Wed Feb 29 08:52:45 2012
@@ -25,14 +25,9 @@ import org.apache.hadoop.hdfs.protocol.p
 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 InterDatanodeProtocolServer
     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));
-  }
 }
\ No newline at end of file

Modified: hadoop/common/trunk/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/protocolPB/InterDatanodeProtocolTranslatorPB.java
URL: http://svn.apache.org/viewvc/hadoop/common/trunk/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/protocolPB/InterDatanodeProtocolTranslatorPB.java?rev=1295017&r1=1295016&r2=1295017&view=diff
==============================================================================
--- hadoop/common/trunk/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/protocolPB/InterDatanodeProtocolTranslatorPB.java (original)
+++ hadoop/common/trunk/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/protocolPB/InterDatanodeProtocolTranslatorPB.java Wed Feb 29 08:52:45 2012
@@ -31,14 +31,12 @@ import org.apache.hadoop.hdfs.protocol.p
 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;
@@ -77,19 +75,6 @@ public class InterDatanodeProtocolTransl
   }
 
   @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 {
     InitReplicaRecoveryRequestProto req = InitReplicaRecoveryRequestProto

Modified: hadoop/common/trunk/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/protocolPB/JournalProtocolPB.java
URL: http://svn.apache.org/viewvc/hadoop/common/trunk/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/protocolPB/JournalProtocolPB.java?rev=1295017&r1=1295016&r2=1295017&view=diff
==============================================================================
--- hadoop/common/trunk/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/protocolPB/JournalProtocolPB.java (original)
+++ hadoop/common/trunk/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/protocolPB/JournalProtocolPB.java Wed Feb 29 08:52:45 2012
@@ -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.VersionedPr
     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 {
 }

Modified: hadoop/common/trunk/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/protocolPB/JournalProtocolServerSideTranslatorPB.java
URL: http://svn.apache.org/viewvc/hadoop/common/trunk/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/protocolPB/JournalProtocolServerSideTranslatorPB.java?rev=1295017&r1=1295016&r2=1295017&view=diff
==============================================================================
--- hadoop/common/trunk/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/protocolPB/JournalProtocolServerSideTranslatorPB.java (original)
+++ hadoop/common/trunk/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/protocolPB/JournalProtocolServerSideTranslatorPB.java Wed Feb 29 08:52:45 2012
@@ -24,11 +24,7 @@ import org.apache.hadoop.hdfs.protocol.p
 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 JournalProtocolServerSideTr
     }
     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));
-  }
 }

Modified: hadoop/common/trunk/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/protocolPB/JournalProtocolTranslatorPB.java
URL: http://svn.apache.org/viewvc/hadoop/common/trunk/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/protocolPB/JournalProtocolTranslatorPB.java?rev=1295017&r1=1295016&r2=1295017&view=diff
==============================================================================
--- hadoop/common/trunk/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/protocolPB/JournalProtocolTranslatorPB.java (original)
+++ hadoop/common/trunk/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/protocolPB/JournalProtocolTranslatorPB.java Wed Feb 29 08:52:45 2012
@@ -66,19 +66,6 @@ public class JournalProtocolTranslatorPB
   }
 
   @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 {
     JournalRequestProto req = JournalRequestProto.newBuilder()

Modified: hadoop/common/trunk/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/protocolPB/NamenodeProtocolPB.java
URL: http://svn.apache.org/viewvc/hadoop/common/trunk/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/protocolPB/NamenodeProtocolPB.java?rev=1295017&r1=1295016&r2=1295017&view=diff
==============================================================================
--- hadoop/common/trunk/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/protocolPB/NamenodeProtocolPB.java (original)
+++ hadoop/common/trunk/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/protocolPB/NamenodeProtocolPB.java Wed Feb 29 08:52:45 2012
@@ -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.Kerber
     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 {
 }

Modified: hadoop/common/trunk/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/protocolPB/NamenodeProtocolServerSideTranslatorPB.java
URL: http://svn.apache.org/viewvc/hadoop/common/trunk/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/protocolPB/NamenodeProtocolServerSideTranslatorPB.java?rev=1295017&r1=1295016&r2=1295017&view=diff
==============================================================================
--- hadoop/common/trunk/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/protocolPB/NamenodeProtocolServerSideTranslatorPB.java (original)
+++ hadoop/common/trunk/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/protocolPB/NamenodeProtocolServerSideTranslatorPB.java Wed Feb 29 08:52:45 2012
@@ -40,7 +40,6 @@ import org.apache.hadoop.hdfs.protocol.p
 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.pro
 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 NamenodeProtocolServerSideT
     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,

Modified: hadoop/common/trunk/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/protocolPB/NamenodeProtocolTranslatorPB.java
URL: http://svn.apache.org/viewvc/hadoop/common/trunk/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/protocolPB/NamenodeProtocolTranslatorPB.java?rev=1295017&r1=1295016&r2=1295017&view=diff
==============================================================================
--- hadoop/common/trunk/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/protocolPB/NamenodeProtocolTranslatorPB.java (original)
+++ hadoop/common/trunk/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/protocolPB/NamenodeProtocolTranslatorPB.java Wed Feb 29 08:52:45 2012
@@ -103,19 +103,6 @@ public class NamenodeProtocolTranslatorP
   }
 
   @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 {
     GetBlocksRequestProto req = GetBlocksRequestProto.newBuilder()

Modified: hadoop/common/trunk/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/protocolPB/RefreshAuthorizationPolicyProtocolClientSideTranslatorPB.java
URL: http://svn.apache.org/viewvc/hadoop/common/trunk/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/protocolPB/RefreshAuthorizationPolicyProtocolClientSideTranslatorPB.java?rev=1295017&r1=1295016&r2=1295017&view=diff
==============================================================================
--- hadoop/common/trunk/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/protocolPB/RefreshAuthorizationPolicyProtocolClientSideTranslatorPB.java (original)
+++ hadoop/common/trunk/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/protocolPB/RefreshAuthorizationPolicyProtocolClientSideTranslatorPB.java Wed Feb 29 08:52:45 2012
@@ -59,19 +59,6 @@ public class RefreshAuthorizationPolicyP
   }
 
   @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);
   }

Modified: hadoop/common/trunk/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/protocolPB/RefreshAuthorizationPolicyProtocolPB.java
URL: http://svn.apache.org/viewvc/hadoop/common/trunk/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/protocolPB/RefreshAuthorizationPolicyProtocolPB.java?rev=1295017&r1=1295016&r2=1295017&view=diff
==============================================================================
--- hadoop/common/trunk/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/protocolPB/RefreshAuthorizationPolicyProtocolPB.java (original)
+++ hadoop/common/trunk/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/protocolPB/RefreshAuthorizationPolicyProtocolPB.java Wed Feb 29 08:52:45 2012
@@ -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.Kerber
 @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 {
 }

Modified: hadoop/common/trunk/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/protocolPB/RefreshAuthorizationPolicyProtocolServerSideTranslatorPB.java
URL: http://svn.apache.org/viewvc/hadoop/common/trunk/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/protocolPB/RefreshAuthorizationPolicyProtocolServerSideTranslatorPB.java?rev=1295017&r1=1295016&r2=1295017&view=diff
==============================================================================
--- hadoop/common/trunk/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/protocolPB/RefreshAuthorizationPolicyProtocolServerSideTranslatorPB.java (original)
+++ hadoop/common/trunk/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/protocolPB/RefreshAuthorizationPolicyProtocolServerSideTranslatorPB.java Wed Feb 29 08:52:45 2012
@@ -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;
@@ -41,42 +38,6 @@ public class RefreshAuthorizationPolicyP
   }
 
   @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)
       throws ServiceException {

Modified: hadoop/common/trunk/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/protocolPB/RefreshUserMappingsProtocolClientSideTranslatorPB.java
URL: http://svn.apache.org/viewvc/hadoop/common/trunk/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/protocolPB/RefreshUserMappingsProtocolClientSideTranslatorPB.java?rev=1295017&r1=1295016&r2=1295017&view=diff
==============================================================================
--- hadoop/common/trunk/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/protocolPB/RefreshUserMappingsProtocolClientSideTranslatorPB.java (original)
+++ hadoop/common/trunk/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/protocolPB/RefreshUserMappingsProtocolClientSideTranslatorPB.java Wed Feb 29 08:52:45 2012
@@ -60,19 +60,6 @@ public class RefreshUserMappingsProtocol
   }
 
   @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);
   }

Modified: hadoop/common/trunk/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/protocolPB/RefreshUserMappingsProtocolPB.java
URL: http://svn.apache.org/viewvc/hadoop/common/trunk/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/protocolPB/RefreshUserMappingsProtocolPB.java?rev=1295017&r1=1295016&r2=1295017&view=diff
==============================================================================
--- hadoop/common/trunk/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/protocolPB/RefreshUserMappingsProtocolPB.java (original)
+++ hadoop/common/trunk/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/protocolPB/RefreshUserMappingsProtocolPB.java Wed Feb 29 08:52:45 2012
@@ -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.Kerber
 @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 {
 }

Modified: hadoop/common/trunk/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/protocolPB/RefreshUserMappingsProtocolServerSideTranslatorPB.java
URL: http://svn.apache.org/viewvc/hadoop/common/trunk/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/protocolPB/RefreshUserMappingsProtocolServerSideTranslatorPB.java?rev=1295017&r1=1295016&r2=1295017&view=diff
==============================================================================
--- hadoop/common/trunk/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/protocolPB/RefreshUserMappingsProtocolServerSideTranslatorPB.java (original)
+++ hadoop/common/trunk/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/protocolPB/RefreshUserMappingsProtocolServerSideTranslatorPB.java Wed Feb 29 08:52:45 2012
@@ -24,9 +24,6 @@ import org.apache.hadoop.hdfs.protocol.p
 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 RefreshUserMappingsProtocol
     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));
-  }
 }

Modified: hadoop/common/trunk/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/DataNode.java
URL: http://svn.apache.org/viewvc/hadoop/common/trunk/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/DataNode.java?rev=1295017&r1=1295016&r2=1295017&view=diff
==============================================================================
--- hadoop/common/trunk/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/DataNode.java (original)
+++ hadoop/common/trunk/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/DataNode.java Wed Feb 29 08:52:45 2012
@@ -702,9 +702,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;
@@ -1861,25 +1864,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;

Modified: hadoop/common/trunk/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/BackupNode.java
URL: http://svn.apache.org/viewvc/hadoop/common/trunk/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/BackupNode.java?rev=1295017&r1=1295016&r2=1295017&view=diff
==============================================================================
--- hadoop/common/trunk/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/BackupNode.java (original)
+++ hadoop/common/trunk/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/BackupNode.java Wed Feb 29 08:52:45 2012
@@ -42,7 +42,6 @@ import org.apache.hadoop.hdfs.server.pro
 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;
 
@@ -222,15 +221,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.