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 sz...@apache.org on 2012/02/27 00:32:14 UTC

svn commit: r1293964 [3/11] - in /hadoop/common/branches/branch-0.23-PB-merge/hadoop-hdfs-project/hadoop-hdfs: ./ src/main/java/org/apache/hadoop/hdfs/ src/main/java/org/apache/hadoop/hdfs/protocol/ src/main/java/org/apache/hadoop/hdfs/protocolPB/ src/...

Added: hadoop/common/branches/branch-0.23-PB-merge/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/protocolPB/ClientNamenodeProtocolTranslatorPB.java
URL: http://svn.apache.org/viewvc/hadoop/common/branches/branch-0.23-PB-merge/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/protocolPB/ClientNamenodeProtocolTranslatorPB.java?rev=1293964&view=auto
==============================================================================
--- hadoop/common/branches/branch-0.23-PB-merge/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/protocolPB/ClientNamenodeProtocolTranslatorPB.java (added)
+++ hadoop/common/branches/branch-0.23-PB-merge/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/protocolPB/ClientNamenodeProtocolTranslatorPB.java Sun Feb 26 23:32:06 2012
@@ -0,0 +1,885 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.hadoop.hdfs.protocolPB;
+
+import java.io.Closeable;
+import java.io.FileNotFoundException;
+import java.io.IOException;
+import java.net.InetSocketAddress;
+import java.util.Arrays;
+import java.util.HashMap;
+import java.util.Map;
+import java.util.concurrent.TimeUnit;
+
+import org.apache.hadoop.classification.InterfaceAudience;
+import org.apache.hadoop.classification.InterfaceStability;
+import org.apache.hadoop.conf.Configuration;
+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.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.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.proto.ClientNamenodeProtocolProtos.AbandonBlockRequestProto;
+import org.apache.hadoop.hdfs.protocol.proto.ClientNamenodeProtocolProtos.AddBlockRequestProto;
+import org.apache.hadoop.hdfs.protocol.proto.ClientNamenodeProtocolProtos.AppendRequestProto;
+import org.apache.hadoop.hdfs.protocol.proto.ClientNamenodeProtocolProtos.AppendResponseProto;
+import org.apache.hadoop.hdfs.protocol.proto.ClientNamenodeProtocolProtos.CancelDelegationTokenRequestProto;
+import org.apache.hadoop.hdfs.protocol.proto.ClientNamenodeProtocolProtos.CompleteRequestProto;
+import org.apache.hadoop.hdfs.protocol.proto.ClientNamenodeProtocolProtos.ConcatRequestProto;
+import org.apache.hadoop.hdfs.protocol.proto.ClientNamenodeProtocolProtos.CreateRequestProto;
+import org.apache.hadoop.hdfs.protocol.proto.ClientNamenodeProtocolProtos.CreateSymlinkRequestProto;
+import org.apache.hadoop.hdfs.protocol.proto.ClientNamenodeProtocolProtos.DeleteRequestProto;
+import org.apache.hadoop.hdfs.protocol.proto.ClientNamenodeProtocolProtos.DistributedUpgradeProgressRequestProto;
+import org.apache.hadoop.hdfs.protocol.proto.ClientNamenodeProtocolProtos.DistributedUpgradeProgressResponseProto;
+import org.apache.hadoop.hdfs.protocol.proto.ClientNamenodeProtocolProtos.FinalizeUpgradeRequestProto;
+import org.apache.hadoop.hdfs.protocol.proto.ClientNamenodeProtocolProtos.FsyncRequestProto;
+import org.apache.hadoop.hdfs.protocol.proto.ClientNamenodeProtocolProtos.GetAdditionalDatanodeRequestProto;
+import org.apache.hadoop.hdfs.protocol.proto.ClientNamenodeProtocolProtos.GetBlockLocationsRequestProto;
+import org.apache.hadoop.hdfs.protocol.proto.ClientNamenodeProtocolProtos.GetBlockLocationsResponseProto;
+import org.apache.hadoop.hdfs.protocol.proto.ClientNamenodeProtocolProtos.GetContentSummaryRequestProto;
+import org.apache.hadoop.hdfs.protocol.proto.ClientNamenodeProtocolProtos.GetDatanodeReportRequestProto;
+import org.apache.hadoop.hdfs.protocol.proto.ClientNamenodeProtocolProtos.GetDelegationTokenRequestProto;
+import org.apache.hadoop.hdfs.protocol.proto.ClientNamenodeProtocolProtos.GetFileInfoRequestProto;
+import org.apache.hadoop.hdfs.protocol.proto.ClientNamenodeProtocolProtos.GetFileInfoResponseProto;
+import org.apache.hadoop.hdfs.protocol.proto.ClientNamenodeProtocolProtos.GetFileLinkInfoRequestProto;
+import org.apache.hadoop.hdfs.protocol.proto.ClientNamenodeProtocolProtos.GetFileLinkInfoResponseProto;
+import org.apache.hadoop.hdfs.protocol.proto.ClientNamenodeProtocolProtos.GetFsStatusRequestProto;
+import org.apache.hadoop.hdfs.protocol.proto.ClientNamenodeProtocolProtos.GetLinkTargetRequestProto;
+import org.apache.hadoop.hdfs.protocol.proto.ClientNamenodeProtocolProtos.GetListingRequestProto;
+import org.apache.hadoop.hdfs.protocol.proto.ClientNamenodeProtocolProtos.GetListingResponseProto;
+import org.apache.hadoop.hdfs.protocol.proto.ClientNamenodeProtocolProtos.GetPreferredBlockSizeRequestProto;
+import org.apache.hadoop.hdfs.protocol.proto.ClientNamenodeProtocolProtos.GetServerDefaultsRequestProto;
+import org.apache.hadoop.hdfs.protocol.proto.ClientNamenodeProtocolProtos.ListCorruptFileBlocksRequestProto;
+import org.apache.hadoop.hdfs.protocol.proto.ClientNamenodeProtocolProtos.MetaSaveRequestProto;
+import org.apache.hadoop.hdfs.protocol.proto.ClientNamenodeProtocolProtos.MkdirsRequestProto;
+import org.apache.hadoop.hdfs.protocol.proto.ClientNamenodeProtocolProtos.RecoverLeaseRequestProto;
+import org.apache.hadoop.hdfs.protocol.proto.ClientNamenodeProtocolProtos.RefreshNodesRequestProto;
+import org.apache.hadoop.hdfs.protocol.proto.ClientNamenodeProtocolProtos.Rename2RequestProto;
+import org.apache.hadoop.hdfs.protocol.proto.ClientNamenodeProtocolProtos.RenameRequestProto;
+import org.apache.hadoop.hdfs.protocol.proto.ClientNamenodeProtocolProtos.RenewDelegationTokenRequestProto;
+import org.apache.hadoop.hdfs.protocol.proto.ClientNamenodeProtocolProtos.RenewLeaseRequestProto;
+import org.apache.hadoop.hdfs.protocol.proto.ClientNamenodeProtocolProtos.ReportBadBlocksRequestProto;
+import org.apache.hadoop.hdfs.protocol.proto.ClientNamenodeProtocolProtos.RestoreFailedStorageRequestProto;
+import org.apache.hadoop.hdfs.protocol.proto.ClientNamenodeProtocolProtos.SaveNamespaceRequestProto;
+import org.apache.hadoop.hdfs.protocol.proto.ClientNamenodeProtocolProtos.SetBalancerBandwidthRequestProto;
+import org.apache.hadoop.hdfs.protocol.proto.ClientNamenodeProtocolProtos.SetOwnerRequestProto;
+import org.apache.hadoop.hdfs.protocol.proto.ClientNamenodeProtocolProtos.SetPermissionRequestProto;
+import org.apache.hadoop.hdfs.protocol.proto.ClientNamenodeProtocolProtos.SetQuotaRequestProto;
+import org.apache.hadoop.hdfs.protocol.proto.ClientNamenodeProtocolProtos.SetReplicationRequestProto;
+import org.apache.hadoop.hdfs.protocol.proto.ClientNamenodeProtocolProtos.SetSafeModeRequestProto;
+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 com.google.protobuf.ByteString;
+import com.google.protobuf.ServiceException;
+
+/**
+ * This class forwards NN's ClientProtocol calls as RPC calls to the NN server
+ * while translating from the parameter types used in ClientProtocol to those
+ * used in protocolR23Compatile.*.
+ */
+@InterfaceAudience.Private
+@InterfaceStability.Stable
+public class ClientNamenodeProtocolTranslatorPB implements
+    ProtocolMetaInterface, ClientProtocol, Closeable {
+  final private ClientNamenodeProtocolPB rpcProxy;
+
+  private static ClientNamenodeProtocolPB createNamenode(
+      InetSocketAddress nameNodeAddr, Configuration conf,
+      UserGroupInformation ugi) throws IOException {
+    RPC.setProtocolEngine(conf, ClientNamenodeProtocolPB.class,
+        ProtobufRpcEngine.class);
+    return RPC.getProxy(ClientNamenodeProtocolPB.class,
+        RPC.getProtocolVersion(ClientNamenodeProtocolPB.class), nameNodeAddr, ugi, conf,
+        NetUtils.getSocketFactory(conf, ClientNamenodeProtocolPB.class));
+  }
+
+  /** Create a {@link NameNode} proxy */
+  static ClientNamenodeProtocolPB createNamenodeWithRetry(
+      ClientNamenodeProtocolPB rpcNamenode) {
+    RetryPolicy createPolicy = RetryPolicies
+        .retryUpToMaximumCountWithFixedSleep(5,
+            HdfsConstants.LEASE_SOFTLIMIT_PERIOD, TimeUnit.MILLISECONDS);
+
+    Map<Class<? extends Exception>, RetryPolicy> remoteExceptionToPolicyMap 
+        = new HashMap<Class<? extends Exception>, RetryPolicy>();
+    remoteExceptionToPolicyMap.put(AlreadyBeingCreatedException.class,
+        createPolicy);
+
+    Map<Class<? extends Exception>, RetryPolicy> exceptionToPolicyMap = 
+        new HashMap<Class<? extends Exception>, RetryPolicy>();
+    exceptionToPolicyMap.put(RemoteException.class, RetryPolicies
+        .retryByRemoteException(RetryPolicies.TRY_ONCE_THEN_FAIL,
+            remoteExceptionToPolicyMap));
+    RetryPolicy methodPolicy = RetryPolicies.retryByException(
+        RetryPolicies.TRY_ONCE_THEN_FAIL, exceptionToPolicyMap);
+    Map<String, RetryPolicy> methodNameToPolicyMap = new HashMap<String, RetryPolicy>();
+
+    methodNameToPolicyMap.put("create", methodPolicy);
+
+    return (ClientNamenodeProtocolPB) RetryProxy.create(
+        ClientNamenodeProtocolPB.class, rpcNamenode, methodNameToPolicyMap);
+  }
+
+  public ClientNamenodeProtocolTranslatorPB(InetSocketAddress nameNodeAddr,
+      Configuration conf, UserGroupInformation ugi) throws IOException {
+    
+    rpcProxy = createNamenodeWithRetry(createNamenode(nameNodeAddr, conf, ugi));
+  }
+
+  public void close() {
+    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,
+      UnresolvedLinkException, IOException {
+    GetBlockLocationsRequestProto req = GetBlockLocationsRequestProto
+        .newBuilder()
+        .setSrc(src)
+        .setOffset(offset)
+        .setLength(length)
+        .build();
+    try {
+      GetBlockLocationsResponseProto resp = rpcProxy.getBlockLocations(null,
+          req);
+      return resp.hasLocations() ? 
+        PBHelper.convert(resp.getLocations()) : null;
+    } catch (ServiceException e) {
+      throw ProtobufHelper.getRemoteException(e);
+    }
+  }
+
+  @Override
+  public FsServerDefaults getServerDefaults() throws IOException {
+    GetServerDefaultsRequestProto req = GetServerDefaultsRequestProto.newBuilder().build();
+    try {
+      return PBHelper
+          .convert(rpcProxy.getServerDefaults(null, req).getServerDefaults());
+    } catch (ServiceException e) {
+      throw ProtobufHelper.getRemoteException(e);
+    }
+  }
+
+  @Override
+  public void create(String src, FsPermission masked, String clientName,
+      EnumSetWritable<CreateFlag> flag, boolean createParent,
+      short replication, long blockSize) throws AccessControlException,
+      AlreadyBeingCreatedException, DSQuotaExceededException,
+      FileAlreadyExistsException, FileNotFoundException,
+      NSQuotaExceededException, ParentNotDirectoryException, SafeModeException,
+      UnresolvedLinkException, IOException {
+    CreateRequestProto req = CreateRequestProto.newBuilder()
+        .setSrc(src)
+        .setMasked(PBHelper.convert(masked))
+        .setClientName(clientName)
+        .setCreateFlag(PBHelper.convertCreateFlag(flag))
+        .setCreateParent(createParent)
+        .setReplication(replication)
+        .setBlockSize(blockSize)
+        .build();
+    try {
+      rpcProxy.create(null, req);
+    } catch (ServiceException e) {
+      throw ProtobufHelper.getRemoteException(e);
+    }
+
+  }
+
+  @Override
+  public LocatedBlock append(String src, String clientName)
+      throws AccessControlException, DSQuotaExceededException,
+      FileNotFoundException, SafeModeException, UnresolvedLinkException,
+      IOException {
+    AppendRequestProto req = AppendRequestProto.newBuilder()
+        .setSrc(src)
+        .setClientName(clientName)
+        .build();
+    try {
+      AppendResponseProto res = rpcProxy.append(null, req);
+      return res.hasBlock() ? PBHelper.convert(res.getBlock()) : null;
+    } catch (ServiceException e) {
+      throw ProtobufHelper.getRemoteException(e);
+    }
+  }
+
+  @Override
+  public boolean setReplication(String src, short replication)
+      throws AccessControlException, DSQuotaExceededException,
+      FileNotFoundException, SafeModeException, UnresolvedLinkException,
+      IOException {
+    SetReplicationRequestProto req = SetReplicationRequestProto.newBuilder()
+        .setSrc(src)
+        .setReplication(replication)
+        .build();
+    try {
+      return rpcProxy.setReplication(null, req).getResult();
+    } catch (ServiceException e) {
+      throw ProtobufHelper.getRemoteException(e);
+    }
+  }
+
+  @Override
+  public void setPermission(String src, FsPermission permission)
+      throws AccessControlException, FileNotFoundException, SafeModeException,
+      UnresolvedLinkException, IOException {
+    SetPermissionRequestProto req = SetPermissionRequestProto.newBuilder()
+        .setSrc(src)
+        .setPermission(PBHelper.convert(permission))
+        .build();
+    try {
+      rpcProxy.setPermission(null, req);
+    } catch (ServiceException e) {
+      throw ProtobufHelper.getRemoteException(e);
+    }
+  }
+
+  @Override
+  public void setOwner(String src, String username, String groupname)
+      throws AccessControlException, FileNotFoundException, SafeModeException,
+      UnresolvedLinkException, IOException {
+    SetOwnerRequestProto.Builder req = SetOwnerRequestProto.newBuilder()
+        .setSrc(src);
+    if (username != null)
+        req.setUsername(username);
+    if (groupname != null)
+        req.setGroupname(groupname);
+    try {
+      rpcProxy.setOwner(null, req.build());
+    } catch (ServiceException e) {
+      throw ProtobufHelper.getRemoteException(e);
+    }
+  }
+
+  @Override
+  public void abandonBlock(ExtendedBlock b, String src, String holder)
+      throws AccessControlException, FileNotFoundException,
+      UnresolvedLinkException, IOException {
+    AbandonBlockRequestProto req = AbandonBlockRequestProto.newBuilder()
+        .setB(PBHelper.convert(b)).setSrc(src).setHolder(holder).build();
+    try {
+      rpcProxy.abandonBlock(null, req);
+    } catch (ServiceException e) {
+      throw ProtobufHelper.getRemoteException(e);
+    }
+  }
+
+  @Override
+  public LocatedBlock addBlock(String src, String clientName,
+      ExtendedBlock previous, DatanodeInfo[] excludeNodes)
+      throws AccessControlException, FileNotFoundException,
+      NotReplicatedYetException, SafeModeException, UnresolvedLinkException,
+      IOException {
+    AddBlockRequestProto.Builder req = AddBlockRequestProto.newBuilder().setSrc(src)
+        .setClientName(clientName);
+    if (previous != null) 
+      req.setPrevious(PBHelper.convert(previous)); 
+    if (excludeNodes != null) 
+      req.addAllExcludeNodes(Arrays.asList(PBHelper.convert(excludeNodes)));
+    try {
+      return PBHelper.convert(rpcProxy.addBlock(null, req.build()).getBlock());
+    } catch (ServiceException e) {
+      throw ProtobufHelper.getRemoteException(e);
+    }
+  }
+
+  @Override
+  public LocatedBlock getAdditionalDatanode(String src, ExtendedBlock blk,
+      DatanodeInfo[] existings, DatanodeInfo[] excludes,
+      int numAdditionalNodes, String clientName) throws AccessControlException,
+      FileNotFoundException, SafeModeException, UnresolvedLinkException,
+      IOException {
+    GetAdditionalDatanodeRequestProto req = GetAdditionalDatanodeRequestProto
+        .newBuilder()
+        .setSrc(src)
+        .setBlk(PBHelper.convert(blk))
+        .addAllExistings(Arrays.asList(PBHelper.convert(existings)))
+        .addAllExcludes(Arrays.asList(PBHelper.convert(excludes)))
+        .setNumAdditionalNodes(numAdditionalNodes)
+        .setClientName(clientName)
+        .build();
+    try {
+      return PBHelper.convert(rpcProxy.getAdditionalDatanode(null, req)
+          .getBlock());
+    } catch (ServiceException e) {
+      throw ProtobufHelper.getRemoteException(e);
+    }
+  }
+
+  @Override
+  public boolean complete(String src, String clientName, ExtendedBlock last)
+      throws AccessControlException, FileNotFoundException, SafeModeException,
+      UnresolvedLinkException, IOException {
+    CompleteRequestProto.Builder req = CompleteRequestProto.newBuilder()
+        .setSrc(src)
+        .setClientName(clientName);   
+    if (last != null)
+      req.setLast(PBHelper.convert(last));
+    try {
+      return rpcProxy.complete(null, req.build()).getResult();
+    } catch (ServiceException e) {
+      throw ProtobufHelper.getRemoteException(e);
+    }
+  }
+
+  @Override
+  public void reportBadBlocks(LocatedBlock[] blocks) throws IOException {
+    ReportBadBlocksRequestProto req = ReportBadBlocksRequestProto.newBuilder()
+        .addAllBlocks(Arrays.asList(PBHelper.convertLocatedBlock(blocks)))
+        .build();
+    try {
+      rpcProxy.reportBadBlocks(null, req);
+    } catch (ServiceException e) {
+      throw ProtobufHelper.getRemoteException(e);
+    }
+  }
+
+  @Override
+  public boolean rename(String src, String dst) throws UnresolvedLinkException,
+      IOException {
+    RenameRequestProto req = RenameRequestProto.newBuilder()
+        .setSrc(src)
+        .setDst(dst).build();
+    try {
+      return rpcProxy.rename(null, req).getResult();
+    } catch (ServiceException e) {
+      throw ProtobufHelper.getRemoteException(e);
+    }
+  }
+  
+
+  @Override
+  public void rename2(String src, String dst, Rename... options)
+      throws AccessControlException, DSQuotaExceededException,
+      FileAlreadyExistsException, FileNotFoundException,
+      NSQuotaExceededException, ParentNotDirectoryException, SafeModeException,
+      UnresolvedLinkException, IOException {
+    boolean overwrite = false;
+    if (options != null) {
+      for (Rename option : options) {
+        if (option == Rename.OVERWRITE) {
+          overwrite = true;
+        }
+      }
+    }
+    Rename2RequestProto req = Rename2RequestProto.newBuilder().
+        setSrc(src).
+        setDst(dst).setOverwriteDest(overwrite).
+        build();
+    try {
+      rpcProxy.rename2(null, req);
+    } catch (ServiceException e) {
+      throw ProtobufHelper.getRemoteException(e);
+    }
+
+  }
+
+  @Override
+  public void concat(String trg, String[] srcs) throws IOException,
+      UnresolvedLinkException {
+    ConcatRequestProto req = ConcatRequestProto.newBuilder().
+        setTrg(trg).
+        addAllSrcs(Arrays.asList(srcs)).build();
+    try {
+      rpcProxy.concat(null, req);
+    } catch (ServiceException e) {
+      throw ProtobufHelper.getRemoteException(e);
+    }
+  }
+
+
+  @Override
+  public boolean delete(String src, boolean recursive)
+      throws AccessControlException, FileNotFoundException, SafeModeException,
+      UnresolvedLinkException, IOException {
+    DeleteRequestProto req = DeleteRequestProto.newBuilder().setSrc(src).setRecursive(recursive).build();
+    try {
+      return rpcProxy.delete(null, req).getResult();
+    } catch (ServiceException e) {
+      throw ProtobufHelper.getRemoteException(e);
+    }
+  }
+
+  @Override
+  public boolean mkdirs(String src, FsPermission masked, boolean createParent)
+      throws AccessControlException, FileAlreadyExistsException,
+      FileNotFoundException, NSQuotaExceededException,
+      ParentNotDirectoryException, SafeModeException, UnresolvedLinkException,
+      IOException {
+    MkdirsRequestProto req = MkdirsRequestProto.newBuilder()
+        .setSrc(src)
+        .setMasked(PBHelper.convert(masked))
+        .setCreateParent(createParent).build();
+
+    try {
+      return rpcProxy.mkdirs(null, req).getResult();
+    } catch (ServiceException e) {
+      throw ProtobufHelper.getRemoteException(e);
+    }
+  }
+
+  @Override
+  public DirectoryListing getListing(String src, byte[] startAfter,
+      boolean needLocation) throws AccessControlException,
+      FileNotFoundException, UnresolvedLinkException, IOException {
+    GetListingRequestProto req = GetListingRequestProto.newBuilder()
+        .setSrc(src)
+        .setStartAfter(ByteString.copyFrom(startAfter))
+        .setNeedLocation(needLocation).build();
+    try {
+      GetListingResponseProto result = rpcProxy.getListing(null, req);
+      
+      if (result.hasDirList()) {
+        return PBHelper.convert(result.getDirList());
+      }
+      return null;
+    } catch (ServiceException e) {
+      throw ProtobufHelper.getRemoteException(e);
+    }
+  }
+
+  @Override
+  public void renewLease(String clientName) throws AccessControlException,
+      IOException {
+    RenewLeaseRequestProto req = RenewLeaseRequestProto.newBuilder()
+        .setClientName(clientName).build();
+    try {
+      rpcProxy.renewLease(null, req);
+    } catch (ServiceException e) {
+      throw ProtobufHelper.getRemoteException(e);
+    }
+  }
+
+  @Override
+  public boolean recoverLease(String src, String clientName)
+      throws IOException {
+    RecoverLeaseRequestProto req = RecoverLeaseRequestProto.newBuilder()
+        .setSrc(src)
+        .setClientName(clientName).build();
+    try {
+      return rpcProxy.recoverLease(null, req).getResult();
+    } catch (ServiceException e) {
+      throw ProtobufHelper.getRemoteException(e);
+    }  
+  }
+
+  @Override
+  public long[] getStats() throws IOException {
+    GetFsStatusRequestProto req = GetFsStatusRequestProto.newBuilder().build();
+    try {
+      return PBHelper.convert(rpcProxy.getFsStats(null, req));
+    } catch (ServiceException e) {
+      throw ProtobufHelper.getRemoteException(e);
+    }
+  }
+
+  @Override
+  public DatanodeInfo[] getDatanodeReport(DatanodeReportType type)
+      throws IOException {
+    GetDatanodeReportRequestProto req = GetDatanodeReportRequestProto
+        .newBuilder()
+        .setType(PBHelper.convert(type)).build();
+    try {
+      return PBHelper.convert(
+          rpcProxy.getDatanodeReport(null, req).getDiList());
+    } catch (ServiceException e) {
+      throw ProtobufHelper.getRemoteException(e);
+    }
+  }
+
+  @Override
+  public long getPreferredBlockSize(String filename) throws IOException,
+      UnresolvedLinkException {
+    GetPreferredBlockSizeRequestProto req = GetPreferredBlockSizeRequestProto
+        .newBuilder()
+        .setFilename(filename)
+        .build();
+    try {
+      return rpcProxy.getPreferredBlockSize(null, req).getBsize();
+    } catch (ServiceException e) {
+      throw ProtobufHelper.getRemoteException(e);
+    }
+  }
+
+  @Override
+  public boolean setSafeMode(SafeModeAction action) throws IOException {
+    SetSafeModeRequestProto req = SetSafeModeRequestProto.newBuilder().
+        setAction(PBHelper.convert(action)).build();
+    try {
+      return rpcProxy.setSafeMode(null, req).getResult();
+    } catch (ServiceException e) {
+      throw ProtobufHelper.getRemoteException(e);
+    }
+  }
+
+  @Override
+  public void saveNamespace() throws AccessControlException, IOException {
+    SaveNamespaceRequestProto req = SaveNamespaceRequestProto.newBuilder()
+        .build();
+    try {
+      rpcProxy.saveNamespace(null, req);
+    } catch (ServiceException e) {
+      throw ProtobufHelper.getRemoteException(e);
+    }
+  }
+
+  @Override
+  public boolean restoreFailedStorage(String arg) 
+      throws AccessControlException, IOException{
+    RestoreFailedStorageRequestProto req = RestoreFailedStorageRequestProto
+        .newBuilder()
+        .setArg(arg).build();
+    try {
+      return rpcProxy.restoreFailedStorage(null, req).getResult();
+    } catch (ServiceException e) {
+      throw ProtobufHelper.getRemoteException(e);
+    }
+  }
+
+  @Override
+  public void refreshNodes() throws IOException {
+    RefreshNodesRequestProto req = RefreshNodesRequestProto.newBuilder().build();
+    try {
+      rpcProxy.refreshNodes(null, req);
+    } catch (ServiceException e) {
+      throw ProtobufHelper.getRemoteException(e);
+    }
+  }
+
+  @Override
+  public void finalizeUpgrade() throws IOException {
+    FinalizeUpgradeRequestProto req = FinalizeUpgradeRequestProto.newBuilder().build();
+    try {
+      rpcProxy.finalizeUpgrade(null, req);
+    } catch (ServiceException e) {
+      throw ProtobufHelper.getRemoteException(e);
+    }
+  }
+
+  @Override
+  public UpgradeStatusReport distributedUpgradeProgress(UpgradeAction action)
+      throws IOException {
+    DistributedUpgradeProgressRequestProto req = 
+        DistributedUpgradeProgressRequestProto.newBuilder().
+        setAction(PBHelper.convert(action)).build();
+    try {
+      DistributedUpgradeProgressResponseProto res = rpcProxy
+          .distributedUpgradeProgress(null, req);
+      return res.hasReport() ? PBHelper.convert(res.getReport()) : null;
+    } catch (ServiceException e) {
+      throw ProtobufHelper.getRemoteException(e);
+    }
+  }
+
+  @Override
+  public CorruptFileBlocks listCorruptFileBlocks(String path, String cookie)
+      throws IOException {
+    ListCorruptFileBlocksRequestProto.Builder req = 
+        ListCorruptFileBlocksRequestProto.newBuilder().setPath(path);   
+    if (cookie != null) 
+      req.setCookie(cookie);
+    try {
+      return PBHelper.convert(
+          rpcProxy.listCorruptFileBlocks(null, req.build()).getCorrupt());
+    } catch (ServiceException e) {
+      throw ProtobufHelper.getRemoteException(e);
+    }
+  }
+
+  @Override
+  public void metaSave(String filename) throws IOException {
+    MetaSaveRequestProto req = MetaSaveRequestProto.newBuilder()
+        .setFilename(filename).build();
+    try {
+      rpcProxy.metaSave(null, req);
+    } catch (ServiceException e) {
+      throw ProtobufHelper.getRemoteException(e);
+    }
+
+  }
+
+  @Override
+  public HdfsFileStatus getFileInfo(String src) throws AccessControlException,
+      FileNotFoundException, UnresolvedLinkException, IOException {
+    GetFileInfoRequestProto req = GetFileInfoRequestProto.newBuilder()
+        .setSrc(src).build();
+    try {
+      GetFileInfoResponseProto res = rpcProxy.getFileInfo(null, req);
+      return res.hasFs() ? PBHelper.convert(res.getFs()) : null;
+    } catch (ServiceException e) {
+      throw ProtobufHelper.getRemoteException(e);
+    }
+  }
+
+  @Override
+  public HdfsFileStatus getFileLinkInfo(String src)
+      throws AccessControlException, UnresolvedLinkException, IOException {
+    GetFileLinkInfoRequestProto req = GetFileLinkInfoRequestProto.newBuilder()
+        .setSrc(src).build();
+    try {
+      GetFileLinkInfoResponseProto result = rpcProxy.getFileLinkInfo(null, req);
+      return result.hasFs() ?  
+          PBHelper.convert(rpcProxy.getFileLinkInfo(null, req).getFs()) : null;
+    } catch (ServiceException e) {
+      throw ProtobufHelper.getRemoteException(e);
+    }
+  }
+
+  @Override
+  public ContentSummary getContentSummary(String path)
+      throws AccessControlException, FileNotFoundException,
+      UnresolvedLinkException, IOException {
+    GetContentSummaryRequestProto req = GetContentSummaryRequestProto
+        .newBuilder()
+        .setPath(path)
+        .build();
+    try {
+      return PBHelper.convert(rpcProxy.getContentSummary(null, req)
+          .getSummary());
+    } catch (ServiceException e) {
+      throw ProtobufHelper.getRemoteException(e);
+    }
+  }
+
+  @Override
+  public void setQuota(String path, long namespaceQuota, long diskspaceQuota)
+      throws AccessControlException, FileNotFoundException,
+      UnresolvedLinkException, IOException {
+    SetQuotaRequestProto req = SetQuotaRequestProto.newBuilder()
+        .setPath(path)
+        .setNamespaceQuota(namespaceQuota)
+        .setDiskspaceQuota(diskspaceQuota)
+        .build();
+    try {
+      rpcProxy.setQuota(null, req);
+    } catch (ServiceException e) {
+      throw ProtobufHelper.getRemoteException(e);
+    }
+  }
+
+  @Override
+  public void fsync(String src, String client) throws AccessControlException,
+      FileNotFoundException, UnresolvedLinkException, IOException {
+    FsyncRequestProto req = FsyncRequestProto.newBuilder()
+        .setSrc(src)
+        .setClient(client)
+        .build();
+    try {
+      rpcProxy.fsync(null, req);
+    } catch (ServiceException e) {
+      throw ProtobufHelper.getRemoteException(e);
+    }
+  }
+
+  @Override
+  public void setTimes(String src, long mtime, long atime)
+      throws AccessControlException, FileNotFoundException,
+      UnresolvedLinkException, IOException {
+    SetTimesRequestProto req = SetTimesRequestProto.newBuilder()
+        .setSrc(src)
+        .setMtime(mtime)
+        .setAtime(atime)
+        .build();
+    try {
+      rpcProxy.setTimes(null, req);
+    } catch (ServiceException e) {
+      throw ProtobufHelper.getRemoteException(e);
+    }
+  }
+
+  @Override
+  public void createSymlink(String target, String link, FsPermission dirPerm,
+      boolean createParent) throws AccessControlException,
+      FileAlreadyExistsException, FileNotFoundException,
+      ParentNotDirectoryException, SafeModeException, UnresolvedLinkException,
+      IOException {
+    CreateSymlinkRequestProto req = CreateSymlinkRequestProto.newBuilder()
+        .setTarget(target)
+        .setLink(link)
+        .setDirPerm(PBHelper.convert(dirPerm))
+        .setCreateParent(createParent)
+        .build();
+    try {
+      rpcProxy.createSymlink(null, req);
+    } catch (ServiceException e) {
+      throw ProtobufHelper.getRemoteException(e);
+    }
+  }
+
+  @Override
+  public String getLinkTarget(String path) throws AccessControlException,
+      FileNotFoundException, IOException {
+    GetLinkTargetRequestProto req = GetLinkTargetRequestProto.newBuilder()
+        .setPath(path).build();
+    try {
+      return rpcProxy.getLinkTarget(null, req).getTargetPath();
+    } catch (ServiceException e) {
+      throw ProtobufHelper.getRemoteException(e);
+    }
+  }
+
+  @Override
+  public LocatedBlock updateBlockForPipeline(ExtendedBlock block,
+      String clientName) throws IOException {
+    UpdateBlockForPipelineRequestProto req = UpdateBlockForPipelineRequestProto
+        .newBuilder()
+        .setBlock(PBHelper.convert(block))
+        .setClientName(clientName)
+        .build();
+    try {
+      return PBHelper.convert(
+          rpcProxy.updateBlockForPipeline(null, req).getBlock());
+    } catch (ServiceException e) {
+      throw ProtobufHelper.getRemoteException(e);
+    }
+  }
+
+  @Override
+  public void updatePipeline(String clientName, ExtendedBlock oldBlock,
+      ExtendedBlock newBlock, DatanodeID[] newNodes) throws IOException {
+    UpdatePipelineRequestProto req = UpdatePipelineRequestProto.newBuilder()
+        .setClientName(clientName)
+        .setOldBlock(PBHelper.convert(oldBlock))
+        .setNewBlock(PBHelper.convert(newBlock))
+        .addAllNewNodes(Arrays.asList(PBHelper.convert(newNodes)))
+        .build();
+    try {
+      rpcProxy.updatePipeline(null, req);
+    } catch (ServiceException e) {
+      throw ProtobufHelper.getRemoteException(e);
+    }
+  }
+
+  @Override
+  public Token<DelegationTokenIdentifier> getDelegationToken(Text renewer)
+      throws IOException {
+    GetDelegationTokenRequestProto req = GetDelegationTokenRequestProto
+        .newBuilder()
+        .setRenewer(renewer.toString())
+        .build();
+    try {
+      return PBHelper.convertDelegationToken(rpcProxy.getDelegationToken(null, req).getToken());
+    } catch (ServiceException e) {
+      throw ProtobufHelper.getRemoteException(e);
+    }
+  }
+
+  @Override
+  public long renewDelegationToken(Token<DelegationTokenIdentifier> token)
+      throws IOException {
+    RenewDelegationTokenRequestProto req = RenewDelegationTokenRequestProto.newBuilder().
+        setToken(PBHelper.convert(token)).
+        build();
+    try {
+      return rpcProxy.renewDelegationToken(null, req).getNewExireTime();
+    } catch (ServiceException e) {
+      throw ProtobufHelper.getRemoteException(e);
+    }
+  }
+
+  @Override
+  public void cancelDelegationToken(Token<DelegationTokenIdentifier> token)
+      throws IOException {
+    CancelDelegationTokenRequestProto req = CancelDelegationTokenRequestProto
+        .newBuilder()
+        .setToken(PBHelper.convert(token))
+        .build();
+    try {
+      rpcProxy.cancelDelegationToken(null, req);
+    } catch (ServiceException e) {
+      throw ProtobufHelper.getRemoteException(e);
+    }
+  }
+
+  @Override
+  public void setBalancerBandwidth(long bandwidth) throws IOException {
+    SetBalancerBandwidthRequestProto req = SetBalancerBandwidthRequestProto.newBuilder()
+        .setBandwidth(bandwidth)
+        .build();
+    try {
+      rpcProxy.setBalancerBandwidth(null, req);
+    } catch (ServiceException e) {
+      throw ProtobufHelper.getRemoteException(e);
+    }
+  }
+
+  @Override
+  public boolean isMethodSupported(String methodName) throws IOException {
+    return RpcClientUtil.isMethodSupported(rpcProxy,
+        ClientNamenodeProtocolPB.class, RpcKind.RPC_PROTOCOL_BUFFER,
+        RPC.getProtocolVersion(ClientNamenodeProtocolPB.class), methodName);
+  }
+}

Added: hadoop/common/branches/branch-0.23-PB-merge/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/protocolPB/DatanodeProtocolClientSideTranslatorPB.java
URL: http://svn.apache.org/viewvc/hadoop/common/branches/branch-0.23-PB-merge/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/protocolPB/DatanodeProtocolClientSideTranslatorPB.java?rev=1293964&view=auto
==============================================================================
--- hadoop/common/branches/branch-0.23-PB-merge/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/protocolPB/DatanodeProtocolClientSideTranslatorPB.java (added)
+++ hadoop/common/branches/branch-0.23-PB-merge/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/protocolPB/DatanodeProtocolClientSideTranslatorPB.java Sun Feb 26 23:32:06 2012
@@ -0,0 +1,331 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.hadoop.hdfs.protocolPB;
+
+import java.io.Closeable;
+import java.io.IOException;
+import java.net.InetSocketAddress;
+import java.util.HashMap;
+import java.util.Map;
+import java.util.concurrent.TimeUnit;
+
+import org.apache.hadoop.classification.InterfaceAudience;
+import org.apache.hadoop.classification.InterfaceStability;
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.hdfs.protocol.AlreadyBeingCreatedException;
+import org.apache.hadoop.hdfs.protocol.DatanodeID;
+import org.apache.hadoop.hdfs.protocol.ExtendedBlock;
+import org.apache.hadoop.hdfs.protocol.HdfsConstants;
+import org.apache.hadoop.hdfs.protocol.LocatedBlock;
+import org.apache.hadoop.hdfs.protocol.proto.DatanodeProtocolProtos.BlockReceivedAndDeletedRequestProto;
+import org.apache.hadoop.hdfs.protocol.proto.DatanodeProtocolProtos.BlockReportRequestProto;
+import org.apache.hadoop.hdfs.protocol.proto.DatanodeProtocolProtos.BlockReportResponseProto;
+import org.apache.hadoop.hdfs.protocol.proto.DatanodeProtocolProtos.CommitBlockSynchronizationRequestProto;
+import org.apache.hadoop.hdfs.protocol.proto.DatanodeProtocolProtos.DatanodeCommandProto;
+import org.apache.hadoop.hdfs.protocol.proto.DatanodeProtocolProtos.ErrorReportRequestProto;
+import org.apache.hadoop.hdfs.protocol.proto.DatanodeProtocolProtos.HeartbeatRequestProto;
+import org.apache.hadoop.hdfs.protocol.proto.DatanodeProtocolProtos.HeartbeatResponseProto;
+import org.apache.hadoop.hdfs.protocol.proto.DatanodeProtocolProtos.ProcessUpgradeRequestProto;
+import org.apache.hadoop.hdfs.protocol.proto.DatanodeProtocolProtos.ProcessUpgradeResponseProto;
+import org.apache.hadoop.hdfs.protocol.proto.DatanodeProtocolProtos.RegisterDatanodeRequestProto;
+import org.apache.hadoop.hdfs.protocol.proto.DatanodeProtocolProtos.RegisterDatanodeResponseProto;
+import org.apache.hadoop.hdfs.protocol.proto.DatanodeProtocolProtos.ReportBadBlocksRequestProto;
+import org.apache.hadoop.hdfs.protocol.proto.DatanodeProtocolProtos.StorageBlockReportProto;
+import org.apache.hadoop.hdfs.protocol.proto.DatanodeProtocolProtos.StorageReceivedDeletedBlocksProto;
+import org.apache.hadoop.hdfs.protocol.proto.HdfsProtos.VersionRequestProto;
+import org.apache.hadoop.hdfs.protocolR23Compatible.ProtocolSignatureWritable;
+import org.apache.hadoop.hdfs.server.namenode.NameNode;
+import org.apache.hadoop.hdfs.server.protocol.DatanodeCommand;
+import org.apache.hadoop.hdfs.server.protocol.DatanodeProtocol;
+import org.apache.hadoop.hdfs.server.protocol.DatanodeRegistration;
+import org.apache.hadoop.hdfs.server.protocol.DatanodeStorage;
+import org.apache.hadoop.hdfs.server.protocol.NamespaceInfo;
+import org.apache.hadoop.hdfs.server.protocol.ReceivedDeletedBlockInfo;
+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.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.UserGroupInformation;
+
+import com.google.protobuf.RpcController;
+import com.google.protobuf.ServiceException;
+
+/**
+ * This class is the client side translator to translate the requests made on
+ * {@link DatanodeProtocol} interfaces to the RPC server implementing
+ * {@link DatanodeProtocolPB}.
+ */
+@InterfaceAudience.Private
+@InterfaceStability.Stable
+public class DatanodeProtocolClientSideTranslatorPB implements
+    ProtocolMetaInterface, DatanodeProtocol, Closeable {
+  
+  /** RpcController is not used and hence is set to null */
+  private final DatanodeProtocolPB rpcProxy;
+  private static final VersionRequestProto VERSION_REQUEST = 
+      VersionRequestProto.newBuilder().build();
+  private final static RpcController NULL_CONTROLLER = null;
+  
+  public DatanodeProtocolClientSideTranslatorPB(InetSocketAddress nameNodeAddr,
+      Configuration conf) throws IOException {
+    RPC.setProtocolEngine(conf, DatanodeProtocolPB.class,
+        ProtobufRpcEngine.class);
+    UserGroupInformation ugi = UserGroupInformation.getCurrentUser();
+    rpcProxy = createNamenodeWithRetry(createNamenode(nameNodeAddr, conf, ugi));
+  }
+
+  private static DatanodeProtocolPB createNamenode(
+      InetSocketAddress nameNodeAddr, Configuration conf,
+      UserGroupInformation ugi) throws IOException {
+    return RPC.getProxy(DatanodeProtocolPB.class,
+        RPC.getProtocolVersion(DatanodeProtocolPB.class), nameNodeAddr, ugi,
+        conf, NetUtils.getSocketFactory(conf, DatanodeProtocolPB.class));
+  }
+
+  /** Create a {@link NameNode} proxy */
+  static DatanodeProtocolPB createNamenodeWithRetry(
+      DatanodeProtocolPB rpcNamenode) {
+    RetryPolicy createPolicy = RetryPolicies
+        .retryUpToMaximumCountWithFixedSleep(5,
+            HdfsConstants.LEASE_SOFTLIMIT_PERIOD, TimeUnit.MILLISECONDS);
+
+    Map<Class<? extends Exception>, RetryPolicy> remoteExceptionToPolicyMap = 
+        new HashMap<Class<? extends Exception>, RetryPolicy>();
+    remoteExceptionToPolicyMap.put(AlreadyBeingCreatedException.class,
+        createPolicy);
+
+    Map<Class<? extends Exception>, RetryPolicy> exceptionToPolicyMap =
+        new HashMap<Class<? extends Exception>, RetryPolicy>();
+    exceptionToPolicyMap.put(RemoteException.class, RetryPolicies
+        .retryByRemoteException(RetryPolicies.TRY_ONCE_THEN_FAIL,
+            remoteExceptionToPolicyMap));
+    RetryPolicy methodPolicy = RetryPolicies.retryByException(
+        RetryPolicies.TRY_ONCE_THEN_FAIL, exceptionToPolicyMap);
+    Map<String, RetryPolicy> methodNameToPolicyMap = new HashMap<String, RetryPolicy>();
+
+    methodNameToPolicyMap.put("create", methodPolicy);
+
+    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 {
+    RPC.stopProxy(rpcProxy);
+  }
+
+  @Override
+  public DatanodeRegistration registerDatanode(DatanodeRegistration registration,
+      DatanodeStorage[] storages) throws IOException {
+    RegisterDatanodeRequestProto.Builder builder = RegisterDatanodeRequestProto
+        .newBuilder().setRegistration(PBHelper.convert(registration));
+    for (DatanodeStorage s : storages) {
+      builder.addStorages(PBHelper.convert(s));
+    }
+    
+    RegisterDatanodeResponseProto resp;
+    try {
+      resp = rpcProxy.registerDatanode(NULL_CONTROLLER, builder.build());
+    } catch (ServiceException se) {
+      throw ProtobufHelper.getRemoteException(se);
+    }
+    return PBHelper.convert(resp.getRegistration());
+  }
+
+  @Override
+  public DatanodeCommand[] sendHeartbeat(DatanodeRegistration registration,
+      StorageReport[] reports, int xmitsInProgress, int xceiverCount,
+      int failedVolumes) throws IOException {
+    HeartbeatRequestProto.Builder builder = HeartbeatRequestProto.newBuilder()
+        .setRegistration(PBHelper.convert(registration))
+        .setXmitsInProgress(xmitsInProgress).setXceiverCount(xceiverCount)
+        .setFailedVolumes(failedVolumes);
+    for (StorageReport r : reports) {
+      builder.addReports(PBHelper.convert(r));
+    }
+    
+    HeartbeatResponseProto resp;
+    try {
+      resp = rpcProxy.sendHeartbeat(NULL_CONTROLLER, builder.build());
+    } catch (ServiceException se) {
+      throw ProtobufHelper.getRemoteException(se);
+    }
+    DatanodeCommand[] cmds = new DatanodeCommand[resp.getCmdsList().size()];
+    int index = 0;
+    for (DatanodeCommandProto p : resp.getCmdsList()) {
+      cmds[index] = PBHelper.convert(p);
+      index++;
+    }
+    return cmds;
+  }
+
+  @Override
+  public DatanodeCommand blockReport(DatanodeRegistration registration,
+      String poolId, StorageBlockReport[] reports) throws IOException {
+    BlockReportRequestProto.Builder builder = BlockReportRequestProto
+        .newBuilder().setRegistration(PBHelper.convert(registration))
+        .setBlockPoolId(poolId);
+    
+    for (StorageBlockReport r : reports) {
+      StorageBlockReportProto.Builder reportBuilder = StorageBlockReportProto
+          .newBuilder().setStorageID(r.getStorageID());
+      long[] blocks = r.getBlocks();
+      for (int i = 0; i < blocks.length; i++) {
+        reportBuilder.addBlocks(blocks[i]);
+      }
+      builder.addReports(reportBuilder.build());
+    }
+    BlockReportResponseProto resp;
+    try {
+      resp = rpcProxy.blockReport(NULL_CONTROLLER, builder.build());
+    } catch (ServiceException se) {
+      throw ProtobufHelper.getRemoteException(se);
+    }
+    return resp.hasCmd() ? PBHelper.convert(resp.getCmd()) : null;
+  }
+
+  @Override
+  public void blockReceivedAndDeleted(DatanodeRegistration registration,
+      String poolId, StorageReceivedDeletedBlocks[] receivedAndDeletedBlocks)
+      throws IOException {
+    BlockReceivedAndDeletedRequestProto.Builder builder = 
+        BlockReceivedAndDeletedRequestProto.newBuilder()
+        .setRegistration(PBHelper.convert(registration))
+        .setBlockPoolId(poolId);
+    for (StorageReceivedDeletedBlocks storageBlock : receivedAndDeletedBlocks) {
+      StorageReceivedDeletedBlocksProto.Builder repBuilder = 
+          StorageReceivedDeletedBlocksProto.newBuilder();
+      repBuilder.setStorageID(storageBlock.getStorageID());
+      for (ReceivedDeletedBlockInfo rdBlock : storageBlock.getBlocks()) {
+        repBuilder.addBlocks(PBHelper.convert(rdBlock));
+      }
+      builder.addBlocks(repBuilder.build());
+    }
+    try {
+      rpcProxy.blockReceivedAndDeleted(NULL_CONTROLLER, builder.build());
+    } catch (ServiceException se) {
+      throw ProtobufHelper.getRemoteException(se);
+    }
+  }
+
+  @Override
+  public void errorReport(DatanodeRegistration registration, int errorCode,
+      String msg) throws IOException {
+    ErrorReportRequestProto req = ErrorReportRequestProto.newBuilder()
+        .setRegistartion(PBHelper.convert(registration))
+        .setErrorCode(errorCode).setMsg(msg).build();
+    try {
+      rpcProxy.errorReport(NULL_CONTROLLER, req);
+    } catch (ServiceException se) {
+      throw ProtobufHelper.getRemoteException(se);
+    }
+  }
+
+  @Override
+  public NamespaceInfo versionRequest() throws IOException {
+    try {
+      return PBHelper.convert(rpcProxy.versionRequest(NULL_CONTROLLER,
+          VERSION_REQUEST).getInfo());
+    } catch (ServiceException e) {
+      throw ProtobufHelper.getRemoteException(e);
+    }
+  }
+
+  @Override
+  public UpgradeCommand processUpgradeCommand(UpgradeCommand comm)
+      throws IOException {
+    ProcessUpgradeRequestProto req = ProcessUpgradeRequestProto.newBuilder()
+        .setCmd(PBHelper.convert(comm)).build();
+    ProcessUpgradeResponseProto resp;
+    try {
+      resp = rpcProxy.processUpgrade(NULL_CONTROLLER, req);
+    } catch (ServiceException se) {
+      throw ProtobufHelper.getRemoteException(se);
+    }
+    return resp.hasCmd() ? PBHelper.convert(resp.getCmd()) : null;
+  }
+
+  @Override
+  public void reportBadBlocks(LocatedBlock[] blocks) throws IOException {
+    ReportBadBlocksRequestProto.Builder builder = ReportBadBlocksRequestProto
+        .newBuilder();
+    for (int i = 0; i < blocks.length; i++) {
+      builder.addBlocks(i, PBHelper.convert(blocks[i]));
+    }
+    ReportBadBlocksRequestProto req = builder.build();
+    try {
+      rpcProxy.reportBadBlocks(NULL_CONTROLLER, req);
+    } catch (ServiceException se) {
+      throw ProtobufHelper.getRemoteException(se);
+    }
+  }
+
+  @Override
+  public void commitBlockSynchronization(ExtendedBlock block,
+      long newgenerationstamp, long newlength, boolean closeFile,
+      boolean deleteblock, DatanodeID[] newtargets) throws IOException {
+    CommitBlockSynchronizationRequestProto.Builder builder = 
+        CommitBlockSynchronizationRequestProto.newBuilder()
+        .setBlock(PBHelper.convert(block)).setNewGenStamp(newgenerationstamp)
+        .setNewLength(newlength).setCloseFile(closeFile)
+        .setDeleteBlock(deleteblock);
+    for (int i = 0; i < newtargets.length; i++) {
+      builder.addNewTaragets(PBHelper.convert(newtargets[i]));
+    }
+    CommitBlockSynchronizationRequestProto req = builder.build();
+    try {
+      rpcProxy.commitBlockSynchronization(NULL_CONTROLLER, req);
+    } catch (ServiceException se) {
+      throw ProtobufHelper.getRemoteException(se);
+    }
+  }
+
+  @Override // ProtocolMetaInterface
+  public boolean isMethodSupported(String methodName)
+      throws IOException {
+    return RpcClientUtil.isMethodSupported(rpcProxy, DatanodeProtocolPB.class,
+        RpcKind.RPC_PROTOCOL_BUFFER,
+        RPC.getProtocolVersion(DatanodeProtocolPB.class), methodName);
+  }
+}

Copied: hadoop/common/branches/branch-0.23-PB-merge/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/protocolPB/DatanodeProtocolPB.java (from r1293950, hadoop/common/branches/branch-0.23-PB-merge/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/protocol/InterDatanodeProtocol.java)
URL: http://svn.apache.org/viewvc/hadoop/common/branches/branch-0.23-PB-merge/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/protocolPB/DatanodeProtocolPB.java?p2=hadoop/common/branches/branch-0.23-PB-merge/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/protocolPB/DatanodeProtocolPB.java&p1=hadoop/common/branches/branch-0.23-PB-merge/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/protocol/InterDatanodeProtocol.java&r1=1293950&r2=1293964&rev=1293964&view=diff
==============================================================================
--- hadoop/common/branches/branch-0.23-PB-merge/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/protocol/InterDatanodeProtocol.java (original)
+++ hadoop/common/branches/branch-0.23-PB-merge/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/protocolPB/DatanodeProtocolPB.java Sun Feb 26 23:32:06 2012
@@ -16,46 +16,33 @@
  * limitations under the License.
  */
 
-package org.apache.hadoop.hdfs.server.protocol;
+package org.apache.hadoop.hdfs.protocolPB;
 
 import java.io.IOException;
 
-import org.apache.commons.logging.Log;
-import org.apache.commons.logging.LogFactory;
 import org.apache.hadoop.classification.InterfaceAudience;
 import org.apache.hadoop.hdfs.DFSConfigKeys;
-import org.apache.hadoop.hdfs.protocol.ExtendedBlock;
-import org.apache.hadoop.hdfs.server.protocol.BlockRecoveryCommand.RecoveringBlock;
+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;
 
-/** An inter-datanode protocol for updating generation stamp
- */
 @KerberosInfo(
-    serverPrincipal = DFSConfigKeys.DFS_DATANODE_USER_NAME_KEY,
+    serverPrincipal = DFSConfigKeys.DFS_NAMENODE_USER_NAME_KEY, 
     clientPrincipal = DFSConfigKeys.DFS_DATANODE_USER_NAME_KEY)
+@ProtocolInfo(
+    protocolName = "org.apache.hadoop.hdfs.server.protocol.DatanodeProtocol", 
+    protocolVersion = 1)
 @InterfaceAudience.Private
-public interface InterDatanodeProtocol extends VersionedProtocol {
-  public static final Log LOG = LogFactory.getLog(InterDatanodeProtocol.class);
-
-  /**
-   * 6: Add block pool ID to Block
-   */
-  public static final long versionID = 6L;
-
-  /**
-   * Initialize a replica recovery.
-   * 
-   * @return actual state of the replica on this data-node or 
-   * null if data-node does not have the replica.
-   */
-  ReplicaRecoveryInfo initReplicaRecovery(RecoveringBlock rBlock)
-  throws IOException;
-
+public interface DatanodeProtocolPB extends
+    DatanodeProtocolService.BlockingInterface, VersionedProtocol {
+  
   /**
-   * Update replica with the new generation stamp and length.  
+   * 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.
    */
-  ExtendedBlock updateReplicaUnderRecovery(ExtendedBlock oldBlock,
-                                   long recoveryId,
-                                   long newLength) throws IOException;
+  public ProtocolSignatureWritable getProtocolSignature2(String protocol,
+      long clientVersion, int clientMethodsHash) throws IOException;
 }

Added: hadoop/common/branches/branch-0.23-PB-merge/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/protocolPB/DatanodeProtocolServerSideTranslatorPB.java
URL: http://svn.apache.org/viewvc/hadoop/common/branches/branch-0.23-PB-merge/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/protocolPB/DatanodeProtocolServerSideTranslatorPB.java?rev=1293964&view=auto
==============================================================================
--- hadoop/common/branches/branch-0.23-PB-merge/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/protocolPB/DatanodeProtocolServerSideTranslatorPB.java (added)
+++ hadoop/common/branches/branch-0.23-PB-merge/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/protocolPB/DatanodeProtocolServerSideTranslatorPB.java Sun Feb 26 23:32:06 2012
@@ -0,0 +1,307 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.hadoop.hdfs.protocolPB;
+
+import java.io.IOException;
+import java.util.List;
+
+import org.apache.hadoop.hdfs.protocol.DatanodeID;
+import org.apache.hadoop.hdfs.protocol.LocatedBlock;
+import org.apache.hadoop.hdfs.protocol.proto.DatanodeProtocolProtos.BlockReceivedAndDeletedRequestProto;
+import org.apache.hadoop.hdfs.protocol.proto.DatanodeProtocolProtos.BlockReceivedAndDeletedResponseProto;
+import org.apache.hadoop.hdfs.protocol.proto.DatanodeProtocolProtos.BlockReportRequestProto;
+import org.apache.hadoop.hdfs.protocol.proto.DatanodeProtocolProtos.BlockReportResponseProto;
+import org.apache.hadoop.hdfs.protocol.proto.DatanodeProtocolProtos.CommitBlockSynchronizationRequestProto;
+import org.apache.hadoop.hdfs.protocol.proto.DatanodeProtocolProtos.CommitBlockSynchronizationResponseProto;
+import org.apache.hadoop.hdfs.protocol.proto.DatanodeProtocolProtos.ErrorReportRequestProto;
+import org.apache.hadoop.hdfs.protocol.proto.DatanodeProtocolProtos.ErrorReportResponseProto;
+import org.apache.hadoop.hdfs.protocol.proto.DatanodeProtocolProtos.HeartbeatRequestProto;
+import org.apache.hadoop.hdfs.protocol.proto.DatanodeProtocolProtos.HeartbeatResponseProto;
+import org.apache.hadoop.hdfs.protocol.proto.DatanodeProtocolProtos.ProcessUpgradeRequestProto;
+import org.apache.hadoop.hdfs.protocol.proto.DatanodeProtocolProtos.ProcessUpgradeResponseProto;
+import org.apache.hadoop.hdfs.protocol.proto.DatanodeProtocolProtos.ReceivedDeletedBlockInfoProto;
+import org.apache.hadoop.hdfs.protocol.proto.DatanodeProtocolProtos.RegisterDatanodeRequestProto;
+import org.apache.hadoop.hdfs.protocol.proto.DatanodeProtocolProtos.RegisterDatanodeResponseProto;
+import org.apache.hadoop.hdfs.protocol.proto.DatanodeProtocolProtos.ReportBadBlocksRequestProto;
+import org.apache.hadoop.hdfs.protocol.proto.DatanodeProtocolProtos.ReportBadBlocksResponseProto;
+import org.apache.hadoop.hdfs.protocol.proto.DatanodeProtocolProtos.StorageBlockReportProto;
+import org.apache.hadoop.hdfs.protocol.proto.DatanodeProtocolProtos.StorageReceivedDeletedBlocksProto;
+import org.apache.hadoop.hdfs.protocol.proto.DatanodeProtocolProtos.StorageReportProto;
+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;
+import org.apache.hadoop.hdfs.server.protocol.DatanodeStorage;
+import org.apache.hadoop.hdfs.server.protocol.NamespaceInfo;
+import org.apache.hadoop.hdfs.server.protocol.ReceivedDeletedBlockInfo;
+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;
+
+public class DatanodeProtocolServerSideTranslatorPB implements
+    DatanodeProtocolPB {
+
+  private final DatanodeProtocol impl;
+  private static final ErrorReportResponseProto ERROR_REPORT_RESPONSE_PROTO = 
+      ErrorReportResponseProto.newBuilder().build();
+  private static final BlockReceivedAndDeletedResponseProto 
+      BLOCK_RECEIVED_AND_DELETE_RESPONSE = 
+          BlockReceivedAndDeletedResponseProto.newBuilder().build();
+  private static final ReportBadBlocksResponseProto REPORT_BAD_BLOCK_RESPONSE = 
+      ReportBadBlocksResponseProto.newBuilder().build();
+  private static final CommitBlockSynchronizationResponseProto 
+      COMMIT_BLOCK_SYNCHRONIZATION_RESPONSE_PROTO =
+          CommitBlockSynchronizationResponseProto.newBuilder().build();
+
+  public DatanodeProtocolServerSideTranslatorPB(DatanodeProtocol impl) {
+    this.impl = impl;
+  }
+
+  @Override
+  public RegisterDatanodeResponseProto registerDatanode(
+      RpcController controller, RegisterDatanodeRequestProto request)
+      throws ServiceException {
+    DatanodeRegistration registration = PBHelper.convert(request
+        .getRegistration());
+    DatanodeRegistration registrationResp;
+    DatanodeStorage[] storages = new DatanodeStorage[request.getStoragesCount()];
+    for (int i = 0; i < request.getStoragesCount(); i++) {
+      storages[i] = PBHelper.convert(request.getStorages(i));
+    }
+    try {
+      registrationResp = impl.registerDatanode(registration, storages);
+    } catch (IOException e) {
+      throw new ServiceException(e);
+    }
+    return RegisterDatanodeResponseProto.newBuilder()
+        .setRegistration(PBHelper.convert(registrationResp)).build();
+  }
+
+  @Override
+  public HeartbeatResponseProto sendHeartbeat(RpcController controller,
+      HeartbeatRequestProto request) throws ServiceException {
+    DatanodeCommand[] cmds = null;
+    try {
+      List<StorageReportProto> list = request.getReportsList();
+      StorageReport[] report = new StorageReport[list.size()];
+      int i = 0;
+      for (StorageReportProto p : list) {
+        report[i++] = new StorageReport(p.getStorageID(), p.getFailed(),
+            p.getCapacity(), p.getDfsUsed(), p.getRemaining(),
+            p.getBlockPoolUsed());
+      }
+      cmds = impl.sendHeartbeat(PBHelper.convert(request.getRegistration()),
+          report, request.getXmitsInProgress(), request.getXceiverCount(),
+          request.getFailedVolumes());
+    } catch (IOException e) {
+      throw new ServiceException(e);
+    }
+    HeartbeatResponseProto.Builder builder = HeartbeatResponseProto
+        .newBuilder();
+    if (cmds != null) {
+      for (int i = 0; i < cmds.length; i++) {
+        if (cmds[i] != null) {
+          builder.addCmds(PBHelper.convert(cmds[i]));
+        }
+      }
+    }
+    return builder.build();
+  }
+
+  @Override
+  public BlockReportResponseProto blockReport(RpcController controller,
+      BlockReportRequestProto request) throws ServiceException {
+    DatanodeCommand cmd = null;
+    StorageBlockReport[] report = 
+        new StorageBlockReport[request.getReportsCount()];
+    
+    int index = 0;
+    for (StorageBlockReportProto s : request.getReportsList()) {
+      List<Long> blockIds = s.getBlocksList();
+      long[] blocks = new long[blockIds.size()];
+      for (int i = 0; i < blockIds.size(); i++) {
+        blocks[i] = blockIds.get(i);
+      }
+      report[index++] = new StorageBlockReport(s.getStorageID(), blocks);
+    }
+    try {
+      cmd = impl.blockReport(PBHelper.convert(request.getRegistration()),
+          request.getBlockPoolId(), report);
+    } catch (IOException e) {
+      throw new ServiceException(e);
+    }
+    BlockReportResponseProto.Builder builder = 
+        BlockReportResponseProto.newBuilder();
+    if (cmd != null) {
+      builder.setCmd(PBHelper.convert(cmd));
+    }
+    return builder.build();
+  }
+
+  @Override
+  public BlockReceivedAndDeletedResponseProto blockReceivedAndDeleted(
+      RpcController controller, BlockReceivedAndDeletedRequestProto request)
+      throws ServiceException {
+    List<StorageReceivedDeletedBlocksProto> sBlocks = request.getBlocksList();
+    StorageReceivedDeletedBlocks[] info = 
+        new StorageReceivedDeletedBlocks[sBlocks.size()];
+    for (int i = 0; i < sBlocks.size(); i++) {
+      StorageReceivedDeletedBlocksProto sBlock = sBlocks.get(i);
+      List<ReceivedDeletedBlockInfoProto> list = sBlock.getBlocksList();
+      ReceivedDeletedBlockInfo[] rdBlocks = 
+          new ReceivedDeletedBlockInfo[list.size()];
+      for (int j = 0; j < list.size(); j++) {
+        rdBlocks[j] = PBHelper.convert(list.get(j));
+      }
+      info[i] = new StorageReceivedDeletedBlocks(sBlock.getStorageID(), rdBlocks);
+    }
+    try {
+      impl.blockReceivedAndDeleted(PBHelper.convert(request.getRegistration()),
+          request.getBlockPoolId(), info);
+    } catch (IOException e) {
+      throw new ServiceException(e);
+    }
+    return BLOCK_RECEIVED_AND_DELETE_RESPONSE;
+  }
+
+  @Override
+  public ErrorReportResponseProto errorReport(RpcController controller,
+      ErrorReportRequestProto request) throws ServiceException {
+    try {
+      impl.errorReport(PBHelper.convert(request.getRegistartion()),
+          request.getErrorCode(), request.getMsg());
+    } catch (IOException e) {
+      throw new ServiceException(e);
+    }
+    return ERROR_REPORT_RESPONSE_PROTO;
+  }
+
+  @Override
+  public VersionResponseProto versionRequest(RpcController controller,
+      VersionRequestProto request) throws ServiceException {
+    NamespaceInfo info;
+    try {
+      info = impl.versionRequest();
+    } catch (IOException e) {
+      throw new ServiceException(e);
+    }
+    return VersionResponseProto.newBuilder()
+        .setInfo(PBHelper.convert(info)).build();
+  }
+
+  @Override
+  public ProcessUpgradeResponseProto processUpgrade(RpcController controller,
+      ProcessUpgradeRequestProto request) throws ServiceException {
+    UpgradeCommand ret;
+    try {
+      UpgradeCommand cmd = request.hasCmd() ? PBHelper
+          .convert(request.getCmd()) : null;
+      ret = impl.processUpgradeCommand(cmd);
+    } catch (IOException e) {
+      throw new ServiceException(e);
+    }
+    ProcessUpgradeResponseProto.Builder builder = 
+        ProcessUpgradeResponseProto.newBuilder();
+    if (ret != null) {
+      builder.setCmd(PBHelper.convert(ret));
+    }
+    return builder.build();
+  }
+
+  @Override
+  public ReportBadBlocksResponseProto reportBadBlocks(RpcController controller,
+      ReportBadBlocksRequestProto request) throws ServiceException {
+    List<LocatedBlockProto> lbps = request.getBlocksList();
+    LocatedBlock [] blocks = new LocatedBlock [lbps.size()];
+    for(int i=0; i<lbps.size(); i++) {
+      blocks[i] = PBHelper.convert(lbps.get(i));
+    }
+    try {
+      impl.reportBadBlocks(blocks);
+    } catch (IOException e) {
+      throw new ServiceException(e);
+    }
+    return REPORT_BAD_BLOCK_RESPONSE;
+  }
+
+  @Override
+  public CommitBlockSynchronizationResponseProto commitBlockSynchronization(
+      RpcController controller, CommitBlockSynchronizationRequestProto request)
+      throws ServiceException {
+    List<DatanodeIDProto> dnprotos = request.getNewTaragetsList();
+    DatanodeID[] dns = new DatanodeID[dnprotos.size()];
+    for (int i = 0; i < dnprotos.size(); i++) {
+      dns[i] = PBHelper.convert(dnprotos.get(i));
+    }
+    try {
+      impl.commitBlockSynchronization(PBHelper.convert(request.getBlock()),
+          request.getNewGenStamp(), request.getNewLength(),
+          request.getCloseFile(), request.getDeleteBlock(), dns);
+    } catch (IOException e) {
+      throw new ServiceException(e);
+    }
+    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));
+  }
+
+}

Added: hadoop/common/branches/branch-0.23-PB-merge/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/protocolPB/GetUserMappingsProtocolClientSideTranslatorPB.java
URL: http://svn.apache.org/viewvc/hadoop/common/branches/branch-0.23-PB-merge/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/protocolPB/GetUserMappingsProtocolClientSideTranslatorPB.java?rev=1293964&view=auto
==============================================================================
--- hadoop/common/branches/branch-0.23-PB-merge/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/protocolPB/GetUserMappingsProtocolClientSideTranslatorPB.java (added)
+++ hadoop/common/branches/branch-0.23-PB-merge/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/protocolPB/GetUserMappingsProtocolClientSideTranslatorPB.java Sun Feb 26 23:32:06 2012
@@ -0,0 +1,99 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.hadoop.hdfs.protocolPB;
+
+import java.io.Closeable;
+import java.io.IOException;
+import java.net.InetSocketAddress;
+
+import org.apache.hadoop.conf.Configuration;
+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.hdfs.server.namenode.NameNode;
+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;
+import org.apache.hadoop.net.NetUtils;
+import org.apache.hadoop.security.UserGroupInformation;
+import org.apache.hadoop.tools.GetUserMappingsProtocol;
+
+import com.google.protobuf.RpcController;
+import com.google.protobuf.ServiceException;
+
+public class GetUserMappingsProtocolClientSideTranslatorPB implements
+    ProtocolMetaInterface, GetUserMappingsProtocol, Closeable {
+
+  /** RpcController is not used and hence is set to null */
+  private final static RpcController NULL_CONTROLLER = null;
+  private final GetUserMappingsProtocolPB rpcProxy;
+
+  public GetUserMappingsProtocolClientSideTranslatorPB(
+      InetSocketAddress nameNodeAddr, UserGroupInformation ugi,
+      Configuration conf) throws IOException {
+    RPC.setProtocolEngine(conf, GetUserMappingsProtocolPB.class,
+        ProtobufRpcEngine.class);
+    rpcProxy = RPC.getProxy(GetUserMappingsProtocolPB.class,
+        RPC.getProtocolVersion(GetUserMappingsProtocolPB.class),
+        NameNode.getAddress(conf), ugi, conf,
+        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);
+  }
+
+  @Override
+  public String[] getGroupsForUser(String user) throws IOException {
+    GetGroupsForUserRequestProto request = GetGroupsForUserRequestProto
+        .newBuilder().setUser(user).build();
+    GetGroupsForUserResponseProto resp;
+    try {
+      resp = rpcProxy.getGroupsForUser(NULL_CONTROLLER, request);
+    } catch (ServiceException se) {
+      throw ProtobufHelper.getRemoteException(se);
+    }
+    return resp.getGroupsList().toArray(new String[resp.getGroupsCount()]);
+  }
+
+  @Override
+  public boolean isMethodSupported(String methodName) throws IOException {
+    return RpcClientUtil.isMethodSupported(rpcProxy,
+        GetUserMappingsProtocolPB.class, RpcKind.RPC_PROTOCOL_BUFFER,
+        RPC.getProtocolVersion(GetUserMappingsProtocolPB.class), methodName);
+  }
+}

Added: hadoop/common/branches/branch-0.23-PB-merge/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/protocolPB/GetUserMappingsProtocolPB.java
URL: http://svn.apache.org/viewvc/hadoop/common/branches/branch-0.23-PB-merge/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/protocolPB/GetUserMappingsProtocolPB.java?rev=1293964&view=auto
==============================================================================
--- hadoop/common/branches/branch-0.23-PB-merge/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/protocolPB/GetUserMappingsProtocolPB.java (added)
+++ hadoop/common/branches/branch-0.23-PB-merge/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/protocolPB/GetUserMappingsProtocolPB.java Sun Feb 26 23:32:06 2012
@@ -0,0 +1,45 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.hadoop.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", 
+    protocolVersion = 1)
+@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;
+}

