Merge r1293034 through r1293500 from 0.23.


git-svn-id: https://svn.apache.org/repos/asf/hadoop/common/branches/branch-0.23-PB@1293501 13f79535-47bb-0310-9956-ffa450edef68
diff --git a/hadoop-common-project/hadoop-common/CHANGES.txt b/hadoop-common-project/hadoop-common/CHANGES.txt
index d321c8d..35ab2ac 100644
--- a/hadoop-common-project/hadoop-common/CHANGES.txt
+++ b/hadoop-common-project/hadoop-common/CHANGES.txt
@@ -1,5 +1,76 @@
 Hadoop Change Log
 
+Release 0.23-PB - Unreleased
+
+  NEW FEATURES                                                                    
+    HADOOP-7920. Remove Avro Rpc. (suresh)
+
+    HADOOP-7773. Add support for protocol buffer based RPC engine.
+    (suresh)
+
+    HADOOP-7875. Add helper class to unwrap protobuf ServiceException.
+    (suresh)
+
+  IMPROVEMENTS
+
+    HADOOP-7524 Change RPC to allow multiple protocols including multuple versions of the same protocol (sanjay Radia)
+
+    HADOOP-7607. Simplify the RPC proxy cleanup process. (atm)
+
+    HADOOP-7635. RetryInvocationHandler should release underlying resources on
+    close (atm)
+
+    HADOOP-7687 Make getProtocolSignature public  (sanjay)
+
+    HADOOP-7693. Enhance AvroRpcEngine to support the new #addProtocol
+    interface introduced in HADOOP-7524.  (cutting)
+
+    HADOOP-7716 RPC protocol registration on SS does not log the protocol name
+    (only the class which may be different) (sanjay)
+
+    HADOOP-7776 Make the Ipc-Header in a RPC-Payload an explicit header (sanjay)
+
+    HADOOP-7862  Move the support for multiple protocols to lower layer so
+    that Writable, PB and Avro can all use it (Sanjay)
+
+    HADOOP-7876. Provided access to encoded key in DelegationKey for
+    use in protobuf based RPCs. (suresh)
+
+    HADOOP-7899. Generate proto java files as part of the build. (tucu)
+
+    HADOOP-7957. Classes deriving GetGroupsBase should be able to override 
+    proxy creation. (jitendra)
+
+    HADOOP-7968. Errant println left in RPC.getHighestSupportedProtocol (Sho Shimauchi via harsh)
+
+    HADOOP-7965. Support for protocol version and signature in PB. (jitendra)
+
+    HADOOP-8070. Add a standalone benchmark for RPC call performance. (todd)
+
+    HADOOP-8084. Updates ProtoBufRpc engine to not do an unnecessary copy 
+    for RPC request/response. (ddas)
+
+    HADOOP-8085. Add RPC metrics to ProtobufRpcEngine. (Hari Mankude via
+    suresh)
+
+  BUG FIXES
+
+    HADOOP-7695. RPC.stopProxy can throw unintended exception while logging
+                 error (atm)
+
+    HADOOP-7833. Fix findbugs warnings in protobuf generated code.
+    (John Lee via suresh)
+
+    HADOOP-7897. ProtobufRpcEngine client side exception mechanism is not
+    consistent with WritableRpcEngine. (suresh)
+
+    HADOOP-7913 Fix bug in ProtoBufRpcEngine  (sanjay)
+
+    HADOOP-7892. IPC logs too verbose after "RpcKind" introduction (todd)
+
+    HADOOP-7931. o.a.h.ipc.WritableRpcEngine should have a way to force
+                 initialization (atm)
+
 Release 0.23.2 - UNRELEASED
 
   NEW FEATURES
diff --git a/hadoop-common-project/hadoop-common/dev-support/findbugsExcludeFile.xml b/hadoop-common-project/hadoop-common/dev-support/findbugsExcludeFile.xml
index 48595ff..115d712 100644
--- a/hadoop-common-project/hadoop-common/dev-support/findbugsExcludeFile.xml
+++ b/hadoop-common-project/hadoop-common/dev-support/findbugsExcludeFile.xml
@@ -270,4 +270,12 @@
       <!-- backward compatibility -->
       <Bug pattern="NM_SAME_SIMPLE_NAME_AS_SUPERCLASS"/>
     </Match>
+    <Match>
+      <!-- protobuf generated code -->
+      <Class name="~org\.apache\.hadoop\.ipc\.protobuf\.HadoopRpcProtos.*"/>
+    </Match>
+    <Match>
+      <!-- protobuf generated code -->
+      <Class name="~org\.apache\.hadoop\.ipc\.protobuf\.ProtocolInfoProtos.*"/>
+    </Match>
  </FindBugsFilter>
diff --git a/hadoop-common-project/hadoop-common/pom.xml b/hadoop-common-project/hadoop-common/pom.xml
index 88e5953..ec87638 100644
--- a/hadoop-common-project/hadoop-common/pom.xml
+++ b/hadoop-common-project/hadoop-common/pom.xml
@@ -240,11 +240,6 @@
       <scope>compile</scope>
     </dependency>
     <dependency>
-      <groupId>org.apache.avro</groupId>
-      <artifactId>avro-ipc</artifactId>
-      <scope>compile</scope>
-    </dependency>
-    <dependency>
       <groupId>net.sf.kosmosfs</groupId>
       <artifactId>kfs</artifactId>
       <scope>compile</scope>
@@ -282,7 +277,6 @@
             <phase>generate-test-sources</phase>
             <goals>
               <goal>schema</goal>
-              <goal>protocol</goal>
             </goals>
           </execution>
         </executions>
diff --git a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/io/retry/DefaultFailoverProxyProvider.java b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/io/retry/DefaultFailoverProxyProvider.java
index 0ea294d..812a46e 100644
--- a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/io/retry/DefaultFailoverProxyProvider.java
+++ b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/io/retry/DefaultFailoverProxyProvider.java
@@ -17,7 +17,10 @@
  */
 package org.apache.hadoop.io.retry;
 
+import java.io.IOException;
+
 import org.apache.hadoop.classification.InterfaceStability;
+import org.apache.hadoop.ipc.RPC;
 
 /**
  * An implementation of {@link FailoverProxyProvider} which does nothing in the
@@ -49,4 +52,9 @@
     // Nothing to do.
   }
 
+  @Override
+  public void close() throws IOException {
+    RPC.stopProxy(proxy);
+  }
+
 }
diff --git a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/io/retry/FailoverProxyProvider.java b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/io/retry/FailoverProxyProvider.java
index cb211c2..707a40d 100644
--- a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/io/retry/FailoverProxyProvider.java
+++ b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/io/retry/FailoverProxyProvider.java
@@ -17,6 +17,8 @@
  */
 package org.apache.hadoop.io.retry;
 
+import java.io.Closeable;
+
 import org.apache.hadoop.classification.InterfaceStability;
 
 /**
@@ -27,7 +29,7 @@
  * {@link RetryPolicy}.
  */
 @InterfaceStability.Evolving
-public interface FailoverProxyProvider {
+public interface FailoverProxyProvider extends Closeable {
 
   /**
    * Get the proxy object which should be used until the next failover event
diff --git a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/io/retry/RetryInvocationHandler.java b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/io/retry/RetryInvocationHandler.java
index 70149e3..51ff6a7 100644
--- a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/io/retry/RetryInvocationHandler.java
+++ b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/io/retry/RetryInvocationHandler.java
@@ -17,17 +17,20 @@
  */
 package org.apache.hadoop.io.retry;
 
-import java.lang.reflect.InvocationHandler;
+import java.io.IOException;
 import java.lang.reflect.InvocationTargetException;
 import java.lang.reflect.Method;
+import java.lang.reflect.Proxy;
 import java.util.Collections;
 import java.util.Map;
 
 import org.apache.commons.logging.Log;
 import org.apache.commons.logging.LogFactory;
 import org.apache.hadoop.io.retry.RetryPolicy.RetryAction;
+import org.apache.hadoop.ipc.Client.ConnectionId;
+import org.apache.hadoop.ipc.RpcInvocationHandler;
 
-class RetryInvocationHandler implements InvocationHandler {
+class RetryInvocationHandler implements RpcInvocationHandler {
   public static final Log LOG = LogFactory.getLog(RetryInvocationHandler.class);
   private FailoverProxyProvider proxyProvider;
   
@@ -103,4 +106,16 @@
     }
   }
 
+  @Override
+  public void close() throws IOException {
+    proxyProvider.close();
+  }
+
+  @Override //RpcInvocationHandler
+  public ConnectionId getConnectionId() {
+    RpcInvocationHandler inv = (RpcInvocationHandler) Proxy
+        .getInvocationHandler(currentProxy);
+    return inv.getConnectionId();
+  }
+
 }
diff --git a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/ipc/AvroRpcEngine.java b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/ipc/AvroRpcEngine.java
deleted file mode 100644
index 5ab379c..0000000
--- a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/ipc/AvroRpcEngine.java
+++ /dev/null
@@ -1,238 +0,0 @@
-/**
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- *     http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-
-package org.apache.hadoop.ipc;
-
-import java.io.Closeable;
-import java.io.DataInput;
-import java.io.DataOutput;
-import java.io.IOException;
-import java.lang.reflect.InvocationHandler;
-import java.lang.reflect.Method;
-import java.lang.reflect.Proxy;
-import java.net.InetSocketAddress;
-import java.nio.ByteBuffer;
-import java.util.ArrayList;
-import java.util.List;
-
-import javax.net.SocketFactory;
-
-import org.apache.avro.ipc.Responder;
-import org.apache.avro.ipc.Transceiver;
-import org.apache.avro.ipc.reflect.ReflectRequestor;
-import org.apache.avro.ipc.reflect.ReflectResponder;
-import org.apache.avro.ipc.specific.SpecificRequestor;
-import org.apache.commons.logging.Log;
-import org.apache.commons.logging.LogFactory;
-import org.apache.hadoop.classification.InterfaceStability;
-import org.apache.hadoop.conf.Configuration;
-import org.apache.hadoop.io.Writable;
-import org.apache.hadoop.security.UserGroupInformation;
-import org.apache.hadoop.security.token.SecretManager;
-import org.apache.hadoop.security.token.TokenIdentifier;
-
-/** Tunnel Avro-format RPC requests over a Hadoop {@link RPC} connection.  This
- * does not give cross-language wire compatibility, since the Hadoop RPC wire
- * format is non-standard, but it does permit use of Avro's protocol versioning
- * features for inter-Java RPCs. */
-@InterfaceStability.Evolving
-public class AvroRpcEngine implements RpcEngine {
-  private static final Log LOG = LogFactory.getLog(RPC.class);
-
-  private static int VERSION = 0;
-
-  // the implementation we tunnel through
-  private static final RpcEngine ENGINE = new WritableRpcEngine();
-
-  /** Tunnel an Avro RPC request and response through Hadoop's RPC. */
-  private static interface TunnelProtocol extends VersionedProtocol {
-    //WritableRpcEngine expects a versionID in every protocol.
-    public static final long versionID = 0L;
-    /** All Avro methods and responses go through this. */
-    BufferListWritable call(BufferListWritable request) throws IOException;
-  }
-
-  /** A Writable that holds a List<ByteBuffer>, The Avro RPC Transceiver's
-   * basic unit of data transfer.*/
-  private static class BufferListWritable implements Writable {
-    private List<ByteBuffer> buffers;
-
-    public BufferListWritable() {}                // required for RPC Writables
-
-    public BufferListWritable(List<ByteBuffer> buffers) {
-      this.buffers = buffers;
-    }
-
-    public void readFields(DataInput in) throws IOException {
-      int size = in.readInt();
-      buffers = new ArrayList<ByteBuffer>(size);
-      for (int i = 0; i < size; i++) {
-        int length = in.readInt();
-        ByteBuffer buffer = ByteBuffer.allocate(length);
-        in.readFully(buffer.array(), 0, length);
-        buffers.add(buffer);
-      }
-    }
-  
-    public void write(DataOutput out) throws IOException {
-      out.writeInt(buffers.size());
-      for (ByteBuffer buffer : buffers) {
-        out.writeInt(buffer.remaining());
-        out.write(buffer.array(), buffer.position(), buffer.remaining());
-      }
-    }
-  }
-
-  /** An Avro RPC Transceiver that tunnels client requests through Hadoop
-   * RPC. */
-  private static class ClientTransceiver extends Transceiver {
-    private TunnelProtocol tunnel;
-    private InetSocketAddress remote;
-  
-    public ClientTransceiver(InetSocketAddress addr,
-                             UserGroupInformation ticket,
-                             Configuration conf, SocketFactory factory,
-                             int rpcTimeout)
-      throws IOException {
-      this.tunnel = ENGINE.getProxy(TunnelProtocol.class, VERSION,
-                                        addr, ticket, conf, factory,
-                                        rpcTimeout).getProxy();
-      this.remote = addr;
-    }
-
-    public String getRemoteName() { return remote.toString(); }
-
-    public List<ByteBuffer> transceive(List<ByteBuffer> request)
-      throws IOException {
-      return tunnel.call(new BufferListWritable(request)).buffers;
-    }
-
-    public List<ByteBuffer> readBuffers() throws IOException {
-      throw new UnsupportedOperationException();
-    }
-
-    public void writeBuffers(List<ByteBuffer> buffers) throws IOException {
-      throw new UnsupportedOperationException();
-    }
-
-    public void close() throws IOException {
-      ENGINE.stopProxy(tunnel);
-    }
-  }
-
-  /** Construct a client-side proxy object that implements the named protocol,
-   * talking to a server at the named address. 
-   * @param <T>*/
-  @SuppressWarnings("unchecked")
-  public <T> ProtocolProxy<T> getProxy(Class<T> protocol, long clientVersion,
-                         InetSocketAddress addr, UserGroupInformation ticket,
-                         Configuration conf, SocketFactory factory,
-                         int rpcTimeout)
-    throws IOException {
-    return new ProtocolProxy<T>(protocol,
-       (T)Proxy.newProxyInstance(
-         protocol.getClassLoader(),
-         new Class[] { protocol },
-         new Invoker(protocol, addr, ticket, conf, factory, rpcTimeout)),
-       false);
-  }
-
-  /** Stop this proxy. */
-  public void stopProxy(Object proxy) {
-    try {
-      ((Invoker)Proxy.getInvocationHandler(proxy)).close();
-    } catch (IOException e) {
-      LOG.warn("Error while stopping "+proxy, e);
-    }
-  }
-
-  private class Invoker implements InvocationHandler, Closeable {
-    private final ClientTransceiver tx;
-    private final SpecificRequestor requestor;
-    public Invoker(Class<?> protocol, InetSocketAddress addr,
-                   UserGroupInformation ticket, Configuration conf,
-                   SocketFactory factory,
-                   int rpcTimeout) throws IOException {
-      this.tx = new ClientTransceiver(addr, ticket, conf, factory, rpcTimeout);
-      this.requestor = createRequestor(protocol, tx);
-    }
-    @Override public Object invoke(Object proxy, Method method, Object[] args) 
-      throws Throwable {
-      return requestor.invoke(proxy, method, args);
-    }
-    public void close() throws IOException {
-      tx.close();
-    }
-  }
-
-  protected SpecificRequestor createRequestor(Class<?> protocol, 
-      Transceiver transeiver) throws IOException {
-    return new ReflectRequestor(protocol, transeiver);
-  }
-
-  protected Responder createResponder(Class<?> iface, Object impl) {
-    return new ReflectResponder(iface, impl);
-  }
-
-  /** An Avro RPC Responder that can process requests passed via Hadoop RPC. */
-  private class TunnelResponder implements TunnelProtocol {
-    private Responder responder;
-    public TunnelResponder(Class<?> iface, Object impl) {
-      responder = createResponder(iface, impl);
-    }
-
-    @Override
-    public long getProtocolVersion(String protocol, long version)
-    throws IOException {
-      return VERSION;
-    }
-
-    @Override
-    public ProtocolSignature getProtocolSignature(
-        String protocol, long version, int clientMethodsHashCode)
-      throws IOException {
-      return new ProtocolSignature(VERSION, null);
-    }
-
-    public BufferListWritable call(final BufferListWritable request)
-      throws IOException {
-      return new BufferListWritable(responder.respond(request.buffers));
-    }
-  }
-
-  public Object[] call(Method method, Object[][] params,
-                       InetSocketAddress[] addrs, UserGroupInformation ticket,
-                       Configuration conf) throws IOException {
-    throw new UnsupportedOperationException();
-  }
-
-  /** Construct a server for a protocol implementation instance listening on a
-   * port and address. */
-  public RPC.Server getServer(Class<?> iface, Object impl, String bindAddress,
-                              int port, int numHandlers, int numReaders,
-                              int queueSizePerHandler, boolean verbose,
-                              Configuration conf, 
-                       SecretManager<? extends TokenIdentifier> secretManager
-                              ) throws IOException {
-    return ENGINE.getServer(TunnelProtocol.class,
-                            new TunnelResponder(iface, impl),
-                            bindAddress, port, numHandlers, numReaders,
-                            queueSizePerHandler, verbose, conf, secretManager);
-  }
-
-}
diff --git a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/ipc/AvroSpecificRpcEngine.java b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/ipc/AvroSpecificRpcEngine.java
deleted file mode 100644
index 995a13a..0000000
--- a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/ipc/AvroSpecificRpcEngine.java
+++ /dev/null
@@ -1,45 +0,0 @@
-/**
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- *     http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-
-package org.apache.hadoop.ipc;
-
-import java.io.IOException;
-
-import org.apache.avro.ipc.Responder;
-import org.apache.avro.ipc.Transceiver;
-import org.apache.avro.ipc.specific.SpecificRequestor;
-import org.apache.avro.ipc.specific.SpecificResponder;
-import org.apache.hadoop.classification.InterfaceStability;
-
-/**
- * AvroRpcEngine which uses Avro's "specific" APIs. The protocols generated 
- * via Avro IDL needs to use this Engine.
- */
-@InterfaceStability.Evolving
-public class AvroSpecificRpcEngine extends AvroRpcEngine {
-
-  protected SpecificRequestor createRequestor(Class<?> protocol, 
-      Transceiver transeiver) throws IOException {
-    return new SpecificRequestor(protocol, transeiver);
-  }
-
-  protected Responder createResponder(Class<?> iface, Object impl) {
-    return new SpecificResponder(iface, impl);
-  }
-
-}
diff --git a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/ipc/Client.java b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/ipc/Client.java
index e3a4baf..929b1e3 100644
--- a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/ipc/Client.java
+++ b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/ipc/Client.java
@@ -50,6 +50,7 @@
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.fs.CommonConfigurationKeys;
 import org.apache.hadoop.fs.CommonConfigurationKeysPublic;
+import org.apache.hadoop.ipc.RpcPayloadHeader.*;
 import org.apache.hadoop.io.IOUtils;
 import org.apache.hadoop.io.Text;
 import org.apache.hadoop.io.Writable;
@@ -153,16 +154,20 @@
     return refCount==0;
   }
 
-  /** A call waiting for a value. */
+  /** 
+   * Class that represents an RPC call
+   */
   private class Call {
-    int id;                                       // call id
-    Writable param;                               // parameter
-    Writable value;                               // value, null if error
-    IOException error;                            // exception, null if value
-    boolean done;                                 // true when call is done
+    final int id;               // call id
+    final Writable rpcRequest;  // the serialized rpc request - RpcPayload
+    Writable rpcResponse;       // null if rpc has error
+    IOException error;          // exception, null if success
+    final RpcKind rpcKind;      // Rpc EngineKind
+    boolean done;               // true when call is done
 
-    protected Call(Writable param) {
-      this.param = param;
+    protected Call(RpcKind rpcKind, Writable param) {
+      this.rpcKind = rpcKind;
+      this.rpcRequest = param;
       synchronized (Client.this) {
         this.id = counter++;
       }
@@ -188,15 +193,15 @@
     /** Set the return value when there is no error. 
      * Notify the caller the call is done.
      * 
-     * @param value return value of the call.
+     * @param rpcResponse return value of the rpc call.
      */
-    public synchronized void setValue(Writable value) {
-      this.value = value;
+    public synchronized void setRpcResponse(Writable rpcResponse) {
+      this.rpcResponse = rpcResponse;
       callComplete();
     }
     
-    public synchronized Writable getValue() {
-      return value;
+    public synchronized Writable getRpcResult() {
+      return rpcResponse;
     }
   }
 
@@ -287,8 +292,8 @@
         authMethod = AuthMethod.KERBEROS;
       }
       
-      header = new ConnectionHeader(protocol == null ? null : protocol
-          .getName(), ticket, authMethod);
+      header = 
+        new ConnectionHeader(RPC.getProtocolName(protocol), ticket, authMethod);
       
       if (LOG.isDebugEnabled())
         LOG.debug("Use " + authMethod + " authentication for protocol "
@@ -728,6 +733,7 @@
       }
     }
 
+    @SuppressWarnings("unused")
     public InetSocketAddress getRemoteAddress() {
       return server;
     }
@@ -789,8 +795,10 @@
           //data to be written
           d = new DataOutputBuffer();
           d.writeInt(0); // placeholder for data length
-          d.writeInt(call.id);
-          call.param.write(d);
+          RpcPayloadHeader header = new RpcPayloadHeader(
+              call.rpcKind, RpcPayloadOperation.RPC_FINAL_PAYLOAD, call.id);
+          header.write(d);
+          call.rpcRequest.write(d);
           byte[] data = d.getData();
           int dataLength = d.getLength() - 4;
           data[0] = (byte)((dataLength >>> 24) & 0xff);
@@ -830,7 +838,7 @@
         if (state == Status.SUCCESS.state) {
           Writable value = ReflectionUtils.newInstance(valueClass, conf);
           value.readFields(in);                 // read value
-          call.setValue(value);
+          call.setRpcResponse(value);
           calls.remove(id);
         } else if (state == Status.ERROR.state) {
           call.setException(new RemoteException(WritableUtils.readString(in),
@@ -914,7 +922,7 @@
     private int index;
     
     public ParallelCall(Writable param, ParallelResults results, int index) {
-      super(param);
+      super(RpcKind.RPC_WRITABLE, param);
       this.results = results;
       this.index = index;
     }
@@ -938,7 +946,7 @@
 
     /** Collect a result. */
     public synchronized void callComplete(ParallelCall call) {
-      values[call.index] = call.getValue();       // store the value
+      values[call.index] = call.getRpcResult();       // store the value
       count++;                                    // count it
       if (count == size)                          // if all values are in
         notify();                                 // then notify waiting caller
@@ -998,15 +1006,25 @@
     }
   }
 
+  /**
+   * Same as {@link #call(RpcPayloadHeader.RpcKind, Writable, ConnectionId)}
+   *  for RPC_BUILTIN
+   */
+  public Writable call(Writable param, InetSocketAddress address)
+  throws InterruptedException, IOException {
+    return call(RpcKind.RPC_BUILTIN, param, address);
+    
+  }
   /** Make a call, passing <code>param</code>, to the IPC server running at
    * <code>address</code>, returning the value.  Throws exceptions if there are
    * network problems or if the remote code threw an exception.
-   * @deprecated Use {@link #call(Writable, ConnectionId)} instead 
+   * @deprecated Use {@link #call(RpcPayloadHeader.RpcKind, Writable,
+   *  ConnectionId)} instead 
    */
   @Deprecated
-  public Writable call(Writable param, InetSocketAddress address)
+  public Writable call(RpcKind rpcKind, Writable param, InetSocketAddress address)
   throws InterruptedException, IOException {
-      return call(param, address, null);
+      return call(rpcKind, param, address, null);
   }
   
   /** Make a call, passing <code>param</code>, to the IPC server running at
@@ -1014,15 +1032,16 @@
    * the value.  
    * Throws exceptions if there are network problems or if the remote code 
    * threw an exception.
-   * @deprecated Use {@link #call(Writable, ConnectionId)} instead 
+   * @deprecated Use {@link #call(RpcPayloadHeader.RpcKind, Writable, 
+   * ConnectionId)} instead 
    */
   @Deprecated
-  public Writable call(Writable param, InetSocketAddress addr, 
+  public Writable call(RpcKind rpcKind, Writable param, InetSocketAddress addr, 
       UserGroupInformation ticket)  
       throws InterruptedException, IOException {
     ConnectionId remoteId = ConnectionId.getConnectionId(addr, null, ticket, 0,
         conf);
-    return call(param, remoteId);
+    return call(rpcKind, param, remoteId);
   }
   
   /** Make a call, passing <code>param</code>, to the IPC server running at
@@ -1031,18 +1050,34 @@
    * timeout, returning the value.  
    * Throws exceptions if there are network problems or if the remote code 
    * threw an exception. 
-   * @deprecated Use {@link #call(Writable, ConnectionId)} instead 
+   * @deprecated Use {@link #call(RpcPayloadHeader.RpcKind, Writable,
+   *  ConnectionId)} instead 
    */
   @Deprecated
-  public Writable call(Writable param, InetSocketAddress addr, 
+  public Writable call(RpcKind rpcKind, Writable param, InetSocketAddress addr, 
                        Class<?> protocol, UserGroupInformation ticket,
                        int rpcTimeout)  
                        throws InterruptedException, IOException {
     ConnectionId remoteId = ConnectionId.getConnectionId(addr, protocol,
         ticket, rpcTimeout, conf);
-    return call(param, remoteId);
+    return call(rpcKind, param, remoteId);
   }
 
+  
+  /**
+   * Same as {@link #call(RpcPayloadHeader.RpcKind, Writable, InetSocketAddress, 
+   * Class, UserGroupInformation, int, Configuration)}
+   * except that rpcKind is writable.
+   */
+  public Writable call(Writable param, InetSocketAddress addr, 
+      Class<?> protocol, UserGroupInformation ticket,
+      int rpcTimeout, Configuration conf)  
+      throws InterruptedException, IOException {
+        ConnectionId remoteId = ConnectionId.getConnectionId(addr, protocol,
+        ticket, rpcTimeout, conf);
+    return call(RpcKind.RPC_BUILTIN, param, remoteId);
+  }
+  
   /**
    * Make a call, passing <code>param</code>, to the IPC server running at
    * <code>address</code> which is servicing the <code>protocol</code> protocol,
@@ -1051,22 +1086,38 @@
    * value. Throws exceptions if there are network problems or if the remote
    * code threw an exception.
    */
-  public Writable call(Writable param, InetSocketAddress addr, 
+  public Writable call(RpcKind rpcKind, Writable param, InetSocketAddress addr, 
                        Class<?> protocol, UserGroupInformation ticket,
                        int rpcTimeout, Configuration conf)  
                        throws InterruptedException, IOException {
     ConnectionId remoteId = ConnectionId.getConnectionId(addr, protocol,
         ticket, rpcTimeout, conf);
-    return call(param, remoteId);
+    return call(rpcKind, param, remoteId);
   }
   
-  /** Make a call, passing <code>param</code>, to the IPC server defined by
-   * <code>remoteId</code>, returning the value.  
-   * Throws exceptions if there are network problems or if the remote code 
-   * threw an exception. */
+  /**
+   * Same as {link {@link #call(RpcPayloadHeader.RpcKind, Writable, ConnectionId)}
+   * except the rpcKind is RPC_BUILTIN
+   */
   public Writable call(Writable param, ConnectionId remoteId)  
       throws InterruptedException, IOException {
-    Call call = new Call(param);
+     return call(RpcKind.RPC_BUILTIN, param, remoteId);
+  }
+  
+  /** 
+   * Make a call, passing <code>rpcRequest</code>, to the IPC server defined by
+   * <code>remoteId</code>, returning the rpc respond.
+   * 
+   * @param rpcKind
+   * @param rpcRequest -  contains serialized method and method parameters
+   * @param remoteId - the target rpc server
+   * @returns the rpc response
+   * Throws exceptions if there are network problems or if the remote code 
+   * threw an exception.
+   */
+  public Writable call(RpcKind rpcKind, Writable rpcRequest,
+      ConnectionId remoteId) throws InterruptedException, IOException {
+    Call call = new Call(rpcKind, rpcRequest);
     Connection connection = getConnection(remoteId, call);
     connection.sendParam(call);                 // send the parameter
     boolean interrupted = false;
@@ -1098,7 +1149,7 @@
                   call.error);
         }
       } else {
-        return call.value;
+        return call.rpcResponse;
       }
     }
   }
diff --git a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/ipc/ProtobufHelper.java b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/ipc/ProtobufHelper.java
new file mode 100644
index 0000000..e30f28a
--- /dev/null
+++ b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/ipc/ProtobufHelper.java
@@ -0,0 +1,49 @@
+/**
+ * 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.ipc;
+
+import java.io.IOException;
+
+import org.apache.hadoop.classification.InterfaceAudience;
+
+import com.google.protobuf.ServiceException;
+
+/**
+ * Helper methods for protobuf related RPC implementation
+ */
+@InterfaceAudience.Private
+public class ProtobufHelper {
+  private ProtobufHelper() {
+    // Hidden constructor for class with only static helper methods
+  }
+
+  /**
+   * Return the IOException thrown by the remote server wrapped in 
+   * ServiceException as cause.
+   * @param se ServiceException that wraps IO exception thrown by the server
+   * @return Exception wrapped in ServiceException or
+   *         a new IOException that wraps the unexpected ServiceException.
+   */
+  public static IOException getRemoteException(ServiceException se) {
+    Throwable e = se.getCause();
+    if (e == null) {
+      return new IOException(se);
+    }
+    return e instanceof IOException ? (IOException) e : new IOException(se);
+  }
+}
diff --git a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/ipc/ProtobufRpcEngine.java b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/ipc/ProtobufRpcEngine.java
new file mode 100644
index 0000000..ff2c1a4a
--- /dev/null
+++ b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/ipc/ProtobufRpcEngine.java
@@ -0,0 +1,471 @@
+/**
+ * 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.ipc;
+
+import java.io.DataInput;
+import java.io.DataOutput;
+import java.io.IOException;
+import java.lang.reflect.Method;
+import java.lang.reflect.Proxy;
+import java.net.InetSocketAddress;
+import java.util.Map;
+import java.util.concurrent.ConcurrentHashMap;
+
+import javax.net.SocketFactory;
+
+import org.apache.commons.logging.Log;
+import org.apache.commons.logging.LogFactory;
+import org.apache.hadoop.classification.InterfaceAudience;
+import org.apache.hadoop.classification.InterfaceStability;
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.io.DataOutputOutputStream;
+import org.apache.hadoop.io.Writable;
+import org.apache.hadoop.ipc.Client.ConnectionId;
+import org.apache.hadoop.ipc.RPC.RpcInvoker;
+import org.apache.hadoop.ipc.RpcPayloadHeader.RpcKind;
+import org.apache.hadoop.ipc.protobuf.HadoopRpcProtos.HadoopRpcExceptionProto;
+import org.apache.hadoop.ipc.protobuf.HadoopRpcProtos.HadoopRpcRequestProto;
+import org.apache.hadoop.ipc.protobuf.HadoopRpcProtos.HadoopRpcResponseProto;
+import org.apache.hadoop.ipc.protobuf.HadoopRpcProtos.HadoopRpcResponseProto.ResponseStatus;
+import org.apache.hadoop.security.UserGroupInformation;
+import org.apache.hadoop.security.token.SecretManager;
+import org.apache.hadoop.security.token.TokenIdentifier;
+import org.apache.hadoop.util.ProtoUtil;
+import org.apache.hadoop.util.StringUtils;
+
+import com.google.common.annotations.VisibleForTesting;
+import com.google.protobuf.BlockingService;
+import com.google.protobuf.Descriptors.MethodDescriptor;
+import com.google.protobuf.Message;
+import com.google.protobuf.ServiceException;
+
+/**
+ * RPC Engine for for protobuf based RPCs.
+ */
+@InterfaceStability.Evolving
+public class ProtobufRpcEngine implements RpcEngine {
+  private static final Log LOG = LogFactory.getLog(ProtobufRpcEngine.class);
+  
+  static { // Register the rpcRequest deserializer for WritableRpcEngine 
+    org.apache.hadoop.ipc.Server.registerProtocolEngine(
+        RpcKind.RPC_PROTOCOL_BUFFER, RpcRequestWritable.class,
+        new Server.ProtoBufRpcInvoker());
+  }
+
+  private static final ClientCache CLIENTS = new ClientCache();
+
+  @Override
+  @SuppressWarnings("unchecked")
+  public <T> ProtocolProxy<T> getProxy(Class<T> protocol, long clientVersion,
+      InetSocketAddress addr, UserGroupInformation ticket, Configuration conf,
+      SocketFactory factory, int rpcTimeout) throws IOException {
+
+    return new ProtocolProxy<T>(protocol, (T) Proxy.newProxyInstance(protocol
+        .getClassLoader(), new Class[] { protocol }, new Invoker(protocol,
+        addr, ticket, conf, factory, rpcTimeout)), false);
+  }
+  
+  @Override
+  public ProtocolProxy<ProtocolMetaInfoPB> getProtocolMetaInfoProxy(
+      ConnectionId connId, Configuration conf, SocketFactory factory)
+      throws IOException {
+    Class<ProtocolMetaInfoPB> protocol = ProtocolMetaInfoPB.class;
+    return new ProtocolProxy<ProtocolMetaInfoPB>(protocol,
+        (ProtocolMetaInfoPB) Proxy.newProxyInstance(protocol.getClassLoader(),
+            new Class[] { protocol }, new Invoker(protocol, connId, conf,
+                factory)), false);
+  }
+
+  private static class Invoker implements RpcInvocationHandler {
+    private final Map<String, Message> returnTypes = 
+        new ConcurrentHashMap<String, Message>();
+    private boolean isClosed = false;
+    private final Client.ConnectionId remoteId;
+    private final Client client;
+    private final long clientProtocolVersion;
+    private final String protocolName;
+
+    public Invoker(Class<?> protocol, InetSocketAddress addr,
+        UserGroupInformation ticket, Configuration conf, SocketFactory factory,
+        int rpcTimeout) throws IOException {
+      this(protocol, Client.ConnectionId.getConnectionId(addr, protocol,
+          ticket, rpcTimeout, conf), conf, factory);
+    }
+    
+    /**
+     * This constructor takes a connectionId, instead of creating a new one.
+     */
+    public Invoker(Class<?> protocol, Client.ConnectionId connId,
+        Configuration conf, SocketFactory factory) {
+      this.remoteId = connId;
+      this.client = CLIENTS.getClient(conf, factory, RpcResponseWritable.class);
+      this.protocolName = RPC.getProtocolName(protocol);
+      this.clientProtocolVersion = RPC
+          .getProtocolVersion(protocol);
+    }
+
+    private HadoopRpcRequestProto constructRpcRequest(Method method,
+        Object[] params) throws ServiceException {
+      HadoopRpcRequestProto rpcRequest;
+      HadoopRpcRequestProto.Builder builder = HadoopRpcRequestProto
+          .newBuilder();
+      builder.setMethodName(method.getName());
+
+      if (params.length != 2) { // RpcController + Message
+        throw new ServiceException("Too many parameters for request. Method: ["
+            + method.getName() + "]" + ", Expected: 2, Actual: "
+            + params.length);
+      }
+      if (params[1] == null) {
+        throw new ServiceException("null param while calling Method: ["
+            + method.getName() + "]");
+      }
+
+      Message param = (Message) params[1];
+      builder.setRequest(param.toByteString());
+      // For protobuf, {@code protocol} used when creating client side proxy is
+      // the interface extending BlockingInterface, which has the annotations 
+      // such as ProtocolName etc.
+      //
+      // Using Method.getDeclaringClass(), as in WritableEngine to get at
+      // the protocol interface will return BlockingInterface, from where 
+      // the annotation ProtocolName and Version cannot be
+      // obtained.
+      //
+      // Hence we simply use the protocol class used to create the proxy.
+      // For PB this may limit the use of mixins on client side.
+      builder.setDeclaringClassProtocolName(protocolName);
+      builder.setClientProtocolVersion(clientProtocolVersion);
+      rpcRequest = builder.build();
+      return rpcRequest;
+    }
+
+    /**
+     * This is the client side invoker of RPC method. It only throws
+     * ServiceException, since the invocation proxy expects only
+     * ServiceException to be thrown by the method in case protobuf service.
+     * 
+     * ServiceException has the following causes:
+     * <ol>
+     * <li>Exceptions encountered on the client side in this method are 
+     * set as cause in ServiceException as is.</li>
+     * <li>Exceptions from the server are wrapped in RemoteException and are
+     * set as cause in ServiceException</li>
+     * </ol>
+     * 
+     * Note that the client calling protobuf RPC methods, must handle
+     * ServiceException by getting the cause from the ServiceException. If the
+     * cause is RemoteException, then unwrap it to get the exception thrown by
+     * the server.
+     */
+    @Override
+    public Object invoke(Object proxy, Method method, Object[] args)
+        throws ServiceException {
+      long startTime = 0;
+      if (LOG.isDebugEnabled()) {
+        startTime = System.currentTimeMillis();
+      }
+
+      HadoopRpcRequestProto rpcRequest = constructRpcRequest(method, args);
+      RpcResponseWritable val = null;
+      try {
+        val = (RpcResponseWritable) client.call(RpcKind.RPC_PROTOCOL_BUFFER,
+            new RpcRequestWritable(rpcRequest), remoteId);
+      } catch (Throwable e) {
+        throw new ServiceException(e);
+      }
+
+      HadoopRpcResponseProto response = val.message;
+      if (LOG.isDebugEnabled()) {
+        long callTime = System.currentTimeMillis() - startTime;
+        LOG.debug("Call: " + method.getName() + " " + callTime);
+      }
+
+      // Wrap the received message
+      ResponseStatus status = response.getStatus();
+      if (status != ResponseStatus.SUCCESS) {
+        RemoteException re =  new RemoteException(response.getException()
+            .getExceptionName(), response.getException().getStackTrace());
+        re.fillInStackTrace();
+        throw new ServiceException(re);
+      }
+
+      Message prototype = null;
+      try {
+        prototype = getReturnProtoType(method);
+      } catch (Exception e) {
+        throw new ServiceException(e);
+      }
+      Message returnMessage;
+      try {
+        returnMessage = prototype.newBuilderForType()
+            .mergeFrom(response.getResponse()).build();
+      } catch (Throwable e) {
+        throw new ServiceException(e);
+      }
+      return returnMessage;
+    }
+
+    public void close() throws IOException {
+      if (!isClosed) {
+        isClosed = true;
+        CLIENTS.stopClient(client);
+      }
+    }
+
+    private Message getReturnProtoType(Method method) throws Exception {
+      if (returnTypes.containsKey(method.getName())) {
+        return returnTypes.get(method.getName());
+      }
+      
+      Class<?> returnType = method.getReturnType();
+      Method newInstMethod = returnType.getMethod("getDefaultInstance");
+      newInstMethod.setAccessible(true);
+      Message prototype = (Message) newInstMethod.invoke(null, (Object[]) null);
+      returnTypes.put(method.getName(), prototype);
+      return prototype;
+    }
+
+    @Override //RpcInvocationHandler
+    public ConnectionId getConnectionId() {
+      return remoteId;
+    }
+  }
+
+  @Override
+  public Object[] call(Method method, Object[][] params,
+      InetSocketAddress[] addrs, UserGroupInformation ticket, Configuration conf) {
+    throw new UnsupportedOperationException();
+  }
+
+  /**
+   * Writable Wrapper for Protocol Buffer Requests
+   */
+  private static class RpcRequestWritable implements Writable {
+    HadoopRpcRequestProto message;
+
+    @SuppressWarnings("unused")
+    public RpcRequestWritable() {
+    }
+
+    RpcRequestWritable(HadoopRpcRequestProto message) {
+      this.message = message;
+    }
+
+    @Override
+    public void write(DataOutput out) throws IOException {
+      ((Message)message).writeDelimitedTo(
+          DataOutputOutputStream.constructOutputStream(out));
+    }
+
+    @Override
+    public void readFields(DataInput in) throws IOException {
+      int length = ProtoUtil.readRawVarint32(in);
+      byte[] bytes = new byte[length];
+      in.readFully(bytes);
+      message = HadoopRpcRequestProto.parseFrom(bytes);
+    }
+  }
+
+  /**
+   * Writable Wrapper for Protocol Buffer Responses
+   */
+  private static class RpcResponseWritable implements Writable {
+    HadoopRpcResponseProto message;
+
+    @SuppressWarnings("unused")
+    public RpcResponseWritable() {
+    }
+
+    public RpcResponseWritable(HadoopRpcResponseProto message) {
+      this.message = message;
+    }
+
+    @Override
+    public void write(DataOutput out) throws IOException {
+      ((Message)message).writeDelimitedTo(
+          DataOutputOutputStream.constructOutputStream(out));      
+    }
+
+    @Override
+    public void readFields(DataInput in) throws IOException {
+      int length = ProtoUtil.readRawVarint32(in);
+      byte[] bytes = new byte[length];
+      in.readFully(bytes);
+      message = HadoopRpcResponseProto.parseFrom(bytes);
+    }
+  }
+
+  @VisibleForTesting
+  @InterfaceAudience.Private
+  @InterfaceStability.Unstable
+  static Client getClient(Configuration conf) {
+    return CLIENTS.getClient(conf, SocketFactory.getDefault(),
+        RpcResponseWritable.class);
+  }
+  
+ 
+
+  @Override
+  public RPC.Server getServer(Class<?> protocol, Object protocolImpl,
+      String bindAddress, int port, int numHandlers, int numReaders,
+      int queueSizePerHandler, boolean verbose, Configuration conf,
+      SecretManager<? extends TokenIdentifier> secretManager)
+      throws IOException {
+    return new Server(protocol, protocolImpl, conf, bindAddress, port,
+        numHandlers, numReaders, queueSizePerHandler, verbose, secretManager);
+  }
+  
+  public static class Server extends RPC.Server {
+    /**
+     * Construct an RPC server.
+     * 
+     * @param protocolClass the class of protocol
+     * @param protocolImpl the protocolImpl whose methods will be called
+     * @param conf the configuration to use
+     * @param bindAddress the address to bind on to listen for connection
+     * @param port the port to listen for connections on
+     * @param numHandlers the number of method handler threads to run
+     * @param verbose whether each call should be logged
+     */
+    public Server(Class<?> protocolClass, Object protocolImpl,
+        Configuration conf, String bindAddress, int port, int numHandlers,
+        int numReaders, int queueSizePerHandler, boolean verbose,
+        SecretManager<? extends TokenIdentifier> secretManager)
+        throws IOException {
+      super(bindAddress, port, null, numHandlers,
+          numReaders, queueSizePerHandler, conf, classNameBase(protocolImpl
+              .getClass().getName()), secretManager);
+      this.verbose = verbose;  
+      registerProtocolAndImpl(RpcKind.RPC_PROTOCOL_BUFFER, protocolClass,
+          protocolImpl);
+    }
+
+    private static RpcResponseWritable handleException(Throwable e) {
+      HadoopRpcExceptionProto exception = HadoopRpcExceptionProto.newBuilder()
+          .setExceptionName(e.getClass().getName())
+          .setStackTrace(StringUtils.stringifyException(e)).build();
+      HadoopRpcResponseProto response = HadoopRpcResponseProto.newBuilder()
+          .setStatus(ResponseStatus.ERRROR).setException(exception).build();
+      return new RpcResponseWritable(response);
+    }
+
+    private static HadoopRpcResponseProto constructProtoSpecificRpcSuccessResponse(
+        Message message) {
+      HadoopRpcResponseProto res = HadoopRpcResponseProto.newBuilder()
+          .setResponse(message.toByteString())
+          .setStatus(ResponseStatus.SUCCESS)
+          .build();
+      return res;
+    }
+    
+    /**
+     * Protobuf invoker for {@link RpcInvoker}
+     */
+    static class ProtoBufRpcInvoker implements RpcInvoker {
+      private static ProtoClassProtoImpl getProtocolImpl(RPC.Server server,
+          String protoName, long version) throws IOException {
+        ProtoNameVer pv = new ProtoNameVer(protoName, version);
+        ProtoClassProtoImpl impl = 
+            server.getProtocolImplMap(RpcKind.RPC_PROTOCOL_BUFFER).get(pv);
+        if (impl == null) { // no match for Protocol AND Version
+          VerProtocolImpl highest = 
+              server.getHighestSupportedProtocol(RpcKind.RPC_PROTOCOL_BUFFER, 
+                  protoName);
+          if (highest == null) {
+            throw new IOException("Unknown protocol: " + protoName);
+          }
+          // protocol supported but not the version that client wants
+          throw new RPC.VersionMismatch(protoName, version,
+              highest.version);
+        }
+        return impl;
+      }
+
+      @Override 
+      /**
+       * This is a server side method, which is invoked over RPC. On success
+       * the return response has protobuf response payload. On failure, the
+       * exception name and the stack trace are return in the resposne.
+       * See {@link HadoopRpcResponseProto}
+       * 
+       * In this method there three types of exceptions possible and they are
+       * returned in response as follows.
+       * <ol>
+       * <li> Exceptions encountered in this method that are returned 
+       * as {@link RpcServerException} </li>
+       * <li> Exceptions thrown by the service is wrapped in ServiceException. 
+       * In that this method returns in response the exception thrown by the 
+       * service.</li>
+       * <li> Other exceptions thrown by the service. They are returned as
+       * it is.</li>
+       * </ol>
+       */
+      public Writable call(RPC.Server server, String protocol,
+          Writable writableRequest, long receiveTime) throws IOException {
+        RpcRequestWritable request = (RpcRequestWritable) writableRequest;
+        HadoopRpcRequestProto rpcRequest = request.message;
+        String methodName = rpcRequest.getMethodName();
+        String protoName = rpcRequest.getDeclaringClassProtocolName();
+        long clientVersion = rpcRequest.getClientProtocolVersion();
+        if (server.verbose)
+          LOG.info("Call: protocol=" + protocol + ", method=" + methodName);
+        
+        ProtoClassProtoImpl protocolImpl = getProtocolImpl(server, protoName,
+            clientVersion);
+        BlockingService service = (BlockingService) protocolImpl.protocolImpl;
+        MethodDescriptor methodDescriptor = service.getDescriptorForType()
+            .findMethodByName(methodName);
+        if (methodDescriptor == null) {
+          String msg = "Unknown method " + methodName + " called on " + protocol
+              + " protocol.";
+          LOG.warn(msg);
+          return handleException(new RpcServerException(msg));
+        }
+        Message prototype = service.getRequestPrototype(methodDescriptor);
+        Message param = prototype.newBuilderForType()
+            .mergeFrom(rpcRequest.getRequest()).build();
+        Message result;
+        try {
+          long startTime = System.currentTimeMillis();
+          server.rpcDetailedMetrics.init(protocolImpl.protocolClass);
+          result = service.callBlockingMethod(methodDescriptor, null, param);
+          int processingTime = (int) (System.currentTimeMillis() - startTime);
+          int qTime = (int) (startTime - receiveTime);
+          if (LOG.isDebugEnabled()) {
+            LOG.info("Served: " + methodName + " queueTime= " + qTime +
+                      " procesingTime= " + processingTime);
+          }
+          server.rpcMetrics.addRpcQueueTime(qTime);
+          server.rpcMetrics.addRpcProcessingTime(processingTime);
+          server.rpcDetailedMetrics.addProcessingTime(methodName,
+              processingTime);
+        } catch (ServiceException e) {
+          Throwable cause = e.getCause();
+          return handleException(cause != null ? cause : e);
+        } catch (Exception e) {
+          return handleException(e);
+        }
+  
+        HadoopRpcResponseProto response = constructProtoSpecificRpcSuccessResponse(result);
+        return new RpcResponseWritable(response);
+      }
+    }
+  }
+}
diff --git a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/ipc/ProtocolInfo.java b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/ipc/ProtocolInfo.java
new file mode 100644
index 0000000..2bdd183
--- /dev/null
+++ b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/ipc/ProtocolInfo.java
@@ -0,0 +1,39 @@
+/**
+ * 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.ipc;
+
+import java.lang.annotation.Retention;
+import java.lang.annotation.RetentionPolicy;
+
+
+/**
+ * The protocol name that is used when a client and server connect.
+ * By default the class name of the protocol interface is the protocol name.
+ * 
+ * Why override the default name (i.e. the class name)?
+ * One use case overriding the default name (i.e. the class name) is when
+ * there are multiple implementations of the same protocol, each with say a
+ *  different version/serialization.
+ * In Hadoop this is used to allow multiple server and client adapters
+ * for different versions of the same protocol service.
+ */
+@Retention(RetentionPolicy.RUNTIME)
+public @interface ProtocolInfo {
+  String protocolName();  // the name of the protocol (i.e. rpc service)
+  long protocolVersion() default -1; // default means not defined use old way
+}
diff --git a/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/ipc/AvroTestProtocol.java b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/ipc/ProtocolMetaInfoPB.java
similarity index 64%
copy from hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/ipc/AvroTestProtocol.java
copy to hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/ipc/ProtocolMetaInfoPB.java
index d5d7396..968f3d0 100644
--- a/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/ipc/AvroTestProtocol.java
+++ b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/ipc/ProtocolMetaInfoPB.java
@@ -15,18 +15,20 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
-
 package org.apache.hadoop.ipc;
 
-import org.apache.avro.AvroRemoteException;
+import org.apache.hadoop.ipc.protobuf.ProtocolInfoProtos.ProtocolInfoService;
 
-@SuppressWarnings("serial")
-public interface AvroTestProtocol {
-  public static class Problem extends AvroRemoteException {
-    public Problem() {}
-  }
-  void ping();
-  String echo(String value);
-  int add(int v1, int v2);
-  int error() throws Problem;
+/**
+ * Protocol to get versions and signatures for supported protocols from the
+ * server.
+ * 
+ * Note: This extends the protocolbuffer service based interface to
+ * add annotations.
+ */
+@ProtocolInfo(
+    protocolName = "org.apache.hadoop.ipc.ProtocolMetaInfoPB", 
+    protocolVersion = 1)
+public interface ProtocolMetaInfoPB extends
+    ProtocolInfoService.BlockingInterface {
 }
diff --git a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/ipc/ProtocolMetaInfoServerSideTranslatorPB.java b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/ipc/ProtocolMetaInfoServerSideTranslatorPB.java
new file mode 100644
index 0000000..aaf71f8
--- /dev/null
+++ b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/ipc/ProtocolMetaInfoServerSideTranslatorPB.java
@@ -0,0 +1,122 @@
+/**
+ * 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.ipc;
+
+import org.apache.hadoop.ipc.RPC.Server.VerProtocolImpl;
+import org.apache.hadoop.ipc.RpcPayloadHeader.RpcKind;
+import org.apache.hadoop.ipc.protobuf.ProtocolInfoProtos.GetProtocolSignatureRequestProto;
+import org.apache.hadoop.ipc.protobuf.ProtocolInfoProtos.GetProtocolSignatureResponseProto;
+import org.apache.hadoop.ipc.protobuf.ProtocolInfoProtos.GetProtocolVersionsRequestProto;
+import org.apache.hadoop.ipc.protobuf.ProtocolInfoProtos.GetProtocolVersionsResponseProto;
+import org.apache.hadoop.ipc.protobuf.ProtocolInfoProtos.ProtocolSignatureProto;
+import org.apache.hadoop.ipc.protobuf.ProtocolInfoProtos.ProtocolVersionProto;
+
+import com.google.protobuf.RpcController;
+import com.google.protobuf.ServiceException;
+
+/**
+ * This class serves the requests for protocol versions and signatures by
+ * looking them up in the server registry.
+ */
+public class ProtocolMetaInfoServerSideTranslatorPB implements
+    ProtocolMetaInfoPB {
+
+  RPC.Server server;
+  
+  public ProtocolMetaInfoServerSideTranslatorPB(RPC.Server server) {
+    this.server = server;
+  }
+  
+  @Override
+  public GetProtocolVersionsResponseProto getProtocolVersions(
+      RpcController controller, GetProtocolVersionsRequestProto request)
+      throws ServiceException {
+    String protocol = request.getProtocol();
+    GetProtocolVersionsResponseProto.Builder builder = 
+        GetProtocolVersionsResponseProto.newBuilder();
+    for (RpcKind r : RpcKind.values()) {
+      long[] versions;
+      try {
+        versions = getProtocolVersionForRpcKind(r, protocol);
+      } catch (ClassNotFoundException e) {
+        throw new ServiceException(e);
+      }
+      ProtocolVersionProto.Builder b = ProtocolVersionProto.newBuilder();
+      if (versions != null) {
+        b.setRpcKind(r.toString());
+        for (long v : versions) {
+          b.addVersions(v);
+        }
+      }
+      builder.addProtocolVersions(b.build());
+    }
+    return builder.build();
+  }
+
+  @Override
+  public GetProtocolSignatureResponseProto getProtocolSignature(
+      RpcController controller, GetProtocolSignatureRequestProto request)
+      throws ServiceException {
+    GetProtocolSignatureResponseProto.Builder builder = GetProtocolSignatureResponseProto
+        .newBuilder();
+    String protocol = request.getProtocol();
+    String rpcKind = request.getRpcKind();
+    long[] versions;
+    try {
+      versions = getProtocolVersionForRpcKind(RpcKind.valueOf(rpcKind),
+          protocol);
+    } catch (ClassNotFoundException e1) {
+      throw new ServiceException(e1);
+    }
+    if (versions == null) {
+      return builder.build();
+    }
+    for (long v : versions) {
+      ProtocolSignatureProto.Builder sigBuilder = ProtocolSignatureProto
+          .newBuilder();
+      sigBuilder.setVersion(v);
+      try {
+        ProtocolSignature signature = ProtocolSignature.getProtocolSignature(
+            protocol, v);
+        for (int m : signature.getMethods()) {
+          sigBuilder.addMethods(m);
+        }
+      } catch (ClassNotFoundException e) {
+        throw new ServiceException(e);
+      }
+      builder.addProtocolSignature(sigBuilder.build());
+    }
+    return builder.build();
+  }
+  
+  private long[] getProtocolVersionForRpcKind(RpcKind rpcKind,
+      String protocol) throws ClassNotFoundException {
+    Class<?> protocolClass = Class.forName(protocol);
+    String protocolName = RPC.getProtocolName(protocolClass);
+    VerProtocolImpl[] vers = server.getSupportedProtocolVersions(rpcKind,
+        protocolName);
+    if (vers == null) {
+      return null;
+    }
+    long [] versions = new long[vers.length];
+    for (int i=0; i<versions.length; i++) {
+      versions[i] = vers[i].version;
+    }
+    return versions;
+  }
+}
diff --git a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/ipc/ProtocolMetaInterface.java b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/ipc/ProtocolMetaInterface.java
new file mode 100644
index 0000000..29c07ac
--- /dev/null
+++ b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/ipc/ProtocolMetaInterface.java
@@ -0,0 +1,42 @@
+/**
+ * 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.ipc;
+
+import java.io.IOException;
+
+import org.apache.hadoop.classification.InterfaceAudience;
+import org.apache.hadoop.classification.InterfaceStability;
+
+/**
+ * This interface is implemented by the client side translators and can be used
+ * to obtain information about underlying protocol e.g. to check if a method is
+ * supported on the server side.
+ */
+@InterfaceAudience.Private
+@InterfaceStability.Stable
+public interface ProtocolMetaInterface {
+  
+  /**
+   * Checks whether the given method name is supported by the server.
+   * It is assumed that all method names are unique for a protocol.
+   * @param methodName The name of the method
+   * @return true if method is supported, otherwise false.
+   * @throws IOException
+   */
+  public boolean isMethodSupported(String methodName) throws IOException;
+}
diff --git a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/ipc/ProtocolProxy.java b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/ipc/ProtocolProxy.java
index 937031c..f15c083 100644
--- a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/ipc/ProtocolProxy.java
+++ b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/ipc/ProtocolProxy.java
@@ -57,19 +57,11 @@
   
   private void fetchServerMethods(Method method) throws IOException {
     long clientVersion;
-    try {
-      Field versionField = method.getDeclaringClass().getField("versionID");
-      versionField.setAccessible(true);
-      clientVersion = versionField.getLong(method.getDeclaringClass());
-    } catch (NoSuchFieldException ex) {
-      throw new RuntimeException(ex);
-    } catch (IllegalAccessException ex) {
-      throw new RuntimeException(ex);
-    }
+    clientVersion = RPC.getProtocolVersion(method.getDeclaringClass());
     int clientMethodsHash = ProtocolSignature.getFingerprint(method
         .getDeclaringClass().getMethods());
     ProtocolSignature serverInfo = ((VersionedProtocol) proxy)
-        .getProtocolSignature(protocol.getName(), clientVersion,
+        .getProtocolSignature(RPC.getProtocolName(protocol), clientVersion,
             clientMethodsHash);
     long serverVersion = serverInfo.getVersion();
     if (serverVersion != clientVersion) {
diff --git a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/ipc/ProtocolSignature.java b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/ipc/ProtocolSignature.java
index a055a7f..1cc269f 100644
--- a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/ipc/ProtocolSignature.java
+++ b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/ipc/ProtocolSignature.java
@@ -29,6 +29,8 @@
 import org.apache.hadoop.io.WritableFactories;
 import org.apache.hadoop.io.WritableFactory;
 
+import com.google.common.annotations.VisibleForTesting;
+
 public class ProtocolSignature implements Writable {
   static {               // register a ctor
     WritableFactories.setFactory
@@ -164,10 +166,15 @@
   /**
    * A cache that maps a protocol's name to its signature & finger print
    */
-  final private static HashMap<String, ProtocolSigFingerprint> 
+  private final static HashMap<String, ProtocolSigFingerprint> 
      PROTOCOL_FINGERPRINT_CACHE = 
        new HashMap<String, ProtocolSigFingerprint>();
   
+  @VisibleForTesting
+  public static void resetCache() {
+    PROTOCOL_FINGERPRINT_CACHE.clear();
+  }
+  
   /**
    * Return a protocol's signature and finger print from cache
    * 
@@ -176,8 +183,8 @@
    * @return its signature and finger print
    */
   private static ProtocolSigFingerprint getSigFingerprint(
-      Class <? extends VersionedProtocol> protocol, long serverVersion) {
-    String protocolName = protocol.getName();
+      Class <?> protocol, long serverVersion) {
+    String protocolName = RPC.getProtocolName(protocol);
     synchronized (PROTOCOL_FINGERPRINT_CACHE) {
       ProtocolSigFingerprint sig = PROTOCOL_FINGERPRINT_CACHE.get(protocolName);
       if (sig == null) {
@@ -199,7 +206,7 @@
    * @param protocol protocol
    * @return the server's protocol signature
    */
-  static ProtocolSignature getProtocolSignature(
+  public static ProtocolSignature getProtocolSignature(
       int clientMethodsHashCode,
       long serverVersion,
       Class<? extends VersionedProtocol> protocol) {
@@ -214,6 +221,12 @@
     return sig.signature;
   }
   
+  public static ProtocolSignature getProtocolSignature(String protocolName,
+      long version) throws ClassNotFoundException {
+    Class<?> protocol = Class.forName(protocolName);
+    return getSigFingerprint(protocol, version).signature;
+  }
+  
   /**
    * Get a server protocol's signature
    *
diff --git a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/ipc/RPC.java b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/ipc/RPC.java
index b42b913..4f85e90 100644
--- a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/ipc/RPC.java
+++ b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/ipc/RPC.java
@@ -18,6 +18,8 @@
 
 package org.apache.hadoop.ipc;
 
+import java.lang.reflect.Field;
+import java.lang.reflect.InvocationHandler;
 import java.lang.reflect.Proxy;
 import java.lang.reflect.Method;
 
@@ -26,6 +28,10 @@
 import java.net.NoRouteToHostException;
 import java.net.SocketTimeoutException;
 import java.io.*;
+import java.io.Closeable;
+import java.util.ArrayList;
+import java.util.Arrays;
+import java.util.List;
 import java.util.Map;
 import java.util.HashMap;
 
@@ -34,6 +40,8 @@
 import org.apache.commons.logging.*;
 
 import org.apache.hadoop.io.*;
+import org.apache.hadoop.ipc.RpcPayloadHeader.RpcKind;
+import org.apache.hadoop.ipc.protobuf.ProtocolInfoProtos.ProtocolInfoService;
 import org.apache.hadoop.net.NetUtils;
 import org.apache.hadoop.security.SaslRpcServer;
 import org.apache.hadoop.security.UserGroupInformation;
@@ -42,6 +50,8 @@
 import org.apache.hadoop.conf.*;
 import org.apache.hadoop.util.ReflectionUtils;
 
+import com.google.protobuf.BlockingService;
+
 /** A simple RPC mechanism.
  *
  * A <i>protocol</i> is a Java interface.  All parameters and return types must
@@ -61,17 +71,100 @@
  * the protocol instance is transmitted.
  */
 public class RPC {
+  
+  interface RpcInvoker {   
+    /**
+     * Process a client call on the server side
+     * @param server the server within whose context this rpc call is made
+     * @param protocol - the protocol name (the class of the client proxy
+     *      used to make calls to the rpc server.
+     * @param rpcRequest  - deserialized
+     * @param receiveTime time at which the call received (for metrics)
+     * @return the call's return
+     * @throws IOException
+     **/
+    public Writable call(Server server, String protocol,
+        Writable rpcRequest, long receiveTime) throws IOException ;
+  }
+  
   static final Log LOG = LogFactory.getLog(RPC.class);
+  
+  /**
+   * Get all superInterfaces that extend VersionedProtocol
+   * @param childInterfaces
+   * @return the super interfaces that extend VersionedProtocol
+   */
+  static Class<?>[] getSuperInterfaces(Class<?>[] childInterfaces) {
+    List<Class<?>> allInterfaces = new ArrayList<Class<?>>();
+
+    for (Class<?> childInterface : childInterfaces) {
+      if (VersionedProtocol.class.isAssignableFrom(childInterface)) {
+          allInterfaces.add(childInterface);
+          allInterfaces.addAll(
+              Arrays.asList(
+                  getSuperInterfaces(childInterface.getInterfaces())));
+      } else {
+        LOG.warn("Interface " + childInterface +
+              " ignored because it does not extend VersionedProtocol");
+      }
+    }
+    return allInterfaces.toArray(new Class[allInterfaces.size()]);
+  }
+  
+  /**
+   * Get all interfaces that the given protocol implements or extends
+   * which are assignable from VersionedProtocol.
+   */
+  static Class<?>[] getProtocolInterfaces(Class<?> protocol) {
+    Class<?>[] interfaces  = protocol.getInterfaces();
+    return getSuperInterfaces(interfaces);
+  }
+  
+  /**
+   * Get the protocol name.
+   *  If the protocol class has a ProtocolAnnotation, then get the protocol
+   *  name from the annotation; otherwise the class name is the protocol name.
+   */
+  static public String getProtocolName(Class<?> protocol) {
+    if (protocol == null) {
+      return null;
+    }
+    ProtocolInfo anno = protocol.getAnnotation(ProtocolInfo.class);
+    return  (anno == null) ? protocol.getName() : anno.protocolName();
+  }
+  
+  /**
+   * Get the protocol version from protocol class.
+   * If the protocol class has a ProtocolAnnotation, then get the protocol
+   * name from the annotation; otherwise the class name is the protocol name.
+   */
+  static public long getProtocolVersion(Class<?> protocol) {
+    if (protocol == null) {
+      throw new IllegalArgumentException("Null protocol");
+    }
+    long version;
+    ProtocolInfo anno = protocol.getAnnotation(ProtocolInfo.class);
+    if (anno != null) {
+      version = anno.protocolVersion();
+      if (version != -1)
+        return version;
+    }
+    try {
+      Field versionField = protocol.getField("versionID");
+      versionField.setAccessible(true);
+      return versionField.getLong(protocol);
+    } catch (NoSuchFieldException ex) {
+      throw new RuntimeException(ex);
+    } catch (IllegalAccessException ex) {
+      throw new RuntimeException(ex);
+    }
+  }
 
   private RPC() {}                                  // no public ctor
 
   // cache of RpcEngines by protocol
-  private static final Map<Class,RpcEngine> PROTOCOL_ENGINES
-    = new HashMap<Class,RpcEngine>();
-
-  // track what RpcEngine is used by a proxy class, for stopProxy()
-  private static final Map<Class,RpcEngine> PROXY_ENGINES
-    = new HashMap<Class,RpcEngine>();
+  private static final Map<Class<?>,RpcEngine> PROTOCOL_ENGINES
+    = new HashMap<Class<?>,RpcEngine>();
 
   private static final String ENGINE_PROP = "rpc.engine";
 
@@ -82,32 +175,23 @@
    * @param engine the RpcEngine impl
    */
   public static void setProtocolEngine(Configuration conf,
-                                Class protocol, Class engine) {
+                                Class<?> protocol, Class<?> engine) {
     conf.setClass(ENGINE_PROP+"."+protocol.getName(), engine, RpcEngine.class);
   }
 
   // return the RpcEngine configured to handle a protocol
-  private static synchronized RpcEngine getProtocolEngine(Class protocol,
-                                                          Configuration conf) {
+  static synchronized RpcEngine getProtocolEngine(Class<?> protocol,
+      Configuration conf) {
     RpcEngine engine = PROTOCOL_ENGINES.get(protocol);
     if (engine == null) {
       Class<?> impl = conf.getClass(ENGINE_PROP+"."+protocol.getName(),
                                     WritableRpcEngine.class);
       engine = (RpcEngine)ReflectionUtils.newInstance(impl, conf);
-      if (protocol.isInterface())
-        PROXY_ENGINES.put(Proxy.getProxyClass(protocol.getClassLoader(),
-                                              protocol),
-                          engine);
       PROTOCOL_ENGINES.put(protocol, engine);
     }
     return engine;
   }
 
-  // return the RpcEngine that handles a proxy object
-  private static synchronized RpcEngine getProxyEngine(Object proxy) {
-    return PROXY_ENGINES.get(proxy.getClass());
-  }
-
   /**
    * A version mismatch for the RPC protocol.
    */
@@ -441,7 +525,16 @@
 
      return getProtocolProxy(protocol, clientVersion, addr, conf).getProxy();
    }
-
+  
+  /**
+   * Returns the server address for a given proxy.
+   */
+  public static InetSocketAddress getServerAddress(Object proxy) {
+    RpcInvocationHandler inv = (RpcInvocationHandler) Proxy
+        .getInvocationHandler(proxy);
+    return inv.getConnectionId().getAddress();
+  }
+   
   /**
    * Get a protocol proxy that contains a proxy connection to a remote server
    * and a set of methods that are supported by the server
@@ -463,13 +556,31 @@
   }
 
   /**
-   * Stop this proxy and release its invoker's resource
-   * @param proxy the proxy to be stopped
+   * Stop this proxy and release its invoker's resource by getting the
+   * invocation handler for the given proxy object and calling
+   * {@link Closeable#close} if that invocation handler implements
+   * {@link Closeable}.
+   * 
+   * @param proxy the RPC proxy object to be stopped
    */
   public static void stopProxy(Object proxy) {
-    RpcEngine rpcEngine;
-    if (proxy!=null && (rpcEngine = getProxyEngine(proxy)) != null) {
-      rpcEngine.stopProxy(proxy);
+    InvocationHandler invocationHandler = null;
+    try {
+      invocationHandler = Proxy.getInvocationHandler(proxy);
+    } catch (IllegalArgumentException e) {
+      LOG.error("Tried to call RPC.stopProxy on an object that is not a proxy.", e);
+    }
+    if (proxy != null && invocationHandler != null &&
+        invocationHandler instanceof Closeable) {
+      try {
+        ((Closeable)invocationHandler).close();
+      } catch (IOException e) {
+        LOG.error("Stopping RPC invocation handler caused exception", e);
+      }
+    } else {
+      LOG.error("Could not get invocation handler " + invocationHandler +
+          " for proxy class " + (proxy == null ? null : proxy.getClass()) +
+          ", or invocation handler is not closeable.");
     }
   }
 
@@ -518,7 +629,7 @@
   }
 
   /** Construct a server for a protocol implementation instance. */
-  public static Server getServer(Class protocol,
+  public static Server getServer(Class<?> protocol,
                                  Object instance, String bindAddress,
                                  int port, Configuration conf) 
     throws IOException {
@@ -529,7 +640,7 @@
    * @deprecated secretManager should be passed.
    */
   @Deprecated
-  public static Server getServer(Class protocol,
+  public static Server getServer(Class<?> protocol,
                                  Object instance, String bindAddress, int port,
                                  int numHandlers,
                                  boolean verbose, Configuration conf) 
@@ -553,8 +664,10 @@
   }
 
   /** Construct a server for a protocol implementation instance. */
-  public static Server getServer(Class<?> protocol,
-                                 Object instance, String bindAddress, int port,
+
+  public static <PROTO extends VersionedProtocol, IMPL extends PROTO> 
+        Server getServer(Class<PROTO> protocol,
+                                 IMPL instance, String bindAddress, int port,
                                  int numHandlers, int numReaders, int queueSizePerHandler,
                                  boolean verbose, Configuration conf,
                                  SecretManager<? extends TokenIdentifier> secretManager) 
@@ -567,6 +680,147 @@
 
   /** An RPC Server. */
   public abstract static class Server extends org.apache.hadoop.ipc.Server {
+   boolean verbose;
+   static String classNameBase(String className) {
+      String[] names = className.split("\\.", -1);
+      if (names == null || names.length == 0) {
+        return className;
+      }
+      return names[names.length-1];
+    }
+   
+   /**
+    * Store a map of protocol and version to its implementation
+    */
+   /**
+    *  The key in Map
+    */
+   static class ProtoNameVer {
+     final String protocol;
+     final long   version;
+     ProtoNameVer(String protocol, long ver) {
+       this.protocol = protocol;
+       this.version = ver;
+     }
+     @Override
+     public boolean equals(Object o) {
+       if (o == null) 
+         return false;
+       if (this == o) 
+         return true;
+       if (! (o instanceof ProtoNameVer))
+         return false;
+       ProtoNameVer pv = (ProtoNameVer) o;
+       return ((pv.protocol.equals(this.protocol)) && 
+           (pv.version == this.version));     
+     }
+     @Override
+     public int hashCode() {
+       return protocol.hashCode() * 37 + (int) version;    
+     }
+   }
+   
+   /**
+    * The value in map
+    */
+   static class ProtoClassProtoImpl {
+     final Class<?> protocolClass;
+     final Object protocolImpl; 
+     ProtoClassProtoImpl(Class<?> protocolClass, Object protocolImpl) {
+       this.protocolClass = protocolClass;
+       this.protocolImpl = protocolImpl;
+     }
+   }
+
+   ArrayList<Map<ProtoNameVer, ProtoClassProtoImpl>> protocolImplMapArray = 
+       new ArrayList<Map<ProtoNameVer, ProtoClassProtoImpl>>(RpcKind.MAX_INDEX);
+   
+   Map<ProtoNameVer, ProtoClassProtoImpl> getProtocolImplMap(RpcKind rpcKind) {
+     if (protocolImplMapArray.size() == 0) {// initialize for all rpc kinds
+       for (int i=0; i <= RpcKind.MAX_INDEX; ++i) {
+         protocolImplMapArray.add(
+             new HashMap<ProtoNameVer, ProtoClassProtoImpl>(10));
+       }
+     }
+     return protocolImplMapArray.get(rpcKind.ordinal());   
+   }
+   
+   // Register  protocol and its impl for rpc calls
+   void registerProtocolAndImpl(RpcKind rpcKind, Class<?> protocolClass, 
+       Object protocolImpl) throws IOException {
+     String protocolName = RPC.getProtocolName(protocolClass);
+     long version;
+     
+
+     try {
+       version = RPC.getProtocolVersion(protocolClass);
+     } catch (Exception ex) {
+       LOG.warn("Protocol "  + protocolClass + 
+            " NOT registered as cannot get protocol version ");
+       return;
+     }
+
+
+     getProtocolImplMap(rpcKind).put(new ProtoNameVer(protocolName, version),
+         new ProtoClassProtoImpl(protocolClass, protocolImpl)); 
+     LOG.debug("RpcKind = " + rpcKind + " Protocol Name = " + protocolName +  " version=" + version +
+         " ProtocolImpl=" + protocolImpl.getClass().getName() + 
+         " protocolClass=" + protocolClass.getName());
+   }
+   
+   static class VerProtocolImpl {
+     final long version;
+     final ProtoClassProtoImpl protocolTarget;
+     VerProtocolImpl(long ver, ProtoClassProtoImpl protocolTarget) {
+       this.version = ver;
+       this.protocolTarget = protocolTarget;
+     }
+   }
+   
+   
+   @SuppressWarnings("unused") // will be useful later.
+   VerProtocolImpl[] getSupportedProtocolVersions(RpcKind rpcKind,
+       String protocolName) {
+     VerProtocolImpl[] resultk = 
+         new  VerProtocolImpl[getProtocolImplMap(rpcKind).size()];
+     int i = 0;
+     for (Map.Entry<ProtoNameVer, ProtoClassProtoImpl> pv :
+                                       getProtocolImplMap(rpcKind).entrySet()) {
+       if (pv.getKey().protocol.equals(protocolName)) {
+         resultk[i++] = 
+             new VerProtocolImpl(pv.getKey().version, pv.getValue());
+       }
+     }
+     if (i == 0) {
+       return null;
+     }
+     VerProtocolImpl[] result = new VerProtocolImpl[i];
+     System.arraycopy(resultk, 0, result, 0, i);
+     return result;
+   }
+   
+   VerProtocolImpl getHighestSupportedProtocol(RpcKind rpcKind, 
+       String protocolName) {    
+     Long highestVersion = 0L;
+     ProtoClassProtoImpl highest = null;
+     if (LOG.isDebugEnabled()) {
+       LOG.debug("Size of protoMap for " + rpcKind + " ="
+           + getProtocolImplMap(rpcKind).size());
+     }
+     for (Map.Entry<ProtoNameVer, ProtoClassProtoImpl> pv : 
+           getProtocolImplMap(rpcKind).entrySet()) {
+       if (pv.getKey().protocol.equals(protocolName)) {
+         if ((highest == null) || (pv.getKey().version > highestVersion)) {
+           highest = pv.getValue();
+           highestVersion = pv.getKey().version;
+         } 
+       }
+     }
+     if (highest == null) {
+       return null;
+     }
+     return new VerProtocolImpl(highestVersion,  highest);   
+   }
   
     protected Server(String bindAddress, int port, 
                      Class<? extends Writable> paramClass, int handlerCount,
@@ -575,7 +829,38 @@
                      SecretManager<? extends TokenIdentifier> secretManager) throws IOException {
       super(bindAddress, port, paramClass, handlerCount, numReaders, queueSizePerHandler,
             conf, serverName, secretManager);
+      initProtocolMetaInfo(conf);
+    }
+    
+    private void initProtocolMetaInfo(Configuration conf)
+        throws IOException {
+      RPC.setProtocolEngine(conf, ProtocolMetaInfoPB.class,
+          ProtobufRpcEngine.class);
+      ProtocolMetaInfoServerSideTranslatorPB xlator = 
+          new ProtocolMetaInfoServerSideTranslatorPB(this);
+      BlockingService protocolInfoBlockingService = ProtocolInfoService
+          .newReflectiveBlockingService(xlator);
+      addProtocol(RpcKind.RPC_PROTOCOL_BUFFER, ProtocolMetaInfoPB.class,
+          protocolInfoBlockingService);
+    }
+    
+    /**
+     * Add a protocol to the existing server.
+     * @param protocolClass - the protocol class
+     * @param protocolImpl - the impl of the protocol that will be called
+     * @return the server (for convenience)
+     */
+    public Server addProtocol(RpcKind rpcKind, Class<?> protocolClass,
+        Object protocolImpl) throws IOException {
+      registerProtocolAndImpl(rpcKind, protocolClass, protocolImpl);
+      return this;
+    }
+    
+    @Override
+    public Writable call(RpcKind rpcKind, String protocol,
+        Writable rpcRequest, long receiveTime) throws IOException {
+      return getRpcInvoker(rpcKind).call(this, protocol, rpcRequest,
+          receiveTime);
     }
   }
-
 }
diff --git a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/ipc/RpcClientUtil.java b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/ipc/RpcClientUtil.java
new file mode 100644
index 0000000..cdbc034
--- /dev/null
+++ b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/ipc/RpcClientUtil.java
@@ -0,0 +1,193 @@
+/**
+ * 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.ipc;
+
+import java.io.IOException;
+import java.lang.reflect.Method;
+import java.lang.reflect.Proxy;
+import java.net.InetSocketAddress;
+import java.util.List;
+import java.util.Map;
+import java.util.TreeMap;
+import java.util.concurrent.ConcurrentHashMap;
+
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.ipc.RpcPayloadHeader.RpcKind;
+import org.apache.hadoop.ipc.protobuf.ProtocolInfoProtos.GetProtocolSignatureRequestProto;
+import org.apache.hadoop.ipc.protobuf.ProtocolInfoProtos.GetProtocolSignatureResponseProto;
+import org.apache.hadoop.ipc.protobuf.ProtocolInfoProtos.ProtocolSignatureProto;
+import org.apache.hadoop.net.NetUtils;
+
+import com.google.protobuf.RpcController;
+import com.google.protobuf.ServiceException;
+
+/**
+ * This class maintains a cache of protocol versions and corresponding protocol
+ * signatures, keyed by server address, protocol and rpc kind.
+ * The cache is lazily populated. 
+ */
+public class RpcClientUtil {
+  private static RpcController NULL_CONTROLLER = null;
+  private static final int PRIME = 16777619;
+  
+  private static class ProtoSigCacheKey {
+    private InetSocketAddress serverAddress;
+    private String protocol;
+    private String rpcKind;
+    
+    ProtoSigCacheKey(InetSocketAddress addr, String p, String rk) {
+      this.serverAddress = addr;
+      this.protocol = p;
+      this.rpcKind = rk;
+    }
+    
+    @Override //Object
+    public int hashCode() {
+      int result = 1;
+      result = PRIME * result
+          + ((serverAddress == null) ? 0 : serverAddress.hashCode());
+      result = PRIME * result + ((protocol == null) ? 0 : protocol.hashCode());
+      result = PRIME * result + ((rpcKind == null) ? 0 : rpcKind.hashCode());
+      return result;
+    }
+    
+    @Override //Object
+    public boolean equals(Object other) {
+      if (other == this) {
+        return true;
+      }
+      if (other instanceof ProtoSigCacheKey) {
+        ProtoSigCacheKey otherKey = (ProtoSigCacheKey) other;
+        return (serverAddress.equals(otherKey.serverAddress) &&
+            protocol.equals(otherKey.protocol) &&
+            rpcKind.equals(otherKey.rpcKind));
+      }
+      return false;
+    }
+  }
+  
+  private static ConcurrentHashMap<ProtoSigCacheKey, Map<Long, ProtocolSignature>> 
+  signatureMap = new ConcurrentHashMap<ProtoSigCacheKey, Map<Long, ProtocolSignature>>();
+
+  private static void putVersionSignatureMap(InetSocketAddress addr,
+      String protocol, String rpcKind, Map<Long, ProtocolSignature> map) {
+    signatureMap.put(new ProtoSigCacheKey(addr, protocol, rpcKind), map);
+  }
+  
+  private static Map<Long, ProtocolSignature> getVersionSignatureMap(
+      InetSocketAddress addr, String protocol, String rpcKind) {
+    return signatureMap.get(new ProtoSigCacheKey(addr, protocol, rpcKind));
+  }
+
+  /**
+   * Returns whether the given method is supported or not.
+   * The protocol signatures are fetched and cached. The connection id for the
+   * proxy provided is re-used.
+   * @param rpcProxy Proxy which provides an existing connection id.
+   * @param protocol Protocol for which the method check is required.
+   * @param rpcKind The RpcKind for which the method check is required.
+   * @param version The version at the client.
+   * @param methodName Name of the method.
+   * @return true if the method is supported, false otherwise.
+   * @throws IOException
+   */
+  public static boolean isMethodSupported(Object rpcProxy, Class<?> protocol,
+      RpcKind rpcKind, long version, String methodName) throws IOException {
+    InetSocketAddress serverAddress = RPC.getServerAddress(rpcProxy);
+    Map<Long, ProtocolSignature> versionMap = getVersionSignatureMap(
+        serverAddress, protocol.getName(), rpcKind.toString());
+
+    if (versionMap == null) {
+      Configuration conf = new Configuration();
+      RPC.setProtocolEngine(conf, ProtocolMetaInfoPB.class,
+          ProtobufRpcEngine.class);
+      ProtocolMetaInfoPB protocolInfoProxy = getProtocolMetaInfoProxy(rpcProxy,
+          conf);
+      GetProtocolSignatureRequestProto.Builder builder = 
+          GetProtocolSignatureRequestProto.newBuilder();
+      builder.setProtocol(protocol.getName());
+      builder.setRpcKind(rpcKind.toString());
+      GetProtocolSignatureResponseProto resp;
+      try {
+        resp = protocolInfoProxy.getProtocolSignature(NULL_CONTROLLER,
+            builder.build());
+      } catch (ServiceException se) {
+        throw ProtobufHelper.getRemoteException(se);
+      }
+      versionMap = convertProtocolSignatureProtos(resp
+          .getProtocolSignatureList());
+      putVersionSignatureMap(serverAddress, protocol.getName(),
+          rpcKind.toString(), versionMap);
+    }
+    // Assuming unique method names.
+    Method desiredMethod;
+    Method[] allMethods = protocol.getMethods();
+    desiredMethod = null;
+    for (Method m : allMethods) {
+      if (m.getName().equals(methodName)) {
+        desiredMethod = m;
+        break;
+      }
+    }
+    if (desiredMethod == null) {
+      return false;
+    }
+    int methodHash = ProtocolSignature.getFingerprint(desiredMethod);
+    return methodExists(methodHash, version, versionMap);
+  }
+  
+  private static Map<Long, ProtocolSignature> 
+  convertProtocolSignatureProtos(List<ProtocolSignatureProto> protoList) {
+    Map<Long, ProtocolSignature> map = new TreeMap<Long, ProtocolSignature>();
+    for (ProtocolSignatureProto p : protoList) {
+      int [] methods = new int[p.getMethodsList().size()];
+      int index=0;
+      for (int m : p.getMethodsList()) {
+        methods[index++] = m;
+      }
+      map.put(p.getVersion(), new ProtocolSignature(p.getVersion(), methods));
+    }
+    return map;
+  }
+
+  private static boolean methodExists(int methodHash, long version,
+      Map<Long, ProtocolSignature> versionMap) {
+    ProtocolSignature sig = versionMap.get(version);
+    if (sig != null) {
+      for (int m : sig.getMethods()) {
+        if (m == methodHash) {
+          return true;
+        }
+      }
+    }
+    return false;
+  }
+  
+  // The proxy returned re-uses the underlying connection. This is a special 
+  // mechanism for ProtocolMetaInfoPB.
+  // Don't do this for any other protocol, it might cause a security hole.
+  private static ProtocolMetaInfoPB getProtocolMetaInfoProxy(Object proxy,
+      Configuration conf) throws IOException {
+    RpcInvocationHandler inv = (RpcInvocationHandler) Proxy
+        .getInvocationHandler(proxy);
+    return RPC
+        .getProtocolEngine(ProtocolMetaInfoPB.class, conf)
+        .getProtocolMetaInfoProxy(inv.getConnectionId(), conf,
+            NetUtils.getDefaultSocketFactory(conf)).getProxy();
+  }
+}
diff --git a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/ipc/RpcEngine.java b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/ipc/RpcEngine.java
index 500cd95..0fc7d60 100644
--- a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/ipc/RpcEngine.java
+++ b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/ipc/RpcEngine.java
@@ -26,6 +26,7 @@
 
 import org.apache.hadoop.classification.InterfaceStability;
 import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.ipc.Client.ConnectionId;
 import org.apache.hadoop.security.UserGroupInformation;
 import org.apache.hadoop.security.token.SecretManager;
 import org.apache.hadoop.security.token.TokenIdentifier;
@@ -41,9 +42,6 @@
                   UserGroupInformation ticket, Configuration conf,
                   SocketFactory factory, int rpcTimeout) throws IOException;
 
-  /** Stop this proxy. */
-  void stopProxy(Object proxy);
-
   /** Expert: Make multiple, parallel calls to a set of servers. */
   Object[] call(Method method, Object[][] params, InetSocketAddress[] addrs,
                 UserGroupInformation ticket, Configuration conf)
@@ -57,4 +55,16 @@
                        SecretManager<? extends TokenIdentifier> secretManager
                        ) throws IOException;
 
+  /**
+   * Returns a proxy for ProtocolMetaInfoPB, which uses the given connection
+   * id.
+   * @param connId, ConnectionId to be used for the proxy.
+   * @param conf, Configuration.
+   * @param factory, Socket factory.
+   * @return Proxy object.
+   * @throws IOException
+   */
+  ProtocolProxy<ProtocolMetaInfoPB> getProtocolMetaInfoProxy(
+      ConnectionId connId, Configuration conf, SocketFactory factory)
+      throws IOException;
 }
diff --git a/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/ipc/AvroTestProtocol.java b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/ipc/RpcInvocationHandler.java
similarity index 65%
copy from hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/ipc/AvroTestProtocol.java
copy to hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/ipc/RpcInvocationHandler.java
index d5d7396..6bcd757 100644
--- a/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/ipc/AvroTestProtocol.java
+++ b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/ipc/RpcInvocationHandler.java
@@ -15,18 +15,22 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
-
 package org.apache.hadoop.ipc;
 
-import org.apache.avro.AvroRemoteException;
+import java.io.Closeable;
+import java.lang.reflect.InvocationHandler;
 
-@SuppressWarnings("serial")
-public interface AvroTestProtocol {
-  public static class Problem extends AvroRemoteException {
-    public Problem() {}
-  }
-  void ping();
-  String echo(String value);
-  int add(int v1, int v2);
-  int error() throws Problem;
+import org.apache.hadoop.ipc.Client.ConnectionId;
+
+/**
+ * This interface must be implemented by all InvocationHandler
+ * implementations.
+ */
+public interface RpcInvocationHandler extends InvocationHandler, Closeable {
+  
+  /**
+   * Returns the connection id associated with the InvocationHandler instance.
+   * @return ConnectionId
+   */
+  ConnectionId getConnectionId();
 }
diff --git a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/ipc/RpcPayloadHeader.java b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/ipc/RpcPayloadHeader.java
new file mode 100644
index 0000000..6e97159
--- /dev/null
+++ b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/ipc/RpcPayloadHeader.java
@@ -0,0 +1,118 @@
+package org.apache.hadoop.ipc;
+
+import java.io.DataInput;
+import java.io.DataOutput;
+import java.io.IOException;
+
+import org.apache.hadoop.io.Writable;
+
+/**
+ * This is the rpc payload header. It is sent with every rpc call
+ * <pre>
+ * The format of RPC call is as follows:
+ * +---------------------------------------------------+
+ * |  Rpc length in bytes (header + payload length)    |
+ * +---------------------------------------------------+
+ * |      Rpc Header       |       Rpc Payload         |
+ * +---------------------------------------------------+
+ * 
+ * The format of Rpc Header is:
+ * +----------------------------------+
+ * |  RpcKind (1 bytes)               |      
+ * +----------------------------------+
+ * |  RpcPayloadOperation (1 bytes)   |      
+ * +----------------------------------+
+ * |  Call ID (4 bytes)               |      
+ * +----------------------------------+
+ * 
+ * {@link RpcKind} determines the type of serialization used for Rpc Payload.
+ * </pre>
+ * <p>
+ * <b>Note this header does NOT have its own version number, 
+ * it used the version number from the connection header. </b>
+ */
+public class RpcPayloadHeader implements Writable {
+  public enum RpcPayloadOperation {
+    RPC_FINAL_PAYLOAD ((short)1),
+    RPC_CONTINUATION_PAYLOAD ((short)2), // not implemented yet
+    RPC_CLOSE_CONNECTION ((short)3);     // close the rpc connection
+    
+    private final short code;
+    private static final short FIRST_INDEX = RPC_FINAL_PAYLOAD.code;
+    RpcPayloadOperation(short val) {
+      this.code = val;
+    }
+    
+    public void write(DataOutput out) throws IOException {  
+      out.writeByte(code);
+    }
+    
+    static RpcPayloadOperation readFields(DataInput in) throws IOException {
+      short inValue = in.readByte();
+      return RpcPayloadOperation.values()[inValue - FIRST_INDEX];
+    }
+  }
+  
+  public enum RpcKind {
+    RPC_BUILTIN ((short) 1),         // Used for built in calls by tests
+    RPC_WRITABLE ((short) 2),        // Use WritableRpcEngine 
+    RPC_PROTOCOL_BUFFER ((short) 3); // Use ProtobufRpcEngine
+    final static short MAX_INDEX = RPC_PROTOCOL_BUFFER.value; // used for array size
+    private static final short FIRST_INDEX = RPC_BUILTIN.value;    
+    private final short value;
+
+    RpcKind(short val) {
+      this.value = val;
+    }
+    
+    public void write(DataOutput out) throws IOException {
+      out.writeByte(value);
+    }
+    
+    static RpcKind readFields(DataInput in) throws IOException {
+      short inValue = in.readByte();
+      return RpcKind.values()[inValue - FIRST_INDEX];
+    }  
+  }
+  
+  private RpcKind kind;
+  private RpcPayloadOperation operation;
+  private int callId;
+  
+  public RpcPayloadHeader() {
+    kind = RpcKind.RPC_WRITABLE;
+    operation = RpcPayloadOperation.RPC_CLOSE_CONNECTION;
+  }
+  
+  public RpcPayloadHeader(RpcKind kind, RpcPayloadOperation op, int callId) {
+    this.kind  = kind;
+    this.operation = op;
+    this.callId = callId;
+  }
+  
+  int getCallId() {
+    return callId;
+  }
+  
+  RpcKind getkind() {
+    return kind;
+  }
+  
+  RpcPayloadOperation getOperation() {
+    return operation;
+  }
+
+  @Override
+  public void write(DataOutput out) throws IOException {
+    kind.write(out);
+    operation.write(out);
+    out.writeInt(callId); 
+  }
+
+  @Override
+  public void readFields(DataInput in) throws IOException {
+    kind = RpcKind.readFields(in);
+    operation = RpcPayloadOperation.readFields(in);
+    this.callId = in.readInt();
+  }
+}
diff --git a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/ipc/RpcServerException.java b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/ipc/RpcServerException.java
index d7742fe..721b10c 100644
--- a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/ipc/RpcServerException.java
+++ b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/ipc/RpcServerException.java
@@ -25,10 +25,9 @@
 
   /**
    * Constructs exception with the specified detail message.
-   * 
-   * @param messages detailed message.
+   * @param message detailed message.
    */
-  RpcServerException(final String message) {
+  public RpcServerException(final String message) {
     super(message);
   }
   
@@ -36,12 +35,11 @@
    * Constructs exception with the specified detail message and cause.
    * 
    * @param message message.
-   * @param cause that cause this exception
    * @param cause the cause (can be retried by the {@link #getCause()} method).
    *          (A <tt>null</tt> value is permitted, and indicates that the cause
    *          is nonexistent or unknown.)
    */
-  RpcServerException(final String message, final Throwable cause) {
+  public RpcServerException(final String message, final Throwable cause) {
     super(message, cause);
   }
 }
diff --git a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/ipc/Server.java b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/ipc/Server.java
index 74755ce..8067260 100644
--- a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/ipc/Server.java
+++ b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/ipc/Server.java
@@ -42,7 +42,9 @@
 import java.nio.channels.WritableByteChannel;
 import java.security.PrivilegedExceptionAction;
 import java.util.ArrayList;
+import java.util.Arrays;
 import java.util.Collections;
+import java.util.HashMap;
 import java.util.Iterator;
 import java.util.LinkedList;
 import java.util.List;
@@ -66,28 +68,33 @@
 import org.apache.hadoop.io.IntWritable;
 import org.apache.hadoop.io.Writable;
 import org.apache.hadoop.io.WritableUtils;
+import org.apache.hadoop.ipc.RPC.RpcInvoker;
 import org.apache.hadoop.ipc.RPC.VersionMismatch;
+import org.apache.hadoop.ipc.RpcPayloadHeader.RpcKind;
+import org.apache.hadoop.ipc.RpcPayloadHeader.RpcPayloadOperation;
 import org.apache.hadoop.ipc.metrics.RpcDetailedMetrics;
 import org.apache.hadoop.ipc.metrics.RpcMetrics;
 import org.apache.hadoop.net.NetUtils;
 import org.apache.hadoop.security.AccessControlException;
 import org.apache.hadoop.security.SaslRpcServer;
 import org.apache.hadoop.security.SaslRpcServer.AuthMethod;
-import org.apache.hadoop.security.SaslRpcServer.SaslStatus;
 import org.apache.hadoop.security.SaslRpcServer.SaslDigestCallbackHandler;
 import org.apache.hadoop.security.SaslRpcServer.SaslGssCallbackHandler;
-import org.apache.hadoop.security.UserGroupInformation.AuthenticationMethod;
+import org.apache.hadoop.security.SaslRpcServer.SaslStatus;
 import org.apache.hadoop.security.UserGroupInformation;
-import org.apache.hadoop.security.authorize.ProxyUsers;
+import org.apache.hadoop.security.UserGroupInformation.AuthenticationMethod;
 import org.apache.hadoop.security.authorize.AuthorizationException;
 import org.apache.hadoop.security.authorize.PolicyProvider;
+import org.apache.hadoop.security.authorize.ProxyUsers;
 import org.apache.hadoop.security.authorize.ServiceAuthorizationManager;
-import org.apache.hadoop.security.token.TokenIdentifier;
 import org.apache.hadoop.security.token.SecretManager;
 import org.apache.hadoop.security.token.SecretManager.InvalidToken;
+import org.apache.hadoop.security.token.TokenIdentifier;
 import org.apache.hadoop.util.ReflectionUtils;
 import org.apache.hadoop.util.StringUtils;
 
+import com.google.common.annotations.VisibleForTesting;
+
 /** An abstract IPC service.  IPC calls take a single {@link Writable} as a
  * parameter, and return a {@link Writable} as their value.  A service runs on
  * a port and is defined by a parameter class and a value class.
@@ -108,12 +115,66 @@
   // 4 : Introduced SASL security layer
   // 5 : Introduced use of {@link ArrayPrimitiveWritable$Internal}
   //     in ObjectWritable to efficiently transmit arrays of primitives
-  public static final byte CURRENT_VERSION = 5;
+  // 6 : Made RPC payload header explicit
+  public static final byte CURRENT_VERSION = 6;
 
   /**
    * Initial and max size of response buffer
    */
   static int INITIAL_RESP_BUF_SIZE = 10240;
+  
+  static class RpcKindMapValue {
+    final Class<? extends Writable> rpcRequestWrapperClass;
+    final RpcInvoker rpcInvoker;
+    RpcKindMapValue (Class<? extends Writable> rpcRequestWrapperClass,
+          RpcInvoker rpcInvoker) {
+      this.rpcInvoker = rpcInvoker;
+      this.rpcRequestWrapperClass = rpcRequestWrapperClass;
+    }   
+  }
+  static Map<RpcKind, RpcKindMapValue> rpcKindMap = new
+      HashMap<RpcKind, RpcKindMapValue>(4);
+  
+  
+
+  /**
+   * Register a RPC kind and the class to deserialize the rpc request.
+   * 
+   * Called by static initializers of rpcKind Engines
+   * @param rpcKind
+   * @param rpcRequestWrapperClass - this class is used to deserialze the
+   *  the rpc request.
+   *  @param rpcInvoker - use to process the calls on SS.
+   */
+  
+  public static void registerProtocolEngine(RpcKind rpcKind, 
+          Class<? extends Writable> rpcRequestWrapperClass,
+          RpcInvoker rpcInvoker) {
+    RpcKindMapValue  old = 
+        rpcKindMap.put(rpcKind, new RpcKindMapValue(rpcRequestWrapperClass, rpcInvoker));
+    if (old != null) {
+      rpcKindMap.put(rpcKind, old);
+      throw new IllegalArgumentException("ReRegistration of rpcKind: " +
+          rpcKind);      
+    }
+    LOG.debug("rpcKind=" + rpcKind + 
+        ", rpcRequestWrapperClass=" + rpcRequestWrapperClass + 
+        ", rpcInvoker=" + rpcInvoker);
+  }
+  
+  public Class<? extends Writable> getRpcRequestWrapper(
+      RpcKind rpcKind) {
+    if (rpcRequestClass != null)
+       return rpcRequestClass;
+    RpcKindMapValue val = rpcKindMap.get(rpcKind);
+    return (val == null) ? null : val.rpcRequestWrapperClass; 
+  }
+  
+  public static RpcInvoker  getRpcInvoker(RpcKind rpcKind) {
+    RpcKindMapValue val = rpcKindMap.get(rpcKind);
+    return (val == null) ? null : val.rpcInvoker; 
+  }
+  
 
   public static final Log LOG = LogFactory.getLog(Server.class);
   public static final Log AUDITLOG = 
@@ -178,7 +239,7 @@
   private int port;                               // port we listen on
   private int handlerCount;                       // number of handler threads
   private int readThreads;                        // number of read threads
-  private Class<? extends Writable> paramClass;   // class of call parameters
+  private Class<? extends Writable> rpcRequestClass;   // class used for deserializing the rpc request
   private int maxIdleTime;                        // the maximum idle time after 
                                                   // which a client may be disconnected
   private int thresholdIdleConnections;           // the number of idle connections
@@ -239,10 +300,21 @@
    * Returns a handle to the rpcMetrics (required in tests)
    * @return rpc metrics
    */
+  @VisibleForTesting
   public RpcMetrics getRpcMetrics() {
     return rpcMetrics;
   }
 
+  @VisibleForTesting
+  public RpcDetailedMetrics getRpcDetailedMetrics() {
+    return rpcDetailedMetrics;
+  }
+  
+  @VisibleForTesting
+  Iterable<? extends Thread> getHandlers() {
+    return Arrays.asList(handlers);
+  }
+
   /**
    * Refresh the service authorization ACL for the service handled by this server.
    */
@@ -261,28 +333,33 @@
 
   /** A call queued for handling. */
   private static class Call {
-    private int id;                               // the client's call id
-    private Writable param;                       // the parameter passed
-    private Connection connection;                // connection to client
-    private long timestamp;     // the time received when response is null
-                                   // the time served when response is not null
-    private ByteBuffer response;                      // the response for this call
+    private final int callId;             // the client's call id
+    private final Writable rpcRequest;    // Serialized Rpc request from client
+    private final Connection connection;  // connection to client
+    private long timestamp;               // time received when response is null
+                                          // time served when response is not null
+    private ByteBuffer rpcResponse;       // the response for this call
+    private final RpcKind rpcKind;
 
-    public Call(int id, Writable param, Connection connection) { 
-      this.id = id;
-      this.param = param;
+    public Call(int id, Writable param, Connection connection) {
+      this( id,  param,  connection, RpcKind.RPC_BUILTIN );    
+    }
+    public Call(int id, Writable param, Connection connection, RpcKind kind) { 
+      this.callId = id;
+      this.rpcRequest = param;
       this.connection = connection;
       this.timestamp = System.currentTimeMillis();
-      this.response = null;
+      this.rpcResponse = null;
+      this.rpcKind = kind;
     }
     
     @Override
     public String toString() {
-      return param.toString() + " from " + connection.toString();
+      return rpcRequest.toString() + " from " + connection.toString();
     }
 
     public void setResponse(ByteBuffer response) {
-      this.response = response;
+      this.rpcResponse = response;
     }
   }
 
@@ -781,17 +858,17 @@
           call = responseQueue.removeFirst();
           SocketChannel channel = call.connection.channel;
           if (LOG.isDebugEnabled()) {
-            LOG.debug(getName() + ": responding to #" + call.id + " from " +
+            LOG.debug(getName() + ": responding to #" + call.callId + " from " +
                       call.connection);
           }
           //
           // Send as much data as we can in the non-blocking fashion
           //
-          int numBytes = channelWrite(channel, call.response);
+          int numBytes = channelWrite(channel, call.rpcResponse);
           if (numBytes < 0) {
             return true;
           }
-          if (!call.response.hasRemaining()) {
+          if (!call.rpcResponse.hasRemaining()) {
             call.connection.decRpcCount();
             if (numElements == 1) {    // last call fully processes.
               done = true;             // no more data for this channel.
@@ -799,7 +876,7 @@
               done = false;            // more calls pending to be sent.
             }
             if (LOG.isDebugEnabled()) {
-              LOG.debug(getName() + ": responding to #" + call.id + " from " +
+              LOG.debug(getName() + ": responding to #" + call.callId + " from " +
                         call.connection + " Wrote " + numBytes + " bytes.");
             }
           } else {
@@ -827,7 +904,7 @@
               }
             }
             if (LOG.isDebugEnabled()) {
-              LOG.debug(getName() + ": responding to #" + call.id + " from " +
+              LOG.debug(getName() + ": responding to #" + call.callId + " from " +
                         call.connection + " Wrote partial " + numBytes + 
                         " bytes.");
             }
@@ -893,7 +970,7 @@
     private InetAddress addr;
     
     ConnectionHeader header = new ConnectionHeader();
-    Class<?> protocol;
+    String protocolName;
     boolean useSasl;
     SaslServer saslServer;
     private AuthMethod authMethod;
@@ -1284,15 +1361,8 @@
       DataInputStream in =
         new DataInputStream(new ByteArrayInputStream(buf));
       header.readFields(in);
-      try {
-        String protocolClassName = header.getProtocol();
-        if (protocolClassName != null) {
-          protocol = getProtocolClass(header.getProtocol(), conf);
-          rpcDetailedMetrics.init(protocol);
-        }
-      } catch (ClassNotFoundException cnfe) {
-        throw new IOException("Unknown protocol: " + header.getProtocol());
-      }
+      protocolName = header.getProtocol();
+
       
       UserGroupInformation protocolUser = header.getUgi();
       if (!useSasl) {
@@ -1384,18 +1454,43 @@
     private void processData(byte[] buf) throws  IOException, InterruptedException {
       DataInputStream dis =
         new DataInputStream(new ByteArrayInputStream(buf));
-      int id = dis.readInt();                    // try to read an id
+      RpcPayloadHeader header = new RpcPayloadHeader();
+      header.readFields(dis);           // Read the RpcPayload header
         
       if (LOG.isDebugEnabled())
-        LOG.debug(" got #" + id);
-      Writable param;
-      try {
-        param = ReflectionUtils.newInstance(paramClass, conf);//read param
-        param.readFields(dis);
+        LOG.debug(" got #" + header.getCallId());
+      if (header.getOperation() != RpcPayloadOperation.RPC_FINAL_PAYLOAD) {
+        throw new IOException("IPC Server does not implement operation" + 
+              header.getOperation());
+      }
+      // If we know the rpc kind, get its class so that we can deserialize
+      // (Note it would make more sense to have the handler deserialize but 
+      // we continue with this original design.
+      Class<? extends Writable> rpcRequestClass = 
+          getRpcRequestWrapper(header.getkind());
+      if (rpcRequestClass == null) {
+        LOG.warn("Unknown rpc kind "  + header.getkind() + 
+            " from client " + getHostAddress());
+        final Call readParamsFailedCall = 
+            new Call(header.getCallId(), null, this);
+        ByteArrayOutputStream responseBuffer = new ByteArrayOutputStream();
+
+        setupResponse(responseBuffer, readParamsFailedCall, Status.FATAL, null,
+            IOException.class.getName(),
+            "Unknown rpc kind "  + header.getkind());
+        responder.doRespond(readParamsFailedCall);
+        return;   
+      }
+      Writable rpcRequest;
+      try { //Read the rpc request
+        rpcRequest = ReflectionUtils.newInstance(rpcRequestClass, conf);
+        rpcRequest.readFields(dis);
       } catch (Throwable t) {
         LOG.warn("Unable to read call parameters for client " +
-                 getHostAddress(), t);
-        final Call readParamsFailedCall = new Call(id, null, this);
+                 getHostAddress() + "on connection protocol " +
+            this.protocolName + " for rpcKind " + header.getkind(),  t);
+        final Call readParamsFailedCall = 
+            new Call(header.getCallId(), null, this);
         ByteArrayOutputStream responseBuffer = new ByteArrayOutputStream();
 
         setupResponse(responseBuffer, readParamsFailedCall, Status.FATAL, null,
@@ -1405,7 +1500,7 @@
         return;
       }
         
-      Call call = new Call(id, param, this);
+      Call call = new Call(header.getCallId(), rpcRequest, this, header.getkind());
       callQueue.put(call);              // queue the call; maybe blocked here
       incRpcCount();  // Increment the rpc count
     }
@@ -1469,8 +1564,8 @@
           final Call call = callQueue.take(); // pop the queue; maybe blocked here
 
           if (LOG.isDebugEnabled())
-            LOG.debug(getName() + ": has #" + call.id + " from " +
-                      call.connection);
+            LOG.debug(getName() + ": has Call#" + call.callId + 
+                "for RpcKind " + call.rpcKind + " from " + call.connection);
           
           String errorClass = null;
           String error = null;
@@ -1481,7 +1576,7 @@
             // Make the call as the user via Subject.doAs, thus associating
             // the call with the Subject
             if (call.connection.user == null) {
-              value = call(call.connection.protocol, call.param, 
+              value = call(call.rpcKind, call.connection.protocolName, call.rpcRequest, 
                            call.timestamp);
             } else {
               value = 
@@ -1490,8 +1585,8 @@
                      @Override
                      public Writable run() throws Exception {
                        // make the call
-                       return call(call.connection.protocol, 
-                                   call.param, call.timestamp);
+                       return call(call.rpcKind, call.connection.protocolName, 
+                                   call.rpcRequest, call.timestamp);
 
                      }
                    }
@@ -1543,24 +1638,33 @@
                   Configuration conf)
     throws IOException 
   {
-    this(bindAddress, port, paramClass, handlerCount, -1, -1, conf, Integer.toString(port), null);
+    this(bindAddress, port, paramClass, handlerCount, -1, -1, conf, Integer
+        .toString(port), null);
   }
   
-  /** Constructs a server listening on the named port and address.  Parameters passed must
+  /** 
+   * Constructs a server listening on the named port and address.  Parameters passed must
    * be of the named class.  The <code>handlerCount</handlerCount> determines
    * the number of handler threads that will be used to process calls.
    * If queueSizePerHandler or numReaders are not -1 they will be used instead of parameters
    * from configuration. Otherwise the configuration will be picked up.
+   * 
+   * If rpcRequestClass is null then the rpcRequestClass must have been 
+   * registered via {@link #registerProtocolEngine(RpcPayloadHeader.RpcKind,
+   *  Class, RPC.RpcInvoker)}
+   * This parameter has been retained for compatibility with existing tests
+   * and usage.
    */
   @SuppressWarnings("unchecked")
-  protected Server(String bindAddress, int port, 
-                  Class<? extends Writable> paramClass, int handlerCount, int numReaders, int queueSizePerHandler,
-                  Configuration conf, String serverName, SecretManager<? extends TokenIdentifier> secretManager) 
+  protected Server(String bindAddress, int port,
+      Class<? extends Writable> rpcRequestClass, int handlerCount,
+      int numReaders, int queueSizePerHandler, Configuration conf,
+      String serverName, SecretManager<? extends TokenIdentifier> secretManager)
     throws IOException {
     this.bindAddress = bindAddress;
     this.conf = conf;
     this.port = port;
-    this.paramClass = paramClass;
+    this.rpcRequestClass = rpcRequestClass; 
     this.handlerCount = handlerCount;
     this.socketSendBufferSize = 0;
     if (queueSizePerHandler != -1) {
@@ -1641,7 +1745,7 @@
   throws IOException {
     response.reset();
     DataOutputStream out = new DataOutputStream(response);
-    out.writeInt(call.id);                // write call id
+    out.writeInt(call.callId);                // write call id
     out.writeInt(status.state);           // write status
 
     if (status == Status.SUCCESS) {
@@ -1758,17 +1862,17 @@
   
   /** 
    * Called for each call. 
-   * @deprecated Use {@link #call(Class, Writable, long)} instead
+   * @deprecated Use  {@link #call(RpcPayloadHeader.RpcKind, String,
+   *  Writable, long)} instead
    */
   @Deprecated
   public Writable call(Writable param, long receiveTime) throws IOException {
-    return call(null, param, receiveTime);
+    return call(RpcKind.RPC_BUILTIN, null, param, receiveTime);
   }
   
   /** Called for each call. */
-  public abstract Writable call(Class<?> protocol,
-                               Writable param, long receiveTime)
-  throws IOException;
+  public abstract Writable call(RpcKind rpcKind, String protocol,
+      Writable param, long receiveTime) throws IOException;
   
   /**
    * Authorize the incoming client connection.
@@ -1918,5 +2022,5 @@
 
     int nBytes = initialRemaining - buf.remaining(); 
     return (nBytes > 0) ? nBytes : ret;
-  }      
+  }
 }
diff --git a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/ipc/VersionedProtocol.java b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/ipc/VersionedProtocol.java
index 4558f21..4d02027 100644
--- a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/ipc/VersionedProtocol.java
+++ b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/ipc/VersionedProtocol.java
@@ -34,7 +34,6 @@
    * @return the version that the server will speak
    * @throws IOException if any IO error occurs
    */
-  @Deprecated
   public long getProtocolVersion(String protocol,
                                  long clientVersion) throws IOException;
 
diff --git a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/ipc/WritableRpcEngine.java b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/ipc/WritableRpcEngine.java
index e587913..beadfa4 100644
--- a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/ipc/WritableRpcEngine.java
+++ b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/ipc/WritableRpcEngine.java
@@ -18,23 +18,23 @@
 
 package org.apache.hadoop.ipc;
 
-import java.lang.reflect.Field;
 import java.lang.reflect.Proxy;
 import java.lang.reflect.Method;
 import java.lang.reflect.Array;
-import java.lang.reflect.InvocationHandler;
 import java.lang.reflect.InvocationTargetException;
 
 import java.net.InetSocketAddress;
 import java.io.*;
-import java.util.Map;
-import java.util.HashMap;
 
 import javax.net.SocketFactory;
 
 import org.apache.commons.logging.*;
 
 import org.apache.hadoop.io.*;
+import org.apache.hadoop.ipc.Client.ConnectionId;
+import org.apache.hadoop.ipc.RPC.RpcInvoker;
+import org.apache.hadoop.ipc.RpcPayloadHeader.RpcKind;
+import org.apache.hadoop.ipc.VersionedProtocol;
 import org.apache.hadoop.security.UserGroupInformation;
 import org.apache.hadoop.security.token.SecretManager;
 import org.apache.hadoop.security.token.TokenIdentifier;
@@ -49,8 +49,38 @@
   
   //writableRpcVersion should be updated if there is a change
   //in format of the rpc messages.
-  public static long writableRpcVersion = 1L;
+  
+  // 2L - added declared class to Invocation
+  public static final long writableRpcVersion = 2L;
+  
+  /**
+   * Whether or not this class has been initialized.
+   */
+  private static boolean isInitialized = false;
+  
+  static { 
+    ensureInitialized();
+  }
+  
+  /**
+   * Initialize this class if it isn't already.
+   */
+  public static synchronized void ensureInitialized() {
+    if (!isInitialized) {
+      initialize();
+    }
+  }
+  
+  /**
+   * Register the rpcRequest deserializer for WritableRpcEngine
+   */
+  private static synchronized void initialize() {
+    org.apache.hadoop.ipc.Server.registerProtocolEngine(RpcKind.RPC_WRITABLE,
+        Invocation.class, new Server.WritableRpcInvoker());
+    isInitialized = true;
+  }
 
+  
   /** A method invocation, including the method name and its parameters.*/
   private static class Invocation implements Writable, Configurable {
     private String methodName;
@@ -59,11 +89,13 @@
     private Configuration conf;
     private long clientVersion;
     private int clientMethodsHash;
+    private String declaringClassProtocolName;
     
     //This could be different from static writableRpcVersion when received
     //at server, if client is using a different version.
     private long rpcVersion;
 
+    @SuppressWarnings("unused") // called when deserializing an invocation
     public Invocation() {}
 
     public Invocation(Method method, Object[] parameters) {
@@ -76,18 +108,12 @@
         clientVersion = 0;
         clientMethodsHash = 0;
       } else {
-        try {
-          Field versionField = method.getDeclaringClass().getField("versionID");
-          versionField.setAccessible(true);
-          this.clientVersion = versionField.getLong(method.getDeclaringClass());
-        } catch (NoSuchFieldException ex) {
-          throw new RuntimeException(ex);
-        } catch (IllegalAccessException ex) {
-          throw new RuntimeException(ex);
-        }
+        this.clientVersion = RPC.getProtocolVersion(method.getDeclaringClass());
         this.clientMethodsHash = ProtocolSignature.getFingerprint(method
             .getDeclaringClass().getMethods());
       }
+      this.declaringClassProtocolName = 
+          RPC.getProtocolName(method.getDeclaringClass());
     }
 
     /** The name of the method invoked. */
@@ -103,6 +129,7 @@
       return clientVersion;
     }
 
+    @SuppressWarnings("unused")
     private int getClientMethodsHash() {
       return clientMethodsHash;
     }
@@ -115,8 +142,10 @@
       return rpcVersion;
     }
 
+    @SuppressWarnings("deprecation")
     public void readFields(DataInput in) throws IOException {
       rpcVersion = in.readLong();
+      declaringClassProtocolName = UTF8.readString(in);
       methodName = UTF8.readString(in);
       clientVersion = in.readLong();
       clientMethodsHash = in.readInt();
@@ -124,13 +153,16 @@
       parameterClasses = new Class[parameters.length];
       ObjectWritable objectWritable = new ObjectWritable();
       for (int i = 0; i < parameters.length; i++) {
-        parameters[i] = ObjectWritable.readObject(in, objectWritable, this.conf);
+        parameters[i] = 
+            ObjectWritable.readObject(in, objectWritable, this.conf);
         parameterClasses[i] = objectWritable.getDeclaredClass();
       }
     }
 
+    @SuppressWarnings("deprecation")
     public void write(DataOutput out) throws IOException {
       out.writeLong(rpcVersion);
+      UTF8.writeString(out, declaringClassProtocolName);
       UTF8.writeString(out, methodName);
       out.writeLong(clientVersion);
       out.writeInt(clientMethodsHash);
@@ -169,7 +201,7 @@
 
   private static ClientCache CLIENTS=new ClientCache();
   
-  private static class Invoker implements InvocationHandler {
+  private static class Invoker implements RpcInvocationHandler {
     private Client.ConnectionId remoteId;
     private Client client;
     private boolean isClosed = false;
@@ -191,7 +223,7 @@
       }
 
       ObjectWritable value = (ObjectWritable)
-        client.call(new Invocation(method, args), remoteId);
+        client.call(RpcKind.RPC_WRITABLE, new Invocation(method, args), remoteId);
       if (LOG.isDebugEnabled()) {
         long callTime = System.currentTimeMillis() - startTime;
         LOG.debug("Call: " + method.getName() + " " + callTime);
@@ -200,12 +232,17 @@
     }
     
     /* close the IPC client that's responsible for this invoker's RPCs */ 
-    synchronized private void close() {
+    synchronized public void close() {
       if (!isClosed) {
         isClosed = true;
         CLIENTS.stopClient(client);
       }
     }
+
+    @Override
+    public ConnectionId getConnectionId() {
+      return remoteId;
+    }
   }
   
   // for unit testing only
@@ -231,15 +268,6 @@
             factory, rpcTimeout));
     return new ProtocolProxy<T>(protocol, proxy, true);
   }
-
-  /**
-   * Stop this proxy and release its invoker's resource
-   * @param proxy the proxy to be stopped
-   */
-  public void stopProxy(Object proxy) {
-    ((Invoker)Proxy.getInvocationHandler(proxy)).close();
-  }
-
   
   /** Expert: Make multiple, parallel calls to a set of servers. */
   public Object[] call(Method method, Object[][] params,
@@ -273,134 +301,238 @@
 
   /** Construct a server for a protocol implementation instance listening on a
    * port and address. */
-  public Server getServer(Class<?> protocol,
-                          Object instance, String bindAddress, int port,
-                          int numHandlers, int numReaders, int queueSizePerHandler,
-                          boolean verbose, Configuration conf,
+  public RPC.Server getServer(Class<?> protocolClass,
+                      Object protocolImpl, String bindAddress, int port,
+                      int numHandlers, int numReaders, int queueSizePerHandler,
+                      boolean verbose, Configuration conf,
                       SecretManager<? extends TokenIdentifier> secretManager) 
     throws IOException {
-    return new Server(instance, conf, bindAddress, port, numHandlers, 
-        numReaders, queueSizePerHandler, verbose, secretManager);
+    return new Server(protocolClass, protocolImpl, conf, bindAddress, port,
+        numHandlers, numReaders, queueSizePerHandler, verbose, secretManager);
   }
 
+
   /** An RPC Server. */
   public static class Server extends RPC.Server {
-    private Object instance;
-    private boolean verbose;
-
-    /** Construct an RPC server.
+    /**
+     * Construct an RPC server.
      * @param instance the instance whose methods will be called
      * @param conf the configuration to use
      * @param bindAddress the address to bind on to listen for connection
      * @param port the port to listen for connections on
+     * 
+     * @deprecated Use #Server(Class, Object, Configuration, String, int)    
      */
-    public Server(Object instance, Configuration conf, String bindAddress, int port) 
-      throws IOException {
-      this(instance, conf,  bindAddress, port, 1, -1, -1, false, null);
+    @Deprecated
+    public Server(Object instance, Configuration conf, String bindAddress,
+        int port) throws IOException {
+      this(null, instance, conf,  bindAddress, port);
     }
     
-    private static String classNameBase(String className) {
-      String[] names = className.split("\\.", -1);
-      if (names == null || names.length == 0) {
-        return className;
-      }
-      return names[names.length-1];
-    }
     
     /** Construct an RPC server.
-     * @param instance the instance whose methods will be called
+     * @param protocolClass class
+     * @param protocolImpl the instance whose methods will be called
+     * @param conf the configuration to use
+     * @param bindAddress the address to bind on to listen for connection
+     * @param port the port to listen for connections on
+     */
+    public Server(Class<?> protocolClass, Object protocolImpl, 
+        Configuration conf, String bindAddress, int port) 
+      throws IOException {
+      this(protocolClass, protocolImpl, conf,  bindAddress, port, 1, -1, -1,
+          false, null);
+    }
+    
+    /** 
+     * Construct an RPC server.
+     * @param protocolImpl the instance whose methods will be called
+     * @param conf the configuration to use
+     * @param bindAddress the address to bind on to listen for connection
+     * @param port the port to listen for connections on
+     * @param numHandlers the number of method handler threads to run
+     * @param verbose whether each call should be logged
+     * 
+     * @deprecated use Server#Server(Class, Object, 
+     *      Configuration, String, int, int, int, int, boolean, SecretManager)
+     */
+    @Deprecated
+    public Server(Object protocolImpl, Configuration conf, String bindAddress,
+        int port, int numHandlers, int numReaders, int queueSizePerHandler,
+        boolean verbose, SecretManager<? extends TokenIdentifier> secretManager) 
+            throws IOException {
+       this(null, protocolImpl,  conf,  bindAddress,   port,
+                   numHandlers,  numReaders,  queueSizePerHandler,  verbose, 
+                   secretManager);
+   
+    }
+    
+    /** 
+     * Construct an RPC server.
+     * @param protocolClass - the protocol being registered
+     *     can be null for compatibility with old usage (see below for details)
+     * @param protocolImpl the protocol impl that will be called
      * @param conf the configuration to use
      * @param bindAddress the address to bind on to listen for connection
      * @param port the port to listen for connections on
      * @param numHandlers the number of method handler threads to run
      * @param verbose whether each call should be logged
      */
-    public Server(Object instance, Configuration conf, String bindAddress,  int port,
-                  int numHandlers, int numReaders, int queueSizePerHandler, boolean verbose, 
-                  SecretManager<? extends TokenIdentifier> secretManager) 
+    public Server(Class<?> protocolClass, Object protocolImpl,
+        Configuration conf, String bindAddress,  int port,
+        int numHandlers, int numReaders, int queueSizePerHandler, 
+        boolean verbose, SecretManager<? extends TokenIdentifier> secretManager) 
         throws IOException {
-      super(bindAddress, port, Invocation.class, numHandlers, numReaders,
+      super(bindAddress, port, null, numHandlers, numReaders,
           queueSizePerHandler, conf,
-          classNameBase(instance.getClass().getName()), secretManager);
-      this.instance = instance;
+          classNameBase(protocolImpl.getClass().getName()), secretManager);
+
       this.verbose = verbose;
+      
+      
+      Class<?>[] protocols;
+      if (protocolClass == null) { // derive protocol from impl
+        /*
+         * In order to remain compatible with the old usage where a single
+         * target protocolImpl is suppled for all protocol interfaces, and
+         * the protocolImpl is derived from the protocolClass(es) 
+         * we register all interfaces extended by the protocolImpl
+         */
+        protocols = RPC.getProtocolInterfaces(protocolImpl.getClass());
+
+      } else {
+        if (!protocolClass.isAssignableFrom(protocolImpl.getClass())) {
+          throw new IOException("protocolClass "+ protocolClass +
+              " is not implemented by protocolImpl which is of class " +
+              protocolImpl.getClass());
+        }
+        // register protocol class and its super interfaces
+        registerProtocolAndImpl(RpcKind.RPC_WRITABLE, protocolClass, protocolImpl);
+        protocols = RPC.getProtocolInterfaces(protocolClass);
+      }
+      for (Class<?> p : protocols) {
+        if (!p.equals(VersionedProtocol.class)) {
+          registerProtocolAndImpl(RpcKind.RPC_WRITABLE, p, protocolImpl);
+        }
+      }
+
     }
 
-    public Writable call(Class<?> protocol, Writable param, long receivedTime) 
-    throws IOException {
-      try {
-        Invocation call = (Invocation)param;
-        if (verbose) log("Call: " + call);
+    private static void log(String value) {
+      if (value!= null && value.length() > 55)
+        value = value.substring(0, 55)+"...";
+      LOG.info(value);
+    }
+    
+    static class WritableRpcInvoker implements RpcInvoker {
 
-        Method method = protocol.getMethod(call.getMethodName(),
-                                           call.getParameterClasses());
-        method.setAccessible(true);
+     @Override
+      public Writable call(org.apache.hadoop.ipc.RPC.Server server,
+          String protocolName, Writable rpcRequest, long receivedTime)
+          throws IOException {
+        try {
+          Invocation call = (Invocation)rpcRequest;
+          if (server.verbose) log("Call: " + call);
 
-        // Verify rpc version
-        if (call.getRpcVersion() != writableRpcVersion) {
-          // Client is using a different version of WritableRpc
-          throw new IOException(
-              "WritableRpc version mismatch, client side version="
-                  + call.getRpcVersion() + ", server side version="
-                  + writableRpcVersion);
-        }
-        
-        //Verify protocol version.
-        //Bypass the version check for VersionedProtocol
-        if (!method.getDeclaringClass().equals(VersionedProtocol.class)) {
-          long clientVersion = call.getProtocolVersion();
-          ProtocolSignature serverInfo = ((VersionedProtocol) instance)
-              .getProtocolSignature(protocol.getCanonicalName(), call
-                  .getProtocolVersion(), call.getClientMethodsHash());
-          long serverVersion = serverInfo.getVersion();
-          if (serverVersion != clientVersion) {
-            LOG.warn("Version mismatch: client version=" + clientVersion
-                + ", server version=" + serverVersion);
-            throw new RPC.VersionMismatch(protocol.getName(), clientVersion,
-                serverVersion);
+          // Verify rpc version
+          if (call.getRpcVersion() != writableRpcVersion) {
+            // Client is using a different version of WritableRpc
+            throw new IOException(
+                "WritableRpc version mismatch, client side version="
+                    + call.getRpcVersion() + ", server side version="
+                    + writableRpcVersion);
           }
-        }
 
-        long startTime = System.currentTimeMillis();
-        Object value = method.invoke(instance, call.getParameters());
-        int processingTime = (int) (System.currentTimeMillis() - startTime);
-        int qTime = (int) (startTime-receivedTime);
-        if (LOG.isDebugEnabled()) {
-          LOG.debug("Served: " + call.getMethodName() +
-                    " queueTime= " + qTime +
-                    " procesingTime= " + processingTime);
-        }
-        rpcMetrics.addRpcQueueTime(qTime);
-        rpcMetrics.addRpcProcessingTime(processingTime);
-        rpcDetailedMetrics.addProcessingTime(call.getMethodName(),
-                                             processingTime);
-        if (verbose) log("Return: "+value);
+          long clientVersion = call.getProtocolVersion();
+          final String protoName;
+          ProtoClassProtoImpl protocolImpl;
+          if (call.declaringClassProtocolName.equals(VersionedProtocol.class.getName())) {
+            // VersionProtocol methods are often used by client to figure out
+            // which version of protocol to use.
+            //
+            // Versioned protocol methods should go the protocolName protocol
+            // rather than the declaring class of the method since the
+            // the declaring class is VersionedProtocol which is not 
+            // registered directly.
+            // Send the call to the highest  protocol version
+            VerProtocolImpl highest = server.getHighestSupportedProtocol(
+                RpcKind.RPC_WRITABLE, protocolName);
+            if (highest == null) {
+              throw new IOException("Unknown protocol: " + protocolName);
+            }
+            protocolImpl = highest.protocolTarget;
+          } else {
+            protoName = call.declaringClassProtocolName;
 
-        return new ObjectWritable(method.getReturnType(), value);
+            // Find the right impl for the protocol based on client version.
+            ProtoNameVer pv = 
+                new ProtoNameVer(call.declaringClassProtocolName, clientVersion);
+            protocolImpl = 
+                server.getProtocolImplMap(RpcKind.RPC_WRITABLE).get(pv);
+            if (protocolImpl == null) { // no match for Protocol AND Version
+               VerProtocolImpl highest = 
+                   server.getHighestSupportedProtocol(RpcKind.RPC_WRITABLE, 
+                       protoName);
+              if (highest == null) {
+                throw new IOException("Unknown protocol: " + protoName);
+              } else { // protocol supported but not the version that client wants
+                throw new RPC.VersionMismatch(protoName, clientVersion,
+                  highest.version);
+              }
+            }
+          }
+          
 
-      } catch (InvocationTargetException e) {
-        Throwable target = e.getTargetException();
-        if (target instanceof IOException) {
-          throw (IOException)target;
-        } else {
-          IOException ioe = new IOException(target.toString());
-          ioe.setStackTrace(target.getStackTrace());
+          // Invoke the protocol method
+
+          long startTime = System.currentTimeMillis();
+          Method method = 
+              protocolImpl.protocolClass.getMethod(call.getMethodName(),
+              call.getParameterClasses());
+          method.setAccessible(true);
+          server.rpcDetailedMetrics.init(protocolImpl.protocolClass);
+          Object value = 
+              method.invoke(protocolImpl.protocolImpl, call.getParameters());
+          int processingTime = (int) (System.currentTimeMillis() - startTime);
+          int qTime = (int) (startTime-receivedTime);
+          if (LOG.isDebugEnabled()) {
+            LOG.debug("Served: " + call.getMethodName() +
+                      " queueTime= " + qTime +
+                      " procesingTime= " + processingTime);
+          }
+          server.rpcMetrics.addRpcQueueTime(qTime);
+          server.rpcMetrics.addRpcProcessingTime(processingTime);
+          server.rpcDetailedMetrics.addProcessingTime(call.getMethodName(),
+                                               processingTime);
+          if (server.verbose) log("Return: "+value);
+
+          return new ObjectWritable(method.getReturnType(), value);
+
+        } catch (InvocationTargetException e) {
+          Throwable target = e.getTargetException();
+          if (target instanceof IOException) {
+            throw (IOException)target;
+          } else {
+            IOException ioe = new IOException(target.toString());
+            ioe.setStackTrace(target.getStackTrace());
+            throw ioe;
+          }
+        } catch (Throwable e) {
+          if (!(e instanceof IOException)) {
+            LOG.error("Unexpected throwable object ", e);
+          }
+          IOException ioe = new IOException(e.toString());
+          ioe.setStackTrace(e.getStackTrace());
           throw ioe;
         }
-      } catch (Throwable e) {
-        if (!(e instanceof IOException)) {
-          LOG.error("Unexpected throwable object ", e);
-        }
-        IOException ioe = new IOException(e.toString());
-        ioe.setStackTrace(e.getStackTrace());
-        throw ioe;
       }
     }
   }
 
-  private static void log(String value) {
-    if (value!= null && value.length() > 55)
-      value = value.substring(0, 55)+"...";
-    LOG.info(value);
+  @Override
+  public ProtocolProxy<ProtocolMetaInfoPB> getProtocolMetaInfoProxy(
+      ConnectionId connId, Configuration conf, SocketFactory factory)
+      throws IOException {
+    throw new UnsupportedOperationException("This proxy is not supported");
   }
 }
diff --git a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/security/token/delegation/DelegationKey.java b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/security/token/delegation/DelegationKey.java
index 28d8253..7cea679 100644
--- a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/security/token/delegation/DelegationKey.java
+++ b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/security/token/delegation/DelegationKey.java
@@ -42,15 +42,20 @@
   @Nullable
   private byte[] keyBytes = null;
 
+  /** Default constructore required for Writable */
   public DelegationKey() {
-    this(0, 0L, null);
+    this(0, 0L, (SecretKey)null);
   }
 
   public DelegationKey(int keyId, long expiryDate, SecretKey key) {
+    this(keyId, expiryDate, key != null ? key.getEncoded() : null);
+  }
+  
+  public DelegationKey(int keyId, long expiryDate, byte[] encodedKey) {
     this.keyId = keyId;
     this.expiryDate = expiryDate;
-    if (key!=null) {
-      this.keyBytes = key.getEncoded();
+    if (encodedKey != null) {
+      this.keyBytes = encodedKey;
     }
   }
 
@@ -70,6 +75,10 @@
       return key;
     }
   }
+  
+  public byte[] getEncodedKey() {
+    return keyBytes;
+  }
 
   public void setExpiryDate(long expiryDate) {
     this.expiryDate = expiryDate;
diff --git a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/tools/GetGroupsBase.java b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/tools/GetGroupsBase.java
index 4d627cb..da7830d 100644
--- a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/tools/GetGroupsBase.java
+++ b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/tools/GetGroupsBase.java
@@ -94,7 +94,7 @@
    * @return A {@link GetUserMappingsProtocol} client proxy.
    * @throws IOException
    */
-  private GetUserMappingsProtocol getUgmProtocol() throws IOException {
+  protected GetUserMappingsProtocol getUgmProtocol() throws IOException {
     GetUserMappingsProtocol userGroupMappingProtocol =
       RPC.getProxy(GetUserMappingsProtocol.class, 
           GetUserMappingsProtocol.versionID,
diff --git a/hadoop-common-project/hadoop-common/src/main/proto/ProtocolInfo.proto b/hadoop-common-project/hadoop-common/src/main/proto/ProtocolInfo.proto
new file mode 100644
index 0000000..53046aa
--- /dev/null
+++ b/hadoop-common-project/hadoop-common/src/main/proto/ProtocolInfo.proto
@@ -0,0 +1,82 @@
+/**
+ * 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.
+ */
+
+option java_package = "org.apache.hadoop.ipc.protobuf";
+option java_outer_classname = "ProtocolInfoProtos";
+option java_generic_services = true;
+option java_generate_equals_and_hash = true;
+
+/**
+ * Request to get protocol versions for all supported rpc kinds.
+ */
+message GetProtocolVersionsRequestProto {
+  required string protocol = 1; // Protocol name
+}
+
+/**
+ * Protocol version with corresponding rpc kind.
+ */
+message ProtocolVersionProto {
+  required string rpcKind = 1; //RPC kind
+  repeated uint64 versions = 2; //Protocol version corresponding to the rpc kind.
+}
+
+/**
+ * Get protocol version response.
+ */
+message GetProtocolVersionsResponseProto { 
+  repeated ProtocolVersionProto protocolVersions = 1;
+}
+
+/**
+ * Get protocol signature request.
+ */
+message GetProtocolSignatureRequestProto {
+  required string protocol = 1; // Protocol name
+  required string rpcKind = 2; // RPC kind
+}
+
+/**
+ * Get protocol signature response.
+ */ 
+message GetProtocolSignatureResponseProto {
+  repeated ProtocolSignatureProto protocolSignature = 1;
+}
+
+message ProtocolSignatureProto {
+  required uint64 version = 1;
+  repeated uint32 methods = 2;
+}
+
+/**
+ * Protocol to get information about protocols.
+ */
+service ProtocolInfoService {
+  /**
+   * Return protocol version corresponding to protocol interface for each
+   * supported rpc kind.
+   */
+  rpc getProtocolVersions(GetProtocolVersionsRequestProto) 
+      returns (GetProtocolVersionsResponseProto);
+
+  /**
+   * Return protocol version corresponding to protocol interface.
+   */
+  rpc getProtocolSignature(GetProtocolSignatureRequestProto) 
+      returns (GetProtocolSignatureResponseProto);
+}
diff --git a/hadoop-common-project/hadoop-common/src/main/proto/hadoop_rpc.proto b/hadoop-common-project/hadoop-common/src/main/proto/hadoop_rpc.proto
new file mode 100644
index 0000000..7af8eae
--- /dev/null
+++ b/hadoop-common-project/hadoop-common/src/main/proto/hadoop_rpc.proto
@@ -0,0 +1,79 @@
+/**
+ * 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.
+ */
+
+/**
+ * These are the messages used by Hadoop RPC to marshal the
+ * request and response in the RPC layer.
+ */
+option java_package = "org.apache.hadoop.ipc.protobuf";
+option java_outer_classname = "HadoopRpcProtos";
+option java_generate_equals_and_hash = true;
+
+/**
+ * Message used to marshal the client request
+ * from RPC client to the RPC server.
+ */
+message HadoopRpcRequestProto {
+  /** Name of the RPC method */
+  required string methodName = 1;
+
+  /** Bytes corresponding to the client protobuf request */
+  optional bytes request = 2;
+  
+  /** protocol name of class declaring the called method */ 
+  required string declaringClassProtocolName = 3;
+  
+  /** protocol version of class declaring the called method */
+  required uint64 clientProtocolVersion = 4;
+}
+
+/**
+ * At the RPC layer, this message is used to indicate
+ * the server side exception the the RPC client.
+ *
+ * Hadoop RPC client throws an exception indicated
+ * by exceptionName with the stackTrace.
+ */
+message HadoopRpcExceptionProto {
+  /** Class name of the exception thrown from the server */
+
+  optional string exceptionName = 1;
+  /** Exception stack trace from the server side */
+  optional string stackTrace = 2;
+}
+
+/**
+ * This message is used to marshal the response from
+ * RPC server to the client.
+ */
+message HadoopRpcResponseProto {
+  /** Status of IPC call */
+  enum ResponseStatus {
+    SUCCESS = 1;
+    ERRROR = 2;
+  }
+
+  required ResponseStatus status = 1;
+
+  // Protobuf response payload from the server, when status is SUCCESS.
+  optional bytes response = 2;
+
+  // Exception when status is ERROR or FATAL
+  optional HadoopRpcExceptionProto exception = 3;
+}
+
diff --git a/hadoop-common-project/hadoop-common/src/test/avro/AvroSpecificTestProtocol.avpr b/hadoop-common-project/hadoop-common/src/test/avro/AvroSpecificTestProtocol.avpr
deleted file mode 100644
index 18960c1..0000000
--- a/hadoop-common-project/hadoop-common/src/test/avro/AvroSpecificTestProtocol.avpr
+++ /dev/null
@@ -1,42 +0,0 @@
-// Licensed to the Apache Software Foundation (ASF) under one
-// or more contributor license agreements.  See the NOTICE file
-// distributed with this work for additional information
-// regarding copyright ownership.  The ASF licenses this file
-// to you under the Apache License, Version 2.0 (the
-// "License"); you may not use this file except in compliance
-// with the License.  You may obtain a copy of the License at
-//
-//     http://www.apache.org/licenses/LICENSE-2.0
-//
-// Unless required by applicable law or agreed to in writing, software
-// distributed under the License is distributed on an "AS IS" BASIS,
-// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
-// See the License for the specific language governing permissions and
-// limitations under the License.
-
-{
-  "protocol" : "AvroSpecificTestProtocol",
-  "namespace" : "org.apache.hadoop.ipc",
-
-  "messages" : {
-    "echo" : {
-      "request" : [ {
-        "name" : "message",
-        "type" : "string"
-      } ],
-      "response" : "string"
-    },
-    
-    "add" : {
-      "request" : [ {
-        "name" : "arg1",
-        "type" : "int"
-      }, {
-        "name" : "arg2",
-        "type" : "int",
-        "default" : 0
-      } ],
-      "response" : "int"
-    }
-  }
-}
\ No newline at end of file
diff --git a/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/io/retry/TestFailoverProxy.java b/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/io/retry/TestFailoverProxy.java
index 4664ab3..295bf13 100644
--- a/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/io/retry/TestFailoverProxy.java
+++ b/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/io/retry/TestFailoverProxy.java
@@ -57,6 +57,11 @@
     public Class<?> getInterface() {
       return iface;
     }
+
+    @Override
+    public void close() throws IOException {
+      // Nothing to do.
+    }
     
   }
   
diff --git a/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/ipc/RPCCallBenchmark.java b/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/ipc/RPCCallBenchmark.java
new file mode 100644
index 0000000..5fe2302
--- /dev/null
+++ b/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/ipc/RPCCallBenchmark.java
@@ -0,0 +1,420 @@
+/**
+ * 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.ipc;
+
+import java.io.IOException;
+import java.lang.management.ManagementFactory;
+import java.lang.management.ThreadMXBean;
+import java.net.InetSocketAddress;
+import java.security.PrivilegedExceptionAction;
+import java.util.concurrent.atomic.AtomicLong;
+
+import org.apache.commons.cli.CommandLine;
+import org.apache.commons.cli.CommandLineParser;
+import org.apache.commons.cli.GnuParser;
+import org.apache.commons.cli.HelpFormatter;
+import org.apache.commons.cli.OptionBuilder;
+import org.apache.commons.cli.Options;
+import org.apache.commons.cli.ParseException;
+import org.apache.hadoop.conf.Configurable;
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.fs.CommonConfigurationKeys;
+import org.apache.hadoop.ipc.RPC.Server;
+import org.apache.hadoop.ipc.TestProtoBufRpc.PBServerImpl;
+import org.apache.hadoop.ipc.TestProtoBufRpc.TestRpcService;
+import org.apache.hadoop.ipc.TestRPC.TestProtocol;
+import org.apache.hadoop.ipc.protobuf.TestProtos.EchoRequestProto;
+import org.apache.hadoop.ipc.protobuf.TestProtos.EchoResponseProto;
+import org.apache.hadoop.ipc.protobuf.TestRpcServiceProtos.TestProtobufRpcProto;
+import org.apache.hadoop.net.NetUtils;
+import org.apache.hadoop.security.UserGroupInformation;
+import org.apache.hadoop.test.MultithreadedTestUtil;
+import org.apache.hadoop.test.MultithreadedTestUtil.TestContext;
+import org.apache.hadoop.util.Tool;
+import org.apache.hadoop.util.ToolRunner;
+
+import com.google.common.base.Joiner;
+import com.google.protobuf.BlockingService;
+
+/**
+ * Benchmark for protobuf RPC.
+ * Run with --help option for usage.
+ */
+public class RPCCallBenchmark implements Tool, Configurable {
+  private Configuration conf;
+  private AtomicLong callCount = new AtomicLong(0);
+  private static ThreadMXBean threadBean =
+    ManagementFactory.getThreadMXBean();
+  
+  private static class MyOptions {
+    private boolean failed = false;
+    private int serverThreads = 0;
+    private int serverReaderThreads = 1;
+    private int clientThreads = 0;
+    private String host = "0.0.0.0";
+    private int port = 12345;
+    public int secondsToRun = 15;
+    private int msgSize = 1024;
+    public Class<? extends RpcEngine> rpcEngine =
+      WritableRpcEngine.class;
+    
+    private MyOptions(String args[]) {
+      try {
+        Options opts = buildOptions();
+        CommandLineParser parser = new GnuParser();
+        CommandLine line = parser.parse(opts, args, true);
+        processOptions(line, opts);
+        validateOptions();
+      } catch (ParseException e) {
+        System.err.println(e.getMessage());
+        System.err.println("Try \"--help\" option for details.");
+        failed = true;
+      }
+    }
+
+    private void validateOptions() throws ParseException {
+      if (serverThreads <= 0 && clientThreads <= 0) {
+        throw new ParseException("Must specify at least -c or -s");
+      }
+    }
+
+    @SuppressWarnings("static-access")
+    private Options buildOptions() {
+      Options opts = new Options();
+      opts.addOption(
+        OptionBuilder.withLongOpt("serverThreads").hasArg(true)
+        .withArgName("numthreads")
+        .withDescription("number of server threads (handlers) to run (or 0 to not run server)")
+        .create("s"));
+      opts.addOption(
+        OptionBuilder.withLongOpt("serverReaderThreads").hasArg(true)
+        .withArgName("threads")
+        .withDescription("number of server reader threads to run")
+        .create("r"));
+
+      
+      opts.addOption(
+        OptionBuilder.withLongOpt("clientThreads").hasArg(true)
+        .withArgName("numthreads")
+        .withDescription("number of client threads to run (or 0 to not run client)")
+        .create("c"));
+
+      opts.addOption(
+        OptionBuilder.withLongOpt("messageSize").hasArg(true)
+        .withArgName("bytes")
+        .withDescription("size of call parameter in bytes")
+        .create("m"));
+
+      opts.addOption(
+          OptionBuilder.withLongOpt("time").hasArg(true)
+          .withArgName("seconds")
+          .withDescription("number of seconds to run clients for")
+          .create("t"));
+      opts.addOption(
+          OptionBuilder.withLongOpt("port").hasArg(true)
+          .withArgName("port")
+          .withDescription("port to listen or connect on")
+          .create("p"));
+      opts.addOption(
+          OptionBuilder.withLongOpt("host").hasArg(true)
+          .withArgName("addr")
+          .withDescription("host to listen or connect on")
+          .create('h'));
+      
+      opts.addOption(
+          OptionBuilder.withLongOpt("engine").hasArg(true)
+          .withArgName("writable|protobuf")
+          .withDescription("engine to use")
+          .create('e'));
+      
+      opts.addOption(
+          OptionBuilder.withLongOpt("help").hasArg(false)
+          .withDescription("show this screen")
+          .create('?'));
+
+      return opts;
+    }
+    
+    private void processOptions(CommandLine line, Options opts)
+      throws ParseException {
+      if (line.hasOption("help") || line.hasOption('?')) {
+        HelpFormatter formatter = new HelpFormatter();
+        System.out.println("Protobuf IPC benchmark.");
+        System.out.println();
+        formatter.printHelp(100,
+            "java ... PBRPCBenchmark [options]",
+            "\nSupported options:", opts, "");
+        return;
+      }
+
+      if (line.hasOption('s')) {
+        serverThreads = Integer.parseInt(line.getOptionValue('s'));
+      }
+      if (line.hasOption('r')) {
+        serverReaderThreads = Integer.parseInt(line.getOptionValue('r'));
+      }
+      if (line.hasOption('c')) {
+        clientThreads = Integer.parseInt(line.getOptionValue('c'));
+      }
+      if (line.hasOption('t')) {
+        secondsToRun = Integer.parseInt(line.getOptionValue('t'));
+      }
+      if (line.hasOption('m')) {
+        msgSize = Integer.parseInt(line.getOptionValue('m'));
+      }
+      if (line.hasOption('p')) {
+        port = Integer.parseInt(line.getOptionValue('p'));
+      }
+      if (line.hasOption('h')) {
+        host = line.getOptionValue('h');
+      }
+      if (line.hasOption('e')) {
+        String eng = line.getOptionValue('e');
+        if ("protobuf".equals(eng)) {
+          rpcEngine = ProtobufRpcEngine.class;
+        } else if ("writable".equals(eng)) {
+          rpcEngine = WritableRpcEngine.class;
+        } else {
+          throw new ParseException("invalid engine: " + eng);
+        }
+      }
+      
+      String[] remainingArgs = line.getArgs();
+      if (remainingArgs.length != 0) {
+        throw new ParseException("Extra arguments: " +
+            Joiner.on(" ").join(remainingArgs));
+      }
+    }
+    
+    @Override
+    public String toString() {
+      return "rpcEngine=" + rpcEngine + "\nserverThreads=" + serverThreads
+          + "\nserverReaderThreads=" + serverReaderThreads + "\nclientThreads="
+          + clientThreads + "\nhost=" + host + "\nport=" + port
+          + "\nsecondsToRun=" + secondsToRun + "\nmsgSize=" + msgSize;
+    }
+  }
+
+
+  
+  private Server startServer(MyOptions opts) throws IOException {
+    if (opts.serverThreads <= 0) {
+      return null;
+    }
+    conf.setInt(CommonConfigurationKeys.IPC_SERVER_RPC_READ_THREADS_KEY,
+        opts.serverReaderThreads);
+    
+    RPC.Server server;
+    // Get RPC server for server side implementation
+    if (opts.rpcEngine == ProtobufRpcEngine.class) {
+      // Create server side implementation
+      PBServerImpl serverImpl = new PBServerImpl();
+      BlockingService service = TestProtobufRpcProto
+          .newReflectiveBlockingService(serverImpl);
+
+      server = RPC.getServer(TestRpcService.class, service,
+          opts.host, opts.port, opts.serverThreads, false, conf, null);
+    } else if (opts.rpcEngine == WritableRpcEngine.class) {
+      server = RPC.getServer(TestProtocol.class, new TestRPC.TestImpl(),
+          opts.host, opts.port, opts.serverThreads, false, conf, null);
+    } else {
+      throw new RuntimeException("Bad engine: " + opts.rpcEngine);
+    }
+    server.start();
+    return server;
+  }
+  
+  private long getTotalCpuTime(Iterable<? extends Thread> threads) {
+    long total = 0;
+    for (Thread t : threads) {
+      long tid = t.getId();
+      total += threadBean.getThreadCpuTime(tid);
+    }
+    return total;
+  }
+  
+  @Override
+  public int run(String[] args) throws Exception {
+    MyOptions opts = new MyOptions(args);
+    if (opts.failed) {
+      return -1;
+    }
+    
+    // Set RPC engine to the configured RPC engine
+    RPC.setProtocolEngine(conf, TestRpcService.class, opts.rpcEngine);
+
+    Server server = startServer(opts);
+    try {
+      
+      TestContext ctx = setupClientTestContext(opts);
+      if (ctx != null) {
+        long totalCalls = 0;
+        ctx.startThreads();
+        long veryStart = System.nanoTime();
+
+        // Loop printing results every second until the specified
+        // time has elapsed
+        for (int i = 0; i < opts.secondsToRun ; i++) {
+          long st = System.nanoTime();
+          ctx.waitFor(1000);
+          long et = System.nanoTime();
+          long ct = callCount.getAndSet(0);
+          totalCalls += ct;
+          double callsPerSec = (ct * 1000000000)/(et - st);
+          System.out.println("Calls per second: " + callsPerSec);
+        }
+        
+        // Print results
+
+        if (totalCalls > 0) {
+          long veryEnd = System.nanoTime();
+          double callsPerSec =
+            (totalCalls * 1000000000)/(veryEnd - veryStart);
+          long cpuNanosClient = getTotalCpuTime(ctx.getTestThreads());
+          long cpuNanosServer = -1;
+          if (server != null) {
+            cpuNanosServer = getTotalCpuTime(server.getHandlers());; 
+          }
+          System.out.println("====== Results ======");
+          System.out.println("Options:\n" + opts);
+          System.out.println("Total calls per second: " + callsPerSec);
+          System.out.println("CPU time per call on client: " +
+              (cpuNanosClient / totalCalls) + " ns");
+          if (server != null) {
+            System.out.println("CPU time per call on server: " +
+                (cpuNanosServer / totalCalls) + " ns");
+          }
+        } else {
+          System.out.println("No calls!");
+        }
+
+        ctx.stop();
+      } else {
+        while (true) {
+          Thread.sleep(10000);
+        }
+      }
+    } finally {
+      if (server != null) {
+        server.stop();
+      }
+    }
+    
+    return 0;
+  }
+
+
+  private TestContext setupClientTestContext(final MyOptions opts)
+      throws IOException, InterruptedException {
+    if (opts.clientThreads <= 0) {
+      return null;
+    }
+
+    // Set up a separate proxy for each client thread,
+    // rather than making them share TCP pipes.
+    int numProxies = opts.clientThreads;
+    final RpcServiceWrapper proxies[] = new RpcServiceWrapper[numProxies];
+    for (int i = 0; i < numProxies; i++) {
+      proxies[i] =
+        UserGroupInformation.createUserForTesting("proxy-" + i,new String[]{})
+        .doAs(new PrivilegedExceptionAction<RpcServiceWrapper>() {
+          @Override
+          public RpcServiceWrapper run() throws Exception {
+            return createRpcClient(opts);
+          }
+        });
+    }
+
+    // Create an echo message of the desired length
+    final StringBuilder msgBuilder = new StringBuilder(opts.msgSize);
+    for (int c = 0; c < opts.msgSize; c++) {
+      msgBuilder.append('x');
+    }
+    final String echoMessage = msgBuilder.toString();
+
+    // Create the clients in a test context
+    TestContext ctx = new TestContext();
+    for (int i = 0; i < opts.clientThreads; i++) {
+      final RpcServiceWrapper proxy = proxies[i % numProxies];
+      
+      ctx.addThread(new MultithreadedTestUtil.RepeatingTestThread(ctx) {
+        @Override
+        public void doAnAction() throws Exception {
+          proxy.doEcho(echoMessage);
+          callCount.incrementAndGet();
+        }
+      });
+    }
+    return ctx;
+  }
+
+  /**
+   * Simple interface that can be implemented either by the
+   * protobuf or writable implementations.
+   */
+  private interface RpcServiceWrapper {
+    public String doEcho(String msg) throws Exception;
+  }
+
+  /**
+   * Create a client proxy for the specified engine.
+   */
+  private RpcServiceWrapper createRpcClient(MyOptions opts) throws IOException {
+    InetSocketAddress addr = NetUtils.createSocketAddr(opts.host, opts.port);
+    
+    if (opts.rpcEngine == ProtobufRpcEngine.class) {
+      final TestRpcService proxy = RPC.getProxy(TestRpcService.class, 0, addr, conf);
+      return new RpcServiceWrapper() {
+        @Override
+        public String doEcho(String msg) throws Exception {
+          EchoRequestProto req = EchoRequestProto.newBuilder()
+            .setMessage(msg)
+            .build();
+          EchoResponseProto responseProto = proxy.echo(null, req);
+          return responseProto.getMessage();
+        }
+      };
+    } else if (opts.rpcEngine == WritableRpcEngine.class) {
+      final TestProtocol proxy = (TestProtocol)RPC.getProxy(
+          TestProtocol.class, TestProtocol.versionID, addr, conf);
+      return new RpcServiceWrapper() {
+        @Override
+        public String doEcho(String msg) throws Exception {
+          return proxy.echo(msg);
+        }
+      };
+    } else {
+      throw new RuntimeException("unsupported engine: " + opts.rpcEngine);
+    }
+  }
+
+  public static void main(String []args) throws Exception {
+    int rc = ToolRunner.run(new RPCCallBenchmark(), args);
+    System.exit(rc);
+  }
+
+  @Override
+  public void setConf(Configuration conf) {
+    this.conf = conf;
+  }
+
+  @Override
+  public Configuration getConf() {
+    return conf;
+  }
+}
diff --git a/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/ipc/TestAvroRpc.java b/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/ipc/TestAvroRpc.java
deleted file mode 100644
index 7f7c510..0000000
--- a/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/ipc/TestAvroRpc.java
+++ /dev/null
@@ -1,219 +0,0 @@
-/**
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- *     http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-
-package org.apache.hadoop.ipc;
-
-import static org.apache.hadoop.fs.CommonConfigurationKeysPublic.HADOOP_SECURITY_AUTHENTICATION;
-
-import java.io.IOException;
-import java.net.InetSocketAddress;
-
-import javax.security.sasl.Sasl;
-
-import junit.framework.Assert;
-import junit.framework.TestCase;
-
-import org.apache.avro.AvroRemoteException;
-import org.apache.avro.util.Utf8;
-import org.apache.commons.logging.Log;
-import org.apache.commons.logging.LogFactory;
-import org.apache.hadoop.conf.Configuration;
-import org.apache.hadoop.io.Text;
-import org.apache.hadoop.ipc.TestSaslRPC.CustomSecurityInfo;
-import org.apache.hadoop.ipc.TestSaslRPC.TestTokenIdentifier;
-import org.apache.hadoop.ipc.TestSaslRPC.TestTokenSecretManager;
-import org.apache.hadoop.net.NetUtils;
-import org.apache.hadoop.security.SaslRpcServer;
-import org.apache.hadoop.security.SecurityInfo;
-import org.apache.hadoop.security.SecurityUtil;
-import org.apache.hadoop.security.UserGroupInformation;
-import org.apache.hadoop.security.token.Token;
-
-/** Unit tests for AvroRpc. */
-public class TestAvroRpc extends TestCase {
-  private static final String ADDRESS = "0.0.0.0";
-
-  public static final Log LOG =
-    LogFactory.getLog(TestAvroRpc.class);
-  
-  int datasize = 1024*100;
-  int numThreads = 50;
-
-  public TestAvroRpc(String name) { super(name); }
-	
-  public static class TestImpl implements AvroTestProtocol {
-
-    public void ping() {}
-    
-    public String echo(String value) { return value; }
-
-    public int add(int v1, int v2) { return v1 + v2; }
-
-    public int error() throws Problem {
-      throw new Problem();
-    }
-  }
-
-  public void testReflect() throws Exception {
-    testReflect(false);
-  }
-
-  public void testSecureReflect() throws Exception {
-    testReflect(true);
-  }
-
-  public void testSpecific() throws Exception {
-    testSpecific(false);
-  }
-
-  public void testSecureSpecific() throws Exception {
-    testSpecific(true);
-  }
-
-  private void testReflect(boolean secure) throws Exception {
-    Configuration conf = new Configuration();
-    TestTokenSecretManager sm = null;
-    if (secure) {
-      makeSecure(conf);
-      sm = new TestTokenSecretManager();
-    }
-    UserGroupInformation.setConfiguration(conf);
-    RPC.setProtocolEngine(conf, AvroTestProtocol.class, AvroRpcEngine.class);
-    Server server = RPC.getServer(AvroTestProtocol.class,
-                                  new TestImpl(), ADDRESS, 0, 5, true, 
-                                  conf, sm);
-    try {
-      server.start();
-      InetSocketAddress addr = NetUtils.getConnectAddress(server);
-
-      if (secure) {
-        addToken(sm, addr);
-        //QOP must be auth
-        Assert.assertEquals("auth", SaslRpcServer.SASL_PROPS.get(Sasl.QOP));
-      }
-
-      AvroTestProtocol proxy =
-        (AvroTestProtocol)RPC.getProxy(AvroTestProtocol.class, 0, addr, conf);
-
-      proxy.ping();
-
-      String echo = proxy.echo("hello world");
-      assertEquals("hello world", echo);
-
-      int intResult = proxy.add(1, 2);
-      assertEquals(3, intResult);
-
-      boolean caught = false;
-      try {
-        proxy.error();
-      } catch (AvroRemoteException e) {
-        if(LOG.isDebugEnabled()) {
-          LOG.debug("Caught " + e);
-        }
-        caught = true;
-      }
-      assertTrue(caught);
-
-    } finally {
-      resetSecurity();
-      server.stop();
-    }
-  }
-
-  private void makeSecure(Configuration conf) {
-    conf.set(HADOOP_SECURITY_AUTHENTICATION, "kerberos");
-    conf.set("hadoop.rpc.socket.factory.class.default", "");
-    //Avro doesn't work with security annotations on protocol.
-    //Avro works ONLY with custom security context
-    SecurityUtil.setSecurityInfoProviders(new CustomSecurityInfo());
-  }
-  
-  private void resetSecurity() {
-    SecurityUtil.setSecurityInfoProviders(new SecurityInfo[0]);
-  }
-
-  private void addToken(TestTokenSecretManager sm, 
-      InetSocketAddress addr) throws IOException {
-    final UserGroupInformation current = UserGroupInformation.getCurrentUser();
-    
-    TestTokenIdentifier tokenId = new TestTokenIdentifier(new Text(current
-        .getUserName()));
-    Token<TestTokenIdentifier> token = new Token<TestTokenIdentifier>(tokenId,
-        sm);
-    Text host = new Text(addr.getAddress().getHostAddress() + ":"
-        + addr.getPort());
-    token.setService(host);
-    LOG.info("Service IP address for token is " + host);
-    current.addToken(token);
-  }
-
-  private void testSpecific(boolean secure) throws Exception {
-    Configuration conf = new Configuration();
-    TestTokenSecretManager sm = null;
-    if (secure) {
-      makeSecure(conf);
-      sm = new TestTokenSecretManager();
-    }
-    UserGroupInformation.setConfiguration(conf);
-    RPC.setProtocolEngine(conf, AvroSpecificTestProtocol.class, 
-        AvroSpecificRpcEngine.class);
-    Server server = RPC.getServer(AvroSpecificTestProtocol.class,
-        new AvroSpecificTestProtocolImpl(), ADDRESS, 0, 5, true, 
-        conf, sm);
-    try {
-      server.start();
-      InetSocketAddress addr = NetUtils.getConnectAddress(server);
-
-      if (secure) {
-        addToken(sm, addr);
-        //QOP must be auth
-        Assert.assertEquals("auth", SaslRpcServer.SASL_PROPS.get(Sasl.QOP));
-      }
-
-      AvroSpecificTestProtocol proxy =
-        (AvroSpecificTestProtocol)RPC.getProxy(AvroSpecificTestProtocol.class, 
-            0, addr, conf);
-      
-      CharSequence echo = proxy.echo("hello world");
-      assertEquals("hello world", echo.toString());
-
-      int intResult = proxy.add(1, 2);
-      assertEquals(3, intResult);
-
-    } finally {
-      resetSecurity();
-      server.stop();
-    }
-  }
-  
-  public static class AvroSpecificTestProtocolImpl implements 
-      AvroSpecificTestProtocol {
-
-    @Override
-    public int add(int arg1, int arg2) throws AvroRemoteException {
-      return arg1 + arg2;
-    }
-
-    @Override
-    public CharSequence echo(CharSequence msg) throws AvroRemoteException {
-      return msg;
-    }
-    
-  }
-
-}
diff --git a/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/ipc/TestIPC.java b/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/ipc/TestIPC.java
index 72409d5..e50d644 100644
--- a/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/ipc/TestIPC.java
+++ b/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/ipc/TestIPC.java
@@ -23,6 +23,7 @@
 import org.apache.hadoop.io.IOUtils;
 import org.apache.hadoop.io.Writable;
 import org.apache.hadoop.io.LongWritable;
+import org.apache.hadoop.ipc.RpcPayloadHeader.RpcKind;
 import org.apache.hadoop.util.StringUtils;
 import org.apache.hadoop.net.NetUtils;
 
@@ -96,8 +97,8 @@
     }
 
     @Override
-    public Writable call(Class<?> protocol, Writable param, long receiveTime)
-        throws IOException {
+    public Writable call(RpcKind rpcKind, String protocol, Writable param,
+        long receiveTime) throws IOException {
       if (sleep) {
         // sleep a bit
         try {
diff --git a/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/ipc/TestIPCServerResponder.java b/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/ipc/TestIPCServerResponder.java
index 3710198..5675cbf 100644
--- a/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/ipc/TestIPCServerResponder.java
+++ b/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/ipc/TestIPCServerResponder.java
@@ -30,6 +30,7 @@
 import org.apache.hadoop.fs.CommonConfigurationKeys;
 import org.apache.hadoop.io.BytesWritable;
 import org.apache.hadoop.io.Writable;
+import org.apache.hadoop.ipc.RpcPayloadHeader.RpcKind;
 import org.apache.hadoop.net.NetUtils;
 
 /**
@@ -72,8 +73,8 @@
     }
 
     @Override
-    public Writable call(Class<?> protocol, Writable param, long receiveTime)
-        throws IOException {
+    public Writable call(RpcKind rpcKind, String protocol, Writable param,
+        long receiveTime) throws IOException {
       if (sleep) {
         try {
           Thread.sleep(RANDOM.nextInt(20)); // sleep a bit
diff --git a/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/ipc/TestMultipleProtocolServer.java b/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/ipc/TestMultipleProtocolServer.java
new file mode 100644
index 0000000..f5acd93
--- /dev/null
+++ b/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/ipc/TestMultipleProtocolServer.java
@@ -0,0 +1,281 @@
+/**
+ * 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.ipc;
+
+import java.io.IOException;
+import java.net.InetSocketAddress;
+
+import org.junit.Assert;
+
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.ipc.RpcPayloadHeader.RpcKind;
+import org.apache.hadoop.ipc.TestProtoBufRpc.PBServerImpl;
+import org.apache.hadoop.ipc.TestProtoBufRpc.TestRpcService;
+import org.apache.hadoop.ipc.protobuf.TestRpcServiceProtos.TestProtobufRpcProto;
+import org.apache.hadoop.net.NetUtils;
+import org.junit.Before;
+import org.junit.After;
+import org.junit.Test;
+import com.google.protobuf.BlockingService;
+
+public class TestMultipleProtocolServer {
+  private static final String ADDRESS = "0.0.0.0";
+  private static InetSocketAddress addr;
+  private static RPC.Server server;
+
+  private static Configuration conf = new Configuration();
+  
+  
+  @ProtocolInfo(protocolName="Foo")
+  interface Foo0 extends VersionedProtocol {
+    public static final long versionID = 0L;
+    String ping() throws IOException;
+    
+  }
+  
+  @ProtocolInfo(protocolName="Foo")
+  interface Foo1 extends VersionedProtocol {
+    public static final long versionID = 1L;
+    String ping() throws IOException;
+    String ping2() throws IOException;
+  }
+  
+  @ProtocolInfo(protocolName="Foo")
+  interface FooUnimplemented extends VersionedProtocol {
+    public static final long versionID = 2L;
+    String ping() throws IOException;  
+  }
+  
+  interface Mixin extends VersionedProtocol{
+    public static final long versionID = 0L;
+    void hello() throws IOException;
+  }
+  interface Bar extends Mixin, VersionedProtocol {
+    public static final long versionID = 0L;
+    int echo(int i) throws IOException;
+  }
+  
+  
+  
+  class Foo0Impl implements Foo0 {
+
+    @Override
+    public long getProtocolVersion(String protocol, long clientVersion)
+        throws IOException {
+      return Foo0.versionID;
+    }
+
+    @SuppressWarnings("unchecked")
+    @Override
+    public ProtocolSignature getProtocolSignature(String protocol,
+        long clientVersion, int clientMethodsHash) throws IOException {
+      Class<? extends VersionedProtocol> inter;
+      try {
+        inter = (Class<? extends VersionedProtocol>)getClass().
+                                          getGenericInterfaces()[0];
+      } catch (Exception e) {
+        throw new IOException(e);
+      }
+      return ProtocolSignature.getProtocolSignature(clientMethodsHash, 
+          getProtocolVersion(protocol, clientVersion), inter);
+    }
+
+    @Override
+    public String ping() {
+      return "Foo0";     
+    }
+    
+  }
+  
+  class Foo1Impl implements Foo1 {
+
+    @Override
+    public long getProtocolVersion(String protocol, long clientVersion)
+        throws IOException {
+      return Foo1.versionID;
+    }
+
+    @SuppressWarnings("unchecked")
+    @Override
+    public ProtocolSignature getProtocolSignature(String protocol,
+        long clientVersion, int clientMethodsHash) throws IOException {
+      Class<? extends VersionedProtocol> inter;
+      try {
+        inter = (Class<? extends VersionedProtocol>)getClass().
+                                        getGenericInterfaces()[0];
+      } catch (Exception e) {
+        throw new IOException(e);
+      }
+      return ProtocolSignature.getProtocolSignature(clientMethodsHash, 
+          getProtocolVersion(protocol, clientVersion), inter);
+    }
+
+    @Override
+    public String ping() {
+      return "Foo1";
+    }
+
+    @Override
+    public String ping2() {
+      return "Foo1";
+      
+    }
+    
+  }
+
+  
+  class BarImpl implements Bar {
+
+    @Override
+    public long getProtocolVersion(String protocol, long clientVersion)
+        throws IOException {
+      return Bar.versionID;
+    }
+
+    @SuppressWarnings("unchecked")
+    @Override
+    public ProtocolSignature getProtocolSignature(String protocol,
+        long clientVersion, int clientMethodsHash) throws IOException {
+      Class<? extends VersionedProtocol> inter;
+      try {
+        inter = (Class<? extends VersionedProtocol>)getClass().
+                                          getGenericInterfaces()[0];
+      } catch (Exception e) {
+        throw new IOException(e);
+      }
+      return ProtocolSignature.getProtocolSignature(clientMethodsHash, 
+          getProtocolVersion(protocol, clientVersion), inter);
+    }
+
+    @Override
+    public int echo(int i) {
+      return i;
+    }
+
+    @Override
+    public void hello() {
+
+      
+    }
+  }
+  @Before
+  public void setUp() throws Exception {
+    // create a server with two handlers
+    server = RPC.getServer(Foo0.class,
+                              new Foo0Impl(), ADDRESS, 0, 2, false, conf, null);
+    server.addProtocol(RpcKind.RPC_WRITABLE, Foo1.class, new Foo1Impl());
+    server.addProtocol(RpcKind.RPC_WRITABLE, Bar.class, new BarImpl());
+    server.addProtocol(RpcKind.RPC_WRITABLE, Mixin.class, new BarImpl());
+    
+    
+    // Add Protobuf server
+    // Create server side implementation
+    PBServerImpl pbServerImpl = 
+        new PBServerImpl();
+    BlockingService service = TestProtobufRpcProto
+        .newReflectiveBlockingService(pbServerImpl);
+    server.addProtocol(RpcKind.RPC_PROTOCOL_BUFFER, TestRpcService.class,
+        service);
+    server.start();
+    addr = NetUtils.getConnectAddress(server);
+  }
+  
+  @After
+  public void tearDown() throws Exception {
+    server.stop();
+  }
+
+  @Test
+  public void test1() throws IOException {
+    ProtocolProxy<?> proxy;
+    proxy = RPC.getProtocolProxy(Foo0.class, Foo0.versionID, addr, conf);
+
+    Foo0 foo0 = (Foo0)proxy.getProxy(); 
+    Assert.assertEquals("Foo0", foo0.ping());
+    
+    
+    proxy = RPC.getProtocolProxy(Foo1.class, Foo1.versionID, addr, conf);
+    
+    
+    Foo1 foo1 = (Foo1)proxy.getProxy(); 
+    Assert.assertEquals("Foo1", foo1.ping());
+    Assert.assertEquals("Foo1", foo1.ping());
+    
+    
+    proxy = RPC.getProtocolProxy(Bar.class, Foo1.versionID, addr, conf);
+    
+    
+    Bar bar = (Bar)proxy.getProxy(); 
+    Assert.assertEquals(99, bar.echo(99));
+    
+    // Now test Mixin class method
+    
+    Mixin mixin = bar;
+    mixin.hello();
+  }
+  
+  
+  // Server does not implement the FooUnimplemented version of protocol Foo.
+  // See that calls to it fail.
+  @Test(expected=IOException.class)
+  public void testNonExistingProtocol() throws IOException {
+    ProtocolProxy<?> proxy;
+    proxy = RPC.getProtocolProxy(FooUnimplemented.class, 
+        FooUnimplemented.versionID, addr, conf);
+
+    FooUnimplemented foo = (FooUnimplemented)proxy.getProxy(); 
+    foo.ping();
+  }
+  
+  
+  /**
+   * getProtocolVersion of an unimplemented version should return highest version
+   * Similarly getProtocolSignature should work.
+   * @throws IOException
+   */
+  @Test
+  public void testNonExistingProtocol2() throws IOException {
+    ProtocolProxy<?> proxy;
+    proxy = RPC.getProtocolProxy(FooUnimplemented.class, 
+        FooUnimplemented.versionID, addr, conf);
+
+    FooUnimplemented foo = (FooUnimplemented)proxy.getProxy(); 
+    Assert.assertEquals(Foo1.versionID, 
+        foo.getProtocolVersion(RPC.getProtocolName(FooUnimplemented.class), 
+        FooUnimplemented.versionID));
+    foo.getProtocolSignature(RPC.getProtocolName(FooUnimplemented.class), 
+        FooUnimplemented.versionID, 0);
+  }
+  
+  @Test(expected=IOException.class)
+  public void testIncorrectServerCreation() throws IOException {
+    RPC.getServer(Foo1.class,
+        new Foo0Impl(), ADDRESS, 0, 2, false, conf, null);
+  } 
+  
+  // Now test a PB service - a server  hosts both PB and Writable Rpcs.
+  @Test
+  public void testPBService() throws Exception {
+    // Set RPC engine to protobuf RPC engine
+    Configuration conf2 = new Configuration();
+    RPC.setProtocolEngine(conf2, TestRpcService.class,
+        ProtobufRpcEngine.class);
+    TestRpcService client = RPC.getProxy(TestRpcService.class, 0, addr, conf2);
+    TestProtoBufRpc.testProtoBufRpc(client);
+  }
+}
diff --git a/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/ipc/TestProtoBufRpc.java b/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/ipc/TestProtoBufRpc.java
new file mode 100644
index 0000000..3b9140a
--- /dev/null
+++ b/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/ipc/TestProtoBufRpc.java
@@ -0,0 +1,204 @@
+/**
+ * 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.ipc;
+
+import static org.apache.hadoop.test.MetricsAsserts.getMetrics;
+import static org.apache.hadoop.test.MetricsAsserts.assertCounterGt;
+
+import java.io.IOException;
+import java.net.InetSocketAddress;
+
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.ipc.RpcPayloadHeader.RpcKind;
+import org.apache.hadoop.ipc.protobuf.TestProtos.EchoRequestProto;
+import org.apache.hadoop.ipc.protobuf.TestProtos.EchoResponseProto;
+import org.apache.hadoop.ipc.protobuf.TestProtos.EmptyRequestProto;
+import org.apache.hadoop.ipc.protobuf.TestProtos.EmptyResponseProto;
+import org.apache.hadoop.ipc.protobuf.TestRpcServiceProtos.TestProtobufRpcProto;
+import org.apache.hadoop.ipc.protobuf.TestRpcServiceProtos.TestProtobufRpc2Proto;
+import org.apache.hadoop.metrics2.MetricsRecordBuilder;
+import org.apache.hadoop.net.NetUtils;
+import org.junit.Assert;
+import org.junit.Test;
+import org.junit.Before;
+import org.junit.After;
+
+import com.google.protobuf.BlockingService;
+import com.google.protobuf.RpcController;
+import com.google.protobuf.ServiceException;
+
+/**
+ * Test for testing protocol buffer based RPC mechanism.
+ * This test depends on test.proto definition of types in src/test/proto
+ * and protobuf service definition from src/test/test_rpc_service.proto
+ */
+public class TestProtoBufRpc {
+  public final static String ADDRESS = "0.0.0.0";
+  public final static int PORT = 0;
+  private static InetSocketAddress addr;
+  private static Configuration conf;
+  private static RPC.Server server;
+  
+  @ProtocolInfo(protocolName = "testProto", protocolVersion = 1)
+  public interface TestRpcService
+      extends TestProtobufRpcProto.BlockingInterface {
+  }
+
+  @ProtocolInfo(protocolName = "testProto2", protocolVersion = 1)
+  public interface TestRpcService2 extends
+      TestProtobufRpc2Proto.BlockingInterface {
+  }
+
+  public static class PBServerImpl implements TestRpcService {
+
+    @Override
+    public EmptyResponseProto ping(RpcController unused,
+        EmptyRequestProto request) throws ServiceException {
+      return EmptyResponseProto.newBuilder().build();
+    }
+
+    @Override
+    public EchoResponseProto echo(RpcController unused, EchoRequestProto request)
+        throws ServiceException {
+      return EchoResponseProto.newBuilder().setMessage(request.getMessage())
+          .build();
+    }
+
+    @Override
+    public EmptyResponseProto error(RpcController unused,
+        EmptyRequestProto request) throws ServiceException {
+      throw new ServiceException("error", new RpcServerException("error"));
+    }
+  }
+  
+  public static class PBServer2Impl implements TestRpcService2 {
+
+    @Override
+    public EmptyResponseProto ping2(RpcController unused,
+        EmptyRequestProto request) throws ServiceException {
+      return EmptyResponseProto.newBuilder().build();
+    }
+
+    @Override
+    public EchoResponseProto echo2(RpcController unused, EchoRequestProto request)
+        throws ServiceException {
+      return EchoResponseProto.newBuilder().setMessage(request.getMessage())
+          .build();
+    }
+  }
+
+  @Before
+  public  void setUp() throws IOException { // Setup server for both protocols
+    conf = new Configuration();
+    // Set RPC engine to protobuf RPC engine
+    RPC.setProtocolEngine(conf, TestRpcService.class, ProtobufRpcEngine.class);
+
+    // Create server side implementation
+    PBServerImpl serverImpl = new PBServerImpl();
+    BlockingService service = TestProtobufRpcProto
+        .newReflectiveBlockingService(serverImpl);
+
+    // Get RPC server for server side implementation
+    server = RPC.getServer(TestRpcService.class, service, ADDRESS, PORT, conf);
+    addr = NetUtils.getConnectAddress(server);
+    
+    // now the second protocol
+    PBServer2Impl server2Impl = new PBServer2Impl();
+    BlockingService service2 = TestProtobufRpc2Proto
+        .newReflectiveBlockingService(server2Impl);
+    
+    server.addProtocol(RpcKind.RPC_PROTOCOL_BUFFER, TestRpcService2.class,
+        service2);
+    server.start();
+  }
+  
+  
+  @After
+  public void tearDown() throws Exception {
+    server.stop();
+  }
+
+  private static TestRpcService getClient() throws IOException {
+    // Set RPC engine to protobuf RPC engine
+    RPC.setProtocolEngine(conf, TestRpcService.class,
+        ProtobufRpcEngine.class);
+        return RPC.getProxy(TestRpcService.class, 0, addr,
+        conf);
+  }
+  
+  private static TestRpcService2 getClient2() throws IOException {
+    // Set RPC engine to protobuf RPC engine
+    RPC.setProtocolEngine(conf, TestRpcService2.class,
+        ProtobufRpcEngine.class);
+        return RPC.getProxy(TestRpcService2.class, 0, addr,
+        conf);
+  }
+
+  @Test
+  public void testProtoBufRpc() throws Exception {
+    TestRpcService client = getClient();
+    testProtoBufRpc(client);
+  }
+  
+  // separated test out so that other tests can call it.
+  public static void testProtoBufRpc(TestRpcService client) throws Exception {  
+    // Test ping method
+    EmptyRequestProto emptyRequest = EmptyRequestProto.newBuilder().build();
+    client.ping(null, emptyRequest);
+    
+    // Test echo method
+    EchoRequestProto echoRequest = EchoRequestProto.newBuilder()
+        .setMessage("hello").build();
+    EchoResponseProto echoResponse = client.echo(null, echoRequest);
+    Assert.assertEquals(echoResponse.getMessage(), "hello");
+    
+    // Test error method - error should be thrown as RemoteException
+    try {
+      client.error(null, emptyRequest);
+      Assert.fail("Expected exception is not thrown");
+    } catch (ServiceException e) {
+      RemoteException re = (RemoteException)e.getCause();
+      RpcServerException rse = (RpcServerException) re
+          .unwrapRemoteException(RpcServerException.class);
+    }
+  }
+  
+  @Test
+  public void testProtoBufRpc2() throws Exception {
+    TestRpcService2 client = getClient2();
+    
+    // Test ping method
+    EmptyRequestProto emptyRequest = EmptyRequestProto.newBuilder().build();
+    client.ping2(null, emptyRequest);
+    
+    // Test echo method
+    EchoRequestProto echoRequest = EchoRequestProto.newBuilder()
+        .setMessage("hello").build();
+    EchoResponseProto echoResponse = client.echo2(null, echoRequest);
+    Assert.assertEquals(echoResponse.getMessage(), "hello");
+    
+    // Ensure RPC metrics are updated
+    MetricsRecordBuilder rpcMetrics = getMetrics(server.getRpcMetrics().name());
+    assertCounterGt("RpcQueueTimeNumOps", 0L, rpcMetrics);
+    assertCounterGt("RpcProcessingTimeNumOps", 0L, rpcMetrics);
+    
+    MetricsRecordBuilder rpcDetailedMetrics = 
+        getMetrics(server.getRpcDetailedMetrics().name());
+    assertCounterGt("Echo2NumOps", 0L, rpcDetailedMetrics);
+  }
+}
\ No newline at end of file
diff --git a/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/ipc/TestRPC.java b/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/ipc/TestRPC.java
index d144349..49e1ed6 100644
--- a/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/ipc/TestRPC.java
+++ b/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/ipc/TestRPC.java
@@ -18,28 +18,39 @@
 
 package org.apache.hadoop.ipc;
 
+import java.io.Closeable;
 import java.io.IOException;
 import java.net.ConnectException;
 import java.net.InetSocketAddress;
 import java.lang.management.ManagementFactory;
 import java.lang.management.ThreadInfo;
 import java.lang.management.ThreadMXBean;
+import java.lang.reflect.InvocationHandler;
 import java.lang.reflect.Method;
+import java.lang.reflect.Proxy;
 import java.util.Arrays;
 
-import junit.framework.TestCase;
+import javax.net.SocketFactory;
 
 import org.apache.commons.logging.*;
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.fs.CommonConfigurationKeys;
 import org.apache.hadoop.io.UTF8;
 import org.apache.hadoop.io.Writable;
+import org.apache.hadoop.io.retry.RetryPolicies;
+import org.apache.hadoop.io.retry.RetryProxy;
+import org.apache.hadoop.ipc.Client.ConnectionId;
 import org.apache.hadoop.metrics2.MetricsRecordBuilder;
 import org.apache.hadoop.net.NetUtils;
 import org.apache.hadoop.security.authorize.AuthorizationException;
 import org.apache.hadoop.security.authorize.PolicyProvider;
 import org.apache.hadoop.security.authorize.Service;
+import org.apache.hadoop.security.token.SecretManager;
+import org.apache.hadoop.security.token.TokenIdentifier;
 import org.apache.hadoop.security.AccessControlException;
+import org.apache.hadoop.security.UserGroupInformation;
+import org.junit.Test;
+import static org.junit.Assert.*;
 
 import com.google.protobuf.DescriptorProtos;
 import com.google.protobuf.DescriptorProtos.EnumDescriptorProto;
@@ -49,18 +60,22 @@
 import static org.mockito.Mockito.*;
 
 /** Unit tests for RPC. */
-public class TestRPC extends TestCase {
+@SuppressWarnings("deprecation")
+public class TestRPC {
   private static final String ADDRESS = "0.0.0.0";
 
   public static final Log LOG =
     LogFactory.getLog(TestRPC.class);
   
   private static Configuration conf = new Configuration();
+  
+  static {
+    conf.setClass("rpc.engine." + StoppedProtocol.class.getName(),
+        StoppedRpcEngine.class, RpcEngine.class);
+  }
 
   int datasize = 1024*100;
   int numThreads = 50;
-
-  public TestRPC(String name) { super(name); }
 	
   public interface TestProtocol extends VersionedProtocol {
     public static final long versionID = 1L;
@@ -207,6 +222,80 @@
     }
   }
   
+  /**
+   * A basic interface for testing client-side RPC resource cleanup.
+   */
+  private static interface StoppedProtocol {
+    long versionID = 0;
+
+    public void stop();
+  }
+  
+  /**
+   * A class used for testing cleanup of client side RPC resources.
+   */
+  private static class StoppedRpcEngine implements RpcEngine {
+
+    @Override
+    public Object[] call(Method method, Object[][] params, InetSocketAddress[] addrs,
+        UserGroupInformation ticket, Configuration conf)
+        throws IOException, InterruptedException {
+      return null;
+    }
+
+    @SuppressWarnings("unchecked")
+    @Override
+    public <T> ProtocolProxy<T> getProxy(Class<T> protocol, long clientVersion,
+        InetSocketAddress addr, UserGroupInformation ticket, Configuration conf,
+        SocketFactory factory, int rpcTimeout) throws IOException {
+      T proxy = (T) Proxy.newProxyInstance(protocol.getClassLoader(),
+              new Class[] { protocol }, new StoppedInvocationHandler());
+      return new ProtocolProxy<T>(protocol, proxy, false);
+    }
+
+    @Override
+    public org.apache.hadoop.ipc.RPC.Server getServer(Class<?> protocol,
+        Object instance, String bindAddress, int port, int numHandlers,
+        int numReaders, int queueSizePerHandler, boolean verbose, Configuration conf,
+        SecretManager<? extends TokenIdentifier> secretManager) throws IOException {
+      return null;
+    }
+
+    @Override
+    public ProtocolProxy<ProtocolMetaInfoPB> getProtocolMetaInfoProxy(
+        ConnectionId connId, Configuration conf, SocketFactory factory)
+        throws IOException {
+      throw new UnsupportedOperationException("This proxy is not supported");
+    }
+  }
+
+  /**
+   * An invocation handler which does nothing when invoking methods, and just
+   * counts the number of times close() is called.
+   */
+  private static class StoppedInvocationHandler
+      implements InvocationHandler, Closeable {
+    
+    private int closeCalled = 0;
+
+    @Override
+    public Object invoke(Object proxy, Method method, Object[] args)
+        throws Throwable {
+          return null;
+    }
+
+    @Override
+    public void close() throws IOException {
+      closeCalled++;
+    }
+    
+    public int getCloseCalled() {
+      return closeCalled;
+    }
+    
+  }
+  
+  @Test
   public void testConfRpc() throws Exception {
     Server server = RPC.getServer(TestProtocol.class,
                                   new TestImpl(), ADDRESS, 0, 1, false, conf, null);
@@ -229,6 +318,7 @@
     server.stop();    
   }
 
+  @Test
   public void testSlowRpc() throws Exception {
     System.out.println("Testing Slow RPC");
     // create a server with two handlers
@@ -273,11 +363,12 @@
     }
   }
   
-  public void testRPCConf(Configuration conf) throws Exception {
-    
+  @Test
+  public void testCalls() throws Exception {
+    testCallsInternal(conf);
   }
-
-  public void testCalls(Configuration conf) throws Exception {
+  
+  private void testCallsInternal(Configuration conf) throws Exception {
     Server server = RPC.getServer(TestProtocol.class,
                                   new TestImpl(), ADDRESS, 0, conf);
     TestProtocol proxy = null;
@@ -384,6 +475,7 @@
     }
   }
   
+  @Test
   public void testStandaloneClient() throws IOException {
     try {
       TestProtocol proxy = RPC.waitForProxy(TestProtocol.class,
@@ -450,6 +542,7 @@
     }
   }
   
+  @Test
   public void testAuthorization() throws Exception {
     Configuration conf = new Configuration();
     conf.setBoolean(CommonConfigurationKeys.HADOOP_SECURITY_AUTHORIZATION,
@@ -481,20 +574,48 @@
     Configuration conf = new Configuration();
     
     conf.setBoolean("ipc.client.ping", false);
-    new TestRPC("testnoPings").testCalls(conf);
+    new TestRPC().testCallsInternal(conf);
     
     conf.setInt(CommonConfigurationKeys.IPC_SERVER_RPC_READ_THREADS_KEY, 2);
-    new TestRPC("testnoPings").testCalls(conf);
+    new TestRPC().testCallsInternal(conf);
   }
 
   /**
    * Test stopping a non-registered proxy
    * @throws Exception
    */
+  @Test
   public void testStopNonRegisteredProxy() throws Exception {
     RPC.stopProxy(mock(TestProtocol.class));
   }
   
+  @Test
+  public void testStopProxy() throws IOException {
+    StoppedProtocol proxy = (StoppedProtocol) RPC.getProxy(StoppedProtocol.class,
+        StoppedProtocol.versionID, null, conf);
+    StoppedInvocationHandler invocationHandler = (StoppedInvocationHandler)
+        Proxy.getInvocationHandler(proxy);
+    assertEquals(invocationHandler.getCloseCalled(), 0);
+    RPC.stopProxy(proxy);
+    assertEquals(invocationHandler.getCloseCalled(), 1);
+  }
+  
+  @Test
+  public void testWrappedStopProxy() throws IOException {
+    StoppedProtocol wrappedProxy = (StoppedProtocol) RPC.getProxy(StoppedProtocol.class,
+        StoppedProtocol.versionID, null, conf);
+    StoppedInvocationHandler invocationHandler = (StoppedInvocationHandler)
+        Proxy.getInvocationHandler(wrappedProxy);
+    
+    StoppedProtocol proxy = (StoppedProtocol) RetryProxy.create(StoppedProtocol.class,
+        wrappedProxy, RetryPolicies.RETRY_FOREVER);
+    
+    assertEquals(invocationHandler.getCloseCalled(), 0);
+    RPC.stopProxy(proxy);
+    assertEquals(invocationHandler.getCloseCalled(), 1);
+  }
+  
+  @Test
   public void testErrorMsgForInsecureClient() throws Exception {
     final Server server = RPC.getServer(TestProtocol.class,
         new TestImpl(), ADDRESS, 0, 5, true, conf, null);
@@ -567,10 +688,10 @@
     return count;
   }
 
-
   /**
    * Test that server.stop() properly stops all threads
    */
+  @Test
   public void testStopsAllThreads() throws Exception {
     int threadsBefore = countThreads("Server$Listener$Reader");
     assertEquals("Expect no Reader threads running before test",
@@ -591,8 +712,7 @@
   }
   
   public static void main(String[] args) throws Exception {
-
-    new TestRPC("test").testCalls(conf);
+    new TestRPC().testCallsInternal(conf);
 
   }
 }
diff --git a/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/ipc/TestRPCCallBenchmark.java b/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/ipc/TestRPCCallBenchmark.java
new file mode 100644
index 0000000..969f728
--- /dev/null
+++ b/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/ipc/TestRPCCallBenchmark.java
@@ -0,0 +1,53 @@
+/**
+ * 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.ipc;
+
+import static org.junit.Assert.*;
+
+import org.apache.hadoop.util.ToolRunner;
+import org.junit.Test;
+
+
+public class TestRPCCallBenchmark {
+
+  @Test(timeout=20000)
+  public void testBenchmarkWithWritable() throws Exception {
+    int rc = ToolRunner.run(new RPCCallBenchmark(),
+        new String[] {
+      "--clientThreads", "30",
+      "--serverThreads", "30",
+      "--time", "5",
+      "--serverReaderThreads", "4",
+      "--messageSize", "1024",
+      "--engine", "writable"});
+    assertEquals(0, rc);
+  }
+  
+  @Test(timeout=20000)
+  public void testBenchmarkWithProto() throws Exception {
+    int rc = ToolRunner.run(new RPCCallBenchmark(),
+        new String[] {
+      "--clientThreads", "30",
+      "--serverThreads", "30",
+      "--time", "5",
+      "--serverReaderThreads", "4",
+      "--messageSize", "1024",
+      "--engine", "protobuf"});
+    assertEquals(0, rc);
+  }
+}
diff --git a/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/ipc/TestRPCCompatibility.java b/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/ipc/TestRPCCompatibility.java
index 02ca2af..aca33ef 100644
--- a/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/ipc/TestRPCCompatibility.java
+++ b/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/ipc/TestRPCCompatibility.java
@@ -31,6 +31,10 @@
 import org.apache.commons.logging.Log;
 import org.apache.commons.logging.LogFactory;
 import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.ipc.RpcPayloadHeader.RpcKind;
+import org.apache.hadoop.ipc.protobuf.ProtocolInfoProtos.GetProtocolSignatureRequestProto;
+import org.apache.hadoop.ipc.protobuf.ProtocolInfoProtos.GetProtocolSignatureResponseProto;
+import org.apache.hadoop.ipc.protobuf.ProtocolInfoProtos.ProtocolSignatureProto;
 import org.apache.hadoop.net.NetUtils;
 import org.junit.After;
 import org.junit.Test;
@@ -39,7 +43,7 @@
 public class TestRPCCompatibility {
   private static final String ADDRESS = "0.0.0.0";
   private static InetSocketAddress addr;
-  private static Server server;
+  private static RPC.Server server;
   private ProtocolProxy<?> proxy;
 
   public static final Log LOG =
@@ -52,10 +56,14 @@
     void ping() throws IOException;    
   }
   
-  public interface TestProtocol1 extends TestProtocol0 {
+  public interface TestProtocol1 extends VersionedProtocol, TestProtocol0 {
     String echo(String value) throws IOException;
   }
 
+  
+  // TestProtocol2 is a compatible impl of TestProtocol1 - hence use its name
+  @ProtocolInfo(protocolName=
+      "org.apache.hadoop.ipc.TestRPCCompatibility$TestProtocol1")
   public interface TestProtocol2 extends TestProtocol1 {
     int echo(int value)  throws IOException;
   }
@@ -89,28 +97,44 @@
   public static class TestImpl1 extends TestImpl0 implements TestProtocol1 {
     @Override
     public String echo(String value) { return value; }
+    @Override
+    public long getProtocolVersion(String protocol,
+        long clientVersion) throws IOException {
+        return TestProtocol1.versionID;
+    }
   }
 
   public static class TestImpl2 extends TestImpl1 implements TestProtocol2 {
     @Override
     public int echo(int value) { return value; }
+
+    @Override
+    public long getProtocolVersion(String protocol,
+        long clientVersion) throws IOException {
+      return TestProtocol2.versionID;
+    }
+
   }
   
   @After
   public void tearDown() throws IOException {
     if (proxy != null) {
       RPC.stopProxy(proxy.getProxy());
+      proxy = null;
     }
     if (server != null) {
       server.stop();
+      server = null;
     }
   }
   
   @Test  // old client vs new server
   public void testVersion0ClientVersion1Server() throws Exception {
     // create a server with two handlers
+    TestImpl1 impl = new TestImpl1();
     server = RPC.getServer(TestProtocol1.class,
-                              new TestImpl1(), ADDRESS, 0, 2, false, conf, null);
+                            impl, ADDRESS, 0, 2, false, conf, null);
+    server.addProtocol(RpcKind.RPC_WRITABLE, TestProtocol0.class, impl);
     server.start();
     addr = NetUtils.getConnectAddress(server);
 
@@ -154,8 +178,10 @@
     
     public int echo(int value) throws IOException, NumberFormatException {
       if (serverInfo.isMethodSupported("echo", int.class)) {
+System.out.println("echo int is supported");
         return -value;  // use version 3 echo long
       } else { // server is version 2
+System.out.println("echo int is NOT supported");
         return Integer.parseInt(proxy2.echo(String.valueOf(value)));
       }
     }
@@ -172,8 +198,10 @@
   @Test // Compatible new client & old server
   public void testVersion2ClientVersion1Server() throws Exception {
     // create a server with two handlers
+    TestImpl1 impl = new TestImpl1();
     server = RPC.getServer(TestProtocol1.class,
-                              new TestImpl1(), ADDRESS, 0, 2, false, conf, null);
+                              impl, ADDRESS, 0, 2, false, conf, null);
+    server.addProtocol(RpcKind.RPC_WRITABLE, TestProtocol0.class, impl);
     server.start();
     addr = NetUtils.getConnectAddress(server);
 
@@ -189,9 +217,12 @@
   
   @Test // equal version client and server
   public void testVersion2ClientVersion2Server() throws Exception {
+    ProtocolSignature.resetCache();
     // create a server with two handlers
+    TestImpl2 impl = new TestImpl2();
     server = RPC.getServer(TestProtocol2.class,
-                              new TestImpl2(), ADDRESS, 0, 2, false, conf, null);
+                             impl, ADDRESS, 0, 2, false, conf, null);
+    server.addProtocol(RpcKind.RPC_WRITABLE, TestProtocol0.class, impl);
     server.start();
     addr = NetUtils.getConnectAddress(server);
 
@@ -250,14 +281,16 @@
     assertEquals(hash1, hash2);
   }
   
+  @ProtocolInfo(protocolName=
+      "org.apache.hadoop.ipc.TestRPCCompatibility$TestProtocol1")
   public interface TestProtocol4 extends TestProtocol2 {
-    public static final long versionID = 1L;
+    public static final long versionID = 4L;
     int echo(int value)  throws IOException;
   }
   
   @Test
   public void testVersionMismatch() throws IOException {
-    server = RPC.getServer(TestProtocol2.class, new TestImpl0(), ADDRESS, 0, 2,
+    server = RPC.getServer(TestProtocol2.class, new TestImpl2(), ADDRESS, 0, 2,
         false, conf, null);
     server.start();
     addr = NetUtils.getConnectAddress(server);
@@ -268,7 +301,76 @@
       proxy.echo(21);
       fail("The call must throw VersionMismatch exception");
     } catch (IOException ex) {
-      Assert.assertTrue(ex.getMessage().contains("VersionMismatch"));
+      Assert.assertTrue("Expected version mismatch but got " + ex.getMessage(), 
+          ex.getMessage().contains("VersionMismatch"));
     }
   }
+  
+  @Test
+  public void testIsMethodSupported() throws IOException {
+    server = RPC.getServer(TestProtocol2.class, new TestImpl2(), ADDRESS, 0, 2,
+        false, conf, null);
+    server.start();
+    addr = NetUtils.getConnectAddress(server);
+
+    TestProtocol2 proxy = RPC.getProxy(TestProtocol2.class,
+        TestProtocol2.versionID, addr, conf);
+    boolean supported = RpcClientUtil.isMethodSupported(proxy,
+        TestProtocol2.class, RpcKind.RPC_WRITABLE,
+        RPC.getProtocolVersion(TestProtocol2.class), "echo");
+    Assert.assertTrue(supported);
+    supported = RpcClientUtil.isMethodSupported(proxy,
+        TestProtocol2.class, RpcKind.RPC_PROTOCOL_BUFFER,
+        RPC.getProtocolVersion(TestProtocol2.class), "echo");
+    Assert.assertFalse(supported);
+  }
+
+  /**
+   * Verify that ProtocolMetaInfoServerSideTranslatorPB correctly looks up
+   * the server registry to extract protocol signatures and versions.
+   */
+  @Test
+  public void testProtocolMetaInfoSSTranslatorPB() throws Exception {
+    TestImpl1 impl = new TestImpl1();
+    server = RPC.getServer(TestProtocol1.class, impl, ADDRESS, 0, 2, false,
+        conf, null);
+    server.addProtocol(RpcKind.RPC_WRITABLE, TestProtocol0.class, impl);
+    server.start();
+
+    ProtocolMetaInfoServerSideTranslatorPB xlator = 
+        new ProtocolMetaInfoServerSideTranslatorPB(server);
+
+    GetProtocolSignatureResponseProto resp = xlator.getProtocolSignature(
+        null,
+        createGetProtocolSigRequestProto(TestProtocol1.class,
+            RpcKind.RPC_PROTOCOL_BUFFER));
+    //No signatures should be found
+    Assert.assertEquals(0, resp.getProtocolSignatureCount());
+    resp = xlator.getProtocolSignature(
+        null,
+        createGetProtocolSigRequestProto(TestProtocol1.class,
+            RpcKind.RPC_WRITABLE));
+    Assert.assertEquals(1, resp.getProtocolSignatureCount());
+    ProtocolSignatureProto sig = resp.getProtocolSignatureList().get(0);
+    Assert.assertEquals(TestProtocol1.versionID, sig.getVersion());
+    boolean found = false;
+    int expected = ProtocolSignature.getFingerprint(TestProtocol1.class
+        .getMethod("echo", String.class));
+    for (int m : sig.getMethodsList()) {
+      if (expected == m) {
+        found = true;
+        break;
+      }
+    }
+    Assert.assertTrue(found);
+  }
+  
+  private GetProtocolSignatureRequestProto createGetProtocolSigRequestProto(
+      Class<?> protocol, RpcKind rpcKind) {
+    GetProtocolSignatureRequestProto.Builder builder = 
+        GetProtocolSignatureRequestProto.newBuilder();
+    builder.setProtocol(protocol.getName());
+    builder.setRpcKind(rpcKind.toString());
+    return builder.build();
+  }
 }
\ No newline at end of file
diff --git a/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/test/MultithreadedTestUtil.java b/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/test/MultithreadedTestUtil.java
index f6c112c..6fab194 100644
--- a/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/test/MultithreadedTestUtil.java
+++ b/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/test/MultithreadedTestUtil.java
@@ -164,6 +164,10 @@
       }
       checkException();
     }
+
+    public Iterable<? extends Thread> getTestThreads() {
+      return testThreads;
+    }
   }
 
   /**
diff --git a/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/ipc/AvroTestProtocol.java b/hadoop-common-project/hadoop-common/src/test/proto/test.proto
similarity index 70%
rename from hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/ipc/AvroTestProtocol.java
rename to hadoop-common-project/hadoop-common/src/test/proto/test.proto
index d5d7396..71f4427 100644
--- a/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/ipc/AvroTestProtocol.java
+++ b/hadoop-common-project/hadoop-common/src/test/proto/test.proto
@@ -16,17 +16,20 @@
  * limitations under the License.
  */
 
-package org.apache.hadoop.ipc;
+option java_package = "org.apache.hadoop.ipc.protobuf";
+option java_outer_classname = "TestProtos";
+option java_generate_equals_and_hash = true;
 
-import org.apache.avro.AvroRemoteException;
+message EmptyRequestProto {
+}
 
-@SuppressWarnings("serial")
-public interface AvroTestProtocol {
-  public static class Problem extends AvroRemoteException {
-    public Problem() {}
-  }
-  void ping();
-  String echo(String value);
-  int add(int v1, int v2);
-  int error() throws Problem;
+message EmptyResponseProto {
+}
+
+message EchoRequestProto {
+  required string message = 1;
+}
+
+message EchoResponseProto {
+  required string message = 1;
 }
diff --git a/hadoop-common-project/hadoop-common/src/test/proto/test_rpc_service.proto b/hadoop-common-project/hadoop-common/src/test/proto/test_rpc_service.proto
new file mode 100644
index 0000000..3dcffe9
--- /dev/null
+++ b/hadoop-common-project/hadoop-common/src/test/proto/test_rpc_service.proto
@@ -0,0 +1,38 @@
+/**
+ * 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.
+ */
+option java_package = "org.apache.hadoop.ipc.protobuf";
+option java_outer_classname = "TestRpcServiceProtos";
+option java_generic_services = true;
+option java_generate_equals_and_hash = true;
+
+import "test.proto";
+
+
+/**
+ * A protobuf service for use in tests
+ */
+service TestProtobufRpcProto {
+  rpc ping(EmptyRequestProto) returns (EmptyResponseProto);
+  rpc echo(EchoRequestProto) returns (EchoResponseProto);
+  rpc error(EmptyRequestProto) returns (EmptyResponseProto);
+}
+
+service TestProtobufRpc2Proto {
+  rpc ping2(EmptyRequestProto) returns (EmptyResponseProto);
+  rpc echo2(EchoRequestProto) returns (EchoResponseProto);
+}
diff --git a/hadoop-hdfs-project/hadoop-hdfs/CHANGES.txt b/hadoop-hdfs-project/hadoop-hdfs/CHANGES.txt
index ac7e337..ea4ffe4 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/CHANGES.txt
+++ b/hadoop-hdfs-project/hadoop-hdfs/CHANGES.txt
@@ -1,5 +1,127 @@
 Hadoop HDFS Change Log
 
+Release 0.23-PB - Unreleased
+
+  INCOMPATIBLE CHANGES
+    HDFS-2676. Remove Avro RPC. (suresh)
+
+  NEW FEATURES
+
+    HDFS-395.  DFS Scalability: Incremental block reports. (Tomasz Nykiel
+    via hairong)
+
+    HDFS-2517. Add protobuf service for JounralProtocol. (suresh)
+
+    HDFS-2518. Add protobuf service for NamenodeProtocol. (suresh)
+
+    HDFS-2520. Add protobuf service for InterDatanodeProtocol. (suresh)
+
+    HDFS-2519. Add protobuf service for DatanodeProtocol. (suresh)
+
+    HDFS-2581. Implement protobuf service for JournalProtocol. (suresh)
+
+    HDFS-2618. Implement protobuf service for NamenodeProtocol. (suresh)
+
+    HDFS-2629. Implement protobuf service for InterDatanodeProtocol. (suresh)
+
+    HDFS-2636. Implement protobuf service for ClientDatanodeProtocol. (suresh)
+
+    HDFS-2642. Protobuf translators for DatanodeProtocol. (jitendra)
+
+    HDFS-2647. Used protobuf based RPC for InterDatanodeProtocol, 
+    ClientDatanodeProtocol, JournalProtocol, NamenodeProtocol. (suresh)
+
+    HDFS-2666. Fix TestBackupNode failure. (suresh)
+
+    HDFS-2663. Optional protobuf parameters are not handled correctly.
+    (suresh)
+
+    HDFS-2661. Enable protobuf RPC for DatanodeProtocol. (jitendra)
+
+    HDFS-2697. Move RefreshAuthPolicy, RefreshUserMappings, GetUserMappings 
+    protocol to protocol buffers. (jitendra)
+
+    HDFS-2880. Protobuf changes in DatanodeProtocol to add multiple storages.
+    (suresh)
+
+    HDFS-2899. Service protocol changes in DatanodeProtocol to add multiple 
+    storages. (suresh)
+
+  IMPROVEMENTS
+
+    HDFS-2018. Move all journal stream management code into one place.
+               (Ivan Kelly via jitendra)
+
+    HDFS-2223. Untangle depencencies between NN components (todd)
+
+    HDFS-2351 Change Namenode and Datanode to register each of their protocols seperately (Sanjay Radia)
+
+    HDFS-2337. DFSClient shouldn't keep multiple RPC proxy references (atm)
+ 
+    HDFS-2181. Separate HDFS Client wire protocol data types (sanjay)
+
+    HDFS-2459. Separate datatypes for Journal Protocol. (suresh)
+
+    HDFS-2480. Separate datatypes for NamenodeProtocol. (suresh)
+
+    HDFS-2489. Move Finalize and Register to separate file out of
+    DatanodeCommand.java. (suresh)
+
+    HDFS-2488. Separate datatypes for InterDatanodeProtocol. (suresh)
+
+    HDFS-2496. Separate datatypes for DatanodeProtocol. (suresh)
+
+    HDFS-2479 HDFS Client Data Types in Protocol Buffers (sanjay)
+
+    HADOOP-7862   Hdfs changes to work with HADOOP 7862: 
+    Move the support for multiple protocols to lower layer so that Writable,
+    PB and Avro can all use it (Sanjay)
+
+    HDFS-2597 ClientNameNodeProtocol in Protocol Buffers (sanjay)
+
+    HDFS-2651 ClientNameNodeProtocol Translators for Protocol Buffers (sanjay)
+
+    HDFS-2650. Replace @inheritDoc with @Override. (Hari Mankude via suresh).
+
+    HDFS-2669 Enable protobuf rpc for ClientNamenodeProtocol
+
+    HDFS-2801. Provide a method in client side translators to check for a 
+    methods supported in underlying protocol. (jitendra)
+
+    HDFS-2895. Remove Writable wire protocol types and translators to
+    complete transition to protocol buffers. (suresh)
+
+  BUG FIXES
+ 
+    HDFS-2481 Unknown protocol: org.apache.hadoop.hdfs.protocol.ClientProtocol (sanjay)
+
+    HDFS-2497 Fix TestBackupNode failure. (suresh)
+
+    HDFS-2499. RPC client is created incorrectly introduced in HDFS-2459.
+    (suresh)
+
+    HDFS-2526. (Client)NamenodeProtocolTranslatorR23 do not need to keep a
+    reference to rpcProxyWithoutRetry (atm)
+
+    HDFS-2532. TestDfsOverAvroRpc timing out in trunk (Uma Maheswara Rao G
+    via todd)
+
+    HDFS-2694. Removal of Avro broke non-PB NN services. (atm)
+
+    HDFS-2687. Tests failing with ClassCastException post protobuf RPC
+    changes. (suresh)
+
+    HDFS-2700. Fix failing TestDataNodeMultipleRegistrations in trunk
+    (Uma Maheswara Rao G via todd)
+
+    HDFS-2739. SecondaryNameNode doesn't start up. (jitendra)
+
+    HDFS-2768. BackupNode stop can not close proxy connections because
+    it is not a proxy instance. (Uma Maheswara Rao G via eli)
+
+    HDFS-2968. Protocol translator for BlockRecoveryCommand broken when
+    multiple blocks need recovery. (todd)
+
 Release 0.23.2 - UNRELEASED
 
   INCOMPATIBLE CHANGES
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/CorruptFileBlockIterator.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/CorruptFileBlockIterator.java
index 2fedd6b..bd3fdd5 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/CorruptFileBlockIterator.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/CorruptFileBlockIterator.java
@@ -81,17 +81,13 @@
     }
   }
 
-  /**
-   * {@inheritDoc}
-   */
+  
   @Override
   public boolean hasNext() {
     return nextPath != null;
   }
 
-  /**
-   * {@inheritDoc}
-   */
+  
   @Override
   public Path next() throws IOException {
     if (!hasNext()) {
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/DFSClient.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/DFSClient.java
index 990bced..cdaf07e 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/DFSClient.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/DFSClient.java
@@ -19,6 +19,7 @@
 package org.apache.hadoop.hdfs;
 
 import java.io.BufferedOutputStream;
+import java.io.Closeable;
 import java.io.DataInputStream;
 import java.io.DataOutputStream;
 import java.io.FileNotFoundException;
@@ -123,7 +124,6 @@
   public static final long SERVER_DEFAULTS_VALIDITY_PERIOD = 60 * 60 * 1000L; // 1 hour
   static final int TCP_WINDOW_SIZE = 128 * 1024; // 128 KB
   final ClientProtocol namenode;
-  final ClientProtocol rpcNamenode;
   private final InetSocketAddress nnAddress;
   final UserGroupInformation ugi;
   volatile boolean clientRunning = true;
@@ -291,11 +291,10 @@
     this.clientName = leaserenewer.getClientName(dfsClientConf.taskId);
     this.socketCache = new SocketCache(dfsClientConf.socketCacheCapacity);
     if (nameNodeAddr != null && rpcNamenode == null) {
-      this.rpcNamenode = DFSUtil.createRPCNamenode(nameNodeAddr, conf, ugi);
-      this.namenode = DFSUtil.createNamenode(this.rpcNamenode);
+      this.namenode = DFSUtil.createNamenode(nameNodeAddr, conf, ugi);
     } else if (nameNodeAddr == null && rpcNamenode != null) {
       //This case is used for testing.
-      this.namenode = this.rpcNamenode = rpcNamenode;
+      this.namenode = rpcNamenode;
     } else {
       throw new IllegalArgumentException(
           "Expecting exactly one of nameNodeAddr and rpcNamenode being null: "
@@ -386,12 +385,31 @@
     }
     return false;
   }
-
+  
+  /**
+   * Close connections the Namenode.
+   * The namenode variable is either a rpcProxy passed by a test or 
+   * created using the protocolTranslator which is closeable.
+   * If closeable then call close, else close using RPC.stopProxy().
+   */
+  void closeConnectionToNamenode() {
+    if (namenode instanceof Closeable) {
+      try {
+        ((Closeable) namenode).close();
+        return;
+      } catch (IOException e) {
+        // fall through - lets try the stopProxy
+        LOG.warn("Exception closing namenode, stopping the proxy");
+      }     
+    }
+    RPC.stopProxy(namenode);
+  }
+  
   /** Abort and release resources held.  Ignore all errors. */
   void abort() {
     clientRunning = false;
     closeAllFilesBeingWritten(true);
-    RPC.stopProxy(rpcNamenode); // close connections to the namenode
+    closeConnectionToNamenode();
   }
 
   /** Close/abort all files being written. */
@@ -431,7 +449,7 @@
       clientRunning = false;
       leaserenewer.closeClient(this);
       // close connections to the namenode
-      RPC.stopProxy(rpcNamenode);
+      closeConnectionToNamenode();
     }
   }
 
@@ -614,7 +632,7 @@
       LOG.info("Renewing " + 
                DelegationTokenIdentifier.stringifyToken(delToken));
       ClientProtocol nn = 
-        DFSUtil.createRPCNamenode
+        DFSUtil.createNamenode
            (SecurityUtil.getTokenServiceAddr(delToken),
             conf, UserGroupInformation.getCurrentUser());
       try {
@@ -632,7 +650,7 @@
           (Token<DelegationTokenIdentifier>) token;
       LOG.info("Cancelling " + 
                DelegationTokenIdentifier.stringifyToken(delToken));
-      ClientProtocol nn = DFSUtil.createRPCNamenode(
+      ClientProtocol nn = DFSUtil.createNamenode(
           SecurityUtil.getTokenServiceAddr(delToken), conf,
           UserGroupInformation.getCurrentUser());
       try {
@@ -1460,7 +1478,8 @@
    * 
    * @see ClientProtocol#restoreFailedStorage(String arg)
    */
-  boolean restoreFailedStorage(String arg) throws AccessControlException {
+  boolean restoreFailedStorage(String arg)
+      throws AccessControlException, IOException{
     return namenode.restoreFailedStorage(arg);
   }
 
@@ -1678,8 +1697,7 @@
     }
   }
   
-  boolean shouldTryShortCircuitRead(InetSocketAddress targetAddr)
-      throws IOException {
+  boolean shouldTryShortCircuitRead(InetSocketAddress targetAddr) {
     if (shortCircuitLocalReads && isLocalAddress(targetAddr)) {
       return true;
     }
@@ -1702,7 +1720,7 @@
     }
   }
 
-  /** {@inheritDoc} */
+  @Override
   public String toString() {
     return getClass().getSimpleName() + "[clientName=" + clientName
         + ", ugi=" + ugi + "]"; 
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/DFSUtil.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/DFSUtil.java
index f323c97..4d413cc 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/DFSUtil.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/DFSUtil.java
@@ -41,11 +41,12 @@
 import java.util.StringTokenizer;
 import java.util.concurrent.TimeUnit;
 
+import javax.net.SocketFactory;
+
 import org.apache.hadoop.HadoopIllegalArgumentException;
 import org.apache.hadoop.classification.InterfaceAudience;
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.fs.BlockLocation;
-import org.apache.hadoop.fs.CommonConfigurationKeysPublic;
 import org.apache.hadoop.fs.Path;
 import org.apache.hadoop.hdfs.protocol.AlreadyBeingCreatedException;
 import org.apache.hadoop.hdfs.protocol.ClientDatanodeProtocol;
@@ -55,16 +56,21 @@
 import org.apache.hadoop.hdfs.protocol.HdfsConstants;
 import org.apache.hadoop.hdfs.protocol.LocatedBlock;
 import org.apache.hadoop.hdfs.protocol.LocatedBlocks;
+import org.apache.hadoop.hdfs.protocolPB.ClientDatanodeProtocolTranslatorPB;
 import org.apache.hadoop.hdfs.server.namenode.NameNode;
 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.ProtobufRpcEngine;
 import org.apache.hadoop.ipc.RPC;
 import org.apache.hadoop.ipc.RemoteException;
+import org.apache.hadoop.ipc.RpcPayloadHeader.RpcKind;
 import org.apache.hadoop.net.NetUtils;
 import org.apache.hadoop.net.NodeBase;
 import org.apache.hadoop.security.UserGroupInformation;
 
+import com.google.protobuf.BlockingService;
+
 @InterfaceAudience.Private
 public class DFSUtil {
   private static final ThreadLocal<Random> RANDOM = new ThreadLocal<Random>() {
@@ -594,12 +600,12 @@
   
   /** Return used as percentage of capacity */
   public static float getPercentUsed(long used, long capacity) {
-    return capacity <= 0 ? 100 : ((float)used * 100.0f)/(float)capacity; 
+    return capacity <= 0 ? 100 : (used * 100.0f)/capacity; 
   }
   
   /** Return remaining as percentage of capacity */
   public static float getPercentRemaining(long remaining, long capacity) {
-    return capacity <= 0 ? 0 : ((float)remaining * 100.0f)/(float)capacity; 
+    return capacity <= 0 ? 0 : (remaining * 100.0f)/capacity; 
   }
 
   /**
@@ -613,25 +619,29 @@
 
 
   /** Create a {@link NameNode} proxy */
-  public static ClientProtocol createNamenode(Configuration conf) throws IOException {
+  public static ClientProtocol createNamenode(Configuration conf)
+      throws IOException {
     return createNamenode(NameNode.getAddress(conf), conf);
   }
 
   /** Create a {@link NameNode} proxy */
   public static ClientProtocol createNamenode( InetSocketAddress nameNodeAddr,
-      Configuration conf) throws IOException {
-    return createNamenode(createRPCNamenode(nameNodeAddr, conf,
-        UserGroupInformation.getCurrentUser()));
-    
+      Configuration conf) throws IOException {   
+    return createNamenode(nameNodeAddr, conf,
+        UserGroupInformation.getCurrentUser());
   }
-
+    
   /** Create a {@link NameNode} proxy */
-  static ClientProtocol createRPCNamenode(InetSocketAddress nameNodeAddr,
-      Configuration conf, UserGroupInformation ugi) 
-    throws IOException {
-    return (ClientProtocol)RPC.getProxy(ClientProtocol.class,
-        ClientProtocol.versionID, nameNodeAddr, ugi, conf,
-        NetUtils.getSocketFactory(conf, ClientProtocol.class));
+  public static ClientProtocol createNamenode( InetSocketAddress nameNodeAddr,
+      Configuration conf, UserGroupInformation ugi) throws IOException {
+    /** 
+     * Currently we have simply burnt-in support for a SINGLE
+     * protocol - protocolPB. This will be replaced
+     * by a way to pick the right protocol based on the 
+     * version of the target server.  
+     */
+    return new org.apache.hadoop.hdfs.protocolPB.
+        ClientNamenodeProtocolTranslatorPB(nameNodeAddr, conf, ugi);
   }
 
   /** Create a {@link NameNode} proxy */
@@ -659,49 +669,27 @@
         rpcNamenode, methodNameToPolicyMap);
   }
   
+  /** Create a {@link ClientDatanodeProtocol} proxy */
+  public static ClientDatanodeProtocol createClientDatanodeProtocolProxy(
+      DatanodeID datanodeid, Configuration conf, int socketTimeout,
+      LocatedBlock locatedBlock) throws IOException {
+    return new ClientDatanodeProtocolTranslatorPB(datanodeid, conf, socketTimeout,
+             locatedBlock);
+  }
+  
   /** Create {@link ClientDatanodeProtocol} proxy using kerberos ticket */
   static ClientDatanodeProtocol createClientDatanodeProtocolProxy(
       DatanodeID datanodeid, Configuration conf, int socketTimeout)
       throws IOException {
-    InetSocketAddress addr = NetUtils.createSocketAddr(datanodeid.getHost()
-        + ":" + datanodeid.getIpcPort());
-    if (ClientDatanodeProtocol.LOG.isDebugEnabled()) {
-      ClientDatanodeProtocol.LOG.info("ClientDatanodeProtocol addr=" + addr);
-    }
-    return (ClientDatanodeProtocol) RPC.getProxy(ClientDatanodeProtocol.class,
-        ClientDatanodeProtocol.versionID, addr,
-        UserGroupInformation.getCurrentUser(), conf,
-        NetUtils.getDefaultSocketFactory(conf), socketTimeout);
+    return new ClientDatanodeProtocolTranslatorPB(
+        datanodeid, conf, socketTimeout);
   }
   
   /** Create a {@link ClientDatanodeProtocol} proxy */
   public static ClientDatanodeProtocol createClientDatanodeProtocolProxy(
-      DatanodeID datanodeid, Configuration conf, int socketTimeout,
-      LocatedBlock locatedBlock)
-      throws IOException {
-    InetSocketAddress addr = NetUtils.createSocketAddr(
-      datanodeid.getHost() + ":" + datanodeid.getIpcPort());
-    if (ClientDatanodeProtocol.LOG.isDebugEnabled()) {
-      ClientDatanodeProtocol.LOG.debug("ClientDatanodeProtocol addr=" + addr);
-    }
-    
-    // Since we're creating a new UserGroupInformation here, we know that no
-    // future RPC proxies will be able to re-use the same connection. And
-    // usages of this proxy tend to be one-off calls.
-    //
-    // This is a temporary fix: callers should really achieve this by using
-    // RPC.stopProxy() on the resulting object, but this is currently not
-    // working in trunk. See the discussion on HDFS-1965.
-    Configuration confWithNoIpcIdle = new Configuration(conf);
-    confWithNoIpcIdle.setInt(CommonConfigurationKeysPublic
-        .IPC_CLIENT_CONNECTION_MAXIDLETIME_KEY, 0);
-
-    UserGroupInformation ticket = UserGroupInformation
-        .createRemoteUser(locatedBlock.getBlock().getLocalBlock().toString());
-    ticket.addToken(locatedBlock.getBlockToken());
-    return (ClientDatanodeProtocol)RPC.getProxy(ClientDatanodeProtocol.class,
-        ClientDatanodeProtocol.versionID, addr, ticket, confWithNoIpcIdle,
-        NetUtils.getDefaultSocketFactory(conf), socketTimeout);
+      InetSocketAddress addr, UserGroupInformation ticket, Configuration conf,
+      SocketFactory factory) throws IOException {
+    return new ClientDatanodeProtocolTranslatorPB(addr, ticket, conf, factory);
   }
   
   /**
@@ -786,4 +774,18 @@
       throw new IllegalArgumentException(ue);
     }
   }
+  
+  /**
+   * Add protobuf based protocol to the {@link org.apache.hadoop.ipc.RPC.Server}
+   * @param conf configuration
+   * @param protocol Protocol interface
+   * @param service service that implements the protocol
+   * @param server RPC server to which the protocol & implementation is added to
+   * @throws IOException
+   */
+  public static void addPBProtocol(Configuration conf, Class<?> protocol,
+      BlockingService service, RPC.Server server) throws IOException {
+    RPC.setProtocolEngine(conf, protocol, ProtobufRpcEngine.class);
+    server.addProtocol(RpcKind.RPC_PROTOCOL_BUFFER, protocol, service);
+  }
 }
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/DistributedFileSystem.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/DistributedFileSystem.java
index 5ccb9ac..119bca9 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/DistributedFileSystem.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/DistributedFileSystem.java
@@ -145,7 +145,7 @@
     workingDir = makeAbsolute(dir);
   }
 
-  /** {@inheritDoc} */
+  
   @Override
   public Path getHomeDirectory() {
     return makeQualified(new Path("/user/" + dfs.ugi.getShortUserName()));
@@ -272,7 +272,7 @@
     dfs.concat(getPathName(trg), srcs);
   }
 
-  /** {@inheritDoc} */
+  
   @SuppressWarnings("deprecation")
   @Override
   public boolean rename(Path src, Path dst) throws IOException {
@@ -281,7 +281,6 @@
   }
 
   /** 
-   * {@inheritDoc}
    * This rename operation is guaranteed to be atomic.
    */
   @SuppressWarnings("deprecation")
@@ -297,7 +296,6 @@
     return dfs.delete(getPathName(f), recursive);
   }
   
-  /** {@inheritDoc} */
   @Override
   public ContentSummary getContentSummary(Path f) throws IOException {
     statistics.incrementReadOps(1);
@@ -478,7 +476,7 @@
     return dfs.primitiveMkdir(getPathName(f), absolutePermission);
   }
 
-  /** {@inheritDoc} */
+ 
   @Override
   public void close() throws IOException {
     try {
@@ -518,7 +516,6 @@
     }
   }
   
-  /** {@inheritDoc} */
   @Override
   public FsStatus getStatus(Path p) throws IOException {
     statistics.incrementReadOps(1);
@@ -580,9 +577,6 @@
     return dfs.getCorruptBlocksCount();
   }
 
-  /**
-   * {@inheritDoc}
-   */
   @Override
   public RemoteIterator<Path> listCorruptFileBlocks(Path path)
     throws IOException {
@@ -625,7 +619,8 @@
    * 
    * @see org.apache.hadoop.hdfs.protocol.ClientProtocol#restoreFailedStorage(String arg)
    */
-  public boolean restoreFailedStorage(String arg) throws AccessControlException {
+  public boolean restoreFailedStorage(String arg)
+      throws AccessControlException, IOException {
     return dfs.restoreFailedStorage(arg);
   }
   
@@ -659,7 +654,6 @@
     dfs.metaSave(pathname);
   }
 
-  /** {@inheritDoc} */
   @Override
   public FsServerDefaults getServerDefaults() throws IOException {
     return dfs.getServerDefaults();
@@ -730,14 +724,12 @@
     }
   }
 
-  /** {@inheritDoc} */
   @Override
   public MD5MD5CRC32FileChecksum getFileChecksum(Path f) throws IOException {
     statistics.incrementReadOps(1);
     return dfs.getFileChecksum(getPathName(f));
   }
 
-  /** {@inheritDoc }*/
   @Override
   public void setPermission(Path p, FsPermission permission
       ) throws IOException {
@@ -745,7 +737,6 @@
     dfs.setPermission(getPathName(p), permission);
   }
 
-  /** {@inheritDoc }*/
   @Override
   public void setOwner(Path p, String username, String groupname
       ) throws IOException {
@@ -756,7 +747,6 @@
     dfs.setOwner(getPathName(p), username, groupname);
   }
 
-  /** {@inheritDoc }*/
   @Override
   public void setTimes(Path p, long mtime, long atime
       ) throws IOException {
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/HftpFileSystem.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/HftpFileSystem.java
index 5ec4584..8fe8cba 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/HftpFileSystem.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/HftpFileSystem.java
@@ -492,7 +492,7 @@
   private class ChecksumParser extends DefaultHandler {
     private FileChecksum filechecksum;
 
-    /** {@inheritDoc} */
+    @Override
     public void startElement(String ns, String localname, String qname,
                 Attributes attrs) throws SAXException {
       if (!MD5MD5CRC32FileChecksum.class.getName().equals(qname)) {
@@ -526,7 +526,7 @@
     }
   }
 
-  /** {@inheritDoc} */
+  @Override
   public FileChecksum getFileChecksum(Path f) throws IOException {
     final String s = makeQualified(f).toUri().getPath();
     return new ChecksumParser().getFileChecksum(s);
@@ -574,7 +574,7 @@
   private class ContentSummaryParser extends DefaultHandler {
     private ContentSummary contentsummary;
 
-    /** {@inheritDoc} */
+    @Override
     public void startElement(String ns, String localname, String qname,
                 Attributes attrs) throws SAXException {
       if (!ContentSummary.class.getName().equals(qname)) {
@@ -660,7 +660,7 @@
     }
   }
 
-  /** {@inheritDoc} */
+  @Override
   public ContentSummary getContentSummary(Path f) throws IOException {
     final String s = makeQualified(f).toUri().getPath();
     final ContentSummary cs = new ContentSummaryParser().getContentSummary(s);
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/protocol/BlockLocalPathInfo.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/protocol/BlockLocalPathInfo.java
index a1823b3..6017c37 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/protocol/BlockLocalPathInfo.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/protocol/BlockLocalPathInfo.java
@@ -67,6 +67,11 @@
   public String getBlockPath() {return localBlockPath;}
   
   /**
+   * @return the Block
+   */
+  public ExtendedBlock getBlock() { return block;}
+  
+  /**
    * Get the Block metadata file.
    * @return Block metadata file.
    */
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/protocol/ClientDatanodeProtocol.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/protocol/ClientDatanodeProtocol.java
index f1be216..96e2dbd 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/protocol/ClientDatanodeProtocol.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/protocol/ClientDatanodeProtocol.java
@@ -39,10 +39,30 @@
     serverPrincipal = DFSConfigKeys.DFS_DATANODE_USER_NAME_KEY)
 @TokenInfo(BlockTokenSelector.class)
 public interface ClientDatanodeProtocol extends VersionedProtocol {
-  public static final Log LOG = LogFactory.getLog(ClientDatanodeProtocol.class);
-
   /**
+   * Until version 9, this class ClientDatanodeProtocol served as both
+   * the client interface to the DN AND the RPC protocol used to 
+   * communicate with the NN.
+   * 
+   * Post version 10 (release 23 of Hadoop), the protocol is implemented in
+   * {@literal ../protocolR23Compatible/ClientDatanodeWireProtocol}
+   * 
+   * This class is used by both the DFSClient and the 
+   * DN server side to insulate from the protocol serialization.
+   * 
+   * If you are adding/changing DN's interface then you need to 
+   * change both this class and ALSO related protocol buffer
+   * wire protocol definition in ClientDatanodeProtocol.proto.
+   * 
+   * For more details on protocol buffer wire protocol, please see 
+   * .../org/apache/hadoop/hdfs/protocolPB/overview.html
+   * 
+   * The log of historical changes can be retrieved from the svn).
    * 9: Added deleteBlockPool method
+   * 
+   * 9 is the last version id when this class was used for protocols
+   *  serialization. DO not update this version any further. 
+   *  Changes are recorded in R23 classes.
    */
   public static final long versionID = 9L;
 
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/protocol/ClientProtocol.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/protocol/ClientProtocol.java
index c840588..65c17fe 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/protocol/ClientProtocol.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/protocol/ClientProtocol.java
@@ -20,8 +20,6 @@
 import java.io.FileNotFoundException;
 import java.io.IOException;
 
-import org.apache.avro.reflect.Nullable;
-
 import org.apache.hadoop.classification.InterfaceAudience;
 import org.apache.hadoop.classification.InterfaceStability;
 import org.apache.hadoop.fs.ContentSummary;
@@ -64,10 +62,29 @@
 public interface ClientProtocol extends VersionedProtocol {
 
   /**
-   * Compared to the previous version the following changes have been introduced:
-   * (Only the latest change is reflected.
+   * Until version 69, this class ClientProtocol served as both
+   * the client interface to the NN AND the RPC protocol used to 
+   * communicate with the NN.
+   * 
+   * Post version 70 (release 23 of Hadoop), the protocol is implemented in
+   * {@literal ../protocolR23Compatible/ClientNamenodeWireProtocol}
+   * 
+   * This class is used by both the DFSClient and the 
+   * NN server side to insulate from the protocol serialization.
+   * 
+   * If you are adding/changing this interface then you need to 
+   * change both this class and ALSO related protocol buffer
+   * wire protocol definition in ClientNamenodeProtocol.proto.
+   * 
+   * For more details on protocol buffer wire protocol, please see 
+   * .../org/apache/hadoop/hdfs/protocolPB/overview.html
+   * 
    * The log of historical changes can be retrieved from the svn).
    * 69: Eliminate overloaded method names.
+   * 
+   * 69L is the last version id when this class was used for protocols
+   *  serialization. DO not update this version any further. 
+   *  Changes are recorded in R23 classes.
    */
   public static final long versionID = 69L;
   
@@ -98,7 +115,6 @@
    * @throws UnresolvedLinkException If <code>src</code> contains a symlink
    * @throws IOException If an I/O error occurred
    */
-  @Nullable
   public LocatedBlocks getBlockLocations(String src,
                                          long offset,
                                          long length) 
@@ -293,7 +309,7 @@
    * @throws IOException If an I/O error occurred
    */
   public LocatedBlock addBlock(String src, String clientName,
-      @Nullable ExtendedBlock previous, @Nullable DatanodeInfo[] excludeNodes)
+      ExtendedBlock previous, DatanodeInfo[] excludeNodes)
       throws AccessControlException, FileNotFoundException,
       NotReplicatedYetException, SafeModeException, UnresolvedLinkException,
       IOException;
@@ -365,11 +381,8 @@
    * @return true if successful, or false if the old name does not exist
    * or if the new name already belongs to the namespace.
    * 
-   * @throws IOException an I/O error occurred
-   * 
-   * @deprecated Use {@link #rename(String, String, Options.Rename...)} instead.
+   * @throws IOException an I/O error occurred 
    */
-  @Deprecated
   public boolean rename(String src, String dst) 
       throws UnresolvedLinkException, IOException;
 
@@ -650,7 +663,8 @@
    * 
    * @throws AccessControlException if the superuser privilege is violated.
    */
-  public boolean restoreFailedStorage(String arg) throws AccessControlException;
+  public boolean restoreFailedStorage(String arg) 
+      throws AccessControlException, IOException;
 
   /**
    * Tells the namenode to reread the hosts and exclude files. 
@@ -674,7 +688,6 @@
    * @return upgrade status information or null if no upgrades are in progress
    * @throws IOException
    */
-  @Nullable
   public UpgradeStatusReport distributedUpgradeProgress(UpgradeAction action) 
       throws IOException;
 
@@ -720,7 +733,6 @@
    * @throws UnresolvedLinkException if the path contains a symlink. 
    * @throws IOException If an I/O error occurred        
    */
-  @Nullable
   public HdfsFileStatus getFileInfo(String src) throws AccessControlException,
       FileNotFoundException, UnresolvedLinkException, IOException;
 
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/protocol/CorruptFileBlocks.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/protocol/CorruptFileBlocks.java
index 7b69f0b..44fd387 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/protocol/CorruptFileBlocks.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/protocol/CorruptFileBlocks.java
@@ -53,9 +53,7 @@
     return cookie;
   }
 
-  /**
-   * {@inheritDoc}
-   */
+  
   @Override
   public void readFields(DataInput in) throws IOException {
     int fileCount = in.readInt();
@@ -66,9 +64,7 @@
     cookie = Text.readString(in);
   }
 
-  /**
-   * {@inheritDoc}
-   */
+  
   @Override
   public void write(DataOutput out) throws IOException {
     out.writeInt(files.length);
@@ -78,9 +74,8 @@
     Text.writeString(out, cookie);
   }
 
-  /**
-   * {@inheritDoc}
-   */
+ 
+  @Override
   public boolean equals(Object obj) {
     if (this == obj) {
       return true;
@@ -93,9 +88,8 @@
       Arrays.equals(files, other.files);
   }
 
-  /**
-   * {@inheritDoc}
-   */
+  
+  @Override
   public int hashCode() {
     int result = cookie.hashCode();
 
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/protocol/DatanodeID.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/protocol/DatanodeID.java
index 6bf4481..21e549d 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/protocol/DatanodeID.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/protocol/DatanodeID.java
@@ -185,14 +185,14 @@
   /////////////////////////////////////////////////
   // Writable
   /////////////////////////////////////////////////
-  /** {@inheritDoc} */
+  @Override
   public void write(DataOutput out) throws IOException {
     DeprecatedUTF8.writeString(out, name);
     DeprecatedUTF8.writeString(out, storageID);
     out.writeShort(infoPort);
   }
 
-  /** {@inheritDoc} */
+  @Override
   public void readFields(DataInput in) throws IOException {
     name = DeprecatedUTF8.readString(in);
     storageID = DeprecatedUTF8.readString(in);
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/protocol/DatanodeInfo.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/protocol/DatanodeInfo.java
index af3283e..80b2d28 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/protocol/DatanodeInfo.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/protocol/DatanodeInfo.java
@@ -36,8 +36,6 @@
 import org.apache.hadoop.net.NodeBase;
 import org.apache.hadoop.util.StringUtils;
 
-import org.apache.avro.reflect.Nullable;
-
 /** 
  * DatanodeInfo represents the status of a DataNode.
  * This object is used for communication in the
@@ -57,7 +55,6 @@
   /** HostName as supplied by the datanode during registration as its 
    * name. Namenode uses datanode IP address as the name.
    */
-  @Nullable
   protected String hostName = null;
   
   // administrative states of a datanode
@@ -75,12 +72,17 @@
     public String toString() {
       return value;
     }
+    
+    public static AdminStates fromValue(final String value) {
+      for (AdminStates as : AdminStates.values()) {
+        if (as.value.equals(value)) return as;
+      }
+      return NORMAL;
+    }
   }
 
-  @Nullable
   protected AdminStates adminState;
 
-
   public DatanodeInfo() {
     super();
     adminState = null;
@@ -110,11 +112,20 @@
     this.adminState = null;    
   }
   
-  protected DatanodeInfo(DatanodeID nodeID, String location, String hostName) {
+  public DatanodeInfo(DatanodeID nodeID, String location, String hostName) {
     this(nodeID);
     this.location = location;
     this.hostName = hostName;
   }
+  
+  public DatanodeInfo(DatanodeID nodeID, String location, String hostName,
+      final long capacity, final long dfsUsed, final long remaining,
+      final long blockPoolUsed, final long lastUpdate, final int xceiverCount,
+      final AdminStates adminState) {
+    this(nodeID.getName(), nodeID.getStorageID(), nodeID.getInfoPort(), nodeID
+        .getIpcPort(), capacity, dfsUsed, remaining, blockPoolUsed, lastUpdate,
+        xceiverCount, location, hostName, adminState);
+  }
 
   /** Constructor */
   public DatanodeInfo(final String name, final String storageID,
@@ -366,7 +377,7 @@
        });
   }
 
-  /** {@inheritDoc} */
+  @Override
   public void write(DataOutput out) throws IOException {
     super.write(out);
 
@@ -384,7 +395,7 @@
     WritableUtils.writeEnum(out, getAdminState());
   }
 
-  /** {@inheritDoc} */
+  @Override
   public void readFields(DataInput in) throws IOException {
     super.readFields(in);
 
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/protocol/HdfsConstants.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/protocol/HdfsConstants.java
index b005145..6b4835f 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/protocol/HdfsConstants.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/protocol/HdfsConstants.java
@@ -30,7 +30,16 @@
   /* Hidden constructor */
   protected HdfsConstants() {
   }
-
+  
+  /**
+   * HDFS Protocol Names:  
+   */
+  public static final String CLIENT_NAMENODE_PROTOCOL_NAME = 
+      "org.apache.hadoop.hdfs.protocol.ClientProtocol";
+  public static final String CLIENT_DATANODE_PROTOCOL_NAME = 
+      "org.apache.hadoop.hdfs.protocol.ClientDatanodeProtocol";
+  
+  
   public static int MIN_BLOCKS_FOR_WRITE = 5;
 
   // Long that indicates "leave current quota unchanged"
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/protocol/HdfsFileStatus.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/protocol/HdfsFileStatus.java
index 84d0e4c..cb6b5d0e 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/protocol/HdfsFileStatus.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/protocol/HdfsFileStatus.java
@@ -31,8 +31,6 @@
 import org.apache.hadoop.io.WritableFactories;
 import org.apache.hadoop.io.WritableFactory;
 
-import org.apache.avro.reflect.Nullable;
-
 /** Interface that represents the over the wire information for a file.
  */
 @InterfaceAudience.Private
@@ -47,7 +45,6 @@
   }
 
   private byte[] path;  // local name of the inode that's encoded in java UTF8
-  @Nullable
   private byte[] symlink; // symlink target encoded in java UTF8 or null
   private long length;
   private boolean isdir;
@@ -241,6 +238,10 @@
   final public String getSymlink() {
     return DFSUtil.bytes2String(symlink);
   }
+  
+  final public byte[] getSymlinkInBytes() {
+    return symlink;
+  }
 
   //////////////////////////////////////////////////
   // Writable
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/protocol/HdfsProtoUtil.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/protocol/HdfsProtoUtil.java
index 739a6d2..c0b63fe 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/protocol/HdfsProtoUtil.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/protocol/HdfsProtoUtil.java
@@ -87,6 +87,7 @@
       .setName(dni.getName())
       .setStorageID(dni.getStorageID())
       .setInfoPort(dni.getInfoPort())
+      .setIpcPort(dni.getIpcPort())
       .build();
   }
   
@@ -95,7 +96,7 @@
         idProto.getName(),
         idProto.getStorageID(),
         idProto.getInfoPort(),
-        -1); // ipc port not serialized in writables either
+        idProto.getIpcPort());
   }
   
   //// DatanodeInfo ////
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/protocol/LocatedBlock.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/protocol/LocatedBlock.java
index 0eace5a8..a8ad641 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/protocol/LocatedBlock.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/protocol/LocatedBlock.java
@@ -54,6 +54,11 @@
   public LocatedBlock() {
     this(new ExtendedBlock(), new DatanodeInfo[0], 0L, false);
   }
+  
+
+  public LocatedBlock(ExtendedBlock eb) {
+    this(eb, new DatanodeInfo[0], 0L, false);
+  }
 
   public LocatedBlock(String bpid, Block b, DatanodeInfo[] locs) {
     this(new ExtendedBlock(bpid, b), locs, -1, false); // startOffset is unknown
@@ -154,7 +159,7 @@
     return lb;
   }
 
-  /** {@inheritDoc} */
+  @Override
   public String toString() {
     return getClass().getSimpleName() + "{" + b
         + "; getBlockSize()=" + getBlockSize()
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/protocol/LocatedBlocks.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/protocol/LocatedBlocks.java
index e9c3122..02dd547 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/protocol/LocatedBlocks.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/protocol/LocatedBlocks.java
@@ -31,8 +31,6 @@
 import org.apache.hadoop.io.WritableFactories;
 import org.apache.hadoop.io.WritableFactory;
 
-import org.apache.avro.reflect.Nullable;
-
 /**
  * Collection of blocks with their locations and the file length.
  */
@@ -42,7 +40,6 @@
   private long fileLength;
   private List<LocatedBlock> blocks; // array of blocks with prioritized locations
   private boolean underConstruction;
-  @Nullable
   private LocatedBlock lastLocatedBlock = null;
   private boolean isLastBlockComplete = false;
 
@@ -225,7 +222,6 @@
     }
   }
 
-  /** {@inheritDoc} */
   @Override
   public String toString() {
     final StringBuilder b = new StringBuilder(getClass().getSimpleName());
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/protocolPB/ClientDatanodeProtocolPB.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/protocolPB/ClientDatanodeProtocolPB.java
new file mode 100644
index 0000000..5829398
--- /dev/null
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/protocolPB/ClientDatanodeProtocolPB.java
@@ -0,0 +1,49 @@
+/**
+ * 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.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;
+
+@KerberosInfo(
+    serverPrincipal = DFSConfigKeys.DFS_DATANODE_USER_NAME_KEY)
+@TokenInfo(BlockTokenSelector.class)
+@ProtocolInfo(protocolName = 
+    "org.apache.hadoop.hdfs.protocol.ClientDatanodeProtocol",
+    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;
+}
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/protocolPB/ClientDatanodeProtocolServerSideTranslatorPB.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/protocolPB/ClientDatanodeProtocolServerSideTranslatorPB.java
new file mode 100644
index 0000000..86302f4
--- /dev/null
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/protocolPB/ClientDatanodeProtocolServerSideTranslatorPB.java
@@ -0,0 +1,159 @@
+/**
+ * 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.hdfs.protocol.BlockLocalPathInfo;
+import org.apache.hadoop.hdfs.protocol.ClientDatanodeProtocol;
+import org.apache.hadoop.hdfs.protocol.proto.ClientDatanodeProtocolProtos.DeleteBlockPoolRequestProto;
+import org.apache.hadoop.hdfs.protocol.proto.ClientDatanodeProtocolProtos.DeleteBlockPoolResponseProto;
+import org.apache.hadoop.hdfs.protocol.proto.ClientDatanodeProtocolProtos.GetBlockLocalPathInfoRequestProto;
+import org.apache.hadoop.hdfs.protocol.proto.ClientDatanodeProtocolProtos.GetBlockLocalPathInfoResponseProto;
+import org.apache.hadoop.hdfs.protocol.proto.ClientDatanodeProtocolProtos.GetReplicaVisibleLengthRequestProto;
+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;
+
+/**
+ * Implementation for protobuf service that forwards requests
+ * received on {@link ClientDatanodeProtocolPB} to the
+ * {@link ClientDatanodeProtocol} server implementation.
+ */
+@InterfaceAudience.Private
+public class ClientDatanodeProtocolServerSideTranslatorPB implements
+    ClientDatanodeProtocolPB {
+  private final static RefreshNamenodesResponseProto REFRESH_NAMENODE_RESP =
+      RefreshNamenodesResponseProto.newBuilder().build();
+  private final static DeleteBlockPoolResponseProto DELETE_BLOCKPOOL_RESP =
+      DeleteBlockPoolResponseProto.newBuilder().build();
+  
+  private final ClientDatanodeProtocol impl;
+
+  public ClientDatanodeProtocolServerSideTranslatorPB(
+      ClientDatanodeProtocol impl) {
+    this.impl = impl;
+  }
+
+  @Override
+  public GetReplicaVisibleLengthResponseProto getReplicaVisibleLength(
+      RpcController unused, GetReplicaVisibleLengthRequestProto request)
+      throws ServiceException {
+    long len;
+    try {
+      len = impl.getReplicaVisibleLength(PBHelper.convert(request.getBlock()));
+    } catch (IOException e) {
+      throw new ServiceException(e);
+    }
+    return GetReplicaVisibleLengthResponseProto.newBuilder().setLength(len)
+        .build();
+  }
+
+  @Override
+  public RefreshNamenodesResponseProto refreshNamenodes(
+      RpcController unused, RefreshNamenodesRequestProto request)
+      throws ServiceException {
+    try {
+      impl.refreshNamenodes();
+    } catch (IOException e) {
+      throw new ServiceException(e);
+    }
+    return REFRESH_NAMENODE_RESP;
+  }
+
+  @Override
+  public DeleteBlockPoolResponseProto deleteBlockPool(RpcController unused,
+      DeleteBlockPoolRequestProto request) throws ServiceException {
+    try {
+      impl.deleteBlockPool(request.getBlockPool(), request.getForce());
+    } catch (IOException e) {
+      throw new ServiceException(e);
+    }
+    return DELETE_BLOCKPOOL_RESP;
+  }
+
+  @Override
+  public GetBlockLocalPathInfoResponseProto getBlockLocalPathInfo(
+      RpcController unused, GetBlockLocalPathInfoRequestProto request)
+      throws ServiceException {
+    BlockLocalPathInfo resp;
+    try {
+      resp = impl.getBlockLocalPathInfo(PBHelper.convert(request.getBlock()), PBHelper.convert(request.getToken()));
+    } catch (IOException e) {
+      throw new ServiceException(e);
+    }
+    return GetBlockLocalPathInfoResponseProto.newBuilder()
+        .setBlock(PBHelper.convert(resp.getBlock()))
+        .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));
+  }
+}
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/protocolPB/ClientDatanodeProtocolTranslatorPB.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/protocolPB/ClientDatanodeProtocolTranslatorPB.java
new file mode 100644
index 0000000..d03f270
--- /dev/null
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/protocolPB/ClientDatanodeProtocolTranslatorPB.java
@@ -0,0 +1,214 @@
+/**
+ * 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 javax.net.SocketFactory;
+
+import org.apache.commons.logging.Log;
+import org.apache.commons.logging.LogFactory;
+import org.apache.hadoop.classification.InterfaceAudience;
+import org.apache.hadoop.classification.InterfaceStability;
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.fs.CommonConfigurationKeysPublic;
+import org.apache.hadoop.hdfs.protocol.BlockLocalPathInfo;
+import org.apache.hadoop.hdfs.protocol.ClientDatanodeProtocol;
+import org.apache.hadoop.hdfs.protocol.DatanodeID;
+import org.apache.hadoop.hdfs.protocol.ExtendedBlock;
+import org.apache.hadoop.hdfs.protocol.LocatedBlock;
+import org.apache.hadoop.hdfs.protocol.proto.ClientDatanodeProtocolProtos.DeleteBlockPoolRequestProto;
+import org.apache.hadoop.hdfs.protocol.proto.ClientDatanodeProtocolProtos.GetBlockLocalPathInfoRequestProto;
+import org.apache.hadoop.hdfs.protocol.proto.ClientDatanodeProtocolProtos.GetBlockLocalPathInfoResponseProto;
+import org.apache.hadoop.hdfs.protocol.proto.ClientDatanodeProtocolProtos.GetReplicaVisibleLengthRequestProto;
+import org.apache.hadoop.hdfs.protocol.proto.ClientDatanodeProtocolProtos.RefreshNamenodesRequestProto;
+import org.apache.hadoop.hdfs.protocolR23Compatible.ProtocolSignatureWritable;
+import org.apache.hadoop.hdfs.security.token.block.BlockTokenIdentifier;
+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.security.token.Token;
+
+import com.google.protobuf.RpcController;
+import com.google.protobuf.ServiceException;
+
+/**
+ * This class is the client side translator to translate the requests made on
+ * {@link ClientDatanodeProtocol} interfaces to the RPC server implementing
+ * {@link ClientDatanodeProtocolPB}.
+ */
+@InterfaceAudience.Private
+@InterfaceStability.Stable
+public class ClientDatanodeProtocolTranslatorPB implements
+    ProtocolMetaInterface, ClientDatanodeProtocol, Closeable {
+  public static final Log LOG = LogFactory
+      .getLog(ClientDatanodeProtocolTranslatorPB.class);
+  
+  /** RpcController is not used and hence is set to null */
+  private final static RpcController NULL_CONTROLLER = null;
+  private final ClientDatanodeProtocolPB rpcProxy;
+  private final static RefreshNamenodesRequestProto REFRESH_NAMENODES = 
+      RefreshNamenodesRequestProto.newBuilder().build();
+
+  public ClientDatanodeProtocolTranslatorPB(DatanodeID datanodeid,
+      Configuration conf, int socketTimeout, LocatedBlock locatedBlock)
+      throws IOException {
+    rpcProxy = createClientDatanodeProtocolProxy( datanodeid, conf, 
+                  socketTimeout, locatedBlock);
+  }
+  
+  public ClientDatanodeProtocolTranslatorPB(InetSocketAddress addr,
+      UserGroupInformation ticket, Configuration conf, SocketFactory factory)
+      throws IOException {
+    rpcProxy = createClientDatanodeProtocolProxy(addr, ticket, conf, factory, 0);
+  }
+  
+  /**
+   * Constructor.
+   * @param datanodeid Datanode to connect to.
+   * @param conf Configuration.
+   * @param socketTimeout Socket timeout to use.
+   * @throws IOException
+   */
+  public ClientDatanodeProtocolTranslatorPB(DatanodeID datanodeid,
+      Configuration conf, int socketTimeout) throws IOException {
+    InetSocketAddress addr = NetUtils.createSocketAddr(datanodeid.getHost()
+        + ":" + datanodeid.getIpcPort());
+    rpcProxy = createClientDatanodeProtocolProxy(addr,
+        UserGroupInformation.getCurrentUser(), conf,
+        NetUtils.getDefaultSocketFactory(conf), socketTimeout);
+  }
+
+  static ClientDatanodeProtocolPB createClientDatanodeProtocolProxy(
+      DatanodeID datanodeid, Configuration conf, int socketTimeout,
+      LocatedBlock locatedBlock) throws IOException {
+    InetSocketAddress addr = NetUtils.createSocketAddr(
+      datanodeid.getHost() + ":" + datanodeid.getIpcPort());
+    if (LOG.isDebugEnabled()) {
+      LOG.debug("ClientDatanodeProtocol addr=" + addr);
+    }
+    
+    // Since we're creating a new UserGroupInformation here, we know that no
+    // future RPC proxies will be able to re-use the same connection. And
+    // usages of this proxy tend to be one-off calls.
+    //
+    // This is a temporary fix: callers should really achieve this by using
+    // RPC.stopProxy() on the resulting object, but this is currently not
+    // working in trunk. See the discussion on HDFS-1965.
+    Configuration confWithNoIpcIdle = new Configuration(conf);
+    confWithNoIpcIdle.setInt(CommonConfigurationKeysPublic
+        .IPC_CLIENT_CONNECTION_MAXIDLETIME_KEY, 0);
+
+    UserGroupInformation ticket = UserGroupInformation
+        .createRemoteUser(locatedBlock.getBlock().getLocalBlock().toString());
+    ticket.addToken(locatedBlock.getBlockToken());
+    return createClientDatanodeProtocolProxy(addr, ticket, confWithNoIpcIdle,
+        NetUtils.getDefaultSocketFactory(conf), socketTimeout);
+  }
+  
+  static ClientDatanodeProtocolPB createClientDatanodeProtocolProxy(
+      InetSocketAddress addr, UserGroupInformation ticket, Configuration conf,
+      SocketFactory factory, int socketTimeout) throws IOException {
+    RPC.setProtocolEngine(conf, ClientDatanodeProtocolPB.class,
+        ProtobufRpcEngine.class);
+    return RPC.getProxy(ClientDatanodeProtocolPB.class,
+        RPC.getProtocolVersion(ClientDatanodeProtocolPB.class), addr, ticket,
+        conf, factory, socketTimeout);
+  }
+
+  @Override
+  public void close() {
+    RPC.stopProxy(rpcProxy);
+  }
+
+  @Override
+  public long getProtocolVersion(String protocolName, long clientVersion)
+      throws IOException {
+    return rpcProxy.getProtocolVersion(protocolName, clientVersion);
+  }
+
+  @Override
+  public ProtocolSignature getProtocolSignature(String protocol,
+      long clientVersion, int clientMethodsHash) throws IOException {
+    return ProtocolSignatureWritable.convert(rpcProxy.getProtocolSignature2(
+        protocol, clientVersion, clientMethodsHash));
+  }
+
+  @Override
+  public long getReplicaVisibleLength(ExtendedBlock b) throws IOException {
+    GetReplicaVisibleLengthRequestProto req = GetReplicaVisibleLengthRequestProto
+        .newBuilder().setBlock(PBHelper.convert(b)).build();
+    try {
+      return rpcProxy.getReplicaVisibleLength(NULL_CONTROLLER, req).getLength();
+    } catch (ServiceException e) {
+      throw ProtobufHelper.getRemoteException(e);
+    }
+  }
+
+  @Override
+  public void refreshNamenodes() throws IOException {
+    try {
+      rpcProxy.refreshNamenodes(NULL_CONTROLLER, REFRESH_NAMENODES);
+    } catch (ServiceException e) {
+      throw ProtobufHelper.getRemoteException(e);
+    }
+  }
+
+  @Override
+  public void deleteBlockPool(String bpid, boolean force) throws IOException {
+    DeleteBlockPoolRequestProto req = DeleteBlockPoolRequestProto.newBuilder()
+        .setBlockPool(bpid).setForce(force).build();
+    try {
+      rpcProxy.deleteBlockPool(NULL_CONTROLLER, req);
+    } catch (ServiceException e) {
+      throw ProtobufHelper.getRemoteException(e);
+    }
+  }
+
+  @Override
+  public BlockLocalPathInfo getBlockLocalPathInfo(ExtendedBlock block,
+      Token<BlockTokenIdentifier> token) throws IOException {
+    GetBlockLocalPathInfoRequestProto req =
+        GetBlockLocalPathInfoRequestProto.newBuilder()
+        .setBlock(PBHelper.convert(block))
+        .setToken(PBHelper.convert(token)).build();
+    GetBlockLocalPathInfoResponseProto resp;
+    try {
+      resp = rpcProxy.getBlockLocalPathInfo(NULL_CONTROLLER, req);
+    } catch (ServiceException e) {
+      throw ProtobufHelper.getRemoteException(e);
+    }
+    return new BlockLocalPathInfo(PBHelper.convert(resp.getBlock()),
+        resp.getLocalPath(), resp.getLocalMetaPath());
+  }
+
+  @Override
+  public boolean isMethodSupported(String methodName) throws IOException {
+    return RpcClientUtil.isMethodSupported(rpcProxy,
+        ClientDatanodeProtocolPB.class, RpcKind.RPC_PROTOCOL_BUFFER,
+        RPC.getProtocolVersion(ClientDatanodeProtocolPB.class), methodName);
+  }
+}
\ No newline at end of file
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/protocolPB/ClientNamenodeProtocolPB.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/protocolPB/ClientNamenodeProtocolPB.java
new file mode 100644
index 0000000..6e577ca
--- /dev/null
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/protocolPB/ClientNamenodeProtocolPB.java
@@ -0,0 +1,58 @@
+/**
+ * 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.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;
+
+
+@InterfaceAudience.Private
+@InterfaceStability.Stable
+@KerberosInfo(
+    serverPrincipal = DFSConfigKeys.DFS_NAMENODE_USER_NAME_KEY)
+@TokenInfo(DelegationTokenSelector.class)
+@ProtocolInfo(protocolName = HdfsConstants.CLIENT_NAMENODE_PROTOCOL_NAME, 
+    protocolVersion = 1)
+/**
+ * Protocol that a clients use to communicate with the NameNode.
+ * 
+ * Note: This extends the protocolbuffer service based interface to
+ * 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;
+}
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/protocolPB/ClientNamenodeProtocolServerSideTranslatorPB.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/protocolPB/ClientNamenodeProtocolServerSideTranslatorPB.java
new file mode 100644
index 0000000..c75c349
--- /dev/null
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/protocolPB/ClientNamenodeProtocolServerSideTranslatorPB.java
@@ -0,0 +1,887 @@
+/**
+ * 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.Arrays;
+import java.util.List;
+
+import org.apache.hadoop.classification.InterfaceAudience;
+import org.apache.hadoop.classification.InterfaceStability;
+import org.apache.hadoop.fs.ContentSummary;
+import org.apache.hadoop.fs.FsServerDefaults;
+import org.apache.hadoop.fs.Options.Rename;
+import org.apache.hadoop.hdfs.protocol.ClientProtocol;
+import org.apache.hadoop.hdfs.protocol.CorruptFileBlocks;
+import org.apache.hadoop.hdfs.protocol.DirectoryListing;
+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.proto.ClientNamenodeProtocolProtos.AbandonBlockRequestProto;
+import org.apache.hadoop.hdfs.protocol.proto.ClientNamenodeProtocolProtos.AbandonBlockResponseProto;
+import org.apache.hadoop.hdfs.protocol.proto.ClientNamenodeProtocolProtos.AddBlockRequestProto;
+import org.apache.hadoop.hdfs.protocol.proto.ClientNamenodeProtocolProtos.AddBlockResponseProto;
+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.CancelDelegationTokenResponseProto;
+import org.apache.hadoop.hdfs.protocol.proto.ClientNamenodeProtocolProtos.CompleteRequestProto;
+import org.apache.hadoop.hdfs.protocol.proto.ClientNamenodeProtocolProtos.CompleteResponseProto;
+import org.apache.hadoop.hdfs.protocol.proto.ClientNamenodeProtocolProtos.ConcatRequestProto;
+import org.apache.hadoop.hdfs.protocol.proto.ClientNamenodeProtocolProtos.ConcatResponseProto;
+import org.apache.hadoop.hdfs.protocol.proto.ClientNamenodeProtocolProtos.CreateRequestProto;
+import org.apache.hadoop.hdfs.protocol.proto.ClientNamenodeProtocolProtos.CreateResponseProto;
+import org.apache.hadoop.hdfs.protocol.proto.ClientNamenodeProtocolProtos.CreateSymlinkRequestProto;
+import org.apache.hadoop.hdfs.protocol.proto.ClientNamenodeProtocolProtos.CreateSymlinkResponseProto;
+import org.apache.hadoop.hdfs.protocol.proto.ClientNamenodeProtocolProtos.DeleteRequestProto;
+import org.apache.hadoop.hdfs.protocol.proto.ClientNamenodeProtocolProtos.DeleteResponseProto;
+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.FinalizeUpgradeResponseProto;
+import org.apache.hadoop.hdfs.protocol.proto.ClientNamenodeProtocolProtos.FsyncRequestProto;
+import org.apache.hadoop.hdfs.protocol.proto.ClientNamenodeProtocolProtos.FsyncResponseProto;
+import org.apache.hadoop.hdfs.protocol.proto.ClientNamenodeProtocolProtos.GetAdditionalDatanodeRequestProto;
+import org.apache.hadoop.hdfs.protocol.proto.ClientNamenodeProtocolProtos.GetAdditionalDatanodeResponseProto;
+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.GetBlockLocationsResponseProto.Builder;
+import org.apache.hadoop.hdfs.protocol.proto.ClientNamenodeProtocolProtos.GetContentSummaryRequestProto;
+import org.apache.hadoop.hdfs.protocol.proto.ClientNamenodeProtocolProtos.GetContentSummaryResponseProto;
+import org.apache.hadoop.hdfs.protocol.proto.ClientNamenodeProtocolProtos.GetDatanodeReportRequestProto;
+import org.apache.hadoop.hdfs.protocol.proto.ClientNamenodeProtocolProtos.GetDatanodeReportResponseProto;
+import org.apache.hadoop.hdfs.protocol.proto.ClientNamenodeProtocolProtos.GetDelegationTokenRequestProto;
+import org.apache.hadoop.hdfs.protocol.proto.ClientNamenodeProtocolProtos.GetDelegationTokenResponseProto;
+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.GetFsStatsResponseProto;
+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.GetLinkTargetResponseProto;
+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.GetPreferredBlockSizeResponseProto;
+import org.apache.hadoop.hdfs.protocol.proto.ClientNamenodeProtocolProtos.GetServerDefaultsRequestProto;
+import org.apache.hadoop.hdfs.protocol.proto.ClientNamenodeProtocolProtos.GetServerDefaultsResponseProto;
+import org.apache.hadoop.hdfs.protocol.proto.ClientNamenodeProtocolProtos.ListCorruptFileBlocksRequestProto;
+import org.apache.hadoop.hdfs.protocol.proto.ClientNamenodeProtocolProtos.ListCorruptFileBlocksResponseProto;
+import org.apache.hadoop.hdfs.protocol.proto.ClientNamenodeProtocolProtos.MetaSaveRequestProto;
+import org.apache.hadoop.hdfs.protocol.proto.ClientNamenodeProtocolProtos.MetaSaveResponseProto;
+import org.apache.hadoop.hdfs.protocol.proto.ClientNamenodeProtocolProtos.MkdirsRequestProto;
+import org.apache.hadoop.hdfs.protocol.proto.ClientNamenodeProtocolProtos.MkdirsResponseProto;
+import org.apache.hadoop.hdfs.protocol.proto.ClientNamenodeProtocolProtos.RecoverLeaseRequestProto;
+import org.apache.hadoop.hdfs.protocol.proto.ClientNamenodeProtocolProtos.RecoverLeaseResponseProto;
+import org.apache.hadoop.hdfs.protocol.proto.ClientNamenodeProtocolProtos.RefreshNodesRequestProto;
+import org.apache.hadoop.hdfs.protocol.proto.ClientNamenodeProtocolProtos.RefreshNodesResponseProto;
+import org.apache.hadoop.hdfs.protocol.proto.ClientNamenodeProtocolProtos.Rename2RequestProto;
+import org.apache.hadoop.hdfs.protocol.proto.ClientNamenodeProtocolProtos.Rename2ResponseProto;
+import org.apache.hadoop.hdfs.protocol.proto.ClientNamenodeProtocolProtos.RenameRequestProto;
+import org.apache.hadoop.hdfs.protocol.proto.ClientNamenodeProtocolProtos.RenameResponseProto;
+import org.apache.hadoop.hdfs.protocol.proto.ClientNamenodeProtocolProtos.RenewDelegationTokenRequestProto;
+import org.apache.hadoop.hdfs.protocol.proto.ClientNamenodeProtocolProtos.RenewDelegationTokenResponseProto;
+import org.apache.hadoop.hdfs.protocol.proto.ClientNamenodeProtocolProtos.RenewLeaseRequestProto;
+import org.apache.hadoop.hdfs.protocol.proto.ClientNamenodeProtocolProtos.RenewLeaseResponseProto;
+import org.apache.hadoop.hdfs.protocol.proto.ClientNamenodeProtocolProtos.ReportBadBlocksRequestProto;
+import org.apache.hadoop.hdfs.protocol.proto.ClientNamenodeProtocolProtos.ReportBadBlocksResponseProto;
+import org.apache.hadoop.hdfs.protocol.proto.ClientNamenodeProtocolProtos.RestoreFailedStorageRequestProto;
+import org.apache.hadoop.hdfs.protocol.proto.ClientNamenodeProtocolProtos.RestoreFailedStorageResponseProto;
+import org.apache.hadoop.hdfs.protocol.proto.ClientNamenodeProtocolProtos.SaveNamespaceRequestProto;
+import org.apache.hadoop.hdfs.protocol.proto.ClientNamenodeProtocolProtos.SaveNamespaceResponseProto;
+import org.apache.hadoop.hdfs.protocol.proto.ClientNamenodeProtocolProtos.SetBalancerBandwidthRequestProto;
+import org.apache.hadoop.hdfs.protocol.proto.ClientNamenodeProtocolProtos.SetBalancerBandwidthResponseProto;
+import org.apache.hadoop.hdfs.protocol.proto.ClientNamenodeProtocolProtos.SetOwnerRequestProto;
+import org.apache.hadoop.hdfs.protocol.proto.ClientNamenodeProtocolProtos.SetOwnerResponseProto;
+import org.apache.hadoop.hdfs.protocol.proto.ClientNamenodeProtocolProtos.SetPermissionRequestProto;
+import org.apache.hadoop.hdfs.protocol.proto.ClientNamenodeProtocolProtos.SetPermissionResponseProto;
+import org.apache.hadoop.hdfs.protocol.proto.ClientNamenodeProtocolProtos.SetQuotaRequestProto;
+import org.apache.hadoop.hdfs.protocol.proto.ClientNamenodeProtocolProtos.SetQuotaResponseProto;
+import org.apache.hadoop.hdfs.protocol.proto.ClientNamenodeProtocolProtos.SetReplicationRequestProto;
+import org.apache.hadoop.hdfs.protocol.proto.ClientNamenodeProtocolProtos.SetReplicationResponseProto;
+import org.apache.hadoop.hdfs.protocol.proto.ClientNamenodeProtocolProtos.SetSafeModeRequestProto;
+import org.apache.hadoop.hdfs.protocol.proto.ClientNamenodeProtocolProtos.SetSafeModeResponseProto;
+import org.apache.hadoop.hdfs.protocol.proto.ClientNamenodeProtocolProtos.SetTimesRequestProto;
+import org.apache.hadoop.hdfs.protocol.proto.ClientNamenodeProtocolProtos.SetTimesResponseProto;
+import org.apache.hadoop.hdfs.protocol.proto.ClientNamenodeProtocolProtos.UpdateBlockForPipelineRequestProto;
+import org.apache.hadoop.hdfs.protocol.proto.ClientNamenodeProtocolProtos.UpdateBlockForPipelineResponseProto;
+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;
+
+/**
+ * This class is used on the server side. Calls come across the wire for the
+ * for protocol {@link ClientNamenodeProtocolPB}.
+ * This class translates the PB data types
+ * to the native data types used inside the NN as specified in the generic
+ * ClientProtocol.
+ */
+@InterfaceAudience.Private
+@InterfaceStability.Stable
+public class ClientNamenodeProtocolServerSideTranslatorPB implements
+    ClientNamenodeProtocolPB {
+  final private ClientProtocol server;
+
+  /**
+   * Constructor
+   * 
+   * @param server - the NN server
+   * @throws IOException
+   */
+  public ClientNamenodeProtocolServerSideTranslatorPB(ClientProtocol server)
+      throws IOException {
+    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)
+      throws ServiceException {
+    try {
+      LocatedBlocks b = server.getBlockLocations(req.getSrc(), req.getOffset(),
+          req.getLength());
+      Builder builder = GetBlockLocationsResponseProto
+          .newBuilder();
+      if (b != null) {
+        builder.setLocations(PBHelper.convert(b)).build();
+      }
+      return builder.build();
+    } catch (IOException e) {
+      throw new ServiceException(e);
+    }
+  }
+
+  @Override
+  public GetServerDefaultsResponseProto getServerDefaults(
+      RpcController controller, GetServerDefaultsRequestProto req)
+      throws ServiceException {
+    try {
+      FsServerDefaults result = server.getServerDefaults();
+      return GetServerDefaultsResponseProto.newBuilder()
+          .setServerDefaults(PBHelper.convert(result))
+          .build();
+    } catch (IOException e) {
+      throw new ServiceException(e);
+    }
+  }
+
+  
+  static final CreateResponseProto VOID_CREATE_RESPONSE = 
+      CreateResponseProto.newBuilder().build();
+  
+  @Override
+  public CreateResponseProto create(RpcController controller,
+      CreateRequestProto req) throws ServiceException {
+    try {
+      server.create(req.getSrc(), PBHelper.convert(req.getMasked()),
+          req.getClientName(), PBHelper.convert(req.getCreateFlag()),
+          req.getCreateParent(), (short) req.getReplication(),
+          req.getBlockSize());
+    } catch (IOException e) {
+      throw new ServiceException(e);
+    }
+    return VOID_CREATE_RESPONSE;
+  }
+  
+  static final AppendResponseProto NULL_APPEND_RESPONSE = 
+      AppendResponseProto.newBuilder().build();
+  
+  @Override
+  public AppendResponseProto append(RpcController controller,
+      AppendRequestProto req) throws ServiceException {
+    try {
+      LocatedBlock result = server.append(req.getSrc(), req.getClientName());
+      if (result != null) {
+        return AppendResponseProto.newBuilder()
+            .setBlock(PBHelper.convert(result)).build();
+      }
+      return NULL_APPEND_RESPONSE;
+    } catch (IOException e) {
+      throw new ServiceException(e);
+    }
+  }
+
+  @Override
+  public SetReplicationResponseProto setReplication(RpcController controller,
+      SetReplicationRequestProto req) throws ServiceException {
+    try {
+      boolean result = 
+          server.setReplication(req.getSrc(), (short) req.getReplication());
+      return SetReplicationResponseProto.newBuilder().setResult(result).build();
+    } catch (IOException e) {
+      throw new ServiceException(e);
+    }
+  }
+
+
+  static final SetPermissionResponseProto VOID_SET_PERM_RESPONSE = 
+      SetPermissionResponseProto.newBuilder().build();
+
+  @Override
+  public SetPermissionResponseProto setPermission(RpcController controller,
+      SetPermissionRequestProto req) throws ServiceException {
+    try {
+      server.setPermission(req.getSrc(), PBHelper.convert(req.getPermission()));
+    } catch (IOException e) {
+      throw new ServiceException(e);
+    }
+    return VOID_SET_PERM_RESPONSE;
+  }
+
+  static final SetOwnerResponseProto VOID_SET_OWNER_RESPONSE = 
+      SetOwnerResponseProto.newBuilder().build();
+
+  @Override
+  public SetOwnerResponseProto setOwner(RpcController controller,
+      SetOwnerRequestProto req) throws ServiceException {
+    try {
+      server.setOwner(req.getSrc(), 
+          req.hasUsername() ? req.getUsername() : null,
+          req.hasGroupname() ? req.getGroupname() : null);
+    } catch (IOException e) {
+      throw new ServiceException(e);
+    }
+    return VOID_SET_OWNER_RESPONSE;
+  }
+
+  static final AbandonBlockResponseProto VOID_ADD_BLOCK_RESPONSE = 
+      AbandonBlockResponseProto.newBuilder().build();
+
+  @Override
+  public AbandonBlockResponseProto abandonBlock(RpcController controller,
+      AbandonBlockRequestProto req) throws ServiceException {
+    try {
+      server.abandonBlock(PBHelper.convert(req.getB()), req.getSrc(),
+          req.getHolder());
+    } catch (IOException e) {
+      throw new ServiceException(e);
+    }
+    return VOID_ADD_BLOCK_RESPONSE;
+  }
+
+  @Override
+  public AddBlockResponseProto addBlock(RpcController controller,
+      AddBlockRequestProto req) throws ServiceException {
+    
+    try {
+      List<DatanodeInfoProto> excl = req.getExcludeNodesList();
+      LocatedBlock result = server.addBlock(req.getSrc(), req.getClientName(),
+          req.hasPrevious() ? PBHelper.convert(req.getPrevious()) : null,
+          (excl == null || 
+           excl.size() == 0) ? null : 
+            PBHelper.convert(excl.toArray(new DatanodeInfoProto[excl.size()])));
+      return AddBlockResponseProto.newBuilder().setBlock(
+          PBHelper.convert(result)).build();
+    } catch (IOException e) {
+      throw new ServiceException(e);
+    }
+  }
+
+  @Override
+  public GetAdditionalDatanodeResponseProto getAdditionalDatanode(
+      RpcController controller, GetAdditionalDatanodeRequestProto req)
+      throws ServiceException {
+    try {
+      List<DatanodeInfoProto> existingList = req.getExistingsList();
+      List<DatanodeInfoProto> excludesList = req.getExcludesList();
+      LocatedBlock result = server.getAdditionalDatanode(
+          req.getSrc(), PBHelper.convert(req.getBlk()),
+          PBHelper.convert(existingList.toArray(
+              new DatanodeInfoProto[existingList.size()])),
+          PBHelper.convert(excludesList.toArray(
+              new DatanodeInfoProto[excludesList.size()])), 
+              req.getNumAdditionalNodes(), req.getClientName());
+      return GetAdditionalDatanodeResponseProto.newBuilder().setBlock(
+          PBHelper.convert(result))
+          .build();
+    } catch (IOException e) {
+      throw new ServiceException(e);
+    }
+  }
+  
+  @Override
+  public CompleteResponseProto complete(RpcController controller,
+      CompleteRequestProto req) throws ServiceException {
+    try {
+      boolean result = 
+          server.complete(req.getSrc(), req.getClientName(),
+          req.hasLast() ? PBHelper.convert(req.getLast()) : null);
+      return CompleteResponseProto.newBuilder().setResult(result).build();
+    } catch (IOException e) {
+      throw new ServiceException(e);
+    }
+  }
+  
+  static final ReportBadBlocksResponseProto VOID_REP_BAD_BLOCK_RESPONSE = 
+      ReportBadBlocksResponseProto.newBuilder().build();
+  
+  @Override
+  public ReportBadBlocksResponseProto reportBadBlocks(RpcController controller,
+      ReportBadBlocksRequestProto req) throws ServiceException {
+    try {
+      List<LocatedBlockProto> bl = req.getBlocksList();
+      server.reportBadBlocks(PBHelper.convertLocatedBlock(
+              bl.toArray(new LocatedBlockProto[bl.size()])));
+    } catch (IOException e) {
+      throw new ServiceException(e);
+    }
+    return VOID_REP_BAD_BLOCK_RESPONSE;
+  }
+
+  static final ConcatResponseProto VOID_CONCAT_RESPONSE = 
+      ConcatResponseProto.newBuilder().build();
+  
+  @Override
+  public ConcatResponseProto concat(RpcController controller,
+      ConcatRequestProto req) throws ServiceException {
+    try {
+      List<String> srcs = req.getSrcsList();
+      server.concat(req.getTrg(), srcs.toArray(new String[srcs.size()]));
+    } catch (IOException e) {
+      throw new ServiceException(e);
+    }
+    return VOID_CONCAT_RESPONSE;
+  }
+
+  @Override
+  public RenameResponseProto rename(RpcController controller,
+      RenameRequestProto req) throws ServiceException {
+    try {
+      boolean result = server.rename(req.getSrc(), req.getDst());
+      return RenameResponseProto.newBuilder().setResult(result).build();
+    } catch (IOException e) {
+      throw new ServiceException(e);
+    }
+  }
+
+  static final Rename2ResponseProto VOID_RENAME2_RESPONSE = 
+      Rename2ResponseProto.newBuilder().build();
+  
+  @Override
+  public Rename2ResponseProto rename2(RpcController controller,
+      Rename2RequestProto req) throws ServiceException {
+
+    try {
+      server.rename2(req.getSrc(), req.getDst(), 
+          req.getOverwriteDest() ? Rename.OVERWRITE : Rename.NONE);
+    } catch (IOException e) {
+      throw new ServiceException(e);
+    }   
+    return VOID_RENAME2_RESPONSE;
+  }
+
+  @Override
+  public DeleteResponseProto delete(RpcController controller,
+    DeleteRequestProto req) throws ServiceException {
+    try {
+      boolean result =  server.delete(req.getSrc(), req.getRecursive());
+      return DeleteResponseProto.newBuilder().setResult(result).build();
+    } catch (IOException e) {
+      throw new ServiceException(e);
+    }
+  }
+
+  @Override
+  public MkdirsResponseProto mkdirs(RpcController controller,
+      MkdirsRequestProto req) throws ServiceException {
+    try {
+      boolean result = server.mkdirs(req.getSrc(),
+          PBHelper.convert(req.getMasked()), req.getCreateParent());
+      return MkdirsResponseProto.newBuilder().setResult(result).build();
+    } catch (IOException e) {
+      throw new ServiceException(e);
+    }
+  }
+
+  static final GetListingResponseProto NULL_GETLISTING_RESPONSE = 
+      GetListingResponseProto.newBuilder().build();
+  @Override
+  public GetListingResponseProto getListing(RpcController controller,
+      GetListingRequestProto req) throws ServiceException {
+    try {
+      DirectoryListing result = server.getListing(
+          req.getSrc(), req.getStartAfter().toByteArray(),
+          req.getNeedLocation());
+      if (result !=null) {
+        return GetListingResponseProto.newBuilder().setDirList(
+          PBHelper.convert(result)).build();
+      } else {
+        return NULL_GETLISTING_RESPONSE;
+      }
+    } catch (IOException e) {
+      throw new ServiceException(e);
+    }
+  }
+  
+  static final RenewLeaseResponseProto VOID_RENEWLEASE_RESPONSE = 
+      RenewLeaseResponseProto.newBuilder().build();
+  
+  @Override
+  public RenewLeaseResponseProto renewLease(RpcController controller,
+      RenewLeaseRequestProto req) throws ServiceException {
+    try {
+      server.renewLease(req.getClientName());
+      return VOID_RENEWLEASE_RESPONSE;
+    } catch (IOException e) {
+      throw new ServiceException(e);
+    }
+  }
+
+  @Override
+  public RecoverLeaseResponseProto recoverLease(RpcController controller,
+      RecoverLeaseRequestProto req) throws ServiceException {
+    try {
+      boolean result = server.recoverLease(req.getSrc(), req.getClientName());
+      return RecoverLeaseResponseProto.newBuilder().setResult(result).build();
+    } catch (IOException e) {
+      throw new ServiceException(e);
+    }
+  }
+  
+  @Override
+  public RestoreFailedStorageResponseProto restoreFailedStorage(
+      RpcController controller, RestoreFailedStorageRequestProto req)
+      throws ServiceException {
+    try {
+      boolean result = server.restoreFailedStorage(req.getArg());
+      return RestoreFailedStorageResponseProto.newBuilder().setResult(result)
+          .build();
+    } catch (IOException e) {
+      throw new ServiceException(e);
+    }
+  }
+
+  @Override
+  public GetFsStatsResponseProto getFsStats(RpcController controller,
+      GetFsStatusRequestProto req) throws ServiceException {
+    try {
+      return PBHelper.convert(server.getStats());
+    } catch (IOException e) {
+      throw new ServiceException(e);
+    }
+  }
+
+  @Override
+  public GetDatanodeReportResponseProto getDatanodeReport(
+      RpcController controller, GetDatanodeReportRequestProto req)
+      throws ServiceException {
+    try {
+      DatanodeInfoProto[] result = PBHelper.convert(server
+          .getDatanodeReport(PBHelper.convert(req.getType())));
+      return GetDatanodeReportResponseProto.newBuilder()
+          .addAllDi(Arrays.asList(result)).build();
+    } catch (IOException e) {
+      throw new ServiceException(e);
+    }
+  }
+
+  @Override
+  public GetPreferredBlockSizeResponseProto getPreferredBlockSize(
+      RpcController controller, GetPreferredBlockSizeRequestProto req)
+      throws ServiceException {
+    try {
+      long result = server.getPreferredBlockSize(req.getFilename());
+      return GetPreferredBlockSizeResponseProto.newBuilder().setBsize(result)
+          .build();
+    } catch (IOException e) {
+      throw new ServiceException(e);
+    }
+  }
+
+  @Override
+  public SetSafeModeResponseProto setSafeMode(RpcController controller,
+      SetSafeModeRequestProto req) throws ServiceException {
+    try {
+      boolean result = server.setSafeMode(PBHelper.convert(req.getAction()));
+      return SetSafeModeResponseProto.newBuilder().setResult(result).build();
+    } catch (IOException e) {
+      throw new ServiceException(e);
+    }
+  }
+  
+  static final SaveNamespaceResponseProto VOID_SAVENAMESPACE_RESPONSE = 
+      SaveNamespaceResponseProto.newBuilder().build();
+
+  @Override
+  public SaveNamespaceResponseProto saveNamespace(RpcController controller,
+      SaveNamespaceRequestProto req) throws ServiceException {
+    try {
+      server.saveNamespace();
+      return VOID_SAVENAMESPACE_RESPONSE;
+    } catch (IOException e) {
+      throw new ServiceException(e);
+    }
+
+  }
+
+  static final RefreshNodesResponseProto VOID_REFRESHNODES_RESPONSE = 
+      RefreshNodesResponseProto.newBuilder().build();
+
+  @Override
+  public RefreshNodesResponseProto refreshNodes(RpcController controller,
+      RefreshNodesRequestProto req) throws ServiceException {
+    try {
+      server.refreshNodes();
+      return VOID_REFRESHNODES_RESPONSE;
+    } catch (IOException e) {
+      throw new ServiceException(e);
+    }
+
+  }
+
+  static final FinalizeUpgradeResponseProto VOID_FINALIZEUPGRADE_RESPONSE = 
+      FinalizeUpgradeResponseProto.newBuilder().build();
+
+  @Override
+  public FinalizeUpgradeResponseProto finalizeUpgrade(RpcController controller,
+      FinalizeUpgradeRequestProto req) throws ServiceException {
+    try {
+      server.finalizeUpgrade();
+      return VOID_FINALIZEUPGRADE_RESPONSE;
+    } catch (IOException e) {
+      throw new ServiceException(e);
+    }
+  }
+
+  @Override
+  public DistributedUpgradeProgressResponseProto distributedUpgradeProgress(
+      RpcController controller, DistributedUpgradeProgressRequestProto req)
+      throws ServiceException {
+    try {
+      UpgradeStatusReport result = server.distributedUpgradeProgress(PBHelper
+          .convert(req.getAction()));
+      DistributedUpgradeProgressResponseProto.Builder builder = 
+          DistributedUpgradeProgressResponseProto.newBuilder();
+      if (result != null) {
+        builder.setReport(PBHelper.convert(result));
+      }
+      return builder.build();
+    } catch (IOException e) {
+      throw new ServiceException(e);
+    }
+  }
+
+  @Override
+  public ListCorruptFileBlocksResponseProto listCorruptFileBlocks(
+      RpcController controller, ListCorruptFileBlocksRequestProto req)
+      throws ServiceException {
+    try {
+      CorruptFileBlocks result = server.listCorruptFileBlocks(
+          req.getPath(), req.hasCookie() ? req.getCookie(): null);
+      return ListCorruptFileBlocksResponseProto.newBuilder()
+          .setCorrupt(PBHelper.convert(result))
+          .build();
+    } catch (IOException e) {
+      throw new ServiceException(e);
+    }
+  }
+
+  static final MetaSaveResponseProto VOID_METASAVE_RESPONSE = 
+      MetaSaveResponseProto.newBuilder().build();
+  
+  @Override
+  public MetaSaveResponseProto metaSave(RpcController controller,
+      MetaSaveRequestProto req) throws ServiceException {
+    try {
+      server.metaSave(req.getFilename());
+      return VOID_METASAVE_RESPONSE;
+    } catch (IOException e) {
+      throw new ServiceException(e);
+    }
+
+  }
+
+  static final GetFileInfoResponseProto NULL_GETFILEINFO_RESPONSE = 
+      GetFileInfoResponseProto.newBuilder().build();
+  @Override
+  public GetFileInfoResponseProto getFileInfo(RpcController controller,
+      GetFileInfoRequestProto req) throws ServiceException {
+    try {
+      HdfsFileStatus result = server.getFileInfo(req.getSrc());
+ 
+      if (result != null) {
+        return GetFileInfoResponseProto.newBuilder().setFs(
+            PBHelper.convert(result)).build();
+      }
+      return NULL_GETFILEINFO_RESPONSE;      
+    } catch (IOException e) {
+      throw new ServiceException(e);
+    }
+  }
+
+  static final GetFileLinkInfoResponseProto NULL_GETFILELINKINFO_RESPONSE = 
+      GetFileLinkInfoResponseProto.newBuilder().build();
+  @Override
+  public GetFileLinkInfoResponseProto getFileLinkInfo(RpcController controller,
+      GetFileLinkInfoRequestProto req) throws ServiceException {
+    try {
+      HdfsFileStatus result = server.getFileLinkInfo(req.getSrc());
+      if (result != null) {
+        System.out.println("got non null result for getFileLinkInfo for " + req.getSrc());
+        return GetFileLinkInfoResponseProto.newBuilder().setFs(
+            PBHelper.convert(result)).build();
+      } else {
+        System.out.println("got  null result for getFileLinkInfo for " + req.getSrc());
+        return NULL_GETFILELINKINFO_RESPONSE;      
+      }
+
+    } catch (IOException e) {
+      throw new ServiceException(e);
+    }
+  }
+
+  @Override
+  public GetContentSummaryResponseProto getContentSummary(
+      RpcController controller, GetContentSummaryRequestProto req)
+      throws ServiceException {
+    try {
+      ContentSummary result = server.getContentSummary(req.getPath());
+      return GetContentSummaryResponseProto.newBuilder()
+          .setSummary(PBHelper.convert(result)).build();
+    } catch (IOException e) {
+      throw new ServiceException(e);
+    }
+  }
+  
+  static final SetQuotaResponseProto VOID_SETQUOTA_RESPONSE = 
+      SetQuotaResponseProto.newBuilder().build();
+  
+  @Override
+  public SetQuotaResponseProto setQuota(RpcController controller,
+      SetQuotaRequestProto req) throws ServiceException {
+    try {
+      server.setQuota(req.getPath(), req.getNamespaceQuota(),
+          req.getDiskspaceQuota());
+      return VOID_SETQUOTA_RESPONSE;
+    } catch (IOException e) {
+      throw new ServiceException(e);
+    }
+  }
+  
+  static final FsyncResponseProto VOID_FSYNC_RESPONSE = 
+      FsyncResponseProto.newBuilder().build();
+
+  @Override
+  public FsyncResponseProto fsync(RpcController controller,
+      FsyncRequestProto req) throws ServiceException {
+    try {
+      server.fsync(req.getSrc(), req.getClient());
+      return VOID_FSYNC_RESPONSE;
+    } catch (IOException e) {
+      throw new ServiceException(e);
+    }
+  }
+
+  static final SetTimesResponseProto VOID_SETTIMES_RESPONSE = 
+      SetTimesResponseProto.newBuilder().build();
+
+  @Override
+  public SetTimesResponseProto setTimes(RpcController controller,
+      SetTimesRequestProto req) throws ServiceException {
+    try {
+      server.setTimes(req.getSrc(), req.getMtime(), req.getAtime());
+      return VOID_SETTIMES_RESPONSE;
+    } catch (IOException e) {
+      throw new ServiceException(e);
+    }
+  }
+
+  static final CreateSymlinkResponseProto VOID_CREATESYMLINK_RESPONSE = 
+      CreateSymlinkResponseProto.newBuilder().build();
+
+  @Override
+  public CreateSymlinkResponseProto createSymlink(RpcController controller,
+      CreateSymlinkRequestProto req) throws ServiceException {
+    try {
+      server.createSymlink(req.getTarget(), req.getLink(),
+          PBHelper.convert(req.getDirPerm()), req.getCreateParent());
+      return VOID_CREATESYMLINK_RESPONSE;
+    } catch (IOException e) {
+      throw new ServiceException(e);
+    }
+  }
+
+  @Override
+  public GetLinkTargetResponseProto getLinkTarget(RpcController controller,
+      GetLinkTargetRequestProto req) throws ServiceException {
+    try {
+      String result = server.getLinkTarget(req.getPath());
+      return GetLinkTargetResponseProto.newBuilder().setTargetPath(result)
+          .build();
+    } catch (IOException e) {
+      throw new ServiceException(e);
+    }
+  }
+
+  @Override
+  public UpdateBlockForPipelineResponseProto updateBlockForPipeline(
+      RpcController controller, UpdateBlockForPipelineRequestProto req)
+      throws ServiceException {
+    try {
+      LocatedBlockProto result = PBHelper.convert(server
+          .updateBlockForPipeline(PBHelper.convert(req.getBlock()),
+              req.getClientName()));
+      return UpdateBlockForPipelineResponseProto.newBuilder().setBlock(result)
+          .build();
+    } catch (IOException e) {
+      throw new ServiceException(e);
+    }
+  }
+
+  static final UpdatePipelineResponseProto VOID_UPDATEPIPELINE_RESPONSE = 
+      UpdatePipelineResponseProto.newBuilder().build();
+
+  @Override
+  public UpdatePipelineResponseProto updatePipeline(RpcController controller,
+      UpdatePipelineRequestProto req) throws ServiceException {
+    try {
+      List<DatanodeIDProto> newNodes = req.getNewNodesList();
+      server
+          .updatePipeline(req.getClientName(), PBHelper.convert(req
+              .getOldBlock()), PBHelper.convert(req.getNewBlock()), PBHelper
+              .convert(newNodes.toArray(new DatanodeIDProto[newNodes.size()])));
+      return VOID_UPDATEPIPELINE_RESPONSE;
+    } catch (IOException e) {
+      throw new ServiceException(e);
+    }
+  }
+
+  @Override
+  public GetDelegationTokenResponseProto getDelegationToken(
+      RpcController controller, GetDelegationTokenRequestProto req)
+      throws ServiceException {
+    try {
+      BlockTokenIdentifierProto result = PBHelper.convert(server
+          .getDelegationToken(new Text(req.getRenewer())));
+      return GetDelegationTokenResponseProto.newBuilder().setToken(result)
+          .build();
+    } catch (IOException e) {
+      throw new ServiceException(e);
+    }
+  }
+
+  @Override
+  public RenewDelegationTokenResponseProto renewDelegationToken(
+      RpcController controller, RenewDelegationTokenRequestProto req)
+      throws ServiceException {
+    try {
+      long result = server.renewDelegationToken(PBHelper
+          .convertDelegationToken(req.getToken()));
+      return RenewDelegationTokenResponseProto.newBuilder()
+          .setNewExireTime(result).build();
+    } catch (IOException e) {
+      throw new ServiceException(e);
+    }
+  }
+
+  static final CancelDelegationTokenResponseProto 
+      VOID_CANCELDELEGATIONTOKEN_RESPONSE = 
+      CancelDelegationTokenResponseProto.newBuilder().build();
+  
+  @Override
+  public CancelDelegationTokenResponseProto cancelDelegationToken(
+      RpcController controller, CancelDelegationTokenRequestProto req)
+      throws ServiceException {
+    try {
+      server.cancelDelegationToken(PBHelper.convertDelegationToken(req
+          .getToken()));
+      return VOID_CANCELDELEGATIONTOKEN_RESPONSE;
+    } catch (IOException e) {
+      throw new ServiceException(e);
+    }
+  }
+
+  static final SetBalancerBandwidthResponseProto 
+    VOID_SETBALANCERBANDWIDTH_RESPONSE = 
+      SetBalancerBandwidthResponseProto.newBuilder().build();
+
+  @Override
+  public SetBalancerBandwidthResponseProto setBalancerBandwidth(
+      RpcController controller, SetBalancerBandwidthRequestProto req)
+      throws ServiceException {
+    try {
+      server.setBalancerBandwidth(req.getBandwidth());
+      return VOID_SETBALANCERBANDWIDTH_RESPONSE;
+    } catch (IOException e) {
+      throw new ServiceException(e);
+    }
+  }
+}
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/protocolPB/ClientNamenodeProtocolTranslatorPB.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/protocolPB/ClientNamenodeProtocolTranslatorPB.java
new file mode 100644
index 0000000..dc9e7aa
--- /dev/null
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/protocolPB/ClientNamenodeProtocolTranslatorPB.java
@@ -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);
+  }
+}
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/protocolPB/DatanodeProtocolClientSideTranslatorPB.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/protocolPB/DatanodeProtocolClientSideTranslatorPB.java
new file mode 100644
index 0000000..7b60456
--- /dev/null
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/protocolPB/DatanodeProtocolClientSideTranslatorPB.java
@@ -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);
+  }
+}
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/protocolPB/DatanodeProtocolPB.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/protocolPB/DatanodeProtocolPB.java
new file mode 100644
index 0000000..60f0c75
--- /dev/null
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/protocolPB/DatanodeProtocolPB.java
@@ -0,0 +1,48 @@
+/**
+ * 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.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(
+    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 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;
+}
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/protocolPB/DatanodeProtocolServerSideTranslatorPB.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/protocolPB/DatanodeProtocolServerSideTranslatorPB.java
new file mode 100644
index 0000000..2ad7f31
--- /dev/null
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/protocolPB/DatanodeProtocolServerSideTranslatorPB.java
@@ -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));
+  }
+
+}
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/protocolPB/GetUserMappingsProtocolClientSideTranslatorPB.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/protocolPB/GetUserMappingsProtocolClientSideTranslatorPB.java
new file mode 100644
index 0000000..a968bfb
--- /dev/null
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/protocolPB/GetUserMappingsProtocolClientSideTranslatorPB.java
@@ -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);
+  }
+}
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/protocolPB/GetUserMappingsProtocolPB.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/protocolPB/GetUserMappingsProtocolPB.java
new file mode 100644
index 0000000..54eeb0a
--- /dev/null
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/protocolPB/GetUserMappingsProtocolPB.java
@@ -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;
+}
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/protocolPB/GetUserMappingsProtocolServerSideTranslatorPB.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/protocolPB/GetUserMappingsProtocolServerSideTranslatorPB.java
new file mode 100644
index 0000000..4dc771d
--- /dev/null
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/protocolPB/GetUserMappingsProtocolServerSideTranslatorPB.java
@@ -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();
+  }
+}
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/protocolPB/InterDatanodeProtocolPB.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/protocolPB/InterDatanodeProtocolPB.java
new file mode 100644
index 0000000..38a1484
--- /dev/null
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/protocolPB/InterDatanodeProtocolPB.java
@@ -0,0 +1,47 @@
+/**
+ * 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.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(
+    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 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;
+}
\ No newline at end of file
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/protocolPB/InterDatanodeProtocolServerSideTranslatorPB.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/protocolPB/InterDatanodeProtocolServerSideTranslatorPB.java
new file mode 100644
index 0000000..495a0b6
--- /dev/null
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/protocolPB/InterDatanodeProtocolServerSideTranslatorPB.java
@@ -0,0 +1,131 @@
+/**
+ * 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.hdfs.protocol.ExtendedBlock;
+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.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;
+
+/**
+ * Implementation for protobuf service that forwards requests
+ * received on {@link InterDatanodeProtocolPB} to the
+ * {@link InterDatanodeProtocol} server implementation.
+ */
+@InterfaceAudience.Private
+public class InterDatanodeProtocolServerSideTranslatorPB implements
+    InterDatanodeProtocolPB {
+  private final InterDatanodeProtocol impl;
+
+  public InterDatanodeProtocolServerSideTranslatorPB(InterDatanodeProtocol impl) {
+    this.impl = impl;
+  }
+
+  @Override
+  public InitReplicaRecoveryResponseProto initReplicaRecovery(
+      RpcController unused, InitReplicaRecoveryRequestProto request)
+      throws ServiceException {
+    RecoveringBlock b = PBHelper.convert(request.getBlock());
+    ReplicaRecoveryInfo r;
+    try {
+      r = impl.initReplicaRecovery(b);
+    } catch (IOException e) {
+      throw new ServiceException(e);
+    }
+    return InitReplicaRecoveryResponseProto.newBuilder()
+        .setBlock(PBHelper.convert(r))
+        .setState(PBHelper.convert(r.getOriginalReplicaState())).build();
+  }
+
+  @Override
+  public UpdateReplicaUnderRecoveryResponseProto updateReplicaUnderRecovery(
+      RpcController unused, UpdateReplicaUnderRecoveryRequestProto request)
+      throws ServiceException {
+    ExtendedBlock b;
+    try {
+      b = impl.updateReplicaUnderRecovery(PBHelper.convert(request.getBlock()),
+          request.getRecoveryId(), request.getNewLength());
+    } catch (IOException e) {
+      throw new ServiceException(e);
+    }
+    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
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/protocolPB/InterDatanodeProtocolTranslatorPB.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/protocolPB/InterDatanodeProtocolTranslatorPB.java
new file mode 100644
index 0000000..f19f289
--- /dev/null
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/protocolPB/InterDatanodeProtocolTranslatorPB.java
@@ -0,0 +1,129 @@
+/**
+ * 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 javax.net.SocketFactory;
+
+import org.apache.hadoop.classification.InterfaceAudience;
+import org.apache.hadoop.classification.InterfaceStability;
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.hdfs.protocol.ExtendedBlock;
+import org.apache.hadoop.hdfs.protocol.proto.HdfsProtos.BlockProto;
+import org.apache.hadoop.hdfs.protocol.proto.InterDatanodeProtocolProtos.InitReplicaRecoveryRequestProto;
+import org.apache.hadoop.hdfs.protocol.proto.InterDatanodeProtocolProtos.InitReplicaRecoveryResponseProto;
+import org.apache.hadoop.hdfs.protocol.proto.InterDatanodeProtocolProtos.UpdateReplicaUnderRecoveryRequestProto;
+import org.apache.hadoop.hdfs.protocolR23Compatible.ProtocolSignatureWritable;
+import org.apache.hadoop.hdfs.server.protocol.BlockRecoveryCommand.RecoveringBlock;
+import org.apache.hadoop.hdfs.server.protocol.InterDatanodeProtocol;
+import org.apache.hadoop.hdfs.server.protocol.ReplicaRecoveryInfo;
+import org.apache.hadoop.ipc.ProtobufHelper;
+import org.apache.hadoop.ipc.ProtobufRpcEngine;
+import org.apache.hadoop.ipc.ProtocolMetaInterface;
+import org.apache.hadoop.ipc.ProtocolSignature;
+import org.apache.hadoop.ipc.RPC;
+import org.apache.hadoop.ipc.RpcClientUtil;
+import org.apache.hadoop.ipc.RpcPayloadHeader.RpcKind;
+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 InterDatanodeProtocol} interfaces to the RPC server implementing
+ * {@link InterDatanodeProtocolPB}.
+ */
+@InterfaceAudience.Private
+@InterfaceStability.Stable
+public class InterDatanodeProtocolTranslatorPB implements
+    ProtocolMetaInterface, InterDatanodeProtocol, Closeable {
+  /** RpcController is not used and hence is set to null */
+  private final static RpcController NULL_CONTROLLER = null;
+  final private InterDatanodeProtocolPB rpcProxy;
+
+  public InterDatanodeProtocolTranslatorPB(InetSocketAddress addr,
+      UserGroupInformation ugi, Configuration conf, SocketFactory factory,
+      int socketTimeout)
+      throws IOException {
+    RPC.setProtocolEngine(conf, InterDatanodeProtocolPB.class,
+        ProtobufRpcEngine.class);
+    rpcProxy = RPC.getProxy(InterDatanodeProtocolPB.class,
+        RPC.getProtocolVersion(InterDatanodeProtocolPB.class), addr, ugi, conf,
+        factory, socketTimeout);
+  }
+
+  @Override
+  public void close() {
+    RPC.stopProxy(rpcProxy);
+  }
+
+  @Override
+  public long getProtocolVersion(String protocolName, long clientVersion)
+      throws IOException {
+    return rpcProxy.getProtocolVersion(protocolName, clientVersion);
+  }
+
+  @Override
+  public ProtocolSignature getProtocolSignature(String protocol,
+      long clientVersion, int clientMethodsHash) throws IOException {
+    return ProtocolSignatureWritable.convert(rpcProxy.getProtocolSignature2(
+        protocol, clientVersion, clientMethodsHash));
+  }
+
+  @Override
+  public ReplicaRecoveryInfo initReplicaRecovery(RecoveringBlock rBlock)
+      throws IOException {
+    InitReplicaRecoveryRequestProto req = InitReplicaRecoveryRequestProto
+        .newBuilder().setBlock(PBHelper.convert(rBlock)).build();
+    InitReplicaRecoveryResponseProto resp;
+    try {
+      resp = rpcProxy.initReplicaRecovery(NULL_CONTROLLER, req);
+    } catch (ServiceException e) {
+      throw ProtobufHelper.getRemoteException(e);
+    }
+    BlockProto b = resp.getBlock();
+    return new ReplicaRecoveryInfo(b.getBlockId(), b.getNumBytes(),
+        b.getGenStamp(), PBHelper.convert(resp.getState()));
+  }
+
+  @Override
+  public ExtendedBlock updateReplicaUnderRecovery(ExtendedBlock oldBlock,
+      long recoveryId, long newLength) throws IOException {
+    UpdateReplicaUnderRecoveryRequestProto req = 
+        UpdateReplicaUnderRecoveryRequestProto.newBuilder()
+        .setBlock(PBHelper.convert(oldBlock))
+        .setNewLength(newLength).setRecoveryId(recoveryId).build();
+    try {
+      return PBHelper.convert(rpcProxy.updateReplicaUnderRecovery(
+          NULL_CONTROLLER, req).getBlock());
+    } catch (ServiceException e) {
+      throw ProtobufHelper.getRemoteException(e);
+    }
+  }
+
+  @Override
+  public boolean isMethodSupported(String methodName) throws IOException {
+    return RpcClientUtil.isMethodSupported(rpcProxy,
+        InterDatanodeProtocolPB.class, RpcKind.RPC_PROTOCOL_BUFFER,
+        RPC.getProtocolVersion(InterDatanodeProtocolPB.class), methodName);
+  }
+}
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/protocolPB/JournalProtocolPB.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/protocolPB/JournalProtocolPB.java
new file mode 100644
index 0000000..ebbdcb3
--- /dev/null
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/protocolPB/JournalProtocolPB.java
@@ -0,0 +1,53 @@
+/**
+ * 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.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.ipc.ProtocolInfo;
+import org.apache.hadoop.ipc.VersionedProtocol;
+
+/**
+ * Protocol used to journal edits to a remote node. Currently,
+ * this is used to publish edits from the NameNode to a BackupNode.
+ * 
+ * Note: This extends the protocolbuffer service based interface to
+ * add annotations required for security.
+ */
+@KerberosInfo(
+    serverPrincipal = DFSConfigKeys.DFS_NAMENODE_USER_NAME_KEY,
+    clientPrincipal = DFSConfigKeys.DFS_NAMENODE_USER_NAME_KEY)
+@ProtocolInfo(protocolName = 
+    "org.apache.hadoop.hdfs.server.protocol.JournalProtocol",
+    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;
+}
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/protocolPB/JournalProtocolServerSideTranslatorPB.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/protocolPB/JournalProtocolServerSideTranslatorPB.java
new file mode 100644
index 0000000..27dda01
--- /dev/null
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/protocolPB/JournalProtocolServerSideTranslatorPB.java
@@ -0,0 +1,123 @@
+/**
+ * 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.hdfs.protocol.proto.JournalProtocolProtos.JournalRequestProto;
+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;
+
+/**
+ * Implementation for protobuf service that forwards requests
+ * received on {@link JournalProtocolPB} to the 
+ * {@link JournalProtocol} server implementation.
+ */
+@InterfaceAudience.Private
+public class JournalProtocolServerSideTranslatorPB implements JournalProtocolPB {
+  /** Server side implementation to delegate the requests to */
+  private final JournalProtocol impl;
+
+  public JournalProtocolServerSideTranslatorPB(JournalProtocol impl) {
+    this.impl = impl;
+  }
+
+  /** @see JournalProtocol#journal */
+  @Override
+  public JournalResponseProto journal(RpcController unused,
+      JournalRequestProto req) throws ServiceException {
+    try {
+      impl.journal(PBHelper.convert(req.getRegistration()),
+          req.getFirstTxnId(), req.getNumTxns(), req.getRecords()
+              .toByteArray());
+    } catch (IOException e) {
+      throw new ServiceException(e);
+    }
+    return JournalResponseProto.newBuilder().build();
+  }
+
+  /** @see JournalProtocol#startLogSegment */
+  @Override
+  public StartLogSegmentResponseProto startLogSegment(RpcController controller,
+      StartLogSegmentRequestProto req) throws ServiceException {
+    try {
+      impl.startLogSegment(PBHelper.convert(req.getRegistration()),
+          req.getTxid());
+    } catch (IOException e) {
+      throw new ServiceException(e);
+    }
+    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));
+  }
+}
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/protocolPB/JournalProtocolTranslatorPB.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/protocolPB/JournalProtocolTranslatorPB.java
new file mode 100644
index 0000000..89d3247
--- /dev/null
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/protocolPB/JournalProtocolTranslatorPB.java
@@ -0,0 +1,117 @@
+/**
+ * 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.classification.InterfaceAudience;
+import org.apache.hadoop.classification.InterfaceStability;
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.hdfs.protocol.proto.JournalProtocolProtos.JournalRequestProto;
+import org.apache.hadoop.hdfs.protocol.proto.JournalProtocolProtos.StartLogSegmentRequestProto;
+import org.apache.hadoop.hdfs.protocolR23Compatible.ProtocolSignatureWritable;
+import org.apache.hadoop.hdfs.server.protocol.JournalProtocol;
+import org.apache.hadoop.hdfs.server.protocol.NamenodeRegistration;
+import org.apache.hadoop.ipc.ProtobufRpcEngine;
+import org.apache.hadoop.ipc.ProtobufHelper;
+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 com.google.protobuf.RpcController;
+import com.google.protobuf.ServiceException;
+
+/**
+ * This class is the client side translator to translate the requests made on
+ * {@link JournalProtocol} interfaces to the RPC server implementing
+ * {@link JournalProtocolPB}.
+ */
+@InterfaceAudience.Private
+@InterfaceStability.Stable
+public class JournalProtocolTranslatorPB implements ProtocolMetaInterface,
+    JournalProtocol, Closeable {
+  /** RpcController is not used and hence is set to null */
+  private final static RpcController NULL_CONTROLLER = null;
+  private final JournalProtocolPB rpcProxy;
+
+  public JournalProtocolTranslatorPB(InetSocketAddress nameNodeAddr,
+      Configuration conf) throws IOException {
+    RPC.setProtocolEngine(conf, JournalProtocolPB.class, ProtobufRpcEngine.class);
+    rpcProxy = RPC.getProxy(JournalProtocolPB.class,
+        RPC.getProtocolVersion(JournalProtocolPB.class), nameNodeAddr, conf);
+  }
+
+  @Override
+  public void close() {
+    RPC.stopProxy(rpcProxy);
+  }
+
+  @Override
+  public long getProtocolVersion(String protocolName, long clientVersion)
+      throws IOException {
+    return rpcProxy.getProtocolVersion(protocolName, clientVersion);
+  }
+
+  @Override
+  public ProtocolSignature getProtocolSignature(String protocol,
+      long clientVersion, int clientMethodsHash) throws IOException {
+    return ProtocolSignatureWritable.convert(rpcProxy.getProtocolSignature2(
+        protocol, clientVersion, clientMethodsHash));
+  }
+
+  @Override
+  public void journal(NamenodeRegistration reg, long firstTxnId,
+      int numTxns, byte[] records) throws IOException {
+    JournalRequestProto req = JournalRequestProto.newBuilder()
+        .setRegistration(PBHelper.convert(reg))
+        .setFirstTxnId(firstTxnId)
+        .setNumTxns(numTxns)
+        .setRecords(PBHelper.getByteString(records))
+        .build();
+    try {
+      rpcProxy.journal(NULL_CONTROLLER, req);
+    } catch (ServiceException e) {
+      throw ProtobufHelper.getRemoteException(e);
+    }
+  }
+
+  @Override
+  public void startLogSegment(NamenodeRegistration registration, long txid)
+      throws IOException {
+    StartLogSegmentRequestProto req = StartLogSegmentRequestProto.newBuilder()
+        .setRegistration(PBHelper.convert(registration))
+        .setTxid(txid)
+        .build();
+    try {
+      rpcProxy.startLogSegment(NULL_CONTROLLER, req);
+    } catch (ServiceException e) {
+      throw ProtobufHelper.getRemoteException(e);
+    }
+  }
+
+  @Override
+  public boolean isMethodSupported(String methodName) throws IOException {
+    return RpcClientUtil.isMethodSupported(rpcProxy, JournalProtocolPB.class,
+        RpcKind.RPC_PROTOCOL_BUFFER,
+        RPC.getProtocolVersion(JournalProtocolPB.class), methodName);
+  }
+}
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/protocolPB/NamenodeProtocolPB.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/protocolPB/NamenodeProtocolPB.java
new file mode 100644
index 0000000..4e50af1
--- /dev/null
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/protocolPB/NamenodeProtocolPB.java
@@ -0,0 +1,54 @@
+/**
+ * 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.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;
+
+/**
+ * Protocol that a secondary NameNode uses to communicate with the NameNode.
+ * It's used to get part of the name node state
+ * 
+ * Note: This extends the protocolbuffer service based interface to
+ * add annotations required for security.
+ */
+@KerberosInfo(
+    serverPrincipal = DFSConfigKeys.DFS_NAMENODE_USER_NAME_KEY,
+    clientPrincipal = DFSConfigKeys.DFS_NAMENODE_USER_NAME_KEY)
+@ProtocolInfo(protocolName = 
+    "org.apache.hadoop.hdfs.server.protocol.NamenodeProtocol",
+    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;
+}
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/protocolPB/NamenodeProtocolServerSideTranslatorPB.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/protocolPB/NamenodeProtocolServerSideTranslatorPB.java
new file mode 100644
index 0000000..5ffe8e5
--- /dev/null
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/protocolPB/NamenodeProtocolServerSideTranslatorPB.java
@@ -0,0 +1,244 @@
+/**
+ * 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.DatanodeInfo;
+import org.apache.hadoop.hdfs.protocol.proto.HdfsProtos.VersionRequestProto;
+import org.apache.hadoop.hdfs.protocol.proto.HdfsProtos.VersionResponseProto;
+import org.apache.hadoop.hdfs.protocol.proto.NamenodeProtocolProtos.EndCheckpointRequestProto;
+import org.apache.hadoop.hdfs.protocol.proto.NamenodeProtocolProtos.EndCheckpointResponseProto;
+import org.apache.hadoop.hdfs.protocol.proto.NamenodeProtocolProtos.ErrorReportRequestProto;
+import org.apache.hadoop.hdfs.protocol.proto.NamenodeProtocolProtos.ErrorReportResponseProto;
+import org.apache.hadoop.hdfs.protocol.proto.NamenodeProtocolProtos.GetBlockKeysRequestProto;
+import org.apache.hadoop.hdfs.protocol.proto.NamenodeProtocolProtos.GetBlockKeysResponseProto;
+import org.apache.hadoop.hdfs.protocol.proto.NamenodeProtocolProtos.GetBlocksRequestProto;
+import org.apache.hadoop.hdfs.protocol.proto.NamenodeProtocolProtos.GetBlocksResponseProto;
+import org.apache.hadoop.hdfs.protocol.proto.NamenodeProtocolProtos.GetEditLogManifestRequestProto;
+import org.apache.hadoop.hdfs.protocol.proto.NamenodeProtocolProtos.GetEditLogManifestResponseProto;
+import org.apache.hadoop.hdfs.protocol.proto.NamenodeProtocolProtos.GetTransactionIdRequestProto;
+import org.apache.hadoop.hdfs.protocol.proto.NamenodeProtocolProtos.GetTransactionIdResponseProto;
+import org.apache.hadoop.hdfs.protocol.proto.NamenodeProtocolProtos.RegisterRequestProto;
+import org.apache.hadoop.hdfs.protocol.proto.NamenodeProtocolProtos.RegisterResponseProto;
+import org.apache.hadoop.hdfs.protocol.proto.NamenodeProtocolProtos.RollEditLogRequestProto;
+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;
+import org.apache.hadoop.hdfs.server.protocol.NamenodeCommand;
+import org.apache.hadoop.hdfs.server.protocol.NamenodeProtocol;
+import org.apache.hadoop.hdfs.server.protocol.NamenodeRegistration;
+import org.apache.hadoop.hdfs.server.protocol.NamespaceInfo;
+import org.apache.hadoop.hdfs.server.protocol.RemoteEditLogManifest;
+import org.apache.hadoop.ipc.ProtocolSignature;
+import org.apache.hadoop.ipc.RPC;
+
+import com.google.protobuf.RpcController;
+import com.google.protobuf.ServiceException;
+
+/**
+ * Implementation for protobuf service that forwards requests
+ * received on {@link NamenodeProtocolPB} to the
+ * {@link NamenodeProtocol} server implementation.
+ */
+public class NamenodeProtocolServerSideTranslatorPB implements
+    NamenodeProtocolPB {
+  private final NamenodeProtocol impl;
+
+  public NamenodeProtocolServerSideTranslatorPB(NamenodeProtocol impl) {
+    this.impl = impl;
+  }
+
+  @Override
+  public GetBlocksResponseProto getBlocks(RpcController unused,
+      GetBlocksRequestProto request) throws ServiceException {
+    DatanodeInfo dnInfo = new DatanodeInfo(PBHelper.convert(request
+        .getDatanode()));
+    BlocksWithLocations blocks;
+    try {
+      blocks = impl.getBlocks(dnInfo, request.getSize());
+    } catch (IOException e) {
+      throw new ServiceException(e);
+    }
+    return GetBlocksResponseProto.newBuilder()
+        .setBlocks(PBHelper.convert(blocks)).build();
+  }
+
+  @Override
+  public GetBlockKeysResponseProto getBlockKeys(RpcController unused,
+      GetBlockKeysRequestProto request) throws ServiceException {
+    ExportedBlockKeys keys;
+    try {
+      keys = impl.getBlockKeys();
+    } catch (IOException e) {
+      throw new ServiceException(e);
+    }
+    return GetBlockKeysResponseProto.newBuilder()
+        .setKeys(PBHelper.convert(keys)).build();
+  }
+
+  @Override
+  public GetTransactionIdResponseProto getTransactionId(RpcController unused,
+      GetTransactionIdRequestProto request) throws ServiceException {
+    long txid;
+    try {
+      txid = impl.getTransactionID();
+    } catch (IOException e) {
+      throw new ServiceException(e);
+    }
+    return GetTransactionIdResponseProto.newBuilder().setTxId(txid).build();
+  }
+
+  @Override
+  public RollEditLogResponseProto rollEditLog(RpcController unused,
+      RollEditLogRequestProto request) throws ServiceException {
+    CheckpointSignature signature;
+    try {
+      signature = impl.rollEditLog();
+    } catch (IOException e) {
+      throw new ServiceException(e);
+    }
+    return RollEditLogResponseProto.newBuilder()
+        .setSignature(PBHelper.convert(signature)).build();
+  }
+
+  @Override
+  public ErrorReportResponseProto errorReport(RpcController unused,
+      ErrorReportRequestProto request) throws ServiceException {
+    try {
+      impl.errorReport(PBHelper.convert(request.getRegistration()),
+          request.getErrorCode(), request.getMsg());
+    } catch (IOException e) {
+      throw new ServiceException(e);
+    }
+    return ErrorReportResponseProto.newBuilder().build();
+  }
+
+  @Override
+  public RegisterResponseProto register(RpcController unused,
+      RegisterRequestProto request) throws ServiceException {
+    NamenodeRegistration reg;
+    try {
+      reg = impl.register(PBHelper.convert(request.getRegistration()));
+    } catch (IOException e) {
+      throw new ServiceException(e);
+    }
+    return RegisterResponseProto.newBuilder()
+        .setRegistration(PBHelper.convert(reg)).build();
+  }
+
+  @Override
+  public StartCheckpointResponseProto startCheckpoint(RpcController unused,
+      StartCheckpointRequestProto request) throws ServiceException {
+    NamenodeCommand cmd;
+    try {
+      cmd = impl.startCheckpoint(PBHelper.convert(request.getRegistration()));
+    } catch (IOException e) {
+      throw new ServiceException(e);
+    }
+    return StartCheckpointResponseProto.newBuilder()
+        .setCommand(PBHelper.convert(cmd)).build();
+  }
+
+  @Override
+  public EndCheckpointResponseProto endCheckpoint(RpcController unused,
+      EndCheckpointRequestProto request) throws ServiceException {
+    try {
+      impl.endCheckpoint(PBHelper.convert(request.getRegistration()),
+          PBHelper.convert(request.getSignature()));
+    } catch (IOException e) {
+      throw new ServiceException(e);
+    }
+    return EndCheckpointResponseProto.newBuilder().build();
+  }
+
+  @Override
+  public GetEditLogManifestResponseProto getEditLogManifest(
+      RpcController unused, GetEditLogManifestRequestProto request)
+      throws ServiceException {
+    RemoteEditLogManifest manifest;
+    try {
+      manifest = impl.getEditLogManifest(request.getSinceTxId());
+    } catch (IOException e) {
+      throw new ServiceException(e);
+    }
+    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,
+      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();
+  }
+}
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/protocolPB/NamenodeProtocolTranslatorPB.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/protocolPB/NamenodeProtocolTranslatorPB.java
new file mode 100644
index 0000000..d5df358
--- /dev/null
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/protocolPB/NamenodeProtocolTranslatorPB.java
@@ -0,0 +1,245 @@
+/**
+ * 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.classification.InterfaceAudience;
+import org.apache.hadoop.classification.InterfaceStability;
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.hdfs.protocol.DatanodeID;
+import org.apache.hadoop.hdfs.protocol.DatanodeInfo;
+import org.apache.hadoop.hdfs.protocol.proto.HdfsProtos.NamenodeCommandProto;
+import org.apache.hadoop.hdfs.protocol.proto.HdfsProtos.VersionRequestProto;
+import org.apache.hadoop.hdfs.protocol.proto.NamenodeProtocolProtos.EndCheckpointRequestProto;
+import org.apache.hadoop.hdfs.protocol.proto.NamenodeProtocolProtos.ErrorReportRequestProto;
+import org.apache.hadoop.hdfs.protocol.proto.NamenodeProtocolProtos.GetBlockKeysRequestProto;
+import org.apache.hadoop.hdfs.protocol.proto.NamenodeProtocolProtos.GetBlocksRequestProto;
+import org.apache.hadoop.hdfs.protocol.proto.NamenodeProtocolProtos.GetEditLogManifestRequestProto;
+import org.apache.hadoop.hdfs.protocol.proto.NamenodeProtocolProtos.GetTransactionIdRequestProto;
+import org.apache.hadoop.hdfs.protocol.proto.NamenodeProtocolProtos.RegisterRequestProto;
+import org.apache.hadoop.hdfs.protocol.proto.NamenodeProtocolProtos.RollEditLogRequestProto;
+import org.apache.hadoop.hdfs.protocol.proto.NamenodeProtocolProtos.StartCheckpointRequestProto;
+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;
+import org.apache.hadoop.hdfs.server.protocol.NamenodeCommand;
+import org.apache.hadoop.hdfs.server.protocol.NamenodeProtocol;
+import org.apache.hadoop.hdfs.server.protocol.NamenodeRegistration;
+import org.apache.hadoop.hdfs.server.protocol.NamespaceInfo;
+import org.apache.hadoop.hdfs.server.protocol.RemoteEditLogManifest;
+import org.apache.hadoop.ipc.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 com.google.protobuf.RpcController;
+import com.google.protobuf.ServiceException;
+
+/**
+ * This class is the client side translator to translate the requests made on
+ * {@link NamenodeProtocol} interfaces to the RPC server implementing
+ * {@link NamenodeProtocolPB}.
+ */
+@InterfaceAudience.Private
+@InterfaceStability.Stable
+public class NamenodeProtocolTranslatorPB implements NamenodeProtocol,
+    ProtocolMetaInterface, Closeable {
+  /** RpcController is not used and hence is set to null */
+  private final static RpcController NULL_CONTROLLER = null;
+  
+  /*
+   * Protobuf requests with no parameters instantiated only once
+   */
+  private static final GetBlockKeysRequestProto GET_BLOCKKEYS = 
+      GetBlockKeysRequestProto.newBuilder().build();
+  private static final GetTransactionIdRequestProto GET_TRANSACTIONID = 
+      GetTransactionIdRequestProto.newBuilder().build();
+  private static final RollEditLogRequestProto ROLL_EDITLOG = 
+      RollEditLogRequestProto.newBuilder().build();
+  private static final VersionRequestProto VERSION_REQUEST = 
+      VersionRequestProto.newBuilder().build();
+
+  final private NamenodeProtocolPB rpcProxy;
+
+  public NamenodeProtocolTranslatorPB(InetSocketAddress nameNodeAddr,
+      Configuration conf, UserGroupInformation ugi) throws IOException {
+    RPC.setProtocolEngine(conf, NamenodeProtocolPB.class,
+        ProtobufRpcEngine.class);
+    rpcProxy = RPC.getProxy(NamenodeProtocolPB.class,
+        RPC.getProtocolVersion(NamenodeProtocolPB.class), nameNodeAddr, ugi,
+        conf, NetUtils.getSocketFactory(conf, NamenodeProtocolPB.class));
+  }
+  
+  public NamenodeProtocolTranslatorPB(NamenodeProtocolPB rpcProxy) {
+    this.rpcProxy = rpcProxy;
+  }
+
+  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 BlocksWithLocations getBlocks(DatanodeInfo datanode, long size)
+      throws IOException {
+    GetBlocksRequestProto req = GetBlocksRequestProto.newBuilder()
+        .setDatanode(PBHelper.convert((DatanodeID)datanode)).setSize(size)
+        .build();
+    try {
+      return PBHelper.convert(rpcProxy.getBlocks(NULL_CONTROLLER, req)
+          .getBlocks());
+    } catch (ServiceException e) {
+      throw ProtobufHelper.getRemoteException(e);
+    }
+  }
+
+  @Override
+  public ExportedBlockKeys getBlockKeys() throws IOException {
+    try {
+      return PBHelper.convert(rpcProxy.getBlockKeys(NULL_CONTROLLER,
+          GET_BLOCKKEYS).getKeys());
+    } catch (ServiceException e) {
+      throw ProtobufHelper.getRemoteException(e);
+    }
+  }
+
+  @Override
+  public long getTransactionID() throws IOException {
+    try {
+      return rpcProxy.getTransactionId(NULL_CONTROLLER, GET_TRANSACTIONID)
+          .getTxId();
+    } catch (ServiceException e) {
+      throw ProtobufHelper.getRemoteException(e);
+    }
+  }
+
+  @Override
+  @SuppressWarnings("deprecation")
+  public CheckpointSignature rollEditLog() throws IOException {
+    try {
+      return PBHelper.convert(rpcProxy.rollEditLog(NULL_CONTROLLER,
+          ROLL_EDITLOG).getSignature());
+    } catch (ServiceException e) {
+      throw ProtobufHelper.getRemoteException(e);
+    }
+  }
+
+  @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 void errorReport(NamenodeRegistration registration, int errorCode,
+      String msg) throws IOException {
+    ErrorReportRequestProto req = ErrorReportRequestProto.newBuilder()
+        .setErrorCode(errorCode).setMsg(msg)
+        .setRegistration(PBHelper.convert(registration)).build();
+    try {
+      rpcProxy.errorReport(NULL_CONTROLLER, req);
+    } catch (ServiceException e) {
+      throw ProtobufHelper.getRemoteException(e);
+    }
+  }
+
+  @Override
+  public NamenodeRegistration register(NamenodeRegistration registration)
+      throws IOException {
+    RegisterRequestProto req = RegisterRequestProto.newBuilder()
+        .setRegistration(PBHelper.convert(registration)).build();
+    try {
+      return PBHelper.convert(rpcProxy.register(NULL_CONTROLLER, req)
+          .getRegistration());
+    } catch (ServiceException e) {
+      throw ProtobufHelper.getRemoteException(e);
+    }
+  }
+
+  @Override
+  public NamenodeCommand startCheckpoint(NamenodeRegistration registration)
+      throws IOException {
+    StartCheckpointRequestProto req = StartCheckpointRequestProto.newBuilder()
+        .setRegistration(PBHelper.convert(registration)).build();
+    NamenodeCommandProto cmd;
+    try {
+      cmd = rpcProxy.startCheckpoint(NULL_CONTROLLER, req).getCommand();
+    } catch (ServiceException e) {
+      throw ProtobufHelper.getRemoteException(e);
+    }
+    return PBHelper.convert(cmd);
+  }
+
+  @Override
+  public void endCheckpoint(NamenodeRegistration registration,
+      CheckpointSignature sig) throws IOException {
+    EndCheckpointRequestProto req = EndCheckpointRequestProto.newBuilder()
+        .setRegistration(PBHelper.convert(registration))
+        .setSignature(PBHelper.convert(sig)).build();
+    try {
+      rpcProxy.endCheckpoint(NULL_CONTROLLER, req);
+    } catch (ServiceException e) {
+      throw ProtobufHelper.getRemoteException(e);
+    }
+  }
+
+  @Override
+  public RemoteEditLogManifest getEditLogManifest(long sinceTxId)
+      throws IOException {
+    GetEditLogManifestRequestProto req = GetEditLogManifestRequestProto
+        .newBuilder().setSinceTxId(sinceTxId).build();
+    try {
+      return PBHelper.convert(rpcProxy.getEditLogManifest(NULL_CONTROLLER, req)
+          .getManifest());
+    } catch (ServiceException e) {
+      throw ProtobufHelper.getRemoteException(e);
+    }
+  }
+
+  @Override
+  public boolean isMethodSupported(String methodName) throws IOException {
+    return RpcClientUtil.isMethodSupported(rpcProxy, NamenodeProtocolPB.class,
+        RpcKind.RPC_PROTOCOL_BUFFER,
+        RPC.getProtocolVersion(NamenodeProtocolPB.class), methodName);
+  }
+}
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/protocolPB/PBHelper.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/protocolPB/PBHelper.java
new file mode 100644
index 0000000..fab9f1f
--- /dev/null
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/protocolPB/PBHelper.java
@@ -0,0 +1,1284 @@
+/**
+ * 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.util.ArrayList;
+import java.util.Arrays;
+import java.util.EnumSet;
+import java.util.List;
+
+import org.apache.hadoop.fs.ContentSummary;
+import org.apache.hadoop.fs.CreateFlag;
+import org.apache.hadoop.fs.FsServerDefaults;
+import org.apache.hadoop.fs.permission.FsPermission;
+import org.apache.hadoop.hdfs.server.protocol.StorageReport;
+import org.apache.hadoop.hdfs.protocol.Block;
+import org.apache.hadoop.hdfs.protocol.ClientProtocol;
+import org.apache.hadoop.hdfs.protocol.CorruptFileBlocks;
+import org.apache.hadoop.hdfs.protocol.DatanodeID;
+import org.apache.hadoop.hdfs.protocol.DatanodeInfo;
+import org.apache.hadoop.hdfs.protocol.DatanodeInfo.AdminStates;
+import org.apache.hadoop.hdfs.protocol.DirectoryListing;
+import org.apache.hadoop.hdfs.protocol.ExtendedBlock;
+import org.apache.hadoop.hdfs.protocol.HdfsConstants.DatanodeReportType;
+import org.apache.hadoop.hdfs.protocol.HdfsConstants.SafeModeAction;
+import org.apache.hadoop.hdfs.protocol.HdfsConstants.UpgradeAction;
+import org.apache.hadoop.hdfs.protocol.HdfsFileStatus;
+import org.apache.hadoop.hdfs.protocol.HdfsLocatedFileStatus;
+import org.apache.hadoop.hdfs.protocol.LocatedBlock;
+import org.apache.hadoop.hdfs.protocol.LocatedBlocks;
+import org.apache.hadoop.hdfs.protocol.proto.ClientNamenodeProtocolProtos;
+import org.apache.hadoop.hdfs.protocol.proto.ClientNamenodeProtocolProtos.CreateFlagProto;
+import org.apache.hadoop.hdfs.protocol.proto.ClientNamenodeProtocolProtos.DatanodeReportTypeProto;
+import org.apache.hadoop.hdfs.protocol.proto.ClientNamenodeProtocolProtos.GetFsStatsResponseProto;
+import org.apache.hadoop.hdfs.protocol.proto.ClientNamenodeProtocolProtos.SafeModeActionProto;
+import org.apache.hadoop.hdfs.protocol.proto.ClientNamenodeProtocolProtos.UpgradeActionProto;
+import org.apache.hadoop.hdfs.protocol.proto.DatanodeProtocolProtos.BalancerBandwidthCommandProto;
+import org.apache.hadoop.hdfs.protocol.proto.DatanodeProtocolProtos.BlockCommandProto;
+import org.apache.hadoop.hdfs.protocol.proto.DatanodeProtocolProtos.BlockRecoveryCommandProto;
+import org.apache.hadoop.hdfs.protocol.proto.DatanodeProtocolProtos.DatanodeCommandProto;
+import org.apache.hadoop.hdfs.protocol.proto.DatanodeProtocolProtos.DatanodeRegistrationProto;
+import org.apache.hadoop.hdfs.protocol.proto.DatanodeProtocolProtos.DatanodeStorageProto;
+import org.apache.hadoop.hdfs.protocol.proto.DatanodeProtocolProtos.DatanodeStorageProto.StorageState;
+import org.apache.hadoop.hdfs.protocol.proto.DatanodeProtocolProtos.FinalizeCommandProto;
+import org.apache.hadoop.hdfs.protocol.proto.DatanodeProtocolProtos.KeyUpdateCommandProto;
+import org.apache.hadoop.hdfs.protocol.proto.DatanodeProtocolProtos.ReceivedDeletedBlockInfoProto;
+import org.apache.hadoop.hdfs.protocol.proto.DatanodeProtocolProtos.RegisterCommandProto;
+import org.apache.hadoop.hdfs.protocol.proto.DatanodeProtocolProtos.StorageReportProto;
+import org.apache.hadoop.hdfs.protocol.proto.DatanodeProtocolProtos.UpgradeCommandProto;
+import org.apache.hadoop.hdfs.protocol.proto.HdfsProtos.BlockKeyProto;
+import org.apache.hadoop.hdfs.protocol.proto.HdfsProtos.BlockProto;
+import org.apache.hadoop.hdfs.protocol.proto.HdfsProtos.BlockTokenIdentifierProto;
+import org.apache.hadoop.hdfs.protocol.proto.HdfsProtos.BlockWithLocationsProto;
+import org.apache.hadoop.hdfs.protocol.proto.HdfsProtos.BlocksWithLocationsProto;
+import org.apache.hadoop.hdfs.protocol.proto.HdfsProtos.CheckpointCommandProto;
+import org.apache.hadoop.hdfs.protocol.proto.HdfsProtos.CheckpointSignatureProto;
+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.DatanodeInfoProto.AdminState;
+import org.apache.hadoop.hdfs.protocol.proto.HdfsProtos.DatanodeInfosProto;
+import org.apache.hadoop.hdfs.protocol.proto.HdfsProtos.DirectoryListingProto;
+import org.apache.hadoop.hdfs.protocol.proto.HdfsProtos.ExportedBlockKeysProto;
+import org.apache.hadoop.hdfs.protocol.proto.HdfsProtos.ExtendedBlockProto;
+import org.apache.hadoop.hdfs.protocol.proto.HdfsProtos.FsPermissionProto;
+import org.apache.hadoop.hdfs.protocol.proto.HdfsProtos.FsServerDefaultsProto;
+import org.apache.hadoop.hdfs.protocol.proto.HdfsProtos.HdfsFileStatusProto;
+import org.apache.hadoop.hdfs.protocol.proto.HdfsProtos.HdfsFileStatusProto.FileType;
+import org.apache.hadoop.hdfs.protocol.proto.HdfsProtos.LocatedBlockProto;
+import org.apache.hadoop.hdfs.protocol.proto.HdfsProtos.LocatedBlockProto.Builder;
+import org.apache.hadoop.hdfs.protocol.proto.HdfsProtos.LocatedBlocksProto;
+import org.apache.hadoop.hdfs.protocol.proto.HdfsProtos.NamenodeCommandProto;
+import org.apache.hadoop.hdfs.protocol.proto.HdfsProtos.NamenodeRegistrationProto;
+import org.apache.hadoop.hdfs.protocol.proto.HdfsProtos.NamespaceInfoProto;
+import org.apache.hadoop.hdfs.protocol.proto.HdfsProtos.RecoveringBlockProto;
+import org.apache.hadoop.hdfs.protocol.proto.HdfsProtos.RemoteEditLogManifestProto;
+import org.apache.hadoop.hdfs.protocol.proto.HdfsProtos.RemoteEditLogProto;
+import org.apache.hadoop.hdfs.protocol.proto.HdfsProtos.NamenodeRegistrationProto.NamenodeRoleProto;
+import org.apache.hadoop.hdfs.protocol.proto.HdfsProtos.ReplicaStateProto;
+import org.apache.hadoop.hdfs.protocol.proto.HdfsProtos.StorageInfoProto;
+import org.apache.hadoop.hdfs.protocol.proto.HdfsProtos.UpgradeStatusReportProto;
+import org.apache.hadoop.hdfs.security.token.block.BlockKey;
+import org.apache.hadoop.hdfs.security.token.block.BlockTokenIdentifier;
+import org.apache.hadoop.hdfs.security.token.block.ExportedBlockKeys;
+import org.apache.hadoop.hdfs.security.token.delegation.DelegationTokenIdentifier;
+import org.apache.hadoop.hdfs.server.common.HdfsServerConstants.NamenodeRole;
+import org.apache.hadoop.hdfs.server.common.HdfsServerConstants.ReplicaState;
+import org.apache.hadoop.hdfs.server.common.StorageInfo;
+import org.apache.hadoop.hdfs.server.common.UpgradeStatusReport;
+import org.apache.hadoop.hdfs.server.namenode.CheckpointSignature;
+import org.apache.hadoop.hdfs.server.protocol.BalancerBandwidthCommand;
+import org.apache.hadoop.hdfs.server.protocol.BlockCommand;
+import org.apache.hadoop.hdfs.server.protocol.BlockRecoveryCommand;
+import org.apache.hadoop.hdfs.server.protocol.BlockRecoveryCommand.RecoveringBlock;
+import org.apache.hadoop.hdfs.server.protocol.BlocksWithLocations;
+import org.apache.hadoop.hdfs.server.protocol.CheckpointCommand;
+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.DatanodeStorage.State;
+import org.apache.hadoop.hdfs.server.protocol.FinalizeCommand;
+import org.apache.hadoop.hdfs.server.protocol.KeyUpdateCommand;
+import org.apache.hadoop.hdfs.server.protocol.NamenodeCommand;
+import org.apache.hadoop.hdfs.server.protocol.NamenodeRegistration;
+import org.apache.hadoop.hdfs.server.protocol.NamespaceInfo;
+import org.apache.hadoop.hdfs.server.protocol.ReceivedDeletedBlockInfo;
+import org.apache.hadoop.hdfs.server.protocol.RegisterCommand;
+import org.apache.hadoop.hdfs.server.protocol.RemoteEditLog;
+import org.apache.hadoop.hdfs.server.protocol.RemoteEditLogManifest;
+import org.apache.hadoop.hdfs.server.protocol.BlocksWithLocations.BlockWithLocations;
+import org.apache.hadoop.hdfs.server.protocol.StorageReport;
+import org.apache.hadoop.hdfs.server.protocol.UpgradeCommand;
+import org.apache.hadoop.io.EnumSetWritable;
+import org.apache.hadoop.io.Text;
+import org.apache.hadoop.security.token.Token;
+
+import com.google.protobuf.ByteString;
+
+/**
+ * Utilities for converting protobuf classes to and from implementation classes.
+ * 
+ * Note that when converting from an internal type to protobuf type, the
+ * converter never return null for protobuf type. The check for internal type
+ * being null must be done before calling the convert() method.
+ */
+public class PBHelper {
+  private static final RegisterCommandProto REG_CMD_PROTO = 
+      RegisterCommandProto.newBuilder().build();
+  private static final RegisterCommand REG_CMD = new RegisterCommand();
+
+  private PBHelper() {
+    /** Hidden constructor */
+  }
+
+  public static ByteString getByteString(byte[] bytes) {
+    return ByteString.copyFrom(bytes);
+  }
+
+  public static NamenodeRole convert(NamenodeRoleProto role) {
+    switch (role) {
+    case NAMENODE:
+      return NamenodeRole.NAMENODE;
+    case BACKUP:
+      return NamenodeRole.BACKUP;
+    case CHECKPOINT:
+      return NamenodeRole.CHECKPOINT;
+    }
+    return null;
+  }
+
+  public static NamenodeRoleProto convert(NamenodeRole role) {
+    switch (role) {
+    case NAMENODE:
+      return NamenodeRoleProto.NAMENODE;
+    case BACKUP:
+      return NamenodeRoleProto.BACKUP;
+    case CHECKPOINT:
+      return NamenodeRoleProto.CHECKPOINT;
+    }
+    return null;
+  }
+
+  public static StorageInfoProto convert(StorageInfo info) {
+    return StorageInfoProto.newBuilder().setClusterID(info.getClusterID())
+        .setCTime(info.getCTime()).setLayoutVersion(info.getLayoutVersion())
+        .setNamespceID(info.getNamespaceID()).build();
+  }
+
+  public static StorageInfo convert(StorageInfoProto info) {
+    return new StorageInfo(info.getLayoutVersion(), info.getNamespceID(),
+        info.getClusterID(), info.getCTime());
+  }
+
+  public static NamenodeRegistrationProto convert(NamenodeRegistration reg) {
+    return NamenodeRegistrationProto.newBuilder()
+        .setHttpAddress(reg.getHttpAddress()).setRole(convert(reg.getRole()))
+        .setRpcAddress(reg.getAddress())
+        .setStorageInfo(convert((StorageInfo) reg)).build();
+  }
+
+  public static NamenodeRegistration convert(NamenodeRegistrationProto reg) {
+    return new NamenodeRegistration(reg.getRpcAddress(), reg.getHttpAddress(),
+        convert(reg.getStorageInfo()), convert(reg.getRole()));
+  }
+
+  // DatanodeId
+  public static DatanodeID convert(DatanodeIDProto dn) {
+    return new DatanodeID(dn.getName(), dn.getStorageID(), dn.getInfoPort(),
+        dn.getIpcPort());
+  }
+
+  public static DatanodeIDProto convert(DatanodeID dn) {
+    return DatanodeIDProto.newBuilder().setName(dn.getName())
+        .setInfoPort(dn.getInfoPort()).setIpcPort(dn.getIpcPort())
+        .setStorageID(dn.getStorageID()).build();
+  }
+
+  // Arrays of DatanodeId
+  public static DatanodeIDProto[] convert(DatanodeID[] did) {
+    if (did == null) return null;
+    final int len = did.length;
+    DatanodeIDProto[] result = new DatanodeIDProto[len];
+    for (int i = 0; i < len; ++i) {
+      result[i] = convert(did[i]);
+    }
+    return result;
+  }
+  
+  public static DatanodeID[] convert(DatanodeIDProto[] did) {
+    if (did == null) return null;
+    final int len = did.length;
+    DatanodeID[] result = new DatanodeID[len];
+    for (int i = 0; i < len; ++i) {
+      result[i] = convert(did[i]);
+    }
+    return result;
+  }
+  
+  // Block
+  public static BlockProto convert(Block b) {
+    return BlockProto.newBuilder().setBlockId(b.getBlockId())
+        .setGenStamp(b.getGenerationStamp()).setNumBytes(b.getNumBytes())
+        .build();
+  }
+
+  public static Block convert(BlockProto b) {
+    return new Block(b.getBlockId(), b.getNumBytes(), b.getGenStamp());
+  }
+
+  public static BlockWithLocationsProto convert(BlockWithLocations blk) {
+    return BlockWithLocationsProto.newBuilder()
+        .setBlock(convert(blk.getBlock()))
+        .addAllDatanodeIDs(Arrays.asList(blk.getDatanodes())).build();
+  }
+
+  public static BlockWithLocations convert(BlockWithLocationsProto b) {
+    return new BlockWithLocations(convert(b.getBlock()), b.getDatanodeIDsList()
+        .toArray(new String[0]));
+  }
+
+  public static BlocksWithLocationsProto convert(BlocksWithLocations blks) {
+    BlocksWithLocationsProto.Builder builder = BlocksWithLocationsProto
+        .newBuilder();
+    for (BlockWithLocations b : blks.getBlocks()) {
+      builder.addBlocks(convert(b));
+    }
+    return builder.build();
+  }
+
+  public static BlocksWithLocations convert(BlocksWithLocationsProto blocks) {
+    List<BlockWithLocationsProto> b = blocks.getBlocksList();
+    BlockWithLocations[] ret = new BlockWithLocations[b.size()];
+    int i = 0;
+    for (BlockWithLocationsProto entry : b) {
+      ret[i++] = convert(entry);
+    }
+    return new BlocksWithLocations(ret);
+  }
+
+  public static BlockKeyProto convert(BlockKey key) {
+    byte[] encodedKey = key.getEncodedKey();
+    ByteString keyBytes = ByteString.copyFrom(encodedKey == null ? new byte[0]
+        : encodedKey);
+    return BlockKeyProto.newBuilder().setKeyId(key.getKeyId())
+        .setKeyBytes(keyBytes).setExpiryDate(key.getExpiryDate()).build();
+  }
+
+  public static BlockKey convert(BlockKeyProto k) {
+    return new BlockKey(k.getKeyId(), k.getExpiryDate(), k.getKeyBytes()
+        .toByteArray());
+  }
+
+  public static ExportedBlockKeysProto convert(ExportedBlockKeys keys) {
+    ExportedBlockKeysProto.Builder builder = ExportedBlockKeysProto
+        .newBuilder();
+    builder.setIsBlockTokenEnabled(keys.isBlockTokenEnabled())
+        .setKeyUpdateInterval(keys.getKeyUpdateInterval())
+        .setTokenLifeTime(keys.getTokenLifetime())
+        .setCurrentKey(convert(keys.getCurrentKey()));
+    for (BlockKey k : keys.getAllKeys()) {
+      builder.addAllKeys(convert(k));
+    }
+    return builder.build();
+  }
+
+  public static ExportedBlockKeys convert(ExportedBlockKeysProto keys) {
+    return new ExportedBlockKeys(keys.getIsBlockTokenEnabled(),
+        keys.getKeyUpdateInterval(), keys.getTokenLifeTime(),
+        convert(keys.getCurrentKey()), convertBlockKeys(keys.getAllKeysList()));
+  }
+
+  public static CheckpointSignatureProto convert(CheckpointSignature s) {
+    return CheckpointSignatureProto.newBuilder()
+        .setBlockPoolId(s.getBlockpoolID())
+        .setCurSegmentTxId(s.getCurSegmentTxId())
+        .setMostRecentCheckpointTxId(s.getMostRecentCheckpointTxId())
+        .setStorageInfo(PBHelper.convert((StorageInfo) s)).build();
+  }
+
+  public static CheckpointSignature convert(CheckpointSignatureProto s) {
+    return new CheckpointSignature(PBHelper.convert(s.getStorageInfo()),
+        s.getBlockPoolId(), s.getMostRecentCheckpointTxId(),
+        s.getCurSegmentTxId());
+  }
+
+  public static RemoteEditLogProto convert(RemoteEditLog log) {
+    return RemoteEditLogProto.newBuilder().setEndTxId(log.getEndTxId())
+        .setStartTxId(log.getStartTxId()).build();
+  }
+
+  public static RemoteEditLog convert(RemoteEditLogProto l) {
+    return new RemoteEditLog(l.getStartTxId(), l.getEndTxId());
+  }
+
+  public static RemoteEditLogManifestProto convert(
+      RemoteEditLogManifest manifest) {
+    RemoteEditLogManifestProto.Builder builder = RemoteEditLogManifestProto
+        .newBuilder();
+    for (RemoteEditLog log : manifest.getLogs()) {
+      builder.addLogs(convert(log));
+    }
+    return builder.build();
+  }
+
+  public static RemoteEditLogManifest convert(
+      RemoteEditLogManifestProto manifest) {
+    List<RemoteEditLog> logs = new ArrayList<RemoteEditLog>(manifest
+        .getLogsList().size());
+    for (RemoteEditLogProto l : manifest.getLogsList()) {
+      logs.add(convert(l));
+    }
+    return new RemoteEditLogManifest(logs);
+  }
+
+  public static CheckpointCommandProto convert(CheckpointCommand cmd) {
+    return CheckpointCommandProto.newBuilder()
+        .setSignature(convert(cmd.getSignature()))
+        .setNeedToReturnImage(cmd.needToReturnImage()).build();
+  }
+
+  public static NamenodeCommandProto convert(NamenodeCommand cmd) {
+    if (cmd instanceof CheckpointCommand) {
+      return NamenodeCommandProto.newBuilder().setAction(cmd.getAction())
+          .setType(NamenodeCommandProto.Type.CheckPointCommand)
+          .setCheckpointCmd(convert((CheckpointCommand) cmd)).build();
+    }
+    return NamenodeCommandProto.newBuilder()
+        .setType(NamenodeCommandProto.Type.NamenodeCommand)
+        .setAction(cmd.getAction()).build();
+  }
+
+  public static BlockKey[] convertBlockKeys(List<BlockKeyProto> list) {
+    BlockKey[] ret = new BlockKey[list.size()];
+    int i = 0;
+    for (BlockKeyProto k : list) {
+      ret[i++] = convert(k);
+    }
+    return ret;
+  }
+
+  public static NamespaceInfo convert(NamespaceInfoProto info) {
+    StorageInfoProto storage = info.getStorageInfo();
+    return new NamespaceInfo(storage.getNamespceID(), storage.getClusterID(),
+        info.getBlockPoolID(), storage.getCTime(), info.getDistUpgradeVersion());
+  }
+
+  public static NamenodeCommand convert(NamenodeCommandProto cmd) {
+    if (cmd == null) return null;
+    switch (cmd.getType()) {
+    case CheckPointCommand:
+      CheckpointCommandProto chkPt = cmd.getCheckpointCmd();
+      return new CheckpointCommand(PBHelper.convert(chkPt.getSignature()),
+          chkPt.getNeedToReturnImage());
+    default:
+      return new NamenodeCommand(cmd.getAction());
+    }
+  }
+  
+  public static ExtendedBlock convert(ExtendedBlockProto eb) {
+    if (eb == null) return null;
+    return new ExtendedBlock( eb.getPoolId(),  eb.getBlockId(),   eb.getNumBytes(),
+       eb.getGenerationStamp());
+  }
+  
+  public static ExtendedBlockProto convert(final ExtendedBlock b) {
+    if (b == null) return null;
+   return ExtendedBlockProto.newBuilder().
+      setPoolId(b.getBlockPoolId()).
+      setBlockId(b.getBlockId()).
+      setNumBytes(b.getNumBytes()).
+      setGenerationStamp(b.getGenerationStamp()).
+      build();
+  }
+  
+  public static RecoveringBlockProto convert(RecoveringBlock b) {
+    if (b == null) {
+      return null;
+    }
+    LocatedBlockProto lb = PBHelper.convert((LocatedBlock)b);
+    return RecoveringBlockProto.newBuilder().setBlock(lb)
+        .setNewGenStamp(b.getNewGenerationStamp()).build();
+  }
+
+  public static RecoveringBlock convert(RecoveringBlockProto b) {
+    ExtendedBlock block = convert(b.getBlock().getB());
+    DatanodeInfo[] locs = convert(b.getBlock().getLocsList());
+    return new RecoveringBlock(block, locs, b.getNewGenStamp());
+  }
+  
+  public static DatanodeInfoProto.AdminState convert(
+      final DatanodeInfo.AdminStates inAs) {
+    switch (inAs) {
+    case NORMAL: return  DatanodeInfoProto.AdminState.NORMAL;
+    case DECOMMISSION_INPROGRESS: 
+        return DatanodeInfoProto.AdminState.DECOMMISSION_INPROGRESS;
+    case DECOMMISSIONED: return DatanodeInfoProto.AdminState.DECOMMISSIONED;
+    default: return DatanodeInfoProto.AdminState.NORMAL;
+    }
+  }
+  
+  static public DatanodeInfo convert(DatanodeInfoProto di) {
+    if (di == null) return null;
+    return new DatanodeInfo(
+        PBHelper.convert(di.getId()),
+        di.hasLocation() ? di.getLocation() : null , 
+        di.hasHostName() ? di.getHostName() : null,
+        di.getCapacity(),  di.getDfsUsed(),  di.getRemaining(),
+        di.getBlockPoolUsed()  ,  di.getLastUpdate() , di.getXceiverCount() ,
+        PBHelper.convert(di.getAdminState())); 
+  }
+  
+  static public DatanodeInfoProto convertDatanodeInfo(DatanodeInfo di) {
+    if (di == null) return null;
+    DatanodeInfoProto.Builder builder = DatanodeInfoProto.newBuilder();
+    if (di.getHostName() != null) {
+      builder.setHostName(di.getHostName());
+    }
+    if (di.getNetworkLocation() != null) {
+      builder.setLocation(di.getNetworkLocation());
+    }
+        
+    return builder.
+     setId(PBHelper.convert((DatanodeID) di)).
+     setCapacity(di.getCapacity()).
+     setDfsUsed(di.getDfsUsed()).
+     setRemaining(di.getRemaining()).
+     setBlockPoolUsed(di.getBlockPoolUsed()).
+     setLastUpdate(di.getLastUpdate()).
+     setXceiverCount(di.getXceiverCount()).
+     setAdminState(PBHelper.convert(di.getAdminState())).
+     build();     
+  }
+  
+  
+  static public DatanodeInfo[] convert(DatanodeInfoProto di[]) {
+    if (di == null) return null;
+    DatanodeInfo[] result = new DatanodeInfo[di.length];
+    for (int i = 0; i < di.length; i++) {
+      result[i] = convert(di[i]);
+    }    
+    return result;
+  }
+  
+  static public DatanodeInfoProto[] convert(DatanodeInfo[] di) {
+    if (di == null) return null;
+    DatanodeInfoProto[] result = new DatanodeInfoProto[di.length];
+    for (int i = 0; i < di.length; i++) {
+      result[i] = PBHelper.convertDatanodeInfo(di[i]);
+    }
+    return result;
+  }
+
+  public static DatanodeInfo[] convert(List<DatanodeInfoProto> list) {
+    DatanodeInfo[] info = new DatanodeInfo[list.size()];
+    for (int i = 0; i < info.length; i++) {
+      info[i] = convert(list.get(i));
+    }
+    return info;
+  }
+  
+  public static DatanodeInfoProto convert(DatanodeInfo info) {
+    DatanodeInfoProto.Builder builder = DatanodeInfoProto.newBuilder();
+    builder.setBlockPoolUsed(info.getBlockPoolUsed());
+    builder.setAdminState(PBHelper.convert(info.getAdminState()));
+    builder.setCapacity(info.getCapacity())
+        .setDfsUsed(info.getDfsUsed())
+        .setHostName(info.getHostName())
+        .setId(PBHelper.convert((DatanodeID)info))
+        .setLastUpdate(info.getLastUpdate())
+        .setLocation(info.getNetworkLocation())
+        .setRemaining(info.getRemaining())
+        .setXceiverCount(info.getXceiverCount())
+        .build();
+    return builder.build();
+  }
+
+  public static AdminStates convert(AdminState adminState) {
+    switch(adminState) {
+    case DECOMMISSION_INPROGRESS:
+      return AdminStates.DECOMMISSION_INPROGRESS;
+    case DECOMMISSIONED:
+      return AdminStates.DECOMMISSIONED;
+    case NORMAL:
+    default:
+      return AdminStates.NORMAL;
+    }
+  }
+  
+  public static LocatedBlockProto convert(LocatedBlock b) {
+    if (b == null) return null;
+    Builder builder = LocatedBlockProto.newBuilder();
+    DatanodeInfo[] locs = b.getLocations();
+    for (int i = 0; i < locs.length; i++) {
+      builder.addLocs(i, PBHelper.convert(locs[i]));
+    }
+    return builder.setB(PBHelper.convert(b.getBlock()))
+        .setBlockToken(PBHelper.convert(b.getBlockToken()))
+        .setCorrupt(b.isCorrupt()).setOffset(b.getStartOffset()).build();
+  }
+  
+  public static LocatedBlock convert(LocatedBlockProto proto) {
+    if (proto == null) return null;
+    List<DatanodeInfoProto> locs = proto.getLocsList();
+    DatanodeInfo[] targets = new DatanodeInfo[locs.size()];
+    for (int i = 0; i < locs.size(); i++) {
+      targets[i] = PBHelper.convert(locs.get(i));
+    }
+    LocatedBlock lb = new LocatedBlock(PBHelper.convert(proto.getB()), targets,
+        proto.getOffset(), proto.getCorrupt());
+    lb.setBlockToken(PBHelper.convert(proto.getBlockToken()));
+    return lb;
+  }
+
+  public static BlockTokenIdentifierProto convert(Token<?> tok) {
+    return BlockTokenIdentifierProto.newBuilder().
+              setIdentifier(ByteString.copyFrom(tok.getIdentifier())).
+              setPassword(ByteString.copyFrom(tok.getPassword())).
+              setKind(tok.getKind().toString()).
+              setService(tok.getService().toString()).build(); 
+  }
+  
+  public static Token<BlockTokenIdentifier> convert(
+      BlockTokenIdentifierProto blockToken) {
+    return new Token<BlockTokenIdentifier>(blockToken.getIdentifier()
+        .toByteArray(), blockToken.getPassword().toByteArray(), new Text(
+        blockToken.getKind()), new Text(blockToken.getService()));
+  }
+
+  
+  public static Token<DelegationTokenIdentifier> convertDelegationToken(
+      BlockTokenIdentifierProto blockToken) {
+    return new Token<DelegationTokenIdentifier>(blockToken.getIdentifier()
+        .toByteArray(), blockToken.getPassword().toByteArray(), new Text(
+        blockToken.getKind()), new Text(blockToken.getService()));
+  }
+
+  public static ReplicaState convert(ReplicaStateProto state) {
+    switch (state) {
+    case RBW:
+      return ReplicaState.RBW;
+    case RUR:
+      return ReplicaState.RUR;
+    case RWR:
+      return ReplicaState.RWR;
+    case TEMPORARY:
+      return ReplicaState.TEMPORARY;
+    case FINALIZED:
+    default:
+      return ReplicaState.FINALIZED;
+    }
+  }
+
+  public static ReplicaStateProto convert(ReplicaState state) {
+    switch (state) {
+    case RBW:
+      return ReplicaStateProto.RBW;
+    case RUR:
+      return ReplicaStateProto.RUR;
+    case RWR:
+      return ReplicaStateProto.RWR;
+    case TEMPORARY:
+      return ReplicaStateProto.TEMPORARY;
+    case FINALIZED:
+    default:
+      return ReplicaStateProto.FINALIZED;
+    }
+  }
+  
+  public static DatanodeRegistrationProto convert(
+      DatanodeRegistration registration) {
+    DatanodeRegistrationProto.Builder builder = DatanodeRegistrationProto
+        .newBuilder();
+    return builder.setDatanodeID(PBHelper.convert((DatanodeID) registration))
+        .setStorageInfo(PBHelper.convert(registration.storageInfo))
+        .setKeys(PBHelper.convert(registration.exportedKeys)).build();
+  }
+
+  public static DatanodeRegistration convert(DatanodeRegistrationProto proto) {
+    return new DatanodeRegistration(PBHelper.convert(proto.getDatanodeID()),
+        PBHelper.convert(proto.getStorageInfo()), PBHelper.convert(proto
+            .getKeys()));
+  }
+
+  public static DatanodeCommand convert(DatanodeCommandProto proto) {
+    switch (proto.getCmdType()) {
+    case BalancerBandwidthCommand:
+      return PBHelper.convert(proto.getBalancerCmd());
+    case BlockCommand:
+      return PBHelper.convert(proto.getBlkCmd());
+    case BlockRecoveryCommand:
+      return PBHelper.convert(proto.getRecoveryCmd());
+    case FinalizeCommand:
+      return PBHelper.convert(proto.getFinalizeCmd());
+    case KeyUpdateCommand:
+      return PBHelper.convert(proto.getKeyUpdateCmd());
+    case RegisterCommand:
+      return REG_CMD;
+    case UpgradeCommand:
+      return PBHelper.convert(proto.getUpgradeCmd());
+    }
+    return null;
+  }
+  
+  public static BalancerBandwidthCommandProto convert(
+      BalancerBandwidthCommand bbCmd) {
+    return BalancerBandwidthCommandProto.newBuilder()
+        .setBandwidth(bbCmd.getBalancerBandwidthValue()).build();
+  }
+
+  public static KeyUpdateCommandProto convert(KeyUpdateCommand cmd) {
+    return KeyUpdateCommandProto.newBuilder()
+        .setKeys(PBHelper.convert(cmd.getExportedKeys())).build();
+  }
+
+  public static BlockRecoveryCommandProto convert(BlockRecoveryCommand cmd) {
+    BlockRecoveryCommandProto.Builder builder = BlockRecoveryCommandProto
+        .newBuilder();
+    for (RecoveringBlock b : cmd.getRecoveringBlocks()) {
+      builder.addBlocks(PBHelper.convert(b));
+    }
+    return builder.build();
+  }
+
+  public static FinalizeCommandProto convert(FinalizeCommand cmd) {
+    return FinalizeCommandProto.newBuilder()
+        .setBlockPoolId(cmd.getBlockPoolId()).build();
+  }
+
+  public static BlockCommandProto convert(BlockCommand cmd) {
+    BlockCommandProto.Builder builder = BlockCommandProto.newBuilder()
+        .setBlockPoolId(cmd.getBlockPoolId());
+    switch (cmd.getAction()) {
+    case DatanodeProtocol.DNA_TRANSFER:
+      builder.setAction(BlockCommandProto.Action.TRANSFER);
+      break;
+    case DatanodeProtocol.DNA_INVALIDATE:
+      builder.setAction(BlockCommandProto.Action.INVALIDATE);
+      break;
+    case DatanodeProtocol.DNA_SHUTDOWN:
+      builder.setAction(BlockCommandProto.Action.SHUTDOWN);
+      break;
+    }
+    Block[] blocks = cmd.getBlocks();
+    for (int i = 0; i < blocks.length; i++) {
+      builder.addBlocks(PBHelper.convert(blocks[i]));
+    }
+    builder.addAllTargets(PBHelper.convert(cmd.getTargets()));
+    return builder.build();
+  }
+
+  private static List<DatanodeInfosProto> convert(DatanodeInfo[][] targets) {
+    DatanodeInfosProto[] ret = new DatanodeInfosProto[targets.length];
+    for (int i = 0; i < targets.length; i++) {
+      ret[i] = DatanodeInfosProto.newBuilder()
+          .addAllDatanodes(Arrays.asList(PBHelper.convert(targets[i]))).build();
+    }
+    return Arrays.asList(ret);
+  }
+
+  public static DatanodeCommandProto convert(DatanodeCommand datanodeCommand) {
+    DatanodeCommandProto.Builder builder = DatanodeCommandProto.newBuilder();
+    if (datanodeCommand == null) {
+      return builder.setCmdType(DatanodeCommandProto.Type.NullDatanodeCommand)
+          .build();
+    }
+    switch (datanodeCommand.getAction()) {
+    case DatanodeProtocol.DNA_BALANCERBANDWIDTHUPDATE:
+      builder.setCmdType(DatanodeCommandProto.Type.BalancerBandwidthCommand)
+          .setBalancerCmd(
+              PBHelper.convert((BalancerBandwidthCommand) datanodeCommand));
+      break;
+    case DatanodeProtocol.DNA_ACCESSKEYUPDATE:
+      builder
+          .setCmdType(DatanodeCommandProto.Type.KeyUpdateCommand)
+          .setKeyUpdateCmd(PBHelper.convert((KeyUpdateCommand) datanodeCommand));
+      break;
+    case DatanodeProtocol.DNA_RECOVERBLOCK:
+      builder.setCmdType(DatanodeCommandProto.Type.BlockRecoveryCommand)
+          .setRecoveryCmd(
+              PBHelper.convert((BlockRecoveryCommand) datanodeCommand));
+      break;
+    case DatanodeProtocol.DNA_FINALIZE:
+      builder.setCmdType(DatanodeCommandProto.Type.FinalizeCommand)
+          .setFinalizeCmd(PBHelper.convert((FinalizeCommand) datanodeCommand));
+      break;
+    case DatanodeProtocol.DNA_REGISTER:
+      builder.setCmdType(DatanodeCommandProto.Type.RegisterCommand)
+          .setRegisterCmd(REG_CMD_PROTO);
+      break;
+    case DatanodeProtocol.DNA_TRANSFER:
+    case DatanodeProtocol.DNA_INVALIDATE:
+    case DatanodeProtocol.DNA_SHUTDOWN:
+      builder.setCmdType(DatanodeCommandProto.Type.BlockCommand).setBlkCmd(
+          PBHelper.convert((BlockCommand) datanodeCommand));
+      break;
+    case DatanodeProtocol.DNA_UC_ACTION_REPORT_STATUS:
+    case DatanodeProtocol.DNA_UC_ACTION_START_UPGRADE:
+      builder.setCmdType(DatanodeCommandProto.Type.UpgradeCommand)
+          .setUpgradeCmd(PBHelper.convert((UpgradeCommand) datanodeCommand));
+      break;
+    case DatanodeProtocol.DNA_UNKNOWN: //Not expected
+    default:
+      builder.setCmdType(DatanodeCommandProto.Type.NullDatanodeCommand);
+    }
+    return builder.build();
+  }
+
+  public static UpgradeCommand convert(UpgradeCommandProto upgradeCmd) {
+    int action = UpgradeCommand.UC_ACTION_UNKNOWN;
+    switch (upgradeCmd.getAction()) {
+    case REPORT_STATUS:
+      action = UpgradeCommand.UC_ACTION_REPORT_STATUS;
+      break;
+    case START_UPGRADE:
+      action = UpgradeCommand.UC_ACTION_START_UPGRADE;
+    }
+    return new UpgradeCommand(action, upgradeCmd.getVersion(),
+        (short) upgradeCmd.getUpgradeStatus());
+  }
+
+  public static KeyUpdateCommand convert(KeyUpdateCommandProto keyUpdateCmd) {
+    return new KeyUpdateCommand(PBHelper.convert(keyUpdateCmd.getKeys()));
+  }
+
+  public static FinalizeCommand convert(FinalizeCommandProto finalizeCmd) {
+    return new FinalizeCommand(finalizeCmd.getBlockPoolId());
+  }
+
+  public static BlockRecoveryCommand convert(
+      BlockRecoveryCommandProto recoveryCmd) {
+    List<RecoveringBlockProto> list = recoveryCmd.getBlocksList();
+    List<RecoveringBlock> recoveringBlocks = new ArrayList<RecoveringBlock>(
+        list.size());
+    
+    for (RecoveringBlockProto rbp : list) {
+      recoveringBlocks.add(PBHelper.convert(rbp));
+    }
+    return new BlockRecoveryCommand(recoveringBlocks);
+  }
+
+  public static BlockCommand convert(BlockCommandProto blkCmd) {
+    List<BlockProto> blockProtoList = blkCmd.getBlocksList();
+    Block[] blocks = new Block[blockProtoList.size()];
+    for (int i = 0; i < blockProtoList.size(); i++) {
+      blocks[i] = PBHelper.convert(blockProtoList.get(i));
+    }
+    List<DatanodeInfosProto> targetList = blkCmd.getTargetsList();
+    DatanodeInfo[][] targets = new DatanodeInfo[targetList.size()][];
+    for (int i = 0; i < targetList.size(); i++) {
+      targets[i] = PBHelper.convert(targetList.get(i));
+    }
+    int action = DatanodeProtocol.DNA_UNKNOWN;
+    switch (blkCmd.getAction()) {
+    case TRANSFER:
+      action = DatanodeProtocol.DNA_TRANSFER;
+      break;
+    case INVALIDATE:
+      action = DatanodeProtocol.DNA_INVALIDATE;
+      break;
+    case SHUTDOWN:
+      action = DatanodeProtocol.DNA_SHUTDOWN;
+      break;
+    }
+    return new BlockCommand(action, blkCmd.getBlockPoolId(), blocks, targets);
+  }
+
+  public static DatanodeInfo[] convert(DatanodeInfosProto datanodeInfosProto) {
+    List<DatanodeInfoProto> proto = datanodeInfosProto.getDatanodesList();
+    DatanodeInfo[] infos = new DatanodeInfo[proto.size()];
+    for (int i = 0; i < infos.length; i++) {
+      infos[i] = PBHelper.convert(proto.get(i));
+    }
+    return infos;
+  }
+
+  public static BalancerBandwidthCommand convert(
+      BalancerBandwidthCommandProto balancerCmd) {
+    return new BalancerBandwidthCommand(balancerCmd.getBandwidth());
+  }
+
+  public static ReceivedDeletedBlockInfoProto convert(
+      ReceivedDeletedBlockInfo receivedDeletedBlockInfo) {
+    ReceivedDeletedBlockInfoProto.Builder builder = 
+        ReceivedDeletedBlockInfoProto.newBuilder();
+    
+    if (receivedDeletedBlockInfo.getDelHints() != null) {
+      builder.setDeleteHint(receivedDeletedBlockInfo.getDelHints());
+    }
+    return builder.setBlock(PBHelper.convert(receivedDeletedBlockInfo.getBlock()))
+        .build();
+  }
+
+  public static UpgradeCommandProto convert(UpgradeCommand comm) {
+    UpgradeCommandProto.Builder builder = UpgradeCommandProto.newBuilder();
+    if (comm == null) {
+      return builder.setAction(UpgradeCommandProto.Action.UNKNOWN)
+          .setVersion(0).setUpgradeStatus(0).build();
+    }
+    builder.setVersion(comm.getVersion()).setUpgradeStatus(
+        comm.getCurrentStatus());
+    switch (comm.getAction()) {
+    case UpgradeCommand.UC_ACTION_REPORT_STATUS:
+      builder.setAction(UpgradeCommandProto.Action.REPORT_STATUS);
+      break;
+    case UpgradeCommand.UC_ACTION_START_UPGRADE:
+      builder.setAction(UpgradeCommandProto.Action.START_UPGRADE);
+      break;
+    default:
+      builder.setAction(UpgradeCommandProto.Action.UNKNOWN);
+      break;
+    }
+    return builder.build();
+  }
+
+  public static ReceivedDeletedBlockInfo convert(
+      ReceivedDeletedBlockInfoProto proto) {
+    return new ReceivedDeletedBlockInfo(PBHelper.convert(proto.getBlock()),
+        proto.hasDeleteHint() ? proto.getDeleteHint() : null);
+  }
+  
+  public static NamespaceInfoProto convert(NamespaceInfo info) {
+    return NamespaceInfoProto.newBuilder()
+        .setBlockPoolID(info.getBlockPoolID())
+        .setBuildVersion(info.getBuildVersion())
+        .setDistUpgradeVersion(info.getDistributedUpgradeVersion())
+        .setStorageInfo(PBHelper.convert((StorageInfo)info)).build();
+  }
+  
+  // Located Block Arrays and Lists
+  public static LocatedBlockProto[] convertLocatedBlock(LocatedBlock[] lb) {
+    if (lb == null) return null;
+    final int len = lb.length;
+    LocatedBlockProto[] result = new LocatedBlockProto[len];
+    for (int i = 0; i < len; ++i) {
+      result[i] = PBHelper.convert(lb[i]);
+    }
+    return result;
+  }
+  
+  public static LocatedBlock[] convertLocatedBlock(LocatedBlockProto[] lb) {
+    if (lb == null) return null;
+    final int len = lb.length;
+    LocatedBlock[] result = new LocatedBlock[len];
+    for (int i = 0; i < len; ++i) {
+      result[i] = new LocatedBlock(
+          PBHelper.convert(lb[i].getB()),
+          PBHelper.convert(lb[i].getLocsList()), 
+          lb[i].getOffset(), lb[i].getCorrupt());
+    }
+    return result;
+  }
+  
+  public static List<LocatedBlock> convertLocatedBlock(
+      List<LocatedBlockProto> lb) {
+    if (lb == null) return null;
+    final int len = lb.size();
+    List<LocatedBlock> result = 
+        new ArrayList<LocatedBlock>(len);
+    for (int i = 0; i < len; ++i) {
+      result.add(PBHelper.convert(lb.get(i)));
+    }
+    return result;
+  }
+  
+  public static List<LocatedBlockProto> convertLocatedBlock2(List<LocatedBlock> lb) {
+    if (lb == null) return null;
+    final int len = lb.size();
+    List<LocatedBlockProto> result = new ArrayList<LocatedBlockProto>(len);
+    for (int i = 0; i < len; ++i) {
+      result.add(PBHelper.convert(lb.get(i)));
+    }
+    return result;
+  }
+  
+  
+  // LocatedBlocks
+  public static LocatedBlocks convert(LocatedBlocksProto lb) {
+    return new LocatedBlocks(
+        lb.getFileLength(), lb.getUnderConstruction(),
+        PBHelper.convertLocatedBlock(lb.getBlocksList()),
+        lb.hasLastBlock() ? PBHelper.convert(lb.getLastBlock()) : null,
+        lb.getIsLastBlockComplete());
+  }
+  
+  public static LocatedBlocksProto convert(LocatedBlocks lb) {
+    if (lb == null) {
+      return null;
+    }
+    LocatedBlocksProto.Builder builder = 
+        LocatedBlocksProto.newBuilder();
+    if (lb.getLastLocatedBlock() != null) {
+      builder.setLastBlock(PBHelper.convert(lb.getLastLocatedBlock()));
+    }
+    return builder.setFileLength(lb.getFileLength())
+        .setUnderConstruction(lb.isUnderConstruction())
+        .addAllBlocks(PBHelper.convertLocatedBlock2(lb.getLocatedBlocks()))
+        .setIsLastBlockComplete(lb.isLastBlockComplete()).build();
+  }
+  
+  public static FsServerDefaults convert(FsServerDefaultsProto fs) {
+    if (fs == null) return null;
+    return new FsServerDefaults(
+        fs.getBlockSize(), fs.getBytesPerChecksum(), 
+        fs.getWritePacketSize(), (short) fs.getReplication(),
+        fs.getFileBufferSize());
+  }
+  
+  public static FsServerDefaultsProto convert(FsServerDefaults fs) {
+    if (fs == null) return null;
+    return FsServerDefaultsProto.newBuilder().
+      setBlockSize(fs.getBlockSize()).
+      setBytesPerChecksum(fs.getBytesPerChecksum()).
+      setWritePacketSize(fs.getWritePacketSize()).setReplication(fs.getReplication()).setFileBufferSize(fs.getFileBufferSize()).build();
+  }
+  
+  public static FsPermissionProto convert(FsPermission p) {
+    if (p == null) return null;
+    return FsPermissionProto.newBuilder().setPerm(p.toShort()).build();
+  }
+  
+  public static FsPermission convert(FsPermissionProto p) {
+    if (p == null) return null;
+    return new FsPermission((short)p.getPerm());
+  }
+  
+  
+  // The creatFlag field in PB is a bitmask whose values are the same a the 
+  // emum values of CreateFlag
+  public static int convertCreateFlag(EnumSetWritable<CreateFlag> flag) {
+    int value = 0;
+    if (flag.contains(CreateFlag.APPEND)) {
+      value |= CreateFlagProto.APPEND.getNumber();
+    }
+    if (flag.contains(CreateFlag.CREATE)) {
+      value |= CreateFlagProto.CREATE.getNumber();
+    }
+    if (flag.contains(CreateFlag.OVERWRITE)) {
+      value |= CreateFlagProto.OVERWRITE.getNumber();
+    }
+    return value;
+  }
+  
+  public static EnumSetWritable<CreateFlag> convert(int flag) {
+    EnumSet<CreateFlag> result = 
+       EnumSet.noneOf(CreateFlag.class);   
+    if ((flag & CreateFlagProto.APPEND_VALUE) == CreateFlagProto.APPEND_VALUE) {
+      result.add(CreateFlag.APPEND);
+    }
+    if ((flag & CreateFlagProto.CREATE_VALUE) == CreateFlagProto.CREATE_VALUE) {
+      result.add(CreateFlag.CREATE);
+    }
+    if ((flag & CreateFlagProto.OVERWRITE_VALUE) 
+        == CreateFlagProto.OVERWRITE_VALUE) {
+      result.add(CreateFlag.OVERWRITE);
+    }
+    return new EnumSetWritable<CreateFlag>(result);
+  }
+  
+  
+  public static HdfsFileStatus convert(HdfsFileStatusProto fs) {
+    if (fs == null)
+      return null;
+    return new HdfsLocatedFileStatus(
+        fs.getLength(), fs.getFileType().equals(FileType.IS_DIR), 
+        fs.getBlockReplication(), fs.getBlocksize(),
+        fs.getModificationTime(), fs.getAccessTime(),
+        PBHelper.convert(fs.getPermission()), fs.getOwner(), fs.getGroup(), 
+        fs.getFileType().equals(FileType.IS_SYMLINK) ? 
+            fs.getSymlink().toByteArray() : null,
+        fs.getPath().toByteArray(),
+        fs.hasLocations() ? PBHelper.convert(fs.getLocations()) : null);
+  }
+
+  public static HdfsFileStatusProto convert(HdfsFileStatus fs) {
+    if (fs == null)
+      return null;
+    FileType fType = FileType.IS_FILE;
+    if (fs.isDir()) {
+      fType = FileType.IS_DIR;
+    } else if (fs.isSymlink()) {
+      fType = FileType.IS_SYMLINK;
+    }
+
+    HdfsFileStatusProto.Builder builder = 
+     HdfsFileStatusProto.newBuilder().
+      setLength(fs.getLen()).
+      setFileType(fType).
+      setBlockReplication(fs.getReplication()).
+      setBlocksize(fs.getBlockSize()).
+      setModificationTime(fs.getModificationTime()).
+      setAccessTime(fs.getAccessTime()).
+      setPermission(PBHelper.convert(fs.getPermission())).
+      setOwner(fs.getOwner()).
+      setGroup(fs.getGroup()).
+      setPath(ByteString.copyFrom(fs.getLocalNameInBytes()));
+    if (fs.isSymlink())  {
+      builder.setSymlink(ByteString.copyFrom(fs.getSymlinkInBytes()));
+    }
+    if (fs instanceof HdfsLocatedFileStatus) {
+      LocatedBlocks locations = ((HdfsLocatedFileStatus)fs).getBlockLocations();
+      if (locations != null) {
+        builder.setLocations(PBHelper.convert(locations));
+      }
+    }
+    return builder.build();
+  }
+  
+  public static HdfsFileStatusProto[] convert(HdfsFileStatus[] fs) {
+    if (fs == null) return null;
+    final int len = fs.length;
+    HdfsFileStatusProto[] result = new HdfsFileStatusProto[len];
+    for (int i = 0; i < len; ++i) {
+      result[i] = PBHelper.convert(fs[i]);
+    }
+    return result;
+  }
+  
+  public static HdfsFileStatus[] convert(HdfsFileStatusProto[] fs) {
+    if (fs == null) return null;
+    final int len = fs.length;
+    HdfsFileStatus[] result = new HdfsFileStatus[len];
+    for (int i = 0; i < len; ++i) {
+      result[i] = PBHelper.convert(fs[i]);
+    }
+    return result;
+  }
+  
+  public static DirectoryListing convert(DirectoryListingProto dl) {
+    if (dl == null)
+      return null;
+    List<HdfsFileStatusProto> partList =  dl.getPartialListingList();
+    return new DirectoryListing( 
+        partList.isEmpty() ? new HdfsLocatedFileStatus[0] 
+          : PBHelper.convert(
+              partList.toArray(new HdfsFileStatusProto[partList.size()])),
+        dl.getRemainingEntries());
+  }
+
+  public static DirectoryListingProto convert(DirectoryListing d) {
+    if (d == null)
+      return null;
+    return DirectoryListingProto.newBuilder().
+        addAllPartialListing(Arrays.asList(
+            PBHelper.convert(d.getPartialListing()))).
+        setRemainingEntries(d.getRemainingEntries()).
+        build();
+  }
+
+  public static long[] convert(GetFsStatsResponseProto res) {
+    long[] result = new long[6];
+    result[ClientProtocol.GET_STATS_CAPACITY_IDX] = res.getCapacity();
+    result[ClientProtocol.GET_STATS_USED_IDX] = res.getUsed();
+    result[ClientProtocol.GET_STATS_REMAINING_IDX] = res.getRemaining();
+    result[ClientProtocol.GET_STATS_UNDER_REPLICATED_IDX] = res.getUnderReplicated();
+    result[ClientProtocol.GET_STATS_CORRUPT_BLOCKS_IDX] = res.getCorruptBlocks();
+    result[ClientProtocol.GET_STATS_MISSING_BLOCKS_IDX] = res.getMissingBlocks();
+    return result;
+  }
+  
+  public static GetFsStatsResponseProto convert(long[] fsStats) {
+    GetFsStatsResponseProto.Builder result = GetFsStatsResponseProto
+        .newBuilder();
+    if (fsStats.length >= ClientProtocol.GET_STATS_CAPACITY_IDX + 1)
+      result.setCapacity(fsStats[ClientProtocol.GET_STATS_CAPACITY_IDX]);
+    if (fsStats.length >= ClientProtocol.GET_STATS_USED_IDX + 1)
+      result.setUsed(fsStats[ClientProtocol.GET_STATS_USED_IDX]);
+    if (fsStats.length >= ClientProtocol.GET_STATS_REMAINING_IDX + 1)
+      result.setRemaining(fsStats[ClientProtocol.GET_STATS_REMAINING_IDX]);
+    if (fsStats.length >= ClientProtocol.GET_STATS_UNDER_REPLICATED_IDX + 1)
+      result.setUnderReplicated(
+              fsStats[ClientProtocol.GET_STATS_UNDER_REPLICATED_IDX]);
+    if (fsStats.length >= ClientProtocol.GET_STATS_CORRUPT_BLOCKS_IDX + 1)
+      result.setCorruptBlocks(
+          fsStats[ClientProtocol.GET_STATS_CORRUPT_BLOCKS_IDX]);
+    if (fsStats.length >= ClientProtocol.GET_STATS_MISSING_BLOCKS_IDX + 1)
+      result.setMissingBlocks(
+          fsStats[ClientProtocol.GET_STATS_MISSING_BLOCKS_IDX]);
+    return result.build();
+  }
+  
+  public static DatanodeReportTypeProto
+    convert(DatanodeReportType t) {
+    switch (t) {
+    case ALL: return DatanodeReportTypeProto.ALL;
+    case LIVE: return DatanodeReportTypeProto.LIVE;
+    case DEAD: return DatanodeReportTypeProto.DEAD;
+    default: 
+      throw new IllegalArgumentException("Unexpected data type report:" + t);
+    }
+  }
+  
+  public static DatanodeReportType 
+    convert(DatanodeReportTypeProto t) {
+    switch (t) {
+    case ALL: return DatanodeReportType.ALL;
+    case LIVE: return DatanodeReportType.LIVE;
+    case DEAD: return DatanodeReportType.DEAD;
+    default: 
+      throw new IllegalArgumentException("Unexpected data type report:" + t);
+    }
+  }
+
+  public static SafeModeActionProto convert(
+      SafeModeAction a) {
+    switch (a) {
+    case SAFEMODE_LEAVE:
+      return SafeModeActionProto.SAFEMODE_LEAVE;
+    case SAFEMODE_ENTER:
+      return SafeModeActionProto.SAFEMODE_ENTER;
+    case SAFEMODE_GET:
+      return SafeModeActionProto.SAFEMODE_GET;
+    default:
+      throw new IllegalArgumentException("Unexpected SafeModeAction :" + a);
+    }
+  }
+  
+  public static SafeModeAction convert(
+      ClientNamenodeProtocolProtos.SafeModeActionProto a) {
+    switch (a) {
+    case SAFEMODE_LEAVE:
+      return SafeModeAction.SAFEMODE_LEAVE;
+    case SAFEMODE_ENTER:
+      return SafeModeAction.SAFEMODE_ENTER;
+    case SAFEMODE_GET:
+      return SafeModeAction.SAFEMODE_GET;
+    default:
+      throw new IllegalArgumentException("Unexpected SafeModeAction :" + a);
+    }
+  }
+  
+  public static UpgradeActionProto convert(
+      UpgradeAction a) {
+    switch (a) {
+    case GET_STATUS:
+      return UpgradeActionProto.GET_STATUS;
+    case DETAILED_STATUS:
+      return UpgradeActionProto.DETAILED_STATUS;
+    case FORCE_PROCEED:
+      return UpgradeActionProto.FORCE_PROCEED;
+    default:
+      throw new IllegalArgumentException("Unexpected UpgradeAction :" + a);
+    }
+  }
+  
+  
+  public static UpgradeAction convert(
+      UpgradeActionProto a) {
+    switch (a) {
+    case GET_STATUS:
+      return UpgradeAction.GET_STATUS;
+    case DETAILED_STATUS:
+      return UpgradeAction.DETAILED_STATUS;
+    case FORCE_PROCEED:
+      return UpgradeAction.FORCE_PROCEED;
+    default:
+      throw new IllegalArgumentException("Unexpected UpgradeAction :" + a);
+    }
+  }
+
+  public static UpgradeStatusReportProto convert(UpgradeStatusReport r) {
+    if (r == null)
+      return null;
+    return UpgradeStatusReportProto.newBuilder()
+        .setVersion(r.getVersion())
+        .setUpgradeStatus(r.getUpgradeStatus())
+        .setFinalized(r.isFinalized())
+        .build();
+  }
+  
+  public static UpgradeStatusReport convert(UpgradeStatusReportProto r) {
+    if (r == null) return null;
+    return new UpgradeStatusReport(r.getVersion(),
+        (short) r.getUpgradeStatus(), r.getFinalized());
+  }
+  
+  public static CorruptFileBlocks convert(CorruptFileBlocksProto c) {
+    if (c == null)
+      return null;
+    List<String> fileList = c.getFilesList();
+    return new CorruptFileBlocks(fileList.toArray(new String[fileList.size()]),
+        c.getCookie());
+  }
+
+  public static CorruptFileBlocksProto convert(CorruptFileBlocks c) {
+    if (c == null)
+      return null;
+    return CorruptFileBlocksProto.newBuilder().
+        addAllFiles(Arrays.asList(c.getFiles())).
+        setCookie(c.getCookie()).
+        build();
+  }
+  
+  public static ContentSummary convert(ContentSummaryProto cs) {
+    if (cs == null) return null;
+    return new ContentSummary(
+      cs.getLength(), cs.getFileCount(), cs.getDirectoryCount(), cs.getQuota(),
+      cs.getSpaceConsumed(), cs.getSpaceQuota());
+  }
+  
+  public static ContentSummaryProto convert(ContentSummary cs) {
+    if (cs == null) return null;
+    return ContentSummaryProto.newBuilder().
+        setLength(cs.getLength()).
+        setFileCount(cs.getFileCount()).
+        setDirectoryCount(cs.getDirectoryCount()).
+        setQuota(cs.getQuota()).
+        setSpaceConsumed(cs.getSpaceConsumed()).
+        setSpaceQuota(cs.getSpaceQuota()).
+        build();
+  }
+
+  public static DatanodeStorageProto convert(DatanodeStorage s) {
+    return DatanodeStorageProto.newBuilder()
+        .setState(PBHelper.convert(s.getState()))
+        .setStorageID(s.getStorageID()).build();
+  }
+
+  private static StorageState convert(State state) {
+    switch(state) {
+    case READ_ONLY:
+      return StorageState.READ_ONLY;
+    case NORMAL:
+    default:
+      return StorageState.NORMAL;
+    }
+  }
+
+  public static DatanodeStorage convert(DatanodeStorageProto s) {
+    return new DatanodeStorage(s.getStorageID(), PBHelper.convert(s.getState()));
+  }
+
+  private static State convert(StorageState state) {
+    switch(state) {
+    case READ_ONLY:
+      return DatanodeStorage.State.READ_ONLY;
+    case NORMAL:
+    default:
+      return DatanodeStorage.State.NORMAL;
+    }
+  }
+
+  public static StorageReportProto convert(StorageReport r) {
+    return StorageReportProto.newBuilder()
+        .setBlockPoolUsed(r.getBlockPoolUsed()).setCapacity(r.getCapacity())
+        .setDfsUsed(r.getDfsUsed()).setRemaining(r.getRemaining())
+        .setStorageID(r.getStorageID()).build();
+  }
+}
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/protocolPB/RefreshAuthorizationPolicyProtocolClientSideTranslatorPB.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/protocolPB/RefreshAuthorizationPolicyProtocolClientSideTranslatorPB.java
new file mode 100644
index 0000000..22b2bcf
--- /dev/null
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/protocolPB/RefreshAuthorizationPolicyProtocolClientSideTranslatorPB.java
@@ -0,0 +1,98 @@
+/**
+ * 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.RefreshAuthorizationPolicyProtocolProtos.RefreshServiceAclRequestProto;
+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.security.authorize.RefreshAuthorizationPolicyProtocol;
+
+import com.google.protobuf.RpcController;
+import com.google.protobuf.ServiceException;
+
+public class RefreshAuthorizationPolicyProtocolClientSideTranslatorPB implements
+    ProtocolMetaInterface, RefreshAuthorizationPolicyProtocol, Closeable {
+
+  /** RpcController is not used and hence is set to null */
+  private final static RpcController NULL_CONTROLLER = null;
+  private final RefreshAuthorizationPolicyProtocolPB rpcProxy;
+
+  public RefreshAuthorizationPolicyProtocolClientSideTranslatorPB(
+      InetSocketAddress nameNodeAddr, UserGroupInformation ugi,
+      Configuration conf) throws IOException {
+    RPC.setProtocolEngine(conf, RefreshAuthorizationPolicyProtocolPB.class,
+        ProtobufRpcEngine.class);
+    rpcProxy = RPC.getProxy(RefreshAuthorizationPolicyProtocolPB.class,
+        RPC.getProtocolVersion(RefreshAuthorizationPolicyProtocolPB.class),
+        NameNode.getAddress(conf), ugi, conf,
+        NetUtils.getSocketFactory(conf, RefreshAuthorizationPolicyProtocol.class));
+  }
+
+  @Override
+  public long getProtocolVersion(String protocol, long clientVersion)
+      throws IOException {
+    return rpcProxy.getProtocolVersion(protocol, clientVersion);
+  }
+
+  @Override
+  public ProtocolSignature getProtocolSignature(String protocol,
+      long clientVersion, int clientMethodsHash) throws IOException {
+    return ProtocolSignatureWritable.convert(rpcProxy.getProtocolSignature2(
+        protocol, clientVersion, clientMethodsHash));
+  }
+
+  @Override
+  public void close() throws IOException {
+    RPC.stopProxy(rpcProxy);
+  }
+
+  @Override
+  public void refreshServiceAcl() throws IOException {
+    RefreshServiceAclRequestProto request = RefreshServiceAclRequestProto
+        .newBuilder().build();
+    try {
+      rpcProxy.refreshServiceAcl(NULL_CONTROLLER, request);
+    } catch (ServiceException se) {
+      throw ProtobufHelper.getRemoteException(se);
+    }
+  }
+
+  @Override
+  public boolean isMethodSupported(String methodName) throws IOException {
+    return RpcClientUtil.isMethodSupported(rpcProxy,
+        RefreshAuthorizationPolicyProtocolPB.class,
+        RpcKind.RPC_PROTOCOL_BUFFER,
+        RPC.getProtocolVersion(RefreshAuthorizationPolicyProtocolPB.class),
+        methodName);
+  }
+}
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/protocolPB/RefreshAuthorizationPolicyProtocolPB.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/protocolPB/RefreshAuthorizationPolicyProtocolPB.java
new file mode 100644
index 0000000..842926c
--- /dev/null
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/protocolPB/RefreshAuthorizationPolicyProtocolPB.java
@@ -0,0 +1,49 @@
+/**
+ * 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.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(
+    serverPrincipal=CommonConfigurationKeys.HADOOP_SECURITY_SERVICE_USER_NAME_KEY)
+@ProtocolInfo(
+    protocolName = "org.apache.hadoop.security.authorize.RefreshAuthorizationPolicyProtocol", 
+    protocolVersion = 1)
+@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;
+}
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/protocolPB/RefreshAuthorizationPolicyProtocolServerSideTranslatorPB.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/protocolPB/RefreshAuthorizationPolicyProtocolServerSideTranslatorPB.java
new file mode 100644
index 0000000..3ae8c71
--- /dev/null
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/protocolPB/RefreshAuthorizationPolicyProtocolServerSideTranslatorPB.java
@@ -0,0 +1,90 @@
+/**
+ * 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.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;
+import com.google.protobuf.ServiceException;
+
+public class RefreshAuthorizationPolicyProtocolServerSideTranslatorPB implements
+    RefreshAuthorizationPolicyProtocolPB {
+
+  private final RefreshAuthorizationPolicyProtocol impl;
+
+  public RefreshAuthorizationPolicyProtocolServerSideTranslatorPB(
+      RefreshAuthorizationPolicyProtocol impl) {
+    this.impl = impl;
+  }
+
+  @Override
+  public long getProtocolVersion(String protocol, long clientVersion)
+      throws IOException {
+    return RPC.getProtocolVersion(RefreshAuthorizationPolicyProtocolPB.class);
+  }
+
+  @Override
+  public ProtocolSignature getProtocolSignature(String protocol,
+      long clientVersion, int clientMethodsHash) throws IOException {
+    /**
+     * Don't forward this to the server. The protocol version and signature is
+     * that of {@link RefreshAuthorizationPolicyProtocol}
+     */
+    if (!protocol.equals(RPC
+        .getProtocolName(RefreshAuthorizationPolicyProtocolPB.class))) {
+      throw new IOException("Namenode Serverside implements "
+          + RPC.getProtocolName(RefreshAuthorizationPolicyProtocolPB.class)
+          + ". The following requested protocol is unknown: " + protocol);
+    }
+
+    return ProtocolSignature.getProtocolSignature(clientMethodsHash,
+        RPC.getProtocolVersion(RefreshAuthorizationPolicyProtocolPB.class),
+        RefreshAuthorizationPolicyProtocolPB.class);
+  }
+
+  @Override
+  public ProtocolSignatureWritable getProtocolSignature2(String protocol,
+      long clientVersion, int clientMethodsHash) throws IOException {
+    /**
+     * Don't forward this to the server. The protocol version and signature is
+     * that of {@link RefreshAuthorizationPolicyProtocolPB}
+     */
+    return ProtocolSignatureWritable.convert(this.getProtocolSignature(
+        protocol, clientVersion, clientMethodsHash));
+  }
+
+  @Override
+  public RefreshServiceAclResponseProto refreshServiceAcl(
+      RpcController controller, RefreshServiceAclRequestProto request)
+      throws ServiceException {
+    try {
+      impl.refreshServiceAcl();
+    } catch (IOException e) {
+      throw new ServiceException(e);
+    }
+    return RefreshServiceAclResponseProto.newBuilder().build();
+  }
+}
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/protocolPB/RefreshUserMappingsProtocolClientSideTranslatorPB.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/protocolPB/RefreshUserMappingsProtocolClientSideTranslatorPB.java
new file mode 100644
index 0000000..c11cf51
--- /dev/null
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/protocolPB/RefreshUserMappingsProtocolClientSideTranslatorPB.java
@@ -0,0 +1,110 @@
+/**
+ * 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.RefreshUserMappingsProtocolProtos.RefreshSuperUserGroupsConfigurationRequestProto;
+import org.apache.hadoop.hdfs.protocol.proto.RefreshUserMappingsProtocolProtos.RefreshUserToGroupsMappingsRequestProto;
+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.RefreshUserMappingsProtocol;
+import org.apache.hadoop.security.UserGroupInformation;
+
+import com.google.protobuf.RpcController;
+import com.google.protobuf.ServiceException;
+
+public class RefreshUserMappingsProtocolClientSideTranslatorPB implements
+    ProtocolMetaInterface, RefreshUserMappingsProtocol, Closeable {
+
+  /** RpcController is not used and hence is set to null */
+  private final static RpcController NULL_CONTROLLER = null;
+  private final RefreshUserMappingsProtocolPB rpcProxy;
+
+  public RefreshUserMappingsProtocolClientSideTranslatorPB(
+      InetSocketAddress nameNodeAddr, UserGroupInformation ugi,
+      Configuration conf) throws IOException {
+    RPC.setProtocolEngine(conf, RefreshUserMappingsProtocolPB.class,
+        ProtobufRpcEngine.class);
+    rpcProxy = RPC.getProxy(RefreshUserMappingsProtocolPB.class,
+        RPC.getProtocolVersion(RefreshUserMappingsProtocolPB.class),
+        NameNode.getAddress(conf), ugi, conf,
+        NetUtils.getSocketFactory(conf, RefreshUserMappingsProtocol.class));
+  }
+
+  @Override
+  public long getProtocolVersion(String protocol, long clientVersion)
+      throws IOException {
+    return rpcProxy.getProtocolVersion(protocol, clientVersion);
+  }
+
+  @Override
+  public ProtocolSignature getProtocolSignature(String protocol,
+      long clientVersion, int clientMethodsHash) throws IOException {
+    return ProtocolSignatureWritable.convert(rpcProxy.getProtocolSignature2(
+        protocol, clientVersion, clientMethodsHash));
+  }
+
+  @Override
+  public void close() throws IOException {
+    RPC.stopProxy(rpcProxy);
+  }
+
+  @Override
+  public void refreshUserToGroupsMappings() throws IOException {
+    RefreshUserToGroupsMappingsRequestProto request = 
+        RefreshUserToGroupsMappingsRequestProto.newBuilder().build();
+    try {
+      rpcProxy.refreshUserToGroupsMappings(NULL_CONTROLLER, request);
+    } catch (ServiceException se) {
+      throw ProtobufHelper.getRemoteException(se);
+    }
+  }
+
+  @Override
+  public void refreshSuperUserGroupsConfiguration() throws IOException {
+    RefreshSuperUserGroupsConfigurationRequestProto request = 
+        RefreshSuperUserGroupsConfigurationRequestProto.newBuilder().build();
+    try {
+      rpcProxy.refreshSuperUserGroupsConfiguration(NULL_CONTROLLER, request);
+    } catch (ServiceException se) {
+      throw ProtobufHelper.getRemoteException(se);
+    }
+  }
+
+  @Override
+  public boolean isMethodSupported(String methodName) throws IOException {
+    return RpcClientUtil
+        .isMethodSupported(rpcProxy, RefreshUserMappingsProtocolPB.class,
+            RpcKind.RPC_PROTOCOL_BUFFER,
+            RPC.getProtocolVersion(RefreshUserMappingsProtocolPB.class),
+            methodName);
+  }
+}
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/protocolPB/RefreshUserMappingsProtocolPB.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/protocolPB/RefreshUserMappingsProtocolPB.java
new file mode 100644
index 0000000..16a77ff
--- /dev/null
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/protocolPB/RefreshUserMappingsProtocolPB.java
@@ -0,0 +1,49 @@
+/**
+ * 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.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(
+    serverPrincipal=CommonConfigurationKeys.HADOOP_SECURITY_SERVICE_USER_NAME_KEY)
+@ProtocolInfo(
+    protocolName = "org.apache.hadoop.security.RefreshUserMappingsProtocol", 
+    protocolVersion = 1)
+@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;
+}
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/protocolPB/RefreshUserMappingsProtocolServerSideTranslatorPB.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/protocolPB/RefreshUserMappingsProtocolServerSideTranslatorPB.java
new file mode 100644
index 0000000..005c654
--- /dev/null
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/protocolPB/RefreshUserMappingsProtocolServerSideTranslatorPB.java
@@ -0,0 +1,105 @@
+/**
+ * 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.RefreshUserMappingsProtocolProtos.RefreshSuperUserGroupsConfigurationRequestProto;
+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;
+import com.google.protobuf.ServiceException;
+
+public class RefreshUserMappingsProtocolServerSideTranslatorPB implements RefreshUserMappingsProtocolPB {
+
+  private final RefreshUserMappingsProtocol impl;
+  
+  public RefreshUserMappingsProtocolServerSideTranslatorPB(RefreshUserMappingsProtocol impl) {
+    this.impl = impl;
+  }
+  
+  @Override
+  public RefreshUserToGroupsMappingsResponseProto 
+      refreshUserToGroupsMappings(RpcController controller, 
+      RefreshUserToGroupsMappingsRequestProto request)
+      throws ServiceException {
+    try {
+      impl.refreshUserToGroupsMappings();
+    } catch (IOException e) {
+      throw new ServiceException(e);
+    }
+    return RefreshUserToGroupsMappingsResponseProto.newBuilder().build();
+  }
+
+  @Override
+  public RefreshSuperUserGroupsConfigurationResponseProto 
+      refreshSuperUserGroupsConfiguration(RpcController controller,
+      RefreshSuperUserGroupsConfigurationRequestProto request)
+      throws ServiceException {
+    try {
+      impl.refreshSuperUserGroupsConfiguration();
+    } catch (IOException e) {
+      throw new ServiceException(e);
+    }
+    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));
+  }
+}
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/protocolPB/overview.html b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/protocolPB/overview.html
new file mode 100644
index 0000000..cf620f3
--- /dev/null
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/protocolPB/overview.html
@@ -0,0 +1,62 @@
+<!DOCTYPE HTML PUBLIC "-//W3C//DTD HTML 4.01 Transitional//EN">
+<html>
+<!--
+   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.
+-->
+<head>
+   <title>Protocol Buffers based data types for NN protocols</title>
+</head>
+<body>
+<p>
+The Protocol Buffers data types for NN protocols that use PB go in this package.
+</p>
+<h1>Steps to add a new protocol</h1>
+<hr/>
+<ol>
+<li>Define the protobuf service for the protocol in &lt;ProtocolName&gt;.proto class. 
+<ul>
+<li>This file should include both the protobuf service definition and the types
+ used for request and response. For example see - NamenodeProtocol.proto
+<li>The naming convention for the protobuf service is &lt;ProtocolName&gt;Service.
+Example: NamenodeProtocolService.
+<li>Every RPC method takes a request and returns a response. The request 
+naming convention is &lt;MethodName&gt;RequestProto. The response naming convention 
+is &lt;MethodName&gt;ResponseProto.
+</ul>
+<li>Generate java files from the proto file using protoc tool.
+<li>Define server side interface that extends BlockingInterface from the 
+generated files (Example: NamenodeProtocolService.BlockingInterface) 
+and VersionedProtocol. See NamenodePBProtocol.java for example.
+<li>Define client side translator to translate the client protocol to 
+protobuf. See NamenodeProtocolTranslator.
+<li>Define server side implementation that implements the server side interface.
+This implementation receives the protobuf requests and delegates it to the 
+server side implementation. See NamenodePBProtocolImpl for example.
+<li>Make changes to register this protocol at the server. See the other 
+protocols on how this is done.
+</ol>
+<h1>Steps to make changes to the existing protocol in a compatible way</h1>
+<hr/>
+<ol>
+<li>Adding new methods is a compatible change.</li>
+<li>When modifying an existing method, do not change the required parameters 
+to optional or optional parameters to required. Only add optional parameters 
+to the request and response.</li>
+<li>When modifying an existing type, do not change the required parameters 
+to optional or optional parameters to require and optional parameters to 
+required. Only add optional parameters to the request and response.</li>
+</ol>
+
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/protocolProtocolBuffers/overview.html b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/protocolProtocolBuffers/overview.html
new file mode 100644
index 0000000..e69de29
--- /dev/null
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/protocolProtocolBuffers/overview.html
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/protocolR23Compatible/ProtocolSignatureWritable.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/protocolR23Compatible/ProtocolSignatureWritable.java
new file mode 100644
index 0000000..9dc929b
--- /dev/null
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/protocolR23Compatible/ProtocolSignatureWritable.java
@@ -0,0 +1,110 @@
+/**
+ * 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.protocolR23Compatible;
+
+import java.io.DataInput;
+import java.io.DataOutput;
+import java.io.IOException;
+
+import org.apache.hadoop.classification.InterfaceAudience;
+import org.apache.hadoop.classification.InterfaceStability;
+import org.apache.hadoop.io.Writable;
+import org.apache.hadoop.io.WritableFactories;
+import org.apache.hadoop.io.WritableFactory;
+
+@InterfaceAudience.Private
+@InterfaceStability.Evolving
+public class ProtocolSignatureWritable implements Writable {
+  static {               // register a ctor
+    WritableFactories.setFactory
+      (ProtocolSignatureWritable.class,
+       new WritableFactory() {
+         public Writable newInstance() { return new ProtocolSignatureWritable(); }
+       });
+  }
+
+  private long version;
+  private int[] methods = null; // an array of method hash codes
+  
+  public static org.apache.hadoop.ipc.ProtocolSignature convert(
+      final ProtocolSignatureWritable ps) {
+    if (ps == null) return null;
+    return new org.apache.hadoop.ipc.ProtocolSignature(
+        ps.getVersion(), ps.getMethods());
+  }
+  
+  public static ProtocolSignatureWritable convert(
+      final org.apache.hadoop.ipc.ProtocolSignature ps) {
+    if (ps == null) return null;
+    return new ProtocolSignatureWritable(ps.getVersion(), ps.getMethods());
+  }
+  
+  /**
+   * default constructor
+   */
+  public ProtocolSignatureWritable() {
+  }
+  
+  /**
+   * Constructor
+   * 
+   * @param version server version
+   * @param methodHashcodes hash codes of the methods supported by server
+   */
+  public ProtocolSignatureWritable(long version, int[] methodHashcodes) {
+    this.version = version;
+    this.methods = methodHashcodes;
+  }
+  
+  public long getVersion() {
+    return version;
+  }
+  
+  public int[] getMethods() {
+    return methods;
+  }
+
+  @Override
+  public void readFields(DataInput in) throws IOException {
+    version = in.readLong();
+    boolean hasMethods = in.readBoolean();
+    if (hasMethods) {
+      int numMethods = in.readInt();
+      methods = new int[numMethods];
+      for (int i=0; i<numMethods; i++) {
+        methods[i] = in.readInt();
+      }
+    }
+  }
+
+  @Override
+  public void write(DataOutput out) throws IOException {
+    out.writeLong(version);
+    if (methods == null) {
+      out.writeBoolean(false);
+    } else {
+      out.writeBoolean(true);
+      out.writeInt(methods.length);
+      for (int method : methods) {
+        out.writeInt(method);
+      }
+    }
+  }
+}
+
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/protocolR23Compatible/overview.html b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/protocolR23Compatible/overview.html
new file mode 100644
index 0000000..7182070
--- /dev/null
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/protocolR23Compatible/overview.html
@@ -0,0 +1,44 @@
+<!DOCTYPE HTML PUBLIC "-//W3C//DTD HTML 4.01 Transitional//EN">
+<html>
+<!--
+   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.
+-->
+<head>
+   <title>Namenode Client Protocols Compatible with the version
+    of Hadoop Release 23</title>
+</head>
+<body>
+<p>
+This package is for ALL versions of HDFS protocols that use writable data types
+and are compatible with the version of the protocol that was
+ shipped with Release 23 of Hadoop.
+</p>
+
+Compatibility should be maintained:
+<ul>
+<li> Do NOT delete any methods </li>
+<li> Do NOT change the signatures of any method:
+ do not  change parameters, parameter types
+or exceptions thrown by the method.</li>
+</ul>
+<p>
+You can add new methods and new types. If you need to  change a method's
+signature, please add a new method instead.
+When you add new methods and new types do NOT change the version number.
+<p> 
+Version number is changed ONLY when compatibility is broken (which
+should be very rare and a big deal).
+</p>
\ No newline at end of file
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/security/token/block/BlockKey.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/security/token/block/BlockKey.java
index 004ff10..8a630ba 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/security/token/block/BlockKey.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/security/token/block/BlockKey.java
@@ -36,4 +36,8 @@
   public BlockKey(int keyId, long expiryDate, SecretKey key) {
     super(keyId, expiryDate, key);
   }
+  
+  public BlockKey(int keyId, long expiryDate, byte[] encodedKey) {
+    super(keyId, expiryDate, encodedKey);
+  }
 }
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/security/token/block/ExportedBlockKeys.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/security/token/block/ExportedBlockKeys.java
index 99fa94b..f317551 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/security/token/block/ExportedBlockKeys.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/security/token/block/ExportedBlockKeys.java
@@ -43,7 +43,7 @@
     this(false, 0, 0, new BlockKey(), new BlockKey[0]);
   }
 
-  ExportedBlockKeys(boolean isBlockTokenEnabled, long keyUpdateInterval,
+  public ExportedBlockKeys(boolean isBlockTokenEnabled, long keyUpdateInterval,
       long tokenLifetime, BlockKey currentKey, BlockKey[] allKeys) {
     this.isBlockTokenEnabled = isBlockTokenEnabled;
     this.keyUpdateInterval = keyUpdateInterval;
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/balancer/NameNodeConnector.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/balancer/NameNodeConnector.java
index 71b2112..83822e4 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/balancer/NameNodeConnector.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/balancer/NameNodeConnector.java
@@ -36,6 +36,8 @@
 import org.apache.hadoop.hdfs.protocol.AlreadyBeingCreatedException;
 import org.apache.hadoop.hdfs.protocol.ClientProtocol;
 import org.apache.hadoop.hdfs.protocol.ExtendedBlock;
+import org.apache.hadoop.hdfs.protocolPB.NamenodeProtocolPB;
+import org.apache.hadoop.hdfs.protocolPB.NamenodeProtocolTranslatorPB;
 import org.apache.hadoop.hdfs.security.token.block.BlockTokenIdentifier;
 import org.apache.hadoop.hdfs.security.token.block.BlockTokenSecretManager;
 import org.apache.hadoop.hdfs.security.token.block.ExportedBlockKeys;
@@ -46,6 +48,7 @@
 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.ProtobufRpcEngine;
 import org.apache.hadoop.ipc.RPC;
 import org.apache.hadoop.ipc.RemoteException;
 import org.apache.hadoop.net.NetUtils;
@@ -206,14 +209,15 @@
     methodNameToPolicyMap.put("getBlocks", methodPolicy);
     methodNameToPolicyMap.put("getAccessKeys", methodPolicy);
 
-    return (NamenodeProtocol) RetryProxy.create(NamenodeProtocol.class,
-        RPC.getProxy(NamenodeProtocol.class,
-            NamenodeProtocol.versionID,
-            address,
-            UserGroupInformation.getCurrentUser(),
-            conf,
-            NetUtils.getDefaultSocketFactory(conf)),
-        methodNameToPolicyMap);
+    RPC.setProtocolEngine(conf, NamenodeProtocolPB.class,
+        ProtobufRpcEngine.class);
+    NamenodeProtocolPB proxy = RPC.getProxy(NamenodeProtocolPB.class,
+            RPC.getProtocolVersion(NamenodeProtocolPB.class), address,
+            UserGroupInformation.getCurrentUser(), conf,
+            NetUtils.getDefaultSocketFactory(conf));
+    NamenodeProtocolPB retryProxy = (NamenodeProtocolPB) RetryProxy.create(
+        NamenodeProtocolPB.class, proxy, methodNameToPolicyMap);
+    return new NamenodeProtocolTranslatorPB(retryProxy);
   }
 
   /**
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/BlockInfoUnderConstruction.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/BlockInfoUnderConstruction.java
index 1695cab..beeb2c8 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/BlockInfoUnderConstruction.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/BlockInfoUnderConstruction.java
@@ -112,7 +112,6 @@
       return (this == obj) || super.equals(obj);
     }
 
-    /** {@inheritDoc} */
     @Override
     public String toString() {
       final StringBuilder b = new StringBuilder(getClass().getSimpleName());
@@ -268,7 +267,6 @@
     return (this == obj) || super.equals(obj);
   }
 
-  /** {@inheritDoc} */
   @Override
   public String toString() {
     final StringBuilder b = new StringBuilder(super.toString());
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/BlockManager.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/BlockManager.java
index 44411fa..282210a 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/BlockManager.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/BlockManager.java
@@ -59,10 +59,12 @@
 import org.apache.hadoop.hdfs.server.namenode.INodeFileUnderConstruction;
 import org.apache.hadoop.hdfs.server.namenode.NameNode;
 import org.apache.hadoop.hdfs.server.namenode.Namesystem;
+import org.apache.hadoop.hdfs.server.protocol.BlockCommand;
 import org.apache.hadoop.hdfs.server.protocol.BlocksWithLocations;
 import org.apache.hadoop.hdfs.server.protocol.BlocksWithLocations.BlockWithLocations;
 import org.apache.hadoop.hdfs.server.protocol.DatanodeCommand;
 import org.apache.hadoop.hdfs.server.protocol.KeyUpdateCommand;
+import org.apache.hadoop.hdfs.server.protocol.ReceivedDeletedBlockInfo;
 import org.apache.hadoop.net.Node;
 import org.apache.hadoop.util.Daemon;
 
@@ -2082,7 +2084,7 @@
    * Modify (block-->datanode) map. Possibly generate replication tasks, if the
    * removed block is still valid.
    */
-  private void removeStoredBlock(Block block, DatanodeDescriptor node) {
+  public void removeStoredBlock(Block block, DatanodeDescriptor node) {
     if(NameNode.stateChangeLog.isDebugEnabled()) {
       NameNode.stateChangeLog.debug("BLOCK* removeStoredBlock: "
           + block + " from " + node.getName());
@@ -2201,27 +2203,48 @@
     }
   }
 
-  /** The given node is reporting that it received a certain block. */
-  public void blockReceived(final DatanodeID nodeID, final String poolId,
-      final Block block, final String delHint) throws IOException {
+  /** The given node is reporting that it received/deleted certain blocks. */
+  public void blockReceivedAndDeleted(final DatanodeID nodeID, 
+     final String poolId, 
+     final ReceivedDeletedBlockInfo receivedAndDeletedBlocks[]
+  ) throws IOException {
     namesystem.writeLock();
+    int received = 0;
+    int deleted = 0;
     try {
       final DatanodeDescriptor node = datanodeManager.getDatanode(nodeID);
       if (node == null || !node.isAlive) {
-        final String s = block + " is received from dead or unregistered node "
-            + nodeID.getName();
-        NameNode.stateChangeLog.warn("BLOCK* blockReceived: " + s);
-        throw new IOException(s);
-      } 
-
-      if (NameNode.stateChangeLog.isDebugEnabled()) {
-        NameNode.stateChangeLog.debug("BLOCK* blockReceived: " + block
-            + " is received from " + nodeID.getName());
+        NameNode.stateChangeLog
+            .warn("BLOCK* blockReceivedDeleted"
+                + " is received from dead or unregistered node "
+                + nodeID.getName());
+        throw new IOException(
+            "Got blockReceivedDeleted message from unregistered or dead node");
       }
 
-      addBlock(node, block, delHint);
+      for (int i = 0; i < receivedAndDeletedBlocks.length; i++) {
+        if (receivedAndDeletedBlocks[i].isDeletedBlock()) {
+          removeStoredBlock(
+              receivedAndDeletedBlocks[i].getBlock(), node);
+          deleted++;
+        } else {
+          addBlock(node, receivedAndDeletedBlocks[i].getBlock(),
+              receivedAndDeletedBlocks[i].getDelHints());
+          received++;
+        }
+        if (NameNode.stateChangeLog.isDebugEnabled()) {
+          NameNode.stateChangeLog.debug("BLOCK* block"
+              + (receivedAndDeletedBlocks[i].isDeletedBlock() ? "Deleted"
+                  : "Received") + ": " + receivedAndDeletedBlocks[i].getBlock()
+              + " is received from " + nodeID.getName());
+        }
+      }
     } finally {
       namesystem.writeUnlock();
+      NameNode.stateChangeLog
+          .debug("*BLOCK* NameNode.blockReceivedAndDeleted: " + "from "
+              + nodeID.getName() + " received: " + received + ", "
+              + " deleted: " + deleted);
     }
   }
 
@@ -2396,6 +2419,7 @@
   }
 
   public void removeBlock(Block block) {
+    block.setNumBytes(BlockCommand.NO_ACK);
     addToInvalidates(block);
     corruptReplicas.removeFromCorruptReplicasMap(block);
     blocksMap.removeBlock(block);
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/BlockPlacementPolicyDefault.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/BlockPlacementPolicyDefault.java
index d40aa52..ac1a7e6 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/BlockPlacementPolicyDefault.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/BlockPlacementPolicyDefault.java
@@ -66,7 +66,7 @@
   BlockPlacementPolicyDefault() {
   }
     
-  /** {@inheritDoc} */
+  @Override
   public void initialize(Configuration conf,  FSClusterStats stats,
                          NetworkTopology clusterMap) {
     this.considerLoad = conf.getBoolean(DFSConfigKeys.DFS_NAMENODE_REPLICATION_CONSIDERLOAD_KEY, true);
@@ -82,7 +82,7 @@
     }
   };
 
-  /** {@inheritDoc} */
+  @Override
   public DatanodeDescriptor[] chooseTarget(String srcPath,
                                     int numOfReplicas,
                                     DatanodeDescriptor writer,
@@ -92,7 +92,6 @@
         null, blocksize);
   }
 
-  /** {@inheritDoc} */
   @Override
   public DatanodeDescriptor[] chooseTarget(String srcPath,
                                     int numOfReplicas,
@@ -528,7 +527,7 @@
     return nodes;
   }
 
-  /** {@inheritDoc} */
+  @Override
   public int verifyBlockPlacement(String srcPath,
                                   LocatedBlock lBlk,
                                   int minRacks) {
@@ -547,7 +546,7 @@
     return minRacks - racks.size();
   }
 
-  /** {@inheritDoc} */
+  @Override
   public DatanodeDescriptor chooseReplicaToDelete(FSInodeInfo inode,
                                                  Block block,
                                                  short replicationFactor,
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/DatanodeManager.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/DatanodeManager.java
index e11355a..5d795e7 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/DatanodeManager.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/DatanodeManager.java
@@ -60,6 +60,7 @@
 import org.apache.hadoop.hdfs.server.protocol.DatanodeProtocol;
 import org.apache.hadoop.hdfs.server.protocol.DatanodeRegistration;
 import org.apache.hadoop.hdfs.server.protocol.DisallowedDatanodeException;
+import org.apache.hadoop.hdfs.server.protocol.RegisterCommand;
 import org.apache.hadoop.hdfs.util.CyclicIteration;
 import org.apache.hadoop.ipc.Server;
 import org.apache.hadoop.net.CachedDNSToSwitchMapping;
@@ -862,7 +863,7 @@
         try {
           nodeinfo = getDatanode(nodeReg);
         } catch(UnregisteredNodeException e) {
-          return new DatanodeCommand[]{DatanodeCommand.REGISTER};
+          return new DatanodeCommand[]{RegisterCommand.REGISTER};
         }
         
         // Check if this datanode should actually be shutdown instead. 
@@ -872,7 +873,7 @@
         }
          
         if (nodeinfo == null || !nodeinfo.isAlive) {
-          return new DatanodeCommand[]{DatanodeCommand.REGISTER};
+          return new DatanodeCommand[]{RegisterCommand.REGISTER};
         }
 
         heartbeatManager.updateHeartbeat(nodeinfo, capacity, dfsUsed,
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/BPOfferService.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/BPOfferService.java
index f75590a..ab37e49 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/BPOfferService.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/BPOfferService.java
@@ -36,6 +36,7 @@
 import org.apache.hadoop.hdfs.protocol.HdfsConstants;
 import org.apache.hadoop.hdfs.protocol.LocatedBlock;
 import org.apache.hadoop.hdfs.protocol.UnregisteredNodeException;
+import org.apache.hadoop.hdfs.protocolPB.DatanodeProtocolClientSideTranslatorPB;
 import org.apache.hadoop.hdfs.server.common.IncorrectVersionException;
 import org.apache.hadoop.hdfs.server.common.Storage;
 import org.apache.hadoop.hdfs.server.namenode.FSNamesystem;
@@ -45,11 +46,17 @@
 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.DisallowedDatanodeException;
+import org.apache.hadoop.hdfs.server.protocol.FinalizeCommand;
 import org.apache.hadoop.hdfs.server.protocol.KeyUpdateCommand;
 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.RPC;
+import org.apache.hadoop.io.IOUtils;
 import org.apache.hadoop.ipc.RemoteException;
 import org.apache.hadoop.util.StringUtils;
 
@@ -86,15 +93,17 @@
   DatanodeRegistration bpRegistration;
   
   long lastBlockReport = 0;
+  long lastDeletedReport = 0;
 
   boolean resetBlockReportTime = true;
 
   Thread bpThread;
-  DatanodeProtocol bpNamenode;
+  DatanodeProtocolClientSideTranslatorPB bpNamenode;
   private long lastHeartbeat = 0;
   private volatile boolean initialized = false;
-  private final LinkedList<Block> receivedBlockList = new LinkedList<Block>();
-  private final LinkedList<String> delHints = new LinkedList<String>();
+  private final LinkedList<ReceivedDeletedBlockInfo> receivedAndDeletedBlockList 
+  	= new LinkedList<ReceivedDeletedBlockInfo>();
+  private volatile int pendingReceivedRequests = 0;
   private volatile boolean shouldServiceRun = true;
   UpgradeManagerDatanode upgradeManager = null;
   private final DataNode dn;
@@ -160,7 +169,7 @@
    * Used to inject a spy NN in the unit tests.
    */
   @VisibleForTesting
-  void setNameNode(DatanodeProtocol dnProtocol) {
+  void setNameNode(DatanodeProtocolClientSideTranslatorPB dnProtocol) {
     bpNamenode = dnProtocol;
   }
 
@@ -220,8 +229,8 @@
 
   private void connectToNNAndHandshake() throws IOException {
     // get NN proxy
-    bpNamenode = (DatanodeProtocol)RPC.waitForProxy(DatanodeProtocol.class,
-          DatanodeProtocol.versionID, nnAddr, dn.getConf());
+    bpNamenode = new DatanodeProtocolClientSideTranslatorPB(nnAddr,
+        dn.getConf());
 
     // First phase of the handshake with NN - get the namespace
     // info.
@@ -270,39 +279,32 @@
    * Report received blocks and delete hints to the Namenode
    * @throws IOException
    */
-  private void reportReceivedBlocks() throws IOException {
-    //check if there are newly received blocks
-    Block [] blockArray=null;
-    String [] delHintArray=null;
-    synchronized(receivedBlockList) {
-      synchronized(delHints){
-        int numBlocks = receivedBlockList.size();
-        if (numBlocks > 0) {
-          if(numBlocks!=delHints.size()) {
-            LOG.warn("Panic: receiveBlockList and delHints are not of " +
-            "the same length" );
-          }
-          //
-          // Send newly-received blockids to namenode
-          //
-          blockArray = receivedBlockList.toArray(new Block[numBlocks]);
-          delHintArray = delHints.toArray(new String[numBlocks]);
-        }
+  private void reportReceivedDeletedBlocks() throws IOException {
+
+    // check if there are newly received blocks
+    ReceivedDeletedBlockInfo[] receivedAndDeletedBlockArray = null;
+    int currentReceivedRequestsCounter;
+    synchronized (receivedAndDeletedBlockList) {
+      currentReceivedRequestsCounter = pendingReceivedRequests;
+      int numBlocks = receivedAndDeletedBlockList.size();
+      if (numBlocks > 0) {
+        //
+        // Send newly-received and deleted blockids to namenode
+        //
+        receivedAndDeletedBlockArray = receivedAndDeletedBlockList
+            .toArray(new ReceivedDeletedBlockInfo[numBlocks]);
       }
     }
-    if (blockArray != null) {
-      if(delHintArray == null || delHintArray.length != blockArray.length ) {
-        LOG.warn("Panic: block array & delHintArray are not the same" );
-      }
-      bpNamenode.blockReceived(bpRegistration, getBlockPoolId(), blockArray,
-          delHintArray);
-      synchronized(receivedBlockList) {
-        synchronized(delHints){
-          for(int i=0; i<blockArray.length; i++) {
-            receivedBlockList.remove(blockArray[i]);
-            delHints.remove(delHintArray[i]);
-          }
+    if (receivedAndDeletedBlockArray != null) {
+      StorageReceivedDeletedBlocks[] report = { new StorageReceivedDeletedBlocks(
+          bpRegistration.getStorageID(), receivedAndDeletedBlockArray) };
+      bpNamenode.blockReceivedAndDeleted(bpRegistration, getBlockPoolId(),
+          report);
+      synchronized (receivedAndDeletedBlockList) {
+        for (int i = 0; i < receivedAndDeletedBlockArray.length; i++) {
+          receivedAndDeletedBlockList.remove(receivedAndDeletedBlockArray[i]);
         }
+        pendingReceivedRequests -= currentReceivedRequestsCounter;
       }
     }
   }
@@ -313,26 +315,42 @@
    * client? For now we don't.
    */
   void notifyNamenodeReceivedBlock(ExtendedBlock block, String delHint) {
-    if(block==null || delHint==null) {
-      throw new IllegalArgumentException(
-          block==null?"Block is null":"delHint is null");
+    if (block == null || delHint == null) {
+      throw new IllegalArgumentException(block == null ? "Block is null"
+          : "delHint is null");
     }
-    
+
     if (!block.getBlockPoolId().equals(getBlockPoolId())) {
       LOG.warn("BlockPool mismatch " + block.getBlockPoolId() + " vs. "
           + getBlockPoolId());
       return;
     }
-    
-    synchronized (receivedBlockList) {
-      synchronized (delHints) {
-        receivedBlockList.add(block.getLocalBlock());
-        delHints.add(delHint);
-        receivedBlockList.notifyAll();
-      }
+
+    synchronized (receivedAndDeletedBlockList) {
+      receivedAndDeletedBlockList.add(new ReceivedDeletedBlockInfo(block
+          .getLocalBlock(), delHint));
+      pendingReceivedRequests++;
+      receivedAndDeletedBlockList.notifyAll();
     }
   }
 
+  void notifyNamenodeDeletedBlock(ExtendedBlock block) {
+    if (block == null) {
+      throw new IllegalArgumentException("Block is null");
+    }
+
+    if (!block.getBlockPoolId().equals(getBlockPoolId())) {
+      LOG.warn("BlockPool mismatch " + block.getBlockPoolId() + " vs. "
+          + getBlockPoolId());
+      return;
+    }
+
+    synchronized (receivedAndDeletedBlockList) {
+      receivedAndDeletedBlockList.add(new ReceivedDeletedBlockInfo(block
+          .getLocalBlock(), ReceivedDeletedBlockInfo.TODELETE_HINT));
+    }
+  }
+  
 
   /**
    * Report the list blocks to the Namenode
@@ -350,8 +368,9 @@
 
       // Send block report
       long brSendStartTime = now();
-      cmd = bpNamenode.blockReport(bpRegistration, getBlockPoolId(), bReport
-          .getBlockListAsLongs());
+      StorageBlockReport[] report = { new StorageBlockReport(
+          bpRegistration.getStorageID(), bReport.getBlockListAsLongs()) };
+      cmd = bpNamenode.blockReport(bpRegistration, getBlockPoolId(), report);
 
       // Log the block report processing stats from Datanode perspective
       long brSendCost = now() - brSendStartTime;
@@ -383,11 +402,11 @@
   
   
   DatanodeCommand [] sendHeartBeat() throws IOException {
-    return bpNamenode.sendHeartbeat(bpRegistration,
-        dn.data.getCapacity(),
-        dn.data.getDfsUsed(),
-        dn.data.getRemaining(),
-        dn.data.getBlockPoolUsed(getBlockPoolId()),
+    // reports number of failed volumes
+    StorageReport[] report = { new StorageReport(bpRegistration.getStorageID(),
+        false, dn.data.getCapacity(), dn.data.getDfsUsed(),
+        dn.data.getRemaining(), dn.data.getBlockPoolUsed(getBlockPoolId())) };
+    return bpNamenode.sendHeartbeat(bpRegistration, report,
         dn.xmitsInProgress.get(),
         dn.getXceiverCount(), dn.data.getNumFailedVolumes());
   }
@@ -433,7 +452,7 @@
     if(upgradeManager != null)
       upgradeManager.shutdownUpgrade();
     shouldServiceRun = false;
-    RPC.stopProxy(bpNamenode);
+    IOUtils.cleanup(LOG, bpNamenode);
     dn.shutdownBlockPool(this);
   }
 
@@ -442,7 +461,8 @@
    * forever calling remote NameNode functions.
    */
   private void offerService() throws Exception {
-    LOG.info("For namenode " + nnAddr + " using BLOCKREPORT_INTERVAL of "
+    LOG.info("For namenode " + nnAddr + " using DELETEREPORT_INTERVAL of "
+        + dnConf.deleteReportInterval + " msec " + " BLOCKREPORT_INTERVAL of "
         + dnConf.blockReportInterval + "msec" + " Initial delay: "
         + dnConf.initialBlockReportDelay + "msec" + "; heartBeatInterval="
         + dnConf.heartBeatInterval);
@@ -480,8 +500,11 @@
             }
           }
         }
-
-        reportReceivedBlocks();
+        if (pendingReceivedRequests > 0
+            || (startTime - lastDeletedReport > dnConf.deleteReportInterval)) {
+          reportReceivedDeletedBlocks();
+          lastDeletedReport = startTime;
+        }
 
         DatanodeCommand cmd = blockReport();
         processCommand(cmd);
@@ -497,10 +520,10 @@
         //
         long waitTime = dnConf.heartBeatInterval - 
         (System.currentTimeMillis() - lastHeartbeat);
-        synchronized(receivedBlockList) {
-          if (waitTime > 0 && receivedBlockList.size() == 0) {
+        synchronized(receivedAndDeletedBlockList) {
+          if (waitTime > 0 && pendingReceivedRequests == 0) {
             try {
-              receivedBlockList.wait(waitTime);
+            	receivedAndDeletedBlockList.wait(waitTime);
             } catch (InterruptedException ie) {
               LOG.warn("BPOfferService for " + this + " interrupted");
             }
@@ -553,7 +576,8 @@
     while (shouldRun()) {
       try {
         // Use returned registration from namenode with updated machine name.
-        bpRegistration = bpNamenode.registerDatanode(bpRegistration);
+        bpRegistration = bpNamenode.registerDatanode(bpRegistration,
+            new DatanodeStorage[0]);
         break;
       } catch(SocketTimeoutException e) {  // namenode is busy
         LOG.info("Problem connecting to server: " + nnAddr);
@@ -699,7 +723,7 @@
       }
       break;
     case DatanodeProtocol.DNA_FINALIZE:
-      String bp = ((DatanodeCommand.Finalize) cmd).getBlockPoolId(); 
+      String bp = ((FinalizeCommand) cmd).getBlockPoolId(); 
       assert getBlockPoolId().equals(bp) :
         "BP " + getBlockPoolId() + " received DNA_FINALIZE " +
         "for other block pool " + bp;
@@ -764,12 +788,12 @@
   }
 
   @VisibleForTesting
-  DatanodeProtocol getBpNamenode() {
+  DatanodeProtocolClientSideTranslatorPB getBpNamenode() {
     return bpNamenode;
   }
 
   @VisibleForTesting
-  void setBpNamenode(DatanodeProtocol bpNamenode) {
+  void setBpNamenode(DatanodeProtocolClientSideTranslatorPB bpNamenode) {
     this.bpNamenode = bpNamenode;
   }
 }
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/DNConf.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/DNConf.java
index f0f4737..e4bf9a6 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/DNConf.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/DNConf.java
@@ -55,6 +55,7 @@
   final long readaheadLength;
   final long heartBeatInterval;
   final long blockReportInterval;
+  final long deleteReportInterval;
   final long initialBlockReportDelay;
   final int writePacketSize;
 
@@ -105,6 +106,7 @@
     heartBeatInterval = conf.getLong(DFS_HEARTBEAT_INTERVAL_KEY,
         DFS_HEARTBEAT_INTERVAL_DEFAULT) * 1000L;
     
+    this.deleteReportInterval = 100 * heartBeatInterval;
     // do we need to sync block file contents to disk when blockfile is closed?
     this.syncOnClose = conf.getBoolean(DFS_DATANODE_SYNCONCLOSE_KEY, 
         DFS_DATANODE_SYNCONCLOSE_DEFAULT);
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/DataNode.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/DataNode.java
index 888a036..7c8202e 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/DataNode.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/DataNode.java
@@ -100,8 +100,16 @@
 import org.apache.hadoop.hdfs.protocol.datatransfer.BlockConstructionStage;
 import org.apache.hadoop.hdfs.protocol.datatransfer.DataTransferProtocol;
 import org.apache.hadoop.hdfs.protocol.datatransfer.Sender;
+import org.apache.hadoop.hdfs.protocol.proto.ClientDatanodeProtocolProtos.ClientDatanodeProtocolService;
 import org.apache.hadoop.hdfs.protocol.proto.DataTransferProtos.DNTransferAckProto;
 import org.apache.hadoop.hdfs.protocol.proto.DataTransferProtos.Status;
+import org.apache.hadoop.hdfs.protocol.proto.InterDatanodeProtocolProtos.InterDatanodeProtocolService;
+import org.apache.hadoop.hdfs.protocolPB.ClientDatanodeProtocolPB;
+import org.apache.hadoop.hdfs.protocolPB.ClientDatanodeProtocolServerSideTranslatorPB;
+import org.apache.hadoop.hdfs.protocolPB.DatanodeProtocolClientSideTranslatorPB;
+import org.apache.hadoop.hdfs.protocolPB.InterDatanodeProtocolPB;
+import org.apache.hadoop.hdfs.protocolPB.InterDatanodeProtocolServerSideTranslatorPB;
+import org.apache.hadoop.hdfs.protocolPB.InterDatanodeProtocolTranslatorPB;
 import org.apache.hadoop.hdfs.security.token.block.BlockPoolTokenSecretManager;
 import org.apache.hadoop.hdfs.security.token.block.BlockTokenIdentifier;
 import org.apache.hadoop.hdfs.security.token.block.BlockTokenSecretManager;
@@ -124,11 +132,13 @@
 import org.apache.hadoop.hdfs.server.protocol.DatanodeRegistration;
 import org.apache.hadoop.hdfs.server.protocol.InterDatanodeProtocol;
 import org.apache.hadoop.hdfs.server.protocol.NamespaceInfo;
+import org.apache.hadoop.hdfs.server.protocol.ReceivedDeletedBlockInfo;
 import org.apache.hadoop.hdfs.server.protocol.ReplicaRecoveryInfo;
 import org.apache.hadoop.hdfs.web.WebHdfsFileSystem;
 import org.apache.hadoop.hdfs.web.resources.Param;
 import org.apache.hadoop.http.HttpServer;
 import org.apache.hadoop.io.IOUtils;
+import org.apache.hadoop.ipc.ProtobufRpcEngine;
 import org.apache.hadoop.ipc.ProtocolSignature;
 import org.apache.hadoop.ipc.RPC;
 import org.apache.hadoop.ipc.RemoteException;
@@ -154,6 +164,7 @@
 import org.apache.hadoop.util.VersionInfo;
 import org.mortbay.util.ajax.JSON;
 
+import com.google.protobuf.BlockingService;
 
 
 /**********************************************************
@@ -385,7 +396,7 @@
   private List<ServicePlugin> plugins;
   
   // For InterDataNodeProtocol
-  public Server ipcServer;
+  public RPC.Server ipcServer;
 
   private SecureResources secureResources = null;
   private AbstractList<File> dataDirs;
@@ -507,11 +518,26 @@
   private void initIpcServer(Configuration conf) throws IOException {
     InetSocketAddress ipcAddr = NetUtils.createSocketAddr(
         conf.get("dfs.datanode.ipc.address"));
-    ipcServer = RPC.getServer(DataNode.class, this, ipcAddr.getHostName(),
-                              ipcAddr.getPort(), 
-                              conf.getInt(DFS_DATANODE_HANDLER_COUNT_KEY, 
-                                          DFS_DATANODE_HANDLER_COUNT_DEFAULT), 
-                              false, conf, blockPoolTokenSecretManager);
+    
+    // Add all the RPC protocols that the Datanode implements    
+    RPC.setProtocolEngine(conf, ClientDatanodeProtocolPB.class,
+        ProtobufRpcEngine.class);
+    ClientDatanodeProtocolServerSideTranslatorPB clientDatanodeProtocolXlator = 
+          new ClientDatanodeProtocolServerSideTranslatorPB(this);
+    BlockingService service = ClientDatanodeProtocolService
+        .newReflectiveBlockingService(clientDatanodeProtocolXlator);
+    ipcServer = RPC.getServer(ClientDatanodeProtocolPB.class, service, ipcAddr
+        .getHostName(), ipcAddr.getPort(), conf.getInt(
+        DFS_DATANODE_HANDLER_COUNT_KEY, DFS_DATANODE_HANDLER_COUNT_DEFAULT),
+        false, conf, blockPoolTokenSecretManager);
+    
+    InterDatanodeProtocolServerSideTranslatorPB interDatanodeProtocolXlator = 
+        new InterDatanodeProtocolServerSideTranslatorPB(this);
+    service = InterDatanodeProtocolService
+        .newReflectiveBlockingService(interDatanodeProtocolXlator);
+    DFSUtil.addPBProtocol(conf, InterDatanodeProtocolPB.class, service,
+        ipcServer);
+    
     // set service-level authorization security policy
     if (conf.getBoolean(
         CommonConfigurationKeys.HADOOP_SECURITY_AUTHORIZATION, false)) {
@@ -634,6 +660,17 @@
     }
   }
   
+  // calls specific to BP
+  protected void notifyNamenodeDeletedBlock(ExtendedBlock block) {
+    BPOfferService bpos = blockPoolManager.get(block.getBlockPoolId());
+    if (bpos != null) {
+      bpos.notifyNamenodeDeletedBlock(block);
+    } else {
+      LOG.warn("Cannot find BPOfferService for reporting block deleted for bpid="
+          + block.getBlockPoolId());
+    }
+  }
+  
   public void reportBadBlocks(ExtendedBlock block) throws IOException{
     BPOfferService bpos = blockPoolManager.get(block.getBlockPoolId());
     if(bpos == null || bpos.bpNamenode == null) {
@@ -952,15 +989,13 @@
     if (InterDatanodeProtocol.LOG.isDebugEnabled()) {
       InterDatanodeProtocol.LOG.debug("InterDatanodeProtocol addr=" + addr);
     }
-    UserGroupInformation loginUgi = UserGroupInformation.getLoginUser();
+    final UserGroupInformation loginUgi = UserGroupInformation.getLoginUser();
     try {
       return loginUgi
           .doAs(new PrivilegedExceptionAction<InterDatanodeProtocol>() {
             public InterDatanodeProtocol run() throws IOException {
-              return (InterDatanodeProtocol) RPC.getProxy(
-                  InterDatanodeProtocol.class, InterDatanodeProtocol.versionID,
-                  addr, UserGroupInformation.getCurrentUser(), conf,
-                  NetUtils.getDefaultSocketFactory(conf), socketTimeout);
+              return new InterDatanodeProtocolTranslatorPB(addr, loginUgi,
+                  conf, NetUtils.getDefaultSocketFactory(conf), socketTimeout);
             }
           });
     } catch (InterruptedException ie) {
@@ -1207,7 +1242,7 @@
 
     //inform NameNodes
     for(BPOfferService bpos: blockPoolManager.getAllNamenodeThreads()) {
-      DatanodeProtocol nn = bpos.bpNamenode;
+      DatanodeProtocolClientSideTranslatorPB nn = bpos.bpNamenode;
       try {
         nn.errorReport(bpos.bpRegistration, dpError, errMsgr);
       } catch(IOException e) {
@@ -1241,7 +1276,8 @@
   private void transferBlock( ExtendedBlock block, 
                               DatanodeInfo xferTargets[] 
                               ) throws IOException {
-    DatanodeProtocol nn = getBPNamenode(block.getBlockPoolId());
+    DatanodeProtocolClientSideTranslatorPB nn = getBPNamenode(block
+        .getBlockPoolId());
     DatanodeRegistration bpReg = getDNRegistrationForBP(block.getBlockPoolId());
     
     if (!data.isValidBlock(block)) {
@@ -1819,7 +1855,7 @@
     return new ExtendedBlock(oldBlock.getBlockPoolId(), r);
   }
 
-  /** {@inheritDoc} */
+  @Override
   public long getProtocolVersion(String protocol, long clientVersion
       ) throws IOException {
     if (protocol.equals(InterDatanodeProtocol.class.getName())) {
@@ -1852,7 +1888,7 @@
       this.rInfo = rInfo;
     }
 
-    /** {@inheritDoc} */
+    @Override
     public String toString() {
       return "block:" + rInfo + " node:" + id;
     }
@@ -1909,7 +1945,8 @@
    * @return Namenode corresponding to the bpid
    * @throws IOException
    */
-  public DatanodeProtocol getBPNamenode(String bpid) throws IOException {
+  public DatanodeProtocolClientSideTranslatorPB getBPNamenode(String bpid)
+      throws IOException {
     BPOfferService bpos = blockPoolManager.get(bpid);
     if(bpos == null || bpos.bpNamenode == null) {
       throw new IOException("cannot find a namnode proxy for bpid=" + bpid);
@@ -1921,7 +1958,8 @@
   void syncBlock(RecoveringBlock rBlock,
                          List<BlockRecord> syncList) throws IOException {
     ExtendedBlock block = rBlock.getBlock();
-    DatanodeProtocol nn = getBPNamenode(block.getBlockPoolId());
+    DatanodeProtocolClientSideTranslatorPB nn = getBPNamenode(block
+        .getBlockPoolId());
     
     long recoveryId = rBlock.getNewGenerationStamp();
     if (LOG.isDebugEnabled()) {
@@ -2036,7 +2074,6 @@
   }
 
   // ClientDataNodeProtocol implementation
-  /** {@inheritDoc} */
   @Override // ClientDataNodeProtocol
   public long getReplicaVisibleLength(final ExtendedBlock block) throws IOException {
     checkWriteAccess(block);
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/FSDataset.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/FSDataset.java
index 5927217..dd5e4d1 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/FSDataset.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/FSDataset.java
@@ -59,7 +59,6 @@
 import org.apache.hadoop.hdfs.server.common.HdfsServerConstants.ReplicaState;
 import org.apache.hadoop.hdfs.server.datanode.metrics.FSDatasetMBean;
 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.io.IOUtils;
 import org.apache.hadoop.metrics2.util.MBeans;
@@ -1122,7 +1121,7 @@
     for (int idx = 0; idx < storage.getNumStorageDirs(); idx++) {
       roots[idx] = storage.getStorageDir(idx).getCurrentDir();
     }
-    asyncDiskService = new FSDatasetAsyncDiskService(roots);
+    asyncDiskService = new FSDatasetAsyncDiskService(this, roots);
     registerMBean(storage.getStorageID());
   }
 
@@ -1202,8 +1201,8 @@
   File getBlockFile(String bpid, Block b) throws IOException {
     File f = validateBlockFile(bpid, b);
     if(f == null) {
-      if (InterDatanodeProtocol.LOG.isDebugEnabled()) {
-        InterDatanodeProtocol.LOG.debug("b=" + b + ", volumeMap=" + volumeMap);
+      if (DataNode.LOG.isDebugEnabled()) {
+        DataNode.LOG.debug("b=" + b + ", volumeMap=" + volumeMap);
       }
       throw new IOException("Block " + b + " is not valid.");
     }
@@ -1964,8 +1963,8 @@
       datanode.checkDiskError();
     }
     
-    if (InterDatanodeProtocol.LOG.isDebugEnabled()) {
-      InterDatanodeProtocol.LOG.debug("b=" + b + ", f=" + f);
+    if (DataNode.LOG.isDebugEnabled()) {
+      DataNode.LOG.debug("b=" + b + ", f=" + f);
     }
     return null;
   }
@@ -2049,15 +2048,19 @@
         volumeMap.remove(bpid, invalidBlks[i]);
       }
       File metaFile = DatanodeUtil.getMetaFile(f, invalidBlks[i].getGenerationStamp());
-      
+
       // Delete the block asynchronously to make sure we can do it fast enough
-      asyncDiskService.deleteAsync(v, bpid, f, metaFile,
-          invalidBlks[i].toString());
+      asyncDiskService.deleteAsync(v, f, metaFile,
+          new ExtendedBlock(bpid, invalidBlks[i]));
     }
     if (error) {
       throw new IOException("Error in deleting blocks.");
     }
   }
+  
+  public void notifyNamenodeDeletedBlock(ExtendedBlock block){
+    datanode.notifyNamenodeDeletedBlock(block);
+  }
 
   @Override // {@link FSDatasetInterface}
   public synchronized boolean contains(final ExtendedBlock block) {
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/FSDatasetAsyncDiskService.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/FSDatasetAsyncDiskService.java
index 0c2523b..408a6af 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/FSDatasetAsyncDiskService.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/FSDatasetAsyncDiskService.java
@@ -28,6 +28,8 @@
 
 import org.apache.commons.logging.Log;
 import org.apache.commons.logging.LogFactory;
+import org.apache.hadoop.hdfs.protocol.ExtendedBlock;
+import org.apache.hadoop.hdfs.server.protocol.BlockCommand;
 
 /*
  * This class is a container of multiple thread pools, each for a volume,
@@ -47,6 +49,8 @@
  */
 class FSDatasetAsyncDiskService {
   
+  final FSDataset dataset;
+  
   public static final Log LOG = LogFactory.getLog(FSDatasetAsyncDiskService.class);
   
   // ThreadPool core pool size
@@ -70,8 +74,8 @@
    * 
    * @param volumes The roots of the data volumes.
    */
-  FSDatasetAsyncDiskService(File[] volumes) {
-
+  FSDatasetAsyncDiskService(FSDataset dataset, File[] volumes) {
+    this.dataset = dataset;
     // Create one ThreadPool per volume
     for (int v = 0 ; v < volumes.length; v++) {
       final File vol = volumes[v];
@@ -147,13 +151,12 @@
    * Delete the block file and meta file from the disk asynchronously, adjust
    * dfsUsed statistics accordingly.
    */
-  void deleteAsync(FSDataset.FSVolume volume, String bpid, File blockFile,
-      File metaFile, String blockName) {
-    DataNode.LOG.info("Scheduling block " + blockName + " file " + blockFile
-        + " for deletion");
-    ReplicaFileDeleteTask deletionTask = 
-        new ReplicaFileDeleteTask(volume, bpid, blockFile, metaFile,
-            blockName);
+  void deleteAsync(FSDataset.FSVolume volume, File blockFile, File metaFile,
+      ExtendedBlock block) {
+    DataNode.LOG.info("Scheduling block " + block.getLocalBlock().toString()
+        + " file " + blockFile + " for deletion");
+    ReplicaFileDeleteTask deletionTask = new ReplicaFileDeleteTask(dataset,
+        volume, blockFile, metaFile, block);
     execute(volume.getCurrentDir(), deletionTask);
   }
   
@@ -161,19 +164,19 @@
    *  as decrement the dfs usage of the volume. 
    */
   static class ReplicaFileDeleteTask implements Runnable {
+    final FSDataset dataset;
     final FSDataset.FSVolume volume;
-    final String blockPoolId;
     final File blockFile;
     final File metaFile;
-    final String blockName;
+    final ExtendedBlock block;
     
-    ReplicaFileDeleteTask(FSDataset.FSVolume volume, String bpid,
-        File blockFile, File metaFile, String blockName) {
+    ReplicaFileDeleteTask(FSDataset dataset, FSDataset.FSVolume volume, File blockFile,
+        File metaFile, ExtendedBlock block) {
+      this.dataset = dataset;
       this.volume = volume;
-      this.blockPoolId = bpid;
       this.blockFile = blockFile;
       this.metaFile = metaFile;
-      this.blockName = blockName;
+      this.block = block;
     }
     
     FSDataset.FSVolume getVolume() {
@@ -183,9 +186,9 @@
     @Override
     public String toString() {
       // Called in AsyncDiskService.execute for displaying error messages.
-      return "deletion of block " + blockPoolId + " " + blockName
-          + " with block file " + blockFile + " and meta file " + metaFile
-          + " from volume " + volume;
+      return "deletion of block " + block.getBlockPoolId() + " "
+          + block.getLocalBlock().toString() + " with block file " + blockFile
+          + " and meta file " + metaFile + " from volume " + volume;
     }
 
     @Override
@@ -193,12 +196,15 @@
       long dfsBytes = blockFile.length() + metaFile.length();
       if ( !blockFile.delete() || ( !metaFile.delete() && metaFile.exists() ) ) {
         DataNode.LOG.warn("Unexpected error trying to delete block "
-            + blockPoolId + " " + blockName + " at file " + blockFile
-            + ". Ignored.");
+            + block.getBlockPoolId() + " " + block.getLocalBlock().toString()
+            + " at file " + blockFile + ". Ignored.");
       } else {
-        volume.decDfsUsed(blockPoolId, dfsBytes);
-        DataNode.LOG.info("Deleted block " + blockPoolId + " " + blockName
-            + " at file " + blockFile);
+        if(block.getLocalBlock().getNumBytes() != BlockCommand.NO_ACK){
+          dataset.notifyNamenodeDeletedBlock(block);
+        }
+        volume.decDfsUsed(block.getBlockPoolId(), dfsBytes);
+        DataNode.LOG.info("Deleted block " + block.getBlockPoolId() + " "
+            + block.getLocalBlock().toString() + " at file " + blockFile);
       }
     }
   };
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/FSDatasetInterface.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/FSDatasetInterface.java
index 48618cd..2487ca6 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/FSDatasetInterface.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/FSDatasetInterface.java
@@ -257,7 +257,7 @@
       this.checksumIn = checksumIn;
     }
 
-    /** {@inheritDoc} */
+    @Override
     public void close() {
       IOUtils.closeStream(dataIn);
       IOUtils.closeStream(checksumIn);
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/BackupImage.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/BackupImage.java
index fae4f8d..e6a093a 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/BackupImage.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/BackupImage.java
@@ -18,18 +18,21 @@
 package org.apache.hadoop.hdfs.server.namenode;
 
 import java.io.IOException;
+import java.util.Collection;
 import java.util.Iterator;
+import java.util.List;
+import java.util.zip.Checksum;
+
 import org.apache.hadoop.classification.InterfaceAudience;
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.hdfs.server.common.HdfsServerConstants;
 import org.apache.hadoop.hdfs.server.common.InconsistentFSStateException;
 import org.apache.hadoop.hdfs.server.common.Storage.StorageDirectory;
 import org.apache.hadoop.hdfs.server.common.Storage.StorageState;
-import org.apache.hadoop.hdfs.server.namenode.FSImageTransactionalStorageInspector.LogLoadPlan;
-import org.apache.hadoop.io.IOUtils;
 import org.apache.hadoop.util.StringUtils;
 
 import com.google.common.base.Preconditions;
+import com.google.common.collect.Lists;
 
 /**
  * Extension of FSImage for the backup node.
@@ -77,6 +80,8 @@
    * {@see #freezeNamespaceAtNextRoll()}
    */
   private boolean stopApplyingEditsOnNextRoll = false;
+  
+  private FSNamesystem namesystem;
 
   /**
    * Construct a backup image.
@@ -88,6 +93,10 @@
     storage.setDisablePreUpgradableLayoutCheck(true);
     bnState = BNState.DROP_UNTIL_NEXT_ROLL;
   }
+  
+  void setNamesystem(FSNamesystem fsn) {
+    this.namesystem = fsn;
+  }
 
   /**
    * Analyze backup storage directories for consistency.<br>
@@ -136,7 +145,7 @@
    * and create empty edits.
    */
   void saveCheckpoint() throws IOException {
-    saveNamespace();
+    saveNamespace(namesystem);
   }
 
   /**
@@ -219,7 +228,7 @@
       }
       lastAppliedTxId += numTxns;
       
-      getFSNamesystem().dir.updateCountForINodeWithQuota(); // inefficient!
+      namesystem.dir.updateCountForINodeWithQuota(); // inefficient!
     } finally {
       backupInputStream.clear();
     }
@@ -257,11 +266,18 @@
         new FSImageTransactionalStorageInspector();
       
       storage.inspectStorageDirs(inspector);
-      LogLoadPlan logLoadPlan = inspector.createLogLoadPlan(lastAppliedTxId,
-          target - 1);
-  
-      logLoadPlan.doRecovery();
-      loadEdits(logLoadPlan.getEditsFiles());
+
+      editLog.recoverUnclosedStreams();
+      Iterable<EditLogInputStream> editStreamsAll 
+        = editLog.selectInputStreams(lastAppliedTxId, target - 1);
+      // remove inprogress
+      List<EditLogInputStream> editStreams = Lists.newArrayList();
+      for (EditLogInputStream s : editStreamsAll) {
+        if (s.getFirstTxId() != editLog.getCurSegmentTxId()) {
+          editStreams.add(s);
+        }
+      }
+      loadEdits(editStreams, namesystem);
     }
     
     // now, need to load the in-progress file
@@ -271,7 +287,24 @@
         return false; // drop lock and try again to load local logs
       }
       
-      EditLogInputStream stream = getEditLog().getInProgressFileInputStream();
+      EditLogInputStream stream = null;
+      Collection<EditLogInputStream> editStreams
+        = getEditLog().selectInputStreams(
+            getEditLog().getCurSegmentTxId(),
+            getEditLog().getCurSegmentTxId());
+      
+      for (EditLogInputStream s : editStreams) {
+        if (s.getFirstTxId() == getEditLog().getCurSegmentTxId()) {
+          stream = s;
+        }
+        break;
+      }
+      if (stream == null) {
+        LOG.warn("Unable to find stream starting with " + editLog.getCurSegmentTxId()
+                 + ". This indicates that there is an error in synchronization in BackupImage");
+        return false;
+      }
+
       try {
         long remainingTxns = getEditLog().getLastWrittenTxId() - lastAppliedTxId;
         
@@ -285,7 +318,7 @@
           "expected to load " + remainingTxns + " but loaded " +
           numLoaded + " from " + stream;
       } finally {
-        IOUtils.closeStream(stream);
+        FSEditLog.closeAllStreams(editStreams);
       }
 
       LOG.info("Successfully synced BackupNode with NameNode at txnid " +
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/BackupJournalManager.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/BackupJournalManager.java
index 7dd2e23..8eb8984 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/BackupJournalManager.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/BackupJournalManager.java
@@ -57,12 +57,31 @@
       throws IOException {
   }
 
+  @Override
+  public long getNumberOfTransactions(long fromTxnId) 
+      throws IOException, CorruptionException {
+    // This JournalManager is never used for input. Therefore it cannot
+    // return any transactions
+    return 0;
+  }
+  
+  @Override
+  public EditLogInputStream getInputStream(long fromTxnId) throws IOException {
+    // This JournalManager is never used for input. Therefore it cannot
+    // return any transactions
+    throw new IOException("Unsupported operation");
+  }
+
+  @Override
+  public void recoverUnfinalizedSegments() throws IOException {
+  }
+
   public boolean matchesRegistration(NamenodeRegistration bnReg) {
     return bnReg.getAddress().equals(this.bnReg.getAddress());
   }
 
   @Override
-  public EditLogInputStream getInProgressInputStream(long segmentStartsAtTxId) {
-    return null;
+  public String toString() {
+    return "BackupJournalManager";
   }
 }
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/BackupNode.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/BackupNode.java
index 93f6047..a8d65d0 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/BackupNode.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/BackupNode.java
@@ -28,16 +28,24 @@
 import org.apache.hadoop.hdfs.DFSUtil;
 import org.apache.hadoop.hdfs.protocol.DatanodeInfo;
 import org.apache.hadoop.hdfs.protocol.HdfsConstants;
+import org.apache.hadoop.hdfs.protocol.proto.JournalProtocolProtos.JournalProtocolService;
+import org.apache.hadoop.hdfs.protocolPB.JournalProtocolPB;
+import org.apache.hadoop.hdfs.protocolPB.JournalProtocolServerSideTranslatorPB;
+import org.apache.hadoop.hdfs.protocolPB.NamenodeProtocolTranslatorPB;
 import org.apache.hadoop.hdfs.server.common.HdfsServerConstants.NamenodeRole;
 import org.apache.hadoop.hdfs.server.common.Storage;
-import org.apache.hadoop.hdfs.server.protocol.JournalProtocol;
 import org.apache.hadoop.hdfs.server.protocol.BlocksWithLocations;
+import org.apache.hadoop.hdfs.server.protocol.JournalProtocol;
 import org.apache.hadoop.hdfs.server.protocol.NamenodeCommand;
 import org.apache.hadoop.hdfs.server.protocol.NamenodeProtocol;
 import org.apache.hadoop.hdfs.server.protocol.NamenodeRegistration;
 import org.apache.hadoop.hdfs.server.protocol.NamespaceInfo;
+import org.apache.hadoop.io.IOUtils;
 import org.apache.hadoop.ipc.RPC;
 import org.apache.hadoop.net.NetUtils;
+import org.apache.hadoop.security.UserGroupInformation;
+
+import com.google.protobuf.BlockingService;
 
 /**
  * BackupNode.
@@ -61,7 +69,7 @@
   private static final String BN_SERVICE_RPC_ADDRESS_KEY = DFSConfigKeys.DFS_NAMENODE_BACKUP_SERVICE_RPC_ADDRESS_KEY;
 
   /** Name-node proxy */
-  NamenodeProtocol namenode;
+  NamenodeProtocolTranslatorPB namenode;
   /** Name-node RPC address */
   String nnRpcAddress;
   /** Name-node HTTP address */
@@ -123,6 +131,7 @@
   protected void loadNamesystem(Configuration conf) throws IOException {
     BackupImage bnImage = new BackupImage(conf);
     this.namesystem = new FSNamesystem(conf, bnImage);
+    bnImage.setNamesystem(namesystem);
     bnImage.recoverCreateRead();
   }
 
@@ -175,7 +184,9 @@
       }
     }
     // Stop the RPC client
-    RPC.stopProxy(namenode);
+    if (namenode != null) {
+      IOUtils.cleanup(LOG, namenode);
+    }
     namenode = null;
     // Stop the checkpoint manager
     if(checkpointManager != null) {
@@ -186,12 +197,19 @@
     super.stop();
   }
 
-  static class BackupNodeRpcServer extends NameNodeRpcServer implements JournalProtocol {
+  static class BackupNodeRpcServer extends NameNodeRpcServer implements
+      JournalProtocol {
     private final String nnRpcAddress;
     
     private BackupNodeRpcServer(Configuration conf, BackupNode nn)
         throws IOException {
       super(conf, nn);
+      JournalProtocolServerSideTranslatorPB journalProtocolTranslator = 
+          new JournalProtocolServerSideTranslatorPB(this);
+      BlockingService service = JournalProtocolService
+          .newReflectiveBlockingService(journalProtocolTranslator);
+      DFSUtil.addPBProtocol(conf, JournalProtocolPB.class, service,
+          this.clientRpcServer);
       nnRpcAddress = nn.nnRpcAddress;
     }
 
@@ -200,9 +218,8 @@
         throws IOException {
       if (protocol.equals(JournalProtocol.class.getName())) {
         return JournalProtocol.versionID;
-      } else {
-        return super.getProtocolVersion(protocol, clientVersion);
       }
+      return super.getProtocolVersion(protocol, clientVersion);
     }
   
     /////////////////////////////////////////////////////
@@ -244,7 +261,7 @@
       verifyRequest(nnReg);
       if(!nnRpcAddress.equals(nnReg.getAddress()))
         throw new IOException("Journal request from unexpected name-node: "
-            + nnReg.getAddress() + " expecting " + rpcAddress);
+            + nnReg.getAddress() + " expecting " + clientRpcAddress);
       getBNImage().journal(firstTxId, numTxns, records);
     }
   
@@ -278,9 +295,8 @@
   private NamespaceInfo handshake(Configuration conf) throws IOException {
     // connect to name node
     InetSocketAddress nnAddress = NameNode.getServiceAddress(conf, true);
-    this.namenode =
-      (NamenodeProtocol) RPC.waitForProxy(NamenodeProtocol.class,
-          NamenodeProtocol.versionID, nnAddress, conf);
+    this.namenode = new NamenodeProtocolTranslatorPB(nnAddress, conf,
+        UserGroupInformation.getCurrentUser());
     this.nnRpcAddress = getHostPortString(nnAddress);
     this.nnHttpAddress = getHostPortString(super.getHttpServerAddress(conf));
     // get version and id info from the name-node
@@ -293,7 +309,9 @@
         LOG.info("Problem connecting to server: " + nnAddress);
         try {
           Thread.sleep(1000);
-        } catch (InterruptedException ie) {}
+        } catch (InterruptedException ie) {
+          LOG.warn("Encountered exception ", e);
+        }
       }
     }
     return nsInfo;
@@ -342,7 +360,9 @@
         LOG.info("Problem connecting to name-node: " + nnRpcAddress);
         try {
           Thread.sleep(1000);
-        } catch (InterruptedException ie) {}
+        } catch (InterruptedException ie) {
+          LOG.warn("Encountered exception ", e);
+        }
       }
     }
 
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/CheckpointSignature.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/CheckpointSignature.java
index 0db5cb1..5f5ebaf 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/CheckpointSignature.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/CheckpointSignature.java
@@ -37,9 +37,7 @@
                       implements WritableComparable<CheckpointSignature> {
   private static final String FIELD_SEPARATOR = ":";
   private static final int NUM_FIELDS = 7;
-
   String blockpoolID = "";
-  
   long mostRecentCheckpointTxId;
   long curSegmentTxId;
 
@@ -67,6 +65,14 @@
     blockpoolID = fields[i++];
   }
 
+  public CheckpointSignature(StorageInfo info, String blockpoolID,
+      long mostRecentCheckpointTxId, long curSegmentTxId) {
+    super(info);
+    this.blockpoolID = blockpoolID;
+    this.mostRecentCheckpointTxId = mostRecentCheckpointTxId;
+    this.curSegmentTxId = curSegmentTxId;
+  }
+
   /**
    * Get the cluster id from CheckpointSignature
    * @return the cluster id
@@ -83,6 +89,14 @@
     return blockpoolID;
   }
 
+  public long getMostRecentCheckpointTxId() {
+    return mostRecentCheckpointTxId;
+  }
+
+  public long getCurSegmentTxId() {
+    return curSegmentTxId;
+  }
+
   /**
    * Set the block pool id of CheckpointSignature.
    * 
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/Checkpointer.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/Checkpointer.java
index 2c861ac21..a4ac99b 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/Checkpointer.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/Checkpointer.java
@@ -224,7 +224,7 @@
         
         LOG.info("Loading image with txid " + sig.mostRecentCheckpointTxId);
         File file = bnStorage.findImageFile(sig.mostRecentCheckpointTxId);
-        bnImage.reloadFromImageFile(file);
+        bnImage.reloadFromImageFile(file, backupNode.getNamesystem());
       }
       
       lastApplied = bnImage.getLastAppliedTxId();
@@ -238,11 +238,11 @@
             backupNode.nnHttpAddress, log, bnStorage);
       }
   
-      rollForwardByApplyingLogs(manifest, bnImage);
+      rollForwardByApplyingLogs(manifest, bnImage, backupNode.getNamesystem());
     }
 
     long txid = bnImage.getLastAppliedTxId();
-    bnImage.saveFSImageInAllDirs(txid);
+    bnImage.saveFSImageInAllDirs(backupNode.getNamesystem(), txid);
     bnStorage.writeAll();
 
     if(cpCmd.needToReturnImage()) {
@@ -272,19 +272,21 @@
 
   static void rollForwardByApplyingLogs(
       RemoteEditLogManifest manifest,
-      FSImage dstImage) throws IOException {
+      FSImage dstImage,
+      FSNamesystem dstNamesystem) throws IOException {
     NNStorage dstStorage = dstImage.getStorage();
   
-    List<File> editsFiles = Lists.newArrayList();
+    List<EditLogInputStream> editsStreams = Lists.newArrayList();    
     for (RemoteEditLog log : manifest.getLogs()) {
       File f = dstStorage.findFinalizedEditsFile(
           log.getStartTxId(), log.getEndTxId());
       if (log.getStartTxId() > dstImage.getLastAppliedTxId()) {
-        editsFiles.add(f);
-      }
+        editsStreams.add(new EditLogFileInputStream(f, log.getStartTxId(), 
+                                                    log.getEndTxId()));
+       }
     }
     LOG.info("Checkpointer about to load edits from " +
-        editsFiles.size() + " file(s).");
-    dstImage.loadEdits(editsFiles);
+        editsStreams.size() + " stream(s).");
+    dstImage.loadEdits(editsStreams, dstNamesystem);
   }
 }
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/ContentSummaryServlet.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/ContentSummaryServlet.java
index 6ed5f49..141cdb6 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/ContentSummaryServlet.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/ContentSummaryServlet.java
@@ -40,7 +40,7 @@
   /** For java.io.Serializable */
   private static final long serialVersionUID = 1L;
   
-  /** {@inheritDoc} */
+  @Override
   public void doGet(final HttpServletRequest request,
       final HttpServletResponse response) throws ServletException, IOException {
     final Configuration conf = 
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/EditLogBackupInputStream.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/EditLogBackupInputStream.java
index 8921bc0..974697d 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/EditLogBackupInputStream.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/EditLogBackupInputStream.java
@@ -21,6 +21,7 @@
 import java.io.ByteArrayInputStream;
 import java.io.IOException;
 
+import org.apache.hadoop.hdfs.protocol.HdfsConstants;
 import com.google.common.base.Preconditions;
 
 /**
@@ -122,4 +123,14 @@
     reader = null;
     this.version = 0;
   }
+
+  @Override
+  public long getFirstTxId() throws IOException {
+    return HdfsConstants.INVALID_TXID;
+  }
+
+  @Override
+  public long getLastTxId() throws IOException {
+    return HdfsConstants.INVALID_TXID;
+  }
 }
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/EditLogBackupOutputStream.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/EditLogBackupOutputStream.java
index 5dd8239..7805bb1 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/EditLogBackupOutputStream.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/EditLogBackupOutputStream.java
@@ -22,11 +22,11 @@
 import java.util.Arrays;
 
 import org.apache.hadoop.hdfs.HdfsConfiguration;
+import org.apache.hadoop.hdfs.protocolPB.JournalProtocolTranslatorPB;
 import org.apache.hadoop.hdfs.server.common.Storage;
-import org.apache.hadoop.hdfs.server.protocol.JournalProtocol;
 import org.apache.hadoop.hdfs.server.protocol.NamenodeRegistration;
 import org.apache.hadoop.io.DataOutputBuffer;
-import org.apache.hadoop.ipc.RPC;
+import org.apache.hadoop.io.IOUtils;
 import org.apache.hadoop.net.NetUtils;
 
 /**
@@ -40,7 +40,7 @@
 class EditLogBackupOutputStream extends EditLogOutputStream {
   static int DEFAULT_BUFFER_SIZE = 256;
 
-  private JournalProtocol backupNode;        // RPC proxy to backup node
+  private JournalProtocolTranslatorPB backupNode;  // RPC proxy to backup node
   private NamenodeRegistration bnRegistration;  // backup node registration
   private NamenodeRegistration nnRegistration;  // active node registration
   private EditsDoubleBuffer doubleBuf;
@@ -57,8 +57,7 @@
     Storage.LOG.info("EditLogBackupOutputStream connects to: " + bnAddress);
     try {
       this.backupNode =
-        RPC.getProxy(JournalProtocol.class,
-            JournalProtocol.versionID, bnAddress, new HdfsConfiguration());
+          new JournalProtocolTranslatorPB(bnAddress, new HdfsConfiguration());
     } catch(IOException e) {
       Storage.LOG.error("Error connecting to: " + bnAddress, e);
       throw e;
@@ -105,14 +104,14 @@
       throw new IOException("BackupEditStream has " + size +
                           " records still to be flushed and cannot be closed.");
     } 
-    RPC.stopProxy(backupNode); // stop the RPC threads
+    IOUtils.cleanup(Storage.LOG, backupNode); // stop the RPC threads
     doubleBuf.close();
     doubleBuf = null;
   }
 
   @Override
   public void abort() throws IOException {
-    RPC.stopProxy(backupNode);
+    IOUtils.cleanup(Storage.LOG, backupNode);
     doubleBuf = null;
   }
 
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/EditLogFileInputStream.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/EditLogFileInputStream.java
index 2e0404e..8685ca5 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/EditLogFileInputStream.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/EditLogFileInputStream.java
@@ -27,6 +27,7 @@
 import org.apache.hadoop.hdfs.protocol.HdfsConstants;
 import org.apache.hadoop.hdfs.server.common.Storage;
 import org.apache.hadoop.io.IOUtils;
+import org.apache.hadoop.hdfs.protocol.HdfsConstants;
 
 import com.google.common.annotations.VisibleForTesting;
 
@@ -37,12 +38,15 @@
 class EditLogFileInputStream extends EditLogInputStream {
   private final File file;
   private final FileInputStream fStream;
+  final private long firstTxId;
+  final private long lastTxId;
   private final int logVersion;
   private final FSEditLogOp.Reader reader;
   private final FSEditLogLoader.PositionTrackingInputStream tracker;
   
   /**
    * Open an EditLogInputStream for the given file.
+   * The file is pretransactional, so has no txids
    * @param name filename to open
    * @throws LogHeaderCorruptException if the header is either missing or
    *         appears to be corrupt/truncated
@@ -51,6 +55,21 @@
    */
   EditLogFileInputStream(File name)
       throws LogHeaderCorruptException, IOException {
+    this(name, HdfsConstants.INVALID_TXID, HdfsConstants.INVALID_TXID);
+  }
+
+  /**
+   * Open an EditLogInputStream for the given file.
+   * @param name filename to open
+   * @param firstTxId first transaction found in file
+   * @param lastTxId last transaction id found in file
+   * @throws LogHeaderCorruptException if the header is either missing or
+   *         appears to be corrupt/truncated
+   * @throws IOException if an actual IO error occurs while reading the
+   *         header
+   */
+  EditLogFileInputStream(File name, long firstTxId, long lastTxId)
+      throws LogHeaderCorruptException, IOException {
     file = name;
     fStream = new FileInputStream(name);
 
@@ -65,6 +84,18 @@
     }
 
     reader = new FSEditLogOp.Reader(in, logVersion);
+    this.firstTxId = firstTxId;
+    this.lastTxId = lastTxId;
+  }
+
+  @Override
+  public long getFirstTxId() throws IOException {
+    return firstTxId;
+  }
+  
+  @Override
+  public long getLastTxId() throws IOException {
+    return lastTxId;
   }
 
   @Override // JournalStream
@@ -116,7 +147,8 @@
       // If it's missing its header, this is equivalent to no transactions
       FSImage.LOG.warn("Log at " + file + " has no valid header",
           corrupt);
-      return new FSEditLogLoader.EditLogValidation(0, 0);
+      return new FSEditLogLoader.EditLogValidation(0, HdfsConstants.INVALID_TXID, 
+                                                   HdfsConstants.INVALID_TXID);
     }
     
     try {
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/EditLogFileOutputStream.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/EditLogFileOutputStream.java
index be75f63..24f2d49 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/EditLogFileOutputStream.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/EditLogFileOutputStream.java
@@ -83,7 +83,6 @@
     return JournalType.FILE;
   }
 
-  /** {@inheritDoc} */
   @Override
   void write(FSEditLogOp op) throws IOException {
     doubleBuf.writeOp(op);
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/EditLogInputStream.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/EditLogInputStream.java
index 52a3dd4..c6f8505 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/EditLogInputStream.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/EditLogInputStream.java
@@ -28,6 +28,17 @@
  * into the #{@link EditLogOutputStream}.
  */
 abstract class EditLogInputStream implements JournalStream, Closeable {
+  /** 
+   * @return the first transaction which will be found in this stream
+   */
+  public abstract long getFirstTxId() throws IOException;
+  
+  /** 
+   * @return the last transaction which will be found in this stream
+   */
+  public abstract long getLastTxId() throws IOException;
+
+
   /**
    * Close the stream.
    * @throws IOException if an error occurred while closing
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSDirectory.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSDirectory.java
index 923e578..aafd3af 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSDirectory.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSDirectory.java
@@ -57,9 +57,10 @@
 import org.apache.hadoop.hdfs.server.blockmanagement.BlockManager;
 import org.apache.hadoop.hdfs.server.blockmanagement.DatanodeDescriptor;
 import org.apache.hadoop.hdfs.server.common.HdfsServerConstants.BlockUCState;
-import org.apache.hadoop.hdfs.server.common.HdfsServerConstants.StartupOption;
 import org.apache.hadoop.hdfs.util.ByteArray;
 
+import com.google.common.base.Preconditions;
+
 /*************************************************
  * FSDirectory stores the filesystem directory state.
  * It handles writing/loading values to disk, and logging
@@ -73,6 +74,7 @@
 
   INodeDirectoryWithQuota rootDir;
   FSImage fsImage;  
+  private final FSNamesystem namesystem;
   private volatile boolean ready = false;
   private static final long UNKNOWN_DISK_SPACE = -1;
   private final int maxComponentLength;
@@ -114,15 +116,9 @@
    */
   private final NameCache<ByteArray> nameCache;
 
-  /** Access an existing dfs name directory. */
-  FSDirectory(FSNamesystem ns, Configuration conf) throws IOException {
-    this(new FSImage(conf), ns, conf);
-  }
-
   FSDirectory(FSImage fsImage, FSNamesystem ns, Configuration conf) {
     this.dirLock = new ReentrantReadWriteLock(true); // fair
     this.cond = dirLock.writeLock().newCondition();
-    fsImage.setFSNamesystem(ns);
     rootDir = new INodeDirectoryWithQuota(INodeDirectory.ROOT_NAME,
         ns.createFsOwnerPermissions(new FsPermission((short)0755)),
         Integer.MAX_VALUE, UNKNOWN_DISK_SPACE);
@@ -146,10 +142,11 @@
     NameNode.LOG.info("Caching file names occuring more than " + threshold
         + " times ");
     nameCache = new NameCache<ByteArray>(threshold);
+    namesystem = ns;
   }
     
   private FSNamesystem getFSNamesystem() {
-    return fsImage.getFSNamesystem();
+    return namesystem;
   }
 
   private BlockManager getBlockManager() {
@@ -157,33 +154,11 @@
   }
 
   /**
-   * Load the filesystem image into memory.
-   *
-   * @param startOpt Startup type as specified by the user.
-   * @throws IOException If image or editlog cannot be read.
+   * Notify that loading of this FSDirectory is complete, and
+   * it is ready for use 
    */
-  void loadFSImage(StartupOption startOpt) 
-      throws IOException {
-    // format before starting up if requested
-    if (startOpt == StartupOption.FORMAT) {
-      fsImage.format(fsImage.getStorage().determineClusterId());// reuse current id
-
-      startOpt = StartupOption.REGULAR;
-    }
-    boolean success = false;
-    try {
-      if (fsImage.recoverTransitionRead(startOpt)) {
-        fsImage.saveNamespace();
-      }
-      fsImage.openEditLog();
-      
-      fsImage.setCheckpointDirectories(null, null);
-      success = true;
-    } finally {
-      if (!success) {
-        fsImage.close();
-      }
-    }
+  void imageLoadComplete() {
+    Preconditions.checkState(!ready, "FSDirectory already loaded");
     writeLock();
     try {
       setReady(true);
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSEditLog.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSEditLog.java
index 2108ded..7f9c218 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSEditLog.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSEditLog.java
@@ -18,6 +18,7 @@
 package org.apache.hadoop.hdfs.server.namenode;
 
 import java.io.IOException;
+import java.util.Collection;
 import java.util.Collections;
 import java.util.Iterator;
 import java.util.List;
@@ -35,11 +36,13 @@
 import org.apache.hadoop.hdfs.server.common.Storage.StorageDirectory;
 import static org.apache.hadoop.hdfs.server.common.Util.now;
 import org.apache.hadoop.hdfs.server.namenode.NNStorage.NameNodeDirType;
+import org.apache.hadoop.hdfs.server.namenode.JournalManager.CorruptionException;
 import org.apache.hadoop.hdfs.server.namenode.metrics.NameNodeMetrics;
 import org.apache.hadoop.hdfs.server.protocol.NamenodeRegistration;
 import org.apache.hadoop.hdfs.server.protocol.RemoteEditLogManifest;
 import org.apache.hadoop.hdfs.server.protocol.RemoteEditLog;
 import org.apache.hadoop.security.token.delegation.DelegationKey;
+import org.apache.hadoop.io.IOUtils;
 
 import com.google.common.annotations.VisibleForTesting;
 import com.google.common.base.Preconditions;
@@ -1069,6 +1072,112 @@
   }
 
   /**
+   * Find the best editlog input stream to read from txid. In this case
+   * best means the editlog which has the largest continuous range of 
+   * transactions starting from the transaction id, fromTxId.
+   *
+   * If a journal throws an CorruptionException while reading from a txn id,
+   * it means that it has more transactions, but can't find any from fromTxId. 
+   * If this is the case and no other journal has transactions, we should throw
+   * an exception as it means more transactions exist, we just can't load them.
+   *
+   * @param fromTxId Transaction id to start from.
+   * @return a edit log input stream with tranactions fromTxId 
+   *         or null if no more exist
+   */
+  private EditLogInputStream selectStream(long fromTxId) 
+      throws IOException {
+    JournalManager bestjm = null;
+    long bestjmNumTxns = 0;
+    CorruptionException corruption = null;
+
+    for (JournalAndStream jas : journals) {
+      JournalManager candidate = jas.getManager();
+      long candidateNumTxns = 0;
+      try {
+        candidateNumTxns = candidate.getNumberOfTransactions(fromTxId);
+      } catch (CorruptionException ce) {
+        corruption = ce;
+      } catch (IOException ioe) {
+        LOG.warn("Error reading number of transactions from " + candidate);
+        continue; // error reading disk, just skip
+      }
+      
+      if (candidateNumTxns > bestjmNumTxns) {
+        bestjm = candidate;
+        bestjmNumTxns = candidateNumTxns;
+      }
+    }
+    
+    
+    if (bestjm == null) {
+      /**
+       * If all candidates either threw a CorruptionException or
+       * found 0 transactions, then a gap exists. 
+       */
+      if (corruption != null) {
+        throw new IOException("Gap exists in logs from " 
+                              + fromTxId, corruption);
+      } else {
+        return null;
+      }
+    }
+
+    return bestjm.getInputStream(fromTxId);
+  }
+
+  /**
+   * Run recovery on all journals to recover any unclosed segments
+   */
+  void recoverUnclosedStreams() {
+    mapJournalsAndReportErrors(new JournalClosure() {
+        @Override
+        public void apply(JournalAndStream jas) throws IOException {
+          jas.manager.recoverUnfinalizedSegments();
+        }
+      }, "recovering unclosed streams");
+  }
+
+  /**
+   * Select a list of input streams to load.
+   * @param fromTxId first transaction in the selected streams
+   * @param toAtLeast the selected streams must contain this transaction
+   */
+  Collection<EditLogInputStream> selectInputStreams(long fromTxId, long toAtLeastTxId) 
+      throws IOException {
+    List<EditLogInputStream> streams = Lists.newArrayList();
+    
+    boolean gapFound = false;
+    EditLogInputStream stream = selectStream(fromTxId);
+    while (stream != null) {
+      fromTxId = stream.getLastTxId() + 1;
+      streams.add(stream);
+      try {
+        stream = selectStream(fromTxId);
+      } catch (IOException ioe) {
+        gapFound = true;
+        break;
+      }
+    }
+    if (fromTxId <= toAtLeastTxId || gapFound) {
+      closeAllStreams(streams);
+      throw new IOException("No non-corrupt logs for txid " 
+                            + fromTxId);
+    }
+    return streams;
+  }
+
+  /** 
+   * Close all the streams in a collection
+   * @param streams The list of streams to close
+   */
+  static void closeAllStreams(Iterable<EditLogInputStream> streams) {
+    for (EditLogInputStream s : streams) {
+      IOUtils.closeStream(s);
+    }
+  }
+
+  /**
    * Container for a JournalManager paired with its currently
    * active stream.
    * 
@@ -1137,30 +1246,5 @@
     JournalManager getManager() {
       return manager;
     }
-
-    private EditLogInputStream getInProgressInputStream() throws IOException {
-      return manager.getInProgressInputStream(segmentStartsAtTxId);
-    }
-  }
-
-  /**
-   * @return an EditLogInputStream that reads from the same log that
-   * the edit log is currently writing. This is used from the BackupNode
-   * during edits synchronization.
-   * @throws IOException if no valid logs are available.
-   */
-  synchronized EditLogInputStream getInProgressFileInputStream()
-      throws IOException {
-    for (JournalAndStream jas : journals) {
-      if (!jas.isActive()) continue;
-      try {
-        EditLogInputStream in = jas.getInProgressInputStream();
-        if (in != null) return in;
-      } catch (IOException ioe) {
-        LOG.warn("Unable to get the in-progress input stream from " + jas,
-            ioe);
-      }
-    }
-    throw new IOException("No in-progress stream provided edits");
   }
 }
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSEditLogLoader.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSEditLogLoader.java
index 7a85d2b..ca65788 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSEditLogLoader.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSEditLogLoader.java
@@ -467,24 +467,6 @@
     }
   }
   
-  static EditLogValidation validateEditLog(File file) throws IOException {
-    EditLogFileInputStream in;
-    try {
-      in = new EditLogFileInputStream(file);
-    } catch (LogHeaderCorruptException corrupt) {
-      // If it's missing its header, this is equivalent to no transactions
-      FSImage.LOG.warn("Log at " + file + " has no valid header",
-          corrupt);
-      return new EditLogValidation(0, 0);
-    }
-    
-    try {
-      return validateEditLog(in);
-    } finally {
-      IOUtils.closeStream(in);
-    }
-  }
-
   /**
    * Return the number of valid transactions in the stream. If the stream is
    * truncated during the header, returns a value indicating that there are
@@ -494,12 +476,26 @@
    *                     if the log does not exist)
    */
   static EditLogValidation validateEditLog(EditLogInputStream in) {
-    long numValid = 0;
     long lastPos = 0;
+    long firstTxId = HdfsConstants.INVALID_TXID;
+    long lastTxId = HdfsConstants.INVALID_TXID;
+    long numValid = 0;
     try {
+      FSEditLogOp op = null;
       while (true) {
         lastPos = in.getPosition();
-        if (in.readOp() == null) {
+        if ((op = in.readOp()) == null) {
+          break;
+        }
+        if (firstTxId == HdfsConstants.INVALID_TXID) {
+          firstTxId = op.txid;
+        }
+        if (lastTxId == HdfsConstants.INVALID_TXID
+            || op.txid == lastTxId + 1) {
+          lastTxId = op.txid;
+        } else {
+          FSImage.LOG.error("Out of order txid found. Found " + op.txid 
+                            + ", expected " + (lastTxId + 1));
           break;
         }
         numValid++;
@@ -510,16 +506,33 @@
       FSImage.LOG.debug("Caught exception after reading " + numValid +
           " ops from " + in + " while determining its valid length.", t);
     }
-    return new EditLogValidation(lastPos, numValid);
+    return new EditLogValidation(lastPos, firstTxId, lastTxId);
   }
   
   static class EditLogValidation {
-    long validLength;
-    long numTransactions;
-    
-    EditLogValidation(long validLength, long numTransactions) {
+    private long validLength;
+    private long startTxId;
+    private long endTxId;
+     
+    EditLogValidation(long validLength, 
+                      long startTxId, long endTxId) {
       this.validLength = validLength;
-      this.numTransactions = numTransactions;
+      this.startTxId = startTxId;
+      this.endTxId = endTxId;
+    }
+    
+    long getValidLength() { return validLength; }
+    
+    long getStartTxId() { return startTxId; }
+    
+    long getEndTxId() { return endTxId; }
+    
+    long getNumTransactions() { 
+      if (endTxId == HdfsConstants.INVALID_TXID
+          || startTxId == HdfsConstants.INVALID_TXID) {
+        return 0;
+      }
+      return (endTxId - startTxId) + 1;
     }
   }
 
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSImage.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSImage.java
index 7604d92..dade5ab 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSImage.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSImage.java
@@ -46,7 +46,7 @@
 import static org.apache.hadoop.hdfs.server.common.Util.now;
 import org.apache.hadoop.hdfs.server.common.HdfsServerConstants.NamenodeRole;
 import org.apache.hadoop.hdfs.server.common.HdfsServerConstants.StartupOption;
-import org.apache.hadoop.hdfs.server.namenode.FSImageStorageInspector.LoadPlan;
+
 import org.apache.hadoop.hdfs.server.namenode.NNStorage.NameNodeDirType;
 import org.apache.hadoop.hdfs.server.namenode.NNStorage.NameNodeFile;
 import org.apache.hadoop.hdfs.server.protocol.CheckpointCommand;
@@ -70,7 +70,6 @@
 public class FSImage implements Closeable {
   protected static final Log LOG = LogFactory.getLog(FSImage.class.getName());
 
-  protected FSNamesystem namesystem = null;
   protected FSEditLog editLog = null;
   private boolean isUpgradeFinalized = false;
 
@@ -82,38 +81,20 @@
    */
   protected long lastAppliedTxId = 0;
 
-  /**
-   * URIs for importing an image from a checkpoint. In the default case,
-   * URIs will represent directories. 
-   */
-  private Collection<URI> checkpointDirs;
-  private Collection<URI> checkpointEditsDirs;
-
   final private Configuration conf;
 
   private final NNStorageRetentionManager archivalManager; 
 
-  /**
-   * Construct an FSImage.
-   * @param conf Configuration
-   * @see #FSImage(Configuration conf, FSNamesystem ns, 
-   *               Collection imageDirs, Collection editsDirs) 
-   * @throws IOException if default directories are invalid.
-   */
-  public FSImage(Configuration conf) throws IOException {
-    this(conf, (FSNamesystem)null);
-  }
 
   /**
    * Construct an FSImage
    * @param conf Configuration
-   * @param ns The FSNamesystem using this image.
-   * @see #FSImage(Configuration conf, FSNamesystem ns, 
+   * @see #FSImage(Configuration conf, 
    *               Collection imageDirs, Collection editsDirs) 
    * @throws IOException if default directories are invalid.
    */
-  private FSImage(Configuration conf, FSNamesystem ns) throws IOException {
-    this(conf, ns,
+  protected FSImage(Configuration conf) throws IOException {
+    this(conf,
          FSNamesystem.getNamespaceDirs(conf),
          FSNamesystem.getNamespaceEditsDirs(conf));
   }
@@ -124,17 +105,14 @@
    * Setup storage and initialize the edit log.
    *
    * @param conf Configuration
-   * @param ns The FSNamesystem using this image.
    * @param imageDirs Directories the image can be stored in.
    * @param editsDirs Directories the editlog can be stored in.
    * @throws IOException if directories are invalid.
    */
-  protected FSImage(Configuration conf, FSNamesystem ns,
+  protected FSImage(Configuration conf,
                     Collection<URI> imageDirs, Collection<URI> editsDirs)
       throws IOException {
     this.conf = conf;
-    setCheckpointDirectories(FSImage.getCheckpointDirs(conf, null),
-                             FSImage.getCheckpointEditsDirs(conf, null));
 
     storage = new NNStorage(conf, imageDirs, editsDirs);
     if(conf.getBoolean(DFSConfigKeys.DFS_NAMENODE_NAME_DIR_RESTORE_KEY,
@@ -143,31 +121,18 @@
     }
 
     this.editLog = new FSEditLog(storage);
-    setFSNamesystem(ns);
     
     archivalManager = new NNStorageRetentionManager(conf, storage, editLog);
   }
-
-  protected FSNamesystem getFSNamesystem() {
-    return namesystem;
-  }
-
-  void setFSNamesystem(FSNamesystem ns) {
-    namesystem = ns;
-    if (ns != null) {
-      storage.setUpgradeManager(ns.upgradeManager);
-    }
-  }
  
-  void setCheckpointDirectories(Collection<URI> dirs,
-                                Collection<URI> editsDirs) {
-    checkpointDirs = dirs;
-    checkpointEditsDirs = editsDirs;
-  }
-  
-  void format(String clusterId) throws IOException {
+  void format(FSNamesystem fsn, String clusterId) throws IOException {
+    long fileCount = fsn.getTotalFiles();
+    // Expect 1 file, which is the root inode
+    Preconditions.checkState(fileCount == 1,
+        "FSImage.format should be called with an uninitialized namesystem, has " +
+        fileCount + " files");
     storage.format(clusterId);
-    saveFSImageInAllDirs(0);    
+    saveFSImageInAllDirs(fsn, 0);
   }
   
   /**
@@ -179,7 +144,7 @@
    * @throws IOException
    * @return true if the image needs to be saved or false otherwise
    */
-  boolean recoverTransitionRead(StartupOption startOpt)
+  boolean recoverTransitionRead(StartupOption startOpt, FSNamesystem target)
       throws IOException {
     assert startOpt != StartupOption.FORMAT : 
       "NameNode formatting should be performed before reading the image";
@@ -187,21 +152,14 @@
     Collection<URI> imageDirs = storage.getImageDirectories();
     Collection<URI> editsDirs = storage.getEditsDirectories();
 
+
     // none of the data dirs exist
     if((imageDirs.size() == 0 || editsDirs.size() == 0) 
                              && startOpt != StartupOption.IMPORT)  
       throw new IOException(
           "All specified directories are not accessible or do not exist.");
     
-    if(startOpt == StartupOption.IMPORT 
-        && (checkpointDirs == null || checkpointDirs.isEmpty()))
-      throw new IOException("Cannot import image from a checkpoint. "
-                            + "\"dfs.namenode.checkpoint.dir\" is not set." );
-
-    if(startOpt == StartupOption.IMPORT 
-        && (checkpointEditsDirs == null || checkpointEditsDirs.isEmpty()))
-      throw new IOException("Cannot import image from a checkpoint. "
-                            + "\"dfs.namenode.checkpoint.dir\" is not set." );
+    storage.setUpgradeManager(target.upgradeManager);
     
     // 1. For each data directory calculate its state and 
     // check whether all is consistent before transitioning.
@@ -261,10 +219,10 @@
     // 3. Do transitions
     switch(startOpt) {
     case UPGRADE:
-      doUpgrade();
+      doUpgrade(target);
       return false; // upgrade saved image already
     case IMPORT:
-      doImportCheckpoint();
+      doImportCheckpoint(target);
       return false; // import checkpoint saved image already
     case ROLLBACK:
       doRollback();
@@ -273,7 +231,7 @@
       // just load the image
     }
     
-    return loadFSImage();
+    return loadFSImage(target);
   }
   
   /**
@@ -324,11 +282,11 @@
     return isFormatted;
   }
 
-  private void doUpgrade() throws IOException {
+  private void doUpgrade(FSNamesystem target) throws IOException {
     if(storage.getDistributedUpgradeState()) {
       // only distributed upgrade need to continue
       // don't do version upgrade
-      this.loadFSImage();
+      this.loadFSImage(target);
       storage.initializeDistributedUpgrade();
       return;
     }
@@ -343,7 +301,7 @@
     }
 
     // load the latest image
-    this.loadFSImage();
+    this.loadFSImage(target);
 
     // Do upgrade for each directory
     long oldCTime = storage.getCTime();
@@ -385,7 +343,7 @@
     storage.reportErrorsOnDirectories(errorSDs);
     errorSDs.clear();
 
-    saveFSImageInAllDirs(editLog.getLastWrittenTxId());
+    saveFSImageInAllDirs(target, editLog.getLastWrittenTxId());
 
     for (Iterator<StorageDirectory> it = storage.dirIterator(); it.hasNext();) {
       StorageDirectory sd = it.next();
@@ -422,7 +380,7 @@
     // a previous fs states in at least one of the storage directories.
     // Directories that don't have previous state do not rollback
     boolean canRollback = false;
-    FSImage prevState = new FSImage(conf, getFSNamesystem());
+    FSImage prevState = new FSImage(conf);
     prevState.getStorage().layoutVersion = HdfsConstants.LAYOUT_VERSION;
     for (Iterator<StorageDirectory> it = storage.dirIterator(); it.hasNext();) {
       StorageDirectory sd = it.next();
@@ -504,19 +462,32 @@
 
   /**
    * Load image from a checkpoint directory and save it into the current one.
+   * @param target the NameSystem to import into
    * @throws IOException
    */
-  void doImportCheckpoint() throws IOException {
-    FSNamesystem fsNamesys = getFSNamesystem();
-    FSImage ckptImage = new FSImage(conf, fsNamesys,
+  void doImportCheckpoint(FSNamesystem target) throws IOException {
+    Collection<URI> checkpointDirs =
+      FSImage.getCheckpointDirs(conf, null);
+    Collection<URI> checkpointEditsDirs =
+      FSImage.getCheckpointEditsDirs(conf, null);
+
+    if (checkpointDirs == null || checkpointDirs.isEmpty()) {
+      throw new IOException("Cannot import image from a checkpoint. "
+                            + "\"dfs.namenode.checkpoint.dir\" is not set." );
+    }
+    
+    if (checkpointEditsDirs == null || checkpointEditsDirs.isEmpty()) {
+      throw new IOException("Cannot import image from a checkpoint. "
+                            + "\"dfs.namenode.checkpoint.dir\" is not set." );
+    }
+
+    FSImage realImage = target.getFSImage();
+    FSImage ckptImage = new FSImage(conf, 
                                     checkpointDirs, checkpointEditsDirs);
-    // replace real image with the checkpoint image
-    FSImage realImage = fsNamesys.getFSImage();
-    assert realImage == this;
-    fsNamesys.dir.fsImage = ckptImage;
+    target.dir.fsImage = ckptImage;
     // load from the checkpoint dirs
     try {
-      ckptImage.recoverTransitionRead(StartupOption.REGULAR);
+      ckptImage.recoverTransitionRead(StartupOption.REGULAR, target);
     } finally {
       ckptImage.close();
     }
@@ -524,10 +495,11 @@
     realImage.getStorage().setStorageInfo(ckptImage.getStorage());
     realImage.getEditLog().setNextTxId(ckptImage.getEditLog().getLastWrittenTxId()+1);
 
-    fsNamesys.dir.fsImage = realImage;
+    target.dir.fsImage = realImage;
     realImage.getStorage().setBlockPoolID(ckptImage.getBlockPoolID());
+
     // and save it but keep the same checkpointTime
-    saveNamespace();
+    saveNamespace(target);
     getStorage().writeAll();
   }
 
@@ -558,11 +530,11 @@
    * Toss the current image and namesystem, reloading from the specified
    * file.
    */
-  void reloadFromImageFile(File file) throws IOException {
-    namesystem.dir.reset();
+  void reloadFromImageFile(File file, FSNamesystem target) throws IOException {
+    target.dir.reset();
 
     LOG.debug("Reloading namespace from " + file);
-    loadFSImage(file);
+    loadFSImage(file, target);
   }
 
   /**
@@ -580,36 +552,42 @@
    * @return whether the image should be saved
    * @throws IOException
    */
-  boolean loadFSImage() throws IOException {
+  boolean loadFSImage(FSNamesystem target) throws IOException {
     FSImageStorageInspector inspector = storage.readAndInspectDirs();
     
     isUpgradeFinalized = inspector.isUpgradeFinalized();
-    
+ 
+    FSImageStorageInspector.FSImageFile imageFile 
+      = inspector.getLatestImage();   
     boolean needToSave = inspector.needToSave();
+
+    Iterable<EditLogInputStream> editStreams = null;
+
+    editLog.recoverUnclosedStreams();
+
+    if (LayoutVersion.supports(Feature.TXID_BASED_LAYOUT, 
+                               getLayoutVersion())) {
+      editStreams = editLog.selectInputStreams(imageFile.getCheckpointTxId() + 1,
+                                               inspector.getMaxSeenTxId());
+    } else {
+      editStreams = FSImagePreTransactionalStorageInspector
+        .getEditLogStreams(storage);
+    }
+ 
+    LOG.debug("Planning to load image :\n" + imageFile);
+    for (EditLogInputStream l : editStreams) {
+      LOG.debug("\t Planning to load edit stream: " + l);
+    }
     
-    // Plan our load. This will throw if it's impossible to load from the
-    // data that's available.
-    LoadPlan loadPlan = inspector.createLoadPlan();    
-    LOG.debug("Planning to load image using following plan:\n" + loadPlan);
-
-    
-    // Recover from previous interrupted checkpoint, if any
-    needToSave |= loadPlan.doRecovery();
-
-    //
-    // Load in bits
-    //
-    StorageDirectory sdForProperties =
-      loadPlan.getStorageDirectoryForProperties();
-    storage.readProperties(sdForProperties);
-    File imageFile = loadPlan.getImageFile();
-
     try {
+      StorageDirectory sdForProperties = imageFile.sd;
+      storage.readProperties(sdForProperties);
+
       if (LayoutVersion.supports(Feature.TXID_BASED_LAYOUT,
                                  getLayoutVersion())) {
         // For txid-based layout, we should have a .md5 file
         // next to the image file
-        loadFSImage(imageFile);
+        loadFSImage(imageFile.getFile(), target);
       } else if (LayoutVersion.supports(Feature.FSIMAGE_CHECKSUM,
                                         getLayoutVersion())) {
         // In 0.22, we have the checksum stored in the VERSION file.
@@ -621,17 +599,19 @@
               NNStorage.DEPRECATED_MESSAGE_DIGEST_PROPERTY +
               " not set for storage directory " + sdForProperties.getRoot());
         }
-        loadFSImage(imageFile, new MD5Hash(md5));
+        loadFSImage(imageFile.getFile(), new MD5Hash(md5), target);
       } else {
         // We don't have any record of the md5sum
-        loadFSImage(imageFile, null);
+        loadFSImage(imageFile.getFile(), null, target);
       }
     } catch (IOException ioe) {
-      throw new IOException("Failed to load image from " + loadPlan.getImageFile(), ioe);
+      FSEditLog.closeAllStreams(editStreams);
+      throw new IOException("Failed to load image from " + imageFile, ioe);
     }
     
-    long numLoaded = loadEdits(loadPlan.getEditsFiles());
-    needToSave |= needsResaveBasedOnStaleCheckpoint(imageFile, numLoaded);
+    long numLoaded = loadEdits(editStreams, target);
+    needToSave |= needsResaveBasedOnStaleCheckpoint(imageFile.getFile(),
+                                                    numLoaded);
     
     // update the txid for the edit log
     editLog.setNextTxId(storage.getMostRecentCheckpointTxId() + numLoaded + 1);
@@ -663,26 +643,30 @@
    * Load the specified list of edit files into the image.
    * @return the number of transactions loaded
    */
-  protected long loadEdits(List<File> editLogs) throws IOException {
-    LOG.debug("About to load edits:\n  " + Joiner.on("\n  ").join(editLogs));
+  protected long loadEdits(Iterable<EditLogInputStream> editStreams,
+                           FSNamesystem target) throws IOException {
+    LOG.debug("About to load edits:\n  " + Joiner.on("\n  ").join(editStreams));
 
     long startingTxId = getLastAppliedTxId() + 1;
-    
-    FSEditLogLoader loader = new FSEditLogLoader(namesystem);
     int numLoaded = 0;
-    // Load latest edits
-    for (File edits : editLogs) {
-      LOG.debug("Reading " + edits + " expecting start txid #" + startingTxId);
-      EditLogFileInputStream editIn = new EditLogFileInputStream(edits);
-      int thisNumLoaded = loader.loadFSEdits(editIn, startingTxId);
-      startingTxId += thisNumLoaded;
-      numLoaded += thisNumLoaded;
-      lastAppliedTxId += thisNumLoaded;
-      editIn.close();
+
+    try {    
+      FSEditLogLoader loader = new FSEditLogLoader(target);
+      
+      // Load latest edits
+      for (EditLogInputStream editIn : editStreams) {
+        LOG.info("Reading " + editIn + " expecting start txid #" + startingTxId);
+        int thisNumLoaded = loader.loadFSEdits(editIn, startingTxId);
+        startingTxId += thisNumLoaded;
+        numLoaded += thisNumLoaded;
+        lastAppliedTxId += thisNumLoaded;
+      }
+    } finally {
+      FSEditLog.closeAllStreams(editStreams);
     }
 
     // update the counts
-    getFSNamesystem().dir.updateCountForINodeWithQuota();    
+    target.dir.updateCountForINodeWithQuota();    
     return numLoaded;
   }
 
@@ -691,13 +675,14 @@
    * Load the image namespace from the given image file, verifying
    * it against the MD5 sum stored in its associated .md5 file.
    */
-  private void loadFSImage(File imageFile) throws IOException {
+  private void loadFSImage(File imageFile, FSNamesystem target)
+      throws IOException {
     MD5Hash expectedMD5 = MD5FileUtils.readStoredMd5ForFile(imageFile);
     if (expectedMD5 == null) {
       throw new IOException("No MD5 file found corresponding to image file "
           + imageFile);
     }
-    loadFSImage(imageFile, expectedMD5);
+    loadFSImage(imageFile, expectedMD5, target);
   }
   
   /**
@@ -705,11 +690,12 @@
    * filenames and blocks.  Return whether we should
    * "re-save" and consolidate the edit-logs
    */
-  private void loadFSImage(File curFile, MD5Hash expectedMd5) throws IOException {
+  private void loadFSImage(File curFile, MD5Hash expectedMd5,
+      FSNamesystem target) throws IOException {
     FSImageFormat.Loader loader = new FSImageFormat.Loader(
-        conf, getFSNamesystem());
+        conf, target);
     loader.load(curFile);
-    namesystem.setBlockPoolId(this.getBlockPoolID());
+    target.setBlockPoolId(this.getBlockPoolID());
 
     // Check that the image digest we loaded matches up with what
     // we expected
@@ -730,13 +716,14 @@
   /**
    * Save the contents of the FS image to the file.
    */
-  void saveFSImage(StorageDirectory sd, long txid) throws IOException {
+  void saveFSImage(FSNamesystem source, StorageDirectory sd, long txid)
+      throws IOException {
     File newFile = NNStorage.getStorageFile(sd, NameNodeFile.IMAGE_NEW, txid);
     File dstFile = NNStorage.getStorageFile(sd, NameNodeFile.IMAGE, txid);
     
     FSImageFormat.Saver saver = new FSImageFormat.Saver();
     FSImageCompression compression = FSImageCompression.createCompression(conf);
-    saver.save(newFile, txid, getFSNamesystem(), compression);
+    saver.save(newFile, txid, source, compression);
     
     MD5FileUtils.saveMD5File(dstFile, saver.getSavedDigest());
     storage.setMostRecentCheckpointInfo(txid, Util.now());
@@ -757,8 +744,11 @@
     private StorageDirectory sd;
     private List<StorageDirectory> errorSDs;
     private final long txid;
+    private final FSNamesystem source;
     
-    FSImageSaver(StorageDirectory sd, List<StorageDirectory> errorSDs, long txid) {
+    FSImageSaver(FSNamesystem source, StorageDirectory sd,
+        List<StorageDirectory> errorSDs, long txid) {
+      this.source = source;
       this.sd = sd;
       this.errorSDs = errorSDs;
       this.txid = txid;
@@ -766,7 +756,7 @@
     
     public void run() {
       try {
-        saveFSImage(sd, txid);
+        saveFSImage(source, sd, txid);
       } catch (Throwable t) {
         LOG.error("Unable to save image for " + sd.getRoot(), t);
         errorSDs.add(sd);
@@ -795,7 +785,7 @@
    * Save the contents of the FS image to a new image file in each of the
    * current storage directories.
    */
-  void saveNamespace() throws IOException {
+  void saveNamespace(FSNamesystem source) throws IOException {
     assert editLog != null : "editLog must be initialized";
     storage.attemptRestoreRemovedStorage();
 
@@ -806,7 +796,7 @@
     }
     long imageTxId = editLog.getLastWrittenTxId();
     try {
-      saveFSImageInAllDirs(imageTxId);
+      saveFSImageInAllDirs(source, imageTxId);
       storage.writeAll();
     } finally {
       if (editLogWasOpen) {
@@ -818,7 +808,8 @@
     
   }
   
-  protected void saveFSImageInAllDirs(long txid) throws IOException {
+  protected void saveFSImageInAllDirs(FSNamesystem source, long txid)
+      throws IOException {
     if (storage.getNumStorageDirs(NameNodeDirType.IMAGE) == 0) {
       throw new IOException("No image directories available!");
     }
@@ -831,7 +822,7 @@
     for (Iterator<StorageDirectory> it
            = storage.dirIterator(NameNodeDirType.IMAGE); it.hasNext();) {
       StorageDirectory sd = it.next();
-      FSImageSaver saver = new FSImageSaver(sd, errorSDs, txid);
+      FSImageSaver saver = new FSImageSaver(source, sd, errorSDs, txid);
       Thread saveThread = new Thread(saver, saver.toString());
       saveThreads.add(saveThread);
       saveThread.start();
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSImageFormat.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSImageFormat.java
index 8579764..c178e04 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSImageFormat.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSImageFormat.java
@@ -556,8 +556,13 @@
       DataOutputStream out = new DataOutputStream(fos);
       try {
         out.writeInt(HdfsConstants.LAYOUT_VERSION);
-        out.writeInt(sourceNamesystem.getFSImage()
-                     .getStorage().getNamespaceID()); // TODO bad dependency
+        // We use the non-locked version of getNamespaceInfo here since
+        // the coordinating thread of saveNamespace already has read-locked
+        // the namespace for us. If we attempt to take another readlock
+        // from the actual saver thread, there's a potential of a
+        // fairness-related deadlock. See the comments on HDFS-2223.
+        out.writeInt(sourceNamesystem.unprotectedGetNamespaceInfo()
+            .getNamespaceID());
         out.writeLong(fsDir.rootDir.numItemsInTree());
         out.writeLong(sourceNamesystem.getGenerationStamp());
         out.writeLong(txid);
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSImagePreTransactionalStorageInspector.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSImagePreTransactionalStorageInspector.java
index cec2eeff..91076ef 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSImagePreTransactionalStorageInspector.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSImagePreTransactionalStorageInspector.java
@@ -32,6 +32,7 @@
 import org.apache.commons.logging.LogFactory;
 import org.apache.hadoop.classification.InterfaceAudience;
 import org.apache.hadoop.classification.InterfaceStability;
+import org.apache.hadoop.hdfs.protocol.HdfsConstants;
 import org.apache.hadoop.hdfs.server.common.Storage.StorageDirectory;
 import org.apache.hadoop.hdfs.server.namenode.NNStorage.NameNodeDirType;
 import org.apache.hadoop.hdfs.server.namenode.NNStorage.NameNodeFile;
@@ -55,6 +56,7 @@
   private boolean hasOutOfDateStorageDirs = false;
   /* Flag set false if there are any "previous" directories found */
   private boolean isUpgradeFinalized = true;
+  private boolean needToSaveAfterRecovery = false;
   
   // Track the name and edits dir with the latest times
   private long latestNameCheckpointTime = Long.MIN_VALUE;
@@ -139,15 +141,15 @@
   boolean isUpgradeFinalized() {
     return isUpgradeFinalized;
   }
-  
+    
   @Override
-  LoadPlan createLoadPlan() throws IOException {
+  FSImageFile getLatestImage() throws IOException {
     // We should have at least one image and one edits dirs
     if (latestNameSD == null)
       throw new IOException("Image file is not found in " + imageDirs);
     if (latestEditsSD == null)
       throw new IOException("Edits file is not found in " + editsDirs);
-
+    
     // Make sure we are loading image and edits from same checkpoint
     if (latestNameCheckpointTime > latestEditsCheckpointTime
         && latestNameSD != latestEditsSD
@@ -168,92 +170,70 @@
                       "image checkpoint time = " + latestNameCheckpointTime +
                       "edits checkpoint time = " + latestEditsCheckpointTime);
     }
+
+    needToSaveAfterRecovery = doRecovery();
     
-    return new PreTransactionalLoadPlan();
+    return new FSImageFile(latestNameSD, 
+        NNStorage.getStorageFile(latestNameSD, NameNodeFile.IMAGE),
+        HdfsConstants.INVALID_TXID);
   }
-  
+
   @Override
   boolean needToSave() {
     return hasOutOfDateStorageDirs ||
       checkpointTimes.size() != 1 ||
-      latestNameCheckpointTime > latestEditsCheckpointTime;
-
+      latestNameCheckpointTime > latestEditsCheckpointTime ||
+      needToSaveAfterRecovery;
   }
   
-  private class PreTransactionalLoadPlan extends LoadPlan {
-
-    @Override
-    boolean doRecovery() throws IOException {
-      LOG.debug(
+  boolean doRecovery() throws IOException {
+    LOG.debug(
         "Performing recovery in "+ latestNameSD + " and " + latestEditsSD);
       
-      boolean needToSave = false;
-      File curFile =
-        NNStorage.getStorageFile(latestNameSD, NameNodeFile.IMAGE);
-      File ckptFile =
-        NNStorage.getStorageFile(latestNameSD, NameNodeFile.IMAGE_NEW);
-
-      //
-      // If we were in the midst of a checkpoint
-      //
-      if (ckptFile.exists()) {
-        needToSave = true;
-        if (NNStorage.getStorageFile(latestEditsSD, NameNodeFile.EDITS_NEW)
-              .exists()) {
-          //
-          // checkpointing migth have uploaded a new
-          // merged image, but we discard it here because we are
-          // not sure whether the entire merged image was uploaded
-          // before the namenode crashed.
-          //
-          if (!ckptFile.delete()) {
-            throw new IOException("Unable to delete " + ckptFile);
-          }
-        } else {
-          //
-          // checkpointing was in progress when the namenode
-          // shutdown. The fsimage.ckpt was created and the edits.new
-          // file was moved to edits. We complete that checkpoint by
-          // moving fsimage.new to fsimage. There is no need to 
-          // update the fstime file here. renameTo fails on Windows
-          // if the destination file already exists.
-          //
+    boolean needToSave = false;
+    File curFile =
+      NNStorage.getStorageFile(latestNameSD, NameNodeFile.IMAGE);
+    File ckptFile =
+      NNStorage.getStorageFile(latestNameSD, NameNodeFile.IMAGE_NEW);
+    
+    //
+    // If we were in the midst of a checkpoint
+    //
+    if (ckptFile.exists()) {
+      needToSave = true;
+      if (NNStorage.getStorageFile(latestEditsSD, NameNodeFile.EDITS_NEW)
+          .exists()) {
+        //
+        // checkpointing migth have uploaded a new
+        // merged image, but we discard it here because we are
+        // not sure whether the entire merged image was uploaded
+        // before the namenode crashed.
+        //
+        if (!ckptFile.delete()) {
+          throw new IOException("Unable to delete " + ckptFile);
+        }
+      } else {
+        //
+        // checkpointing was in progress when the namenode
+        // shutdown. The fsimage.ckpt was created and the edits.new
+        // file was moved to edits. We complete that checkpoint by
+        // moving fsimage.new to fsimage. There is no need to 
+        // update the fstime file here. renameTo fails on Windows
+        // if the destination file already exists.
+        //
+        if (!ckptFile.renameTo(curFile)) {
+          if (!curFile.delete())
+            LOG.warn("Unable to delete dir " + curFile + " before rename");
           if (!ckptFile.renameTo(curFile)) {
-            if (!curFile.delete())
-              LOG.warn("Unable to delete dir " + curFile + " before rename");
-            if (!ckptFile.renameTo(curFile)) {
-              throw new IOException("Unable to rename " + ckptFile +
-                                    " to " + curFile);
-            }
+            throw new IOException("Unable to rename " + ckptFile +
+                                  " to " + curFile);
           }
         }
       }
-      return needToSave;
     }
-
-    @Override
-    File getImageFile() {
-      return NNStorage.getStorageFile(latestNameSD, NameNodeFile.IMAGE);
-    }
-
-    @Override
-    List<File> getEditsFiles() {
-      if (latestNameCheckpointTime > latestEditsCheckpointTime) {
-        // the image is already current, discard edits
-        LOG.debug(
-          "Name checkpoint time is newer than edits, not loading edits.");
-        return Collections.<File>emptyList();
-      }
-      
-      return getEditsInStorageDir(latestEditsSD);
-    }
-
-    @Override
-    StorageDirectory getStorageDirectoryForProperties() {
-      return latestNameSD;
-    }    
+    return needToSave;
   }
-
+  
   /**
    * @return a list with the paths to EDITS and EDITS_NEW (if it exists)
    * in a given storage directory.
@@ -269,4 +249,33 @@
     }
     return files;
   }
+  
+  private List<File> getLatestEditsFiles() {
+    if (latestNameCheckpointTime > latestEditsCheckpointTime) {
+      // the image is already current, discard edits
+      LOG.debug(
+          "Name checkpoint time is newer than edits, not loading edits.");
+      return Collections.<File>emptyList();
+    }
+    
+    return getEditsInStorageDir(latestEditsSD);
+  }
+  
+  @Override
+  long getMaxSeenTxId() {
+    return 0L;
+  }
+
+  static Iterable<EditLogInputStream> getEditLogStreams(NNStorage storage)
+      throws IOException {
+    FSImagePreTransactionalStorageInspector inspector 
+      = new FSImagePreTransactionalStorageInspector();
+    storage.inspectStorageDirs(inspector);
+
+    List<EditLogInputStream> editStreams = new ArrayList<EditLogInputStream>();
+    for (File f : inspector.getLatestEditsFiles()) {
+      editStreams.add(new EditLogFileInputStream(f));
+    }
+    return editStreams;
+  }
 }
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSImageStorageInspector.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSImageStorageInspector.java
index 65bfa0a..a7c2949 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSImageStorageInspector.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSImageStorageInspector.java
@@ -21,6 +21,7 @@
 import java.io.IOException;
 import java.util.List;
 
+import org.apache.hadoop.hdfs.protocol.HdfsConstants;
 import org.apache.hadoop.classification.InterfaceAudience;
 import org.apache.hadoop.classification.InterfaceStability;
 import org.apache.hadoop.hdfs.server.common.Storage.StorageDirectory;
@@ -43,11 +44,16 @@
   abstract boolean isUpgradeFinalized();
   
   /**
-   * Create a plan to load the image from the set of inspected storage directories.
+   * Get the image files which should be loaded into the filesystem.
    * @throws IOException if not enough files are available (eg no image found in any directory)
    */
-  abstract LoadPlan createLoadPlan() throws IOException;
-  
+  abstract FSImageFile getLatestImage() throws IOException;
+
+  /** 
+   * Get the minimum tx id which should be loaded with this set of images.
+   */
+  abstract long getMaxSeenTxId();
+
   /**
    * @return true if the directories are in such a state that the image should be re-saved
    * following the load
@@ -55,49 +61,6 @@
   abstract boolean needToSave();
 
   /**
-   * A plan to load the namespace from disk, providing the locations from which to load
-   * the image and a set of edits files.
-   */
-  abstract static class LoadPlan {
-    /**
-     * Execute atomic move sequence in the chosen storage directories,
-     * in order to recover from an interrupted checkpoint.
-     * @return true if some recovery action was taken
-     */
-    abstract boolean doRecovery() throws IOException;
-
-    /**
-     * @return the file from which to load the image data
-     */
-    abstract File getImageFile();
-    
-    /**
-     * @return a list of flies containing edits to replay
-     */
-    abstract List<File> getEditsFiles();
-    
-    /**
-     * @return the storage directory containing the VERSION file that should be
-     * loaded.
-     */
-    abstract StorageDirectory getStorageDirectoryForProperties();
-    
-    @Override
-    public String toString() {
-      StringBuilder sb = new StringBuilder();
-      sb.append("Will load image file: ").append(getImageFile()).append("\n");
-      sb.append("Will load edits files:").append("\n");
-      for (File f : getEditsFiles()) {
-        sb.append("  ").append(f).append("\n");
-      }
-      sb.append("Will load metadata from: ")
-        .append(getStorageDirectoryForProperties())
-        .append("\n");
-      return sb.toString();
-    }
-  }
-
-  /**
    * Record of an image that has been located and had its filename parsed.
    */
   static class FSImageFile {
@@ -106,7 +69,8 @@
     private final File file;
     
     FSImageFile(StorageDirectory sd, File file, long txId) {
-      assert txId >= 0 : "Invalid txid on " + file +": " + txId;
+      assert txId >= 0 || txId == HdfsConstants.INVALID_TXID 
+        : "Invalid txid on " + file +": " + txId;
       
       this.sd = sd;
       this.txId = txId;
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSImageTransactionalStorageInspector.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSImageTransactionalStorageInspector.java
index 303b8e6..1a52373 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSImageTransactionalStorageInspector.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSImageTransactionalStorageInspector.java
@@ -39,7 +39,6 @@
 import org.apache.hadoop.hdfs.server.common.Storage.StorageDirectory;
 import org.apache.hadoop.hdfs.server.namenode.NNStorage.NameNodeDirType;
 import org.apache.hadoop.hdfs.server.namenode.NNStorage.NameNodeFile;
-import org.apache.hadoop.hdfs.server.namenode.FileJournalManager.EditLogFile;
 import com.google.common.base.Joiner;
 import com.google.common.collect.ImmutableList;
 import com.google.common.collect.Lists;
@@ -52,9 +51,7 @@
   private boolean isUpgradeFinalized = true;
   
   List<FSImageFile> foundImages = new ArrayList<FSImageFile>();
-  List<EditLogFile> foundEditLogs = new ArrayList<EditLogFile>();
-  SortedMap<Long, LogGroup> logGroups = new TreeMap<Long, LogGroup>();
-  long maxSeenTxId = 0;
+  private long maxSeenTxId = 0;
   
   private static final Pattern IMAGE_REGEX = Pattern.compile(
     NameNodeFile.IMAGE.getName() + "_(\\d+)");
@@ -68,6 +65,8 @@
       return;
     }
     
+    maxSeenTxId = Math.max(maxSeenTxId, NNStorage.readTransactionIdFile(sd));
+
     File currentDir = sd.getCurrentDir();
     File filesInStorage[];
     try {
@@ -110,34 +109,10 @@
       LOG.warn("Unable to determine the max transaction ID seen by " + sd, ioe);
     }
     
-    List<EditLogFile> editLogs 
-      = FileJournalManager.matchEditLogs(filesInStorage);
-    if (sd.getStorageDirType().isOfType(NameNodeDirType.EDITS)) {
-      for (EditLogFile log : editLogs) {
-        addEditLog(log);
-      }
-    } else if (!editLogs.isEmpty()){
-      LOG.warn("Found the following edit log file(s) in " + sd +
-          " even though it was not configured to store edits:\n" +
-          "  " + Joiner.on("\n  ").join(editLogs));
-          
-    }
-    
     // set finalized flag
     isUpgradeFinalized = isUpgradeFinalized && !sd.getPreviousDir().exists();
   }
 
-  private void addEditLog(EditLogFile foundEditLog) {
-    foundEditLogs.add(foundEditLog);
-    LogGroup group = logGroups.get(foundEditLog.getFirstTxId());
-    if (group == null) {
-      group = new LogGroup(foundEditLog.getFirstTxId());
-      logGroups.put(foundEditLog.getFirstTxId(), group);
-    }
-    group.add(foundEditLog);
-  }
-
-
   @Override
   public boolean isUpgradeFinalized() {
     return isUpgradeFinalized;
@@ -148,9 +123,13 @@
    * If there are multiple storage directories which contain equal images 
    * the storage directory that was inspected first will be preferred.
    * 
-   * Returns null if no images were found.
+   * @throws FileNotFoundException if not images are found.
    */
-  FSImageFile getLatestImage() {
+  FSImageFile getLatestImage() throws IOException {
+    if (foundImages.isEmpty()) {
+      throw new FileNotFoundException("No valid image files found");
+    }
+
     FSImageFile ret = null;
     for (FSImageFile img : foundImages) {
       if (ret == null || img.txId > ret.txId) {
@@ -164,349 +143,13 @@
     return ImmutableList.copyOf(foundImages);
   }
   
-  public List<EditLogFile> getEditLogFiles() {
-    return ImmutableList.copyOf(foundEditLogs);
-  }
-
-  @Override
-  public LoadPlan createLoadPlan() throws IOException {
-    if (foundImages.isEmpty()) {
-      throw new FileNotFoundException("No valid image files found");
-    }
-
-    FSImageFile recoveryImage = getLatestImage();
-    LogLoadPlan logPlan = createLogLoadPlan(recoveryImage.txId, Long.MAX_VALUE);
-
-    return new TransactionalLoadPlan(recoveryImage,
-        logPlan);
-  }
-  
-  /**
-   * Plan which logs to load in order to bring the namespace up-to-date.
-   * Transactions will be considered in the range (sinceTxId, maxTxId]
-   * 
-   * @param sinceTxId the highest txid that is already loaded 
-   *                  (eg from the image checkpoint)
-   * @param maxStartTxId ignore any log files that start after this txid
-   */
-  LogLoadPlan createLogLoadPlan(long sinceTxId, long maxStartTxId) throws IOException {
-    long expectedTxId = sinceTxId + 1;
-    
-    List<EditLogFile> recoveryLogs = new ArrayList<EditLogFile>();
-    
-    SortedMap<Long, LogGroup> tailGroups = logGroups.tailMap(expectedTxId);
-    if (logGroups.size() > tailGroups.size()) {
-      LOG.debug("Excluded " + (logGroups.size() - tailGroups.size()) + 
-          " groups of logs because they start with a txid less than image " +
-          "txid " + sinceTxId);
-    }
-    
-    SortedMap<Long, LogGroup> usefulGroups;
-    if (maxStartTxId > sinceTxId) {
-      usefulGroups = tailGroups.headMap(maxStartTxId);
-    } else {
-      usefulGroups = new TreeMap<Long, LogGroup>();
-    }
-    
-    if (usefulGroups.size() > tailGroups.size()) {
-      LOG.debug("Excluded " + (tailGroups.size() - usefulGroups.size()) + 
-        " groups of logs because they start with a txid higher than max " +
-        "txid " + sinceTxId);
-    }
-
-
-    for (Map.Entry<Long, LogGroup> entry : usefulGroups.entrySet()) {
-      long logStartTxId = entry.getKey();
-      LogGroup logGroup = entry.getValue();
-      
-      logGroup.planRecovery();
-      
-      if (expectedTxId != HdfsConstants.INVALID_TXID && logStartTxId != expectedTxId) {
-        throw new IOException("Expected next log group would start at txid " +
-            expectedTxId + " but starts at txid " + logStartTxId);
-      }
-      
-      // We can pick any of the non-corrupt logs here
-      recoveryLogs.add(logGroup.getBestNonCorruptLog());
-      
-      // If this log group was finalized, we know to expect the next
-      // log group to start at the following txid (ie no gaps)
-      if (logGroup.hasKnownLastTxId()) {
-        expectedTxId = logGroup.getLastTxId() + 1;
-      } else {
-        // the log group was in-progress so we don't know what ID
-        // the next group should start from.
-        expectedTxId = HdfsConstants.INVALID_TXID;
-      }
-    }
-    
-    long lastLogGroupStartTxId = usefulGroups.isEmpty() ?
-        0 : usefulGroups.lastKey();
-    if (maxSeenTxId > sinceTxId &&
-        maxSeenTxId > lastLogGroupStartTxId) {
-      String msg = "At least one storage directory indicated it has seen a " +
-        "log segment starting at txid " + maxSeenTxId;
-      if (usefulGroups.isEmpty()) {
-        msg += " but there are no logs to load.";
-      } else {
-        msg += " but the most recent log file found starts with txid " +
-          lastLogGroupStartTxId;
-      }
-      throw new IOException(msg);
-    }
-    
-    return new LogLoadPlan(recoveryLogs,
-        Lists.newArrayList(usefulGroups.values()));
-
-  }
-
   @Override
   public boolean needToSave() {
     return needToSave;
   }
-  
-  /**
-   * A group of logs that all start at the same txid.
-   * 
-   * Handles determining which logs are corrupt and which should be considered
-   * candidates for loading.
-   */
-  static class LogGroup {
-    long startTxId;
-    List<EditLogFile> logs = new ArrayList<EditLogFile>();;
-    private Set<Long> endTxIds = new TreeSet<Long>();
-    private boolean hasInProgress = false;
-    private boolean hasFinalized = false;
-        
-    LogGroup(long startTxId) {
-      this.startTxId = startTxId;
-    }
-    
-    EditLogFile getBestNonCorruptLog() {
-      // First look for non-corrupt finalized logs
-      for (EditLogFile log : logs) {
-        if (!log.isCorrupt() && !log.isInProgress()) {
-          return log;
-        }
-      }
-      // Then look for non-corrupt in-progress logs
-      for (EditLogFile log : logs) {
-        if (!log.isCorrupt()) {
-          return log;
-        }
-      }
 
-      // We should never get here, because we don't get to the planning stage
-      // without calling planRecovery first, and if we've called planRecovery,
-      // we would have already thrown if there were no non-corrupt logs!
-      throw new IllegalStateException(
-        "No non-corrupt logs for txid " + startTxId);
-    }
-
-    /**
-     * @return true if we can determine the last txid in this log group.
-     */
-    boolean hasKnownLastTxId() {
-      for (EditLogFile log : logs) {
-        if (!log.isInProgress()) {
-          return true;
-        }
-      }
-      return false;
-    }
-
-    /**
-     * @return the last txid included in the logs in this group
-     * @throws IllegalStateException if it is unknown -
-     *                               {@see #hasKnownLastTxId()}
-     */
-    long getLastTxId() {
-      for (EditLogFile log : logs) {
-        if (!log.isInProgress()) {
-          return log.getLastTxId();
-        }
-      }
-      throw new IllegalStateException("LogGroup only has in-progress logs");
-    }
-
-    
-    void add(EditLogFile log) {
-      assert log.getFirstTxId() == startTxId;
-      logs.add(log);
-      
-      if (log.isInProgress()) {
-        hasInProgress = true;
-      } else {
-        hasFinalized = true;
-        endTxIds.add(log.getLastTxId());
-      }
-    }
-    
-    void planRecovery() throws IOException {
-      assert hasInProgress || hasFinalized;
-      
-      checkConsistentEndTxIds();
-        
-      if (hasFinalized && hasInProgress) {
-        planMixedLogRecovery();
-      } else if (!hasFinalized && hasInProgress) {
-        planAllInProgressRecovery();
-      } else if (hasFinalized && !hasInProgress) {
-        LOG.debug("No recovery necessary for logs starting at txid " +
-                  startTxId);
-      }
-    }
-
-    /**
-     * Recovery case for when some logs in the group were in-progress, and
-     * others were finalized. This happens when one of the storage
-     * directories fails.
-     *
-     * The in-progress logs in this case should be considered corrupt.
-     */
-    private void planMixedLogRecovery() throws IOException {
-      for (EditLogFile log : logs) {
-        if (log.isInProgress()) {
-          LOG.warn("Log at " + log.getFile() + " is in progress, but " +
-                   "other logs starting at the same txid " + startTxId +
-                   " are finalized. Moving aside.");
-          log.markCorrupt();
-        }
-      }
-    }
-    
-    /**
-     * Recovery case for when all of the logs in the group were in progress.
-     * This happens if the NN completely crashes and restarts. In this case
-     * we check the non-zero lengths of each log file, and any logs that are
-     * less than the max of these lengths are considered corrupt.
-     */
-    private void planAllInProgressRecovery() throws IOException {
-      // We only have in-progress logs. We need to figure out which logs have
-      // the latest data to reccover them
-      LOG.warn("Logs beginning at txid " + startTxId + " were are all " +
-               "in-progress (probably truncated due to a previous NameNode " +
-               "crash)");
-      if (logs.size() == 1) {
-        // Only one log, it's our only choice!
-        EditLogFile log = logs.get(0);
-        if (log.validateLog().numTransactions == 0) {
-          // If it has no transactions, we should consider it corrupt just
-          // to be conservative.
-          // See comment below for similar case
-          LOG.warn("Marking log at " + log.getFile() + " as corrupt since " +
-              "it has no transactions in it.");
-          log.markCorrupt();          
-        }
-        return;
-      }
-
-      long maxValidTxnCount = Long.MIN_VALUE;
-      for (EditLogFile log : logs) {
-        long validTxnCount = log.validateLog().numTransactions;
-        LOG.warn("  Log " + log.getFile() +
-            " valid txns=" + validTxnCount +
-            " valid len=" + log.validateLog().validLength);
-        maxValidTxnCount = Math.max(maxValidTxnCount, validTxnCount);
-      }        
-
-      for (EditLogFile log : logs) {
-        long txns = log.validateLog().numTransactions;
-        if (txns < maxValidTxnCount) {
-          LOG.warn("Marking log at " + log.getFile() + " as corrupt since " +
-                   "it is has only " + txns + " valid txns whereas another " +
-                   "log has " + maxValidTxnCount);
-          log.markCorrupt();
-        } else if (txns == 0) {
-          // this can happen if the NN crashes right after rolling a log
-          // but before the START_LOG_SEGMENT txn is written. Since the log
-          // is empty, we can just move it aside to its corrupt name.
-          LOG.warn("Marking log at " + log.getFile() + " as corrupt since " +
-              "it has no transactions in it.");
-          log.markCorrupt();
-        }
-      }
-    }
-
-    /**
-     * Check for the case when we have multiple finalized logs and they have
-     * different ending transaction IDs. This violates an invariant that all
-     * log directories should roll together. We should abort in this case.
-     */
-    private void checkConsistentEndTxIds() throws IOException {
-      if (hasFinalized && endTxIds.size() > 1) {
-        throw new IOException("More than one ending txid was found " +
-            "for logs starting at txid " + startTxId + ". " +
-            "Found: " + StringUtils.join(endTxIds, ','));
-      }
-    }
-
-    void recover() throws IOException {
-      for (EditLogFile log : logs) {
-        if (log.isCorrupt()) {
-          log.moveAsideCorruptFile();
-        } else if (log.isInProgress()) {
-          log.finalizeLog();
-        }
-      }
-    }    
-  }
-  
-  static class TransactionalLoadPlan extends LoadPlan {
-    final FSImageFile image;
-    final LogLoadPlan logPlan;
-    
-    public TransactionalLoadPlan(FSImageFile image,
-        LogLoadPlan logPlan) {
-      super();
-      this.image = image;
-      this.logPlan = logPlan;
-    }
-
-    @Override
-    boolean doRecovery() throws IOException {
-      logPlan.doRecovery();
-      return false;
-    }
-
-    @Override
-    File getImageFile() {
-      return image.getFile();
-    }
-
-    @Override
-    List<File> getEditsFiles() {
-      return logPlan.getEditsFiles();
-    }
-
-    @Override
-    StorageDirectory getStorageDirectoryForProperties() {
-      return image.sd;
-    }
-  }
-  
-  static class LogLoadPlan {
-    final List<EditLogFile> editLogs;
-    final List<LogGroup> logGroupsToRecover;
-    
-    LogLoadPlan(List<EditLogFile> editLogs,
-        List<LogGroup> logGroupsToRecover) {
-      this.editLogs = editLogs;
-      this.logGroupsToRecover = logGroupsToRecover;
-    }
-
-    public void doRecovery() throws IOException {
-      for (LogGroup g : logGroupsToRecover) {
-        g.recover();
-      }
-    }
-
-    public List<File> getEditsFiles() {
-      List<File> ret = new ArrayList<File>();
-      for (EditLogFile log : editLogs) {
-        ret.add(log.getFile());
-      }
-      return ret;
-    }
+  @Override
+  long getMaxSeenTxId() {
+    return maxSeenTxId;
   }
 }
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSNamesystem.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSNamesystem.java
index 06d7736..20f5971 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSNamesystem.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSNamesystem.java
@@ -171,6 +171,7 @@
 import org.apache.hadoop.util.VersionInfo;
 import org.mortbay.util.ajax.JSON;
 
+import com.google.common.base.Preconditions;
 import com.google.common.annotations.VisibleForTesting;
 
 /***************************************************
@@ -295,12 +296,43 @@
   // lock to protect FSNamesystem.
   private ReentrantReadWriteLock fsLock;
 
+  
   /**
-   * FSNamesystem constructor.
+   * Instantiates an FSNamesystem loaded from the image and edits
+   * directories specified in the passed Configuration.
+   * 
+   * @param conf the Configuration which specifies the storage directories
+   *             from which to load
+   * @return an FSNamesystem which contains the loaded namespace
+   * @throws IOException if loading fails
    */
-  FSNamesystem(Configuration conf) throws IOException {
+  public static FSNamesystem loadFromDisk(Configuration conf) throws IOException {
+    FSImage fsImage = new FSImage(conf);
+    FSNamesystem namesystem = new FSNamesystem(conf, fsImage);
+
+    long loadStart = now();
+    StartupOption startOpt = NameNode.getStartupOption(conf);
+    namesystem.loadFSImage(startOpt, fsImage);
+    long timeTakenToLoadFSImage = now() - loadStart;
+    LOG.info("Finished loading FSImage in " + timeTakenToLoadFSImage + " msecs");
+    NameNode.getNameNodeMetrics().setFsImageLoadTime(
+                              (int) timeTakenToLoadFSImage);
+    return namesystem;
+  }
+
+  /**
+   * Create an FSNamesystem associated with the specified image.
+   * 
+   * Note that this does not load any data off of disk -- if you would
+   * like that behavior, use {@link #loadFromDisk(Configuration)}
+
+   * @param fnImage The FSImage to associate with
+   * @param conf configuration
+   * @throws IOException on bad configuration
+   */
+  FSNamesystem(Configuration conf, FSImage fsImage) throws IOException {
     try {
-      initialize(conf, null);
+      initialize(conf, fsImage);
     } catch(IOException e) {
       LOG.error(getClass().getSimpleName() + " initialization failed.", e);
       close();
@@ -316,29 +348,41 @@
     resourceRecheckInterval = conf.getLong(
         DFS_NAMENODE_RESOURCE_CHECK_INTERVAL_KEY,
         DFS_NAMENODE_RESOURCE_CHECK_INTERVAL_DEFAULT);
-    nnResourceChecker = new NameNodeResourceChecker(conf);
-    checkAvailableResources();
     this.systemStart = now();
     this.blockManager = new BlockManager(this, this, conf);
     this.datanodeStatistics = blockManager.getDatanodeManager().getDatanodeStatistics();
     this.fsLock = new ReentrantReadWriteLock(true); // fair locking
     setConfigurationParameters(conf);
     dtSecretManager = createDelegationTokenSecretManager(conf);
-    this.registerMBean(); // register the MBean for the FSNamesystemState
-    if(fsImage == null) {
-      this.dir = new FSDirectory(this, conf);
-      StartupOption startOpt = NameNode.getStartupOption(conf);
-      this.dir.loadFSImage(startOpt);
-      long timeTakenToLoadFSImage = now() - systemStart;
-      LOG.info("Finished loading FSImage in " + timeTakenToLoadFSImage + " msecs");
-      NameNode.getNameNodeMetrics().setFsImageLoadTime(
-                                (int) timeTakenToLoadFSImage);
-    } else {
-      this.dir = new FSDirectory(fsImage, this, conf);
-    }
+    this.dir = new FSDirectory(fsImage, this, conf);
     this.safeMode = new SafeModeInfo(conf);
   }
 
+  void loadFSImage(StartupOption startOpt, FSImage fsImage)
+      throws IOException {
+    // format before starting up if requested
+    if (startOpt == StartupOption.FORMAT) {
+      
+      fsImage.format(this, fsImage.getStorage().determineClusterId());// reuse current id
+
+      startOpt = StartupOption.REGULAR;
+    }
+    boolean success = false;
+    try {
+      if (fsImage.recoverTransitionRead(startOpt, this)) {
+        fsImage.saveNamespace(this);
+      }
+      fsImage.openEditLog();
+      
+      success = true;
+    } finally {
+      if (!success) {
+        fsImage.close();
+      }
+    }
+    dir.imageLoadComplete();
+  }
+
   void activateSecretManager() throws IOException {
     if (dtSecretManager != null) {
       dtSecretManager.startThreads();
@@ -349,8 +393,13 @@
    * Activate FSNamesystem daemons.
    */
   void activate(Configuration conf) throws IOException {
+    this.registerMBean(); // register the MBean for the FSNamesystemState
+
     writeLock();
     try {
+      nnResourceChecker = new NameNodeResourceChecker(conf);
+      checkAvailableResources();
+
       setBlockTotal();
       blockManager.activate(conf);
 
@@ -434,37 +483,6 @@
   }
 
   /**
-   * dirs is a list of directories where the filesystem directory state 
-   * is stored
-   */
-  FSNamesystem(FSImage fsImage, Configuration conf) throws IOException {
-    this.fsLock = new ReentrantReadWriteLock(true);
-    this.blockManager = new BlockManager(this, this, conf);
-    setConfigurationParameters(conf);
-    this.dir = new FSDirectory(fsImage, this, conf);
-    dtSecretManager = createDelegationTokenSecretManager(conf);
-  }
-
-  /**
-   * Create FSNamesystem for {@link BackupNode}.
-   * Should do everything that would be done for the NameNode,
-   * except for loading the image.
-   * 
-   * @param bnImage {@link BackupImage}
-   * @param conf configuration
-   * @throws IOException
-   */
-  FSNamesystem(Configuration conf, BackupImage bnImage) throws IOException {
-    try {
-      initialize(conf, bnImage);
-    } catch(IOException e) {
-      LOG.error(getClass().getSimpleName() + " initialization failed.", e);
-      close();
-      throw e;
-    }
-  }
-
-  /**
    * Initializes some of the members from configuration
    */
   private void setConfigurationParameters(Configuration conf) 
@@ -512,16 +530,23 @@
   NamespaceInfo getNamespaceInfo() {
     readLock();
     try {
-      return new NamespaceInfo(dir.fsImage.getStorage().getNamespaceID(),
-          getClusterId(), getBlockPoolId(),
-          dir.fsImage.getStorage().getCTime(),
-          upgradeManager.getUpgradeVersion());
+      return unprotectedGetNamespaceInfo();
     } finally {
       readUnlock();
     }
   }
 
   /**
+   * Version of {@see #getNamespaceInfo()} that is not protected by a lock.
+   */
+  NamespaceInfo unprotectedGetNamespaceInfo() {
+    return new NamespaceInfo(dir.fsImage.getStorage().getNamespaceID(),
+        getClusterId(), getBlockPoolId(),
+        dir.fsImage.getStorage().getCTime(),
+        upgradeManager.getUpgradeVersion());
+  }
+
+  /**
    * Close down this file system manager.
    * Causes heartbeat and lease daemons to stop; waits briefly for
    * them to finish, but a short timeout returns control back to caller.
@@ -2565,6 +2590,8 @@
    * @throws IOException
    */
   private void checkAvailableResources() throws IOException {
+    Preconditions.checkState(nnResourceChecker != null,
+        "nnResourceChecker not initialized");
     hasResourcesAvailable = nnResourceChecker.hasAvailableDiskSpace();
   }
 
@@ -2750,7 +2777,7 @@
         throw new IOException("Safe mode should be turned ON " +
                               "in order to create namespace image.");
       }
-      getFSImage().saveNamespace();
+      getFSImage().saveNamespace(this);
       LOG.info("New namespace image has been created.");
     } finally {
       readUnlock();
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FileChecksumServlets.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FileChecksumServlets.java
index 023e3b6..7bd1f27 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FileChecksumServlets.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FileChecksumServlets.java
@@ -80,7 +80,7 @@
           dtParam + addrParam);
     }
 
-    /** {@inheritDoc} */
+    @Override
     public void doGet(HttpServletRequest request, HttpServletResponse response
         ) throws ServletException, IOException {
       final ServletContext context = getServletContext();
@@ -104,7 +104,7 @@
     /** For java.io.Serializable */
     private static final long serialVersionUID = 1L;
     
-    /** {@inheritDoc} */
+    @Override
     public void doGet(HttpServletRequest request, HttpServletResponse response
         ) throws ServletException, IOException {
       final PrintWriter out = response.getWriter();
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FileJournalManager.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FileJournalManager.java
index 991d7f5..6e4c171 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FileJournalManager.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FileJournalManager.java
@@ -23,11 +23,14 @@
 import java.io.File;
 import java.io.IOException;
 import java.util.List;
+import java.util.HashMap;
 import java.util.Comparator;
+import java.util.Collections;
 import java.util.regex.Matcher;
 import java.util.regex.Pattern;
 
 import org.apache.hadoop.fs.FileUtil;
+import org.apache.hadoop.hdfs.protocol.HdfsConstants;
 import org.apache.hadoop.hdfs.server.common.Storage.StorageDirectory;
 import org.apache.hadoop.hdfs.server.namenode.NNStorageRetentionManager.StoragePurger;
 import org.apache.hadoop.hdfs.server.namenode.FSEditLogLoader.EditLogValidation;
@@ -57,6 +60,9 @@
   private static final Pattern EDITS_INPROGRESS_REGEX = Pattern.compile(
     NameNodeFile.EDITS_INPROGRESS.getName() + "_(\\d+)");
 
+  private File currentInProgress = null;
+  private long maxSeenTransaction = 0L;
+
   @VisibleForTesting
   StoragePurger purger
     = new NNStorageRetentionManager.DeletionStoragePurger();
@@ -66,19 +72,20 @@
   }
 
   @Override
-  public EditLogOutputStream startLogSegment(long txid) throws IOException {    
-    File newInProgress = NNStorage.getInProgressEditsFile(sd, txid);
-    EditLogOutputStream stm = new EditLogFileOutputStream(newInProgress,
+  synchronized public EditLogOutputStream startLogSegment(long txid) 
+      throws IOException {
+    currentInProgress = NNStorage.getInProgressEditsFile(sd, txid);
+    EditLogOutputStream stm = new EditLogFileOutputStream(currentInProgress,
         outputBufferCapacity);
     stm.create();
     return stm;
   }
 
   @Override
-  public void finalizeLogSegment(long firstTxId, long lastTxId)
+  synchronized public void finalizeLogSegment(long firstTxId, long lastTxId)
       throws IOException {
-    File inprogressFile = NNStorage.getInProgressEditsFile(
-        sd, firstTxId);
+    File inprogressFile = NNStorage.getInProgressEditsFile(sd, firstTxId);
+
     File dstFile = NNStorage.getFinalizedEditsFile(
         sd, firstTxId, lastTxId);
     LOG.debug("Finalizing edits file " + inprogressFile + " -> " + dstFile);
@@ -89,6 +96,9 @@
     if (!inprogressFile.renameTo(dstFile)) {
       throw new IOException("Unable to finalize edits file " + inprogressFile);
     }
+    if (inprogressFile.equals(currentInProgress)) {
+      currentInProgress = null;
+    }
   }
 
   @VisibleForTesting
@@ -97,12 +107,7 @@
   }
 
   @Override
-  public String toString() {
-    return "FileJournalManager for storage directory " + sd;
-  }
-
-  @Override
-  public void setOutputBufferCapacity(int size) {
+  synchronized public void setOutputBufferCapacity(int size) {
     this.outputBufferCapacity = size;
   }
 
@@ -120,13 +125,6 @@
     }
   }
 
-  @Override
-  public EditLogInputStream getInProgressInputStream(long segmentStartsAtTxId)
-      throws IOException {
-    File f = NNStorage.getInProgressEditsFile(sd, segmentStartsAtTxId);
-    return new EditLogFileInputStream(f);
-  }
-  
   /**
    * Find all editlog segments starting at or above the given txid.
    * @param fromTxId the txnid which to start looking
@@ -178,17 +176,156 @@
         try {
           long startTxId = Long.valueOf(inProgressEditsMatch.group(1));
           ret.add(
-            new EditLogFile(f, startTxId, EditLogFile.UNKNOWN_END));
+              new EditLogFile(f, startTxId, startTxId, true));
         } catch (NumberFormatException nfe) {
           LOG.error("In-progress edits file " + f + " has improperly " +
                     "formatted transaction ID");
           // skip
-        }          
+        }
       }
     }
     return ret;
   }
 
+  @Override
+  synchronized public EditLogInputStream getInputStream(long fromTxId) 
+      throws IOException {
+    for (EditLogFile elf : getLogFiles(fromTxId)) {
+      if (elf.getFirstTxId() == fromTxId) {
+        if (elf.isInProgress()) {
+          elf.validateLog();
+        }
+        if (LOG.isTraceEnabled()) {
+          LOG.trace("Returning edit stream reading from " + elf);
+        }
+        return new EditLogFileInputStream(elf.getFile(), 
+            elf.getFirstTxId(), elf.getLastTxId());
+      }
+    }
+
+    throw new IOException("Cannot find editlog file with " + fromTxId
+        + " as first first txid");
+  }
+
+  @Override
+  public long getNumberOfTransactions(long fromTxId) 
+      throws IOException, CorruptionException {
+    long numTxns = 0L;
+    
+    for (EditLogFile elf : getLogFiles(fromTxId)) {
+      if (LOG.isTraceEnabled()) {
+        LOG.trace("Counting " + elf);
+      }
+      if (elf.getFirstTxId() > fromTxId) { // there must be a gap
+        LOG.warn("Gap in transactions in " + sd.getRoot() + ". Gap is "
+            + fromTxId + " - " + (elf.getFirstTxId() - 1));
+        break;
+      } else if (fromTxId == elf.getFirstTxId()) {
+        if (elf.isInProgress()) {
+          elf.validateLog();
+        } 
+
+        if (elf.isCorrupt()) {
+          break;
+        }
+        fromTxId = elf.getLastTxId() + 1;
+        numTxns += fromTxId - elf.getFirstTxId();
+        
+        if (elf.isInProgress()) {
+          break;
+        }
+      } // else skip
+    }
+
+    if (LOG.isDebugEnabled()) {
+      LOG.debug("Journal " + this + " has " + numTxns 
+                + " txns from " + fromTxId);
+    }
+
+    long max = findMaxTransaction();
+    // fromTxId should be greater than max, as it points to the next 
+    // transaction we should expect to find. If it is less than or equal
+    // to max, it means that a transaction with txid == max has not been found
+    if (numTxns == 0 && fromTxId <= max) { 
+      String error = String.format("Gap in transactions, max txnid is %d"
+                                   + ", 0 txns from %d", max, fromTxId);
+      LOG.error(error);
+      throw new CorruptionException(error);
+    }
+
+    return numTxns;
+  }
+
+  @Override
+  synchronized public void recoverUnfinalizedSegments() throws IOException {
+    File currentDir = sd.getCurrentDir();
+    List<EditLogFile> allLogFiles = matchEditLogs(currentDir.listFiles());
+    
+    // make sure journal is aware of max seen transaction before moving corrupt 
+    // files aside
+    findMaxTransaction();
+
+    for (EditLogFile elf : allLogFiles) {
+      if (elf.getFile().equals(currentInProgress)) {
+        continue;
+      }
+      if (elf.isInProgress()) {
+        elf.validateLog();
+
+        if (elf.isCorrupt()) {
+          elf.moveAsideCorruptFile();
+          continue;
+        }
+        finalizeLogSegment(elf.getFirstTxId(), elf.getLastTxId());
+      }
+    }
+  }
+
+  private List<EditLogFile> getLogFiles(long fromTxId) throws IOException {
+    File currentDir = sd.getCurrentDir();
+    List<EditLogFile> allLogFiles = matchEditLogs(currentDir.listFiles());
+    List<EditLogFile> logFiles = Lists.newArrayList();
+    
+    for (EditLogFile elf : allLogFiles) {
+      if (fromTxId > elf.getFirstTxId()
+          && fromTxId <= elf.getLastTxId()) {
+        throw new IOException("Asked for fromTxId " + fromTxId
+            + " which is in middle of file " + elf.file);
+      }
+      if (fromTxId <= elf.getFirstTxId()) {
+        logFiles.add(elf);
+      }
+    }
+    
+    Collections.sort(logFiles, EditLogFile.COMPARE_BY_START_TXID);
+
+    return logFiles;
+  }
+
+  /** 
+   * Find the maximum transaction in the journal.
+   * This gets stored in a member variable, as corrupt edit logs
+   * will be moved aside, but we still need to remember their first
+   * tranaction id in the case that it was the maximum transaction in
+   * the journal.
+   */
+  private long findMaxTransaction()
+      throws IOException {
+    for (EditLogFile elf : getLogFiles(0)) {
+      if (elf.isInProgress()) {
+        maxSeenTransaction = Math.max(elf.getFirstTxId(), maxSeenTransaction);
+        elf.validateLog();
+      }
+      maxSeenTransaction = Math.max(elf.getLastTxId(), maxSeenTransaction);
+    }
+    return maxSeenTransaction;
+  }
+
+  @Override
+  public String toString() {
+    return String.format("FileJournalManager(root=%s)", sd.getRoot());
+  }
+
   /**
    * Record of an edit log that has been located and had its filename parsed.
    */
@@ -196,12 +333,10 @@
     private File file;
     private final long firstTxId;
     private long lastTxId;
-    
-    private EditLogValidation cachedValidation = null;
+
     private boolean isCorrupt = false;
-    
-    static final long UNKNOWN_END = -1;
-    
+    private final boolean isInProgress;
+
     final static Comparator<EditLogFile> COMPARE_BY_START_TXID 
       = new Comparator<EditLogFile>() {
       public int compare(EditLogFile a, EditLogFile b) {
@@ -214,30 +349,24 @@
 
     EditLogFile(File file,
         long firstTxId, long lastTxId) {
-      assert lastTxId == UNKNOWN_END || lastTxId >= firstTxId;
-      assert firstTxId > 0;
+      this(file, firstTxId, lastTxId, false);
+      assert (lastTxId != HdfsConstants.INVALID_TXID)
+        && (lastTxId >= firstTxId);
+    }
+    
+    EditLogFile(File file, long firstTxId, 
+                long lastTxId, boolean isInProgress) { 
+      assert (lastTxId == HdfsConstants.INVALID_TXID && isInProgress)
+        || (lastTxId != HdfsConstants.INVALID_TXID && lastTxId >= firstTxId);
+      assert (firstTxId > 0) || (firstTxId == HdfsConstants.INVALID_TXID);
       assert file != null;
       
       this.firstTxId = firstTxId;
       this.lastTxId = lastTxId;
       this.file = file;
+      this.isInProgress = isInProgress;
     }
     
-    public void finalizeLog() throws IOException {
-      long numTransactions = validateLog().numTransactions;
-      long lastTxId = firstTxId + numTransactions - 1;
-      File dst = new File(file.getParentFile(),
-          NNStorage.getFinalizedEditsFileName(firstTxId, lastTxId));
-      LOG.info("Finalizing edits log " + file + " by renaming to "
-          + dst.getName());
-      if (!file.renameTo(dst)) {
-        throw new IOException("Couldn't finalize log " +
-            file + " to " + dst);
-      }
-      this.lastTxId = lastTxId;
-      file = dst;
-    }
-
     long getFirstTxId() {
       return firstTxId;
     }
@@ -246,15 +375,22 @@
       return lastTxId;
     }
 
-    EditLogValidation validateLog() throws IOException {
-      if (cachedValidation == null) {
-        cachedValidation = EditLogFileInputStream.validateEditLog(file);
+    /** 
+     * Count the number of valid transactions in a log.
+     * This will update the lastTxId of the EditLogFile or
+     * mark it as corrupt if it is.
+     */
+    void validateLog() throws IOException {
+      EditLogValidation val = EditLogFileInputStream.validateEditLog(file);
+      if (val.getNumTransactions() == 0) {
+        markCorrupt();
+      } else {
+        this.lastTxId = val.getEndTxId();
       }
-      return cachedValidation;
     }
 
     boolean isInProgress() {
-      return (lastTxId == UNKNOWN_END);
+      return isInProgress;
     }
 
     File getFile() {
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/INode.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/INode.java
index 9885f23..c4bc1cc 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/INode.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/INode.java
@@ -261,13 +261,13 @@
     this.name = name;
   }
 
-  /** {@inheritDoc} */
+  @Override
   public String getFullPathName() {
     // Get the full path name of this inode.
     return FSDirectory.getFullPathName(this);
   }
 
-  /** {@inheritDoc} */
+  @Override
   public String toString() {
     return "\"" + getFullPathName() + "\":"
     + getUserName() + ":" + getGroupName() + ":"
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/INodeDirectory.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/INodeDirectory.java
index f4d9e78..0f1a8a2 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/INodeDirectory.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/INodeDirectory.java
@@ -372,7 +372,7 @@
     return parent;
   }
 
-  /** {@inheritDoc} */
+  @Override
   DirCounts spaceConsumedInTree(DirCounts counts) {
     counts.nsCount += 1;
     if (children != null) {
@@ -383,7 +383,7 @@
     return counts;    
   }
 
-  /** {@inheritDoc} */
+  @Override
   long[] computeContentSummary(long[] summary) {
     // Walk through the children of this node, using a new summary array
     // for the (sub)tree rooted at this node
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/INodeFile.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/INodeFile.java
index 7077dc1..2b3a012 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/INodeFile.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/INodeFile.java
@@ -168,7 +168,7 @@
     return 1;
   }
 
-  /** {@inheritDoc} */
+  @Override
   long[] computeContentSummary(long[] summary) {
     summary[0] += computeFileSize(true);
     summary[1]++;
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/JournalManager.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/JournalManager.java
index f2138ff..64c4ce7 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/JournalManager.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/JournalManager.java
@@ -39,6 +39,25 @@
    */
   void finalizeLogSegment(long firstTxId, long lastTxId) throws IOException;
 
+   /**
+   * Get the input stream starting with fromTxnId from this journal manager
+   * @param fromTxnId the first transaction id we want to read
+   * @return the stream starting with transaction fromTxnId
+   * @throws IOException if a stream cannot be found.
+   */
+  EditLogInputStream getInputStream(long fromTxnId) throws IOException;
+
+  /**
+   * Get the number of transaction contiguously available from fromTxnId.
+   *
+   * @param fromTxnId Transaction id to count from
+   * @return The number of transactions available from fromTxnId
+   * @throws IOException if the journal cannot be read.
+   * @throws CorruptionException if there is a gap in the journal at fromTxnId.
+   */
+  long getNumberOfTransactions(long fromTxnId) 
+      throws IOException, CorruptionException;
+
   /**
    * Set the amount of memory that this stream should use to buffer edits
    */
@@ -57,10 +76,21 @@
     throws IOException;
 
   /**
-   * @return an EditLogInputStream that reads from the same log that
-   * the edit log is currently writing. May return null if this journal
-   * manager does not support this operation.
-   */  
-  EditLogInputStream getInProgressInputStream(long segmentStartsAtTxId)
-    throws IOException;
+   * Recover segments which have not been finalized.
+   */
+  void recoverUnfinalizedSegments() throws IOException;
+
+  /** 
+   * Indicate that a journal is cannot be used to load a certain range of 
+   * edits.
+   * This exception occurs in the case of a gap in the transactions, or a
+   * corrupt edit file.
+   */
+  public static class CorruptionException extends IOException {
+    static final long serialVersionUID = -4687802717006172702L;
+    
+    public CorruptionException(String reason) {
+      super(reason);
+    }
+  }
 }
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/LeaseManager.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/LeaseManager.java
index 4485773..6f03452 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/LeaseManager.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/LeaseManager.java
@@ -245,13 +245,13 @@
       return paths.remove(src);
     }
 
-    /** {@inheritDoc} */
+    @Override
     public String toString() {
       return "[Lease.  Holder: " + holder
           + ", pendingcreates: " + paths.size() + "]";
     }
   
-    /** {@inheritDoc} */
+    @Override
     public int compareTo(Lease o) {
       Lease l1 = this;
       Lease l2 = o;
@@ -266,7 +266,7 @@
       }
     }
   
-    /** {@inheritDoc} */
+    @Override
     public boolean equals(Object o) {
       if (!(o instanceof Lease)) {
         return false;
@@ -279,7 +279,7 @@
       return false;
     }
   
-    /** {@inheritDoc} */
+    @Override
     public int hashCode() {
       return holder.hashCode();
     }
@@ -429,7 +429,7 @@
     }
   }
 
-  /** {@inheritDoc} */
+  @Override
   public synchronized String toString() {
     return getClass().getSimpleName() + "= {"
         + "\n leases=" + leases
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/NameNode.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/NameNode.java
index f2ed1ec..ca7c4fd 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/NameNode.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/NameNode.java
@@ -296,7 +296,7 @@
   }
 
   protected void loadNamesystem(Configuration conf) throws IOException {
-    this.namesystem = new FSNamesystem(conf);
+    this.namesystem = FSNamesystem.loadFromDisk(conf);
   }
 
   NamenodeRegistration getRegistration() {
@@ -602,16 +602,16 @@
     }
     System.out.println("Formatting using clusterid: " + clusterId);
     
-    FSImage fsImage = new FSImage(conf, null, dirsToFormat, editDirsToFormat);
-    FSNamesystem nsys = new FSNamesystem(fsImage, conf);
-    nsys.dir.fsImage.format(clusterId);
+    FSImage fsImage = new FSImage(conf, dirsToFormat, editDirsToFormat);
+    FSNamesystem fsn = new FSNamesystem(conf, fsImage);
+    fsImage.format(fsn, clusterId);
     return false;
   }
 
   private static boolean finalize(Configuration conf,
                                boolean isConfirmationNeeded
                                ) throws IOException {
-    FSNamesystem nsys = new FSNamesystem(new FSImage(conf), conf);
+    FSNamesystem nsys = new FSNamesystem(conf, new FSImage(conf));
     System.err.print(
         "\"finalize\" will remove the previous state of the files system.\n"
         + "Recent upgrade will become permanent.\n"
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/NameNodeHttpServer.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/NameNodeHttpServer.java
index acee179..5cb267d 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/NameNodeHttpServer.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/NameNodeHttpServer.java
@@ -195,7 +195,9 @@
   }
   
   public void stop() throws Exception {
-    httpServer.stop();
+    if (httpServer != null) {
+      httpServer.stop();
+    }
   }
 
   public InetSocketAddress getHttpAddress() {
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/NameNodeResourceChecker.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/NameNodeResourceChecker.java
index ed638a73..4d7cfd8 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/NameNodeResourceChecker.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/NameNodeResourceChecker.java
@@ -64,7 +64,7 @@
 
     duReserved = conf.getLong(DFSConfigKeys.DFS_NAMENODE_DU_RESERVED_KEY,
         DFSConfigKeys.DFS_NAMENODE_DU_RESERVED_DEFAULT);
-
+  
     Collection<URI> extraCheckedVolumes = Util.stringCollectionAsURIs(conf
         .getTrimmedStringCollection(DFSConfigKeys.DFS_NAMENODE_CHECKED_VOLUMES_KEY));
 
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/NameNodeRpcServer.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/NameNodeRpcServer.java
index 087079e..7bd2b44 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/NameNodeRpcServer.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/NameNodeRpcServer.java
@@ -38,6 +38,8 @@
 import org.apache.hadoop.fs.permission.FsPermission;
 import org.apache.hadoop.fs.permission.PermissionStatus;
 import static org.apache.hadoop.hdfs.DFSConfigKeys.*;
+
+import org.apache.hadoop.hdfs.DFSUtil;
 import org.apache.hadoop.hdfs.HDFSPolicyProvider;
 import org.apache.hadoop.hdfs.HdfsConfiguration;
 import org.apache.hadoop.hdfs.protocol.Block;
@@ -57,6 +59,24 @@
 import org.apache.hadoop.hdfs.protocol.HdfsConstants.DatanodeReportType;
 import org.apache.hadoop.hdfs.protocol.HdfsConstants.SafeModeAction;
 import org.apache.hadoop.hdfs.protocol.HdfsConstants.UpgradeAction;
+import org.apache.hadoop.hdfs.protocol.proto.ClientNamenodeProtocolProtos.ClientNamenodeProtocol;
+import org.apache.hadoop.hdfs.protocol.proto.NamenodeProtocolProtos.NamenodeProtocolService;
+import org.apache.hadoop.hdfs.protocol.proto.DatanodeProtocolProtos.DatanodeProtocolService;
+import org.apache.hadoop.hdfs.protocol.proto.GetUserMappingsProtocolProtos.GetUserMappingsProtocolService;
+import org.apache.hadoop.hdfs.protocol.proto.RefreshAuthorizationPolicyProtocolProtos.RefreshAuthorizationPolicyProtocolService;
+import org.apache.hadoop.hdfs.protocol.proto.RefreshUserMappingsProtocolProtos.RefreshUserMappingsProtocolService;
+import org.apache.hadoop.hdfs.protocolPB.DatanodeProtocolPB;
+import org.apache.hadoop.hdfs.protocolPB.DatanodeProtocolServerSideTranslatorPB;
+import org.apache.hadoop.hdfs.protocolPB.GetUserMappingsProtocolPB;
+import org.apache.hadoop.hdfs.protocolPB.GetUserMappingsProtocolServerSideTranslatorPB;
+import org.apache.hadoop.hdfs.protocolPB.NamenodeProtocolPB;
+import org.apache.hadoop.hdfs.protocolPB.NamenodeProtocolServerSideTranslatorPB;
+import org.apache.hadoop.hdfs.protocolPB.ClientNamenodeProtocolPB;
+import org.apache.hadoop.hdfs.protocolPB.ClientNamenodeProtocolServerSideTranslatorPB;
+import org.apache.hadoop.hdfs.protocolPB.RefreshAuthorizationPolicyProtocolPB;
+import org.apache.hadoop.hdfs.protocolPB.RefreshAuthorizationPolicyProtocolServerSideTranslatorPB;
+import org.apache.hadoop.hdfs.protocolPB.RefreshUserMappingsProtocolPB;
+import org.apache.hadoop.hdfs.protocolPB.RefreshUserMappingsProtocolServerSideTranslatorPB;
 import org.apache.hadoop.hdfs.security.token.block.ExportedBlockKeys;
 import org.apache.hadoop.hdfs.security.token.delegation.DelegationTokenIdentifier;
 import org.apache.hadoop.hdfs.server.common.IncorrectVersionException;
@@ -68,6 +88,8 @@
 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.FinalizeCommand;
 import org.apache.hadoop.hdfs.server.protocol.NamenodeCommand;
 import org.apache.hadoop.hdfs.server.protocol.NamenodeProtocol;
 import org.apache.hadoop.hdfs.server.protocol.NamenodeProtocols;
@@ -75,12 +97,17 @@
 import org.apache.hadoop.hdfs.server.protocol.NamespaceInfo;
 import org.apache.hadoop.hdfs.server.protocol.NodeRegistration;
 import org.apache.hadoop.hdfs.server.protocol.RemoteEditLogManifest;
+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.EnumSetWritable;
 import org.apache.hadoop.io.Text;
+import org.apache.hadoop.ipc.ProtobufRpcEngine;
 import org.apache.hadoop.ipc.ProtocolSignature;
 import org.apache.hadoop.ipc.RPC;
 import org.apache.hadoop.ipc.Server;
+import org.apache.hadoop.ipc.WritableRpcEngine;
 import org.apache.hadoop.net.Node;
 import org.apache.hadoop.security.AccessControlException;
 import org.apache.hadoop.security.Groups;
@@ -93,6 +120,8 @@
 import org.apache.hadoop.security.token.SecretManager.InvalidToken;
 import org.apache.hadoop.tools.GetUserMappingsProtocol;
 
+import com.google.protobuf.BlockingService;
+
 /**
  * This class is responsible for handling all of the RPC calls to the NameNode.
  * It is created, started, and stopped by {@link NameNode}.
@@ -114,8 +143,8 @@
   private final InetSocketAddress serviceRPCAddress;
   
   /** The RPC server that listens to requests from clients */
-  protected final RPC.Server server;
-  protected final InetSocketAddress rpcAddress;
+  protected final RPC.Server clientRpcServer;
+  protected final InetSocketAddress clientRpcAddress;
 
   public NameNodeRpcServer(Configuration conf, NameNode nn)
       throws IOException {
@@ -127,46 +156,107 @@
       conf.getInt(DFS_NAMENODE_HANDLER_COUNT_KEY, 
                   DFS_NAMENODE_HANDLER_COUNT_DEFAULT);
     InetSocketAddress socAddr = nn.getRpcServerAddress(conf);
+		RPC.setProtocolEngine(conf, ClientNamenodeProtocolPB.class,
+         ProtobufRpcEngine.class);
+     ClientNamenodeProtocolServerSideTranslatorPB 
+       clientProtocolServerTranslator = 
+         new ClientNamenodeProtocolServerSideTranslatorPB(this);
+     BlockingService clientNNPbService = ClientNamenodeProtocol.
+         newReflectiveBlockingService(clientProtocolServerTranslator);
+    
+    DatanodeProtocolServerSideTranslatorPB dnProtoPbTranslator = 
+        new DatanodeProtocolServerSideTranslatorPB(this);
+    BlockingService dnProtoPbService = DatanodeProtocolService
+        .newReflectiveBlockingService(dnProtoPbTranslator);
 
+    NamenodeProtocolServerSideTranslatorPB namenodeProtocolXlator = 
+        new NamenodeProtocolServerSideTranslatorPB(this);
+	  BlockingService NNPbService = NamenodeProtocolService
+          .newReflectiveBlockingService(namenodeProtocolXlator);
+	  
+    RefreshAuthorizationPolicyProtocolServerSideTranslatorPB refreshAuthPolicyXlator = 
+        new RefreshAuthorizationPolicyProtocolServerSideTranslatorPB(this);
+    BlockingService refreshAuthService = RefreshAuthorizationPolicyProtocolService
+        .newReflectiveBlockingService(refreshAuthPolicyXlator);
+
+    RefreshUserMappingsProtocolServerSideTranslatorPB refreshUserMappingXlator = 
+        new RefreshUserMappingsProtocolServerSideTranslatorPB(this);
+    BlockingService refreshUserMappingService = RefreshUserMappingsProtocolService
+        .newReflectiveBlockingService(refreshUserMappingXlator);
+
+    GetUserMappingsProtocolServerSideTranslatorPB getUserMappingXlator = 
+        new GetUserMappingsProtocolServerSideTranslatorPB(this);
+    BlockingService getUserMappingService = GetUserMappingsProtocolService
+        .newReflectiveBlockingService(getUserMappingXlator);
+	  
+    WritableRpcEngine.ensureInitialized();
+    
     InetSocketAddress dnSocketAddr = nn.getServiceRpcServerAddress(conf);
     if (dnSocketAddr != null) {
       int serviceHandlerCount =
         conf.getInt(DFS_NAMENODE_SERVICE_HANDLER_COUNT_KEY,
                     DFS_NAMENODE_SERVICE_HANDLER_COUNT_DEFAULT);
-      this.serviceRpcServer = RPC.getServer(NamenodeProtocols.class, this,
-          dnSocketAddr.getHostName(), dnSocketAddr.getPort(), serviceHandlerCount,
+      // Add all the RPC protocols that the namenode implements
+      this.serviceRpcServer = 
+          RPC.getServer(org.apache.hadoop.hdfs.protocolPB.
+              ClientNamenodeProtocolPB.class, clientNNPbService,
+          dnSocketAddr.getHostName(), dnSocketAddr.getPort(), 
+          serviceHandlerCount,
           false, conf, namesystem.getDelegationTokenSecretManager());
+      DFSUtil.addPBProtocol(conf, NamenodeProtocolPB.class, NNPbService,
+          serviceRpcServer);
+      DFSUtil.addPBProtocol(conf, DatanodeProtocolPB.class, dnProtoPbService,
+          serviceRpcServer);
+      DFSUtil.addPBProtocol(conf, RefreshAuthorizationPolicyProtocolPB.class,
+          refreshAuthService, serviceRpcServer);
+      DFSUtil.addPBProtocol(conf, RefreshUserMappingsProtocolPB.class, 
+          refreshUserMappingService, serviceRpcServer);
+      DFSUtil.addPBProtocol(conf, GetUserMappingsProtocolPB.class, 
+          getUserMappingService, serviceRpcServer);
+  
       this.serviceRPCAddress = this.serviceRpcServer.getListenerAddress();
       nn.setRpcServiceServerAddress(conf, serviceRPCAddress);
     } else {
       serviceRpcServer = null;
       serviceRPCAddress = null;
     }
-    this.server = RPC.getServer(NamenodeProtocols.class, this,
-                                socAddr.getHostName(), socAddr.getPort(),
-                                handlerCount, false, conf, 
-                                namesystem.getDelegationTokenSecretManager());
+    // Add all the RPC protocols that the namenode implements
+    this.clientRpcServer = RPC.getServer(
+        org.apache.hadoop.hdfs.protocolPB.ClientNamenodeProtocolPB.class, 
+        clientNNPbService, socAddr.getHostName(),
+            socAddr.getPort(), handlerCount, false, conf,
+            namesystem.getDelegationTokenSecretManager());
+    DFSUtil.addPBProtocol(conf, NamenodeProtocolPB.class, NNPbService,
+        clientRpcServer);
+    DFSUtil.addPBProtocol(conf, DatanodeProtocolPB.class, dnProtoPbService,
+        clientRpcServer);
+    DFSUtil.addPBProtocol(conf, RefreshAuthorizationPolicyProtocolPB.class, 
+        refreshAuthService, clientRpcServer);
+    DFSUtil.addPBProtocol(conf, RefreshUserMappingsProtocolPB.class, 
+        refreshUserMappingService, clientRpcServer);
+    DFSUtil.addPBProtocol(conf, GetUserMappingsProtocolPB.class, 
+        getUserMappingService, clientRpcServer);
 
     // set service-level authorization security policy
     if (serviceAuthEnabled =
           conf.getBoolean(
             CommonConfigurationKeys.HADOOP_SECURITY_AUTHORIZATION, false)) {
-      this.server.refreshServiceAcl(conf, new HDFSPolicyProvider());
+      this.clientRpcServer.refreshServiceAcl(conf, new HDFSPolicyProvider());
       if (this.serviceRpcServer != null) {
         this.serviceRpcServer.refreshServiceAcl(conf, new HDFSPolicyProvider());
       }
     }
 
     // The rpc-server port can be ephemeral... ensure we have the correct info
-    this.rpcAddress = this.server.getListenerAddress(); 
-    nn.setRpcServerAddress(conf, rpcAddress);
+    this.clientRpcAddress = this.clientRpcServer.getListenerAddress(); 
+    nn.setRpcServerAddress(conf, clientRpcAddress);
   }
   
   /**
    * Actually start serving requests.
    */
   void start() {
-    server.start();  //start RPC server
+    clientRpcServer.start();  //start RPC server
     if (serviceRpcServer != null) {
       serviceRpcServer.start();      
     }
@@ -176,11 +266,11 @@
    * Wait until the RPC server has shut down.
    */
   void join() throws InterruptedException {
-    this.server.join();
+    this.clientRpcServer.join();
   }
   
   void stop() {
-    if(server != null) server.stop();
+    if(clientRpcServer != null) clientRpcServer.stop();
     if(serviceRpcServer != null) serviceRpcServer.stop();
   }
   
@@ -189,7 +279,7 @@
   }
 
   InetSocketAddress getRpcAddress() {
-    return rpcAddress;
+    return clientRpcAddress;
   }
   
   @Override // VersionedProtocol
@@ -203,7 +293,8 @@
   public long getProtocolVersion(String protocol, 
                                  long clientVersion) throws IOException {
     if (protocol.equals(ClientProtocol.class.getName())) {
-      return ClientProtocol.versionID; 
+      throw new IOException("Old Namenode Client protocol is not supported:" + 
+      protocol + "Switch your clientside to " + ClientNamenodeProtocol.class); 
     } else if (protocol.equals(DatanodeProtocol.class.getName())){
       return DatanodeProtocol.versionID;
     } else if (protocol.equals(NamenodeProtocol.class.getName())){
@@ -756,8 +847,8 @@
 
 
   @Override // DatanodeProtocol
-  public DatanodeRegistration registerDatanode(DatanodeRegistration nodeReg)
-      throws IOException {
+  public DatanodeRegistration registerDatanode(DatanodeRegistration nodeReg,
+      DatanodeStorage[] storages) throws IOException {
     verifyVersion(nodeReg.getVersion());
     namesystem.registerDatanode(nodeReg);
       
@@ -766,19 +857,20 @@
 
   @Override // DatanodeProtocol
   public DatanodeCommand[] sendHeartbeat(DatanodeRegistration nodeReg,
-      long capacity, long dfsUsed, long remaining, long blockPoolUsed,
-      int xmitsInProgress, int xceiverCount, int failedVolumes)
-      throws IOException {
+      StorageReport[] report, int xmitsInProgress, int xceiverCount,
+      int failedVolumes) throws IOException {
     verifyRequest(nodeReg);
-    return namesystem.handleHeartbeat(nodeReg, capacity, dfsUsed, remaining,
-        blockPoolUsed, xceiverCount, xmitsInProgress, failedVolumes);
+    return namesystem.handleHeartbeat(nodeReg, report[0].getCapacity(),
+        report[0].getDfsUsed(), report[0].getRemaining(),
+        report[0].getBlockPoolUsed(), xceiverCount, xmitsInProgress,
+        failedVolumes);
   }
 
   @Override // DatanodeProtocol
   public DatanodeCommand blockReport(DatanodeRegistration nodeReg,
-      String poolId, long[] blocks) throws IOException {
+      String poolId, StorageBlockReport[] reports) throws IOException {
     verifyRequest(nodeReg);
-    BlockListAsLongs blist = new BlockListAsLongs(blocks);
+    BlockListAsLongs blist = new BlockListAsLongs(reports[0].getBlocks());
     if(stateChangeLog.isDebugEnabled()) {
       stateChangeLog.debug("*BLOCK* NameNode.blockReport: "
            + "from " + nodeReg.getName() + " " + blist.getNumberOfBlocks()
@@ -787,22 +879,21 @@
 
     namesystem.getBlockManager().processReport(nodeReg, poolId, blist);
     if (nn.getFSImage().isUpgradeFinalized())
-      return new DatanodeCommand.Finalize(poolId);
+      return new FinalizeCommand(poolId);
     return null;
   }
 
   @Override // DatanodeProtocol
-  public void blockReceived(DatanodeRegistration nodeReg, String poolId,
-      Block blocks[], String delHints[]) throws IOException {
+  public void blockReceivedAndDeleted(DatanodeRegistration nodeReg, String poolId,
+      StorageReceivedDeletedBlocks[] receivedAndDeletedBlocks) throws IOException {
     verifyRequest(nodeReg);
     if(stateChangeLog.isDebugEnabled()) {
-      stateChangeLog.debug("*BLOCK* NameNode.blockReceived: "
-          +"from "+nodeReg.getName()+" "+blocks.length+" blocks.");
+      stateChangeLog.debug("*BLOCK* NameNode.blockReceivedAndDeleted: "
+          +"from "+nodeReg.getName()+" "+receivedAndDeletedBlocks.length
+          +" blocks.");
     }
-    for (int i = 0; i < blocks.length; i++) {
-      namesystem.getBlockManager().blockReceived(
-          nodeReg, poolId, blocks[i], delHints[i]);
-    }
+    namesystem.getBlockManager().blockReceivedAndDeleted(
+        nodeReg, poolId, receivedAndDeletedBlocks[0].getBlocks());
   }
   
   @Override // DatanodeProtocol
@@ -862,7 +953,7 @@
       throw new AuthorizationException("Service Level Authorization not enabled!");
     }
 
-    this.server.refreshServiceAcl(new Configuration(), new HDFSPolicyProvider());
+    this.clientRpcServer.refreshServiceAcl(new Configuration(), new HDFSPolicyProvider());
     if (this.serviceRpcServer != null) {
       this.serviceRpcServer.refreshServiceAcl(new Configuration(), new HDFSPolicyProvider());
     }
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/NamenodeFsck.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/NamenodeFsck.java
index 4e40c9a..b823a00 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/NamenodeFsck.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/NamenodeFsck.java
@@ -646,7 +646,7 @@
       return (float) (totalReplicas) / (float) totalBlocks;
     }
     
-    /** {@inheritDoc} */
+    @Override
     public String toString() {
       StringBuilder res = new StringBuilder();
       res.append("Status: ").append((isHealthy() ? "HEALTHY" : "CORRUPT"))
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/SecondaryNameNode.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/SecondaryNameNode.java
index 9fba7d6..50dca62 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/SecondaryNameNode.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/SecondaryNameNode.java
@@ -48,6 +48,7 @@
 import org.apache.hadoop.hdfs.DFSUtil.ErrorSimulator;
 import org.apache.hadoop.hdfs.HdfsConfiguration;
 import org.apache.hadoop.hdfs.protocol.HdfsConstants;
+import org.apache.hadoop.hdfs.protocolPB.NamenodeProtocolTranslatorPB;
 import org.apache.hadoop.hdfs.server.common.HdfsServerConstants;
 import org.apache.hadoop.hdfs.server.common.InconsistentFSStateException;
 import org.apache.hadoop.hdfs.server.common.JspHelper;
@@ -58,7 +59,6 @@
 import org.apache.hadoop.hdfs.server.protocol.RemoteEditLogManifest;
 import org.apache.hadoop.http.HttpServer;
 import org.apache.hadoop.io.MD5Hash;
-import org.apache.hadoop.ipc.RPC;
 import org.apache.hadoop.ipc.RemoteException;
 import org.apache.hadoop.metrics2.lib.DefaultMetricsSystem;
 import org.apache.hadoop.metrics2.source.JvmMetrics;
@@ -123,8 +123,10 @@
   /** checkpoint once every this many transactions, regardless of time */
   private long checkpointTxnCount;
 
+  private FSNamesystem namesystem;
 
-  /** {@inheritDoc} */
+
+  @Override
   public String toString() {
     return getClass().getSimpleName() + " Status" 
       + "\nName Node Address    : " + nameNodeAddr   
@@ -214,9 +216,8 @@
     nameNodeAddr = NameNode.getServiceAddress(conf, true);
 
     this.conf = conf;
-    this.namenode =
-        (NamenodeProtocol) RPC.waitForProxy(NamenodeProtocol.class,
-            NamenodeProtocol.versionID, nameNodeAddr, conf);
+    this.namenode = new NamenodeProtocolTranslatorPB(nameNodeAddr, conf,
+        UserGroupInformation.getCurrentUser());
 
     // initialize checkpoint directories
     fsName = getInfoServer();
@@ -226,6 +227,8 @@
                                   "/tmp/hadoop/dfs/namesecondary");    
     checkpointImage = new CheckpointStorage(conf, checkpointDirs, checkpointEditsDirs);
     checkpointImage.recoverCreate(commandLineOpts.shouldFormat());
+    
+    namesystem = new FSNamesystem(conf, checkpointImage);
 
     // Initialize other scheduling parameters from the configuration
     checkpointCheckPeriod = conf.getLong(
@@ -527,7 +530,7 @@
 
     boolean loadImage = downloadCheckpointFiles(
         fsName, checkpointImage, sig, manifest);   // Fetch fsimage and edits
-    doMerge(sig, manifest, loadImage, checkpointImage);
+    doMerge(sig, manifest, loadImage, checkpointImage, namesystem);
     
     //
     // Upload the new image into the NameNode. Then tell the Namenode
@@ -757,8 +760,7 @@
     CheckpointStorage(Configuration conf, 
                       Collection<URI> imageDirs,
                       Collection<URI> editsDirs) throws IOException {
-      super(conf, (FSNamesystem)null, imageDirs, editsDirs);
-      setFSNamesystem(new FSNamesystem(this, conf));
+      super(conf, imageDirs, editsDirs);
       
       // the 2NN never writes edits -- it only downloads them. So
       // we shouldn't have any editLog instance. Setting to null
@@ -844,7 +846,8 @@
     
   static void doMerge(
       CheckpointSignature sig, RemoteEditLogManifest manifest,
-      boolean loadImage, FSImage dstImage) throws IOException {   
+      boolean loadImage, FSImage dstImage, FSNamesystem dstNamesystem)
+      throws IOException {   
     NNStorage dstStorage = dstImage.getStorage();
     
     dstStorage.setStorageInfo(sig);
@@ -855,11 +858,11 @@
             sig.mostRecentCheckpointTxId + " even though it should have " +
             "just been downloaded");
       }
-      dstImage.reloadFromImageFile(file);
+      dstImage.reloadFromImageFile(file, dstNamesystem);
     }
     
-    Checkpointer.rollForwardByApplyingLogs(manifest, dstImage);
-    dstImage.saveFSImageInAllDirs(dstImage.getLastAppliedTxId());
+    Checkpointer.rollForwardByApplyingLogs(manifest, dstImage, dstNamesystem);
+    dstImage.saveFSImageInAllDirs(dstNamesystem, dstImage.getLastAppliedTxId());
     dstStorage.writeAll();
   }
 }
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/SerialNumberManager.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/SerialNumberManager.java
index d042e15..9a71184 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/SerialNumberManager.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/SerialNumberManager.java
@@ -64,7 +64,7 @@
       return i2t.get(i);
     }
 
-    /** {@inheritDoc} */
+    @Override
     public String toString() {
       return "max=" + max + ",\n  t2i=" + t2i + ",\n  i2t=" + i2t;
     }
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/protocol/BlockCommand.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/protocol/BlockCommand.java
index b4e4ab7..cfbfb0a 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/protocol/BlockCommand.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/protocol/BlockCommand.java
@@ -44,6 +44,16 @@
 @InterfaceAudience.Private
 @InterfaceStability.Evolving
 public class BlockCommand extends DatanodeCommand {
+  
+  /**
+   * This constant is used to indicate that the block deletion does not need
+   * explicit ACK from the datanode. When a block is put into the list of blocks
+   * to be deleted, it's size is set to this constant. We assume that no block
+   * would actually have this size. Otherwise, we would miss ACKs for blocks
+   * with such size. Positive number is used for compatibility reasons.
+   */
+  public static final long NO_ACK = Long.MAX_VALUE;
+  
   String poolId;
   Block blocks[];
   DatanodeInfo targets[][];
@@ -57,7 +67,6 @@
   public BlockCommand(int action, String poolId,
       List<BlockTargetPair> blocktargetlist) {
     super(action);
-
     this.poolId = poolId;
     blocks = new Block[blocktargetlist.size()]; 
     targets = new DatanodeInfo[blocks.length][];
@@ -75,12 +84,21 @@
    * @param blocks blocks related to the action
    */
   public BlockCommand(int action, String poolId, Block blocks[]) {
+    this(action, poolId, blocks, EMPTY_TARGET);
+  }
+
+  /**
+   * Create BlockCommand for the given action
+   * @param blocks blocks related to the action
+   */
+  public BlockCommand(int action, String poolId, Block[] blocks,
+      DatanodeInfo[][] targets) {
     super(action);
     this.poolId = poolId;
     this.blocks = blocks;
-    this.targets = EMPTY_TARGET;
+    this.targets = targets;
   }
-
+  
   public String getBlockPoolId() {
     return poolId;
   }
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/protocol/BlockRecoveryCommand.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/protocol/BlockRecoveryCommand.java
index 992deb8..5f2ae8e 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/protocol/BlockRecoveryCommand.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/protocol/BlockRecoveryCommand.java
@@ -32,6 +32,8 @@
 import org.apache.hadoop.io.WritableFactories;
 import org.apache.hadoop.io.WritableFactory;
 
+import com.google.common.base.Joiner;
+
 /**
  * BlockRecoveryCommand is an instruction to a data-node to recover
  * the specified blocks.
@@ -117,8 +119,12 @@
    * the specified capacity for recovering blocks.
    */
   public BlockRecoveryCommand(int capacity) {
+    this(new ArrayList<RecoveringBlock>(capacity));
+  }
+  
+  public BlockRecoveryCommand(Collection<RecoveringBlock> blocks) {
     super(DatanodeProtocol.DNA_RECOVERBLOCK);
-    recoveringBlocks = new ArrayList<RecoveringBlock>(capacity);
+    recoveringBlocks = blocks;
   }
 
   /**
@@ -134,6 +140,15 @@
   public void add(RecoveringBlock block) {
     recoveringBlocks.add(block);
   }
+  
+  @Override
+  public String toString() {
+    StringBuilder sb = new StringBuilder();
+    sb.append("BlockRecoveryCommand(\n  ");
+    Joiner.on("\n  ").appendTo(sb, recoveringBlocks);
+    sb.append("\n)");
+    return sb.toString();
+  }
 
   ///////////////////////////////////////////
   // Writable
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/protocol/DatanodeCommand.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/protocol/DatanodeCommand.java
index 52396d2..6d59274 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/protocol/DatanodeCommand.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/protocol/DatanodeCommand.java
@@ -17,74 +17,16 @@
  */
 package org.apache.hadoop.hdfs.server.protocol;
 
-import java.io.DataInput;
-import java.io.DataOutput;
-import java.io.IOException;
-
 import org.apache.hadoop.classification.InterfaceAudience;
 import org.apache.hadoop.classification.InterfaceStability;
-import org.apache.hadoop.io.Writable;
-import org.apache.hadoop.io.WritableFactory;
-import org.apache.hadoop.io.WritableFactories;
-import org.apache.hadoop.io.WritableUtils;
-import org.apache.avro.reflect.Union;
 
 /**
  * Base class for data-node command.
  * Issued by the name-node to notify data-nodes what should be done.
  */
-
-// Declare subclasses for Avro's denormalized representation
-@Union({Void.class,
-      DatanodeCommand.Register.class, DatanodeCommand.Finalize.class,
-      BlockCommand.class, UpgradeCommand.class,
-      BlockRecoveryCommand.class, KeyUpdateCommand.class})
-
 @InterfaceAudience.Private
 @InterfaceStability.Evolving
 public abstract class DatanodeCommand extends ServerCommand {
-  static class Register extends DatanodeCommand {
-    private Register() {super(DatanodeProtocol.DNA_REGISTER);}
-    public void readFields(DataInput in) {}
-    public void write(DataOutput out) {}
-  }
-
-  public static class Finalize extends DatanodeCommand {
-    String blockPoolId;
-    private Finalize() {
-      super(DatanodeProtocol.DNA_FINALIZE);
-    }
-    
-    public Finalize(String bpid) {
-      super(DatanodeProtocol.DNA_FINALIZE);
-      blockPoolId = bpid;
-    }
-    
-    public String getBlockPoolId() {
-      return blockPoolId;
-    }
-    
-    public void readFields(DataInput in) throws IOException {
-      blockPoolId = WritableUtils.readString(in);
-    }
-    public void write(DataOutput out) throws IOException {
-      WritableUtils.writeString(out, blockPoolId);
-    }
-  }
-
-  static {                                      // register a ctor
-    WritableFactories.setFactory(Register.class,
-        new WritableFactory() {
-          public Writable newInstance() {return new Register();}
-        });
-    WritableFactories.setFactory(Finalize.class,
-        new WritableFactory() {
-          public Writable newInstance() {return new Finalize();}
-        });
-  }
-
-  public static final DatanodeCommand REGISTER = new Register();
-  
   public DatanodeCommand() {
     super();
   }
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/protocol/DatanodeProtocol.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/protocol/DatanodeProtocol.java
index f60bef7..a12b042 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/protocol/DatanodeProtocol.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/protocol/DatanodeProtocol.java
@@ -22,15 +22,12 @@
 
 import org.apache.hadoop.classification.InterfaceAudience;
 import org.apache.hadoop.hdfs.DFSConfigKeys;
-import org.apache.hadoop.hdfs.protocol.Block;
 import org.apache.hadoop.hdfs.protocol.ExtendedBlock;
 import org.apache.hadoop.hdfs.protocol.DatanodeID;
 import org.apache.hadoop.hdfs.protocol.LocatedBlock;
 import org.apache.hadoop.ipc.VersionedProtocol;
 import org.apache.hadoop.security.KerberosInfo;
 
-import org.apache.avro.reflect.Nullable;
-
 /**********************************************************************
  * Protocol that a DFS datanode uses to communicate with the NameNode.
  * It's used to upload current load information and block reports.
@@ -45,7 +42,15 @@
 @InterfaceAudience.Private
 public interface DatanodeProtocol extends VersionedProtocol {
   /**
-   * 28: Add Balancer Bandwidth Command protocol.
+   * This class is used by both the Namenode (client) and BackupNode (server) 
+   * to insulate from the protocol serialization.
+   * 
+   * If you are adding/changing DN's interface then you need to 
+   * change both this class and ALSO related protocol buffer
+   * wire protocol definition in DatanodeProtocol.proto.
+   * 
+   * For more details on protocol buffer wire protocol, please see 
+   * .../org/apache/hadoop/hdfs/protocolPB/overview.html
    */
   public static final long versionID = 28L;
   
@@ -68,18 +73,23 @@
   final static int DNA_RECOVERBLOCK = 6;  // request a block recovery
   final static int DNA_ACCESSKEYUPDATE = 7;  // update access key
   final static int DNA_BALANCERBANDWIDTHUPDATE = 8; // update balancer bandwidth
+  final static int DNA_UC_ACTION_REPORT_STATUS = 100; // Report upgrade status
+  final static int DNA_UC_ACTION_START_UPGRADE = 101; // start upgrade
 
   /** 
    * Register Datanode.
    *
    * @see org.apache.hadoop.hdfs.server.namenode.FSNamesystem#registerDatanode(DatanodeRegistration)
-   * 
+   * @param registration datanode registration information
+   * @param storages list of storages on the datanode``
    * @return updated {@link org.apache.hadoop.hdfs.server.protocol.DatanodeRegistration}, which contains 
    * new storageID if the datanode did not have one and
    * registration ID for further communication.
    */
-  public DatanodeRegistration registerDatanode(DatanodeRegistration registration
-                                       ) throws IOException;
+  public DatanodeRegistration registerDatanode(
+      DatanodeRegistration registration, DatanodeStorage[] storages)
+      throws IOException;
+  
   /**
    * sendHeartbeat() tells the NameNode that the DataNode is still
    * alive and well.  Includes some status info, too. 
@@ -88,20 +98,14 @@
    * A DatanodeCommand tells the DataNode to invalidate local block(s), 
    * or to copy them to other DataNodes, etc.
    * @param registration datanode registration information
-   * @param capacity total storage capacity available at the datanode
-   * @param dfsUsed storage used by HDFS
-   * @param remaining remaining storage available for HDFS
-   * @param blockPoolUsed storage used by the block pool
+   * @param reports utilization report per storage
    * @param xmitsInProgress number of transfers from this datanode to others
    * @param xceiverCount number of active transceiver threads
    * @param failedVolumes number of failed volumes
    * @throws IOException on error
    */
-  @Nullable
   public DatanodeCommand[] sendHeartbeat(DatanodeRegistration registration,
-                                       long capacity,
-                                       long dfsUsed, long remaining,
-                                       long blockPoolUsed,
+                                       StorageReport[] reports,
                                        int xmitsInProgress,
                                        int xceiverCount,
                                        int failedVolumes) throws IOException;
@@ -114,7 +118,7 @@
    * infrequently afterwards.
    * @param registration
    * @param poolId - the block pool ID for the blocks
-   * @param blocks - the block list as an array of longs.
+   * @param reports - report of blocks per storage
    *     Each block is represented as 2 longs.
    *     This is done instead of Block[] to reduce memory used by block reports.
    *     
@@ -122,21 +126,22 @@
    * @throws IOException
    */
   public DatanodeCommand blockReport(DatanodeRegistration registration,
-                                     String poolId,
-                                     long[] blocks) throws IOException;
+      String poolId, StorageBlockReport[] reports) throws IOException;
     
   /**
-   * blockReceived() allows the DataNode to tell the NameNode about
-   * recently-received block data, with a hint for pereferred replica
-   * to be deleted when there is any excessive blocks.
+   * blockReceivedAndDeleted() allows the DataNode to tell the NameNode about
+   * recently-received and -deleted block data. 
+   * 
+   * For the case of received blocks, a hint for preferred replica to be 
+   * deleted when there is any excessive blocks is provided.
    * For example, whenever client code
    * writes a new Block here, or another DataNode copies a Block to
    * this DataNode, it will call blockReceived().
    */
-  public void blockReceived(DatanodeRegistration registration,
+  public void blockReceivedAndDeleted(DatanodeRegistration registration,
                             String poolId,
-                            Block blocks[],
-                            String[] delHints) throws IOException;
+                            StorageReceivedDeletedBlocks[] rcvdAndDeletedBlocks)
+                            throws IOException;
 
   /**
    * errorReport() tells the NameNode about something that has gone
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/protocol/DatanodeRegistration.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/protocol/DatanodeRegistration.java
index 5dc8825..d21b92e 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/protocol/DatanodeRegistration.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/protocol/DatanodeRegistration.java
@@ -63,9 +63,21 @@
    * Create DatanodeRegistration
    */
   public DatanodeRegistration(String nodeName) {
+    this(nodeName, new StorageInfo(), new ExportedBlockKeys());
+  }
+  
+  public DatanodeRegistration(DatanodeID dn, StorageInfo info,
+      ExportedBlockKeys keys) {
+    super(dn);
+    this.storageInfo = info;
+    this.exportedKeys = keys;
+  }
+  
+  public DatanodeRegistration(String nodeName, StorageInfo info,
+      ExportedBlockKeys keys) {
     super(nodeName);
-    this.storageInfo = new StorageInfo();
-    this.exportedKeys = new ExportedBlockKeys();
+    this.storageInfo = info;
+    this.exportedKeys = keys;
   }
   
   public void setStorageInfo(StorageInfo storage) {
@@ -101,7 +113,7 @@
   /////////////////////////////////////////////////
   // Writable
   /////////////////////////////////////////////////
-  /** {@inheritDoc} */
+  @Override
   public void write(DataOutput out) throws IOException {
     super.write(out);
 
@@ -112,7 +124,7 @@
     exportedKeys.write(out);
   }
 
-  /** {@inheritDoc} */
+  @Override
   public void readFields(DataInput in) throws IOException {
     super.readFields(in);
 
diff --git a/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/ipc/AvroTestProtocol.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/protocol/DatanodeStorage.java
similarity index 63%
copy from hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/ipc/AvroTestProtocol.java
copy to hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/protocol/DatanodeStorage.java
index d5d7396..5166b09 100644
--- a/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/ipc/AvroTestProtocol.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/protocol/DatanodeStorage.java
@@ -15,18 +15,30 @@
  * 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.ipc;
-
-import org.apache.avro.AvroRemoteException;
-
-@SuppressWarnings("serial")
-public interface AvroTestProtocol {
-  public static class Problem extends AvroRemoteException {
-    public Problem() {}
+/**
+ * Class capatures information about a storage in Datanode
+ */
+public class DatanodeStorage {
+  public enum State {
+    NORMAL,
+    READ_ONLY
   }
-  void ping();
-  String echo(String value);
-  int add(int v1, int v2);
-  int error() throws Problem;
+  
+  private final String storageID;
+  private final State state;
+  
+  public DatanodeStorage(String sid, State s) {
+    storageID = sid;
+    state = s;
+  }
+
+  public String getStorageID() {
+    return storageID;
+  }
+
+  public State getState() {
+    return state;
+  }
 }
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/protocol/FinalizeCommand.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/protocol/FinalizeCommand.java
new file mode 100644
index 0000000..3bc8b11
--- /dev/null
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/protocol/FinalizeCommand.java
@@ -0,0 +1,68 @@
+/**
+ * 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.server.protocol;
+
+import java.io.DataInput;
+import java.io.DataOutput;
+import java.io.IOException;
+
+import org.apache.hadoop.classification.InterfaceAudience;
+import org.apache.hadoop.classification.InterfaceStability;
+import org.apache.hadoop.io.Writable;
+import org.apache.hadoop.io.WritableFactories;
+import org.apache.hadoop.io.WritableFactory;
+import org.apache.hadoop.io.WritableUtils;
+
+/**
+ * A BlockCommand is an instruction to a datanode to register with the namenode.
+ */
+@InterfaceAudience.Private
+@InterfaceStability.Evolving
+public class FinalizeCommand extends DatanodeCommand {
+  // /////////////////////////////////////////
+  // Writable
+  // /////////////////////////////////////////
+  static { // register a ctor
+    WritableFactories.setFactory(FinalizeCommand.class, new WritableFactory() {
+      public Writable newInstance() {
+        return new FinalizeCommand();
+      }
+    });
+  }
+  
+  String blockPoolId;
+  private FinalizeCommand() {
+    super(DatanodeProtocol.DNA_FINALIZE);
+  }
+  
+  public FinalizeCommand(String bpid) {
+    super(DatanodeProtocol.DNA_FINALIZE);
+    blockPoolId = bpid;
+  }
+  
+  public String getBlockPoolId() {
+    return blockPoolId;
+  }
+  
+  public void readFields(DataInput in) throws IOException {
+    blockPoolId = WritableUtils.readString(in);
+  }
+  public void write(DataOutput out) throws IOException {
+    WritableUtils.writeString(out, blockPoolId);
+  }
+}
\ No newline at end of file
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/protocol/InterDatanodeProtocol.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/protocol/InterDatanodeProtocol.java
index 1c62a60..8702b08 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/protocol/InterDatanodeProtocol.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/protocol/InterDatanodeProtocol.java
@@ -39,6 +39,24 @@
   public static final Log LOG = LogFactory.getLog(InterDatanodeProtocol.class);
 
   /**
+   * Until version 9, this class InterDatanodeProtocol served as both
+   * the interface to the DN AND the RPC protocol used to communicate with the 
+   * DN.
+   * 
+   * Post version 6L (release 23 of Hadoop), the protocol is implemented in
+   * {@literal ../protocolR23Compatible/InterDatanodeWireProtocol}
+   * 
+   * This class is used by both the DN to insulate from the protocol 
+   * serialization.
+   * 
+   * If you are adding/changing DN's interface then you need to 
+   * change both this class and ALSO related protocol buffer
+   * wire protocol definition in InterDatanodeProtocol.proto.
+   * 
+   * For more details on protocol buffer wire protocol, please see 
+   * .../org/apache/hadoop/hdfs/protocolPB/overview.html
+   * 
+   * The log of historical changes can be retrieved from the svn).
    * 6: Add block pool ID to Block
    */
   public static final long versionID = 6L;
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/protocol/JournalProtocol.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/protocol/JournalProtocol.java
index 224208d..4457048 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/protocol/JournalProtocol.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/protocol/JournalProtocol.java
@@ -33,6 +33,18 @@
     clientPrincipal = DFSConfigKeys.DFS_NAMENODE_USER_NAME_KEY)
 @InterfaceAudience.Private
 public interface JournalProtocol extends VersionedProtocol {
+  /**
+   * 
+   * This class is used by both the Namenode (client) and BackupNode (server) 
+   * to insulate from the protocol serialization.
+   * 
+   * If you are adding/changing DN's interface then you need to 
+   * change both this class and ALSO related protocol buffer
+   * wire protocol definition in JournalProtocol.proto.
+   * 
+   * For more details on protocol buffer wire protocol, please see 
+   * .../org/apache/hadoop/hdfs/protocolPB/overview.html
+   */
   public static final long versionID = 1L;
 
   /**
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/protocol/NamenodeProtocol.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/protocol/NamenodeProtocol.java
index a58d0d0..a5c310b 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/protocol/NamenodeProtocol.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/protocol/NamenodeProtocol.java
@@ -38,9 +38,22 @@
 @InterfaceAudience.Private
 public interface NamenodeProtocol extends VersionedProtocol {
   /**
-   * Compared to the previous version the following changes have been introduced:
-   * (Only the latest change is reflected.
-   * The log of historical changes can be retrieved from the svn).
+   * Until version 6L, this class served as both
+   * the client interface to the NN AND the RPC protocol used to 
+   * communicate with the NN.
+   * 
+   * Post version 70 (release 23 of Hadoop), the protocol is implemented in
+   * {@literal ../protocolR23Compatible/ClientNamenodeWireProtocol}
+   * 
+   * This class is used by both the DFSClient and the 
+   * NN server side to insulate from the protocol serialization.
+   * 
+   * If you are adding/changing NN's interface then you need to 
+   * change both this class and ALSO related protocol buffer
+   * wire protocol definition in NamenodeProtocol.proto.
+   * 
+   * For more details on protocol buffer wire protocol, please see 
+   * .../org/apache/hadoop/hdfs/protocolPB/overview.html
    * 
    * 6: Switch to txid-based file naming for image and edits
    */
@@ -62,7 +75,7 @@
    * @param datanode  a data node
    * @param size      requested size
    * @return          a list of blocks & their locations
-   * @throws RemoteException if size is less than or equal to 0 or
+   * @throws IOException if size is less than or equal to 0 or
                                    datanode does not exist
    */
   public BlocksWithLocations getBlocks(DatanodeInfo datanode, long size)
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/protocol/NamenodeRegistration.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/protocol/NamenodeRegistration.java
index aa98ab1..2ee4d40 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/protocol/NamenodeRegistration.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/protocol/NamenodeRegistration.java
@@ -52,10 +52,9 @@
                               String httpAddress,
                               StorageInfo storageInfo,
                               NamenodeRole role) {
-    super();
+    super(storageInfo);
     this.rpcAddress = address;
     this.httpAddress = httpAddress;
-    this.setStorageInfo(storageInfo);
     this.role = role;
   }
 
@@ -64,6 +63,10 @@
     return rpcAddress;
   }
   
+  public String getHttpAddress() {
+    return httpAddress;
+  }
+  
   @Override // NodeRegistration
   public String getRegistrationID() {
     return Storage.getRegistrationID(this);
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/protocol/ReceivedDeletedBlockInfo.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/protocol/ReceivedDeletedBlockInfo.java
new file mode 100644
index 0000000..45014ad
--- /dev/null
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/protocol/ReceivedDeletedBlockInfo.java
@@ -0,0 +1,101 @@
+/**
+ * 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.server.protocol;
+
+import java.io.DataInput;
+import java.io.DataOutput;
+import java.io.IOException;
+
+import org.apache.hadoop.hdfs.protocol.Block;
+import org.apache.hadoop.io.Text;
+import org.apache.hadoop.io.Writable;
+
+/**
+ * A data structure to store Block and delHints together, used to send
+ * received/deleted ACKs.
+ */
+public class ReceivedDeletedBlockInfo implements Writable {
+  Block block;
+  String delHints;
+
+  public final static String TODELETE_HINT = "-";
+
+  public ReceivedDeletedBlockInfo() {
+  }
+
+  public ReceivedDeletedBlockInfo(Block blk, String delHints) {
+    this.block = blk;
+    this.delHints = delHints;
+  }
+
+  public Block getBlock() {
+    return this.block;
+  }
+
+  public void setBlock(Block blk) {
+    this.block = blk;
+  }
+
+  public String getDelHints() {
+    return this.delHints;
+  }
+
+  public void setDelHints(String hints) {
+    this.delHints = hints;
+  }
+
+  public boolean equals(Object o) {
+    if (!(o instanceof ReceivedDeletedBlockInfo)) {
+      return false;
+    }
+    ReceivedDeletedBlockInfo other = (ReceivedDeletedBlockInfo) o;
+    return this.block.equals(other.getBlock())
+        && this.delHints.equals(other.delHints);
+  }
+
+  public int hashCode() {
+    assert false : "hashCode not designed";
+    return 0; 
+  }
+
+  public boolean blockEquals(Block b) {
+    return this.block.equals(b);
+  }
+
+  public boolean isDeletedBlock() {
+    return delHints.equals(TODELETE_HINT);
+  }
+
+  @Override
+  public void write(DataOutput out) throws IOException {
+    this.block.write(out);
+    Text.writeString(out, this.delHints);
+  }
+
+  @Override
+  public void readFields(DataInput in) throws IOException {
+    this.block = new Block();
+    this.block.readFields(in);
+    this.delHints = Text.readString(in);
+  }
+
+  public String toString() {
+    return block.toString() + ", delHint: " + delHints;
+  }
+}
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/protocol/RegisterCommand.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/protocol/RegisterCommand.java
new file mode 100644
index 0000000..0584347
--- /dev/null
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/protocol/RegisterCommand.java
@@ -0,0 +1,57 @@
+/**
+ * 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.server.protocol;
+
+import java.io.DataInput;
+import java.io.DataOutput;
+
+import org.apache.hadoop.classification.InterfaceAudience;
+import org.apache.hadoop.classification.InterfaceStability;
+import org.apache.hadoop.io.Writable;
+import org.apache.hadoop.io.WritableFactories;
+import org.apache.hadoop.io.WritableFactory;
+
+/**
+ * A BlockCommand is an instruction to a datanode to register with the namenode.
+ */
+@InterfaceAudience.Private
+@InterfaceStability.Evolving
+public class RegisterCommand extends DatanodeCommand {
+  // /////////////////////////////////////////
+  // Writable
+  // /////////////////////////////////////////
+  static { // register a ctor
+    WritableFactories.setFactory(RegisterCommand.class, new WritableFactory() {
+      public Writable newInstance() {
+        return new RegisterCommand();
+      }
+    });
+  }
+  
+  public static final DatanodeCommand REGISTER = new RegisterCommand();
+
+  public RegisterCommand() {
+    super(DatanodeProtocol.DNA_REGISTER);
+  }
+
+  @Override
+  public void readFields(DataInput in) { }
+ 
+  @Override
+  public void write(DataOutput out) { }
+}
\ No newline at end of file
diff --git a/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/ipc/AvroTestProtocol.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/protocol/StorageBlockReport.java
similarity index 65%
copy from hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/ipc/AvroTestProtocol.java
copy to hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/protocol/StorageBlockReport.java
index d5d7396..d734406 100644
--- a/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/ipc/AvroTestProtocol.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/protocol/StorageBlockReport.java
@@ -16,17 +16,25 @@
  * limitations under the License.
  */
 
-package org.apache.hadoop.ipc;
+package org.apache.hadoop.hdfs.server.protocol;
 
-import org.apache.avro.AvroRemoteException;
-
-@SuppressWarnings("serial")
-public interface AvroTestProtocol {
-  public static class Problem extends AvroRemoteException {
-    public Problem() {}
+/**
+ * Block report for a Datanode storage
+ */
+public class StorageBlockReport {
+  private final String storageID;
+  private final long[] blocks;
+  
+  public StorageBlockReport(String sid, long[] blocks) {
+    this.storageID = sid;
+    this.blocks = blocks;
   }
-  void ping();
-  String echo(String value);
-  int add(int v1, int v2);
-  int error() throws Problem;
+
+  public String getStorageID() {
+    return storageID;
+  }
+
+  public long[] getBlocks() {
+    return blocks;
+  }
 }
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/protocol/StorageReceivedDeletedBlocks.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/protocol/StorageReceivedDeletedBlocks.java
new file mode 100644
index 0000000..c3d44ab
--- /dev/null
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/protocol/StorageReceivedDeletedBlocks.java
@@ -0,0 +1,42 @@
+/**
+ * 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.server.protocol;
+
+/**
+ * Report of block received and deleted per Datanode
+ * storage.
+ */
+public class StorageReceivedDeletedBlocks {
+  private final String storageID;
+  private final ReceivedDeletedBlockInfo[] blocks;
+  
+  public String getStorageID() {
+    return storageID;
+  }
+
+  public ReceivedDeletedBlockInfo[] getBlocks() {
+    return blocks;
+  }
+
+  public StorageReceivedDeletedBlocks(final String storageID,
+      final ReceivedDeletedBlockInfo[] blocks) {
+    this.storageID = storageID;
+    this.blocks = blocks;
+  }
+}
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/protocol/StorageReport.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/protocol/StorageReport.java
new file mode 100644
index 0000000..ff43c2f
--- /dev/null
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/protocol/StorageReport.java
@@ -0,0 +1,47 @@
+package org.apache.hadoop.hdfs.server.protocol;
+
+/**
+ * Utilization report for a Datanode storage
+ */
+public class StorageReport {
+  private final String storageID;
+  private final boolean failed;
+  private final long capacity;
+  private final long dfsUsed;
+  private final long remaining;
+  private final long blockPoolUsed;
+  
+  public StorageReport(String sid, boolean failed, long capacity, long dfsUsed,
+      long remaining, long bpUsed) {
+    this.storageID = sid;
+    this.failed = failed;
+    this.capacity = capacity;
+    this.dfsUsed = dfsUsed;
+    this.remaining = remaining;
+    this.blockPoolUsed = bpUsed;
+  }
+
+  public String getStorageID() {
+    return storageID;
+  }
+
+  public boolean isFailed() {
+    return failed;
+  }
+
+  public long getCapacity() {
+    return capacity;
+  }
+
+  public long getDfsUsed() {
+    return dfsUsed;
+  }
+
+  public long getRemaining() {
+    return remaining;
+  }
+
+  public long getBlockPoolUsed() {
+    return blockPoolUsed;
+  }
+}
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/protocol/UpgradeCommand.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/protocol/UpgradeCommand.java
index a225790..a6de55d 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/protocol/UpgradeCommand.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/protocol/UpgradeCommand.java
@@ -40,9 +40,11 @@
 @InterfaceAudience.Private
 @InterfaceStability.Evolving
 public class UpgradeCommand extends DatanodeCommand {
-  final static int UC_ACTION_UNKNOWN = DatanodeProtocol.DNA_UNKNOWN;
-  public final static int UC_ACTION_REPORT_STATUS = 100; // report upgrade status
-  public final static int UC_ACTION_START_UPGRADE = 101; // start upgrade
+  public final static int UC_ACTION_UNKNOWN = DatanodeProtocol.DNA_UNKNOWN;
+  public final static int UC_ACTION_REPORT_STATUS = 
+      DatanodeProtocol.DNA_UC_ACTION_REPORT_STATUS;
+  public final static int UC_ACTION_START_UPGRADE =
+      DatanodeProtocol.DNA_UC_ACTION_START_UPGRADE;
 
   private int version;
   private short upgradeStatus;
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/tools/DFSAdmin.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/tools/DFSAdmin.java
index cca82a8..1025880 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/tools/DFSAdmin.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/tools/DFSAdmin.java
@@ -43,14 +43,14 @@
 import org.apache.hadoop.hdfs.protocol.HdfsConstants;
 import org.apache.hadoop.hdfs.protocol.HdfsConstants.DatanodeReportType;
 import org.apache.hadoop.hdfs.protocol.HdfsConstants.UpgradeAction;
+import org.apache.hadoop.hdfs.protocolPB.RefreshAuthorizationPolicyProtocolClientSideTranslatorPB;
+import org.apache.hadoop.hdfs.protocolPB.RefreshUserMappingsProtocolClientSideTranslatorPB;
 import org.apache.hadoop.hdfs.server.common.UpgradeStatusReport;
 import org.apache.hadoop.hdfs.server.namenode.NameNode;
 import org.apache.hadoop.ipc.RPC;
 import org.apache.hadoop.ipc.RemoteException;
 import org.apache.hadoop.net.NetUtils;
-import org.apache.hadoop.security.RefreshUserMappingsProtocol;
 import org.apache.hadoop.security.UserGroupInformation;
-import org.apache.hadoop.security.authorize.RefreshAuthorizationPolicyProtocol;
 import org.apache.hadoop.util.StringUtils;
 import org.apache.hadoop.util.ToolRunner;
 
@@ -791,13 +791,9 @@
         conf.get(DFSConfigKeys.DFS_NAMENODE_USER_NAME_KEY, ""));
 
     // Create the client
-    RefreshAuthorizationPolicyProtocol refreshProtocol = 
-      (RefreshAuthorizationPolicyProtocol) 
-      RPC.getProxy(RefreshAuthorizationPolicyProtocol.class, 
-                   RefreshAuthorizationPolicyProtocol.versionID, 
-                   NameNode.getAddress(conf), getUGI(), conf,
-                   NetUtils.getSocketFactory(conf, 
-                                             RefreshAuthorizationPolicyProtocol.class));
+    RefreshAuthorizationPolicyProtocolClientSideTranslatorPB refreshProtocol = 
+        new RefreshAuthorizationPolicyProtocolClientSideTranslatorPB(
+        NameNode.getAddress(conf), getUGI(), conf);
     
     // Refresh the authorization policy in-effect
     refreshProtocol.refreshServiceAcl();
@@ -821,13 +817,9 @@
         conf.get(DFSConfigKeys.DFS_NAMENODE_USER_NAME_KEY, ""));
  
     // Create the client
-    RefreshUserMappingsProtocol refreshProtocol = 
-      (RefreshUserMappingsProtocol) 
-      RPC.getProxy(RefreshUserMappingsProtocol.class, 
-          RefreshUserMappingsProtocol.versionID, 
-          NameNode.getAddress(conf), getUGI(), conf,
-          NetUtils.getSocketFactory(conf, 
-              RefreshUserMappingsProtocol.class));
+    RefreshUserMappingsProtocolClientSideTranslatorPB refreshProtocol = 
+        new RefreshUserMappingsProtocolClientSideTranslatorPB(
+        NameNode.getAddress(conf), getUGI(), conf);
 
     // Refresh the user-to-groups mappings
     refreshProtocol.refreshUserToGroupsMappings();
@@ -852,13 +844,9 @@
         conf.get(DFSConfigKeys.DFS_NAMENODE_USER_NAME_KEY, ""));
 
     // Create the client
-    RefreshUserMappingsProtocol refreshProtocol = 
-      (RefreshUserMappingsProtocol) 
-      RPC.getProxy(RefreshUserMappingsProtocol.class, 
-          RefreshUserMappingsProtocol.versionID, 
-          NameNode.getAddress(conf), getUGI(), conf,
-          NetUtils.getSocketFactory(conf, 
-              RefreshUserMappingsProtocol.class));
+    RefreshUserMappingsProtocolClientSideTranslatorPB refreshProtocol = 
+        new RefreshUserMappingsProtocolClientSideTranslatorPB(
+        NameNode.getAddress(conf), getUGI(), conf);
 
     // Refresh the user-to-groups mappings
     refreshProtocol.refreshSuperUserGroupsConfiguration();
@@ -1148,10 +1136,9 @@
         conf.get(DFSConfigKeys.DFS_DATANODE_USER_NAME_KEY, ""));
 
     // Create the client
-    ClientDatanodeProtocol dnProtocol = RPC.getProxy(
-        ClientDatanodeProtocol.class, ClientDatanodeProtocol.versionID,
-        datanodeAddr, getUGI(), conf, NetUtils.getSocketFactory(conf,
-            ClientDatanodeProtocol.class));
+    ClientDatanodeProtocol dnProtocol =     
+        DFSUtil.createClientDatanodeProtocolProxy(datanodeAddr, getUGI(), conf,
+            NetUtils.getSocketFactory(conf, ClientDatanodeProtocol.class));
     return dnProtocol;
   }
   
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/tools/GetGroups.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/tools/GetGroups.java
index 1f2b98f..5ad227d 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/tools/GetGroups.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/tools/GetGroups.java
@@ -23,8 +23,11 @@
 
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.hdfs.HdfsConfiguration;
+import org.apache.hadoop.hdfs.protocolPB.GetUserMappingsProtocolClientSideTranslatorPB;
 import org.apache.hadoop.hdfs.server.namenode.NameNode;
+import org.apache.hadoop.security.UserGroupInformation;
 import org.apache.hadoop.tools.GetGroupsBase;
+import org.apache.hadoop.tools.GetUserMappingsProtocol;
 import org.apache.hadoop.util.ToolRunner;
 
 /**
@@ -51,6 +54,13 @@
       throws IOException {
     return NameNode.getAddress(conf);
   }
+  
+  @Override
+  protected GetUserMappingsProtocol getUgmProtocol() throws IOException {
+    return new GetUserMappingsProtocolClientSideTranslatorPB(
+        NameNode.getAddress(getConf()), UserGroupInformation.getCurrentUser(),
+        getConf());
+  }
 
   public static void main(String[] argv) throws Exception {
     int res = ToolRunner.run(new GetGroups(new HdfsConfiguration()), argv);
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/proto/ClientDatanodeProtocol.proto b/hadoop-hdfs-project/hadoop-hdfs/src/main/proto/ClientDatanodeProtocol.proto
new file mode 100644
index 0000000..a74d7a2
--- /dev/null
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/proto/ClientDatanodeProtocol.proto
@@ -0,0 +1,122 @@
+/**
+ * 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.
+ */
+
+// This file contains protocol buffers that are used throughout HDFS -- i.e.
+// by the client, server, and data transfer protocols.
+
+option java_package = "org.apache.hadoop.hdfs.protocol.proto";
+option java_outer_classname = "ClientDatanodeProtocolProtos";
+option java_generic_services = true;
+option java_generate_equals_and_hash = true;
+
+import "hdfs.proto";
+
+/**
+ * block - block for which visible length is requested
+ */
+message GetReplicaVisibleLengthRequestProto {
+  required ExtendedBlockProto block = 1;
+}
+
+/**
+ * length - visible length of the block
+ */
+message GetReplicaVisibleLengthResponseProto {
+  required uint64 length = 1;
+}
+
+/**
+ * void request
+ */
+message RefreshNamenodesRequestProto {
+}
+
+/**
+ * void response
+ */
+message RefreshNamenodesResponseProto {
+}
+
+/**
+ * blockPool - block pool to be deleted
+ * force - if false, delete the block pool only if it is empty.
+ *         if true, delete the block pool even if it has blocks.
+ */
+message DeleteBlockPoolRequestProto {
+  required string blockPool = 1;
+  required bool force = 2;
+}
+
+/**
+ * void response
+ */
+message DeleteBlockPoolResponseProto {
+}
+
+/**
+ * Gets the file information where block and its metadata is stored
+ * block - block for which path information is being requested
+ * token - block token
+ */
+message GetBlockLocalPathInfoRequestProto {
+  required ExtendedBlockProto block = 1;
+  required BlockTokenIdentifierProto token = 2;
+}
+
+/**
+ * block - block for which file path information is being returned
+ * localPath - file path where the block data is stored
+ * localMetaPath - file path where the block meta data is stored
+ */
+message GetBlockLocalPathInfoResponseProto {
+  required ExtendedBlockProto block = 1;
+  required string localPath = 2;
+  required string localMetaPath = 3;
+}
+
+/**
+ * Protocol used from client to the Datanode.
+ * See the request and response for details of rpc call.
+ */
+service ClientDatanodeProtocolService {
+  /**
+   * Returns the visible length of the replica
+   */
+  rpc getReplicaVisibleLength(GetReplicaVisibleLengthRequestProto)
+      returns(GetReplicaVisibleLengthResponseProto);
+
+  /**
+   * Refresh the list of federated namenodes from updated configuration.
+   * Adds new namenodes and stops the deleted namenodes.
+   */
+  rpc refreshNamenodes(RefreshNamenodesRequestProto)
+      returns(RefreshNamenodesResponseProto);
+
+  /**
+   * Delete the block pool from the datanode.
+   */
+  rpc deleteBlockPool(DeleteBlockPoolRequestProto)
+      returns(DeleteBlockPoolResponseProto);
+
+  /**
+   * Retrieves the path names of the block file and metadata file stored on the
+   * local file system.
+   */
+  rpc getBlockLocalPathInfo(GetBlockLocalPathInfoRequestProto)
+      returns(GetBlockLocalPathInfoResponseProto);
+}
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/proto/ClientNamenodeProtocol.proto b/hadoop-hdfs-project/hadoop-hdfs/src/main/proto/ClientNamenodeProtocol.proto
new file mode 100644
index 0000000..10f39ea
--- /dev/null
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/proto/ClientNamenodeProtocol.proto
@@ -0,0 +1,516 @@
+/**
+ * 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.
+ */
+
+option java_package = "org.apache.hadoop.hdfs.protocol.proto";
+option java_outer_classname = "ClientNamenodeProtocolProtos";
+option java_generic_services = true;
+option java_generate_equals_and_hash = true;
+
+import "hdfs.proto";
+
+/**
+ * The ClientNamenodeProtocol Service defines the interface between a client 
+ * (as runnign inside a MR Task) and the Namenode.
+ * See org.apache.hadoop.hdfs.protocol.ClientProtocol for the javadoc 
+ * for each of the methods.
+ * The exceptions declared in the above class also apply to this protocol.
+ * Exceptions are unwrapped and thrown by the  PB libraries.
+ */
+
+message GetBlockLocationsRequestProto {
+  required string src = 1;     // file name
+  required uint64 offset = 2;  // range start offset
+  required uint64 length = 3;  // range length
+}
+
+message GetBlockLocationsResponseProto {
+  optional LocatedBlocksProto locations = 1;
+}
+
+message GetServerDefaultsRequestProto { // No parameters
+}
+
+message GetServerDefaultsResponseProto {
+  required FsServerDefaultsProto serverDefaults = 1;
+}
+
+enum CreateFlagProto {
+  CREATE = 0x01;    // Create a file
+  OVERWRITE = 0x02; // Truncate/overwrite a file. Same as POSIX O_TRUNC
+  APPEND = 0x04;    // Append to a file
+}
+
+message CreateRequestProto {
+  required string src = 1;
+  required FsPermissionProto masked = 2;
+  required string clientName = 3;
+  required uint32 createFlag = 4;  // bits set using CreateFlag
+  required bool createParent = 5;
+  required uint32 replication = 6; // Short: Only 16 bits used
+  required uint64 blockSize = 7;
+}
+
+message CreateResponseProto { // void response
+}
+
+message AppendRequestProto {
+  required string src = 1;
+  required string clientName = 2;
+}
+
+message AppendResponseProto {
+  optional LocatedBlockProto block = 1;
+}
+
+message SetReplicationRequestProto {
+  required string src = 1;
+  required uint32 replication = 2; // Short: Only 16 bits used
+}
+
+message SetReplicationResponseProto {
+  required bool result = 1;
+}
+
+message SetPermissionRequestProto {
+  required string src = 1;
+  required FsPermissionProto permission = 2;
+}
+
+message SetPermissionResponseProto { // void response
+}
+
+message SetOwnerRequestProto {
+  required string src = 1;
+  optional string username = 2;
+  optional string groupname = 3;
+}
+
+message SetOwnerResponseProto { // void response
+}
+
+message AbandonBlockRequestProto {
+  required ExtendedBlockProto b = 1;
+  required string src = 2;
+  required string holder = 3;
+}
+
+message AbandonBlockResponseProto { // void response
+}
+
+message AddBlockRequestProto {
+  required string src = 1;
+  required string clientName = 2;
+  optional ExtendedBlockProto previous = 3;
+  repeated DatanodeInfoProto excludeNodes = 4;
+}
+
+message AddBlockResponseProto {
+  required LocatedBlockProto block = 1;
+}
+
+message GetAdditionalDatanodeRequestProto {
+  required string src = 1;
+  required ExtendedBlockProto blk = 2;
+  repeated DatanodeInfoProto existings = 3;
+  repeated DatanodeInfoProto excludes = 4;
+  required uint32 numAdditionalNodes = 5;
+  required string clientName = 6;
+}
+
+message GetAdditionalDatanodeResponseProto {
+  required LocatedBlockProto block = 1;
+}
+
+message CompleteRequestProto {
+  required string src = 1;
+  required string clientName = 2;
+  optional ExtendedBlockProto last = 3;
+}
+
+message CompleteResponseProto {
+  required bool result = 1;
+}
+
+message ReportBadBlocksRequestProto {
+  repeated LocatedBlockProto blocks = 1;
+}
+
+message ReportBadBlocksResponseProto { // void response
+}
+
+message ConcatRequestProto {
+  required string trg = 1;
+  repeated string srcs = 2;
+}
+
+message ConcatResponseProto { // void response
+}
+
+message RenameRequestProto {
+  required string src = 1;
+  required string dst = 2;
+}
+
+message RenameResponseProto { // void response
+  required bool result = 1;
+}
+
+
+message Rename2RequestProto {
+  required string src = 1;
+  required string dst = 2;
+  required bool overwriteDest = 3;
+}
+
+message Rename2ResponseProto { // void response
+}
+
+message DeleteRequestProto {
+  required string src = 1;
+  required bool recursive = 2;
+}
+
+message DeleteResponseProto {
+    required bool result = 1;
+}
+
+message MkdirsRequestProto {
+  required string src = 1;
+  required FsPermissionProto masked = 2;
+  required bool createParent = 3;
+}
+message MkdirsResponseProto {
+    required bool result = 1;
+}
+
+message GetListingRequestProto {
+  required string src = 1;
+  required bytes startAfter = 2;
+  required bool needLocation = 3;
+}
+message GetListingResponseProto {
+  optional DirectoryListingProto dirList = 1;
+}
+
+message RenewLeaseRequestProto {
+  required string clientName = 1;
+}
+
+message RenewLeaseResponseProto { //void response
+}
+
+message RecoverLeaseRequestProto {
+  required string src = 1;
+  required string clientName = 2;
+}
+message RecoverLeaseResponseProto {
+  required bool result = 1;
+}
+
+message GetFsStatusRequestProto { // no input paramters
+}
+
+message GetFsStatsResponseProto {
+  required uint64 capacity = 1;
+  required uint64 used = 2;
+  required uint64 remaining = 3;
+  required uint64 under_replicated = 4;
+  required uint64 corrupt_blocks = 5;
+  required uint64 missing_blocks = 6;
+}
+
+enum DatanodeReportTypeProto {  // type of the datanode report
+  ALL = 1;
+  LIVE = 2;
+  DEAD = 3;
+}
+
+message GetDatanodeReportRequestProto {
+  required DatanodeReportTypeProto type = 1;
+}
+
+message GetDatanodeReportResponseProto {
+  repeated DatanodeInfoProto di = 1;
+}
+
+message GetPreferredBlockSizeRequestProto {
+  required string filename = 1;
+}
+
+message GetPreferredBlockSizeResponseProto {
+  required uint64 bsize = 1;
+}
+
+enum SafeModeActionProto {
+  SAFEMODE_LEAVE = 1;
+  SAFEMODE_ENTER = 2;
+  SAFEMODE_GET = 3;
+}
+
+message SetSafeModeRequestProto {
+  required SafeModeActionProto action = 1;
+}
+
+message SetSafeModeResponseProto {
+  required bool result = 1;
+}
+
+message SaveNamespaceRequestProto { // no parameters
+}
+
+message SaveNamespaceResponseProto { // void response
+}
+
+message RestoreFailedStorageRequestProto {
+  required string arg = 1;
+}
+
+message RestoreFailedStorageResponseProto {
+    required bool result = 1;
+}
+
+message RefreshNodesRequestProto { // no parameters
+}
+
+message RefreshNodesResponseProto { // void response
+}
+
+message FinalizeUpgradeRequestProto { // no parameters
+}
+
+message FinalizeUpgradeResponseProto { // void response
+}
+
+enum UpgradeActionProto {
+  GET_STATUS = 1;
+  DETAILED_STATUS = 2;
+  FORCE_PROCEED = 3;
+}
+
+message DistributedUpgradeProgressRequestProto {
+  required UpgradeActionProto action = 1;
+}
+message DistributedUpgradeProgressResponseProto {
+  optional UpgradeStatusReportProto report = 1;
+}
+
+message ListCorruptFileBlocksRequestProto {
+  required string path = 1;
+  optional string cookie = 2;
+}
+
+message ListCorruptFileBlocksResponseProto {
+  required CorruptFileBlocksProto corrupt = 1;
+}
+
+message MetaSaveRequestProto {
+  required string filename = 1;
+}
+
+message MetaSaveResponseProto { // void response
+}
+
+message GetFileInfoRequestProto {
+  required string src = 1;
+}
+
+message GetFileInfoResponseProto {
+  optional HdfsFileStatusProto fs = 1;
+}
+
+message GetFileLinkInfoRequestProto {
+  required string src = 1;
+}
+
+message GetFileLinkInfoResponseProto {
+  optional HdfsFileStatusProto fs = 1;
+}
+
+message GetContentSummaryRequestProto {
+  required string path = 1;
+}
+
+message GetContentSummaryResponseProto {
+  required ContentSummaryProto summary = 1;
+}
+
+message SetQuotaRequestProto {
+  required string path = 1;
+  required uint64 namespaceQuota = 2;
+  required uint64 diskspaceQuota = 3;
+}
+
+message SetQuotaResponseProto { // void response
+}
+
+message FsyncRequestProto {
+  required string src = 1;
+  required string client = 2;
+}
+
+message FsyncResponseProto { // void response
+}
+
+message SetTimesRequestProto {
+  required string src = 1;
+  required uint64 mtime = 2;
+  required uint64 atime = 3;
+}
+
+message SetTimesResponseProto { // void response
+}
+
+message CreateSymlinkRequestProto {
+  required string target = 1;
+  required string link = 2;
+  required FsPermissionProto dirPerm = 3;
+  required bool createParent = 4;
+}
+
+message CreateSymlinkResponseProto { // void response
+}
+
+message GetLinkTargetRequestProto {
+  required string path = 1;
+}
+message GetLinkTargetResponseProto {
+  required string targetPath = 1;
+}
+
+message UpdateBlockForPipelineRequestProto {
+  required ExtendedBlockProto block = 1;
+  required string clientName = 2;
+}
+
+message UpdateBlockForPipelineResponseProto {
+  required LocatedBlockProto block = 1;
+}
+
+message UpdatePipelineRequestProto {
+  required string clientName = 1;
+  required ExtendedBlockProto oldBlock = 2;
+  required ExtendedBlockProto newBlock = 3;
+  repeated DatanodeIDProto newNodes = 4;
+}
+
+message UpdatePipelineResponseProto { // void response
+}
+
+message GetDelegationTokenRequestProto {
+  required string renewer = 1;
+}
+
+message GetDelegationTokenResponseProto {
+  required BlockTokenIdentifierProto token = 1;
+}
+
+message RenewDelegationTokenRequestProto {
+  required BlockTokenIdentifierProto token = 1;
+}
+
+message RenewDelegationTokenResponseProto {
+  required uint64 newExireTime = 1;
+}
+
+message CancelDelegationTokenRequestProto {
+  required BlockTokenIdentifierProto token = 1;
+}
+
+message CancelDelegationTokenResponseProto { // void response
+}
+
+message SetBalancerBandwidthRequestProto {
+  required int64 bandwidth = 1;
+}
+
+message SetBalancerBandwidthResponseProto { // void response
+}
+
+
+service ClientNamenodeProtocol {
+  rpc getBlockLocations(GetBlockLocationsRequestProto)
+      returns(GetBlockLocationsResponseProto);
+  rpc getServerDefaults(GetServerDefaultsRequestProto)
+      returns(GetServerDefaultsResponseProto);
+  rpc create(CreateRequestProto)returns(CreateResponseProto);
+  rpc append(AppendRequestProto) returns(AppendResponseProto);
+  rpc setReplication(SetReplicationRequestProto)
+      returns(SetReplicationResponseProto);
+  rpc setPermission(SetPermissionRequestProto)
+      returns(SetPermissionResponseProto);
+  rpc setOwner(SetOwnerRequestProto) returns(SetOwnerResponseProto);
+  rpc abandonBlock(AbandonBlockRequestProto) returns(AbandonBlockResponseProto);
+  rpc addBlock(AddBlockRequestProto) returns(AddBlockResponseProto);
+  rpc getAdditionalDatanode(GetAdditionalDatanodeRequestProto)
+      returns(GetAdditionalDatanodeResponseProto);
+  rpc complete(CompleteRequestProto) returns(CompleteResponseProto);
+  rpc reportBadBlocks(ReportBadBlocksRequestProto)
+      returns(ReportBadBlocksResponseProto);
+  rpc concat(ConcatRequestProto) returns(ConcatResponseProto);
+  rpc rename(RenameRequestProto) returns(RenameResponseProto);
+  rpc rename2(Rename2RequestProto) returns(Rename2ResponseProto);
+  rpc delete(DeleteRequestProto) returns(DeleteResponseProto);
+  rpc mkdirs(MkdirsRequestProto) returns(MkdirsResponseProto);
+  rpc getListing(GetListingRequestProto) returns(GetListingResponseProto);
+  rpc renewLease(RenewLeaseRequestProto) returns(RenewLeaseResponseProto);
+  rpc recoverLease(RecoverLeaseRequestProto)
+      returns(RecoverLeaseResponseProto);
+  rpc getFsStats(GetFsStatusRequestProto) returns(GetFsStatsResponseProto);
+  rpc getDatanodeReport(GetDatanodeReportRequestProto)
+      returns(GetDatanodeReportResponseProto);
+  rpc getPreferredBlockSize(GetPreferredBlockSizeRequestProto)
+      returns(GetPreferredBlockSizeResponseProto);
+  rpc setSafeMode(SetSafeModeRequestProto)
+      returns(SetSafeModeResponseProto);
+  rpc saveNamespace(SaveNamespaceRequestProto)
+      returns(SaveNamespaceResponseProto);
+  rpc restoreFailedStorage(RestoreFailedStorageRequestProto)
+      returns(RestoreFailedStorageResponseProto);
+  rpc refreshNodes(RefreshNodesRequestProto) returns(RefreshNodesResponseProto);
+  rpc finalizeUpgrade(FinalizeUpgradeRequestProto)
+      returns(FinalizeUpgradeResponseProto);
+  rpc distributedUpgradeProgress(DistributedUpgradeProgressRequestProto)
+      returns(DistributedUpgradeProgressResponseProto);
+  rpc listCorruptFileBlocks(ListCorruptFileBlocksRequestProto)
+      returns(ListCorruptFileBlocksResponseProto);
+  rpc metaSave(MetaSaveRequestProto) returns(MetaSaveResponseProto);
+  rpc getFileInfo(GetFileInfoRequestProto) returns(GetFileInfoResponseProto);
+  rpc getFileLinkInfo(GetFileLinkInfoRequestProto)
+      returns(GetFileLinkInfoResponseProto);
+  rpc getContentSummary(GetContentSummaryRequestProto)
+      returns(GetContentSummaryResponseProto);
+  rpc setQuota(SetQuotaRequestProto) returns(SetQuotaResponseProto);
+  rpc fsync(FsyncRequestProto) returns(FsyncResponseProto);
+  rpc setTimes(SetTimesRequestProto) returns(SetTimesResponseProto);
+  rpc createSymlink(CreateSymlinkRequestProto)
+      returns(CreateSymlinkResponseProto);
+  rpc getLinkTarget(GetLinkTargetRequestProto)
+      returns(GetLinkTargetResponseProto);
+  rpc updateBlockForPipeline(UpdateBlockForPipelineRequestProto)
+      returns(UpdateBlockForPipelineResponseProto);
+  rpc updatePipeline(UpdatePipelineRequestProto)
+      returns(UpdatePipelineResponseProto);
+  rpc getDelegationToken(GetDelegationTokenRequestProto)
+      returns(GetDelegationTokenResponseProto);
+  rpc renewDelegationToken(RenewDelegationTokenRequestProto)
+      returns(RenewDelegationTokenResponseProto);
+  rpc cancelDelegationToken(CancelDelegationTokenRequestProto)
+      returns(CancelDelegationTokenResponseProto);
+  rpc  setBalancerBandwidth(SetBalancerBandwidthRequestProto)
+      returns(SetBalancerBandwidthResponseProto);
+}
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/proto/DatanodeProtocol.proto b/hadoop-hdfs-project/hadoop-hdfs/src/main/proto/DatanodeProtocol.proto
new file mode 100644
index 0000000..81ca74b
--- /dev/null
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/proto/DatanodeProtocol.proto
@@ -0,0 +1,385 @@
+/**
+ * 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.
+ */
+
+// This file contains protocol buffers that are used throughout HDFS -- i.e.
+// by the client, server, and data transfer protocols.
+
+option java_package = "org.apache.hadoop.hdfs.protocol.proto";
+option java_outer_classname = "DatanodeProtocolProtos";
+option java_generic_services = true;
+option java_generate_equals_and_hash = true;
+
+import "hdfs.proto";
+
+/**
+ * Information to identify a datanode to a namenode
+ */
+message DatanodeRegistrationProto {
+  required DatanodeIDProto datanodeID = 1;    // Datanode information
+  required StorageInfoProto storageInfo = 2;  // Node information
+  required ExportedBlockKeysProto keys = 3;   // Block keys
+}
+
+/**
+ * Represents a storage available on the datanode
+ */
+message DatanodeStorageProto {
+  enum StorageState {
+    NORMAL = 0;
+    READ_ONLY = 1;
+  }
+
+  required string storageID = 1;    // Unique identifier for the storage
+  optional StorageState state = 2 [default = NORMAL];
+}
+
+/**
+ * Commands sent from namenode to the datanodes
+ */
+message DatanodeCommandProto {
+  enum Type {
+    BalancerBandwidthCommand = 0;
+    BlockCommand = 1;
+    BlockRecoveryCommand = 2;
+    FinalizeCommand = 3;
+    KeyUpdateCommand = 4;
+    RegisterCommand = 5;
+    UpgradeCommand = 6;
+    NullDatanodeCommand = 7;
+  }
+
+  required Type cmdType = 1;    // Type of the command
+
+  // One of the following command is available when the corresponding
+  // cmdType is set
+  optional BalancerBandwidthCommandProto balancerCmd = 2;
+  optional BlockCommandProto blkCmd = 3;
+  optional BlockRecoveryCommandProto recoveryCmd = 4;
+  optional FinalizeCommandProto finalizeCmd = 5;
+  optional KeyUpdateCommandProto keyUpdateCmd = 6;
+  optional RegisterCommandProto registerCmd = 7;
+  optional UpgradeCommandProto upgradeCmd = 8;
+}
+
+/**
+ * Command sent from namenode to datanode to set the
+ * maximum bandwidth to be used for balancing.
+ */
+message BalancerBandwidthCommandProto {
+
+  // Maximum bandwidth to be used by datanode for balancing
+  required uint64 bandwidth = 1;
+}
+
+/**
+ * Command to instruct datanodes to perform certain action
+ * on the given set of blocks.
+ */
+message BlockCommandProto {
+  enum Action {  
+    TRANSFER = 1;   // Transfer blocks to another datanode
+    INVALIDATE = 2; // Invalidate blocks
+    SHUTDOWN = 3; // Shutdown the datanode
+  }
+  required Action action = 1;
+  required string blockPoolId = 2;
+  repeated BlockProto blocks = 3;
+  repeated DatanodeInfosProto targets = 4;
+}
+
+/**
+ * List of blocks to be recovered by the datanode
+ */
+message BlockRecoveryCommandProto {
+  repeated RecoveringBlockProto blocks = 1;
+}
+
+/**
+ * Finalize the upgrade at the datanode
+ */
+message FinalizeCommandProto {
+  required string blockPoolId = 1; // Block pool to be finalized
+}
+
+/**
+ * Update the block keys at the datanode
+ */
+message KeyUpdateCommandProto {
+  required ExportedBlockKeysProto keys = 1;
+}
+
+/**
+ * Instruct datanode to register with the namenode
+ */
+message RegisterCommandProto {
+  // void
+}
+
+/**
+ * Generic distributed upgrade Command
+ */
+message UpgradeCommandProto {
+  enum Action {
+    UNKNOWN = 0;          // Unknown action
+    REPORT_STATUS = 100;  // Report upgrade status
+    START_UPGRADE = 101;  // Start upgrade
+  }
+  required Action action = 1;  // Upgrade action
+  required uint32 version = 2; // Version of the upgrade
+  required uint32 upgradeStatus = 3; // % completed in range 0 & 100
+}
+
+/**
+ * registration - Information of the datanode registering with the namenode
+ */
+message RegisterDatanodeRequestProto {
+  required DatanodeRegistrationProto registration = 1; // Datanode info
+  repeated DatanodeStorageProto storages = 2; // Storages on the datanode
+}
+
+/**
+ * registration - Update registration of the datanode that successfully 
+ *                registered. StorageInfo will be updated to include new 
+ *                storage ID if the datanode did not have one in the request.
+ */
+message RegisterDatanodeResponseProto {
+  required DatanodeRegistrationProto registration = 1; // Datanode info
+}
+
+/**
+ * registration - datanode registration information
+ * capacity - total storage capacity available at the datanode
+ * dfsUsed - storage used by HDFS
+ * remaining - remaining storage available for HDFS
+ * blockPoolUsed - storage used by the block pool
+ * xmitsInProgress - number of transfers from this datanode to others
+ * xceiverCount - number of active transceiver threads
+ * failedVolumes - number of failed volumes
+ */
+message HeartbeatRequestProto {
+  required DatanodeRegistrationProto registration = 1; // Datanode info
+  repeated StorageReportProto reports = 2;
+  optional uint32 xmitsInProgress = 3 [ default = 0 ];
+  optional uint32 xceiverCount = 4 [ default = 0 ];
+  optional uint32 failedVolumes = 5 [ default = 0 ];
+}
+
+message StorageReportProto {
+  required string storageID = 1;
+  optional bool failed = 2 [ default = false ];
+  optional uint64 capacity = 3 [ default = 0 ];
+  optional uint64 dfsUsed = 4 [ default = 0 ];
+  optional uint64 remaining = 5 [ default = 0 ];
+  optional uint64 blockPoolUsed = 6 [ default = 0 ];
+}
+
+/**
+ * cmds - Commands from namenode to datanode.
+ */
+message HeartbeatResponseProto {
+  repeated DatanodeCommandProto cmds = 1; // Returned commands can be null
+}
+
+/**
+ * registration - datanode registration information
+ * blockPoolID  - block pool ID of the reported blocks
+ * blocks       - each block is represented as two longs in the array.
+ *                first long represents block ID
+ *                second long represents length
+ */
+message BlockReportRequestProto {
+  required DatanodeRegistrationProto registration = 1;
+  required string blockPoolId = 2;
+  repeated StorageBlockReportProto reports = 3;
+}
+
+/**
+ * Report of blocks in a storage
+ */
+message StorageBlockReportProto {
+  required string storageID = 1;    // Storage ID
+  repeated uint64 blocks = 2 [packed=true];
+}
+
+/**
+ * cmd - Command from namenode to the datanode
+ */
+message BlockReportResponseProto {
+  optional DatanodeCommandProto cmd = 1;
+} 
+
+/**
+ * Data structure to send received or deleted block information
+ * from datanode to namenode.
+ *
+ * deleteHint set to "-" indicates block deletion.
+ * other deleteHint indicates block addition.
+ */
+message ReceivedDeletedBlockInfoProto {
+  required BlockProto block = 1;
+  optional string deleteHint = 2;
+}
+
+/**
+ * List of blocks received and deleted for a storage.
+ */
+message StorageReceivedDeletedBlocksProto {
+  required string storageID = 1;
+  repeated ReceivedDeletedBlockInfoProto blocks = 2;
+}
+
+/**
+ * registration - datanode registration information
+ * blockPoolID  - block pool ID of the reported blocks
+ * blocks       - Received/deleted block list
+ */
+message BlockReceivedAndDeletedRequestProto {
+  required DatanodeRegistrationProto registration = 1;
+  required string blockPoolId = 2;
+  repeated StorageReceivedDeletedBlocksProto blocks = 3;
+}
+
+/**
+ * void response
+ */
+message BlockReceivedAndDeletedResponseProto {
+}
+
+/**
+ * registartion - Datanode reporting the error
+ * errorCode - error code indicating the error
+ * msg - Free text description of the error
+ */
+message ErrorReportRequestProto {
+  enum ErrorCode {
+    NOTIFY = 0;           // Error report to be logged at the namenode
+    DISK_ERROR = 1;       // DN has disk errors but still has valid volumes
+    INVALID_BLOCK = 2;    // Command from namenode has invalid block ID
+    FATAL_DISK_ERROR = 3; // No valid volumes left on datanode
+  }
+  required DatanodeRegistrationProto registartion = 1; // Registartion info
+  required uint32 errorCode = 2;  // Error code
+  required string msg = 3;        // Error message
+}
+
+/**
+ * void response
+ */
+message ErrorReportResponseProto {
+}
+
+/**
+ * cmd - Upgrade command sent from datanode to namenode
+ */
+message ProcessUpgradeRequestProto {
+  optional UpgradeCommandProto cmd = 1;
+}
+
+/**
+ * cmd - Upgrade command sent from namenode to datanode
+ */
+message ProcessUpgradeResponseProto {
+  optional UpgradeCommandProto cmd = 1;
+}
+
+/**
+ * blocks - list of blocks that are reported as corrupt
+ */
+message ReportBadBlocksRequestProto {
+  repeated LocatedBlockProto blocks = 1;
+}
+
+/**
+ * void response
+ */
+message ReportBadBlocksResponseProto {
+}
+
+/**
+ * Commit block synchronization request during lease recovery
+ */
+message CommitBlockSynchronizationRequestProto {
+  required ExtendedBlockProto block = 1;
+  required uint64 newGenStamp = 2;
+  required uint64 newLength = 3;
+  required bool closeFile = 4;
+  required bool deleteBlock = 5;
+  repeated DatanodeIDProto newTaragets = 6;
+}
+
+/**
+ * void response
+ */
+message CommitBlockSynchronizationResponseProto {
+}
+
+/**
+ * Protocol used from datanode to the namenode
+ * See the request and response for details of rpc call.
+ */
+service DatanodeProtocolService {
+  /**
+   * Register a datanode at a namenode
+   */
+  rpc registerDatanode(RegisterDatanodeRequestProto)
+      returns(RegisterDatanodeResponseProto);
+
+  /**
+   * Send heartbeat from datanode to namenode
+   */
+  rpc sendHeartbeat(HeartbeatRequestProto) returns(HeartbeatResponseProto);
+
+  /**
+   * Report blocks at a given datanode to the namenode
+   */
+  rpc blockReport(BlockReportRequestProto) returns(BlockReportResponseProto);
+
+  /**
+   * Report from datanode about recently received or deleted block
+   */
+  rpc blockReceivedAndDeleted(BlockReceivedAndDeletedRequestProto) 
+      returns(BlockReceivedAndDeletedResponseProto);
+
+  /**
+   * Report from a datanode of an error to the active namenode.
+   * Used for debugging.
+   */
+  rpc errorReport(ErrorReportRequestProto) returns(ErrorReportResponseProto);
+  
+  /**
+   * Request the version
+   */
+  rpc versionRequest(VersionRequestProto) returns(VersionResponseProto);
+
+  /**
+   * Generic way to send commands from datanode to namenode during
+   * distributed upgrade process.
+   */
+  rpc processUpgrade(ProcessUpgradeRequestProto) returns(ProcessUpgradeResponseProto);
+
+  /**
+   * Report corrupt blocks at the specified location
+   */
+  rpc reportBadBlocks(ReportBadBlocksRequestProto) returns(ReportBadBlocksResponseProto);
+
+  /**
+   * Commit block synchronization during lease recovery.
+   */
+  rpc commitBlockSynchronization(CommitBlockSynchronizationRequestProto)
+      returns(CommitBlockSynchronizationResponseProto);
+}
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/proto/GetUserMappingsProtocol.proto b/hadoop-hdfs-project/hadoop-hdfs/src/main/proto/GetUserMappingsProtocol.proto
new file mode 100644
index 0000000..d3e2321
--- /dev/null
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/proto/GetUserMappingsProtocol.proto
@@ -0,0 +1,49 @@
+/**
+ * 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.
+ */
+
+option java_package = "org.apache.hadoop.hdfs.protocol.proto";
+option java_outer_classname = "GetUserMappingsProtocolProtos";
+option java_generic_services = true;
+option java_generate_equals_and_hash = true;
+
+/**
+ *  Get groups for user request.
+ */
+message GetGroupsForUserRequestProto {
+  required string user = 1;
+}
+
+/**
+ * Response for get groups.
+ */
+message GetGroupsForUserResponseProto {
+  repeated string groups = 1;
+}
+
+
+/**
+ * Protocol implemented by the Name Node and Job Tracker which maps users to
+ * groups.
+ */
+service GetUserMappingsProtocolService {
+  /**
+   * Get the groups which are mapped to the given user.
+   */
+  rpc getGroupsForUser(GetGroupsForUserRequestProto)
+      returns(GetGroupsForUserResponseProto);
+}
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/proto/InterDatanodeProtocol.proto b/hadoop-hdfs-project/hadoop-hdfs/src/main/proto/InterDatanodeProtocol.proto
new file mode 100644
index 0000000..7621c85
--- /dev/null
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/proto/InterDatanodeProtocol.proto
@@ -0,0 +1,78 @@
+/**
+ * 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.
+ */
+
+// This file contains protocol buffers that are used throughout HDFS -- i.e.
+// by the client, server, and data transfer protocols.
+
+option java_package = "org.apache.hadoop.hdfs.protocol.proto";
+option java_outer_classname = "InterDatanodeProtocolProtos";
+option java_generic_services = true;
+option java_generate_equals_and_hash = true;
+
+import "hdfs.proto";
+
+/**
+ * Block with location information and new generation stamp
+ * to be used for recovery.
+ */
+message InitReplicaRecoveryRequestProto {
+  required RecoveringBlockProto block = 1;
+}
+
+/**
+ * Repica recovery information
+ */
+message InitReplicaRecoveryResponseProto {
+  required ReplicaStateProto state = 1; // State of the replica
+  required BlockProto block = 2;   // block information
+}
+
+/**
+ * Update replica with new generation stamp and length
+ */
+message UpdateReplicaUnderRecoveryRequestProto {
+  required ExtendedBlockProto block = 1; // Block identifier
+  required uint64 recoveryId = 2;        // New genstamp of the replica
+  required uint64 newLength = 3;         // New length of the replica
+}
+
+/**
+ * Response returns updated block information
+ */
+message UpdateReplicaUnderRecoveryResponseProto {
+  required ExtendedBlockProto block = 1; // Updated block information
+}
+
+/**
+ * Protocol used between datanodes for block recovery.
+ *
+ * See the request and response for details of rpc call.
+ */
+service InterDatanodeProtocolService {
+  /**
+   * Initialize recovery of a replica
+   */
+  rpc initReplicaRecovery(InitReplicaRecoveryRequestProto)
+      returns(InitReplicaRecoveryResponseProto);
+
+  /**
+   * Update a replica with new generation stamp and length
+   */
+  rpc updateReplicaUnderRecovery(UpdateReplicaUnderRecoveryRequestProto)
+      returns(UpdateReplicaUnderRecoveryResponseProto);
+}
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/proto/JournalProtocol.proto b/hadoop-hdfs-project/hadoop-hdfs/src/main/proto/JournalProtocol.proto
new file mode 100644
index 0000000..fa7ed15
--- /dev/null
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/proto/JournalProtocol.proto
@@ -0,0 +1,83 @@
+/**
+ * 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.
+ */
+
+// This file contains protocol buffers that are used throughout HDFS -- i.e.
+// by the client, server, and data transfer protocols.
+
+option java_package = "org.apache.hadoop.hdfs.protocol.proto";
+option java_outer_classname = "JournalProtocolProtos";
+option java_generic_services = true;
+option java_generate_equals_and_hash = true;
+
+import "hdfs.proto";
+
+/**
+ * registration - the registration info of the active NameNode
+ * firstTxnId - the first txid in the rolled edit log
+ * numTxns - Number of transactions in editlog
+ * records - bytes containing serialized journal records
+ */
+message JournalRequestProto {
+  required NamenodeRegistrationProto registration = 1; // Registration info
+  required uint64 firstTxnId = 2; // Transaction ID
+  required uint32 numTxns = 3;    // Transaction ID
+  required bytes records = 4;     // Journal record
+}
+
+/**
+ * void response
+ */
+message JournalResponseProto { 
+}
+
+/**
+ * registration - the registration info of the active NameNode
+ * txid - first txid in the new log
+ */
+message StartLogSegmentRequestProto {
+  required NamenodeRegistrationProto registration = 1; // Registration info
+  required uint64 txid = 2; // Transaction ID
+}
+
+/**
+ * void response
+ */
+message StartLogSegmentResponseProto { 
+}
+
+/**
+ * Protocol used to journal edits to a remote node. Currently,
+ * this is used to publish edits from the NameNode to a BackupNode.
+ *
+ * See the request and response for details of rpc call.
+ */
+service JournalProtocolService {
+  /**
+   * Request sent by active namenode to backup node via 
+   * EditLogBackupOutputStream to stream editlog records.
+   */
+  rpc journal(JournalRequestProto) returns (JournalResponseProto);
+
+  /**
+   * Request sent by active namenode to backup node to notify 
+   * that the NameNode has rolled its edit logs and is now writing a 
+   * new log segment.
+   */
+  rpc startLogSegment(StartLogSegmentRequestProto) 
+      returns (StartLogSegmentResponseProto);
+}
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/proto/NamenodeProtocol.proto b/hadoop-hdfs-project/hadoop-hdfs/src/main/proto/NamenodeProtocol.proto
new file mode 100644
index 0000000..e1ddcf0
--- /dev/null
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/proto/NamenodeProtocol.proto
@@ -0,0 +1,230 @@
+/**
+ * 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.
+ */
+
+// This file contains protocol buffers that are used throughout HDFS -- i.e.
+// by the client, server, and data transfer protocols.
+
+option java_package = "org.apache.hadoop.hdfs.protocol.proto";
+option java_outer_classname = "NamenodeProtocolProtos";
+option java_generic_services = true;
+option java_generate_equals_and_hash = true;
+
+import "hdfs.proto";
+
+/**
+ * Get list of blocks for a given datanode with the total length 
+ * of adding up to given size
+ * datanode - Datanode ID to get list of block from
+ * size - size to which the block lengths must add up to
+ */
+message GetBlocksRequestProto {
+  required DatanodeIDProto datanode = 1; // Datanode ID
+  required uint64 size = 2;              // Size in bytes
+}
+
+ 
+/**
+ * blocks - List of returned blocks
+ */
+message GetBlocksResponseProto {
+  required BlocksWithLocationsProto blocks = 1; // List of blocks
+}
+
+/**
+ * void request
+ */
+message GetBlockKeysRequestProto {
+}
+
+/**
+ * keys - Information about block keys at the active namenode
+ */
+message GetBlockKeysResponseProto {
+  required ExportedBlockKeysProto keys = 1;
+}
+
+/**
+ * void request
+ */
+message GetTransactionIdRequestProto {
+}
+
+/**
+ * txId - Transaction ID of the most recently persisted edit log record
+ */
+message GetTransactionIdResponseProto {
+  required uint64 txId = 1;   // Transaction ID
+}
+
+/**
+ * void request
+ */
+message RollEditLogRequestProto {
+}
+
+/**
+ * signature - A unique token to identify checkpoint transaction
+ */
+message RollEditLogResponseProto {
+  required CheckpointSignatureProto signature = 1;
+}
+
+/**
+ * registration - Namenode reporting the error
+ * errorCode - error code indicating the error
+ * msg - Free text description of the error
+ */
+message ErrorReportRequestProto {
+  required NamenodeRegistrationProto registration = 1; // Registration info
+  required uint32 errorCode = 2;  // Error code
+  required string msg = 3;        // Error message
+}
+
+/**
+ * void response
+ */
+message ErrorReportResponseProto {
+}
+
+/**
+ * registration - Information of the namenode registering with primary namenode
+ */
+message RegisterRequestProto {
+  required NamenodeRegistrationProto registration = 1; // Registration info
+}
+
+/**
+ * registration - Updated registration information of the newly registered
+ *                datanode.
+ */
+message RegisterResponseProto {
+  required NamenodeRegistrationProto registration = 1; // Registration info
+}
+
+/**
+ * Start checkpoint request
+ * registration - Namenode that is starting the checkpoint
+ */
+message StartCheckpointRequestProto {
+  required NamenodeRegistrationProto registration = 1; // Registration info
+}
+
+/**
+ * command - Command returned by the active namenode to be
+ *           be handled by the caller.
+ */
+message StartCheckpointResponseProto {
+  required NamenodeCommandProto command = 1;
+}
+
+/**
+ * End or finalize the previously started checkpoint
+ * registration - Namenode that is ending the checkpoint
+ * signature - unique token to identify checkpoint transaction,
+ *             that was received when checkpoint was started.
+ */
+message EndCheckpointRequestProto {
+  required NamenodeRegistrationProto registration = 1; // Registration info
+  required CheckpointSignatureProto signature = 2;
+}
+
+/**
+ * void response
+ */
+message EndCheckpointResponseProto {
+}
+
+/**
+ * sinceTxId - return the editlog information for transactions >= sinceTxId
+ */
+message GetEditLogManifestRequestProto {
+  required uint64 sinceTxId = 1;  // Transaction ID
+}
+
+/**
+ * manifest - Enumeration of editlogs from namenode for 
+ *            logs >= sinceTxId in the request
+ */
+message GetEditLogManifestResponseProto {
+  required RemoteEditLogManifestProto manifest = 1; 
+}
+
+/**
+ * Protocol used by the sub-ordinate namenode to send requests
+ * the active/primary namenode.
+ *
+ * See the request and response for details of rpc call.
+ */
+service NamenodeProtocolService {
+  /**
+   * Get list of blocks for a given datanode with length
+   * of blocks adding up to given size.
+   */
+  rpc getBlocks(GetBlocksRequestProto) returns(GetBlocksResponseProto);
+
+  /**
+   * Get the current block keys
+   */
+  rpc getBlockKeys(GetBlockKeysRequestProto) returns(GetBlockKeysResponseProto);
+
+  /**
+   * Get the transaction ID of the most recently persisted editlog record
+   */
+  rpc getTransactionId(GetTransactionIdRequestProto) 
+      returns(GetTransactionIdResponseProto);
+
+  /**
+   * Close the current editlog and open a new one for checkpointing purposes
+   */
+  rpc rollEditLog(RollEditLogRequestProto) returns(RollEditLogResponseProto);
+
+  /**
+   * Close the current editlog and open a new one for checkpointing purposes
+   */
+  rpc versionRequest(VersionRequestProto) returns(VersionResponseProto);
+
+  /**
+   * Report from a sub-ordinate namenode of an error to the active namenode.
+   * Active namenode may decide to unregister the reporting namenode 
+   * depending on the error.
+   */
+  rpc errorReport(ErrorReportRequestProto) returns(ErrorReportResponseProto);
+
+  /**
+   * Request to register a sub-ordinate namenode
+   */
+  rpc register(RegisterRequestProto) returns(RegisterResponseProto);
+
+  /**
+   * Request to start a checkpoint. 
+   */
+  rpc startCheckpoint(StartCheckpointRequestProto) 
+      returns(StartCheckpointResponseProto);
+
+  /**
+   * End of finalize the previously started checkpoint
+   */
+  rpc endCheckpoint(EndCheckpointRequestProto) 
+      returns(EndCheckpointResponseProto);
+
+  /**
+   * Get editlog manifests from the active namenode for all the editlogs
+   */
+  rpc getEditLogManifest(GetEditLogManifestRequestProto) 
+      returns(GetEditLogManifestResponseProto);
+}
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/proto/RefreshAuthorizationPolicyProtocol.proto b/hadoop-hdfs-project/hadoop-hdfs/src/main/proto/RefreshAuthorizationPolicyProtocol.proto
new file mode 100644
index 0000000..45bfaf8
--- /dev/null
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/proto/RefreshAuthorizationPolicyProtocol.proto
@@ -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.
+ */
+
+option java_package = "org.apache.hadoop.hdfs.protocol.proto";
+option java_outer_classname = "RefreshAuthorizationPolicyProtocolProtos";
+option java_generic_services = true;
+option java_generate_equals_and_hash = true;
+
+/**
+ *  Refresh service acl request.
+ */
+message RefreshServiceAclRequestProto {
+}
+
+/**
+ * void response
+ */
+message RefreshServiceAclResponseProto {
+}
+
+/**
+ * Protocol which is used to refresh the authorization policy in use currently.
+ */
+service RefreshAuthorizationPolicyProtocolService {
+  /**
+   * Refresh the service-level authorization policy in-effect.
+   */
+  rpc refreshServiceAcl(RefreshServiceAclRequestProto)
+      returns(RefreshServiceAclResponseProto);
+}
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/proto/RefreshUserMappingsProtocol.proto b/hadoop-hdfs-project/hadoop-hdfs/src/main/proto/RefreshUserMappingsProtocol.proto
new file mode 100644
index 0000000..4e45d52
--- /dev/null
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/proto/RefreshUserMappingsProtocol.proto
@@ -0,0 +1,63 @@
+/**
+ * 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.
+ */
+
+option java_package = "org.apache.hadoop.hdfs.protocol.proto";
+option java_outer_classname = "RefreshUserMappingsProtocolProtos";
+option java_generic_services = true;
+option java_generate_equals_and_hash = true;
+
+/**
+ *  Refresh user to group mappings request.
+ */
+message RefreshUserToGroupsMappingsRequestProto {
+}
+
+/**
+ * void response
+ */
+message RefreshUserToGroupsMappingsResponseProto {
+}
+
+/**
+ * Refresh superuser configuration request.
+ */
+message RefreshSuperUserGroupsConfigurationRequestProto {
+}
+
+/**
+ * void response
+ */
+message RefreshSuperUserGroupsConfigurationResponseProto {
+}
+
+/**
+ * Protocol to refresh the user mappings.
+ */
+service RefreshUserMappingsProtocolService {
+  /**
+   * Refresh user to group mappings.
+   */
+  rpc refreshUserToGroupsMappings(RefreshUserToGroupsMappingsRequestProto)
+      returns(RefreshUserToGroupsMappingsResponseProto);
+
+  /**
+   * Refresh superuser proxy group list.
+   */
+  rpc refreshSuperUserGroupsConfiguration(RefreshSuperUserGroupsConfigurationRequestProto)
+      returns(RefreshSuperUserGroupsConfigurationResponseProto);
+}
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/proto/hdfs.proto b/hadoop-hdfs-project/hadoop-hdfs/src/main/proto/hdfs.proto
index d11dbfa..cc45593b 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/proto/hdfs.proto
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/proto/hdfs.proto
@@ -23,13 +23,20 @@
 option java_outer_classname = "HdfsProtos";
 option java_generate_equals_and_hash = true;
 
+/**
+ * Extended block idenfies a block
+ */
 message ExtendedBlockProto {
-  required string poolId = 1;
-  required uint64 blockId = 2;
-  required uint64 numBytes = 3;
-  required uint64 generationStamp = 4;
+  required string poolId = 1;   // Block pool id - gloablly unique across clusters
+  required uint64 blockId = 2;  // the local id within a pool
+  required uint64 generationStamp = 3;
+  optional uint64 numBytes = 4 [default = 0];  // len does not belong in ebid 
+                                               // here for historical reasons
 }
 
+/**
+ * Block Token
+ */
 message BlockTokenIdentifierProto {
   required bytes identifier = 1;
   required bytes password = 2;
@@ -37,20 +44,34 @@
   required string service = 4;
 }
 
+/**
+ * Identifies a Datanode
+ */
 message DatanodeIDProto {
-  required string name = 1;
-  required string storageID = 2;
-  required uint32 infoPort = 3;
+  required string name = 1;      // hostname:portNumber
+  required string storageID = 2; // Unique storage id
+  required uint32 infoPort = 3;  // the port where the infoserver is running
+  required uint32 ipcPort = 4;   // the port where the ipc Server is running
 }
 
+/**
+ * DatanodeInfo array
+ */
+message DatanodeInfosProto {
+  repeated DatanodeInfoProto datanodes = 1;
+}
+
+/**
+ * The status of a Datanode
+ */
 message DatanodeInfoProto {
   required DatanodeIDProto id = 1;
-  optional uint64 capacity = 2;
-  optional uint64 dfsUsed = 3;
-  optional uint64 remaining = 4;
-  optional uint64 blockPoolUsed = 5;
-  optional uint64 lastUpdate = 6;
-  optional uint32 xceiverCount = 7;
+  optional uint64 capacity = 2 [default = 0];
+  optional uint64 dfsUsed = 3 [default = 0];
+  optional uint64 remaining = 4 [default = 0];
+  optional uint64 blockPoolUsed = 5 [default = 0];
+  optional uint64 lastUpdate = 6 [default = 0];
+  optional uint32 xceiverCount = 7 [default = 0];
   optional string location = 8;
   optional string hostName = 9;
   enum AdminState {
@@ -59,6 +80,283 @@
     DECOMMISSIONED = 2;
   }
 
-  optional AdminState adminState = 10;
+  optional AdminState adminState = 10 [default = NORMAL];
+}
+
+/**
+ * Summary of a file or directory
+ */
+message ContentSummaryProto {
+  required uint64 length = 1;
+  required uint64 fileCount = 2;
+  required uint64 directoryCount = 3;
+  required uint64 quota = 4;
+  required uint64 spaceConsumed = 5;
+  required uint64 spaceQuota = 6;
+}
+
+/**
+ * Contains a list of paths corresponding to corrupt files and a cookie
+ * used for iterative calls to NameNode.listCorruptFileBlocks.
+ *
+ */
+message CorruptFileBlocksProto {
+ repeated string files = 1;
+ required string   cookie = 2;
+}
+
+/**
+ * File or Directory permision - same spec as posix
+ */
+message FsPermissionProto {
+  required uint32 perm = 1;       // Actually a short - only 16bits used
+}
+
+
+/**
+ * A LocatedBlock gives information about a block and its location.
+ */ 
+message LocatedBlockProto {
+  required ExtendedBlockProto b  = 1;
+  required uint64 offset = 2;           // offset of first byte of block in the file
+  repeated DatanodeInfoProto locs = 3;  // Locations ordered by proximity to client ip
+  required bool corrupt = 4;            // true if all replicas of a block are corrupt, else false
+                                        // If block has few corrupt replicas, they are filtered and 
+                                        // their locations are not part of this object
+
+  required BlockTokenIdentifierProto blockToken = 5;
+ }
+
+
+/**
+ * A set of file blocks and their locations.
+ */
+message LocatedBlocksProto {
+  required uint64 fileLength = 1;
+  repeated LocatedBlockProto blocks = 2;
+  required bool underConstruction = 3;
+  optional LocatedBlockProto lastBlock = 4;
+  required bool isLastBlockComplete = 5;
+}
+
+
+/**
+ * Status of a file, directory  or symlink
+ * Optionally includes a file's block locations if requested by client on the rpc call.
+ */
+message HdfsFileStatusProto {
+  enum FileType {
+    IS_DIR = 1;
+    IS_FILE = 2;
+    IS_SYMLINK = 3;
+  }
+  required FileType fileType = 1;
+  required bytes path = 2;          // local name of inode encoded java UTF8
+  required uint64 length = 3;
+  required FsPermissionProto permission = 4;
+  required string owner = 5;
+  required string group = 6;
+  required uint64 modification_time = 7;
+  required uint64 access_time = 8;
+
+  // Optional fields for symlink
+  optional bytes symlink = 9;             // if symlink, target encoded java UTF8 
+
+  // Optional fields for file
+  optional uint32 block_replication = 10 [default = 0]; // only 16bits used
+  optional uint64 blocksize = 11 [default = 0];
+  optional LocatedBlocksProto locations = 12;  // suppled only if asked by client
+} 
+
+/**
+ * HDFS Server Defaults
+ */
+message FsServerDefaultsProto {
+  required uint64 blockSize = 1;
+  required uint32 bytesPerChecksum = 2;
+  required uint32 writePacketSize = 3;
+  required uint32 replication = 4; // Actually a short - only 16 bits used
+  required uint32 fileBufferSize = 5;
+}
+
+
+/**
+ * Directory listing
+ */
+message DirectoryListingProto {
+  repeated HdfsFileStatusProto partialListing = 1;
+  required uint32 remainingEntries  = 2;
+}
+
+/**
+ * Status of current cluster upgrade from one version to another
+ */
+message UpgradeStatusReportProto {
+  required uint32 version = 1;;
+  required uint32 upgradeStatus = 2; // % completed in range 0 & 100
+	required bool finalized = 3;
+}
+
+/**
+ * Common node information shared by all the nodes in the cluster
+ */
+message StorageInfoProto {
+  required uint32 layoutVersion = 1; // Layout version of the file system
+  required uint32 namespceID = 2;    // File system namespace ID
+  required string clusterID = 3;     // ID of the cluster
+  required uint64 cTime = 4;         // File system creation time
+}
+
+/**
+ * Information sent by a namenode to identify itself to the primary namenode.
+ */
+message NamenodeRegistrationProto {
+  required string rpcAddress = 1;    // host:port of the namenode RPC address
+  required string httpAddress = 2;   // host:port of the namenode http server
+  enum NamenodeRoleProto {
+    NAMENODE = 1;
+    BACKUP = 2;
+    CHECKPOINT = 3;
+  }
+  required StorageInfoProto storageInfo = 3;  // Node information
+  optional NamenodeRoleProto role = 4 [default = NAMENODE];        // Namenode role
+}
+
+/**
+ * Unique signature to identify checkpoint transactions.
+ */
+message CheckpointSignatureProto {
+  required string blockPoolId = 1;
+  required uint64 mostRecentCheckpointTxId = 2;
+  required uint64 curSegmentTxId = 3;
+  required StorageInfoProto storageInfo = 4;
+}
+
+/**
+ * Command sent from one namenode to another namenode.
+ */
+message NamenodeCommandProto {
+  enum Type {
+    NamenodeCommand = 0;      // Base command
+    CheckPointCommand = 1;    // Check point command
+  }
+  required uint32 action = 1;
+  required Type type = 2;
+  optional CheckpointCommandProto checkpointCmd = 3;
+}
+
+/**
+ * Command returned from primary to checkpointing namenode.
+ * This command has checkpoint signature that identifies
+ * checkpoint transaction and is needed for further
+ * communication related to checkpointing.
+ */
+message CheckpointCommandProto {
+  // Unique signature to identify checkpoint transation
+  required CheckpointSignatureProto signature = 1; 
+
+  // If true, return transfer image to primary upon the completion of checkpoint
+  required bool needToReturnImage = 2;
+}
+
+/**
+ * Block information
+ */
+message BlockProto {
+  required uint64 blockId = 1;
+  required uint64 genStamp = 2;
+  optional uint64 numBytes = 3 [default = 0];
+}
+
+/**
+ * Block and datanodes where is it located
+ */
+message BlockWithLocationsProto {
+  required BlockProto block = 1;   // Block
+  repeated string datanodeIDs = 2; // Datanodes with replicas of the block
+}
+
+/**
+ * List of block with locations
+ */
+message BlocksWithLocationsProto {
+  repeated BlockWithLocationsProto blocks = 1;
+}
+
+/**
+ * Editlog information with available transactions
+ */
+message RemoteEditLogProto {
+  required uint64 startTxId = 1;  // Starting available edit log transaction
+  required uint64 endTxId = 2;    // Ending available edit log transaction
+}
+
+/**
+ * Enumeration of editlogs available on a remote namenode
+ */
+message RemoteEditLogManifestProto {
+  repeated RemoteEditLogProto logs = 1;
+}
+
+/**
+ * Namespace information that describes namespace on a namenode
+ */
+message NamespaceInfoProto {
+  required string buildVersion = 1;         // Software build version
+  required uint32 distUpgradeVersion = 2;   // Distributed upgrade version
+  required string blockPoolID = 3;          // block pool used by the namespace
+  required StorageInfoProto storageInfo = 4;// Noe information
+}
+
+/**
+ * Block access token information
+ */
+message BlockKeyProto {
+  required uint32 keyId = 1;      // Key identifier
+  required uint64 expiryDate = 2; // Expiry time in milliseconds
+  optional bytes keyBytes = 3;    // Key secret
+}
+
+/**
+ * Current key and set of block keys at the namenode.
+ */
+message ExportedBlockKeysProto {
+  required bool isBlockTokenEnabled = 1;
+  required uint64 keyUpdateInterval = 2;
+  required uint64 tokenLifeTime = 3;
+  required BlockKeyProto currentKey = 4;
+  repeated BlockKeyProto allKeys = 5;
+}
+
+/**
+ * State of a block replica at a datanode
+ */
+enum ReplicaStateProto {
+  FINALIZED = 0;  // State of a replica when it is not modified
+  RBW = 1;        // State of replica that is being written to
+  RWR = 2;        // State of replica that is waiting to be recovered
+  RUR = 3;        // State of replica that is under recovery
+  TEMPORARY = 4;  // State of replica that is created for replication
+}
+
+/**
+ * Block that needs to be recovered with at a given location
+ */
+message RecoveringBlockProto {
+  required uint64 newGenStamp = 1;      // New genstamp post recovery
+  required LocatedBlockProto block = 2; // Block to be recovered
+}
+
+/**
+ * void request
+ */
+message VersionRequestProto {
+}
+
+/**
+ * Version response from namenode.
+ */
+message VersionResponseProto {
+  required NamespaceInfoProto info = 1;
 }
 
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/MiniDFSCluster.java b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/MiniDFSCluster.java
index 66a4f94..655310f 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/MiniDFSCluster.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/MiniDFSCluster.java
@@ -44,12 +44,11 @@
 import org.apache.hadoop.fs.FileUtil;
 import org.apache.hadoop.hdfs.protocol.Block;
 import org.apache.hadoop.hdfs.protocol.BlockListAsLongs;
-import org.apache.hadoop.hdfs.protocol.ClientProtocol;
 import org.apache.hadoop.hdfs.protocol.DatanodeInfo;
 import org.apache.hadoop.hdfs.protocol.ExtendedBlock;
 import org.apache.hadoop.hdfs.protocol.HdfsConstants.DatanodeReportType;
-import org.apache.hadoop.hdfs.server.common.HdfsServerConstants.StartupOption;
 import org.apache.hadoop.hdfs.server.common.Storage;
+import org.apache.hadoop.hdfs.server.common.HdfsServerConstants.StartupOption;
 import org.apache.hadoop.hdfs.server.datanode.DataNode;
 import org.apache.hadoop.hdfs.server.datanode.DataNodeTestUtils;
 import org.apache.hadoop.hdfs.server.datanode.DataStorage;
@@ -58,20 +57,15 @@
 import org.apache.hadoop.hdfs.server.namenode.FSNamesystem;
 import org.apache.hadoop.hdfs.server.namenode.NameNode;
 import org.apache.hadoop.hdfs.server.namenode.NameNodeAdapter;
-import org.apache.hadoop.hdfs.server.protocol.DatanodeProtocol;
 import org.apache.hadoop.hdfs.server.protocol.DatanodeRegistration;
-import org.apache.hadoop.hdfs.server.protocol.NamenodeProtocol;
 import org.apache.hadoop.hdfs.server.protocol.NamenodeProtocols;
 import org.apache.hadoop.hdfs.tools.DFSAdmin;
 import org.apache.hadoop.metrics2.lib.DefaultMetricsSystem;
 import org.apache.hadoop.net.DNSToSwitchMapping;
 import org.apache.hadoop.net.NetUtils;
 import org.apache.hadoop.net.StaticMapping;
-import org.apache.hadoop.security.RefreshUserMappingsProtocol;
 import org.apache.hadoop.security.UserGroupInformation;
 import org.apache.hadoop.security.authorize.ProxyUsers;
-import org.apache.hadoop.security.authorize.RefreshAuthorizationPolicyProtocol;
-import org.apache.hadoop.tools.GetUserMappingsProtocol;
 import org.apache.hadoop.util.StringUtils;
 import org.apache.hadoop.util.ToolRunner;
 
@@ -504,29 +498,6 @@
     this.federation = federation;
     this.waitSafeMode = waitSafeMode;
     
-    // use alternate RPC engine if spec'd
-    String rpcEngineName = System.getProperty("hdfs.rpc.engine");
-    if (rpcEngineName != null && !"".equals(rpcEngineName)) {
-      
-      LOG.info("HDFS using RPCEngine: " + rpcEngineName);
-      try {
-        Class<?> rpcEngine = conf.getClassByName(rpcEngineName);
-        setRpcEngine(conf, NamenodeProtocols.class, rpcEngine);
-        setRpcEngine(conf, NamenodeProtocol.class, rpcEngine);
-        setRpcEngine(conf, ClientProtocol.class, rpcEngine);
-        setRpcEngine(conf, DatanodeProtocol.class, rpcEngine);
-        setRpcEngine(conf, RefreshAuthorizationPolicyProtocol.class, rpcEngine);
-        setRpcEngine(conf, RefreshUserMappingsProtocol.class, rpcEngine);
-        setRpcEngine(conf, GetUserMappingsProtocol.class, rpcEngine);
-      } catch (ClassNotFoundException e) {
-        throw new RuntimeException(e);
-      }
-
-      // disable service authorization, as it does not work with tunnelled RPC
-      conf.setBoolean(CommonConfigurationKeys.HADOOP_SECURITY_AUTHORIZATION,
-                      false);
-    }
-    
     int replication = conf.getInt(DFSConfigKeys.DFS_REPLICATION_KEY, 3);
     conf.setInt(DFSConfigKeys.DFS_REPLICATION_KEY, Math.min(replication, numDataNodes));
     conf.setInt(DFSConfigKeys.DFS_NAMENODE_SAFEMODE_EXTENSION_KEY, 0);
@@ -659,10 +630,6 @@
     nameNodes[nnIndex] = new NameNodeInfo(nn, new Configuration(conf));
   }
 
-  private void setRpcEngine(Configuration conf, Class<?> protocol, Class<?> engine) {
-    conf.setClass("rpc.engine."+protocol.getName(), engine, Object.class);
-  }
-
   /**
    * @return URI of the namenode from a single namenode MiniDFSCluster
    */
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestDFSClientRetries.java b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestDFSClientRetries.java
index 7feeb2a..257ddd3 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestDFSClientRetries.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestDFSClientRetries.java
@@ -61,6 +61,7 @@
 import org.apache.hadoop.ipc.RemoteException;
 import org.apache.hadoop.ipc.RPC;
 import org.apache.hadoop.ipc.Server;
+import org.apache.hadoop.ipc.RpcPayloadHeader.RpcKind;
 import org.apache.hadoop.net.NetUtils;
 import org.mockito.internal.stubbing.answers.ThrowsException;
 import org.mockito.invocation.InvocationOnMock;
@@ -96,7 +97,7 @@
     }
 
     @Override
-    public Writable call(Class<?> protocol, Writable param, long receiveTime)
+    public Writable call(RpcKind rpcKind, String protocol, Writable param, long receiveTime)
         throws IOException {
       if (sleep) {
         // sleep a bit
@@ -636,7 +637,7 @@
       proxy = DFSUtil.createClientDatanodeProtocolProxy(
           fakeDnId, conf, 500, fakeBlock);
 
-      proxy.getReplicaVisibleLength(null);
+      proxy.getReplicaVisibleLength(new ExtendedBlock("bpid", 1));
       fail ("Did not get expected exception: SocketTimeoutException");
     } catch (SocketTimeoutException e) {
       LOG.info("Got the expected Exception: SocketTimeoutException");
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestDFSRollback.java b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestDFSRollback.java
index 5e14b96..b07bad25 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestDFSRollback.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestDFSRollback.java
@@ -248,7 +248,7 @@
       baseDirs = UpgradeUtilities.createNameNodeStorageDirs(nameNodeDirs, "previous");
       deleteMatchingFiles(baseDirs, "edits.*");
       startNameNodeShouldFail(StartupOption.ROLLBACK,
-          "but there are no logs to load");
+          "No non-corrupt logs for txid ");
       UpgradeUtilities.createEmptyDirs(nameNodeDirs);
       
       log("NameNode rollback with no image file", numDirs);
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestDfsOverAvroRpc.java b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestDfsOverAvroRpc.java
deleted file mode 100644
index a32a55b8..0000000
--- a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestDfsOverAvroRpc.java
+++ /dev/null
@@ -1,35 +0,0 @@
-/**
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- *     http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-package org.apache.hadoop.hdfs;
-
-/** Test for simple signs of life using Avro RPC.  Not an exhaustive test
- * yet, just enough to catch fundamental problems using Avro reflection to
- * infer namenode RPC protocols. */
-public class TestDfsOverAvroRpc extends TestLocalDFS {
-
-  // Commenting the test in 0.23. This can be uncommented once
-  // HADOOP-7524 and HADOOP-7693 is merged into 0.23
-  /*
-  public void testWorkingDirectory() throws IOException {
-    System.setProperty("hdfs.rpc.engine",
-                       "org.apache.hadoop.ipc.AvroRpcEngine");
-    super.testWorkingDirectory();
-  }
-  */
-
-}
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestGetBlocks.java b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestGetBlocks.java
index 53a433b..8693885 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestGetBlocks.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestGetBlocks.java
@@ -25,6 +25,7 @@
 import org.apache.hadoop.hdfs.protocol.Block;
 import org.apache.hadoop.hdfs.protocol.DatanodeInfo;
 import org.apache.hadoop.hdfs.protocol.LocatedBlock;
+import org.apache.hadoop.hdfs.protocolPB.NamenodeProtocolTranslatorPB;
 import org.apache.hadoop.hdfs.server.common.GenerationStamp;
 import org.apache.hadoop.hdfs.server.namenode.NameNode;
 import org.apache.hadoop.hdfs.server.protocol.NamenodeProtocol;
@@ -32,9 +33,7 @@
 import org.apache.hadoop.fs.FSDataOutputStream;
 import org.apache.hadoop.fs.FileSystem;
 import org.apache.hadoop.fs.Path;
-import org.apache.hadoop.ipc.RPC;
 import org.apache.hadoop.ipc.RemoteException;
-import org.apache.hadoop.net.NetUtils;
 import org.apache.hadoop.security.UserGroupInformation;
 
 import junit.framework.TestCase;
@@ -98,10 +97,8 @@
       // get RPC client to namenode
       InetSocketAddress addr = new InetSocketAddress("localhost",
           cluster.getNameNodePort());
-      NamenodeProtocol namenode = (NamenodeProtocol) RPC.getProxy(
-          NamenodeProtocol.class, NamenodeProtocol.versionID, addr,
-          UserGroupInformation.getCurrentUser(), CONF,
-          NetUtils.getDefaultSocketFactory(CONF));
+      NamenodeProtocol namenode = new NamenodeProtocolTranslatorPB(addr, CONF,
+          UserGroupInformation.getCurrentUser());
 
       // get blocks of size fileLen from dataNodes[0]
       BlockWithLocations[] locs;
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestIsMethodSupported.java b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestIsMethodSupported.java
new file mode 100644
index 0000000..0d8174e
--- /dev/null
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestIsMethodSupported.java
@@ -0,0 +1,154 @@
+/**
+ * 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;
+
+import java.io.IOException;
+import java.net.InetSocketAddress;
+
+import junit.framework.Assert;
+
+import org.apache.hadoop.hdfs.protocolPB.ClientDatanodeProtocolTranslatorPB;
+import org.apache.hadoop.hdfs.protocolPB.ClientNamenodeProtocolTranslatorPB;
+import org.apache.hadoop.hdfs.protocolPB.DatanodeProtocolClientSideTranslatorPB;
+import org.apache.hadoop.hdfs.protocolPB.GetUserMappingsProtocolClientSideTranslatorPB;
+import org.apache.hadoop.hdfs.protocolPB.InterDatanodeProtocolTranslatorPB;
+import org.apache.hadoop.hdfs.protocolPB.JournalProtocolTranslatorPB;
+import org.apache.hadoop.hdfs.protocolPB.NamenodeProtocolTranslatorPB;
+import org.apache.hadoop.hdfs.protocolPB.RefreshAuthorizationPolicyProtocolClientSideTranslatorPB;
+import org.apache.hadoop.hdfs.protocolPB.RefreshUserMappingsProtocolClientSideTranslatorPB;
+import org.apache.hadoop.net.NetUtils;
+import org.apache.hadoop.security.UserGroupInformation;
+import org.junit.AfterClass;
+import org.junit.BeforeClass;
+import org.junit.Test;
+
+/**
+ * Test cases to verify that client side translators correctly implement the
+ * isMethodSupported method in ProtocolMetaInterface.
+ */
+public class TestIsMethodSupported {
+  private static MiniDFSCluster cluster = null;
+  private static HdfsConfiguration conf = new HdfsConfiguration();
+  private static InetSocketAddress nnAddress = null;
+  private static InetSocketAddress dnAddress = null;
+  
+  @BeforeClass
+  public static void setUp() throws Exception {
+    cluster = (new MiniDFSCluster.Builder(conf)).numNameNodes(1)
+        .numDataNodes(1).build();
+    nnAddress = cluster.getNameNode().getNameNodeAddress();
+    dnAddress = new InetSocketAddress(cluster.getDataNodes().get(0)
+        .getDatanodeId().getHost(), cluster.getDataNodes().get(0).getIpcPort());
+  }
+
+  @AfterClass
+  public static void tearDown() throws Exception {
+    if (cluster != null) {
+      cluster.shutdown();
+    }
+  }
+
+  @Test
+  public void testNamenodeProtocol() throws IOException {
+    NamenodeProtocolTranslatorPB translator =
+        new NamenodeProtocolTranslatorPB(nnAddress, conf,
+            UserGroupInformation.getCurrentUser());
+    boolean exists = translator.isMethodSupported("rollEditLog");
+    Assert.assertTrue(exists);
+    exists = translator.isMethodSupported("bogusMethod");
+    Assert.assertFalse(exists);
+  }
+  
+  @Test
+  public void testDatanodeProtocol() throws IOException {
+    DatanodeProtocolClientSideTranslatorPB translator = 
+        new DatanodeProtocolClientSideTranslatorPB(nnAddress, conf);
+    Assert.assertTrue(translator.isMethodSupported("sendHeartbeat"));
+  }
+  
+  @Test
+  public void testClientDatanodeProtocol() throws IOException {
+    ClientDatanodeProtocolTranslatorPB translator = 
+        new ClientDatanodeProtocolTranslatorPB(nnAddress, 
+            UserGroupInformation.getCurrentUser(), conf,
+        NetUtils.getDefaultSocketFactory(conf));
+    //Namenode doesn't implement ClientDatanodeProtocol
+    Assert.assertFalse(translator.isMethodSupported("refreshNamenodes"));
+    
+    translator = new ClientDatanodeProtocolTranslatorPB(
+        dnAddress, UserGroupInformation.getCurrentUser(), conf,
+        NetUtils.getDefaultSocketFactory(conf));
+    Assert.assertTrue(translator.isMethodSupported("refreshNamenodes"));
+  }
+  
+  @Test
+  public void testClientNamenodeProtocol() throws IOException {
+    ClientNamenodeProtocolTranslatorPB translator = 
+        new ClientNamenodeProtocolTranslatorPB(nnAddress, conf, 
+            UserGroupInformation.getCurrentUser()); 
+    Assert.assertTrue(translator.isMethodSupported("mkdirs"));
+  }
+  
+  @Test
+  public void tesJournalProtocol() throws IOException {
+    JournalProtocolTranslatorPB translator = 
+        new JournalProtocolTranslatorPB(nnAddress, conf);
+    //Nameode doesn't implement JournalProtocol
+    Assert.assertFalse(translator.isMethodSupported("startLogSegment"));
+  }
+  
+  @Test
+  public void testInterDatanodeProtocol() throws IOException {
+    InterDatanodeProtocolTranslatorPB translator = 
+        new InterDatanodeProtocolTranslatorPB(
+            nnAddress, UserGroupInformation.getCurrentUser(), conf,
+            NetUtils.getDefaultSocketFactory(conf), 0);
+    //Not supported at namenode
+    Assert.assertFalse(translator.isMethodSupported("initReplicaRecovery"));
+    
+    translator = new InterDatanodeProtocolTranslatorPB(
+        dnAddress, UserGroupInformation.getCurrentUser(), conf,
+        NetUtils.getDefaultSocketFactory(conf), 0);
+    Assert.assertTrue(translator.isMethodSupported("initReplicaRecovery"));
+  }
+  
+  @Test
+  public void testGetUserMappingsProtocol() throws IOException {
+    GetUserMappingsProtocolClientSideTranslatorPB translator = 
+        new GetUserMappingsProtocolClientSideTranslatorPB(
+            nnAddress, UserGroupInformation.getCurrentUser(), conf);
+    Assert.assertTrue(translator.isMethodSupported("getGroupsForUser"));
+  }
+  
+  @Test
+  public void testRefreshAuthorizationPolicyProtocol() throws IOException {
+    RefreshAuthorizationPolicyProtocolClientSideTranslatorPB translator =
+        new RefreshAuthorizationPolicyProtocolClientSideTranslatorPB(
+            nnAddress, UserGroupInformation.getCurrentUser(), conf);
+    Assert.assertTrue(translator.isMethodSupported("refreshServiceAcl"));
+  }
+  
+  @Test
+  public void testRefreshUserMappingsProtocol() throws IOException {
+    RefreshUserMappingsProtocolClientSideTranslatorPB translator =
+        new RefreshUserMappingsProtocolClientSideTranslatorPB(
+            nnAddress, UserGroupInformation.getCurrentUser(), conf);
+    Assert.assertTrue(
+        translator.isMethodSupported("refreshUserToGroupsMappings"));
+  }
+}
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/protocolPB/TestPBHelper.java b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/protocolPB/TestPBHelper.java
new file mode 100644
index 0000000..fd31df0
--- /dev/null
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/protocolPB/TestPBHelper.java
@@ -0,0 +1,468 @@
+/**
+ * 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 static junit.framework.Assert.*;
+
+import java.util.ArrayList;
+import java.util.Arrays;
+import java.util.List;
+
+import org.apache.hadoop.hdfs.protocol.Block;
+import org.apache.hadoop.hdfs.protocol.DatanodeID;
+import org.apache.hadoop.hdfs.protocol.DatanodeInfo;
+import org.apache.hadoop.hdfs.protocol.ExtendedBlock;
+import org.apache.hadoop.hdfs.protocol.LocatedBlock;
+import org.apache.hadoop.hdfs.protocol.DatanodeInfo.AdminStates;
+import org.apache.hadoop.hdfs.protocol.proto.DatanodeProtocolProtos.BlockCommandProto;
+import org.apache.hadoop.hdfs.protocol.proto.DatanodeProtocolProtos.BlockRecoveryCommandProto;
+import org.apache.hadoop.hdfs.protocol.proto.DatanodeProtocolProtos.DatanodeRegistrationProto;
+import org.apache.hadoop.hdfs.protocol.proto.HdfsProtos.BlockKeyProto;
+import org.apache.hadoop.hdfs.protocol.proto.HdfsProtos.BlockProto;
+import org.apache.hadoop.hdfs.protocol.proto.HdfsProtos.BlockTokenIdentifierProto;
+import org.apache.hadoop.hdfs.protocol.proto.HdfsProtos.BlockWithLocationsProto;
+import org.apache.hadoop.hdfs.protocol.proto.HdfsProtos.BlocksWithLocationsProto;
+import org.apache.hadoop.hdfs.protocol.proto.HdfsProtos.CheckpointSignatureProto;
+import org.apache.hadoop.hdfs.protocol.proto.HdfsProtos.DatanodeIDProto;
+import org.apache.hadoop.hdfs.protocol.proto.HdfsProtos.ExportedBlockKeysProto;
+import org.apache.hadoop.hdfs.protocol.proto.HdfsProtos.ExtendedBlockProto;
+import org.apache.hadoop.hdfs.protocol.proto.HdfsProtos.LocatedBlockProto;
+import org.apache.hadoop.hdfs.protocol.proto.HdfsProtos.NamenodeRegistrationProto;
+import org.apache.hadoop.hdfs.protocol.proto.HdfsProtos.NamenodeRegistrationProto.NamenodeRoleProto;
+import org.apache.hadoop.hdfs.protocol.proto.HdfsProtos.NamespaceInfoProto;
+import org.apache.hadoop.hdfs.protocol.proto.HdfsProtos.RecoveringBlockProto;
+import org.apache.hadoop.hdfs.protocol.proto.HdfsProtos.RemoteEditLogManifestProto;
+import org.apache.hadoop.hdfs.protocol.proto.HdfsProtos.RemoteEditLogProto;
+import org.apache.hadoop.hdfs.protocol.proto.HdfsProtos.StorageInfoProto;
+import org.apache.hadoop.hdfs.security.token.block.BlockKey;
+import org.apache.hadoop.hdfs.security.token.block.BlockTokenIdentifier;
+import org.apache.hadoop.hdfs.security.token.block.ExportedBlockKeys;
+import org.apache.hadoop.hdfs.server.common.HdfsServerConstants.NamenodeRole;
+import org.apache.hadoop.hdfs.server.common.StorageInfo;
+import org.apache.hadoop.hdfs.server.namenode.CheckpointSignature;
+import org.apache.hadoop.hdfs.server.protocol.BlockCommand;
+import org.apache.hadoop.hdfs.server.protocol.BlockRecoveryCommand.RecoveringBlock;
+import org.apache.hadoop.hdfs.server.protocol.BlocksWithLocations;
+import org.apache.hadoop.hdfs.server.protocol.BlocksWithLocations.BlockWithLocations;
+import org.apache.hadoop.hdfs.server.protocol.BlockRecoveryCommand;
+import org.apache.hadoop.hdfs.server.protocol.DatanodeProtocol;
+import org.apache.hadoop.hdfs.server.protocol.DatanodeRegistration;
+import org.apache.hadoop.hdfs.server.protocol.NamenodeRegistration;
+import org.apache.hadoop.hdfs.server.protocol.NamespaceInfo;
+import org.apache.hadoop.hdfs.server.protocol.RemoteEditLog;
+import org.apache.hadoop.hdfs.server.protocol.RemoteEditLogManifest;
+import org.apache.hadoop.io.Text;
+import org.apache.hadoop.security.token.Token;
+import org.junit.Test;
+
+import com.google.common.base.Joiner;
+import com.google.common.collect.ImmutableList;
+import com.google.common.collect.Lists;
+
+/**
+ * Tests for {@link PBHelper}
+ */
+public class TestPBHelper {
+  @Test
+  public void testConvertNamenodeRole() {
+    assertEquals(NamenodeRoleProto.BACKUP,
+        PBHelper.convert(NamenodeRole.BACKUP));
+    assertEquals(NamenodeRoleProto.CHECKPOINT,
+        PBHelper.convert(NamenodeRole.CHECKPOINT));
+    assertEquals(NamenodeRoleProto.NAMENODE,
+        PBHelper.convert(NamenodeRole.NAMENODE));
+    assertEquals(NamenodeRole.BACKUP,
+        PBHelper.convert(NamenodeRoleProto.BACKUP));
+    assertEquals(NamenodeRole.CHECKPOINT,
+        PBHelper.convert(NamenodeRoleProto.CHECKPOINT));
+    assertEquals(NamenodeRole.NAMENODE,
+        PBHelper.convert(NamenodeRoleProto.NAMENODE));
+  }
+
+  private static StorageInfo getStorageInfo() {
+    return new StorageInfo(1, 2, "cid", 3);
+  }
+
+  @Test
+  public void testConvertStoragInfo() {
+    StorageInfo info = getStorageInfo();
+    StorageInfoProto infoProto = PBHelper.convert(info);
+    StorageInfo info2 = PBHelper.convert(infoProto);
+    assertEquals(info.getClusterID(), info2.getClusterID());
+    assertEquals(info.getCTime(), info2.getCTime());
+    assertEquals(info.getLayoutVersion(), info2.getLayoutVersion());
+    assertEquals(info.getNamespaceID(), info2.getNamespaceID());
+  }
+
+  @Test
+  public void testConvertNamenodeRegistration() {
+    StorageInfo info = getStorageInfo();
+    NamenodeRegistration reg = new NamenodeRegistration("address:999",
+        "http:1000", info, NamenodeRole.NAMENODE);
+    NamenodeRegistrationProto regProto = PBHelper.convert(reg);
+    NamenodeRegistration reg2 = PBHelper.convert(regProto);
+    assertEquals(reg.getAddress(), reg2.getAddress());
+    assertEquals(reg.getClusterID(), reg2.getClusterID());
+    assertEquals(reg.getCTime(), reg2.getCTime());
+    assertEquals(reg.getHttpAddress(), reg2.getHttpAddress());
+    assertEquals(reg.getLayoutVersion(), reg2.getLayoutVersion());
+    assertEquals(reg.getNamespaceID(), reg2.getNamespaceID());
+    assertEquals(reg.getRegistrationID(), reg2.getRegistrationID());
+    assertEquals(reg.getRole(), reg2.getRole());
+    assertEquals(reg.getVersion(), reg2.getVersion());
+
+  }
+
+  @Test
+  public void testConvertDatanodeID() {
+    DatanodeID dn = new DatanodeID("node", "sid", 1, 2);
+    DatanodeIDProto dnProto = PBHelper.convert(dn);
+    DatanodeID dn2 = PBHelper.convert(dnProto);
+    compare(dn, dn2);
+  }
+  
+  void compare(DatanodeID dn, DatanodeID dn2) {
+    assertEquals(dn.getHost(), dn2.getHost());
+    assertEquals(dn.getInfoPort(), dn2.getInfoPort());
+    assertEquals(dn.getIpcPort(), dn2.getIpcPort());
+    assertEquals(dn.getName(), dn2.getName());
+    assertEquals(dn.getPort(), dn2.getPort());
+    assertEquals(dn.getStorageID(), dn2.getStorageID());
+  }
+
+  @Test
+  public void testConvertBlock() {
+    Block b = new Block(1, 100, 3);
+    BlockProto bProto = PBHelper.convert(b);
+    Block b2 = PBHelper.convert(bProto);
+    assertEquals(b, b2);
+  }
+
+  private static BlockWithLocations getBlockWithLocations(int bid) {
+    return new BlockWithLocations(new Block(bid, 0, 1), new String[] { "dn1",
+        "dn2", "dn3" });
+  }
+
+  private void compare(BlockWithLocations locs1, BlockWithLocations locs2) {
+    assertEquals(locs1.getBlock(), locs2.getBlock());
+    assertTrue(Arrays.equals(locs1.getDatanodes(), locs2.getDatanodes()));
+  }
+
+  @Test
+  public void testConvertBlockWithLocations() {
+    BlockWithLocations locs = getBlockWithLocations(1);
+    BlockWithLocationsProto locsProto = PBHelper.convert(locs);
+    BlockWithLocations locs2 = PBHelper.convert(locsProto);
+    compare(locs, locs2);
+  }
+
+  @Test
+  public void testConvertBlocksWithLocations() {
+    BlockWithLocations[] list = new BlockWithLocations[] {
+        getBlockWithLocations(1), getBlockWithLocations(2) };
+    BlocksWithLocations locs = new BlocksWithLocations(list);
+    BlocksWithLocationsProto locsProto = PBHelper.convert(locs);
+    BlocksWithLocations locs2 = PBHelper.convert(locsProto);
+    BlockWithLocations[] blocks = locs.getBlocks();
+    BlockWithLocations[] blocks2 = locs2.getBlocks();
+    assertEquals(blocks.length, blocks2.length);
+    for (int i = 0; i < blocks.length; i++) {
+      compare(blocks[i], blocks2[i]);
+    }
+  }
+
+  private static BlockKey getBlockKey(int keyId) {
+    return new BlockKey(keyId, 10, "encodedKey".getBytes());
+  }
+
+  private void compare(BlockKey k1, BlockKey k2) {
+    assertEquals(k1.getExpiryDate(), k2.getExpiryDate());
+    assertEquals(k1.getKeyId(), k2.getKeyId());
+    assertTrue(Arrays.equals(k1.getEncodedKey(), k2.getEncodedKey()));
+  }
+
+  @Test
+  public void testConvertBlockKey() {
+    BlockKey key = getBlockKey(1);
+    BlockKeyProto keyProto = PBHelper.convert(key);
+    BlockKey key1 = PBHelper.convert(keyProto);
+    compare(key, key1);
+  }
+
+  @Test
+  public void testConvertExportedBlockKeys() {
+    BlockKey[] keys = new BlockKey[] { getBlockKey(2), getBlockKey(3) };
+    ExportedBlockKeys expKeys = new ExportedBlockKeys(true, 9, 10,
+        getBlockKey(1), keys);
+    ExportedBlockKeysProto expKeysProto = PBHelper.convert(expKeys);
+    ExportedBlockKeys expKeys1 = PBHelper.convert(expKeysProto);
+    compare(expKeys, expKeys1);
+  }
+  
+  void compare(ExportedBlockKeys expKeys, ExportedBlockKeys expKeys1) {
+    BlockKey[] allKeys = expKeys.getAllKeys();
+    BlockKey[] allKeys1 = expKeys1.getAllKeys();
+    assertEquals(allKeys.length, allKeys1.length);
+    for (int i = 0; i < allKeys.length; i++) {
+      compare(allKeys[i], allKeys1[i]);
+    }
+    compare(expKeys.getCurrentKey(), expKeys1.getCurrentKey());
+    assertEquals(expKeys.getKeyUpdateInterval(),
+        expKeys1.getKeyUpdateInterval());
+    assertEquals(expKeys.getTokenLifetime(), expKeys1.getTokenLifetime());
+  }
+
+  @Test
+  public void testConvertCheckpointSignature() {
+    CheckpointSignature s = new CheckpointSignature(getStorageInfo(), "bpid",
+        100, 1);
+    CheckpointSignatureProto sProto = PBHelper.convert(s);
+    CheckpointSignature s1 = PBHelper.convert(sProto);
+    assertEquals(s.getBlockpoolID(), s1.getBlockpoolID());
+    assertEquals(s.getClusterID(), s1.getClusterID());
+    assertEquals(s.getCTime(), s1.getCTime());
+    assertEquals(s.getCurSegmentTxId(), s1.getCurSegmentTxId());
+    assertEquals(s.getLayoutVersion(), s1.getLayoutVersion());
+    assertEquals(s.getMostRecentCheckpointTxId(),
+        s1.getMostRecentCheckpointTxId());
+    assertEquals(s.getNamespaceID(), s1.getNamespaceID());
+  }
+  
+  private static void compare(RemoteEditLog l1, RemoteEditLog l2) {
+    assertEquals(l1.getEndTxId(), l2.getEndTxId());
+    assertEquals(l1.getStartTxId(), l2.getStartTxId());
+  }
+  
+  @Test
+  public void testConvertRemoteEditLog() {
+    RemoteEditLog l = new RemoteEditLog(1, 100);
+    RemoteEditLogProto lProto = PBHelper.convert(l);
+    RemoteEditLog l1 = PBHelper.convert(lProto);
+    compare(l, l1);
+  }
+  
+  @Test
+  public void testConvertRemoteEditLogManifest() {
+    List<RemoteEditLog> logs = new ArrayList<RemoteEditLog>();
+    logs.add(new RemoteEditLog(1, 10));
+    logs.add(new RemoteEditLog(11, 20));
+    RemoteEditLogManifest m = new RemoteEditLogManifest(logs);
+    RemoteEditLogManifestProto mProto = PBHelper.convert(m);
+    RemoteEditLogManifest m1 = PBHelper.convert(mProto);
+    
+    List<RemoteEditLog> logs1 = m1.getLogs();
+    assertEquals(logs.size(), logs1.size());
+    for (int i = 0; i < logs.size(); i++) {
+      compare(logs.get(i), logs1.get(i));
+    }
+  }
+  public ExtendedBlock getExtendedBlock() {
+    return getExtendedBlock(1);
+  }
+  
+  public ExtendedBlock getExtendedBlock(long blkid) {
+    return new ExtendedBlock("bpid", blkid, 100, 2);
+  }
+  
+  public DatanodeInfo getDNInfo() {
+    return new DatanodeInfo(new DatanodeID("node", "sid", 1, 2));
+  }
+  
+  private void compare(DatanodeInfo dn1, DatanodeInfo dn2) {
+      assertEquals(dn1.getAdminState(), dn2.getAdminState());
+      assertEquals(dn1.getBlockPoolUsed(), dn2.getBlockPoolUsed());
+      assertEquals(dn1.getBlockPoolUsedPercent(), dn2.getBlockPoolUsedPercent());
+      assertEquals(dn1.getCapacity(), dn2.getCapacity());
+      assertEquals(dn1.getDatanodeReport(), dn2.getDatanodeReport());
+      assertEquals(dn1.getDfsUsed(), dn1.getDfsUsed());
+      assertEquals(dn1.getDfsUsedPercent(), dn1.getDfsUsedPercent());
+      assertEquals(dn1.getHost(), dn2.getHost());
+      assertEquals(dn1.getHostName(), dn2.getHostName());
+      assertEquals(dn1.getInfoPort(), dn2.getInfoPort());
+      assertEquals(dn1.getIpcPort(), dn2.getIpcPort());
+      assertEquals(dn1.getLastUpdate(), dn2.getLastUpdate());
+      assertEquals(dn1.getLevel(), dn2.getLevel());
+      assertEquals(dn1.getNetworkLocation(), dn2.getNetworkLocation());
+  }
+  
+  @Test
+  public void testConvertExtendedBlock() {
+    ExtendedBlock b = getExtendedBlock();
+    ExtendedBlockProto bProto = PBHelper.convert(b);
+    ExtendedBlock b1 = PBHelper.convert(bProto);
+    assertEquals(b, b1);
+    
+    b.setBlockId(-1);
+    bProto = PBHelper.convert(b);
+    b1 = PBHelper.convert(bProto);
+    assertEquals(b, b1);
+  }
+  
+  @Test
+  public void testConvertRecoveringBlock() {
+    DatanodeInfo[] dnInfo = new DatanodeInfo[] { getDNInfo(), getDNInfo() };
+    RecoveringBlock b = new RecoveringBlock(getExtendedBlock(), dnInfo, 3);
+    RecoveringBlockProto bProto = PBHelper.convert(b);
+    RecoveringBlock b1 = PBHelper.convert(bProto);
+    assertEquals(b.getBlock(), b1.getBlock());
+    DatanodeInfo[] dnInfo1 = b1.getLocations();
+    assertEquals(dnInfo.length, dnInfo1.length);
+    for (int i=0; i < dnInfo.length; i++) {
+      compare(dnInfo[0], dnInfo1[0]);
+    }
+  }
+  
+  @Test
+  public void testConvertBlockRecoveryCommand() {
+    DatanodeInfo[] dnInfo = new DatanodeInfo[] { getDNInfo(), getDNInfo() };
+
+    List<RecoveringBlock> blks = ImmutableList.of(
+      new RecoveringBlock(getExtendedBlock(1), dnInfo, 3),
+      new RecoveringBlock(getExtendedBlock(2), dnInfo, 3)
+    );
+    
+    BlockRecoveryCommand cmd = new BlockRecoveryCommand(blks);
+    BlockRecoveryCommandProto proto = PBHelper.convert(cmd);
+    assertEquals(1, proto.getBlocks(0).getBlock().getB().getBlockId());
+    assertEquals(2, proto.getBlocks(1).getBlock().getB().getBlockId());
+    
+    BlockRecoveryCommand cmd2 = PBHelper.convert(proto);
+    
+    List<RecoveringBlock> cmd2Blks = Lists.newArrayList(
+        cmd2.getRecoveringBlocks());
+    assertEquals(blks.get(0).getBlock(), cmd2Blks.get(0).getBlock());
+    assertEquals(blks.get(1).getBlock(), cmd2Blks.get(1).getBlock());
+    assertEquals(Joiner.on(",").join(blks), Joiner.on(",").join(cmd2Blks));
+    assertEquals(cmd.toString(), cmd2.toString());
+  }
+  
+  
+  @Test
+  public void testConvertText() {
+    Text t = new Text("abc".getBytes());
+    String s = t.toString();
+    Text t1 = new Text(s);
+    assertEquals(t, t1);
+  }
+  
+  @Test
+  public void testConvertBlockToken() {
+    Token<BlockTokenIdentifier> token = new Token<BlockTokenIdentifier>(
+        "identifier".getBytes(), "password".getBytes(), new Text("kind"),
+        new Text("service"));
+    BlockTokenIdentifierProto tokenProto = PBHelper.convert(token);
+    Token<BlockTokenIdentifier> token2 = PBHelper.convert(tokenProto);
+    compare(token, token2);
+  }
+  
+  @Test
+  public void testConvertNamespaceInfo() {
+    NamespaceInfo info = new NamespaceInfo(37, "clusterID", "bpID", 2300, 53);
+    NamespaceInfoProto proto = PBHelper.convert(info);
+    NamespaceInfo info2 = PBHelper.convert(proto);
+    compare(info, info2); //Compare the StorageInfo
+    assertEquals(info.getBlockPoolID(), info2.getBlockPoolID());
+    assertEquals(info.getBuildVersion(), info2.getBuildVersion());
+    assertEquals(info.getDistributedUpgradeVersion(),
+        info2.getDistributedUpgradeVersion());
+  }
+
+  private void compare(StorageInfo expected, StorageInfo actual) {
+    assertEquals(expected.clusterID, actual.clusterID);
+    assertEquals(expected.namespaceID, actual.namespaceID);
+    assertEquals(expected.cTime, actual.cTime);
+    assertEquals(expected.layoutVersion, actual.layoutVersion);
+  }
+
+  private void compare(Token<BlockTokenIdentifier> expected,
+      Token<BlockTokenIdentifier> actual) {
+    assertTrue(Arrays.equals(expected.getIdentifier(), actual.getIdentifier()));
+    assertTrue(Arrays.equals(expected.getPassword(), actual.getPassword()));
+    assertEquals(expected.getKind(), actual.getKind());
+    assertEquals(expected.getService(), actual.getService());
+  }
+  
+  @Test
+  public void testConvertLocatedBlock() {
+    DatanodeInfo [] dnInfos = new DatanodeInfo[3];
+    dnInfos[0] = new DatanodeInfo("host0", "0", 5000, 5001, 20000, 10001, 9999,
+        59, 69, 32, "local", "host0", AdminStates.DECOMMISSION_INPROGRESS);
+    dnInfos[1] = new DatanodeInfo("host1", "1", 5000, 5001, 20000, 10001, 9999,
+        59, 69, 32, "local", "host1", AdminStates.DECOMMISSIONED);
+    dnInfos[2] = new DatanodeInfo("host2", "2", 5000, 5001, 20000, 10001, 9999,
+        59, 69, 32, "local", "host1", AdminStates.NORMAL);
+    LocatedBlock lb = new LocatedBlock(
+        new ExtendedBlock("bp12", 12345, 10, 53), dnInfos, 5, false);
+    LocatedBlockProto lbProto = PBHelper.convert(lb);
+    LocatedBlock lb2 = PBHelper.convert(lbProto);
+    assertEquals(lb.getBlock(), lb2.getBlock());
+    compare(lb.getBlockToken(), lb2.getBlockToken());
+    assertEquals(lb.getStartOffset(), lb2.getStartOffset());
+    assertEquals(lb.isCorrupt(), lb2.isCorrupt());
+    DatanodeInfo [] dnInfos2 = lb2.getLocations();
+    assertEquals(dnInfos.length, dnInfos2.length);
+    for (int i = 0; i < dnInfos.length ; i++) {
+      compare(dnInfos[i], dnInfos2[i]);
+    }
+  }
+  
+  @Test
+  public void testConvertDatanodeRegistration() {
+    DatanodeID dnId = new DatanodeID("host", "xyz", 1, 0);
+    BlockKey[] keys = new BlockKey[] { getBlockKey(2), getBlockKey(3) };
+    ExportedBlockKeys expKeys = new ExportedBlockKeys(true, 9, 10,
+        getBlockKey(1), keys);
+    DatanodeRegistration reg = new DatanodeRegistration(dnId,
+        new StorageInfo(), expKeys);
+    DatanodeRegistrationProto proto = PBHelper.convert(reg);
+    DatanodeRegistration reg2 = PBHelper.convert(proto);
+    compare(reg.storageInfo, reg2.storageInfo);
+    compare(reg.exportedKeys, reg2.exportedKeys);
+    compare((DatanodeID)reg, (DatanodeID)reg2);
+  }
+  
+  @Test
+  public void testConvertBlockCommand() {
+    Block[] blocks = new Block[] { new Block(21), new Block(22) };
+    DatanodeInfo[][] dnInfos = new DatanodeInfo[][] { new DatanodeInfo[1],
+        new DatanodeInfo[2] };
+    dnInfos[0][0] = new DatanodeInfo();
+    dnInfos[1][0] = new DatanodeInfo();
+    dnInfos[1][1] = new DatanodeInfo();
+    BlockCommand bc = new BlockCommand(DatanodeProtocol.DNA_TRANSFER, "bp1",
+        blocks, dnInfos);
+    BlockCommandProto bcProto = PBHelper.convert(bc);
+    BlockCommand bc2 = PBHelper.convert(bcProto);
+    assertEquals(bc.getAction(), bc2.getAction());
+    assertEquals(bc.getBlocks().length, bc2.getBlocks().length);
+    Block[] blocks2 = bc2.getBlocks();
+    for (int i = 0; i < blocks.length; i++) {
+      assertEquals(blocks[i], blocks2[i]);
+    }
+    DatanodeInfo[][] dnInfos2 = bc2.getTargets();
+    assertEquals(dnInfos.length, dnInfos2.length);
+    for (int i = 0; i < dnInfos.length; i++) {
+      DatanodeInfo[] d1 = dnInfos[i];
+      DatanodeInfo[] d2 = dnInfos2[i];
+      assertEquals(d1.length, d2.length);
+      for (int j = 0; j < d1.length; j++) {
+        compare(d1[j], d2[j]);
+      }
+    }
+  }
+}
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/security/token/block/TestBlockToken.java b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/security/token/block/TestBlockToken.java
index 9ad87fe..a58394a 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/security/token/block/TestBlockToken.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/security/token/block/TestBlockToken.java
@@ -51,14 +51,20 @@
 import org.apache.hadoop.hdfs.HdfsConfiguration;
 import org.apache.hadoop.hdfs.MiniDFSCluster;
 import org.apache.hadoop.hdfs.protocol.Block;
-import org.apache.hadoop.hdfs.protocol.ClientDatanodeProtocol;
 import org.apache.hadoop.hdfs.protocol.DatanodeID;
 import org.apache.hadoop.hdfs.protocol.DatanodeInfo;
 import org.apache.hadoop.hdfs.protocol.ExtendedBlock;
 import org.apache.hadoop.hdfs.protocol.LocatedBlock;
 import org.apache.hadoop.hdfs.protocol.LocatedBlocks;
+import org.apache.hadoop.hdfs.protocol.ClientDatanodeProtocol;
+import org.apache.hadoop.hdfs.protocol.proto.ClientDatanodeProtocolProtos.ClientDatanodeProtocolService;
+import org.apache.hadoop.hdfs.protocol.proto.ClientDatanodeProtocolProtos.GetReplicaVisibleLengthRequestProto;
+import org.apache.hadoop.hdfs.protocol.proto.ClientDatanodeProtocolProtos.GetReplicaVisibleLengthResponseProto;
+import org.apache.hadoop.hdfs.protocolPB.ClientDatanodeProtocolPB;
+import org.apache.hadoop.hdfs.protocolPB.PBHelper;
 import org.apache.hadoop.io.TestWritable;
 import org.apache.hadoop.ipc.Client;
+import org.apache.hadoop.ipc.ProtobufRpcEngine;
 import org.apache.hadoop.ipc.ProtocolSignature;
 import org.apache.hadoop.ipc.RPC;
 import org.apache.hadoop.ipc.Server;
@@ -76,6 +82,10 @@
 import org.mockito.invocation.InvocationOnMock;
 import org.mockito.stubbing.Answer;
 
+import com.google.protobuf.BlockingService;
+import com.google.protobuf.RpcController;
+import com.google.protobuf.ServiceException;
+
 /** Unit tests for block tokens */
 public class TestBlockToken {
   public static final Log LOG = LogFactory.getLog(TestBlockToken.class);
@@ -96,9 +106,9 @@
     ((Log4JLogger) SaslRpcServer.LOG).getLogger().setLevel(Level.ALL);
     ((Log4JLogger) SaslInputStream.LOG).getLogger().setLevel(Level.ALL);
   }
-  
+
   /** Directory where we can count our open file descriptors under Linux */
-  static File FD_DIR = new File("/proc/self/fd/");  
+  static File FD_DIR = new File("/proc/self/fd/");
 
   long blockKeyUpdateInterval = 10 * 60 * 1000; // 10 mins
   long blockTokenLifetime = 2 * 60 * 1000; // 2 mins
@@ -106,21 +116,24 @@
   ExtendedBlock block2 = new ExtendedBlock("10", 10L);
   ExtendedBlock block3 = new ExtendedBlock("-10", -108L);
 
-  private static class getLengthAnswer implements Answer<Long> {
+  private static class GetLengthAnswer implements
+      Answer<GetReplicaVisibleLengthResponseProto> {
     BlockTokenSecretManager sm;
     BlockTokenIdentifier ident;
 
-    public getLengthAnswer(BlockTokenSecretManager sm,
+    public GetLengthAnswer(BlockTokenSecretManager sm,
         BlockTokenIdentifier ident) {
       this.sm = sm;
       this.ident = ident;
     }
 
     @Override
-    public Long answer(InvocationOnMock invocation) throws IOException {
+    public GetReplicaVisibleLengthResponseProto answer(
+        InvocationOnMock invocation) throws IOException {
       Object args[] = invocation.getArguments();
-      assertEquals(1, args.length);
-      ExtendedBlock block = (ExtendedBlock) args[0];
+      assertEquals(2, args.length);
+      GetReplicaVisibleLengthRequestProto req = 
+          (GetReplicaVisibleLengthRequestProto) args[1];
       Set<TokenIdentifier> tokenIds = UserGroupInformation.getCurrentUser()
           .getTokenIdentifiers();
       assertEquals("Only one BlockTokenIdentifier expected", 1, tokenIds.size());
@@ -129,15 +142,18 @@
         BlockTokenIdentifier id = (BlockTokenIdentifier) tokenId;
         LOG.info("Got: " + id.toString());
         assertTrue("Received BlockTokenIdentifier is wrong", ident.equals(id));
-        sm.checkAccess(id, null, block, BlockTokenSecretManager.AccessMode.WRITE);
+        sm.checkAccess(id, null, PBHelper.convert(req.getBlock()),
+            BlockTokenSecretManager.AccessMode.WRITE);
         result = id.getBlockId();
       }
-      return result;
+      return GetReplicaVisibleLengthResponseProto.newBuilder()
+          .setLength(result).build();
     }
   }
 
   private BlockTokenIdentifier generateTokenId(BlockTokenSecretManager sm,
-      ExtendedBlock block, EnumSet<BlockTokenSecretManager.AccessMode> accessModes)
+      ExtendedBlock block,
+      EnumSet<BlockTokenSecretManager.AccessMode> accessModes)
       throws IOException {
     Token<BlockTokenIdentifier> token = sm.generateToken(block, accessModes);
     BlockTokenIdentifier id = sm.createIdentifier();
@@ -151,12 +167,12 @@
     TestWritable.testWritable(new BlockTokenIdentifier());
     BlockTokenSecretManager sm = new BlockTokenSecretManager(true,
         blockKeyUpdateInterval, blockTokenLifetime);
-    TestWritable.testWritable(generateTokenId(sm, block1, EnumSet
-        .allOf(BlockTokenSecretManager.AccessMode.class)));
-    TestWritable.testWritable(generateTokenId(sm, block2, EnumSet
-        .of(BlockTokenSecretManager.AccessMode.WRITE)));
-    TestWritable.testWritable(generateTokenId(sm, block3, EnumSet
-        .noneOf(BlockTokenSecretManager.AccessMode.class)));
+    TestWritable.testWritable(generateTokenId(sm, block1,
+        EnumSet.allOf(BlockTokenSecretManager.AccessMode.class)));
+    TestWritable.testWritable(generateTokenId(sm, block2,
+        EnumSet.of(BlockTokenSecretManager.AccessMode.WRITE)));
+    TestWritable.testWritable(generateTokenId(sm, block3,
+        EnumSet.noneOf(BlockTokenSecretManager.AccessMode.class)));
   }
 
   private void tokenGenerationAndVerification(BlockTokenSecretManager master,
@@ -176,8 +192,8 @@
       slave.checkAccess(token2, null, block2, mode);
     }
     // multi-mode tokens
-    Token<BlockTokenIdentifier> mtoken = master.generateToken(block3, EnumSet
-        .allOf(BlockTokenSecretManager.AccessMode.class));
+    Token<BlockTokenIdentifier> mtoken = master.generateToken(block3,
+        EnumSet.allOf(BlockTokenSecretManager.AccessMode.class));
     for (BlockTokenSecretManager.AccessMode mode : BlockTokenSecretManager.AccessMode
         .values()) {
       master.checkAccess(mtoken, null, block3, mode);
@@ -202,25 +218,32 @@
     slaveHandler.setKeys(keys);
     tokenGenerationAndVerification(masterHandler, slaveHandler);
   }
-  
+
   private Server createMockDatanode(BlockTokenSecretManager sm,
-      Token<BlockTokenIdentifier> token) throws IOException {
-    ClientDatanodeProtocol mockDN = mock(ClientDatanodeProtocol.class);
+      Token<BlockTokenIdentifier> token) throws IOException, ServiceException {
+    ClientDatanodeProtocolPB mockDN = mock(ClientDatanodeProtocolPB.class);
     when(mockDN.getProtocolVersion(anyString(), anyLong())).thenReturn(
-        ClientDatanodeProtocol.versionID);
-    doReturn(ProtocolSignature.getProtocolSignature(
-        mockDN, ClientDatanodeProtocol.class.getName(),
-        ClientDatanodeProtocol.versionID, 0))
-      .when(mockDN).getProtocolSignature(anyString(), anyLong(), anyInt());
+        RPC.getProtocolVersion(ClientDatanodeProtocolPB.class));
+    doReturn(
+        ProtocolSignature.getProtocolSignature(mockDN,
+            ClientDatanodeProtocolPB.class.getName(),
+            RPC.getProtocolVersion(ClientDatanodeProtocolPB.class), 0)).when(
+        mockDN).getProtocolSignature(anyString(), anyLong(), anyInt());
 
     BlockTokenIdentifier id = sm.createIdentifier();
     id.readFields(new DataInputStream(new ByteArrayInputStream(token
         .getIdentifier())));
-    doAnswer(new getLengthAnswer(sm, id)).when(mockDN).getReplicaVisibleLength(
-        any(ExtendedBlock.class));
+    
+    doAnswer(new GetLengthAnswer(sm, id)).when(mockDN)
+        .getReplicaVisibleLength(any(RpcController.class),
+            any(GetReplicaVisibleLengthRequestProto.class));
 
-    return RPC.getServer(ClientDatanodeProtocol.class, mockDN,
-        ADDRESS, 0, 5, true, conf, sm);
+    RPC.setProtocolEngine(conf, ClientDatanodeProtocolPB.class,
+        ProtobufRpcEngine.class);
+    BlockingService service = ClientDatanodeProtocolService
+        .newReflectiveBlockingService(mockDN);
+    return RPC.getServer(ClientDatanodeProtocolPB.class, service, ADDRESS, 0, 5,
+        true, conf, sm);
   }
 
   @Test
@@ -241,9 +264,8 @@
 
     ClientDatanodeProtocol proxy = null;
     try {
-      proxy = RPC.getProxy(
-          ClientDatanodeProtocol.class, ClientDatanodeProtocol.versionID, addr,
-          ticket, conf, NetUtils.getDefaultSocketFactory(conf));
+      proxy = DFSUtil.createClientDatanodeProtocolProxy(addr, ticket, conf,
+          NetUtils.getDefaultSocketFactory(conf));
       assertEquals(block3.getBlockId(), proxy.getReplicaVisibleLength(block3));
     } finally {
       server.stop();
@@ -255,8 +277,8 @@
 
   /**
    * Test that fast repeated invocations of createClientDatanodeProtocolProxy
-   * will not end up using up thousands of sockets. This is a regression test for
-   * HDFS-1965.
+   * will not end up using up thousands of sockets. This is a regression test
+   * for HDFS-1965.
    */
   @Test
   public void testBlockTokenRpcLeak() throws Exception {
@@ -270,9 +292,9 @@
     server.start();
 
     final InetSocketAddress addr = NetUtils.getConnectAddress(server);
-    DatanodeID fakeDnId = new DatanodeID(
-        "localhost:" + addr.getPort(), "fake-storage", 0, addr.getPort());
-    
+    DatanodeID fakeDnId = new DatanodeID("localhost:" + addr.getPort(),
+        "fake-storage", 0, addr.getPort());
+
     ExtendedBlock b = new ExtendedBlock("fake-pool", new Block(12345L));
     LocatedBlock fakeBlock = new LocatedBlock(b, new DatanodeInfo[0]);
     fakeBlock.setBlockToken(token);
@@ -282,19 +304,19 @@
     // RPC "Client" object to stay above 0 such that RPC.stopProxy doesn't
     // actually close the TCP connections to the real target DN.
     ClientDatanodeProtocol proxyToNoWhere = RPC.getProxy(
-        ClientDatanodeProtocol.class, ClientDatanodeProtocol.versionID, 
+        ClientDatanodeProtocol.class, ClientDatanodeProtocol.versionID,
         new InetSocketAddress("1.1.1.1", 1),
-        UserGroupInformation.createRemoteUser("junk"),
-        conf, NetUtils.getDefaultSocketFactory(conf));
-    
+        UserGroupInformation.createRemoteUser("junk"), conf,
+        NetUtils.getDefaultSocketFactory(conf));
+
     ClientDatanodeProtocol proxy = null;
 
     int fdsAtStart = countOpenFileDescriptors();
     try {
       long endTime = System.currentTimeMillis() + 3000;
       while (System.currentTimeMillis() < endTime) {
-        proxy = DFSUtil.createClientDatanodeProtocolProxy(
-            fakeDnId, conf, 1000, fakeBlock);
+        proxy = DFSUtil.createClientDatanodeProtocolProxy(fakeDnId, conf, 1000,
+            fakeBlock);
         assertEquals(block3.getBlockId(), proxy.getReplicaVisibleLength(block3));
         if (proxy != null) {
           RPC.stopProxy(proxy);
@@ -303,32 +325,31 @@
       }
 
       int fdsAtEnd = countOpenFileDescriptors();
-      
+
       if (fdsAtEnd - fdsAtStart > 50) {
         fail("Leaked " + (fdsAtEnd - fdsAtStart) + " fds!");
       }
     } finally {
       server.stop();
     }
-    
+
     RPC.stopProxy(proxyToNoWhere);
   }
 
   /**
-   * @return the current number of file descriptors open by this
-   * process.
+   * @return the current number of file descriptors open by this process.
    */
-  private static int countOpenFileDescriptors() throws IOException {
+  private static int countOpenFileDescriptors() {
     return FD_DIR.list().length;
   }
 
-  /** 
+  /**
    * Test {@link BlockPoolTokenSecretManager}
    */
   @Test
   public void testBlockPoolTokenSecretManager() throws Exception {
     BlockPoolTokenSecretManager bpMgr = new BlockPoolTokenSecretManager();
-    
+
     // Test BlockPoolSecretManager with upto 10 block pools
     for (int i = 0; i < 10; i++) {
       String bpid = Integer.toString(i);
@@ -337,12 +358,11 @@
       BlockTokenSecretManager slaveHandler = new BlockTokenSecretManager(false,
           blockKeyUpdateInterval, blockTokenLifetime);
       bpMgr.addBlockPool(bpid, slaveHandler);
-      
-      
+
       ExportedBlockKeys keys = masterHandler.exportKeys();
       bpMgr.setKeys(bpid, keys);
       tokenGenerationAndVerification(masterHandler, bpMgr.get(bpid));
-      
+
       // Test key updating
       masterHandler.updateKeys();
       tokenGenerationAndVerification(masterHandler, bpMgr.get(bpid));
@@ -351,11 +371,12 @@
       tokenGenerationAndVerification(masterHandler, bpMgr.get(bpid));
     }
   }
-  
+
   /**
-   * This test writes a file and gets the block locations without closing
-   * the file, and tests the block token in the last block. Block token is
-   * verified by ensuring it is of correct kind.
+   * This test writes a file and gets the block locations without closing the
+   * file, and tests the block token in the last block. Block token is verified
+   * by ensuring it is of correct kind.
+   * 
    * @throws IOException
    * @throws InterruptedException
    */
@@ -389,5 +410,5 @@
     } finally {
       cluster.shutdown();
     }
-  } 
+  }
 }
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/common/TestDistributedUpgrade.java b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/common/TestDistributedUpgrade.java
index d518db0..2bfcbde 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/common/TestDistributedUpgrade.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/common/TestDistributedUpgrade.java
@@ -34,6 +34,7 @@
 import org.apache.hadoop.hdfs.server.protocol.DatanodeProtocol;
 import org.apache.hadoop.hdfs.server.protocol.UpgradeCommand;
 import org.apache.hadoop.hdfs.tools.DFSAdmin;
+import org.apache.hadoop.test.GenericTestUtils;
 
 import org.junit.Test;
 import static org.junit.Assert.*;
@@ -61,7 +62,8 @@
    * Attempts to start a NameNode with the given operation.  Starting
    * the NameNode should throw an exception.
    */
-  void startNameNodeShouldFail(StartupOption operation) {
+  void startNameNodeShouldFail(StartupOption operation,
+      String exceptionSubstring) {
     try {
       //cluster = new MiniDFSCluster.Builder(conf).numDataNodes(0).startupOption(operation).build(); // should fail
       // we set manage dirs to true as NN has to start from untar'ed image with 
@@ -73,8 +75,8 @@
                                               .build(); // should fail
       throw new AssertionError("NameNode should have failed to start");
     } catch (Exception expected) {
-      expected = null;
-      // expected
+      GenericTestUtils.assertExceptionContains(
+          exceptionSubstring, expected);
     }
   }
   
@@ -117,7 +119,7 @@
     conf.setInt(DFSConfigKeys.DFS_DATANODE_SCAN_PERIOD_HOURS_KEY, -1); // block scanning off
 
     log("NameNode start in regular mode when dustributed upgrade is required", numDirs);
-    startNameNodeShouldFail(StartupOption.REGULAR);
+    startNameNodeShouldFail(StartupOption.REGULAR, "contains an old layout version");
 
     log("Start NameNode only distributed upgrade", numDirs);
     // cluster = new MiniDFSCluster.Builder(conf).numDataNodes(0).format(false)
@@ -130,10 +132,12 @@
     cluster.shutdown();
 
     log("NameNode start in regular mode when dustributed upgrade has been started", numDirs);
-    startNameNodeShouldFail(StartupOption.REGULAR);
+    startNameNodeShouldFail(StartupOption.REGULAR,
+        "Previous distributed upgrade was not completed");
 
     log("NameNode rollback to the old version that require a dustributed upgrade", numDirs);
-    startNameNodeShouldFail(StartupOption.ROLLBACK);
+    startNameNodeShouldFail(StartupOption.ROLLBACK,
+        "Cannot rollback to storage version -7 using this version");
 
     log("Normal distributed upgrade for the cluster", numDirs);
     cluster = new MiniDFSCluster.Builder(conf)
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/datanode/DataNodeAdapter.java b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/datanode/DataNodeAdapter.java
index 842616c..cf7db6f 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/datanode/DataNodeAdapter.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/datanode/DataNodeAdapter.java
@@ -19,9 +19,8 @@
 
 import java.io.IOException;
 
-import org.apache.hadoop.hdfs.server.protocol.DatanodeProtocol;
+import org.apache.hadoop.hdfs.protocolPB.DatanodeProtocolClientSideTranslatorPB;
 import org.apache.hadoop.hdfs.server.namenode.NameNode;
-import org.apache.hadoop.test.GenericTestUtils;
 import org.mockito.Mockito;
 
 import com.google.common.base.Preconditions;
@@ -57,7 +56,7 @@
    * the given NameNode. This can be used to delay or wait for
    * RPC calls on the datanode->NN path.
    */
-  public static DatanodeProtocol spyOnBposToNN(
+  public static DatanodeProtocolClientSideTranslatorPB spyOnBposToNN(
       DataNode dn, NameNode nn) {
     String bpid = nn.getNamesystem().getBlockPoolId();
     
@@ -71,12 +70,8 @@
     Preconditions.checkArgument(bpos != null,
         "No such bpid: %s", bpid);
 
-    // When protobufs are merged, the following can be converted
-    // to a simple spy. Because you can't spy on proxy objects,
-    // we have to use the DelegateAnswer trick.
-    DatanodeProtocol origNN = bpos.getBpNamenode();
-    DatanodeProtocol spy = Mockito.mock(DatanodeProtocol.class,
-        new GenericTestUtils.DelegateAnswer(origNN));
+    DatanodeProtocolClientSideTranslatorPB origNN = bpos.getBpNamenode();
+    DatanodeProtocolClientSideTranslatorPB spy = Mockito.spy(origNN);
 
     bpos.setBpNamenode(spy);
     return spy;
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/datanode/TestBlockReport.java b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/datanode/TestBlockReport.java
index 8c53fd9..cb820cc 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/datanode/TestBlockReport.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/datanode/TestBlockReport.java
@@ -40,6 +40,7 @@
 import org.apache.hadoop.hdfs.server.protocol.DatanodeProtocol;
 import org.apache.hadoop.hdfs.server.protocol.DatanodeRegistration;
 import org.apache.hadoop.io.IOUtils;
+import org.apache.hadoop.hdfs.server.protocol.StorageBlockReport;
 import org.apache.hadoop.test.GenericTestUtils;
 import org.apache.hadoop.test.GenericTestUtils.DelayAnswer;
 import org.apache.log4j.Level;
@@ -146,8 +147,9 @@
     DataNode dn = cluster.getDataNodes().get(DN_N0);
     String poolId = cluster.getNamesystem().getBlockPoolId();
     DatanodeRegistration dnR = dn.getDNRegistrationForBP(poolId);
-    cluster.getNameNodeRpc().blockReport(dnR, poolId,
-      new BlockListAsLongs(blocks, null).getBlockListAsLongs());
+    StorageBlockReport[] report = { new StorageBlockReport(dnR.getStorageID(),
+        new BlockListAsLongs(blocks, null).getBlockListAsLongs()) };
+    cluster.getNameNodeRpc().blockReport(dnR, poolId, report);
 
     List<LocatedBlock> blocksAfterReport =
       DFSTestUtil.getAllBlocks(fs.open(filePath));
@@ -180,7 +182,7 @@
 
     Path filePath = new Path("/" + METHOD_NAME + ".dat");
     DFSTestUtil.createFile(fs, filePath,
-      (long) FILE_SIZE, REPL_FACTOR, rand.nextLong());
+      FILE_SIZE, REPL_FACTOR, rand.nextLong());
 
     // mock around with newly created blocks and delete some
     File dataDir = new File(cluster.getDataDirectory());
@@ -226,8 +228,9 @@
     DataNode dn = cluster.getDataNodes().get(DN_N0);
     String poolId = cluster.getNamesystem().getBlockPoolId();
     DatanodeRegistration dnR = dn.getDNRegistrationForBP(poolId);
-    cluster.getNameNodeRpc().blockReport(dnR, poolId,
-      new BlockListAsLongs(blocks, null).getBlockListAsLongs());
+    StorageBlockReport[] report = { new StorageBlockReport(dnR.getStorageID(),
+        new BlockListAsLongs(blocks, null).getBlockListAsLongs()) };
+    cluster.getNameNodeRpc().blockReport(dnR, poolId, report);
 
     BlockManagerTestUtil.getComputedDatanodeWork(cluster.getNamesystem()
         .getBlockManager());
@@ -266,9 +269,10 @@
     DataNode dn = cluster.getDataNodes().get(DN_N0);
     String poolId = cluster.getNamesystem().getBlockPoolId();
     DatanodeRegistration dnR = dn.getDNRegistrationForBP(poolId);
+    StorageBlockReport[] report = { new StorageBlockReport(dnR.getStorageID(),
+        new BlockListAsLongs(blocks, null).getBlockListAsLongs()) };
     DatanodeCommand dnCmd =
-      cluster.getNameNodeRpc().blockReport(dnR, poolId,
-        new BlockListAsLongs(blocks, null).getBlockListAsLongs());
+      cluster.getNameNodeRpc().blockReport(dnR, poolId, report);
     if(LOG.isDebugEnabled()) {
       LOG.debug("Got the command: " + dnCmd);
     }
@@ -284,9 +288,8 @@
    * This test isn't a representative case for BlockReport
    * The empty method is going to be left here to keep the naming
    * of the test plan in synch with the actual implementation
-   * @throws IOException in case of errors
    */
-  public void blockReport_04() throws IOException {
+  public void blockReport_04() {
   }
 
   // Client requests new block from NN. The test corrupts this very block
@@ -295,7 +298,7 @@
   // BlockScanner which is out of scope of this test
   // Keeping the name to be in synch with the test plan
   //
-  public void blockReport_05() throws IOException {
+  public void blockReport_05() {
   }
 
   /**
@@ -319,8 +322,9 @@
     DataNode dn = cluster.getDataNodes().get(DN_N1);
     String poolId = cluster.getNamesystem().getBlockPoolId();
     DatanodeRegistration dnR = dn.getDNRegistrationForBP(poolId);
-    cluster.getNameNodeRpc().blockReport(dnR, poolId,
-        new BlockListAsLongs(blocks, null).getBlockListAsLongs());
+    StorageBlockReport[] report = { new StorageBlockReport(dnR.getStorageID(),
+        new BlockListAsLongs(blocks, null).getBlockListAsLongs()) };
+    cluster.getNameNodeRpc().blockReport(dnR, poolId, report);
     printStats();
     assertEquals("Wrong number of PendingReplication Blocks",
       0, cluster.getNamesystem().getUnderReplicatedBlocks());
@@ -368,8 +372,9 @@
     DataNode dn = cluster.getDataNodes().get(DN_N1);
     String poolId = cluster.getNamesystem().getBlockPoolId();
     DatanodeRegistration dnR = dn.getDNRegistrationForBP(poolId);
-    cluster.getNameNodeRpc().blockReport(dnR, poolId,
-        new BlockListAsLongs(blocks, null).getBlockListAsLongs());
+    StorageBlockReport[] report = { new StorageBlockReport(dnR.getStorageID(),
+        new BlockListAsLongs(blocks, null).getBlockListAsLongs()) };
+    cluster.getNameNodeRpc().blockReport(dnR, poolId, report);
     printStats();
     assertEquals("Wrong number of Corrupted blocks",
       1, cluster.getNamesystem().getCorruptReplicaBlocks() +
@@ -390,8 +395,9 @@
       LOG.debug("Done corrupting length of " + corruptedBlock.getBlockName());
     }
     
-    cluster.getNameNodeRpc().blockReport(dnR, poolId,
+    report[0] = new StorageBlockReport(dnR.getStorageID(),
         new BlockListAsLongs(blocks, null).getBlockListAsLongs());
+    cluster.getNameNodeRpc().blockReport(dnR, poolId, report);
     printStats();
 
     assertEquals("Wrong number of Corrupted blocks",
@@ -440,8 +446,9 @@
       DataNode dn = cluster.getDataNodes().get(DN_N1);
       String poolId = cluster.getNamesystem().getBlockPoolId();
       DatanodeRegistration dnR = dn.getDNRegistrationForBP(poolId);
-      cluster.getNameNodeRpc().blockReport(dnR, poolId,
-          new BlockListAsLongs(blocks, null).getBlockListAsLongs());
+      StorageBlockReport[] report = { new StorageBlockReport(dnR.getStorageID(),
+          new BlockListAsLongs(blocks, null).getBlockListAsLongs()) };
+      cluster.getNameNodeRpc().blockReport(dnR, poolId, report);
       printStats();
       assertEquals("Wrong number of PendingReplication blocks",
         blocks.size(), cluster.getNamesystem().getPendingReplicationBlocks());
@@ -486,8 +493,9 @@
       DataNode dn = cluster.getDataNodes().get(DN_N1);
       String poolId = cluster.getNamesystem().getBlockPoolId();
       DatanodeRegistration dnR = dn.getDNRegistrationForBP(poolId);
-      cluster.getNameNodeRpc().blockReport(dnR, poolId,
-          new BlockListAsLongs(blocks, null).getBlockListAsLongs());
+      StorageBlockReport[] report = { new StorageBlockReport(dnR.getStorageID(),
+          new BlockListAsLongs(blocks, null).getBlockListAsLongs()) };
+      cluster.getNameNodeRpc().blockReport(dnR, poolId, report);
       printStats();
       assertEquals("Wrong number of PendingReplication blocks",
         2, cluster.getNamesystem().getPendingReplicationBlocks());
@@ -550,7 +558,7 @@
         .when(spy).blockReport(
           Mockito.<DatanodeRegistration>anyObject(),
           Mockito.anyString(),
-          Mockito.<long[]>anyObject());
+          Mockito.<StorageBlockReport[]>anyObject());
       
       // Force a block report to be generated. The block report will have
       // an RBW replica in it. Wait for the RPC to be sent, but block
@@ -638,8 +646,7 @@
   // Write file and start second data node.
   private ArrayList<Block> writeFile(final String METHOD_NAME,
                                                final long fileSize,
-                                               Path filePath)
-    throws IOException {
+                                               Path filePath) {
     ArrayList<Block> blocks = null;
     try {
       REPL_FACTOR = 2;
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/datanode/TestDataNodeVolumeFailure.java b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/datanode/TestDataNodeVolumeFailure.java
index 27b8c05..1ebee2f 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/datanode/TestDataNodeVolumeFailure.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/datanode/TestDataNodeVolumeFailure.java
@@ -45,6 +45,7 @@
 import org.apache.hadoop.hdfs.server.namenode.FSNamesystem;
 import org.apache.hadoop.hdfs.server.protocol.DatanodeRegistration;
 import org.apache.hadoop.hdfs.server.protocol.NamenodeProtocols;
+import org.apache.hadoop.hdfs.server.protocol.StorageBlockReport;
 import org.apache.hadoop.net.NetUtils;
 import org.junit.After;
 import org.junit.Before;
@@ -144,8 +145,9 @@
     DataNode dn = cluster.getDataNodes().get(1); //corresponds to dir data3
     String bpid = cluster.getNamesystem().getBlockPoolId();
     DatanodeRegistration dnR = dn.getDNRegistrationForBP(bpid);
-    long[] bReport = dn.getFSDataset().getBlockReport(bpid).getBlockListAsLongs();
-    cluster.getNameNodeRpc().blockReport(dnR, bpid, bReport);
+    StorageBlockReport[] report = { new StorageBlockReport(dnR.getStorageID(),
+        dn.getFSDataset().getBlockReport(bpid).getBlockListAsLongs()) };
+    cluster.getNameNodeRpc().blockReport(dnR, bpid, report);
 
     // verify number of blocks and files...
     verify(filename, filesize);
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/datanode/TestDatanodeRegister.java b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/datanode/TestDatanodeRegister.java
index 2c47b74..09b2b15 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/datanode/TestDatanodeRegister.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/datanode/TestDatanodeRegister.java
@@ -24,9 +24,8 @@
 import static org.mockito.Mockito.*;
 import org.apache.commons.logging.Log;
 import org.apache.commons.logging.LogFactory;
-import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.hdfs.protocolPB.DatanodeProtocolClientSideTranslatorPB;
 import org.apache.hadoop.hdfs.server.common.IncorrectVersionException;
-import org.apache.hadoop.hdfs.server.protocol.DatanodeProtocol;
 import org.apache.hadoop.hdfs.server.protocol.DatanodeRegistration;
 import org.apache.hadoop.hdfs.server.protocol.NamespaceInfo;
 import org.junit.Test;
@@ -49,7 +48,8 @@
 
     NamespaceInfo fakeNSInfo = mock(NamespaceInfo.class);
     when(fakeNSInfo.getBuildVersion()).thenReturn("NSBuildVersion");
-    DatanodeProtocol fakeDNProt = mock(DatanodeProtocol.class);
+    DatanodeProtocolClientSideTranslatorPB fakeDNProt = 
+        mock(DatanodeProtocolClientSideTranslatorPB.class);
     when(fakeDNProt.versionRequest()).thenReturn(fakeNSInfo);
 
     bpos.setNameNode( fakeDNProt );
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/datanode/TestInterDatanodeProtocol.java b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/datanode/TestInterDatanodeProtocol.java
index 953f5e6..26fc102 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/datanode/TestInterDatanodeProtocol.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/datanode/TestInterDatanodeProtocol.java
@@ -28,6 +28,7 @@
 import org.apache.hadoop.io.Writable;
 import org.apache.hadoop.io.LongWritable;
 import org.apache.hadoop.ipc.RPC;
+import org.apache.hadoop.ipc.RpcPayloadHeader.RpcKind;
 import org.apache.hadoop.ipc.Server;
 import org.apache.hadoop.net.NetUtils;
 
@@ -81,7 +82,7 @@
     }
 
     @Override
-    public Writable call(Class<?> protocol, Writable param, long receiveTime)
+    public Writable call(RpcKind rpcKind, String protocol, Writable param, long receiveTime)
         throws IOException {
       if (sleep) {
         // sleep a bit
@@ -149,7 +150,6 @@
       DataNode datanode = cluster.getDataNode(datanodeinfo[0].getIpcPort());
       InterDatanodeProtocol idp = DataNode.createInterDataNodeProtocolProxy(
           datanodeinfo[0], conf, datanode.getDnConf().socketTimeout);
-      assertTrue(datanode != null);
       
       //stop block scanner, so we could compare lastScanTime
       if (datanode.blockScanner != null) {
@@ -346,8 +346,8 @@
   /** Test to verify that InterDatanode RPC timesout as expected when
    *  the server DN does not respond.
    */
-  @Test
-  public void testInterDNProtocolTimeout() throws Exception {
+  @Test(expected=SocketTimeoutException.class)
+  public void testInterDNProtocolTimeout() throws Throwable {
     final Server server = new TestServer(1, true);
     server.start();
 
@@ -360,10 +360,9 @@
     try {
       proxy = DataNode.createInterDataNodeProtocolProxy(
           dInfo, conf, 500);
-      proxy.initReplicaRecovery(null);
+      proxy.initReplicaRecovery(new RecoveringBlock(
+          new ExtendedBlock("bpid", 1), null, 100));
       fail ("Expected SocketTimeoutException exception, but did not get.");
-    } catch (SocketTimeoutException e) {
-      DataNode.LOG.info("Got expected Exception: SocketTimeoutException" + e);
     } finally {
       if (proxy != null) {
         RPC.stopProxy(proxy);
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/FSImageTestUtil.java b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/FSImageTestUtil.java
index a539f55..9fa1fce 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/FSImageTestUtil.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/FSImageTestUtil.java
@@ -397,12 +397,9 @@
    */
   public static EditLogFile findLatestEditsLog(StorageDirectory sd)
   throws IOException {
-    FSImageTransactionalStorageInspector inspector =
-      new FSImageTransactionalStorageInspector();
-    inspector.inspectDirectory(sd);
-    
-    List<EditLogFile> foundEditLogs = Lists.newArrayList(
-        inspector.getEditLogFiles());
+    File currentDir = sd.getCurrentDir();
+    List<EditLogFile> foundEditLogs 
+      = Lists.newArrayList(FileJournalManager.matchEditLogs(currentDir.listFiles()));
     return Collections.max(foundEditLogs, EditLogFile.COMPARE_BY_START_TXID);
   }
 
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/NNThroughputBenchmark.java b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/NNThroughputBenchmark.java
index bac38d2..d128167 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/NNThroughputBenchmark.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/NNThroughputBenchmark.java
@@ -25,8 +25,6 @@
 import java.util.EnumSet;
 import java.util.List;
 
-import javax.security.auth.login.LoginException;
-
 import org.apache.commons.logging.Log;
 import org.apache.commons.logging.LogFactory;
 import org.apache.commons.logging.impl.Log4JLogger;
@@ -46,8 +44,13 @@
 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.NamenodeProtocols;
 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.HdfsConfiguration;
 import org.apache.hadoop.hdfs.DFSConfigKeys;
 import org.apache.hadoop.io.EnumSetWritable;
@@ -103,7 +106,7 @@
   static NameNode nameNode;
   static NamenodeProtocols nameNodeProto;
 
-  NNThroughputBenchmark(Configuration conf) throws IOException, LoginException {
+  NNThroughputBenchmark(Configuration conf) throws IOException {
     config = conf;
     // We do not need many handlers, since each thread simulates a handler
     // by calling name-node methods directly
@@ -124,7 +127,7 @@
     nameNodeProto = nameNode.getRpcServer();
   }
 
-  void close() throws IOException {
+  void close() {
     nameNode.stop();
   }
 
@@ -794,7 +797,10 @@
       dnRegistration.setStorageInfo(new DataStorage(nsInfo, ""));
       DataNode.setNewStorageID(dnRegistration);
       // register datanode
-      dnRegistration = nameNodeProto.registerDatanode(dnRegistration);
+      
+      DatanodeStorage[] storages = { new DatanodeStorage(
+          dnRegistration.getStorageID(), DatanodeStorage.State.NORMAL) };
+      dnRegistration = nameNodeProto.registerDatanode(dnRegistration, storages);
     }
 
     /**
@@ -804,8 +810,10 @@
     void sendHeartbeat() throws IOException {
       // register datanode
       // TODO:FEDERATION currently a single block pool is supported
+      StorageReport[] rep = { new StorageReport(dnRegistration.getStorageID(),
+          false, DF_CAPACITY, DF_USED, DF_CAPACITY - DF_USED, DF_USED) };
       DatanodeCommand[] cmds = nameNodeProto.sendHeartbeat(dnRegistration,
-          DF_CAPACITY, DF_USED, DF_CAPACITY - DF_USED, DF_USED, 0, 0, 0);
+          rep, 0, 0, 0);
       if(cmds != null) {
         for (DatanodeCommand cmd : cmds ) {
           if(LOG.isDebugEnabled()) {
@@ -848,9 +856,10 @@
     @SuppressWarnings("unused") // keep it for future blockReceived benchmark
     int replicateBlocks() throws IOException {
       // register datanode
-      // TODO:FEDERATION currently a single block pool is supported
+      StorageReport[] rep = { new StorageReport(dnRegistration.getStorageID(),
+          false, DF_CAPACITY, DF_USED, DF_CAPACITY - DF_USED, DF_USED) };
       DatanodeCommand[] cmds = nameNodeProto.sendHeartbeat(dnRegistration,
-          DF_CAPACITY, DF_USED, DF_CAPACITY - DF_USED, DF_USED, 0, 0, 0);
+          rep, 0, 0, 0);
       if (cmds != null) {
         for (DatanodeCommand cmd : cmds) {
           if (cmd.getAction() == DatanodeProtocol.DNA_TRANSFER) {
@@ -880,10 +889,12 @@
           receivedDNReg.setStorageInfo(
                           new DataStorage(nsInfo, dnInfo.getStorageID()));
           receivedDNReg.setInfoPort(dnInfo.getInfoPort());
-          nameNodeProto.blockReceived( receivedDNReg, 
-                                  nameNode.getNamesystem().getBlockPoolId(),
-                                  new Block[] {blocks[i]},
-                                  new String[] {DataNode.EMPTY_DEL_HINT});
+          ReceivedDeletedBlockInfo[] rdBlocks = { new ReceivedDeletedBlockInfo(
+                  blocks[i], DataNode.EMPTY_DEL_HINT) };
+          StorageReceivedDeletedBlocks[] report = { new StorageReceivedDeletedBlocks(
+              receivedDNReg.getStorageID(), rdBlocks) };
+          nameNodeProto.blockReceivedAndDeleted(receivedDNReg, nameNode
+              .getNamesystem().getBlockPoolId(), report);
         }
       }
       return blocks.length;
@@ -915,7 +926,7 @@
       config.setLong(DFSConfigKeys.DFS_HEARTBEAT_INTERVAL_KEY, 3 * 60);
       parseArguments(args);
       // adjust replication to the number of data-nodes
-      this.replication = (short)Math.min((int)replication, getNumDatanodes());
+      this.replication = (short)Math.min(replication, getNumDatanodes());
     }
 
     /**
@@ -995,11 +1006,12 @@
         for(DatanodeInfo dnInfo : loc.getLocations()) {
           int dnIdx = Arrays.binarySearch(datanodes, dnInfo.getName());
           datanodes[dnIdx].addBlock(loc.getBlock().getLocalBlock());
-          nameNodeProto.blockReceived(
-              datanodes[dnIdx].dnRegistration, 
-              loc.getBlock().getBlockPoolId(),
-              new Block[] {loc.getBlock().getLocalBlock()},
-              new String[] {""});
+          ReceivedDeletedBlockInfo[] rdBlocks = { new ReceivedDeletedBlockInfo(
+              loc.getBlock().getLocalBlock(), "") };
+          StorageReceivedDeletedBlocks[] report = { new StorageReceivedDeletedBlocks(
+              datanodes[dnIdx].dnRegistration.getStorageID(), rdBlocks) };
+          nameNodeProto.blockReceivedAndDeleted(datanodes[dnIdx].dnRegistration, loc
+              .getBlock().getBlockPoolId(), report);
         }
       }
       return prevBlock;
@@ -1016,8 +1028,10 @@
       assert daemonId < numThreads : "Wrong daemonId.";
       TinyDatanode dn = datanodes[daemonId];
       long start = System.currentTimeMillis();
+      StorageBlockReport[] report = { new StorageBlockReport(
+          dn.dnRegistration.getStorageID(), dn.getBlockReportList()) };
       nameNodeProto.blockReport(dn.dnRegistration, nameNode.getNamesystem()
-          .getBlockPoolId(), dn.getBlockReportList());
+          .getBlockPoolId(), report);
       long end = System.currentTimeMillis();
       return end-start;
     }
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/NameNodeAdapter.java b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/NameNodeAdapter.java
index 2183016..fb1fc6b 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/NameNodeAdapter.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/NameNodeAdapter.java
@@ -53,7 +53,7 @@
    * @return rpc server
    */
   public static Server getRpcServer(NameNode namenode) {
-    return ((NameNodeRpcServer)namenode.getRpcServer()).server;
+    return ((NameNodeRpcServer)namenode.getRpcServer()).clientRpcServer;
   }
 
   public static DelegationTokenSecretManager getDtSecretManager(
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestCheckPointForSecurityTokens.java b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestCheckPointForSecurityTokens.java
index 3454f7f..fbbcfc7 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestCheckPointForSecurityTokens.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestCheckPointForSecurityTokens.java
@@ -84,8 +84,10 @@
       for (StorageDirectory sd : nn.getFSImage().getStorage().dirIterable(null)) {
         EditLogFile log = FSImageTestUtil.findLatestEditsLog(sd);
         assertTrue(log.isInProgress());
+        log.validateLog();
+        long numTransactions = (log.getLastTxId() - log.getFirstTxId()) + 1;
         assertEquals("In-progress log " + log + " should have 5 transactions",
-            5, log.validateLog().numTransactions);
+                     5, numTransactions);;
       }
 
       // Saving image in safe mode should succeed
@@ -99,8 +101,10 @@
       for (StorageDirectory sd : nn.getFSImage().getStorage().dirIterable(null)) {
         EditLogFile log = FSImageTestUtil.findLatestEditsLog(sd);
         assertTrue(log.isInProgress());
+        log.validateLog();
+        long numTransactions = (log.getLastTxId() - log.getFirstTxId()) + 1;
         assertEquals("In-progress log " + log + " should only have START txn",
-            1, log.validateLog().numTransactions);
+            1, numTransactions);
       }
 
       // restart cluster
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestCheckpoint.java b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestCheckpoint.java
index 924f4d0..25db608 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestCheckpoint.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestCheckpoint.java
@@ -1200,7 +1200,7 @@
       CheckpointStorage spyImage1 = spyOnSecondaryImage(secondary1);
       DelayAnswer delayer = new DelayAnswer(LOG);
       Mockito.doAnswer(delayer).when(spyImage1)
-        .saveFSImageInAllDirs(Mockito.anyLong());
+        .saveFSImageInAllDirs(Mockito.<FSNamesystem>any(), Mockito.anyLong());
 
       // Set up a thread to do a checkpoint from the first 2NN
       DoCheckpointThread checkpointThread = new DoCheckpointThread(secondary1);
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestClusterId.java b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestClusterId.java
index 64e7a96..aed6787 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestClusterId.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestClusterId.java
@@ -48,7 +48,7 @@
     // see if cluster id not empty.
     Collection<URI> dirsToFormat = FSNamesystem.getNamespaceDirs(config);
     Collection<URI> editsToFormat = new ArrayList<URI>(0);
-    FSImage fsImage = new FSImage(config, null, dirsToFormat, editsToFormat);
+    FSImage fsImage = new FSImage(config, dirsToFormat, editsToFormat);
     
     Iterator<StorageDirectory> sdit = 
       fsImage.getStorage().dirIterator(NNStorage.NameNodeDirType.IMAGE);
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestDeadDatanode.java b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestDeadDatanode.java
index 021d60a..65b763c 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestDeadDatanode.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestDeadDatanode.java
@@ -36,6 +36,11 @@
 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.RegisterCommand;
+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.junit.After;
 import org.junit.Test;
 
@@ -104,21 +109,24 @@
 
     DatanodeProtocol dnp = cluster.getNameNodeRpc();
     
-    Block[] blocks = new Block[] { new Block(0) };
-    String[] delHints = new String[] { "" };
+    ReceivedDeletedBlockInfo[] blocks = { new ReceivedDeletedBlockInfo(
+        new Block(0), "") };
+    StorageReceivedDeletedBlocks[] storageBlocks = { 
+        new StorageReceivedDeletedBlocks(reg.getStorageID(), blocks) };
     
     // Ensure blockReceived call from dead datanode is rejected with IOException
     try {
-      dnp.blockReceived(reg, poolId, blocks, delHints);
+      dnp.blockReceivedAndDeleted(reg, poolId, storageBlocks);
       Assert.fail("Expected IOException is not thrown");
     } catch (IOException ex) {
       // Expected
     }
 
     // Ensure blockReport from dead datanode is rejected with IOException
-    long[] blockReport = new long[] { 0L, 0L, 0L };
+    StorageBlockReport[] report = { new StorageBlockReport(reg.getStorageID(),
+        new long[] { 0L, 0L, 0L }) };
     try {
-      dnp.blockReport(reg, poolId, blockReport);
+      dnp.blockReport(reg, poolId, report);
       Assert.fail("Expected IOException is not thrown");
     } catch (IOException ex) {
       // Expected
@@ -126,9 +134,11 @@
 
     // Ensure heartbeat from dead datanode is rejected with a command
     // that asks datanode to register again
-    DatanodeCommand[] cmd = dnp.sendHeartbeat(reg, 0, 0, 0, 0, 0, 0, 0);
+    StorageReport[] rep = { new StorageReport(reg.getStorageID(), false, 0, 0,
+        0, 0) };
+    DatanodeCommand[] cmd = dnp.sendHeartbeat(reg, rep, 0, 0, 0);
     Assert.assertEquals(1, cmd.length);
-    Assert.assertEquals(cmd[0].getAction(), DatanodeCommand.REGISTER
+    Assert.assertEquals(cmd[0].getAction(), RegisterCommand.REGISTER
         .getAction());
   }
 }
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestEditLog.java b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestEditLog.java
index bc40e03..4ec9202 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestEditLog.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestEditLog.java
@@ -23,6 +23,9 @@
 import java.util.Collection;
 import java.util.Iterator;
 import java.util.List;
+import java.util.ArrayList;
+import java.util.Collections;
+import java.util.Arrays;
 import java.util.concurrent.Callable;
 import java.util.concurrent.ExecutorService;
 import java.util.concurrent.Executors;
@@ -37,6 +40,7 @@
 import org.apache.hadoop.fs.FileSystem;
 import org.apache.hadoop.fs.Path;
 import org.apache.hadoop.fs.permission.*;
+import org.apache.hadoop.hdfs.protocol.HdfsConstants;
 
 import org.apache.hadoop.hdfs.DFSConfigKeys;
 import org.apache.hadoop.hdfs.HdfsConfiguration;
@@ -54,6 +58,7 @@
 import org.mockito.Mockito;
 import org.junit.Test;
 
+import com.google.common.collect.ImmutableList;
 import com.google.common.collect.Lists;
 
 import static org.apache.hadoop.test.MetricsAsserts.*;
@@ -76,7 +81,7 @@
   static final int NUM_TRANSACTIONS = 100;
   static final int NUM_THREADS = 100;
   
-  private static final File TEST_DIR = new File(
+  static final File TEST_DIR = new File(
     System.getProperty("test.build.data","build/test/data"));
 
   /** An edits log with 3 edits from 0.20 - the result of
@@ -631,13 +636,23 @@
   }
   
   public void testCrashRecoveryEmptyLogOneDir() throws Exception {
-    doTestCrashRecoveryEmptyLog(false);
+    doTestCrashRecoveryEmptyLog(false, true);
   }
   
   public void testCrashRecoveryEmptyLogBothDirs() throws Exception {
-    doTestCrashRecoveryEmptyLog(true);
+    doTestCrashRecoveryEmptyLog(true, true);
+  }
+
+  public void testCrashRecoveryEmptyLogOneDirNoUpdateSeenTxId() 
+      throws Exception {
+    doTestCrashRecoveryEmptyLog(false, false);
   }
   
+  public void testCrashRecoveryEmptyLogBothDirsNoUpdateSeenTxId()
+      throws Exception {
+    doTestCrashRecoveryEmptyLog(true, false);
+  }
+
   /**
    * Test that the NN handles the corruption properly
    * after it crashes just after creating an edit log
@@ -650,8 +665,14 @@
    * will only be in one of the directories. In both cases, the
    * NN should fail to start up, because it's aware that txid 3
    * was reached, but unable to find a non-corrupt log starting there.
+   * @param updateTransactionIdFile if true update the seen_txid file.
+   * If false, the it will not be updated. This will simulate a case 
+   * where the NN crashed between creating the new segment and updating
+   * seen_txid. 
    */
-  private void doTestCrashRecoveryEmptyLog(boolean inBothDirs) throws Exception {
+  private void doTestCrashRecoveryEmptyLog(boolean inBothDirs, 
+                                           boolean updateTransactionIdFile) 
+      throws Exception {
     // start a cluster 
     Configuration conf = new HdfsConfiguration();
     MiniDFSCluster cluster = null;
@@ -669,6 +690,14 @@
       // Make a truncated edits_3_inprogress
       File log = new File(currentDir,
           NNStorage.getInProgressEditsFileName(3));
+      NNStorage storage = new NNStorage(conf, 
+                                        Collections.<URI>emptyList(),
+                                        Lists.newArrayList(uri));
+      if (updateTransactionIdFile) {
+        storage.writeTransactionIdFileToStorage(3);
+      }
+      storage.close();
+
       new EditLogFileOutputStream(log, 1024).create();
       if (!inBothDirs) {
         break;
@@ -679,9 +708,9 @@
       cluster = new MiniDFSCluster.Builder(conf)
         .numDataNodes(NUM_DATA_NODES).format(false).build();
       fail("Did not fail to start with all-corrupt logs");
-    } catch (IllegalStateException ise) {
+    } catch (IOException ioe) {
       GenericTestUtils.assertExceptionContains(
-          "No non-corrupt logs for txid 3", ise);
+          "No non-corrupt logs for txid 3", ioe);
     }
     cluster.shutdown();
   }
@@ -706,8 +735,18 @@
             
       reader = new FSEditLogOp.Reader(in, version);
     }
+  
+    @Override
+    public long getFirstTxId() throws IOException {
+      return HdfsConstants.INVALID_TXID;
+    }
     
     @Override
+    public long getLastTxId() throws IOException {
+      return HdfsConstants.INVALID_TXID;
+    }
+  
+    @Override
     public long length() throws IOException {
       return len;
     }
@@ -856,6 +895,168 @@
     Mockito.doReturn(sds).when(storage).dirIterable(NameNodeDirType.EDITS);
     return storage;
   }
-  
-  
+
+  /** 
+   * Specification for a failure during #setupEdits
+   */
+  static class AbortSpec {
+    final int roll;
+    final int logindex;
+    
+    /**
+     * Construct the failure specification. 
+     * @param roll number to fail after. e.g. 1 to fail after the first roll
+     * @param loginfo index of journal to fail. 
+     */
+    AbortSpec(int roll, int logindex) {
+      this.roll = roll;
+      this.logindex = logindex;
+    }
+  }
+
+  final static int TXNS_PER_ROLL = 10;  
+  final static int TXNS_PER_FAIL = 2;
+    
+  /**
+   * Set up directories for tests. 
+   *
+   * Each rolled file is 10 txns long. 
+   * A failed file is 2 txns long.
+   * 
+   * @param editUris directories to create edit logs in
+   * @param numrolls number of times to roll the edit log during setup
+   * @param abortAtRolls Specifications for when to fail, see AbortSpec
+   */
+  public static NNStorage setupEdits(List<URI> editUris, int numrolls, 
+                                     AbortSpec... abortAtRolls)
+      throws IOException {
+    List<AbortSpec> aborts = new ArrayList<AbortSpec>(Arrays.asList(abortAtRolls));
+    NNStorage storage = new NNStorage(new Configuration(),
+                                      Collections.<URI>emptyList(),
+                                      editUris);
+    storage.format("test-cluster-id");
+    FSEditLog editlog = new FSEditLog(storage);    
+    // open the edit log and add two transactions
+    // logGenerationStamp is used, simply because it doesn't 
+    // require complex arguments.
+    editlog.open();
+    for (int i = 2; i < TXNS_PER_ROLL; i++) {
+      editlog.logGenerationStamp((long)0);
+    }
+    editlog.logSync();
+    
+    // Go into edit log rolling loop.
+    // On each roll, the abortAtRolls abort specs are 
+    // checked to see if an abort is required. If so the 
+    // the specified journal is aborted. It will be brought
+    // back into rotation automatically by rollEditLog
+    for (int i = 0; i < numrolls; i++) {
+      editlog.rollEditLog();
+      
+      editlog.logGenerationStamp((long)i);
+      editlog.logSync();
+
+      while (aborts.size() > 0 
+             && aborts.get(0).roll == (i+1)) {
+        AbortSpec spec = aborts.remove(0);
+        editlog.getJournals().get(spec.logindex).abort();
+      } 
+      
+      for (int j = 3; j < TXNS_PER_ROLL; j++) {
+        editlog.logGenerationStamp((long)i);
+      }
+      editlog.logSync();
+    }
+    editlog.close();
+
+    FSImageTestUtil.logStorageContents(LOG, storage);
+    return storage;
+  }
+
+  /** 
+   * Test loading an editlog which has had both its storage fail
+   * on alternating rolls. Two edit log directories are created.
+   * The first on fails on odd rolls, the second on even. Test
+   * that we are able to load the entire editlog regardless.
+   */
+  @Test
+  public void testAlternatingJournalFailure() throws IOException {
+    File f1 = new File(TEST_DIR + "/alternatingjournaltest0");
+    File f2 = new File(TEST_DIR + "/alternatingjournaltest1");
+
+    List<URI> editUris = ImmutableList.of(f1.toURI(), f2.toURI());
+    
+    NNStorage storage = setupEdits(editUris, 10,
+                                   new AbortSpec(1, 0),
+                                   new AbortSpec(2, 1),
+                                   new AbortSpec(3, 0),
+                                   new AbortSpec(4, 1),
+                                   new AbortSpec(5, 0),
+                                   new AbortSpec(6, 1),
+                                   new AbortSpec(7, 0),
+                                   new AbortSpec(8, 1),
+                                   new AbortSpec(9, 0),
+                                   new AbortSpec(10, 1));
+    long totaltxnread = 0;
+    FSEditLog editlog = new FSEditLog(storage);
+    long startTxId = 1;
+    Iterable<EditLogInputStream> editStreams = editlog.selectInputStreams(startTxId, 
+                                                                          TXNS_PER_ROLL*11);
+
+    for (EditLogInputStream edits : editStreams) {
+      FSEditLogLoader.EditLogValidation val = FSEditLogLoader.validateEditLog(edits);
+      long read = val.getNumTransactions();
+      LOG.info("Loading edits " + edits + " read " + read);
+      assertEquals(startTxId, val.getStartTxId());
+      startTxId += read;
+      totaltxnread += read;
+    }
+
+    editlog.close();
+    storage.close();
+    assertEquals(TXNS_PER_ROLL*11, totaltxnread);    
+  }
+
+  /** 
+   * Test loading an editlog with gaps. A single editlog directory
+   * is set up. On of the edit log files is deleted. This should
+   * fail when selecting the input streams as it will not be able 
+   * to select enough streams to load up to 4*TXNS_PER_ROLL.
+   * There should be 4*TXNS_PER_ROLL transactions as we rolled 3
+   * times. 
+   */
+  @Test
+  public void testLoadingWithGaps() throws IOException {
+    File f1 = new File(TEST_DIR + "/gaptest0");
+    List<URI> editUris = ImmutableList.of(f1.toURI());
+
+    NNStorage storage = setupEdits(editUris, 3);
+    
+    final long startGapTxId = 1*TXNS_PER_ROLL + 1;
+    final long endGapTxId = 2*TXNS_PER_ROLL;
+
+    File[] files = new File(f1, "current").listFiles(new FilenameFilter() {
+        public boolean accept(File dir, String name) {
+          if (name.startsWith(NNStorage.getFinalizedEditsFileName(startGapTxId, 
+                                  endGapTxId))) {
+            return true;
+          }
+          return false;
+        }
+      });
+    assertEquals(1, files.length);
+    assertTrue(files[0].delete());
+    
+    FSEditLog editlog = new FSEditLog(storage);
+    long startTxId = 1;
+    try {
+      Iterable<EditLogInputStream> editStreams 
+        = editlog.selectInputStreams(startTxId, 4*TXNS_PER_ROLL);
+      
+      fail("Should have thrown exception");
+    } catch (IOException ioe) {
+      GenericTestUtils.assertExceptionContains(
+          "No non-corrupt logs for txid " + startGapTxId, ioe);
+    }
+  }
 }
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestEditLogFileOutputStream.java b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestEditLogFileOutputStream.java
index a673c5f3..1228bef 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestEditLogFileOutputStream.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestEditLogFileOutputStream.java
@@ -63,8 +63,8 @@
 
     EditLogValidation validation = EditLogFileInputStream.validateEditLog(editLog);
     assertEquals("Edit log should contain a header as valid length",
-        HEADER_LEN, validation.validLength);
-    assertEquals(1, validation.numTransactions);
+        HEADER_LEN, validation.getValidLength());
+    assertEquals(1, validation.getNumTransactions());
     assertEquals("Edit log should have 1MB of bytes allocated",
         1024*1024, editLog.length());
     
@@ -72,12 +72,12 @@
     cluster.getFileSystem().mkdirs(new Path("/tmp"),
         new FsPermission((short)777));
 
-    long oldLength = validation.validLength;
+    long oldLength = validation.getValidLength();
     validation = EditLogFileInputStream.validateEditLog(editLog);
     assertTrue("Edit log should have more valid data after writing a txn " +
-        "(was: " + oldLength + " now: " + validation.validLength + ")",
-        validation.validLength > oldLength);
-    assertEquals(2, validation.numTransactions);
+        "(was: " + oldLength + " now: " + validation.getValidLength() + ")",
+        validation.getValidLength() > oldLength);
+    assertEquals(2, validation.getNumTransactions());
 
     assertEquals("Edit log should be 1MB long",
         1024 * 1024, editLog.length());
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestEditLogRace.java b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestEditLogRace.java
index b305296..bc5aa16 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestEditLogRace.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestEditLogRace.java
@@ -350,7 +350,7 @@
     Configuration conf = getConf();
     NameNode.initMetrics(conf, NamenodeRole.NAMENODE);
     DFSTestUtil.formatNameNode(conf);
-    final FSNamesystem namesystem = new FSNamesystem(conf);
+    final FSNamesystem namesystem = FSNamesystem.loadFromDisk(conf);
 
     try {
       FSImage fsimage = namesystem.getFSImage();
@@ -448,7 +448,7 @@
     Configuration conf = getConf();
     NameNode.initMetrics(conf, NamenodeRole.NAMENODE);
     DFSTestUtil.formatNameNode(conf);
-    final FSNamesystem namesystem = new FSNamesystem(conf);
+    final FSNamesystem namesystem = FSNamesystem.loadFromDisk(conf);
 
     try {
       FSImage fsimage = namesystem.getFSImage();
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestFSEditLogLoader.java b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestFSEditLogLoader.java
index 15b831e..e820705 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestFSEditLogLoader.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestFSEditLogLoader.java
@@ -189,8 +189,8 @@
     // Make sure that uncorrupted log has the expected length and number
     // of transactions.
     EditLogValidation validation = EditLogFileInputStream.validateEditLog(logFile);
-    assertEquals(NUM_TXNS + 2, validation.numTransactions);
-    assertEquals(validLength, validation.validLength);
+    assertEquals(NUM_TXNS + 2, validation.getNumTransactions());
+    assertEquals(validLength, validation.getValidLength());
     
     // Back up the uncorrupted log
     File logFileBak = new File(testDir, logFile.getName() + ".bak");
@@ -206,8 +206,8 @@
       truncateFile(logFile, txOffset);
       validation = EditLogFileInputStream.validateEditLog(logFile);
       assertEquals("Failed when truncating to length " + txOffset,
-          txid - 1, validation.numTransactions);
-      assertEquals(txOffset, validation.validLength);
+          txid - 1, validation.getNumTransactions());
+      assertEquals(txOffset, validation.getValidLength());
 
       // Restore backup, truncate the file with one byte in the txn,
       // also isn't valid
@@ -215,24 +215,24 @@
       truncateFile(logFile, txOffset + 1);
       validation = EditLogFileInputStream.validateEditLog(logFile);
       assertEquals("Failed when truncating to length " + (txOffset + 1),
-          txid - 1, validation.numTransactions);
-      assertEquals(txOffset, validation.validLength);
+          txid - 1, validation.getNumTransactions());
+      assertEquals(txOffset, validation.getValidLength());
 
       // Restore backup, corrupt the txn opcode
       Files.copy(logFileBak, logFile);
       corruptByteInFile(logFile, txOffset);
       validation = EditLogFileInputStream.validateEditLog(logFile);
       assertEquals("Failed when corrupting txn opcode at " + txOffset,
-          txid - 1, validation.numTransactions);
-      assertEquals(txOffset, validation.validLength);
+          txid - 1, validation.getNumTransactions());
+      assertEquals(txOffset, validation.getValidLength());
 
       // Restore backup, corrupt a byte a few bytes into the txn
       Files.copy(logFileBak, logFile);
       corruptByteInFile(logFile, txOffset+5);
       validation = EditLogFileInputStream.validateEditLog(logFile);
       assertEquals("Failed when corrupting txn data at " + (txOffset+5),
-          txid - 1, validation.numTransactions);
-      assertEquals(txOffset, validation.validLength);
+          txid - 1, validation.getNumTransactions());
+      assertEquals(txOffset, validation.getValidLength());
     }
     
     // Corrupt the log at every offset to make sure that validation itself
@@ -243,8 +243,8 @@
       Files.copy(logFileBak, logFile);
       corruptByteInFile(logFile, offset);
       EditLogValidation val = EditLogFileInputStream.validateEditLog(logFile);
-      assertTrue(val.numTransactions >= prevNumValid);
-      prevNumValid = val.numTransactions;
+      assertTrue(val.getNumTransactions() >= prevNumValid);
+      prevNumValid = val.getNumTransactions();
     }
   }
 
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestFSImageStorageInspector.java b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestFSImageStorageInspector.java
index 113dcbc..649c415 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestFSImageStorageInspector.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestFSImageStorageInspector.java
@@ -36,9 +36,6 @@
 
 import org.apache.hadoop.hdfs.server.namenode.FileJournalManager.EditLogFile;
 import org.apache.hadoop.hdfs.server.namenode.FSImageStorageInspector.FSImageFile;
-import org.apache.hadoop.hdfs.server.namenode.FSImageTransactionalStorageInspector.TransactionalLoadPlan;
-import org.apache.hadoop.hdfs.server.namenode.FSImageTransactionalStorageInspector.LogGroup;
-import org.apache.hadoop.hdfs.server.namenode.FSImageStorageInspector.LoadPlan;
 import org.junit.Test;
 import org.mockito.Mockito;
 
@@ -63,335 +60,14 @@
         "/foo/current/" + getInProgressEditsFileName(457));
 
     inspector.inspectDirectory(mockDir);
-    mockLogValidation(inspector,
-        "/foo/current/" + getInProgressEditsFileName(457), 10);
-    
-    assertEquals(2, inspector.foundEditLogs.size());
     assertEquals(2, inspector.foundImages.size());
-    assertTrue(inspector.foundEditLogs.get(1).isInProgress());
-    
+
     FSImageFile latestImage = inspector.getLatestImage();
     assertEquals(456, latestImage.txId);
     assertSame(mockDir, latestImage.sd);
     assertTrue(inspector.isUpgradeFinalized());
     
-    LoadPlan plan = inspector.createLoadPlan();
-    LOG.info("Plan: " + plan);
-    
     assertEquals(new File("/foo/current/"+getImageFileName(456)), 
-                 plan.getImageFile());
-    assertArrayEquals(new File[] {
-        new File("/foo/current/" + getInProgressEditsFileName(457)) },
-        plan.getEditsFiles().toArray(new File[0]));
-  }
-  
-  /**
-   * Test that we check for gaps in txids when devising a load plan.
-   */
-  @Test
-  public void testPlanWithGaps() throws IOException {
-    FSImageTransactionalStorageInspector inspector =
-        new FSImageTransactionalStorageInspector();
-    
-    StorageDirectory mockDir = FSImageTestUtil.mockStorageDirectory(
-        NameNodeDirType.IMAGE_AND_EDITS,
-        false,
-        "/foo/current/" + getImageFileName(123),
-        "/foo/current/" + getImageFileName(456),
-        "/foo/current/" + getFinalizedEditsFileName(457,900),
-        "/foo/current/" + getFinalizedEditsFileName(901,950),
-        "/foo/current/" + getFinalizedEditsFileName(952,1000)); // <-- missing edit 951!
-
-    inspector.inspectDirectory(mockDir);
-    try {
-      inspector.createLoadPlan();
-      fail("Didn't throw IOE trying to load with gaps in edits");
-    } catch (IOException ioe) {
-      assertTrue(ioe.getMessage().contains(
-          "would start at txid 951 but starts at txid 952"));
-    }
-  }
-  
-  /**
-   * Test the case where an in-progress log comes in the middle of a sequence
-   * of logs
-   */
-  @Test
-  public void testPlanWithInProgressInMiddle() throws IOException {
-    FSImageTransactionalStorageInspector inspector =
-        new FSImageTransactionalStorageInspector();
-    
-    StorageDirectory mockDir = FSImageTestUtil.mockStorageDirectory(
-        NameNodeDirType.IMAGE_AND_EDITS,
-        false,
-        "/foo/current/" + getImageFileName(123),
-        "/foo/current/" + getImageFileName(456),
-        "/foo/current/" + getFinalizedEditsFileName(457,900),
-        "/foo/current/" + getInProgressEditsFileName(901), // <-- inprogress in middle
-        "/foo/current/" + getFinalizedEditsFileName(952,1000));
-
-    inspector.inspectDirectory(mockDir);
-    mockLogValidation(inspector,
-        "/foo/current/" + getInProgressEditsFileName(901), 51);
-
-    LoadPlan plan = inspector.createLoadPlan();
-    LOG.info("Plan: " + plan);
-    
-    assertEquals(new File("/foo/current/" + getImageFileName(456)), 
-                 plan.getImageFile());
-    assertArrayEquals(new File[] {
-        new File("/foo/current/" + getFinalizedEditsFileName(457,900)),
-        new File("/foo/current/" + getInProgressEditsFileName(901)),
-        new File("/foo/current/" + getFinalizedEditsFileName(952,1000)) },
-        plan.getEditsFiles().toArray(new File[0]));
-
-  }
-
-  
-  /**
-   * Test case for the usual case where no recovery of a log group is necessary
-   * (i.e all logs have the same start and end txids and finalized)
-   */
-  @Test
-  public void testLogGroupRecoveryNoop() throws IOException {
-    FSImageTransactionalStorageInspector inspector =
-        new FSImageTransactionalStorageInspector();
-
-    inspector.inspectDirectory(
-        mockDirectoryWithEditLogs("/foo1/current/" 
-                                  + getFinalizedEditsFileName(123,456)));
-    inspector.inspectDirectory(
-        mockDirectoryWithEditLogs("/foo2/current/"
-                                  + getFinalizedEditsFileName(123,456)));
-    inspector.inspectDirectory(
-        mockDirectoryWithEditLogs("/foo3/current/"
-                                  + getFinalizedEditsFileName(123,456)));
-    LogGroup lg = inspector.logGroups.get(123L);
-    assertEquals(3, lg.logs.size());
-    
-    lg.planRecovery();
-    
-    assertFalse(lg.logs.get(0).isCorrupt());
-    assertFalse(lg.logs.get(1).isCorrupt());
-    assertFalse(lg.logs.get(2).isCorrupt());
-  }
-  
-  /**
-   * Test case where we have some in-progress and some finalized logs
-   * for a given txid.
-   */
-  @Test
-  public void testLogGroupRecoveryMixed() throws IOException {
-    FSImageTransactionalStorageInspector inspector =
-        new FSImageTransactionalStorageInspector();
-
-    inspector.inspectDirectory(
-        mockDirectoryWithEditLogs("/foo1/current/"
-                                  + getFinalizedEditsFileName(123,456)));
-    inspector.inspectDirectory(
-        mockDirectoryWithEditLogs("/foo2/current/"
-                                  + getFinalizedEditsFileName(123,456)));
-    inspector.inspectDirectory(
-        mockDirectoryWithEditLogs("/foo3/current/"
-                                  + getInProgressEditsFileName(123)));
-    inspector.inspectDirectory(FSImageTestUtil.mockStorageDirectory(
-        NameNodeDirType.IMAGE,
-        false,
-        "/foo4/current/" + getImageFileName(122)));
-
-    LogGroup lg = inspector.logGroups.get(123L);
-    assertEquals(3, lg.logs.size());
-    EditLogFile inProgressLog = lg.logs.get(2);
-    assertTrue(inProgressLog.isInProgress());
-    
-    LoadPlan plan = inspector.createLoadPlan();
-
-    // Check that it was marked corrupt.
-    assertFalse(lg.logs.get(0).isCorrupt());
-    assertFalse(lg.logs.get(1).isCorrupt());
-    assertTrue(lg.logs.get(2).isCorrupt());
-
-    
-    // Calling recover should move it aside
-    inProgressLog = spy(inProgressLog);
-    Mockito.doNothing().when(inProgressLog).moveAsideCorruptFile();
-    lg.logs.set(2, inProgressLog);
-    
-    plan.doRecovery();
-    
-    Mockito.verify(inProgressLog).moveAsideCorruptFile();
-  }
-  
-  /**
-   * Test case where we have finalized logs with different end txids
-   */
-  @Test
-  public void testLogGroupRecoveryInconsistentEndTxIds() throws IOException {
-    FSImageTransactionalStorageInspector inspector =
-        new FSImageTransactionalStorageInspector();
-    inspector.inspectDirectory(
-        mockDirectoryWithEditLogs("/foo1/current/"
-                                  + getFinalizedEditsFileName(123,456)));
-    inspector.inspectDirectory(
-        mockDirectoryWithEditLogs("/foo2/current/"
-                                  + getFinalizedEditsFileName(123,678)));
-
-    LogGroup lg = inspector.logGroups.get(123L);
-    assertEquals(2, lg.logs.size());
-
-    try {
-      lg.planRecovery();
-      fail("Didn't throw IOE on inconsistent end txids");
-    } catch (IOException ioe) {
-      assertTrue(ioe.getMessage().contains("More than one ending txid"));
-    }
-  }
-
-  /**
-   * Test case where we have only in-progress logs and need to synchronize
-   * based on valid length.
-   */
-  @Test
-  public void testLogGroupRecoveryInProgress() throws IOException {
-    String paths[] = new String[] {
-        "/foo1/current/" + getInProgressEditsFileName(123),
-        "/foo2/current/" + getInProgressEditsFileName(123),
-        "/foo3/current/" + getInProgressEditsFileName(123)
-    };
-    FSImageTransactionalStorageInspector inspector =
-        new FSImageTransactionalStorageInspector();
-    inspector.inspectDirectory(mockDirectoryWithEditLogs(paths[0]));
-    inspector.inspectDirectory(mockDirectoryWithEditLogs(paths[1]));
-    inspector.inspectDirectory(mockDirectoryWithEditLogs(paths[2]));
-
-    // Inject spies to return the valid counts we would like to see
-    mockLogValidation(inspector, paths[0], 2000);
-    mockLogValidation(inspector, paths[1], 2000);
-    mockLogValidation(inspector, paths[2], 1000);
-
-    LogGroup lg = inspector.logGroups.get(123L);
-    assertEquals(3, lg.logs.size());
-    
-    lg.planRecovery();
-    
-    // Check that the short one was marked corrupt
-    assertFalse(lg.logs.get(0).isCorrupt());
-    assertFalse(lg.logs.get(1).isCorrupt());
-    assertTrue(lg.logs.get(2).isCorrupt());
-    
-    // Calling recover should move it aside
-    EditLogFile badLog = lg.logs.get(2);
-    Mockito.doNothing().when(badLog).moveAsideCorruptFile();
-    Mockito.doNothing().when(lg.logs.get(0)).finalizeLog();
-    Mockito.doNothing().when(lg.logs.get(1)).finalizeLog();
-    
-    lg.recover();
-    
-    Mockito.verify(badLog).moveAsideCorruptFile();
-    Mockito.verify(lg.logs.get(0)).finalizeLog();
-    Mockito.verify(lg.logs.get(1)).finalizeLog();
-  }
-
-  /**
-   * Mock out the log at the given path to return a specified number
-   * of transactions upon validation.
-   */
-  private void mockLogValidation(
-      FSImageTransactionalStorageInspector inspector,
-      String path, int numValidTransactions) throws IOException {
-    
-    for (LogGroup lg : inspector.logGroups.values()) {
-      List<EditLogFile> logs = lg.logs;
-      for (int i = 0; i < logs.size(); i++) {
-        EditLogFile log = logs.get(i);
-        if (log.getFile().getPath().equals(path)) {
-          // mock out its validation
-          EditLogFile spyLog = spy(log);
-          doReturn(new FSEditLogLoader.EditLogValidation(-1, numValidTransactions))
-            .when(spyLog).validateLog();
-          logs.set(i, spyLog);
-          return;
-        }
-      }
-    }
-    fail("No log found to mock out at " + path);
-  }
-
-  /**
-   * Test when edits and image are in separate directories.
-   */
-  @Test
-  public void testCurrentSplitEditsAndImage() throws IOException {
-    FSImageTransactionalStorageInspector inspector =
-        new FSImageTransactionalStorageInspector();
-    
-    StorageDirectory mockImageDir = FSImageTestUtil.mockStorageDirectory(
-        NameNodeDirType.IMAGE,
-        false,
-        "/foo/current/" + getImageFileName(123));
-    StorageDirectory mockImageDir2 = FSImageTestUtil.mockStorageDirectory(
-        NameNodeDirType.IMAGE,
-        false,
-        "/foo2/current/" + getImageFileName(456));
-    StorageDirectory mockEditsDir = FSImageTestUtil.mockStorageDirectory(
-        NameNodeDirType.EDITS,
-        false,
-        "/foo3/current/" + getFinalizedEditsFileName(123, 456),
-        "/foo3/current/" + getInProgressEditsFileName(457));
-    
-    inspector.inspectDirectory(mockImageDir);
-    inspector.inspectDirectory(mockEditsDir);
-    inspector.inspectDirectory(mockImageDir2);
-    
-    mockLogValidation(inspector,
-        "/foo3/current/" + getInProgressEditsFileName(457), 2);
-
-    assertEquals(2, inspector.foundEditLogs.size());
-    assertEquals(2, inspector.foundImages.size());
-    assertTrue(inspector.foundEditLogs.get(1).isInProgress());
-    assertTrue(inspector.isUpgradeFinalized());    
-
-    // Check plan
-    TransactionalLoadPlan plan =
-      (TransactionalLoadPlan)inspector.createLoadPlan();
-    FSImageFile pickedImage = plan.image;
-    assertEquals(456, pickedImage.txId);
-    assertSame(mockImageDir2, pickedImage.sd);
-    assertEquals(new File("/foo2/current/" + getImageFileName(456)),
-                 plan.getImageFile());
-    assertArrayEquals(new File[] {
-        new File("/foo3/current/" + getInProgressEditsFileName(457))
-      }, plan.getEditsFiles().toArray(new File[0]));
-  }
-  
-  /**
-   * Test case where an in-progress log is in an earlier name directory
-   * than a finalized log. Previously, getEditLogManifest wouldn't
-   * see this log.
-   */
-  @Test
-  public void testLogManifestInProgressComesFirst() throws IOException { 
-    FSImageTransactionalStorageInspector inspector =
-        new FSImageTransactionalStorageInspector();
-    inspector.inspectDirectory(
-        mockDirectoryWithEditLogs("/foo1/current/" 
-                                  + getFinalizedEditsFileName(2622,2623),
-                                  "/foo1/current/"
-                                  + getFinalizedEditsFileName(2624,2625),
-                                  "/foo1/current/"
-                                  + getInProgressEditsFileName(2626)));
-    inspector.inspectDirectory(
-        mockDirectoryWithEditLogs("/foo2/current/"
-                                  + getFinalizedEditsFileName(2622,2623),
-                                  "/foo2/current/"
-                                  + getFinalizedEditsFileName(2624,2625),
-                                  "/foo2/current/"
-                                  + getFinalizedEditsFileName(2626,2627),
-                                  "/foo2/current/"
-                                  + getFinalizedEditsFileName(2628,2629)));
-  }  
-  
-  static StorageDirectory mockDirectoryWithEditLogs(String... fileNames) {
-    return FSImageTestUtil.mockStorageDirectory(NameNodeDirType.EDITS, false, fileNames);
+        latestImage.getFile());
   }
 }
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestFileJournalManager.java b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestFileJournalManager.java
index 748caf4..d2f9781 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestFileJournalManager.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestFileJournalManager.java
@@ -19,17 +19,277 @@
 
 import static org.junit.Assert.*;
 
-import java.io.IOException;
+import java.net.URI;
+import java.util.Collections;
+import java.util.Arrays;
+import java.util.List;
+import java.util.ArrayList;
+import java.util.Iterator;
 
+import java.io.RandomAccessFile;
+import java.io.File;
+import java.io.FilenameFilter;
+import java.io.BufferedInputStream;
+import java.io.DataInputStream;
+import java.io.IOException;
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.fs.FileSystem;
+import org.apache.hadoop.hdfs.DFSConfigKeys;
+import org.apache.hadoop.security.SecurityUtil;
+import org.junit.Test;
 import org.apache.hadoop.hdfs.server.common.Storage.StorageDirectory;
 import org.apache.hadoop.hdfs.server.namenode.NNStorage.NameNodeDirType;
 import org.apache.hadoop.test.GenericTestUtils;
-import org.junit.Test;
+import static org.apache.hadoop.hdfs.server.namenode.TestEditLog.setupEdits;
+import static org.apache.hadoop.hdfs.server.namenode.TestEditLog.AbortSpec;
+import static org.apache.hadoop.hdfs.server.namenode.TestEditLog.TXNS_PER_ROLL;
+import static org.apache.hadoop.hdfs.server.namenode.TestEditLog.TXNS_PER_FAIL;
 
+import com.google.common.collect.ImmutableList;
 import com.google.common.base.Joiner;
 
+import java.util.zip.CheckedInputStream;
+import java.util.zip.Checksum;
+
 public class TestFileJournalManager {
 
+  /** 
+   * Test the normal operation of loading transactions from
+   * file journal manager. 3 edits directories are setup without any
+   * failures. Test that we read in the expected number of transactions.
+   */
+  @Test
+  public void testNormalOperation() throws IOException {
+    File f1 = new File(TestEditLog.TEST_DIR + "/normtest0");
+    File f2 = new File(TestEditLog.TEST_DIR + "/normtest1");
+    File f3 = new File(TestEditLog.TEST_DIR + "/normtest2");
+    
+    List<URI> editUris = ImmutableList.of(f1.toURI(), f2.toURI(), f3.toURI());
+    NNStorage storage = setupEdits(editUris, 5);
+    
+    long numJournals = 0;
+    for (StorageDirectory sd : storage.dirIterable(NameNodeDirType.EDITS)) {
+      FileJournalManager jm = new FileJournalManager(sd);
+      assertEquals(6*TXNS_PER_ROLL, jm.getNumberOfTransactions(1));
+      numJournals++;
+    }
+    assertEquals(3, numJournals);
+  }
+
+  /**
+   * Test that inprogress files are handled correct. Set up a single
+   * edits directory. Fail on after the last roll. Then verify that the 
+   * logs have the expected number of transactions.
+   */
+  @Test
+  public void testInprogressRecovery() throws IOException {
+    File f = new File(TestEditLog.TEST_DIR + "/filejournaltest0");
+    // abort after the 5th roll 
+    NNStorage storage = setupEdits(Collections.<URI>singletonList(f.toURI()),
+                                   5, new AbortSpec(5, 0));
+    StorageDirectory sd = storage.dirIterator(NameNodeDirType.EDITS).next();
+
+    FileJournalManager jm = new FileJournalManager(sd);
+    assertEquals(5*TXNS_PER_ROLL + TXNS_PER_FAIL, 
+                 jm.getNumberOfTransactions(1));
+  }
+
+  /**
+   * Test a mixture of inprogress files and finalised. Set up 3 edits 
+   * directories and fail the second on the last roll. Verify that reading
+   * the transactions, reads from the finalised directories.
+   */
+  @Test
+  public void testInprogressRecoveryMixed() throws IOException {
+    File f1 = new File(TestEditLog.TEST_DIR + "/mixtest0");
+    File f2 = new File(TestEditLog.TEST_DIR + "/mixtest1");
+    File f3 = new File(TestEditLog.TEST_DIR + "/mixtest2");
+    
+    List<URI> editUris = ImmutableList.of(f1.toURI(), f2.toURI(), f3.toURI());
+
+    // abort after the 5th roll 
+    NNStorage storage = setupEdits(editUris,
+                                   5, new AbortSpec(5, 1));
+    Iterator<StorageDirectory> dirs = storage.dirIterator(NameNodeDirType.EDITS);
+    StorageDirectory sd = dirs.next();
+    FileJournalManager jm = new FileJournalManager(sd);
+    assertEquals(6*TXNS_PER_ROLL, jm.getNumberOfTransactions(1));
+    
+    sd = dirs.next();
+    jm = new FileJournalManager(sd);
+    assertEquals(5*TXNS_PER_ROLL + TXNS_PER_FAIL, jm.getNumberOfTransactions(1));
+
+    sd = dirs.next();
+    jm = new FileJournalManager(sd);
+    assertEquals(6*TXNS_PER_ROLL, jm.getNumberOfTransactions(1));
+  }
+
+  /** 
+   * Test that FileJournalManager behaves correctly despite inprogress
+   * files in all its edit log directories. Set up 3 directories and fail
+   * all on the last roll. Verify that the correct number of transaction 
+   * are then loaded.
+   */
+  @Test
+  public void testInprogressRecoveryAll() throws IOException {
+    File f1 = new File(TestEditLog.TEST_DIR + "/failalltest0");
+    File f2 = new File(TestEditLog.TEST_DIR + "/failalltest1");
+    File f3 = new File(TestEditLog.TEST_DIR + "/failalltest2");
+    
+    List<URI> editUris = ImmutableList.of(f1.toURI(), f2.toURI(), f3.toURI());
+    // abort after the 5th roll 
+    NNStorage storage = setupEdits(editUris, 5, 
+                                   new AbortSpec(5, 0),
+                                   new AbortSpec(5, 1),
+                                   new AbortSpec(5, 2));
+    Iterator<StorageDirectory> dirs = storage.dirIterator(NameNodeDirType.EDITS);
+    StorageDirectory sd = dirs.next();
+    FileJournalManager jm = new FileJournalManager(sd);
+    assertEquals(5*TXNS_PER_ROLL + TXNS_PER_FAIL, jm.getNumberOfTransactions(1));
+    
+    sd = dirs.next();
+    jm = new FileJournalManager(sd);
+    assertEquals(5*TXNS_PER_ROLL + TXNS_PER_FAIL, jm.getNumberOfTransactions(1));
+
+    sd = dirs.next();
+    jm = new FileJournalManager(sd);
+    assertEquals(5*TXNS_PER_ROLL + TXNS_PER_FAIL, jm.getNumberOfTransactions(1));
+  }
+
+  /** 
+   * Corrupt an edit log file after the start segment transaction
+   */
+  private void corruptAfterStartSegment(File f) throws IOException {
+    RandomAccessFile raf = new RandomAccessFile(f, "rw");
+    raf.seek(0x16); // skip version and first tranaction and a bit of next transaction
+    for (int i = 0; i < 1000; i++) {
+      raf.writeInt(0xdeadbeef);
+    }
+    raf.close();
+  }
+
+  /** 
+   * Test that we can read from a stream created by FileJournalManager.
+   * Create a single edits directory, failing it on the final roll.
+   * Then try loading from the point of the 3rd roll. Verify that we read 
+   * the correct number of transactions from this point.
+   */
+  @Test 
+  public void testReadFromStream() throws IOException {
+    File f = new File(TestEditLog.TEST_DIR + "/filejournaltest1");
+    // abort after 10th roll
+    NNStorage storage = setupEdits(Collections.<URI>singletonList(f.toURI()),
+                                   10, new AbortSpec(10, 0));
+    StorageDirectory sd = storage.dirIterator(NameNodeDirType.EDITS).next();
+
+    FileJournalManager jm = new FileJournalManager(sd);
+    long expectedTotalTxnCount = TXNS_PER_ROLL*10 + TXNS_PER_FAIL;
+    assertEquals(expectedTotalTxnCount, jm.getNumberOfTransactions(1));
+
+    long skippedTxns = (3*TXNS_PER_ROLL); // skip first 3 files
+    long startingTxId = skippedTxns + 1; 
+
+    long numTransactionsToLoad = jm.getNumberOfTransactions(startingTxId);
+    long numLoaded = 0;
+    while (numLoaded < numTransactionsToLoad) {
+      EditLogInputStream editIn = jm.getInputStream(startingTxId);
+      FSEditLogLoader.EditLogValidation val = FSEditLogLoader.validateEditLog(editIn);
+      long count = val.getNumTransactions();
+
+      editIn.close();
+      startingTxId += count;
+      numLoaded += count;
+    }
+
+    assertEquals(expectedTotalTxnCount - skippedTxns, numLoaded); 
+  }
+
+  /**
+   * Try to make a request with a start transaction id which doesn't
+   * match the start ID of some log segment. 
+   * This should fail as edit logs must currently be treated as indevisable 
+   * units.
+   */
+  @Test(expected=IOException.class)
+  public void testAskForTransactionsMidfile() throws IOException {
+    File f = new File(TestEditLog.TEST_DIR + "/filejournaltest2");
+    NNStorage storage = setupEdits(Collections.<URI>singletonList(f.toURI()), 
+                                   10);
+    StorageDirectory sd = storage.dirIterator(NameNodeDirType.EDITS).next();
+    
+    FileJournalManager jm = new FileJournalManager(sd);
+    jm.getNumberOfTransactions(2);    
+  }
+
+  /** 
+   * Test that we receive the correct number of transactions when we count
+   * the number of transactions around gaps.
+   * Set up a single edits directory, with no failures. Delete the 4th logfile.
+   * Test that getNumberOfTransactions returns the correct number of 
+   * transactions before this gap and after this gap. Also verify that if you
+   * try to count on the gap that an exception is thrown.
+   */
+  @Test
+  public void testManyLogsWithGaps() throws IOException {
+    File f = new File(TestEditLog.TEST_DIR + "/filejournaltest3");
+    NNStorage storage = setupEdits(Collections.<URI>singletonList(f.toURI()), 10);
+    StorageDirectory sd = storage.dirIterator(NameNodeDirType.EDITS).next();
+
+    final long startGapTxId = 3*TXNS_PER_ROLL + 1;
+    final long endGapTxId = 4*TXNS_PER_ROLL;
+    File[] files = new File(f, "current").listFiles(new FilenameFilter() {
+        public boolean accept(File dir, String name) {
+          if (name.startsWith(NNStorage.getFinalizedEditsFileName(startGapTxId, endGapTxId))) {
+            return true;
+          }
+          return false;
+        }
+      });
+    assertEquals(1, files.length);
+    assertTrue(files[0].delete());
+    
+    FileJournalManager jm = new FileJournalManager(sd);
+    assertEquals(startGapTxId-1, jm.getNumberOfTransactions(1));
+
+    try {
+      jm.getNumberOfTransactions(startGapTxId);
+      fail("Should have thrown an exception by now");
+    } catch (IOException ioe) {
+      assertTrue(true);
+    }
+
+    // rolled 10 times so there should be 11 files.
+    assertEquals(11*TXNS_PER_ROLL - endGapTxId, 
+                 jm.getNumberOfTransactions(endGapTxId+1));
+  }
+
+  /** 
+   * Test that we can load an edits directory with a corrupt inprogress file.
+   * The corrupt inprogress file should be moved to the side.
+   */
+  @Test
+  public void testManyLogsWithCorruptInprogress() throws IOException {
+    File f = new File(TestEditLog.TEST_DIR + "/filejournaltest5");
+    NNStorage storage = setupEdits(Collections.<URI>singletonList(f.toURI()), 10, new AbortSpec(10, 0));
+    StorageDirectory sd = storage.dirIterator(NameNodeDirType.EDITS).next();
+
+    File[] files = new File(f, "current").listFiles(new FilenameFilter() {
+        public boolean accept(File dir, String name) {
+          if (name.startsWith("edits_inprogress")) {
+            return true;
+          }
+          return false;
+        }
+      });
+    assertEquals(files.length, 1);
+    
+    corruptAfterStartSegment(files[0]);
+
+    FileJournalManager jm = new FileJournalManager(sd);
+    assertEquals(10*TXNS_PER_ROLL+1, 
+                 jm.getNumberOfTransactions(1)); 
+  }
+
   @Test
   public void testGetRemoteEditLog() throws IOException {
     StorageDirectory sd = FSImageTestUtil.mockStorageDirectory(
@@ -58,5 +318,4 @@
       FileJournalManager fjm, long firstTxId) throws IOException {
     return Joiner.on(",").join(fjm.getRemoteEditLogs(firstTxId));
   }
-  
 }
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestNameEditsConfigs.java b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestNameEditsConfigs.java
index e038051..d237bac 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestNameEditsConfigs.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestNameEditsConfigs.java
@@ -20,6 +20,7 @@
 import junit.framework.TestCase;
 import java.io.*;
 import java.util.Random;
+import java.util.List;
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.hdfs.HdfsConfiguration;
 import org.apache.hadoop.hdfs.MiniDFSCluster;
@@ -80,10 +81,12 @@
       assertTrue("Expect no images in " + dir, ins.foundImages.isEmpty());      
     }
 
+    List<FileJournalManager.EditLogFile> editlogs 
+      = FileJournalManager.matchEditLogs(new File(dir, "current").listFiles()); 
     if (shouldHaveEdits) {
-      assertTrue("Expect edits in " + dir, ins.foundEditLogs.size() > 0);
+      assertTrue("Expect edits in " + dir, editlogs.size() > 0);
     } else {
-      assertTrue("Expect no edits in " + dir, ins.foundEditLogs.isEmpty());
+      assertTrue("Expect no edits in " + dir, editlogs.isEmpty());
     }
   }
 
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestSaveNamespace.java b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestSaveNamespace.java
index 8689934b..05fe32f 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestSaveNamespace.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestSaveNamespace.java
@@ -84,7 +84,7 @@
 
     public Void answer(InvocationOnMock invocation) throws Throwable {
       Object[] args = invocation.getArguments();
-      StorageDirectory sd = (StorageDirectory)args[0];
+      StorageDirectory sd = (StorageDirectory)args[1];
 
       if (count++ == 1) {
         LOG.info("Injecting fault for sd: " + sd);
@@ -111,7 +111,7 @@
     Configuration conf = getConf();
     NameNode.initMetrics(conf, NamenodeRole.NAMENODE);
     DFSTestUtil.formatNameNode(conf);
-    FSNamesystem fsn = new FSNamesystem(conf);
+    FSNamesystem fsn = FSNamesystem.loadFromDisk(conf);
 
     // Replace the FSImage with a spy
     FSImage originalImage = fsn.dir.fsImage;
@@ -129,19 +129,22 @@
     case SAVE_SECOND_FSIMAGE_RTE:
       // The spy throws a RuntimeException when writing to the second directory
       doAnswer(new FaultySaveImage(true)).
-        when(spyImage).saveFSImage((StorageDirectory)anyObject(), anyLong());
+        when(spyImage).saveFSImage(Mockito.eq(fsn),
+            (StorageDirectory)anyObject(), anyLong());
       shouldFail = false;
       break;
     case SAVE_SECOND_FSIMAGE_IOE:
       // The spy throws an IOException when writing to the second directory
       doAnswer(new FaultySaveImage(false)).
-        when(spyImage).saveFSImage((StorageDirectory)anyObject(), anyLong());
+        when(spyImage).saveFSImage(Mockito.eq(fsn),
+            (StorageDirectory)anyObject(), anyLong());
       shouldFail = false;
       break;
     case SAVE_ALL_FSIMAGES:
       // The spy throws IOException in all directories
       doThrow(new RuntimeException("Injected")).
-        when(spyImage).saveFSImage((StorageDirectory)anyObject(), anyLong());
+        when(spyImage).saveFSImage(Mockito.eq(fsn),
+            (StorageDirectory)anyObject(), anyLong());
       shouldFail = true;
       break;
     case WRITE_STORAGE_ALL:
@@ -189,7 +192,7 @@
 
       // Start a new namesystem, which should be able to recover
       // the namespace from the previous incarnation.
-      fsn = new FSNamesystem(conf);
+      fsn = FSNamesystem.loadFromDisk(conf);
 
       // Make sure the image loaded including our edits.
       checkEditExists(fsn, 1);
@@ -214,7 +217,7 @@
 
     NameNode.initMetrics(conf, NamenodeRole.NAMENODE);
     DFSTestUtil.formatNameNode(conf);
-    FSNamesystem fsn = new FSNamesystem(conf);
+    FSNamesystem fsn = FSNamesystem.loadFromDisk(conf);
 
     // Replace the FSImage with a spy
     FSImage originalImage = fsn.dir.fsImage;
@@ -268,7 +271,7 @@
       // Start a new namesystem, which should be able to recover
       // the namespace from the previous incarnation.
       LOG.info("Loading new FSmage from disk.");
-      fsn = new FSNamesystem(conf);
+      fsn = FSNamesystem.loadFromDisk(conf);
 
       // Make sure the image loaded including our edit.
       LOG.info("Checking reloaded image.");
@@ -349,7 +352,7 @@
     Configuration conf = getConf();
     NameNode.initMetrics(conf, NamenodeRole.NAMENODE);
     DFSTestUtil.formatNameNode(conf);
-    FSNamesystem fsn = new FSNamesystem(conf);
+    FSNamesystem fsn = FSNamesystem.loadFromDisk(conf);
 
     // Replace the FSImage with a spy
     final FSImage originalImage = fsn.dir.fsImage;
@@ -365,8 +368,9 @@
         FSNamesystem.getNamespaceEditsDirs(conf));
 
     doThrow(new IOException("Injected fault: saveFSImage")).
-      when(spyImage).saveFSImage((StorageDirectory)anyObject(),
-                                 Mockito.anyLong());
+      when(spyImage).saveFSImage(
+          Mockito.eq(fsn), (StorageDirectory)anyObject(),
+          Mockito.anyLong());
 
     try {
       doAnEdit(fsn, 1);
@@ -395,7 +399,7 @@
 
       // Start a new namesystem, which should be able to recover
       // the namespace from the previous incarnation.
-      fsn = new FSNamesystem(conf);
+      fsn = FSNamesystem.loadFromDisk(conf);
 
       // Make sure the image loaded including our edits.
       checkEditExists(fsn, 1);
@@ -411,7 +415,7 @@
     Configuration conf = getConf();
     NameNode.initMetrics(conf, NamenodeRole.NAMENODE);
     DFSTestUtil.formatNameNode(conf);
-    FSNamesystem fsn = new FSNamesystem(conf);
+    FSNamesystem fsn = FSNamesystem.loadFromDisk(conf);
 
     try {
       doAnEdit(fsn, 1);
@@ -430,7 +434,7 @@
 
       // Start a new namesystem, which should be able to recover
       // the namespace from the previous incarnation.
-      fsn = new FSNamesystem(conf);
+      fsn = FSNamesystem.loadFromDisk(conf);
 
       // Make sure the image loaded including our edits.
       checkEditExists(fsn, 1);
@@ -447,7 +451,7 @@
     Configuration conf = getConf();
     NameNode.initMetrics(conf, NamenodeRole.NAMENODE);
     DFSTestUtil.formatNameNode(conf);
-    FSNamesystem fsn = new FSNamesystem(conf);
+    FSNamesystem fsn = FSNamesystem.loadFromDisk(conf);
 
     try {
       // We have a BEGIN_LOG_SEGMENT txn to start
@@ -469,7 +473,7 @@
       assertEquals(5, fsn.getEditLog().getLastWrittenTxId());
       fsn = null;
       
-      fsn = new FSNamesystem(conf);
+      fsn = FSNamesystem.loadFromDisk(conf);
       // 1 more txn to start new segment on restart
       assertEquals(6, fsn.getEditLog().getLastWrittenTxId());
       
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/test/unit/org/apache/hadoop/hdfs/server/namenode/TestNNLeaseRecovery.java b/hadoop-hdfs-project/hadoop-hdfs/src/test/unit/org/apache/hadoop/hdfs/server/namenode/TestNNLeaseRecovery.java
index b2533a0..ba76a7b 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/test/unit/org/apache/hadoop/hdfs/server/namenode/TestNNLeaseRecovery.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/unit/org/apache/hadoop/hdfs/server/namenode/TestNNLeaseRecovery.java
@@ -84,7 +84,7 @@
     FileSystem.setDefaultUri(conf, "hdfs://localhost:0");
     conf.set(DFSConfigKeys.DFS_NAMENODE_HTTP_ADDRESS_KEY, "0.0.0.0:0");
     DFSTestUtil.formatNameNode(conf);
-    fsn = spy(new FSNamesystem(conf));
+    fsn = spy(FSNamesystem.loadFromDisk(conf));
   }
 
   /**
@@ -428,7 +428,6 @@
                             
     when(fsn.getFSImage()).thenReturn(fsImage);
     when(fsn.getFSImage().getEditLog()).thenReturn(editLog);
-    fsn.getFSImage().setFSNamesystem(fsn);
     
     switch (fileBlocksNumber) {
       case 0:
diff --git a/hadoop-mapreduce-project/CHANGES.txt b/hadoop-mapreduce-project/CHANGES.txt
index 2206f4b..e22241b 100644
--- a/hadoop-mapreduce-project/CHANGES.txt
+++ b/hadoop-mapreduce-project/CHANGES.txt
@@ -1,5 +1,34 @@
 Hadoop MapReduce Change Log
 
+Release 0.23-PB - Unreleased
+
+  INCOMPATIBLE CHANGES
+
+    MAPREDUCE-3545. Remove Avro RPC. (suresh)
+
+  IMPROVEMENTS
+
+    MAPREDUCE-2887 due to HADOOP-7524 Change RPC to allow multiple protocols
+                   including multuple versions of the same protocol (sanjay Radia)
+
+    MAPREDUCE-2934. MR portion of HADOOP-7607 - Simplify the RPC proxy cleanup
+                    process (atm)
+
+    HADOOP-7862   MR changes to work with HADOOP 7862: 
+    Move the support for multiple protocols to lower layer so that Writable,
+    PB and Avro can all use it (Sanjay)
+
+    MAPREDUCE-3740. Fixed broken mapreduce compilation after the patch for
+    HADOOP-7965. (Devaraj K via vinodkv) 
+
+  BUG FIXES
+
+    MAPREDUCE-3818. Fixed broken compilation in TestSubmitJob after the patch
+    for HDFS-2895. (Suresh Srinivas via vinodkv)
+
+   MAPREDUCE-2942. TestNMAuditLogger.testNMAuditLoggerWithIP failing (Thomas Graves 
+   via mahadev)
+
 Release 0.23.2 - UNRELEASED
 
   INCOMPATIBLE CHANGES
@@ -64,8 +93,6 @@
  
 Release 0.23.1 - 2012-02-17 
 
-  INCOMPATIBLE CHANGES
-
   NEW FEATURES
 
     MAPREDUCE-778. Rumen Anonymizer. (Amar Kamat and Chris Douglas via amarrk)
diff --git a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-app/src/test/java/org/apache/hadoop/mapreduce/v2/app/TestRuntimeEstimators.java b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-app/src/test/java/org/apache/hadoop/mapreduce/v2/app/TestRuntimeEstimators.java
index 4df8df2..96cb6e2 100644
--- a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-app/src/test/java/org/apache/hadoop/mapreduce/v2/app/TestRuntimeEstimators.java
+++ b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-app/src/test/java/org/apache/hadoop/mapreduce/v2/app/TestRuntimeEstimators.java
@@ -758,8 +758,6 @@
   }
 
   class MyAppContext implements AppContext {
-    // I'll be making Avro objects by hand.  Please don't do that very often.
-
     private final ApplicationAttemptId myAppAttemptID;
     private final ApplicationId myApplicationID;
     private final JobId myJobID;
diff --git a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-common/src/main/avro/MRClientProtocol.genavro b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-common/src/main/avro/MRClientProtocol.genavro
deleted file mode 100644
index fdf98ab..0000000
--- a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-common/src/main/avro/MRClientProtocol.genavro
+++ /dev/null
@@ -1,153 +0,0 @@
-@namespace("org.apache.hadoop.mapreduce.v2.api")
-protocol MRClientProtocol {
-
-  import idl "./yarn/yarn-api/src/main/avro/yarn-types.genavro";
-
-  enum TaskType {
-    MAP,
-    REDUCE
-  }
-
-  record JobID {
-    org.apache.hadoop.yarn.ApplicationID appID;
-    int id;
-  }
-
-  record TaskID {
-    JobID jobID;
-    TaskType taskType;
-    int id;
-  }
-
-  record TaskAttemptID {
-    TaskID taskID; 
-    int id;
-  }
-
-  enum TaskState {
-    NEW,
-    SCHEDULED,
-    RUNNING,
-    SUCCEEDED,
-    FAILED,
-    KILL_WAIT,
-    KILLED
-  }
-
-  enum Phase {
-    STARTING,
-    MAP,
-    SHUFFLE,
-    SORT,
-    REDUCE,
-    CLEANUP
-  }
-
-  record Counter {
-    string name;
-    string displayName;
-    long value; 
-  }
-
-  record CounterGroup {
-    string name;
-    string displayname;
-    map<Counter> counters;
-  }
-
-  record Counters {
-    map<CounterGroup> groups;
-  }
-
-  record TaskReport {
-    TaskID id;
-    TaskState state;
-    float progress;
-    long startTime;
-    long finishTime;
-    Counters counters;
-    array<TaskAttemptID> runningAttempts;
-    union{TaskAttemptID, null} successfulAttempt;
-    array<string> diagnostics;
-  }
-
-  enum TaskAttemptState {
-    NEW,
-    UNASSIGNED,
-    ASSIGNED,
-    RUNNING,
-    COMMIT_PENDING,
-    SUCCESS_CONTAINER_CLEANUP,
-    SUCCEEDED,
-    FAIL_CONTAINER_CLEANUP,
-    FAIL_TASK_CLEANUP,
-    FAILED,
-    KILL_CONTAINER_CLEANUP,
-    KILL_TASK_CLEANUP,
-    KILLED
-  }
-
-  record TaskAttemptReport {
-    TaskAttemptID id;
-    TaskAttemptState state;
-    float progress;
-    long startTime;
-    long finishTime;
-    Counters counters;
-    string diagnosticInfo;
-    string stateString;
-    Phase phase;
-  }
-
-  enum JobState {
-    NEW,
-    INITED,
-    RUNNING,
-    SUCCEEDED,
-    FAILED,
-    KILL_WAIT,
-    KILLED,
-    ERROR
-  }
-
-  record JobReport {
-    JobID id;
-    JobState state;
-    float mapProgress;
-    float reduceProgress;
-    float cleanupProgress;
-    float setupProgress;
-    long startTime;
-    long finishTime;
-  }
-
-  enum TaskAttemptCompletionEventStatus {
-    FAILED,
-    KILLED,
-    SUCCEEDED,
-    OBSOLETE,
-    TIPFAILED
-  }
-
-  record TaskAttemptCompletionEvent {
-    TaskAttemptID attemptId;
-    TaskAttemptCompletionEventStatus status;
-    string mapOutputServerAddress;
-    int attemptRunTime;
-    int eventId;
-  }
-
-  JobReport getJobReport(JobID jobID) throws org.apache.hadoop.yarn.YarnRemoteException;
-  TaskReport getTaskReport(TaskID taskID) throws org.apache.hadoop.yarn.YarnRemoteException;
-  TaskAttemptReport getTaskAttemptReport(TaskAttemptID taskAttemptID) throws org.apache.hadoop.yarn.YarnRemoteException;
-  Counters getCounters(JobID jobID) throws org.apache.hadoop.yarn.YarnRemoteException;
-  array<TaskAttemptCompletionEvent> getTaskAttemptCompletionEvents(JobID jobID, int fromEventId, int maxEvents) throws org.apache.hadoop.yarn.YarnRemoteException;
-  array<TaskReport> getTaskReports(JobID jobID, TaskType taskType) throws org.apache.hadoop.yarn.YarnRemoteException;
-  array<string> getDiagnostics(TaskAttemptID taskAttemptID) throws org.apache.hadoop.yarn.YarnRemoteException;
-
-  void killJob(JobID jobID) throws org.apache.hadoop.yarn.YarnRemoteException;
-  void killTask(TaskID taskID) throws org.apache.hadoop.yarn.YarnRemoteException;
-  void killTaskAttempt(TaskAttemptID taskAttemptID) throws org.apache.hadoop.yarn.YarnRemoteException;
-  void failTaskAttempt(TaskAttemptID taskAttemptID) throws org.apache.hadoop.yarn.YarnRemoteException;
-
-}
diff --git a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/test/java/org/apache/hadoop/mapred/TestAuditLogger.java b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/test/java/org/apache/hadoop/mapred/TestAuditLogger.java
index 38edc90a..a6aebb0 100644
--- a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/test/java/org/apache/hadoop/mapred/TestAuditLogger.java
+++ b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/test/java/org/apache/hadoop/mapred/TestAuditLogger.java
@@ -21,6 +21,7 @@
 import java.net.InetSocketAddress;
 
 import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.ipc.ProtocolInfo;
 import org.apache.hadoop.ipc.RPC;
 import org.apache.hadoop.ipc.Server;
 import org.apache.hadoop.ipc.TestRPC.TestImpl;
@@ -123,6 +124,7 @@
    * A special extension of {@link TestImpl} RPC server with
    * {@link TestImpl#ping()} testing the audit logs.
    */
+  @ProtocolInfo(protocolName = "org.apache.hadoop.ipc.TestRPC$TestProtocol")
   private class MyTestRPCServer extends TestImpl {
     @Override
     public void ping() {
diff --git a/hadoop-mapreduce-project/hadoop-yarn/hadoop-yarn-api/src/main/avro/AMRMProtocol.genavro b/hadoop-mapreduce-project/hadoop-yarn/hadoop-yarn-api/src/main/avro/AMRMProtocol.genavro
deleted file mode 100644
index d36922e..0000000
--- a/hadoop-mapreduce-project/hadoop-yarn/hadoop-yarn-api/src/main/avro/AMRMProtocol.genavro
+++ /dev/null
@@ -1,27 +0,0 @@
-@namespace("org.apache.hadoop.yarn")
-protocol AMRMProtocol {
-
-  import idl "yarn/yarn-api/src/main/avro/yarn-types.genavro";
-  
-   // Scheduler
-   record Priority {
-     int priority;
-   }
-  
-   record ResourceRequest {
-    Priority priority;
-    string hostName;
-    Resource capability;
-    int numContainers;
-  }
-  record AMResponse {
-    boolean reboot;
-    int responseId;
-    array<Container> containers;
-  }
-  
-  void registerApplicationMaster(ApplicationMaster applicationMaster) throws YarnRemoteException;
-  void finishApplicationMaster(ApplicationMaster applicationMaster) throws YarnRemoteException;
-  AMResponse allocate(ApplicationStatus status, array<ResourceRequest> ask, array<Container> release) throws YarnRemoteException;
-  
- }
diff --git a/hadoop-mapreduce-project/hadoop-yarn/hadoop-yarn-api/src/main/avro/ClientRMProtocol.genavro b/hadoop-mapreduce-project/hadoop-yarn/hadoop-yarn-api/src/main/avro/ClientRMProtocol.genavro
deleted file mode 100644
index a37fc03..0000000
--- a/hadoop-mapreduce-project/hadoop-yarn/hadoop-yarn-api/src/main/avro/ClientRMProtocol.genavro
+++ /dev/null
@@ -1,45 +0,0 @@
-@namespace("org.apache.hadoop.yarn")
-protocol ClientRMProtocol {
-
-  import idl "yarn/yarn-api/src/main/avro/yarn-types.genavro";
-
-  record Priority {
-   int priority;
-  }
-
-  record ApplicationSubmissionContext {
-    ApplicationID applicationId;
-    union {null, string} applicationName;
-    Resource masterCapability; // TODO: Needs RM validation
-
-    //all the files required by the container to run the ApplicationMaster
-    //KEY-> destination dir name
-    //VALUE-> source path
-    map<URL> resources;
-    union {null, map<LocalResource>} resources_todo;
-    
-    // TODO - Remove fsTokens (url encoded)
-    union {null, array<string>} fsTokens;
-    union {null, bytes} fsTokens_todo;
-    
-    //env to be set before launching the command for ApplicationMaster
-    //KEY-> env variable name
-    //VALUE -> env variable value.
-    map<string> environment;
-    //command-line of the container that is going to launch the ApplicationMaster.
-    array<string> command;
-    union {null, string} queue;
-    union {null, Priority} priority;
-    string user; // TODO: Shouldn't pass it like this.
-  }
-  
-  record YarnClusterMetrics {
-    int numNodeManagers;
-  }
-
-  ApplicationID getNewApplicationId() throws YarnRemoteException;
-  ApplicationMaster getApplicationMaster(ApplicationID applicationId) throws YarnRemoteException;
-  void submitApplication(ApplicationSubmissionContext context) throws YarnRemoteException;
-  void finishApplication(ApplicationID applicationId) throws YarnRemoteException;
-  YarnClusterMetrics getClusterMetrics() throws YarnRemoteException;  
-}
diff --git a/hadoop-mapreduce-project/hadoop-yarn/hadoop-yarn-api/src/main/avro/ContainerManager.genavro b/hadoop-mapreduce-project/hadoop-yarn/hadoop-yarn-api/src/main/avro/ContainerManager.genavro
deleted file mode 100644
index 0ba1fb8..0000000
--- a/hadoop-mapreduce-project/hadoop-yarn/hadoop-yarn-api/src/main/avro/ContainerManager.genavro
+++ /dev/null
@@ -1,37 +0,0 @@
-@namespace("org.apache.hadoop.yarn")
-protocol ContainerManager {
-
-  import idl "yarn/yarn-api/src/main/avro/yarn-types.genavro";
-
-  record ContainerLaunchContext {
-    ContainerID id;
-    string user; // TODO: Shouldn't pass it like this.
-    Resource resource; // TODO: Needs RM validation
-    union {null, map<LocalResource>} resources;
-
-    union {null, bytes} containerTokens; // FileSystem related and other application specific tokens.
-    union {null, map<bytes>} serviceData;
-
-    //env to be set before launching the command
-    //KEY-> env variable name
-    //VALUE -> env variable value.
-    map<string> env;
-
-    //commandline to launch the container. All resources are downloaded in the 
-    //working directory of the command.
-    array<string> command;
-  }
-
-  record ContainerStatus {
-    ContainerID containerID;
-    ContainerState state;
-    int exitStatus;
-  }
-
-  void startContainer(ContainerLaunchContext container) throws YarnRemoteException;
-  void stopContainer(ContainerID containerID) throws YarnRemoteException;
-  void cleanupContainer(ContainerID containerID) throws YarnRemoteException;
-
-  ContainerStatus getContainerStatus(ContainerID containerID) throws YarnRemoteException;
-
-}
diff --git a/hadoop-mapreduce-project/hadoop-yarn/hadoop-yarn-api/src/main/avro/yarn-types.genavro b/hadoop-mapreduce-project/hadoop-yarn/hadoop-yarn-api/src/main/avro/yarn-types.genavro
deleted file mode 100644
index 51d2077..0000000
--- a/hadoop-mapreduce-project/hadoop-yarn/hadoop-yarn-api/src/main/avro/yarn-types.genavro
+++ /dev/null
@@ -1,109 +0,0 @@
-@namespace("org.apache.hadoop.yarn")
-protocol types {
-
-  record ApplicationID {
-    int id;
-    long clusterTimeStamp;
-  }
-
-  record ContainerID {
-    ApplicationID appID; // the application id to which this container belong.
-    int id;// unique string for this application
-  }
-
-  error YarnRemoteException {
-    union { null, string } message;
-    union { null, string } trace; //stackTrace
-    union { null, YarnRemoteException } cause;
-  }
-  
-  record Resource {
-    int memory;
-    //int diskspace;
-  }
-
-  // State of the container on the ContainerManager.
-  enum ContainerState {
-    INTIALIZING,
-    RUNNING,
-    COMPLETE
-  }
-
-  record ContainerToken {
-    bytes identifier;
-    bytes password;
-    string kind;
-    string service;
-  }
-
-  record Container {
-    ContainerID id;
-    string hostName;
-    Resource resource;
-    ContainerState state;
-    union {ContainerToken, null} containerToken;
-  }
-
-  enum ApplicationState {
-    PENDING,
-    ALLOCATING,
-   	ALLOCATED,
-   	EXPIRED_PENDING,
-   	LAUNCHING,
-   	LAUNCHED,
-    RUNNING,
-    PAUSED,
-    CLEANUP,
-    COMPLETED,
-    KILLED,
-    FAILED
-  }
- 
- record ApplicationStatus {
-    int responseID; // TODO: This should be renamed as previousResponseID
-    ApplicationID applicationId;
-    float progress;
-    long lastSeen;
-  }
-  
-  record ApplicationMaster {
-    ApplicationID applicationId;
-    union { null, string } host;
-    int rpcPort;
-    int httpPort;
-    ApplicationStatus status;
-    ApplicationState state;
-    union { null, string } clientToken;
-  }
-
-  record URL {
-    string scheme;
-    union { null, string } host;
-    int port;
-    string file;
-  }
-
-  enum LocalResourceVisibility {
-    // accessible to applications from all users
-    PUBLIC,
-    // accessible only to applications from the submitting user
-    PRIVATE,
-    // accessible only to this application
-    APPLICATION
-  }
-
-  enum LocalResourceType {
-    // an archive to be expanded
-    ARCHIVE,
-    // uninterpreted file
-    FILE
-  }
-
-  record LocalResource {
-    URL resource;
-    long size;
-    long timestamp;
-    LocalResourceType type;
-    LocalResourceVisibility state;
-  }
-}
diff --git a/hadoop-mapreduce-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/api/ApplicationConstants.java b/hadoop-mapreduce-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/api/ApplicationConstants.java
index cb47d1a..5302047 100644
--- a/hadoop-mapreduce-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/api/ApplicationConstants.java
+++ b/hadoop-mapreduce-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/api/ApplicationConstants.java
@@ -24,7 +24,6 @@
  * This is the API for the applications comprising of constants that YARN sets
  * up for the applications and the containers.
  * 
- * TODO: Should also be defined in avro/pb IDLs
  * TODO: Investigate the semantics and security of each cross-boundary refs.
  */
 public interface ApplicationConstants {
diff --git a/hadoop-mapreduce-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/ipc/ProtoOverHadoopRpcEngine.java b/hadoop-mapreduce-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/ipc/ProtoOverHadoopRpcEngine.java
index 6900b0f..bbece2f 100644
--- a/hadoop-mapreduce-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/ipc/ProtoOverHadoopRpcEngine.java
+++ b/hadoop-mapreduce-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/ipc/ProtoOverHadoopRpcEngine.java
@@ -36,10 +36,13 @@
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.io.Writable;
 import org.apache.hadoop.ipc.Client;
+import org.apache.hadoop.ipc.ProtocolMetaInfoPB;
 import org.apache.hadoop.ipc.ProtocolProxy;
 import org.apache.hadoop.ipc.RPC;
 import org.apache.hadoop.ipc.RpcEngine;
 import org.apache.hadoop.ipc.ClientCache;
+import org.apache.hadoop.ipc.Client.ConnectionId;
+import org.apache.hadoop.ipc.RpcPayloadHeader.RpcKind;
 import org.apache.hadoop.security.UserGroupInformation;
 import org.apache.hadoop.security.token.SecretManager;
 import org.apache.hadoop.security.token.TokenIdentifier;
@@ -73,14 +76,16 @@
   }
 
   @Override
-  public void stopProxy(Object proxy) {
-    try {
-      ((Invoker) Proxy.getInvocationHandler(proxy)).close();
-    } catch (IOException e) {
-      LOG.warn("Error while stopping " + proxy, e);
-    }
+  public ProtocolProxy<ProtocolMetaInfoPB> getProtocolMetaInfoProxy(
+      ConnectionId connId, Configuration conf, SocketFactory factory)
+      throws IOException {
+    Class<ProtocolMetaInfoPB> protocol = ProtocolMetaInfoPB.class;
+    return new ProtocolProxy<ProtocolMetaInfoPB>(protocol,
+        (ProtocolMetaInfoPB) Proxy.newProxyInstance(protocol.getClassLoader(),
+            new Class[] { protocol }, new Invoker(protocol, connId, conf,
+                factory)), false);
   }
-
+  
   private static class Invoker implements InvocationHandler, Closeable {
     private Map<String, Message> returnTypes = new ConcurrentHashMap<String, Message>();
     private boolean isClosed = false;
@@ -90,8 +95,13 @@
     public Invoker(Class<?> protocol, InetSocketAddress addr,
         UserGroupInformation ticket, Configuration conf, SocketFactory factory,
         int rpcTimeout) throws IOException {
-      this.remoteId = Client.ConnectionId.getConnectionId(addr, protocol,
-          ticket, rpcTimeout, conf);
+      this(protocol, Client.ConnectionId.getConnectionId(addr, protocol,
+          ticket, rpcTimeout, conf), conf, factory);
+    }
+    
+    public Invoker(Class<?> protocol, Client.ConnectionId connId,
+        Configuration conf, SocketFactory factory) {
+      this.remoteId = connId;
       this.client = CLIENTS.getClient(conf, factory,
           ProtoSpecificResponseWritable.class);
     }
@@ -317,13 +327,13 @@
     }
 
     @Override
-    public Writable call(Class<?> protocol, Writable writableRequest,
-        long receiveTime) throws IOException {
+    public Writable call(RpcKind rpcKind, String protocol, 
+        Writable writableRequest, long receiveTime) throws IOException {
       ProtoSpecificRequestWritable request = (ProtoSpecificRequestWritable) writableRequest;
       ProtoSpecificRpcRequest rpcRequest = request.message;
       String methodName = rpcRequest.getMethodName();
       if (verbose) {
-        log("Call: protocol=" + protocol.getCanonicalName() + ", method="
+        log("Call: protocol=" + protocol + ", method="
             + methodName);
       }
       MethodDescriptor methodDescriptor = service.getDescriptorForType()
diff --git a/hadoop-mapreduce-project/hadoop-yarn/hadoop-yarn-common/src/test/java/org/apache/hadoop/yarn/TestRPC.java b/hadoop-mapreduce-project/hadoop-yarn/hadoop-yarn-common/src/test/java/org/apache/hadoop/yarn/TestRPC.java
index a53175b..e007ad6 100644
--- a/hadoop-mapreduce-project/hadoop-yarn/hadoop-yarn-common/src/test/java/org/apache/hadoop/yarn/TestRPC.java
+++ b/hadoop-mapreduce-project/hadoop-yarn/hadoop-yarn-common/src/test/java/org/apache/hadoop/yarn/TestRPC.java
@@ -58,16 +58,6 @@
   private static final String EXCEPTION_CAUSE = "exception cause";
   private static final RecordFactory recordFactory = RecordFactoryProvider.getRecordFactory(null);
   
-//  @Test
-//  public void testAvroRPC() throws Exception {
-//    test(AvroYarnRPC.class.getName());
-//  }
-//
-//  @Test
-//  public void testHadoopNativeRPC() throws Exception {
-//    test(HadoopYarnRPC.class.getName());
-//  }
-
   @Test
   public void testUnknownCall() {
     Configuration conf = new Configuration();
diff --git a/hadoop-mapreduce-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/src/main/avro/ResourceTracker.genavro b/hadoop-mapreduce-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/src/main/avro/ResourceTracker.genavro
deleted file mode 100644
index b1da44f..0000000
--- a/hadoop-mapreduce-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/src/main/avro/ResourceTracker.genavro
+++ /dev/null
@@ -1,40 +0,0 @@
-@namespace("org.apache.hadoop.yarn")
-protocol ResourceTracker {
-
-  import idl "yarn/yarn-api/src/main/avro/yarn-types.genavro";
-  
-  // ResourceTracker
-  record NodeID {
-    int id;
-  }
-
-  record NodeHealthStatus {
-    boolean isNodeHealthy;
-    union {string, null} healthReport;
-    long lastHealthReportTime;
-  }
-
-  record NodeStatus {
-    NodeID nodeId;
-	int responseId;
-	long lastSeen;
-    map<array<org.apache.hadoop.yarn.Container>> containers;
-	NodeHealthStatus nodeHealthStatus;
-  }
-
-  record RegistrationResponse {
-    NodeID nodeID;
-    union {bytes, null} secretKey;
-  }
-
-  record HeartbeatResponse {
-    int responseId;
-    boolean reboot;
-    array<org.apache.hadoop.yarn.Container> containersToCleanup;
-    array<org.apache.hadoop.yarn.ApplicationID> appplicationsToCleanup;
-  }
-  
-  RegistrationResponse registerNodeManager(string node, org.apache.hadoop.yarn.Resource resource) throws YarnRemoteException;
-  HeartbeatResponse nodeHeartbeat(NodeStatus nodeStatus) throws YarnRemoteException;
-  
-}
diff --git a/hadoop-mapreduce-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/avro/LocalizationProtocol.genavro b/hadoop-mapreduce-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/avro/LocalizationProtocol.genavro
deleted file mode 100644
index 01e0c07..0000000
--- a/hadoop-mapreduce-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/avro/LocalizationProtocol.genavro
+++ /dev/null
@@ -1,11 +0,0 @@
-@namespace("org.apache.hadoop.yarn")
-protocol LocalizationProtocol {
-
-  import idl "yarn/yarn-api/src/main/avro/yarn-types.genavro";
-
-  void successfulLocalization(string user, LocalResource resource, URL path) 
-    throws YarnRemoteException;
-
-  void failedLocalization(string user, LocalResource resource, YarnRemoteException path)
-    throws YarnRemoteException;
-}
diff --git a/hadoop-mapreduce-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/test/java/org/apache/hadoop/yarn/server/nodemanager/TestNMAuditLogger.java b/hadoop-mapreduce-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/test/java/org/apache/hadoop/yarn/server/nodemanager/TestNMAuditLogger.java
index b642279..8a6dabf 100644
--- a/hadoop-mapreduce-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/test/java/org/apache/hadoop/yarn/server/nodemanager/TestNMAuditLogger.java
+++ b/hadoop-mapreduce-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/test/java/org/apache/hadoop/yarn/server/nodemanager/TestNMAuditLogger.java
@@ -32,6 +32,7 @@
 import org.apache.hadoop.yarn.server.nodemanager.NMAuditLogger;
 import org.apache.hadoop.yarn.server.nodemanager.NMAuditLogger.AuditConstants;
 import org.apache.hadoop.yarn.server.nodemanager.NMAuditLogger.Keys;
+import org.apache.hadoop.yarn.server.security.ContainerTokenSecretManager;
 
 import org.apache.hadoop.net.NetUtils;
 
@@ -211,7 +212,8 @@
   public void testNMAuditLoggerWithIP() throws Exception {
     Configuration conf = new Configuration();
     // start the IPC server
-    Server server = RPC.getServer(new MyTestRPCServer(), "0.0.0.0", 0, conf);
+    Server server = RPC.getServer(TestProtocol.class,
+        new MyTestRPCServer(), "0.0.0.0", 0, 5, true, conf, null);
     server.start();
 
     InetSocketAddress addr = NetUtils.getConnectAddress(server);
diff --git a/hadoop-mapreduce-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/TestRMAuditLogger.java b/hadoop-mapreduce-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/TestRMAuditLogger.java
index 9291b49..4f6aadd 100644
--- a/hadoop-mapreduce-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/TestRMAuditLogger.java
+++ b/hadoop-mapreduce-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/TestRMAuditLogger.java
@@ -33,6 +33,7 @@
 import org.apache.hadoop.yarn.server.resourcemanager.RMAuditLogger;
 import org.apache.hadoop.yarn.server.resourcemanager.RMAuditLogger.AuditConstants;
 import org.apache.hadoop.yarn.server.resourcemanager.RMAuditLogger.Keys;
+import org.apache.hadoop.yarn.server.security.ContainerTokenSecretManager;
 
 import org.apache.hadoop.net.NetUtils;
 
@@ -228,7 +229,8 @@
   public void testRMAuditLoggerWithIP() throws Exception {
     Configuration conf = new Configuration();
     // start the IPC server
-    Server server = RPC.getServer(new MyTestRPCServer(), "0.0.0.0", 0, conf);
+    Server server = RPC.getServer(TestProtocol.class,
+        new MyTestRPCServer(), "0.0.0.0", 0, 5, true, conf, null);
     server.start();
 
     InetSocketAddress addr = NetUtils.getConnectAddress(server);
diff --git a/hadoop-mapreduce-project/hadoop-yarn/pom.xml b/hadoop-mapreduce-project/hadoop-yarn/pom.xml
index d52704d..40f9476 100644
--- a/hadoop-mapreduce-project/hadoop-yarn/pom.xml
+++ b/hadoop-mapreduce-project/hadoop-yarn/pom.xml
@@ -49,36 +49,6 @@
 
   <dependencies>
     <dependency>
-      <groupId>org.apache.avro</groupId>
-      <artifactId>avro</artifactId>
-      <exclusions>
-        <exclusion>
-          <groupId>org.mortbay.jetty</groupId>
-          <artifactId>jetty</artifactId>
-        </exclusion>
-        <exclusion>
-          <groupId>org.apache.ant</groupId>
-          <artifactId>ant</artifactId>
-        </exclusion>
-        <exclusion>
-          <groupId>org.jboss.netty</groupId>
-          <artifactId>netty</artifactId>
-        </exclusion>
-        <exclusion>
-          <groupId>org.apache.velocity</groupId>
-          <artifactId>velocity</artifactId>
-        </exclusion>
-        <exclusion>
-          <groupId>org.slf4j</groupId>
-          <artifactId>slf4j-api</artifactId>
-        </exclusion>
-        <exclusion>
-          <artifactId>paranamer-ant</artifactId>
-          <groupId>com.thoughtworks.paranamer</groupId>
-        </exclusion>
-      </exclusions>
-    </dependency>
-    <dependency>
       <groupId>org.apache.hadoop</groupId>
       <artifactId>hadoop-common</artifactId>
       <scope>provided</scope>
@@ -141,7 +111,6 @@
       <groupId>com.cenqua.clover</groupId>
       <artifactId>clover</artifactId>
     </dependency>
-
     <dependency>
       <groupId>com.google.protobuf</groupId>
       <artifactId>protobuf-java</artifactId>
diff --git a/hadoop-mapreduce-project/src/java/org/apache/hadoop/mapred/JobTracker.java b/hadoop-mapreduce-project/src/java/org/apache/hadoop/mapred/JobTracker.java
index 8a07128..c578635 100644
--- a/hadoop-mapreduce-project/src/java/org/apache/hadoop/mapred/JobTracker.java
+++ b/hadoop-mapreduce-project/src/java/org/apache/hadoop/mapred/JobTracker.java
@@ -1488,7 +1488,7 @@
     taskScheduler = (TaskScheduler) ReflectionUtils.newInstance(schedulerClass, conf);
     
     int handlerCount = conf.getInt(JT_IPC_HANDLER_COUNT, 10);
-    this.interTrackerServer = RPC.getServer(ClientProtocol.class,
+    this.interTrackerServer = RPC.getServer(JobTracker.class, // All protocols in JobTracker
                                             this,
                                             addr.getHostName(), 
                                             addr.getPort(), handlerCount, 
diff --git a/hadoop-mapreduce-project/src/test/mapred/org/apache/hadoop/mapred/TestSubmitJob.java b/hadoop-mapreduce-project/src/test/mapred/org/apache/hadoop/mapred/TestSubmitJob.java
index 3478bd3..98563d5 100644
--- a/hadoop-mapreduce-project/src/test/mapred/org/apache/hadoop/mapred/TestSubmitJob.java
+++ b/hadoop-mapreduce-project/src/test/mapred/org/apache/hadoop/mapred/TestSubmitJob.java
@@ -33,6 +33,7 @@
 import org.apache.hadoop.fs.Path;
 import org.apache.hadoop.hdfs.MiniDFSCluster;
 import org.apache.hadoop.hdfs.protocol.HdfsFileStatus;
+import org.apache.hadoop.hdfs.protocolPB.ClientNamenodeProtocolTranslatorPB;
 import org.apache.hadoop.hdfs.server.namenode.NameNode;
 import org.apache.hadoop.ipc.RPC;
 import org.apache.hadoop.ipc.RemoteException;
@@ -148,15 +149,9 @@
   }
 
   static org.apache.hadoop.hdfs.protocol.ClientProtocol getDFSClient(
-      Configuration conf, UserGroupInformation ugi) 
-  throws IOException {
-    return (org.apache.hadoop.hdfs.protocol.ClientProtocol) 
-      RPC.getProxy(org.apache.hadoop.hdfs.protocol.ClientProtocol.class, 
-        org.apache.hadoop.hdfs.protocol.ClientProtocol.versionID, 
-        NameNode.getAddress(conf), ugi, 
-        conf, 
-        NetUtils.getSocketFactory(conf, 
-            org.apache.hadoop.hdfs.protocol.ClientProtocol.class));
+      Configuration conf, UserGroupInformation ugi) throws IOException {
+    return new ClientNamenodeProtocolTranslatorPB(NameNode.getAddress(conf),
+        conf, ugi);
   }
   
   /**
diff --git a/hadoop-project/pom.xml b/hadoop-project/pom.xml
index a0463cd..a59b636 100644
--- a/hadoop-project/pom.xml
+++ b/hadoop-project/pom.xml
@@ -558,11 +558,6 @@
         <version>1.5.3</version>
       </dependency>
       <dependency>
-        <groupId>org.apache.avro</groupId>
-        <artifactId>avro-ipc</artifactId>
-        <version>1.5.3</version>
-      </dependency>
-      <dependency>
         <groupId>net.sf.kosmosfs</groupId>
         <artifactId>kfs</artifactId>
         <version>0.3</version>