Added: hadoop/common/branches/branch-0.23-PB-merge/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/protocolPB/GetUserMappingsProtocolServerSideTranslatorPB.java
URL: http://svn.apache.org/viewvc/hadoop/common/branches/branch-0.23-PB-merge/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/protocolPB/GetUserMappingsProtocolServerSideTranslatorPB.java?rev=1293964&view=auto
==============================================================================
--- hadoop/common/branches/branch-0.23-PB-merge/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/protocolPB/GetUserMappingsProtocolServerSideTranslatorPB.java (added)
+++ hadoop/common/branches/branch-0.23-PB-merge/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/protocolPB/GetUserMappingsProtocolServerSideTranslatorPB.java Sun Feb 26 23:32:06 2012
@@ -0,0 +1,96 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.hadoop.hdfs.protocolPB;
+
+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;
+import com.google.protobuf.ServiceException;
+
+public class GetUserMappingsProtocolServerSideTranslatorPB implements
+    GetUserMappingsProtocolPB {
+
+  private final GetUserMappingsProtocol impl;
+
+  public GetUserMappingsProtocolServerSideTranslatorPB(
+      GetUserMappingsProtocol impl) {
+    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)
+      throws ServiceException {
+    String[] groups;
+    try {
+      groups = impl.getGroupsForUser(request.getUser());
+    } catch (IOException e) {
+      throw new ServiceException(e);
+    }
+    GetGroupsForUserResponseProto.Builder builder = GetGroupsForUserResponseProto
+        .newBuilder();
+    for (String g : groups) {
+      builder.addGroups(g);
+    }
+    return builder.build();
+  }
+}

Copied: hadoop/common/branches/branch-0.23-PB-merge/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/protocolPB/InterDatanodeProtocolPB.java (from r1293950, hadoop/common/branches/branch-0.23-PB-merge/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/protocol/InterDatanodeProtocol.java)
URL: http://svn.apache.org/viewvc/hadoop/common/branches/branch-0.23-PB-merge/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/protocolPB/InterDatanodeProtocolPB.java?p2=hadoop/common/branches/branch-0.23-PB-merge/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/protocolPB/InterDatanodeProtocolPB.java&p1=hadoop/common/branches/branch-0.23-PB-merge/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/protocol/InterDatanodeProtocol.java&r1=1293950&r2=1293964&rev=1293964&view=diff
==============================================================================
--- hadoop/common/branches/branch-0.23-PB-merge/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/protocol/InterDatanodeProtocol.java (original)
+++ hadoop/common/branches/branch-0.23-PB-merge/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/protocolPB/InterDatanodeProtocolPB.java Sun Feb 26 23:32:06 2012
@@ -15,47 +15,33 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
-
-package org.apache.hadoop.hdfs.server.protocol;
+package org.apache.hadoop.hdfs.protocolPB;
 
 import java.io.IOException;
 
-import org.apache.commons.logging.Log;
-import org.apache.commons.logging.LogFactory;
 import org.apache.hadoop.classification.InterfaceAudience;
 import org.apache.hadoop.hdfs.DFSConfigKeys;
-import org.apache.hadoop.hdfs.protocol.ExtendedBlock;
-import org.apache.hadoop.hdfs.server.protocol.BlockRecoveryCommand.RecoveringBlock;
+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;
 
-/** An inter-datanode protocol for updating generation stamp
- */
 @KerberosInfo(
     serverPrincipal = DFSConfigKeys.DFS_DATANODE_USER_NAME_KEY,
     clientPrincipal = DFSConfigKeys.DFS_DATANODE_USER_NAME_KEY)
+@ProtocolInfo(protocolName = 
+    "org.apache.hadoop.hdfs.server.protocol.InterDatanodeProtocol",
+    protocolVersion = 1)
 @InterfaceAudience.Private
-public interface InterDatanodeProtocol extends VersionedProtocol {
-  public static final Log LOG = LogFactory.getLog(InterDatanodeProtocol.class);
-
-  /**
-   * 6: Add block pool ID to Block
-   */
-  public static final long versionID = 6L;
-
-  /**
-   * Initialize a replica recovery.
-   * 
-   * @return actual state of the replica on this data-node or 
-   * null if data-node does not have the replica.
-   */
-  ReplicaRecoveryInfo initReplicaRecovery(RecoveringBlock rBlock)
-  throws IOException;
+public interface InterDatanodeProtocolPB extends
+    InterDatanodeProtocolService.BlockingInterface, VersionedProtocol {
 
   /**
-   * Update replica with the new generation stamp and length.  
+   * 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.
    */
-  ExtendedBlock updateReplicaUnderRecovery(ExtendedBlock oldBlock,
-                                   long recoveryId,
-                                   long newLength) throws IOException;
-}
+  public ProtocolSignatureWritable getProtocolSignature2(String protocol,
+      long clientVersion, int clientMethodsHash) throws IOException;
+}
\ No newline at end of file