HDFS-2058. Change Data Transfer wire protocol to use protocol buffers. Contributed by Todd Lipcon.

git-svn-id: https://svn.apache.org/repos/asf/hadoop/hdfs/trunk@1134492 13f79535-47bb-0310-9956-ffa450edef68
diff --git a/CHANGES.txt b/CHANGES.txt
index f059e47..52bdbf5 100644
--- a/CHANGES.txt
+++ b/CHANGES.txt
@@ -285,6 +285,9 @@
     layout to be consistent across the binary tgz, rpm, and deb.
     (Eric Yang via omalley)
 
+    HDFS-2058. Change Data Transfer wire protocol to use protocol buffers.
+    (todd)
+
   IMPROVEMENTS
 
     HDFS-1875. MiniDFSCluster hard-codes dfs.datanode.address to localhost
diff --git a/build.xml b/build.xml
index 92b662e..b221132 100644
--- a/build.xml
+++ b/build.xml
@@ -40,6 +40,7 @@
 
   <property name="src.dir" value="${basedir}/src"/>  	
   <property name="java.src.dir" value="${src.dir}/java"/>
+  <property name="proto.src.dir" value="${src.dir}/proto"/>
   <property name="anttasks.dir" value="${basedir}/src/ant"/>
   <property name="lib.dir" value="${basedir}/lib"/>
   <property name="conf.dir" value="${basedir}/conf"/>
@@ -201,6 +202,9 @@
   <property name="build.dir.eclipse-test-classes" value="${build.dir.eclipse}/classes-test"/>
   <property name="build.dir.eclipse-contrib-classes" value="${build.dir.eclipse}/classes-contrib"/>
 
+  <!-- Protoc properties -->
+  <property name="protoc" value="protoc" />
+
   <property name="clover.jar" location="${clover.home}/lib/clover.jar"/>
   <available property="clover.present" file="${clover.jar}" />
 
@@ -923,7 +927,9 @@
       bottom="Copyright &amp;copy; ${year} The Apache Software Foundation"
       maxmemory="${javadoc.maxmemory}">
 
-        <packageset dir="${java.src.dir}"/>
+        <packageset dir="${java.src.dir}">
+          <exclude name="org/apache/hadoop/hdfs/protocol/proto" />
+        </packageset>
         <link href="${javadoc.link.java}"/>
         <classpath >
           <path refid="classpath" />
@@ -943,7 +949,9 @@
          <param name="-apidir" value="${jdiff.xml.dir}"/>
          <param name="-apiname" value="hadoop-hdfs ${version}"/>
        </doclet>
-       <packageset dir="src/java"/>
+       <packageset dir="${java.src.dir}">
+         <exclude name="org/apache/hadoop/hdfs/protocol/proto" />
+       </packageset>
        <classpath >
          <path refid="classpath" />
          <path refid="jdiff-classpath" />
@@ -1365,6 +1373,8 @@
         <exclude name="lib/jdiff/"/>
         <exclude name="**/conf/*" />
         <exclude name="webapps/**/WEB-INF/web.xml"/>
+        <!-- generated code for protobufs don't have headers -->
+        <exclude name="src/java/org/apache/hadoop/hdfs/protocol/proto/*Protos.java" />
         <exclude name="src/docs/releasenotes.html" />
         <exclude name="src/test/hdfs/org/apache/hadoop/cli/clitest_data/" />
         <exclude name="src/test/hdfs/org/apache/hadoop/hdfs/tools/offlineEditsViewer/editsStored*" />
@@ -1672,6 +1682,27 @@
     </copy>
   </target>
 
+  <target name="generate-protos"
+    description="Generate Java code from protocol buffer definitions">
+    <exec executable="bash" resultproperty="which.protoc.result" outputproperty="">
+      <arg value="-c" />
+      <arg value="which ${protoc}" />
+    </exec>
+    <condition property="protoc.found">
+      <equals arg1="${which.protoc.result}" arg2="0" />
+    </condition>
+    <fail unless="protoc.found"
+      message="No protoc compiler found. Please pass -Dprotoc=/path/to/protoc if it is not on your path." />
+
+    <exec executable="${protoc}" failonerror="true">
+      <arg value="--java_out=${java.src.dir}" />
+      <arg value="--proto_path=${proto.src.dir}" />
+      <arg value="${proto.src.dir}/hdfs.proto" />
+      <arg value="${proto.src.dir}/datatransfer.proto" />
+    </exec>
+    <echo message="Generated protocol buffer code successfully." />
+  </target>
+
   <target name="ivy-init-dirs">
     <mkdir dir="${build.ivy.dir}" />
     <mkdir dir="${build.ivy.lib.dir}" />
diff --git a/ivy.xml b/ivy.xml
index 8c3cc14..d52b8d5 100644
--- a/ivy.xml
+++ b/ivy.xml
@@ -67,6 +67,7 @@
     <dependency org="commons-logging" name="commons-logging" rev="${commons-logging.version}" conf="compile->master"/>
     <dependency org="commons-daemon" name="commons-daemon" rev="${commons-daemon.version}" conf="hdfs->default" />
     <dependency org="log4j" name="log4j" rev="${log4j.version}" conf="common->master"/>
+    <dependency org="com.google.protobuf" name="protobuf-java" rev="2.4.0a" conf="common->master"/>
     <dependency org="org.apache.hadoop" name="avro" rev="${avro.version}" conf="compile->master">
       <exclude module="ant"/>
       <exclude module="jetty"/>
diff --git a/src/java/org/apache/hadoop/hdfs/BlockReader.java b/src/java/org/apache/hadoop/hdfs/BlockReader.java
index c7578eb..9b98e1b 100644
--- a/src/java/org/apache/hadoop/hdfs/BlockReader.java
+++ b/src/java/org/apache/hadoop/hdfs/BlockReader.java
@@ -17,10 +17,6 @@
  */
 package org.apache.hadoop.hdfs;
 
-import static org.apache.hadoop.hdfs.protocol.DataTransferProtocol.Status.CHECKSUM_OK;
-import static org.apache.hadoop.hdfs.protocol.DataTransferProtocol.Status.ERROR_ACCESS_TOKEN;
-import static org.apache.hadoop.hdfs.protocol.DataTransferProtocol.Status.SUCCESS;
-
 import java.io.BufferedInputStream;
 import java.io.BufferedOutputStream;
 import java.io.DataInputStream;
@@ -37,6 +33,10 @@
 import org.apache.hadoop.hdfs.protocol.DataTransferProtocol;
 import org.apache.hadoop.hdfs.protocol.ExtendedBlock;
 import org.apache.hadoop.hdfs.protocol.DataTransferProtocol.PacketHeader;
+import org.apache.hadoop.hdfs.protocol.HdfsProtoUtil;
+import org.apache.hadoop.hdfs.protocol.proto.DataTransferProtos.BlockOpResponseProto;
+import org.apache.hadoop.hdfs.protocol.proto.DataTransferProtos.ClientReadStatusProto;
+import org.apache.hadoop.hdfs.protocol.proto.DataTransferProtos.Status;
 import org.apache.hadoop.hdfs.security.token.block.BlockTokenIdentifier;
 import org.apache.hadoop.hdfs.security.token.block.InvalidBlockTokenException;
 import org.apache.hadoop.hdfs.server.common.HdfsConstants;
@@ -45,6 +45,9 @@
 import org.apache.hadoop.security.token.Token;
 import org.apache.hadoop.util.DataChecksum;
 
+import static org.apache.hadoop.hdfs.protocol.HdfsProtoUtil.vintPrefixed;
+
+
 /** This is a wrapper around connection to datanode
  * and understands checksum, offset etc.
  *
@@ -138,9 +141,9 @@
     // if eos was set in the previous read, send a status code to the DN
     if (eos && !eosBefore && nRead >= 0) {
       if (needChecksum()) {
-        sendReadResult(dnSock, CHECKSUM_OK);
+        sendReadResult(dnSock, Status.CHECKSUM_OK);
       } else {
-        sendReadResult(dnSock, SUCCESS);
+        sendReadResult(dnSock, Status.SUCCESS);
       }
     }
     return nRead;
@@ -313,20 +316,13 @@
         pos + bytesToRead >= bytesNeededToFinish) {
 
       // Read header
-      int packetLen = in.readInt();
-      long offsetInBlock = in.readLong();
-      long seqno = in.readLong();
-      boolean lastPacketInBlock = in.readBoolean();
-      int dataLen = in.readInt();
+      PacketHeader hdr = new PacketHeader();
+      hdr.readFields(in);
 
-      if (!lastPacketInBlock ||
-          dataLen != 0) {
+      if (!hdr.isLastPacketInBlock() ||
+          hdr.getDataLen() != 0) {
         throw new IOException("Expected empty end-of-read packet! Header: " +
-                              "(packetLen : " + packetLen + 
-                              ", offsetInBlock : " + offsetInBlock +
-                              ", seqno : " + seqno + 
-                              ", lastInBlock : " + lastPacketInBlock +
-                              ", dataLen : " + dataLen);
+                              hdr);
       }
 
       eos = true;
@@ -422,9 +418,10 @@
         new BufferedInputStream(NetUtils.getInputStream(sock), 
                                 bufferSize));
     
-    DataTransferProtocol.Status status = DataTransferProtocol.Status.read(in);
-    if (status != SUCCESS) {
-      if (status == ERROR_ACCESS_TOKEN) {
+    BlockOpResponseProto status = BlockOpResponseProto.parseFrom(
+        vintPrefixed(in));
+    if (status.getStatus() != Status.SUCCESS) {
+      if (status.getStatus() == Status.ERROR_ACCESS_TOKEN) {
         throw new InvalidBlockTokenException(
             "Got access token error for OP_READ_BLOCK, self="
                 + sock.getLocalSocketAddress() + ", remote="
@@ -499,11 +496,16 @@
    * closing our connection (which we will re-open), but won't affect
    * data correctness.
    */
-  void sendReadResult(Socket sock, DataTransferProtocol.Status statusCode) {
+  void sendReadResult(Socket sock, Status statusCode) {
     assert !sentStatusCode : "already sent status code to " + sock;
     try {
       OutputStream out = NetUtils.getOutputStream(sock, HdfsConstants.WRITE_TIMEOUT);
-      statusCode.writeOutputStream(out);
+      
+      ClientReadStatusProto.newBuilder()
+        .setStatus(statusCode)
+        .build()
+        .writeDelimitedTo(out);
+
       out.flush();
       sentStatusCode = true;
     } catch (IOException e) {
diff --git a/src/java/org/apache/hadoop/hdfs/DFSClient.java b/src/java/org/apache/hadoop/hdfs/DFSClient.java
index 9ae968b..76de6c2 100644
--- a/src/java/org/apache/hadoop/hdfs/DFSClient.java
+++ b/src/java/org/apache/hadoop/hdfs/DFSClient.java
@@ -19,8 +19,6 @@
 package org.apache.hadoop.hdfs;
 
 import static org.apache.hadoop.hdfs.protocol.DataTransferProtocol.Op.BLOCK_CHECKSUM;
-import static org.apache.hadoop.hdfs.protocol.DataTransferProtocol.Status.ERROR_ACCESS_TOKEN;
-import static org.apache.hadoop.hdfs.protocol.DataTransferProtocol.Status.SUCCESS;
 
 import java.io.BufferedOutputStream;
 import java.io.ByteArrayInputStream;
@@ -75,7 +73,11 @@
 import org.apache.hadoop.hdfs.protocol.LocatedBlock;
 import org.apache.hadoop.hdfs.protocol.LocatedBlocks;
 import org.apache.hadoop.hdfs.protocol.NSQuotaExceededException;
+import org.apache.hadoop.hdfs.protocol.HdfsProtoUtil;
 import org.apache.hadoop.hdfs.protocol.UnresolvedPathException;
+import org.apache.hadoop.hdfs.protocol.proto.DataTransferProtos.BlockOpResponseProto;
+import org.apache.hadoop.hdfs.protocol.proto.DataTransferProtos.OpBlockChecksumResponseProto;
+import org.apache.hadoop.hdfs.protocol.proto.DataTransferProtos.Status;
 import org.apache.hadoop.hdfs.security.token.delegation.DelegationTokenIdentifier;
 import org.apache.hadoop.hdfs.server.common.HdfsConstants;
 import org.apache.hadoop.hdfs.server.common.UpgradeStatusReport;
@@ -1118,9 +1120,11 @@
           DataTransferProtocol.Sender.opBlockChecksum(out, block,
               lb.getBlockToken());
 
-          final DataTransferProtocol.Status reply = DataTransferProtocol.Status.read(in);
-          if (reply != SUCCESS) {
-            if (reply == ERROR_ACCESS_TOKEN
+          final BlockOpResponseProto reply =
+            BlockOpResponseProto.parseFrom(HdfsProtoUtil.vintPrefixed(in));
+
+          if (reply.getStatus() != Status.SUCCESS) {
+            if (reply.getStatus() == Status.ERROR_ACCESS_TOKEN
                 && i > lastRetriedIndex) {
               if (LOG.isDebugEnabled()) {
                 LOG.debug("Got access token error in response to OP_BLOCK_CHECKSUM "
@@ -1138,9 +1142,12 @@
                   + block + " from datanode " + datanodes[j].getName());
             }
           }
+          
+          OpBlockChecksumResponseProto checksumData =
+            reply.getChecksumResponse();
 
           //read byte-per-checksum
-          final int bpc = in.readInt(); 
+          final int bpc = checksumData.getBytesPerCrc();
           if (i == 0) { //first block
             bytesPerCRC = bpc;
           }
@@ -1150,13 +1157,14 @@
           }
           
           //read crc-per-block
-          final long cpb = in.readLong();
+          final long cpb = checksumData.getCrcPerBlock();
           if (locatedblocks.size() > 1 && i == 0) {
             crcPerBlock = cpb;
           }
 
           //read md5
-          final MD5Hash md5 = MD5Hash.read(in);
+          final MD5Hash md5 = new MD5Hash(
+              checksumData.getMd5().toByteArray());
           md5.write(md5out);
           
           done = true;
diff --git a/src/java/org/apache/hadoop/hdfs/DFSOutputStream.java b/src/java/org/apache/hadoop/hdfs/DFSOutputStream.java
index 6f310ee..8843982 100644
--- a/src/java/org/apache/hadoop/hdfs/DFSOutputStream.java
+++ b/src/java/org/apache/hadoop/hdfs/DFSOutputStream.java
@@ -17,8 +17,9 @@
  */
 package org.apache.hadoop.hdfs;
 
-import static org.apache.hadoop.hdfs.protocol.DataTransferProtocol.Status.ERROR_ACCESS_TOKEN;
-import static org.apache.hadoop.hdfs.protocol.DataTransferProtocol.Status.SUCCESS;
+import static org.apache.hadoop.hdfs.protocol.proto.DataTransferProtos.Status.SUCCESS;
+import static org.apache.hadoop.hdfs.protocol.proto.DataTransferProtos.Status.ERROR;
+
 
 import java.io.BufferedOutputStream;
 import java.io.DataInputStream;
@@ -56,7 +57,10 @@
 import org.apache.hadoop.hdfs.protocol.HdfsFileStatus;
 import org.apache.hadoop.hdfs.protocol.LocatedBlock;
 import org.apache.hadoop.hdfs.protocol.NSQuotaExceededException;
+import org.apache.hadoop.hdfs.protocol.HdfsProtoUtil;
 import org.apache.hadoop.hdfs.protocol.UnresolvedPathException;
+import org.apache.hadoop.hdfs.protocol.proto.DataTransferProtos.BlockOpResponseProto;
+import org.apache.hadoop.hdfs.protocol.proto.DataTransferProtos.Status;
 import org.apache.hadoop.hdfs.security.token.block.BlockTokenIdentifier;
 import org.apache.hadoop.hdfs.security.token.block.InvalidBlockTokenException;
 import org.apache.hadoop.hdfs.server.datanode.DataNode;
@@ -75,6 +79,8 @@
 import org.apache.hadoop.util.PureJavaCrc32;
 import org.apache.hadoop.util.StringUtils;
 
+
+
 /****************************************************************
  * DFSOutputStream creates files from a stream of bytes.
  *
@@ -650,7 +656,7 @@
             long seqno = ack.getSeqno();
             // processes response status from datanodes.
             for (int i = ack.getNumOfReplies()-1; i >=0  && dfsClient.clientRunning; i--) {
-              final DataTransferProtocol.Status reply = ack.getReply(i);
+              final Status reply = ack.getReply(i);
               if (reply != SUCCESS) {
                 errorIndex = i; // first bad datanode
                 throw new IOException("Bad response " + reply +
@@ -848,7 +854,9 @@
 
         //ack
         in = new DataInputStream(NetUtils.getInputStream(sock));
-        if (SUCCESS != DataTransferProtocol.Status.read(in)) {
+        BlockOpResponseProto response =
+          BlockOpResponseProto.parseFrom(HdfsProtoUtil.vintPrefixed(in));
+        if (SUCCESS != response.getStatus()) {
           throw new IOException("Failed to add a datanode");
         }
       } finally {
@@ -990,7 +998,7 @@
     //
     private boolean createBlockOutputStream(DatanodeInfo[] nodes, long newGS,
         boolean recoveryFlag) {
-      DataTransferProtocol.Status pipelineStatus = SUCCESS;
+      Status pipelineStatus = SUCCESS;
       String firstBadLink = "";
       if (DFSClient.LOG.isDebugEnabled()) {
         for (int i = 0; i < nodes.length; i++) {
@@ -1023,10 +1031,13 @@
         out.flush();
 
         // receive ack for connect
-        pipelineStatus = DataTransferProtocol.Status.read(blockReplyStream);
-        firstBadLink = Text.readString(blockReplyStream);
+        BlockOpResponseProto resp = BlockOpResponseProto.parseFrom(
+            HdfsProtoUtil.vintPrefixed(blockReplyStream));
+        pipelineStatus = resp.getStatus();
+        firstBadLink = resp.getFirstBadLink();
+        
         if (pipelineStatus != SUCCESS) {
-          if (pipelineStatus == ERROR_ACCESS_TOKEN) {
+          if (pipelineStatus == Status.ERROR_ACCESS_TOKEN) {
             throw new InvalidBlockTokenException(
                 "Got access token error for connect ack with firstBadLink as "
                     + firstBadLink);
diff --git a/src/java/org/apache/hadoop/hdfs/protocol/DataTransferProtoUtil.java b/src/java/org/apache/hadoop/hdfs/protocol/DataTransferProtoUtil.java
new file mode 100644
index 0000000..783f65c
--- /dev/null
+++ b/src/java/org/apache/hadoop/hdfs/protocol/DataTransferProtoUtil.java
@@ -0,0 +1,64 @@
+/**
+ * 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.protocol;
+
+
+import org.apache.hadoop.hdfs.protocol.DataTransferProtocol.BlockConstructionStage;
+import org.apache.hadoop.hdfs.protocol.proto.DataTransferProtos.BaseHeaderProto;
+import org.apache.hadoop.hdfs.protocol.proto.DataTransferProtos.ClientOperationHeaderProto;
+import org.apache.hadoop.hdfs.protocol.proto.DataTransferProtos.OpWriteBlockProto;
+import org.apache.hadoop.hdfs.security.token.block.BlockTokenIdentifier;
+import org.apache.hadoop.security.token.Token;
+
+
+/**
+ * Static utilities for dealing with the protocol buffers used by the
+ * Data Transfer Protocol.
+ */
+abstract class DataTransferProtoUtil {
+
+  static DataTransferProtocol.BlockConstructionStage fromProto(
+      OpWriteBlockProto.BlockConstructionStage stage) {
+    return BlockConstructionStage.valueOf(BlockConstructionStage.class,
+        stage.name());
+  }
+
+  static OpWriteBlockProto.BlockConstructionStage toProto(
+      BlockConstructionStage stage) {
+    return OpWriteBlockProto.BlockConstructionStage.valueOf(
+        stage.name());
+  }
+
+  static ClientOperationHeaderProto buildClientHeader(ExtendedBlock blk,
+      String client, Token<BlockTokenIdentifier> blockToken) {
+    ClientOperationHeaderProto header =
+      ClientOperationHeaderProto.newBuilder()
+        .setBaseHeader(buildBaseHeader(blk, blockToken))
+        .setClientName(client)
+        .build();
+    return header;
+  }
+
+  static BaseHeaderProto buildBaseHeader(ExtendedBlock blk,
+      Token<BlockTokenIdentifier> blockToken) {
+    return BaseHeaderProto.newBuilder()
+      .setBlock(HdfsProtoUtil.toProto(blk))
+      .setToken(HdfsProtoUtil.toProto(blockToken))
+      .build();
+  }
+}
diff --git a/src/java/org/apache/hadoop/hdfs/protocol/DataTransferProtocol.java b/src/java/org/apache/hadoop/hdfs/protocol/DataTransferProtocol.java
index 9923c1f..c340804 100644
--- a/src/java/org/apache/hadoop/hdfs/protocol/DataTransferProtocol.java
+++ b/src/java/org/apache/hadoop/hdfs/protocol/DataTransferProtocol.java
@@ -22,8 +22,10 @@
 import java.io.DataOutput;
 import java.io.DataOutputStream;
 import java.io.IOException;
+import java.io.InputStream;
 import java.io.OutputStream;
 import java.nio.ByteBuffer;
+import java.util.Arrays;
 
 import org.apache.commons.logging.Log;
 import org.apache.commons.logging.LogFactory;
@@ -32,12 +34,31 @@
 import org.apache.hadoop.classification.InterfaceStability;
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.hdfs.DFSConfigKeys;
+import org.apache.hadoop.hdfs.protocol.proto.DataTransferProtos;
+import org.apache.hadoop.hdfs.protocol.proto.DataTransferProtos.ClientOperationHeaderProto;
+import org.apache.hadoop.hdfs.protocol.proto.DataTransferProtos.OpBlockChecksumProto;
+import org.apache.hadoop.hdfs.protocol.proto.DataTransferProtos.OpCopyBlockProto;
+import org.apache.hadoop.hdfs.protocol.proto.DataTransferProtos.OpReadBlockProto;
+import org.apache.hadoop.hdfs.protocol.proto.DataTransferProtos.OpReplaceBlockProto;
+import org.apache.hadoop.hdfs.protocol.proto.DataTransferProtos.OpTransferBlockProto;
+import org.apache.hadoop.hdfs.protocol.proto.DataTransferProtos.OpWriteBlockProto;
+import org.apache.hadoop.hdfs.protocol.proto.DataTransferProtos.PacketHeaderProto;
+import org.apache.hadoop.hdfs.protocol.proto.DataTransferProtos.PipelineAckProto;
+import org.apache.hadoop.hdfs.protocol.proto.DataTransferProtos.Status;
 import org.apache.hadoop.hdfs.security.token.block.BlockTokenIdentifier;
-import org.apache.hadoop.io.Text;
-import org.apache.hadoop.io.Writable;
-import org.apache.hadoop.io.WritableUtils;
+import org.apache.hadoop.hdfs.util.ByteBufferOutputStream;
 import org.apache.hadoop.security.token.Token;
 
+import static org.apache.hadoop.hdfs.protocol.DataTransferProtoUtil.fromProto;
+import static org.apache.hadoop.hdfs.protocol.DataTransferProtoUtil.toProto;
+import static org.apache.hadoop.hdfs.protocol.HdfsProtoUtil.fromProto;
+import static org.apache.hadoop.hdfs.protocol.HdfsProtoUtil.fromProtos;
+import static org.apache.hadoop.hdfs.protocol.HdfsProtoUtil.toProto;
+import static org.apache.hadoop.hdfs.protocol.HdfsProtoUtil.toProtos;
+import static org.apache.hadoop.hdfs.protocol.HdfsProtoUtil.vintPrefixed;
+
+import com.google.protobuf.Message;
+
 /**
  * Transfer data to/from datanode using a streaming protocol.
  */
@@ -89,373 +110,8 @@
     public void write(DataOutput out) throws IOException {
       out.write(code);
     }
-
-    /** Base class for all headers. */
-    private static abstract class BaseHeader implements Writable {
-      private ExtendedBlock block;
-      private Token<BlockTokenIdentifier> blockToken;
-      
-      private BaseHeader() {}
-      
-      private BaseHeader(
-          final ExtendedBlock block,
-          final Token<BlockTokenIdentifier> blockToken) {
-        this.block = block;
-        this.blockToken = blockToken;
-      }
-
-      /** @return the extended block. */
-      public final ExtendedBlock getBlock() {
-        return block;
-      }
-
-      /** @return the block token. */
-      public final Token<BlockTokenIdentifier> getBlockToken() {
-        return blockToken;
-      }
-
-      @Override
-      public void write(DataOutput out) throws IOException {
-        block.writeId(out);
-        blockToken.write(out);
-      }
-
-      @Override
-      public void readFields(DataInput in) throws IOException {
-        block = new ExtendedBlock();
-        block.readId(in);
-
-        blockToken = new Token<BlockTokenIdentifier>();
-        blockToken.readFields(in);
-      }
-    }
-
-    /** Base header for all client operation. */
-    private static abstract class ClientOperationHeader extends BaseHeader {
-      private String clientName;
-      
-      private ClientOperationHeader() {}
-      
-      private ClientOperationHeader(
-          final ExtendedBlock block,
-          final Token<BlockTokenIdentifier> blockToken,
-          final String clientName) {
-        super(block, blockToken);
-        this.clientName = clientName;
-      }
-
-      /** @return client name. */
-      public final String getClientName() {
-        return clientName;
-      }
-
-      @Override
-      public void write(DataOutput out) throws IOException {
-        super.write(out);
-        Text.writeString(out, clientName);
-      }
-
-      @Override
-      public void readFields(DataInput in) throws IOException {
-        super.readFields(in);
-        clientName = Text.readString(in);
-      }
-    }
-
-    /** {@link Op#READ_BLOCK} header. */
-    public static class ReadBlockHeader extends ClientOperationHeader {
-      private long offset;
-      private long length;
-
-      /** Default constructor */
-      public ReadBlockHeader() {}
-
-      /** Constructor with all parameters */
-      public ReadBlockHeader(
-          final ExtendedBlock blk,
-          final Token<BlockTokenIdentifier> blockToken,
-          final String clientName,
-          final long offset,
-          final long length) {
-        super(blk, blockToken, clientName);
-        this.offset = offset;
-        this.length = length;
-      }
-
-      /** @return the offset */
-      public long getOffset() {
-        return offset;
-      }
-
-      /** @return the length */
-      public long getLength() {
-        return length;
-      }
-
-      @Override
-      public void write(DataOutput out) throws IOException {
-        super.write(out);
-        out.writeLong(offset);
-        out.writeLong(length);
-      }
-
-      @Override
-      public void readFields(DataInput in) throws IOException {
-        super.readFields(in);
-        offset = in.readLong();
-        length = in.readLong();
-      }
-    }
-
-    /** {@link Op#WRITE_BLOCK} header. */
-    public static class WriteBlockHeader extends ClientOperationHeader {
-      private DatanodeInfo[] targets;
-
-      private DatanodeInfo source;
-      private BlockConstructionStage stage;
-      private int pipelineSize;
-      private long minBytesRcvd;
-      private long maxBytesRcvd;
-      private long latestGenerationStamp;
-      
-      /** Default constructor */
-      public WriteBlockHeader() {}
-
-      /** Constructor with all parameters */
-      public WriteBlockHeader(
-          final ExtendedBlock blk,
-          final Token<BlockTokenIdentifier> blockToken,
-          final String clientName,
-          final DatanodeInfo[] targets,
-          final DatanodeInfo source,
-          final BlockConstructionStage stage,
-          final int pipelineSize,
-          final long minBytesRcvd,
-          final long maxBytesRcvd,
-          final long latestGenerationStamp
-          ) throws IOException {
-        super(blk, blockToken, clientName);
-        this.targets = targets;
-        this.source = source;
-        this.stage = stage;
-        this.pipelineSize = pipelineSize;
-        this.minBytesRcvd = minBytesRcvd;
-        this.maxBytesRcvd = maxBytesRcvd;
-        this.latestGenerationStamp = latestGenerationStamp;
-      }
-
-      /** @return targets. */
-      public DatanodeInfo[] getTargets() {
-        return targets;
-      }
-
-      /** @return the source */
-      public DatanodeInfo getSource() {
-        return source;
-      }
-
-      /** @return the stage */
-      public BlockConstructionStage getStage() {
-        return stage;
-      }
-
-      /** @return the pipeline size */
-      public int getPipelineSize() {
-        return pipelineSize;
-      }
-
-      /** @return the minimum bytes received. */
-      public long getMinBytesRcvd() {
-        return minBytesRcvd;
-      }
-
-      /** @return the maximum bytes received. */
-      public long getMaxBytesRcvd() {
-        return maxBytesRcvd;
-      }
-
-      /** @return the latest generation stamp */
-      public long getLatestGenerationStamp() {
-        return latestGenerationStamp;
-      }
-
-      @Override
-      public void write(DataOutput out) throws IOException {
-        super.write(out);
-        Sender.write(out, 1, targets);
-
-        out.writeBoolean(source != null);
-        if (source != null) {
-          source.write(out);
-        }
-
-        stage.write(out);
-        out.writeInt(pipelineSize);
-        WritableUtils.writeVLong(out, minBytesRcvd);
-        WritableUtils.writeVLong(out, maxBytesRcvd);
-        WritableUtils.writeVLong(out, latestGenerationStamp);
-      }
-
-      @Override
-      public void readFields(DataInput in) throws IOException {
-        super.readFields(in);
-        targets = Receiver.readDatanodeInfos(in);
-
-        source = in.readBoolean()? DatanodeInfo.read(in): null;
-        stage = BlockConstructionStage.readFields(in);
-        pipelineSize = in.readInt(); // num of datanodes in entire pipeline
-        minBytesRcvd = WritableUtils.readVLong(in);
-        maxBytesRcvd = WritableUtils.readVLong(in);
-        latestGenerationStamp = WritableUtils.readVLong(in);
-      }
-    }
-
-    /** {@link Op#TRANSFER_BLOCK} header. */
-    public static class TransferBlockHeader extends ClientOperationHeader {
-      private DatanodeInfo[] targets;
-
-      /** Default constructor */
-      public TransferBlockHeader() {}
-
-      /** Constructor with all parameters */
-      public TransferBlockHeader(
-          final ExtendedBlock blk,
-          final Token<BlockTokenIdentifier> blockToken,
-          final String clientName,
-          final DatanodeInfo[] targets) throws IOException {
-        super(blk, blockToken, clientName);
-        this.targets = targets;
-      }
-
-      /** @return targets. */
-      public DatanodeInfo[] getTargets() {
-        return targets;
-      }
-
-      @Override
-      public void write(DataOutput out) throws IOException {
-        super.write(out);
-        Sender.write(out, 0, targets);
-      }
-
-      @Override
-      public void readFields(DataInput in) throws IOException {
-        super.readFields(in);
-        targets = Receiver.readDatanodeInfos(in);
-      }
-    }
-
-    /** {@link Op#REPLACE_BLOCK} header. */
-    public static class ReplaceBlockHeader extends BaseHeader {
-      private String delHint;
-      private DatanodeInfo source;
-
-      /** Default constructor */
-      public ReplaceBlockHeader() {}
-
-      /** Constructor with all parameters */
-      public ReplaceBlockHeader(final ExtendedBlock blk,
-          final Token<BlockTokenIdentifier> blockToken,
-          final String storageId,
-          final DatanodeInfo src) throws IOException {
-        super(blk, blockToken);
-        this.delHint = storageId;
-        this.source = src;
-      }
-
-      /** @return delete-hint. */
-      public String getDelHint() {
-        return delHint;
-      }
-
-      /** @return source datanode. */
-      public DatanodeInfo getSource() {
-        return source;
-      }
-
-      @Override
-      public void write(DataOutput out) throws IOException {
-        super.write(out);
-        Text.writeString(out, delHint);
-        source.write(out);
-      }
-
-      @Override
-      public void readFields(DataInput in) throws IOException {
-        super.readFields(in);
-        delHint = Text.readString(in);
-        source = DatanodeInfo.read(in);
-      }
-    }
-
-    /** {@link Op#COPY_BLOCK} header. */
-    public static class CopyBlockHeader extends BaseHeader {
-      /** Default constructor */
-      public CopyBlockHeader() {}
-
-      /** Constructor with all parameters */
-      public CopyBlockHeader(
-          final ExtendedBlock block,
-          final Token<BlockTokenIdentifier> blockToken) {
-        super(block, blockToken);
-      }
-    }
-
-    /** {@link Op#BLOCK_CHECKSUM} header. */
-    public static class BlockChecksumHeader extends BaseHeader {
-      /** Default constructor */
-      public BlockChecksumHeader() {}
-
-      /** Constructor with all parameters */
-      public BlockChecksumHeader(
-          final ExtendedBlock block,
-          final Token<BlockTokenIdentifier> blockToken) {
-        super(block, blockToken);
-      }
-    }
   }
-
-
-  /** Status */
-  public enum Status {
-    SUCCESS(0),
-    ERROR(1),
-    ERROR_CHECKSUM(2),
-    ERROR_INVALID(3),
-    ERROR_EXISTS(4),
-    ERROR_ACCESS_TOKEN(5),
-    CHECKSUM_OK(6);
-
-    /** The code for this operation. */
-    private final int code;
     
-    private Status(int code) {
-      this.code = code;
-    }
-
-    private static final int FIRST_CODE = values()[0].code;
-    /** Return the object represented by the code. */
-    private static Status valueOf(int code) {
-      final int i = code - FIRST_CODE;
-      return i < 0 || i >= values().length? null: values()[i];
-    }
-
-    /** Read from in */
-    public static Status read(DataInput in) throws IOException {
-      return valueOf(in.readShort());
-    }
-
-    /** Write to out */
-    public void write(DataOutput out) throws IOException {
-      out.writeShort(code);
-    }
-
-    /** Write to out */
-    public void writeOutputStream(OutputStream out) throws IOException {
-      out.write(new byte[] {(byte)(code >>> 8), (byte)code});
-    }
-  };
-  
   public enum BlockConstructionStage {
     /** The enumerates are always listed as regular stage followed by the
      * recovery stage. 
@@ -492,23 +148,9 @@
         return values()[ordinal()|RECOVERY_BIT];
       }
     }
-    
-    private static BlockConstructionStage valueOf(byte code) {
-      return code < 0 || code >= values().length? null: values()[code];
-    }
-    
-    /** Read from in */
-    private static BlockConstructionStage readFields(DataInput in)
-    throws IOException {
-      return valueOf(in.readByte());
-    }
-
-    /** write to out */
-    private void write(DataOutput out) throws IOException {
-      out.writeByte(ordinal());
-    }
   }    
 
+  
   /** Sender */
   @InterfaceAudience.Private
   @InterfaceStability.Evolving
@@ -520,11 +162,10 @@
       op.write(out);
     }
 
-    /** Send an operation request. */
     private static void send(final DataOutputStream out, final Op opcode,
-        final Op.BaseHeader parameters) throws IOException {
+        final Message proto) throws IOException {
       op(out, opcode);
-      parameters.write(out);
+      proto.writeDelimitedTo(out);
       out.flush();
     }
 
@@ -533,59 +174,90 @@
         long blockOffset, long blockLen, String clientName,
         Token<BlockTokenIdentifier> blockToken)
         throws IOException {
-      send(out, Op.READ_BLOCK, new Op.ReadBlockHeader(blk, blockToken,
-          clientName, blockOffset, blockLen));
+
+      OpReadBlockProto proto = OpReadBlockProto.newBuilder()
+        .setHeader(DataTransferProtoUtil.buildClientHeader(blk, clientName, blockToken))
+        .setOffset(blockOffset)
+        .setLen(blockLen)
+        .build();
+
+      send(out, Op.READ_BLOCK, proto);
     }
     
+
     /** Send OP_WRITE_BLOCK */
     public static void opWriteBlock(DataOutputStream out, ExtendedBlock blk,
         int pipelineSize, BlockConstructionStage stage, long newGs,
         long minBytesRcvd, long maxBytesRcvd, String client, DatanodeInfo src,
         DatanodeInfo[] targets, Token<BlockTokenIdentifier> blockToken)
         throws IOException {
-      send(out, Op.WRITE_BLOCK, new Op.WriteBlockHeader(blk, blockToken,
-          client, targets, src, stage, pipelineSize, minBytesRcvd, maxBytesRcvd,
-          newGs));
+      ClientOperationHeaderProto header = DataTransferProtoUtil.buildClientHeader(blk, client,
+          blockToken);
+      
+      OpWriteBlockProto.Builder proto = OpWriteBlockProto.newBuilder()
+        .setHeader(header)
+        .addAllTargets(
+            toProtos(targets, 1))
+        .setStage(toProto(stage))
+        .setPipelineSize(pipelineSize)
+        .setMinBytesRcvd(minBytesRcvd)
+        .setMaxBytesRcvd(maxBytesRcvd)
+        .setLatestGenerationStamp(newGs);
+      
+      if (src != null) {
+        proto.setSource(toProto(src));
+      }
+
+      send(out, Op.WRITE_BLOCK, proto.build());
     }
 
     /** Send {@link Op#TRANSFER_BLOCK} */
     public static void opTransferBlock(DataOutputStream out, ExtendedBlock blk,
         String client, DatanodeInfo[] targets,
         Token<BlockTokenIdentifier> blockToken) throws IOException {
-      send(out, Op.TRANSFER_BLOCK, new Op.TransferBlockHeader(blk, blockToken,
-          client, targets));
+      
+      OpTransferBlockProto proto = OpTransferBlockProto.newBuilder()
+        .setHeader(DataTransferProtoUtil.buildClientHeader(
+            blk, client, blockToken))
+        .addAllTargets(toProtos(targets, 0))
+        .build();
+
+      send(out, Op.TRANSFER_BLOCK, proto);
     }
 
     /** Send OP_REPLACE_BLOCK */
     public static void opReplaceBlock(DataOutputStream out,
         ExtendedBlock blk, String delHint, DatanodeInfo src,
         Token<BlockTokenIdentifier> blockToken) throws IOException {
-      send(out, Op.REPLACE_BLOCK, new Op.ReplaceBlockHeader(blk, blockToken,
-          delHint, src));
+      OpReplaceBlockProto proto = OpReplaceBlockProto.newBuilder()
+        .setHeader(DataTransferProtoUtil.buildBaseHeader(blk, blockToken))
+        .setDelHint(delHint)
+        .setSource(toProto(src))
+        .build();
+      
+      send(out, Op.REPLACE_BLOCK, proto);
     }
 
     /** Send OP_COPY_BLOCK */
     public static void opCopyBlock(DataOutputStream out, ExtendedBlock blk,
         Token<BlockTokenIdentifier> blockToken)
         throws IOException {
-      send(out, Op.COPY_BLOCK, new Op.CopyBlockHeader(blk, blockToken));
+      OpCopyBlockProto proto = OpCopyBlockProto.newBuilder()
+        .setHeader(DataTransferProtoUtil.buildBaseHeader(blk, blockToken))
+        .build();
+      
+      send(out, Op.COPY_BLOCK, proto);
     }
 
     /** Send OP_BLOCK_CHECKSUM */
     public static void opBlockChecksum(DataOutputStream out, ExtendedBlock blk,
         Token<BlockTokenIdentifier> blockToken)
         throws IOException {
-      send(out, Op.BLOCK_CHECKSUM, new Op.BlockChecksumHeader(blk, blockToken));
-    }
-
-    /** Write an array of {@link DatanodeInfo} */
-    private static void write(final DataOutput out,
-        final int start, 
-        final DatanodeInfo[] datanodeinfos) throws IOException {
-      out.writeInt(datanodeinfos.length - start);
-      for (int i = start; i < datanodeinfos.length; i++) {
-        datanodeinfos[i].write(out);
-      }
+      OpBlockChecksumProto proto = OpBlockChecksumProto.newBuilder()
+        .setHeader(DataTransferProtoUtil.buildBaseHeader(blk, blockToken))
+        .build();
+      
+      send(out, Op.BLOCK_CHECKSUM, proto);
     }
   }
 
@@ -631,12 +303,16 @@
 
     /** Receive OP_READ_BLOCK */
     private void opReadBlock(DataInputStream in) throws IOException {
-      final Op.ReadBlockHeader h = new Op.ReadBlockHeader();
-      h.readFields(in);
-      opReadBlock(in, h.getBlock(), h.getOffset(), h.getLength(),
-          h.getClientName(), h.getBlockToken());
-    }
+      OpReadBlockProto proto = OpReadBlockProto.parseFrom(vintPrefixed(in));
+      
+      ExtendedBlock b = fromProto(
+          proto.getHeader().getBaseHeader().getBlock());
+      Token<BlockTokenIdentifier> token = fromProto(
+          proto.getHeader().getBaseHeader().getToken());
 
+      opReadBlock(in, b, proto.getOffset(), proto.getLen(),
+          proto.getHeader().getClientName(), token);
+    }
     /**
      * Abstract OP_READ_BLOCK method. Read a block.
      */
@@ -646,12 +322,17 @@
     
     /** Receive OP_WRITE_BLOCK */
     private void opWriteBlock(DataInputStream in) throws IOException {
-      final Op.WriteBlockHeader h = new Op.WriteBlockHeader();
-      h.readFields(in);
-      opWriteBlock(in, h.getBlock(), h.getPipelineSize(), h.getStage(),
-          h.getLatestGenerationStamp(),
-          h.getMinBytesRcvd(), h.getMaxBytesRcvd(),
-          h.getClientName(), h.getSource(), h.getTargets(), h.getBlockToken());
+      final OpWriteBlockProto proto = OpWriteBlockProto.parseFrom(vintPrefixed(in));
+      opWriteBlock(in,
+          fromProto(proto.getHeader().getBaseHeader().getBlock()),
+          proto.getPipelineSize(),
+          fromProto(proto.getStage()),
+          proto.getLatestGenerationStamp(),
+          proto.getMinBytesRcvd(), proto.getMaxBytesRcvd(),
+          proto.getHeader().getClientName(),
+          fromProto(proto.getSource()),
+          fromProtos(proto.getTargetsList()),
+          fromProto(proto.getHeader().getBaseHeader().getToken()));
     }
 
     /**
@@ -666,10 +347,14 @@
 
     /** Receive {@link Op#TRANSFER_BLOCK} */
     private void opTransferBlock(DataInputStream in) throws IOException {
-      final Op.TransferBlockHeader h = new Op.TransferBlockHeader();
-      h.readFields(in);
-      opTransferBlock(in, h.getBlock(), h.getClientName(), h.getTargets(),
-          h.getBlockToken());
+      final OpTransferBlockProto proto =
+        OpTransferBlockProto.parseFrom(vintPrefixed(in));
+
+      opTransferBlock(in,
+          fromProto(proto.getHeader().getBaseHeader().getBlock()),
+          proto.getHeader().getClientName(),
+          fromProtos(proto.getTargetsList()),
+          fromProto(proto.getHeader().getBaseHeader().getToken()));
     }
 
     /**
@@ -684,10 +369,13 @@
 
     /** Receive OP_REPLACE_BLOCK */
     private void opReplaceBlock(DataInputStream in) throws IOException {
-      final Op.ReplaceBlockHeader h = new Op.ReplaceBlockHeader();
-      h.readFields(in);
-      opReplaceBlock(in, h.getBlock(), h.getDelHint(), h.getSource(),
-          h.getBlockToken());
+      OpReplaceBlockProto proto = OpReplaceBlockProto.parseFrom(vintPrefixed(in));
+
+      opReplaceBlock(in,
+          fromProto(proto.getHeader().getBlock()),
+          proto.getDelHint(),
+          fromProto(proto.getSource()),
+          fromProto(proto.getHeader().getToken()));
     }
 
     /**
@@ -700,9 +388,11 @@
 
     /** Receive OP_COPY_BLOCK */
     private void opCopyBlock(DataInputStream in) throws IOException {
-      final Op.CopyBlockHeader h = new Op.CopyBlockHeader();
-      h.readFields(in);
-      opCopyBlock(in, h.getBlock(), h.getBlockToken());
+      OpCopyBlockProto proto = OpCopyBlockProto.parseFrom(vintPrefixed(in));
+      
+      opCopyBlock(in,
+          fromProto(proto.getHeader().getBlock()),
+          fromProto(proto.getHeader().getToken()));
     }
 
     /**
@@ -715,9 +405,11 @@
 
     /** Receive OP_BLOCK_CHECKSUM */
     private void opBlockChecksum(DataInputStream in) throws IOException {
-      final Op.BlockChecksumHeader h = new Op.BlockChecksumHeader();
-      h.readFields(in);
-      opBlockChecksum(in, h.getBlock(), h.getBlockToken());
+      OpBlockChecksumProto proto = OpBlockChecksumProto.parseFrom(vintPrefixed(in));
+      
+      opBlockChecksum(in,
+          fromProto(proto.getHeader().getBlock()),
+          fromProto(proto.getHeader().getToken()));
     }
 
     /**
@@ -727,29 +419,13 @@
     protected abstract void opBlockChecksum(DataInputStream in,
         ExtendedBlock blk, Token<BlockTokenIdentifier> blockToken)
         throws IOException;
-
-    /** Read an array of {@link DatanodeInfo} */
-    private static DatanodeInfo[] readDatanodeInfos(final DataInput in
-        ) throws IOException {
-      final int n = in.readInt();
-      if (n < 0) {
-        throw new IOException("Mislabelled incoming datastream: "
-            + n + " = n < 0");
-      }
-      final DatanodeInfo[] datanodeinfos= new DatanodeInfo[n];
-      for (int i = 0; i < datanodeinfos.length; i++) {
-        datanodeinfos[i] = DatanodeInfo.read(in);
-      }
-      return datanodeinfos;
-    }
   }
   
   /** reply **/
   @InterfaceAudience.Private
   @InterfaceStability.Evolving
-  public static class PipelineAck implements Writable {
-    private long seqno;
-    private Status replies[];
+  public static class PipelineAck {
+    PipelineAckProto proto;
     public final static long UNKOWN_SEQNO = -2;
 
     /** default constructor **/
@@ -762,8 +438,10 @@
      * @param replies an array of replies
      */
     public PipelineAck(long seqno, Status[] replies) {
-      this.seqno = seqno;
-      this.replies = replies;
+      proto = PipelineAckProto.newBuilder()
+        .setSeqno(seqno)
+        .addAllStatus(Arrays.asList(replies))
+        .build();
     }
     
     /**
@@ -771,7 +449,7 @@
      * @return the sequence number
      */
     public long getSeqno() {
-      return seqno;
+      return proto.getSeqno();
     }
     
     /**
@@ -779,7 +457,7 @@
      * @return the number of replies
      */
     public short getNumOfReplies() {
-      return (short)replies.length;
+      return (short)proto.getStatusCount();
     }
     
     /**
@@ -787,11 +465,7 @@
      * @return the the ith reply
      */
     public Status getReply(int i) {
-      if (i<0 || i>=replies.length) {
-        throw new IllegalArgumentException("The input parameter " + i + 
-            " should in the range of [0, " + replies.length);
-      }
-      return replies[i];
+      return proto.getStatus(i);
     }
     
     /**
@@ -799,8 +473,8 @@
      * @return true if all statuses are SUCCESS
      */
     public boolean isSuccess() {
-      for (Status reply : replies) {
-        if (reply != Status.SUCCESS) {
+      for (DataTransferProtos.Status reply : proto.getStatusList()) {
+        if (reply != DataTransferProtos.Status.SUCCESS) {
           return false;
         }
       }
@@ -808,54 +482,37 @@
     }
     
     /**** Writable interface ****/
-    @Override // Writable
-    public void readFields(DataInput in) throws IOException {
-      seqno = in.readLong();
-      short numOfReplies = in.readShort();
-      replies = new Status[numOfReplies];
-      for (int i=0; i<numOfReplies; i++) {
-        replies[i] = Status.read(in);
-      }
+    public void readFields(InputStream in) throws IOException {
+      proto = PipelineAckProto.parseFrom(vintPrefixed(in));
     }
 
-    @Override // Writable
-    public void write(DataOutput out) throws IOException {
-      //WritableUtils.writeVLong(out, seqno);
-      out.writeLong(seqno);
-      out.writeShort((short)replies.length);
-      for(Status reply : replies) {
-        reply.write(out);
-      }
+    public void write(OutputStream out) throws IOException {
+      proto.writeDelimitedTo(out);
     }
     
     @Override //Object
     public String toString() {
-      StringBuilder ack = new StringBuilder("Replies for seqno ");
-      ack.append( seqno ).append( " are" );
-      for(Status reply : replies) {
-        ack.append(" ");
-        ack.append(reply);
-      }
-      return ack.toString();
+      return proto.toString();
     }
   }
 
   /**
    * Header data for each packet that goes through the read/write pipelines.
    */
-  public static class PacketHeader implements Writable {
+  public static class PacketHeader {
     /** Header size for a packet */
-    public static final int PKT_HEADER_LEN = ( 4 + /* Packet payload length */
-                                               8 + /* offset in block */
-                                               8 + /* seqno */
-                                               1 + /* isLastPacketInBlock */
-                                               4   /* data length */ );
+    private static final int PROTO_SIZE = 
+      PacketHeaderProto.newBuilder()
+        .setOffsetInBlock(0)
+        .setSeqno(0)
+        .setLastPacketInBlock(false)
+        .setDataLen(0)
+        .build().getSerializedSize();
+    public static final int PKT_HEADER_LEN =
+      6 + PROTO_SIZE;
 
     private int packetLen;
-    private long offsetInBlock;
-    private long seqno;
-    private boolean lastPacketInBlock;
-    private int dataLen;
+    private PacketHeaderProto proto;
 
     public PacketHeader() {
     }
@@ -863,26 +520,28 @@
     public PacketHeader(int packetLen, long offsetInBlock, long seqno,
                         boolean lastPacketInBlock, int dataLen) {
       this.packetLen = packetLen;
-      this.offsetInBlock = offsetInBlock;
-      this.seqno = seqno;
-      this.lastPacketInBlock = lastPacketInBlock;
-      this.dataLen = dataLen;
+      proto = PacketHeaderProto.newBuilder()
+        .setOffsetInBlock(offsetInBlock)
+        .setSeqno(seqno)
+        .setLastPacketInBlock(lastPacketInBlock)
+        .setDataLen(dataLen)
+        .build();
     }
 
     public int getDataLen() {
-      return dataLen;
+      return proto.getDataLen();
     }
 
     public boolean isLastPacketInBlock() {
-      return lastPacketInBlock;
+      return proto.getLastPacketInBlock();
     }
 
     public long getSeqno() {
-      return seqno;
+      return proto.getSeqno();
     }
 
     public long getOffsetInBlock() {
-      return offsetInBlock;
+      return proto.getOffsetInBlock();
     }
 
     public int getPacketLen() {
@@ -891,57 +550,50 @@
 
     @Override
     public String toString() {
-      StringBuilder sb = new StringBuilder();
-      sb.append("PacketHeader(")
-        .append("packetLen=").append(packetLen)
-        .append(" offsetInBlock=").append(offsetInBlock)
-        .append(" seqno=").append(seqno)
-        .append(" lastPacketInBlock=").append(lastPacketInBlock)
-        .append(" dataLen=").append(dataLen)
-        .append(")");
-      return sb.toString();
+      return "PacketHeader with packetLen=" + packetLen +
+        "Header data: " + 
+        proto.toString();
     }
-
-    @Override
-    public void readFields(DataInput in) throws IOException {
-      // Note that it's important for packetLen to come first and not
-      // change format -
-      // this is used by BlockReceiver to read entire packets with
-      // a single read call.
-      packetLen = in.readInt();
-      offsetInBlock = in.readLong();
-      seqno = in.readLong();
-      lastPacketInBlock = in.readBoolean();
-      dataLen = in.readInt();
-    }
-
+    
     public void readFields(ByteBuffer buf) throws IOException {
       packetLen = buf.getInt();
-      offsetInBlock = buf.getLong();
-      seqno = buf.getLong();
-      lastPacketInBlock = (buf.get() != 0);
-      dataLen = buf.getInt();
+      short protoLen = buf.getShort();
+      byte[] data = new byte[protoLen];
+      buf.get(data);
+      proto = PacketHeaderProto.parseFrom(data);
+    }
+    
+    public void readFields(DataInputStream in) throws IOException {
+      this.packetLen = in.readInt();
+      short protoLen = in.readShort();
+      byte[] data = new byte[protoLen];
+      in.readFully(data);
+      proto = PacketHeaderProto.parseFrom(data);
     }
 
-    @Override
-    public void write(DataOutput out) throws IOException {
-      out.writeInt(packetLen);
-      out.writeLong(offsetInBlock);
-      out.writeLong(seqno);
-      out.writeBoolean(lastPacketInBlock);
-      out.writeInt(dataLen);
-    }
 
     /**
      * Write the header into the buffer.
      * This requires that PKT_HEADER_LEN bytes are available.
      */
-    public void putInBuffer(ByteBuffer buf) {
-      buf.putInt(packetLen)
-        .putLong(offsetInBlock)
-        .putLong(seqno)
-        .put((byte)(lastPacketInBlock ? 1 : 0))
-        .putInt(dataLen);
+    public void putInBuffer(final ByteBuffer buf) {
+      assert proto.getSerializedSize() == PROTO_SIZE
+        : "Expected " + (PROTO_SIZE) + " got: " + proto.getSerializedSize();
+      try {
+        buf.putInt(packetLen);
+        buf.putShort((short) proto.getSerializedSize());
+        proto.writeTo(new ByteBufferOutputStream(buf));
+      } catch (IOException e) {
+        throw new RuntimeException(e);
+      }
+    }
+    
+    public void write(DataOutputStream out) throws IOException {
+      assert proto.getSerializedSize() == PROTO_SIZE
+      : "Expected " + (PROTO_SIZE) + " got: " + proto.getSerializedSize();
+      out.writeInt(packetLen);
+      out.writeShort(proto.getSerializedSize());
+      proto.writeTo(out);
     }
 
     /**
@@ -951,11 +603,11 @@
      */
     public boolean sanityCheck(long lastSeqNo) {
       // We should only have a non-positive data length for the last packet
-      if (dataLen <= 0 && lastPacketInBlock) return false;
+      if (proto.getDataLen() <= 0 && proto.getLastPacketInBlock()) return false;
       // The last packet should not contain data
-      if (lastPacketInBlock && dataLen != 0) return false;
+      if (proto.getLastPacketInBlock() && proto.getDataLen() != 0) return false;
       // Seqnos should always increase by 1 with each packet received
-      if (seqno != lastSeqNo + 1) return false;
+      if (proto.getSeqno() != lastSeqNo + 1) return false;
       return true;
     }
 
@@ -963,16 +615,12 @@
     public boolean equals(Object o) {
       if (!(o instanceof PacketHeader)) return false;
       PacketHeader other = (PacketHeader)o;
-      return (other.packetLen == packetLen &&
-              other.offsetInBlock == offsetInBlock &&
-              other.seqno == seqno &&
-              other.lastPacketInBlock == lastPacketInBlock &&
-              other.dataLen == dataLen);
+      return this.proto.equals(other.proto);
     }
 
     @Override
     public int hashCode() {
-      return (int)seqno;
+      return (int)proto.getSeqno();
     }
   }
 
diff --git a/src/java/org/apache/hadoop/hdfs/protocol/DatanodeInfo.java b/src/java/org/apache/hadoop/hdfs/protocol/DatanodeInfo.java
index 948f151..17a09f6 100644
--- a/src/java/org/apache/hadoop/hdfs/protocol/DatanodeInfo.java
+++ b/src/java/org/apache/hadoop/hdfs/protocol/DatanodeInfo.java
@@ -159,6 +159,11 @@
   public void setCapacity(long capacity) { 
     this.capacity = capacity; 
   }
+  
+  /** Sets the used space for the datanode. */
+  public void setDfsUsed(long dfsUsed) {
+    this.dfsUsed = dfsUsed;
+  }
 
   /** Sets raw free space. */
   public void setRemaining(long remaining) { 
diff --git a/src/java/org/apache/hadoop/hdfs/protocol/HdfsProtoUtil.java b/src/java/org/apache/hadoop/hdfs/protocol/HdfsProtoUtil.java
new file mode 100644
index 0000000..739a6d2
--- /dev/null
+++ b/src/java/org/apache/hadoop/hdfs/protocol/HdfsProtoUtil.java
@@ -0,0 +1,166 @@
+/**
+ * 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.protocol;
+
+import java.io.EOFException;
+import java.io.IOException;
+import java.io.InputStream;
+import java.util.ArrayList;
+import java.util.List;
+
+import org.apache.hadoop.classification.InterfaceAudience;
+import org.apache.hadoop.classification.InterfaceStability;
+import org.apache.hadoop.hdfs.protocol.proto.HdfsProtos;
+import org.apache.hadoop.hdfs.security.token.block.BlockTokenIdentifier;
+import org.apache.hadoop.hdfs.util.ExactSizeInputStream;
+import org.apache.hadoop.io.Text;
+import org.apache.hadoop.security.token.Token;
+
+import com.google.common.collect.Lists;
+import com.google.protobuf.ByteString;
+import com.google.protobuf.CodedInputStream;
+
+/**
+ * Utilities for converting to and from protocol buffers used in the
+ * HDFS wire protocol, as well as some generic utilities useful
+ * for dealing with protocol buffers.
+ */
+@InterfaceAudience.Private
+@InterfaceStability.Evolving
+public abstract class HdfsProtoUtil {
+  
+  //// Block Token ////
+  
+  public static HdfsProtos.BlockTokenIdentifierProto toProto(Token<?> blockToken) {
+    return HdfsProtos.BlockTokenIdentifierProto.newBuilder()
+      .setIdentifier(ByteString.copyFrom(blockToken.getIdentifier()))
+      .setPassword(ByteString.copyFrom(blockToken.getPassword()))
+      .setKind(blockToken.getKind().toString())
+      .setService(blockToken.getService().toString())
+      .build();
+  }
+
+  public static Token<BlockTokenIdentifier> fromProto(HdfsProtos.BlockTokenIdentifierProto proto) {
+    return new Token<BlockTokenIdentifier>(proto.getIdentifier().toByteArray(),
+        proto.getPassword().toByteArray(),
+        new Text(proto.getKind()),
+        new Text(proto.getService()));
+  }
+
+  //// Extended Block ////
+  
+  public static HdfsProtos.ExtendedBlockProto toProto(ExtendedBlock block) {
+    return HdfsProtos.ExtendedBlockProto.newBuilder()
+      .setBlockId(block.getBlockId())
+      .setPoolId(block.getBlockPoolId())
+      .setNumBytes(block.getNumBytes())
+      .setGenerationStamp(block.getGenerationStamp())
+      .build();
+  }
+    
+  public static ExtendedBlock fromProto(HdfsProtos.ExtendedBlockProto proto) {
+    return new ExtendedBlock(
+        proto.getPoolId(), proto.getBlockId(),
+        proto.getNumBytes(), proto.getGenerationStamp());
+  }
+
+  //// DatanodeID ////
+  
+  private static HdfsProtos.DatanodeIDProto toProto(
+      DatanodeID dni) {
+    return HdfsProtos.DatanodeIDProto.newBuilder()
+      .setName(dni.getName())
+      .setStorageID(dni.getStorageID())
+      .setInfoPort(dni.getInfoPort())
+      .build();
+  }
+  
+  private static DatanodeID fromProto(HdfsProtos.DatanodeIDProto idProto) {
+    return new DatanodeID(
+        idProto.getName(),
+        idProto.getStorageID(),
+        idProto.getInfoPort(),
+        -1); // ipc port not serialized in writables either
+  }
+  
+  //// DatanodeInfo ////
+  
+  public static HdfsProtos.DatanodeInfoProto toProto(DatanodeInfo dni) {
+    return HdfsProtos.DatanodeInfoProto.newBuilder()
+      .setId(toProto((DatanodeID)dni))
+      .setCapacity(dni.getCapacity())
+      .setDfsUsed(dni.getDfsUsed())
+      .setRemaining(dni.getRemaining())
+      .setBlockPoolUsed(dni.getBlockPoolUsed())
+      .setLastUpdate(dni.getLastUpdate())
+      .setXceiverCount(dni.getXceiverCount())
+      .setLocation(dni.getNetworkLocation())
+      .setHostName(dni.getHostName())
+      .setAdminState(HdfsProtos.DatanodeInfoProto.AdminState.valueOf(
+          dni.getAdminState().name()))
+      .build();
+  }
+
+  public static DatanodeInfo fromProto(HdfsProtos.DatanodeInfoProto dniProto) {
+    DatanodeInfo dniObj = new DatanodeInfo(fromProto(dniProto.getId()),
+        dniProto.getLocation(), dniProto.getHostName());
+
+    dniObj.setCapacity(dniProto.getCapacity());
+    dniObj.setDfsUsed(dniProto.getDfsUsed());
+    dniObj.setRemaining(dniProto.getRemaining());
+    dniObj.setBlockPoolUsed(dniProto.getBlockPoolUsed());
+    dniObj.setLastUpdate(dniProto.getLastUpdate());
+    dniObj.setXceiverCount(dniProto.getXceiverCount());
+    dniObj.setAdminState(DatanodeInfo.AdminStates.valueOf(
+        dniProto.getAdminState().name()));
+    return dniObj;
+  }
+  
+  public static ArrayList<? extends HdfsProtos.DatanodeInfoProto> toProtos(
+      DatanodeInfo[] dnInfos, int startIdx) {
+    ArrayList<HdfsProtos.DatanodeInfoProto> protos =
+      Lists.newArrayListWithCapacity(dnInfos.length);
+    for (int i = startIdx; i < dnInfos.length; i++) {
+      protos.add(toProto(dnInfos[i]));
+    }
+    return protos;
+  }
+  
+  public static DatanodeInfo[] fromProtos(
+      List<HdfsProtos.DatanodeInfoProto> targetsList) {
+    DatanodeInfo[] ret = new DatanodeInfo[targetsList.size()];
+    int i = 0;
+    for (HdfsProtos.DatanodeInfoProto proto : targetsList) {
+      ret[i++] = fromProto(proto);
+    }
+    return ret;
+  }
+
+  public static InputStream vintPrefixed(final InputStream input)
+  throws IOException {
+    final int firstByte = input.read();
+    if (firstByte == -1) {
+      throw new EOFException("Premature EOF: no length prefix available");
+    }
+    
+    int size = CodedInputStream.readRawVarint32(firstByte, input);
+    assert size >= 0;
+  
+    return new ExactSizeInputStream(input, size);
+  }
+}
\ No newline at end of file
diff --git a/src/java/org/apache/hadoop/hdfs/protocol/proto/DataTransferProtos.java b/src/java/org/apache/hadoop/hdfs/protocol/proto/DataTransferProtos.java
new file mode 100644
index 0000000..e4f2d1f
--- /dev/null
+++ b/src/java/org/apache/hadoop/hdfs/protocol/proto/DataTransferProtos.java
@@ -0,0 +1,9122 @@
+// Generated by the protocol buffer compiler.  DO NOT EDIT!
+// source: datatransfer.proto
+
+package org.apache.hadoop.hdfs.protocol.proto;
+
+public final class DataTransferProtos {
+  private DataTransferProtos() {}
+  public static void registerAllExtensions(
+      com.google.protobuf.ExtensionRegistry registry) {
+  }
+  public enum Status
+      implements com.google.protobuf.ProtocolMessageEnum {
+    SUCCESS(0, 0),
+    ERROR(1, 1),
+    ERROR_CHECKSUM(2, 2),
+    ERROR_INVALID(3, 3),
+    ERROR_EXISTS(4, 4),
+    ERROR_ACCESS_TOKEN(5, 5),
+    CHECKSUM_OK(6, 6),
+    ;
+    
+    public static final int SUCCESS_VALUE = 0;
+    public static final int ERROR_VALUE = 1;
+    public static final int ERROR_CHECKSUM_VALUE = 2;
+    public static final int ERROR_INVALID_VALUE = 3;
+    public static final int ERROR_EXISTS_VALUE = 4;
+    public static final int ERROR_ACCESS_TOKEN_VALUE = 5;
+    public static final int CHECKSUM_OK_VALUE = 6;
+    
+    
+    public final int getNumber() { return value; }
+    
+    public static Status valueOf(int value) {
+      switch (value) {
+        case 0: return SUCCESS;
+        case 1: return ERROR;
+        case 2: return ERROR_CHECKSUM;
+        case 3: return ERROR_INVALID;
+        case 4: return ERROR_EXISTS;
+        case 5: return ERROR_ACCESS_TOKEN;
+        case 6: return CHECKSUM_OK;
+        default: return null;
+      }
+    }
+    
+    public static com.google.protobuf.Internal.EnumLiteMap<Status>
+        internalGetValueMap() {
+      return internalValueMap;
+    }
+    private static com.google.protobuf.Internal.EnumLiteMap<Status>
+        internalValueMap =
+          new com.google.protobuf.Internal.EnumLiteMap<Status>() {
+            public Status findValueByNumber(int number) {
+              return Status.valueOf(number);
+            }
+          };
+    
+    public final com.google.protobuf.Descriptors.EnumValueDescriptor
+        getValueDescriptor() {
+      return getDescriptor().getValues().get(index);
+    }
+    public final com.google.protobuf.Descriptors.EnumDescriptor
+        getDescriptorForType() {
+      return getDescriptor();
+    }
+    public static final com.google.protobuf.Descriptors.EnumDescriptor
+        getDescriptor() {
+      return org.apache.hadoop.hdfs.protocol.proto.DataTransferProtos.getDescriptor().getEnumTypes().get(0);
+    }
+    
+    private static final Status[] VALUES = {
+      SUCCESS, ERROR, ERROR_CHECKSUM, ERROR_INVALID, ERROR_EXISTS, ERROR_ACCESS_TOKEN, CHECKSUM_OK, 
+    };
+    
+    public static Status valueOf(
+        com.google.protobuf.Descriptors.EnumValueDescriptor desc) {
+      if (desc.getType() != getDescriptor()) {
+        throw new java.lang.IllegalArgumentException(
+          "EnumValueDescriptor is not for this type.");
+      }
+      return VALUES[desc.getIndex()];
+    }
+    
+    private final int index;
+    private final int value;
+    
+    private Status(int index, int value) {
+      this.index = index;
+      this.value = value;
+    }
+    
+    // @@protoc_insertion_point(enum_scope:Status)
+  }
+  
+  public interface BaseHeaderProtoOrBuilder
+      extends com.google.protobuf.MessageOrBuilder {
+    
+    // required .ExtendedBlockProto block = 1;
+    boolean hasBlock();
+    org.apache.hadoop.hdfs.protocol.proto.HdfsProtos.ExtendedBlockProto getBlock();
+    org.apache.hadoop.hdfs.protocol.proto.HdfsProtos.ExtendedBlockProtoOrBuilder getBlockOrBuilder();
+    
+    // optional .BlockTokenIdentifierProto token = 2;
+    boolean hasToken();
+    org.apache.hadoop.hdfs.protocol.proto.HdfsProtos.BlockTokenIdentifierProto getToken();
+    org.apache.hadoop.hdfs.protocol.proto.HdfsProtos.BlockTokenIdentifierProtoOrBuilder getTokenOrBuilder();
+  }
+  public static final class BaseHeaderProto extends
+      com.google.protobuf.GeneratedMessage
+      implements BaseHeaderProtoOrBuilder {
+    // Use BaseHeaderProto.newBuilder() to construct.
+    private BaseHeaderProto(Builder builder) {
+      super(builder);
+    }
+    private BaseHeaderProto(boolean noInit) {}
+    
+    private static final BaseHeaderProto defaultInstance;
+    public static BaseHeaderProto getDefaultInstance() {
+      return defaultInstance;
+    }
+    
+    public BaseHeaderProto getDefaultInstanceForType() {
+      return defaultInstance;
+    }
+    
+    public static final com.google.protobuf.Descriptors.Descriptor
+        getDescriptor() {
+      return org.apache.hadoop.hdfs.protocol.proto.DataTransferProtos.internal_static_BaseHeaderProto_descriptor;
+    }
+    
+    protected com.google.protobuf.GeneratedMessage.FieldAccessorTable
+        internalGetFieldAccessorTable() {
+      return org.apache.hadoop.hdfs.protocol.proto.DataTransferProtos.internal_static_BaseHeaderProto_fieldAccessorTable;
+    }
+    
+    private int bitField0_;
+    // required .ExtendedBlockProto block = 1;
+    public static final int BLOCK_FIELD_NUMBER = 1;
+    private org.apache.hadoop.hdfs.protocol.proto.HdfsProtos.ExtendedBlockProto block_;
+    public boolean hasBlock() {
+      return ((bitField0_ & 0x00000001) == 0x00000001);
+    }
+    public org.apache.hadoop.hdfs.protocol.proto.HdfsProtos.ExtendedBlockProto getBlock() {
+      return block_;
+    }
+    public org.apache.hadoop.hdfs.protocol.proto.HdfsProtos.ExtendedBlockProtoOrBuilder getBlockOrBuilder() {
+      return block_;
+    }
+    
+    // optional .BlockTokenIdentifierProto token = 2;
+    public static final int TOKEN_FIELD_NUMBER = 2;
+    private org.apache.hadoop.hdfs.protocol.proto.HdfsProtos.BlockTokenIdentifierProto token_;
+    public boolean hasToken() {
+      return ((bitField0_ & 0x00000002) == 0x00000002);
+    }
+    public org.apache.hadoop.hdfs.protocol.proto.HdfsProtos.BlockTokenIdentifierProto getToken() {
+      return token_;
+    }
+    public org.apache.hadoop.hdfs.protocol.proto.HdfsProtos.BlockTokenIdentifierProtoOrBuilder getTokenOrBuilder() {
+      return token_;
+    }
+    
+    private void initFields() {
+      block_ = org.apache.hadoop.hdfs.protocol.proto.HdfsProtos.ExtendedBlockProto.getDefaultInstance();
+      token_ = org.apache.hadoop.hdfs.protocol.proto.HdfsProtos.BlockTokenIdentifierProto.getDefaultInstance();
+    }
+    private byte memoizedIsInitialized = -1;
+    public final boolean isInitialized() {
+      byte isInitialized = memoizedIsInitialized;
+      if (isInitialized != -1) return isInitialized == 1;
+      
+      if (!hasBlock()) {
+        memoizedIsInitialized = 0;
+        return false;
+      }
+      if (!getBlock().isInitialized()) {
+        memoizedIsInitialized = 0;
+        return false;
+      }
+      if (hasToken()) {
+        if (!getToken().isInitialized()) {
+          memoizedIsInitialized = 0;
+          return false;
+        }
+      }
+      memoizedIsInitialized = 1;
+      return true;
+    }
+    
+    public void writeTo(com.google.protobuf.CodedOutputStream output)
+                        throws java.io.IOException {
+      getSerializedSize();
+      if (((bitField0_ & 0x00000001) == 0x00000001)) {
+        output.writeMessage(1, block_);
+      }
+      if (((bitField0_ & 0x00000002) == 0x00000002)) {
+        output.writeMessage(2, token_);
+      }
+      getUnknownFields().writeTo(output);
+    }
+    
+    private int memoizedSerializedSize = -1;
+    public int getSerializedSize() {
+      int size = memoizedSerializedSize;
+      if (size != -1) return size;
+    
+      size = 0;
+      if (((bitField0_ & 0x00000001) == 0x00000001)) {
+        size += com.google.protobuf.CodedOutputStream
+          .computeMessageSize(1, block_);
+      }
+      if (((bitField0_ & 0x00000002) == 0x00000002)) {
+        size += com.google.protobuf.CodedOutputStream
+          .computeMessageSize(2, token_);
+      }
+      size += getUnknownFields().getSerializedSize();
+      memoizedSerializedSize = size;
+      return size;
+    }
+    
+    private static final long serialVersionUID = 0L;
+    @java.lang.Override
+    protected java.lang.Object writeReplace()
+        throws java.io.ObjectStreamException {
+      return super.writeReplace();
+    }
+    
+    @java.lang.Override
+    public boolean equals(final java.lang.Object obj) {
+      if (obj == this) {
+       return true;
+      }
+      if (!(obj instanceof org.apache.hadoop.hdfs.protocol.proto.DataTransferProtos.BaseHeaderProto)) {
+        return super.equals(obj);
+      }
+      org.apache.hadoop.hdfs.protocol.proto.DataTransferProtos.BaseHeaderProto other = (org.apache.hadoop.hdfs.protocol.proto.DataTransferProtos.BaseHeaderProto) obj;
+      
+      boolean result = true;
+      result = result && (hasBlock() == other.hasBlock());
+      if (hasBlock()) {
+        result = result && getBlock()
+            .equals(other.getBlock());
+      }
+      result = result && (hasToken() == other.hasToken());
+      if (hasToken()) {
+        result = result && getToken()
+            .equals(other.getToken());
+      }
+      result = result &&
+          getUnknownFields().equals(other.getUnknownFields());
+      return result;
+    }
+    
+    @java.lang.Override
+    public int hashCode() {
+      int hash = 41;
+      hash = (19 * hash) + getDescriptorForType().hashCode();
+      if (hasBlock()) {
+        hash = (37 * hash) + BLOCK_FIELD_NUMBER;
+        hash = (53 * hash) + getBlock().hashCode();
+      }
+      if (hasToken()) {
+        hash = (37 * hash) + TOKEN_FIELD_NUMBER;
+        hash = (53 * hash) + getToken().hashCode();
+      }
+      hash = (29 * hash) + getUnknownFields().hashCode();
+      return hash;
+    }
+    
+    public static org.apache.hadoop.hdfs.protocol.proto.DataTransferProtos.BaseHeaderProto parseFrom(
+        com.google.protobuf.ByteString data)
+        throws com.google.protobuf.InvalidProtocolBufferException {
+      return newBuilder().mergeFrom(data).buildParsed();
+    }
+    public static org.apache.hadoop.hdfs.protocol.proto.DataTransferProtos.BaseHeaderProto parseFrom(
+        com.google.protobuf.ByteString data,
+        com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+        throws com.google.protobuf.InvalidProtocolBufferException {
+      return newBuilder().mergeFrom(data, extensionRegistry)
+               .buildParsed();
+    }
+    public static org.apache.hadoop.hdfs.protocol.proto.DataTransferProtos.BaseHeaderProto parseFrom(byte[] data)
+        throws com.google.protobuf.InvalidProtocolBufferException {
+      return newBuilder().mergeFrom(data).buildParsed();
+    }
+    public static org.apache.hadoop.hdfs.protocol.proto.DataTransferProtos.BaseHeaderProto parseFrom(
+        byte[] data,
+        com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+        throws com.google.protobuf.InvalidProtocolBufferException {
+      return newBuilder().mergeFrom(data, extensionRegistry)
+               .buildParsed();
+    }
+    public static org.apache.hadoop.hdfs.protocol.proto.DataTransferProtos.BaseHeaderProto parseFrom(java.io.InputStream input)
+        throws java.io.IOException {
+      return newBuilder().mergeFrom(input).buildParsed();
+    }
+    public static org.apache.hadoop.hdfs.protocol.proto.DataTransferProtos.BaseHeaderProto parseFrom(
+        java.io.InputStream input,
+        com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+        throws java.io.IOException {
+      return newBuilder().mergeFrom(input, extensionRegistry)
+               .buildParsed();
+    }
+    public static org.apache.hadoop.hdfs.protocol.proto.DataTransferProtos.BaseHeaderProto parseDelimitedFrom(java.io.InputStream input)
+        throws java.io.IOException {
+      Builder builder = newBuilder();
+      if (builder.mergeDelimitedFrom(input)) {
+        return builder.buildParsed();
+      } else {
+        return null;
+      }
+    }
+    public static org.apache.hadoop.hdfs.protocol.proto.DataTransferProtos.BaseHeaderProto parseDelimitedFrom(
+        java.io.InputStream input,
+        com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+        throws java.io.IOException {
+      Builder builder = newBuilder();
+      if (builder.mergeDelimitedFrom(input, extensionRegistry)) {
+        return builder.buildParsed();
+      } else {
+        return null;
+      }
+    }
+    public static org.apache.hadoop.hdfs.protocol.proto.DataTransferProtos.BaseHeaderProto parseFrom(
+        com.google.protobuf.CodedInputStream input)
+        throws java.io.IOException {
+      return newBuilder().mergeFrom(input).buildParsed();
+    }
+    public static org.apache.hadoop.hdfs.protocol.proto.DataTransferProtos.BaseHeaderProto parseFrom(
+        com.google.protobuf.CodedInputStream input,
+        com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+        throws java.io.IOException {
+      return newBuilder().mergeFrom(input, extensionRegistry)
+               .buildParsed();
+    }
+    
+    public static Builder newBuilder() { return Builder.create(); }
+    public Builder newBuilderForType() { return newBuilder(); }
+    public static Builder newBuilder(org.apache.hadoop.hdfs.protocol.proto.DataTransferProtos.BaseHeaderProto prototype) {
+      return newBuilder().mergeFrom(prototype);
+    }
+    public Builder toBuilder() { return newBuilder(this); }
+    
+    @java.lang.Override
+    protected Builder newBuilderForType(
+        com.google.protobuf.GeneratedMessage.BuilderParent parent) {
+      Builder builder = new Builder(parent);
+      return builder;
+    }
+    public static final class Builder extends
+        com.google.protobuf.GeneratedMessage.Builder<Builder>
+       implements org.apache.hadoop.hdfs.protocol.proto.DataTransferProtos.BaseHeaderProtoOrBuilder {
+      public static final com.google.protobuf.Descriptors.Descriptor
+          getDescriptor() {
+        return org.apache.hadoop.hdfs.protocol.proto.DataTransferProtos.internal_static_BaseHeaderProto_descriptor;
+      }
+      
+      protected com.google.protobuf.GeneratedMessage.FieldAccessorTable
+          internalGetFieldAccessorTable() {
+        return org.apache.hadoop.hdfs.protocol.proto.DataTransferProtos.internal_static_BaseHeaderProto_fieldAccessorTable;
+      }
+      
+      // Construct using org.apache.hadoop.hdfs.protocol.proto.DataTransferProtos.BaseHeaderProto.newBuilder()
+      private Builder() {
+        maybeForceBuilderInitialization();
+      }
+      
+      private Builder(BuilderParent parent) {
+        super(parent);
+        maybeForceBuilderInitialization();
+      }
+      private void maybeForceBuilderInitialization() {
+        if (com.google.protobuf.GeneratedMessage.alwaysUseFieldBuilders) {
+          getBlockFieldBuilder();
+          getTokenFieldBuilder();
+        }
+      }
+      private static Builder create() {
+        return new Builder();
+      }
+      
+      public Builder clear() {
+        super.clear();
+        if (blockBuilder_ == null) {
+          block_ = org.apache.hadoop.hdfs.protocol.proto.HdfsProtos.ExtendedBlockProto.getDefaultInstance();
+        } else {
+          blockBuilder_.clear();
+        }
+        bitField0_ = (bitField0_ & ~0x00000001);
+        if (tokenBuilder_ == null) {
+          token_ = org.apache.hadoop.hdfs.protocol.proto.HdfsProtos.BlockTokenIdentifierProto.getDefaultInstance();
+        } else {
+          tokenBuilder_.clear();
+        }
+        bitField0_ = (bitField0_ & ~0x00000002);
+        return this;
+      }
+      
+      public Builder clone() {
+        return create().mergeFrom(buildPartial());
+      }
+      
+      public com.google.protobuf.Descriptors.Descriptor
+          getDescriptorForType() {
+        return org.apache.hadoop.hdfs.protocol.proto.DataTransferProtos.BaseHeaderProto.getDescriptor();
+      }
+      
+      public org.apache.hadoop.hdfs.protocol.proto.DataTransferProtos.BaseHeaderProto getDefaultInstanceForType() {
+        return org.apache.hadoop.hdfs.protocol.proto.DataTransferProtos.BaseHeaderProto.getDefaultInstance();
+      }
+      
+      public org.apache.hadoop.hdfs.protocol.proto.DataTransferProtos.BaseHeaderProto build() {
+        org.apache.hadoop.hdfs.protocol.proto.DataTransferProtos.BaseHeaderProto result = buildPartial();
+        if (!result.isInitialized()) {
+          throw newUninitializedMessageException(result);
+        }
+        return result;
+      }
+      
+      private org.apache.hadoop.hdfs.protocol.proto.DataTransferProtos.BaseHeaderProto buildParsed()
+          throws com.google.protobuf.InvalidProtocolBufferException {
+        org.apache.hadoop.hdfs.protocol.proto.DataTransferProtos.BaseHeaderProto result = buildPartial();
+        if (!result.isInitialized()) {
+          throw newUninitializedMessageException(
+            result).asInvalidProtocolBufferException();
+        }
+        return result;
+      }
+      
+      public org.apache.hadoop.hdfs.protocol.proto.DataTransferProtos.BaseHeaderProto buildPartial() {
+        org.apache.hadoop.hdfs.protocol.proto.DataTransferProtos.BaseHeaderProto result = new org.apache.hadoop.hdfs.protocol.proto.DataTransferProtos.BaseHeaderProto(this);
+        int from_bitField0_ = bitField0_;
+        int to_bitField0_ = 0;
+        if (((from_bitField0_ & 0x00000001) == 0x00000001)) {
+          to_bitField0_ |= 0x00000001;
+        }
+        if (blockBuilder_ == null) {
+          result.block_ = block_;
+        } else {
+          result.block_ = blockBuilder_.build();
+        }
+        if (((from_bitField0_ & 0x00000002) == 0x00000002)) {
+          to_bitField0_ |= 0x00000002;
+        }
+        if (tokenBuilder_ == null) {
+          result.token_ = token_;
+        } else {
+          result.token_ = tokenBuilder_.build();
+        }
+        result.bitField0_ = to_bitField0_;
+        onBuilt();
+        return result;
+      }
+      
+      public Builder mergeFrom(com.google.protobuf.Message other) {
+        if (other instanceof org.apache.hadoop.hdfs.protocol.proto.DataTransferProtos.BaseHeaderProto) {
+          return mergeFrom((org.apache.hadoop.hdfs.protocol.proto.DataTransferProtos.BaseHeaderProto)other);
+        } else {
+          super.mergeFrom(other);
+          return this;
+        }
+      }
+      
+      public Builder mergeFrom(org.apache.hadoop.hdfs.protocol.proto.DataTransferProtos.BaseHeaderProto other) {
+        if (other == org.apache.hadoop.hdfs.protocol.proto.DataTransferProtos.BaseHeaderProto.getDefaultInstance()) return this;
+        if (other.hasBlock()) {
+          mergeBlock(other.getBlock());
+        }
+        if (other.hasToken()) {
+          mergeToken(other.getToken());
+        }
+        this.mergeUnknownFields(other.getUnknownFields());
+        return this;
+      }
+      
+      public final boolean isInitialized() {
+        if (!hasBlock()) {
+          
+          return false;
+        }
+        if (!getBlock().isInitialized()) {
+          
+          return false;
+        }
+        if (hasToken()) {
+          if (!getToken().isInitialized()) {
+            
+            return false;
+          }
+        }
+        return true;
+      }
+      
+      public Builder mergeFrom(
+          com.google.protobuf.CodedInputStream input,
+          com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+          throws java.io.IOException {
+        com.google.protobuf.UnknownFieldSet.Builder unknownFields =
+          com.google.protobuf.UnknownFieldSet.newBuilder(
+            this.getUnknownFields());
+        while (true) {
+          int tag = input.readTag();
+          switch (tag) {
+            case 0:
+              this.setUnknownFields(unknownFields.build());
+              onChanged();
+              return this;
+            default: {
+              if (!parseUnknownField(input, unknownFields,
+                                     extensionRegistry, tag)) {
+                this.setUnknownFields(unknownFields.build());
+                onChanged();
+                return this;
+              }
+              break;
+            }
+            case 10: {
+              org.apache.hadoop.hdfs.protocol.proto.HdfsProtos.ExtendedBlockProto.Builder subBuilder = org.apache.hadoop.hdfs.protocol.proto.HdfsProtos.ExtendedBlockProto.newBuilder();
+              if (hasBlock()) {
+                subBuilder.mergeFrom(getBlock());
+              }
+              input.readMessage(subBuilder, extensionRegistry);
+              setBlock(subBuilder.buildPartial());
+              break;
+            }
+            case 18: {
+              org.apache.hadoop.hdfs.protocol.proto.HdfsProtos.BlockTokenIdentifierProto.Builder subBuilder = org.apache.hadoop.hdfs.protocol.proto.HdfsProtos.BlockTokenIdentifierProto.newBuilder();
+              if (hasToken()) {
+                subBuilder.mergeFrom(getToken());
+              }
+              input.readMessage(subBuilder, extensionRegistry);
+              setToken(subBuilder.buildPartial());
+              break;
+            }
+          }
+        }
+      }
+      
+      private int bitField0_;
+      
+      // required .ExtendedBlockProto block = 1;
+      private org.apache.hadoop.hdfs.protocol.proto.HdfsProtos.ExtendedBlockProto block_ = org.apache.hadoop.hdfs.protocol.proto.HdfsProtos.ExtendedBlockProto.getDefaultInstance();
+      private com.google.protobuf.SingleFieldBuilder<
+          org.apache.hadoop.hdfs.protocol.proto.HdfsProtos.ExtendedBlockProto, org.apache.hadoop.hdfs.protocol.proto.HdfsProtos.ExtendedBlockProto.Builder, org.apache.hadoop.hdfs.protocol.proto.HdfsProtos.ExtendedBlockProtoOrBuilder> blockBuilder_;
+      public boolean hasBlock() {
+        return ((bitField0_ & 0x00000001) == 0x00000001);
+      }
+      public org.apache.hadoop.hdfs.protocol.proto.HdfsProtos.ExtendedBlockProto getBlock() {
+        if (blockBuilder_ == null) {
+          return block_;
+        } else {
+          return blockBuilder_.getMessage();
+        }
+      }
+      public Builder setBlock(org.apache.hadoop.hdfs.protocol.proto.HdfsProtos.ExtendedBlockProto value) {
+        if (blockBuilder_ == null) {
+          if (value == null) {
+            throw new NullPointerException();
+          }
+          block_ = value;
+          onChanged();
+        } else {
+          blockBuilder_.setMessage(value);
+        }
+        bitField0_ |= 0x00000001;
+        return this;
+      }
+      public Builder setBlock(
+          org.apache.hadoop.hdfs.protocol.proto.HdfsProtos.ExtendedBlockProto.Builder builderForValue) {
+        if (blockBuilder_ == null) {
+          block_ = builderForValue.build();
+          onChanged();
+        } else {
+          blockBuilder_.setMessage(builderForValue.build());
+        }
+        bitField0_ |= 0x00000001;
+        return this;
+      }
+      public Builder mergeBlock(org.apache.hadoop.hdfs.protocol.proto.HdfsProtos.ExtendedBlockProto value) {
+        if (blockBuilder_ == null) {
+          if (((bitField0_ & 0x00000001) == 0x00000001) &&
+              block_ != org.apache.hadoop.hdfs.protocol.proto.HdfsProtos.ExtendedBlockProto.getDefaultInstance()) {
+            block_ =
+              org.apache.hadoop.hdfs.protocol.proto.HdfsProtos.ExtendedBlockProto.newBuilder(block_).mergeFrom(value).buildPartial();
+          } else {
+            block_ = value;
+          }
+          onChanged();
+        } else {
+          blockBuilder_.mergeFrom(value);
+        }
+        bitField0_ |= 0x00000001;
+        return this;
+      }
+      public Builder clearBlock() {
+        if (blockBuilder_ == null) {
+          block_ = org.apache.hadoop.hdfs.protocol.proto.HdfsProtos.ExtendedBlockProto.getDefaultInstance();
+          onChanged();
+        } else {
+          blockBuilder_.clear();
+        }
+        bitField0_ = (bitField0_ & ~0x00000001);
+        return this;
+      }
+      public org.apache.hadoop.hdfs.protocol.proto.HdfsProtos.ExtendedBlockProto.Builder getBlockBuilder() {
+        bitField0_ |= 0x00000001;
+        onChanged();
+        return getBlockFieldBuilder().getBuilder();
+      }
+      public org.apache.hadoop.hdfs.protocol.proto.HdfsProtos.ExtendedBlockProtoOrBuilder getBlockOrBuilder() {
+        if (blockBuilder_ != null) {
+          return blockBuilder_.getMessageOrBuilder();
+        } else {
+          return block_;
+        }
+      }
+      private com.google.protobuf.SingleFieldBuilder<
+          org.apache.hadoop.hdfs.protocol.proto.HdfsProtos.ExtendedBlockProto, org.apache.hadoop.hdfs.protocol.proto.HdfsProtos.ExtendedBlockProto.Builder, org.apache.hadoop.hdfs.protocol.proto.HdfsProtos.ExtendedBlockProtoOrBuilder> 
+          getBlockFieldBuilder() {
+        if (blockBuilder_ == null) {
+          blockBuilder_ = new com.google.protobuf.SingleFieldBuilder<
+              org.apache.hadoop.hdfs.protocol.proto.HdfsProtos.ExtendedBlockProto, org.apache.hadoop.hdfs.protocol.proto.HdfsProtos.ExtendedBlockProto.Builder, org.apache.hadoop.hdfs.protocol.proto.HdfsProtos.ExtendedBlockProtoOrBuilder>(
+                  block_,
+                  getParentForChildren(),
+                  isClean());
+          block_ = null;
+        }
+        return blockBuilder_;
+      }
+      
+      // optional .BlockTokenIdentifierProto token = 2;
+      private org.apache.hadoop.hdfs.protocol.proto.HdfsProtos.BlockTokenIdentifierProto token_ = org.apache.hadoop.hdfs.protocol.proto.HdfsProtos.BlockTokenIdentifierProto.getDefaultInstance();
+      private com.google.protobuf.SingleFieldBuilder<
+          org.apache.hadoop.hdfs.protocol.proto.HdfsProtos.BlockTokenIdentifierProto, org.apache.hadoop.hdfs.protocol.proto.HdfsProtos.BlockTokenIdentifierProto.Builder, org.apache.hadoop.hdfs.protocol.proto.HdfsProtos.BlockTokenIdentifierProtoOrBuilder> tokenBuilder_;
+      public boolean hasToken() {
+        return ((bitField0_ & 0x00000002) == 0x00000002);
+      }
+      public org.apache.hadoop.hdfs.protocol.proto.HdfsProtos.BlockTokenIdentifierProto getToken() {
+        if (tokenBuilder_ == null) {
+          return token_;
+        } else {
+          return tokenBuilder_.getMessage();
+        }
+      }
+      public Builder setToken(org.apache.hadoop.hdfs.protocol.proto.HdfsProtos.BlockTokenIdentifierProto value) {
+        if (tokenBuilder_ == null) {
+          if (value == null) {
+            throw new NullPointerException();
+          }
+          token_ = value;
+          onChanged();
+        } else {
+          tokenBuilder_.setMessage(value);
+        }
+        bitField0_ |= 0x00000002;
+        return this;
+      }
+      public Builder setToken(
+          org.apache.hadoop.hdfs.protocol.proto.HdfsProtos.BlockTokenIdentifierProto.Builder builderForValue) {
+        if (tokenBuilder_ == null) {
+          token_ = builderForValue.build();
+          onChanged();
+        } else {
+          tokenBuilder_.setMessage(builderForValue.build());
+        }
+        bitField0_ |= 0x00000002;
+        return this;
+      }
+      public Builder mergeToken(org.apache.hadoop.hdfs.protocol.proto.HdfsProtos.BlockTokenIdentifierProto value) {
+        if (tokenBuilder_ == null) {
+          if (((bitField0_ & 0x00000002) == 0x00000002) &&
+              token_ != org.apache.hadoop.hdfs.protocol.proto.HdfsProtos.BlockTokenIdentifierProto.getDefaultInstance()) {
+            token_ =
+              org.apache.hadoop.hdfs.protocol.proto.HdfsProtos.BlockTokenIdentifierProto.newBuilder(token_).mergeFrom(value).buildPartial();
+          } else {
+            token_ = value;
+          }
+          onChanged();
+        } else {
+          tokenBuilder_.mergeFrom(value);
+        }
+        bitField0_ |= 0x00000002;
+        return this;
+      }
+      public Builder clearToken() {
+        if (tokenBuilder_ == null) {
+          token_ = org.apache.hadoop.hdfs.protocol.proto.HdfsProtos.BlockTokenIdentifierProto.getDefaultInstance();
+          onChanged();
+        } else {
+          tokenBuilder_.clear();
+        }
+        bitField0_ = (bitField0_ & ~0x00000002);
+        return this;
+      }
+      public org.apache.hadoop.hdfs.protocol.proto.HdfsProtos.BlockTokenIdentifierProto.Builder getTokenBuilder() {
+        bitField0_ |= 0x00000002;
+        onChanged();
+        return getTokenFieldBuilder().getBuilder();
+      }
+      public org.apache.hadoop.hdfs.protocol.proto.HdfsProtos.BlockTokenIdentifierProtoOrBuilder getTokenOrBuilder() {
+        if (tokenBuilder_ != null) {
+          return tokenBuilder_.getMessageOrBuilder();
+        } else {
+          return token_;
+        }
+      }
+      private com.google.protobuf.SingleFieldBuilder<
+          org.apache.hadoop.hdfs.protocol.proto.HdfsProtos.BlockTokenIdentifierProto, org.apache.hadoop.hdfs.protocol.proto.HdfsProtos.BlockTokenIdentifierProto.Builder, org.apache.hadoop.hdfs.protocol.proto.HdfsProtos.BlockTokenIdentifierProtoOrBuilder> 
+          getTokenFieldBuilder() {
+        if (tokenBuilder_ == null) {
+          tokenBuilder_ = new com.google.protobuf.SingleFieldBuilder<
+              org.apache.hadoop.hdfs.protocol.proto.HdfsProtos.BlockTokenIdentifierProto, org.apache.hadoop.hdfs.protocol.proto.HdfsProtos.BlockTokenIdentifierProto.Builder, org.apache.hadoop.hdfs.protocol.proto.HdfsProtos.BlockTokenIdentifierProtoOrBuilder>(
+                  token_,
+                  getParentForChildren(),
+                  isClean());
+          token_ = null;
+        }
+        return tokenBuilder_;
+      }
+      
+      // @@protoc_insertion_point(builder_scope:BaseHeaderProto)
+    }
+    
+    static {
+      defaultInstance = new BaseHeaderProto(true);
+      defaultInstance.initFields();
+    }
+    
+    // @@protoc_insertion_point(class_scope:BaseHeaderProto)
+  }
+  
+  public interface ClientOperationHeaderProtoOrBuilder
+      extends com.google.protobuf.MessageOrBuilder {
+    
+    // required .BaseHeaderProto baseHeader = 1;
+    boolean hasBaseHeader();
+    org.apache.hadoop.hdfs.protocol.proto.DataTransferProtos.BaseHeaderProto getBaseHeader();
+    org.apache.hadoop.hdfs.protocol.proto.DataTransferProtos.BaseHeaderProtoOrBuilder getBaseHeaderOrBuilder();
+    
+    // required string clientName = 2;
+    boolean hasClientName();
+    String getClientName();
+  }
+  public static final class ClientOperationHeaderProto extends
+      com.google.protobuf.GeneratedMessage
+      implements ClientOperationHeaderProtoOrBuilder {
+    // Use ClientOperationHeaderProto.newBuilder() to construct.
+    private ClientOperationHeaderProto(Builder builder) {
+      super(builder);
+    }
+    private ClientOperationHeaderProto(boolean noInit) {}
+    
+    private static final ClientOperationHeaderProto defaultInstance;
+    public static ClientOperationHeaderProto getDefaultInstance() {
+      return defaultInstance;
+    }
+    
+    public ClientOperationHeaderProto getDefaultInstanceForType() {
+      return defaultInstance;
+    }
+    
+    public static final com.google.protobuf.Descriptors.Descriptor
+        getDescriptor() {
+      return org.apache.hadoop.hdfs.protocol.proto.DataTransferProtos.internal_static_ClientOperationHeaderProto_descriptor;
+    }
+    
+    protected com.google.protobuf.GeneratedMessage.FieldAccessorTable
+        internalGetFieldAccessorTable() {
+      return org.apache.hadoop.hdfs.protocol.proto.DataTransferProtos.internal_static_ClientOperationHeaderProto_fieldAccessorTable;
+    }
+    
+    private int bitField0_;
+    // required .BaseHeaderProto baseHeader = 1;
+    public static final int BASEHEADER_FIELD_NUMBER = 1;
+    private org.apache.hadoop.hdfs.protocol.proto.DataTransferProtos.BaseHeaderProto baseHeader_;
+    public boolean hasBaseHeader() {
+      return ((bitField0_ & 0x00000001) == 0x00000001);
+    }
+    public org.apache.hadoop.hdfs.protocol.proto.DataTransferProtos.BaseHeaderProto getBaseHeader() {
+      return baseHeader_;
+    }
+    public org.apache.hadoop.hdfs.protocol.proto.DataTransferProtos.BaseHeaderProtoOrBuilder getBaseHeaderOrBuilder() {
+      return baseHeader_;
+    }
+    
+    // required string clientName = 2;
+    public static final int CLIENTNAME_FIELD_NUMBER = 2;
+    private java.lang.Object clientName_;
+    public boolean hasClientName() {
+      return ((bitField0_ & 0x00000002) == 0x00000002);
+    }
+    public String getClientName() {
+      java.lang.Object ref = clientName_;
+      if (ref instanceof String) {
+        return (String) ref;
+      } else {
+        com.google.protobuf.ByteString bs = 
+            (com.google.protobuf.ByteString) ref;
+        String s = bs.toStringUtf8();
+        if (com.google.protobuf.Internal.isValidUtf8(bs)) {
+          clientName_ = s;
+        }
+        return s;
+      }
+    }
+    private com.google.protobuf.ByteString getClientNameBytes() {
+      java.lang.Object ref = clientName_;
+      if (ref instanceof String) {
+        com.google.protobuf.ByteString b = 
+            com.google.protobuf.ByteString.copyFromUtf8((String) ref);
+        clientName_ = b;
+        return b;
+      } else {
+        return (com.google.protobuf.ByteString) ref;
+      }
+    }
+    
+    private void initFields() {
+      baseHeader_ = org.apache.hadoop.hdfs.protocol.proto.DataTransferProtos.BaseHeaderProto.getDefaultInstance();
+      clientName_ = "";
+    }
+    private byte memoizedIsInitialized = -1;
+    public final boolean isInitialized() {
+      byte isInitialized = memoizedIsInitialized;
+      if (isInitialized != -1) return isInitialized == 1;
+      
+      if (!hasBaseHeader()) {
+        memoizedIsInitialized = 0;
+        return false;
+      }
+      if (!hasClientName()) {
+        memoizedIsInitialized = 0;
+        return false;
+      }
+      if (!getBaseHeader().isInitialized()) {
+        memoizedIsInitialized = 0;
+        return false;
+      }
+      memoizedIsInitialized = 1;
+      return true;
+    }
+    
+    public void writeTo(com.google.protobuf.CodedOutputStream output)
+                        throws java.io.IOException {
+      getSerializedSize();
+      if (((bitField0_ & 0x00000001) == 0x00000001)) {
+        output.writeMessage(1, baseHeader_);
+      }
+      if (((bitField0_ & 0x00000002) == 0x00000002)) {
+        output.writeBytes(2, getClientNameBytes());
+      }
+      getUnknownFields().writeTo(output);
+    }
+    
+    private int memoizedSerializedSize = -1;
+    public int getSerializedSize() {
+      int size = memoizedSerializedSize;
+      if (size != -1) return size;
+    
+      size = 0;
+      if (((bitField0_ & 0x00000001) == 0x00000001)) {
+        size += com.google.protobuf.CodedOutputStream
+          .computeMessageSize(1, baseHeader_);
+      }
+      if (((bitField0_ & 0x00000002) == 0x00000002)) {
+        size += com.google.protobuf.CodedOutputStream
+          .computeBytesSize(2, getClientNameBytes());
+      }
+      size += getUnknownFields().getSerializedSize();
+      memoizedSerializedSize = size;
+      return size;
+    }
+    
+    private static final long serialVersionUID = 0L;
+    @java.lang.Override
+    protected java.lang.Object writeReplace()
+        throws java.io.ObjectStreamException {
+      return super.writeReplace();
+    }
+    
+    @java.lang.Override
+    public boolean equals(final java.lang.Object obj) {
+      if (obj == this) {
+       return true;
+      }
+      if (!(obj instanceof org.apache.hadoop.hdfs.protocol.proto.DataTransferProtos.ClientOperationHeaderProto)) {
+        return super.equals(obj);
+      }
+      org.apache.hadoop.hdfs.protocol.proto.DataTransferProtos.ClientOperationHeaderProto other = (org.apache.hadoop.hdfs.protocol.proto.DataTransferProtos.ClientOperationHeaderProto) obj;
+      
+      boolean result = true;
+      result = result && (hasBaseHeader() == other.hasBaseHeader());
+      if (hasBaseHeader()) {
+        result = result && getBaseHeader()
+            .equals(other.getBaseHeader());
+      }
+      result = result && (hasClientName() == other.hasClientName());
+      if (hasClientName()) {
+        result = result && getClientName()
+            .equals(other.getClientName());
+      }
+      result = result &&
+          getUnknownFields().equals(other.getUnknownFields());
+      return result;
+    }
+    
+    @java.lang.Override
+    public int hashCode() {
+      int hash = 41;
+      hash = (19 * hash) + getDescriptorForType().hashCode();
+      if (hasBaseHeader()) {
+        hash = (37 * hash) + BASEHEADER_FIELD_NUMBER;
+        hash = (53 * hash) + getBaseHeader().hashCode();
+      }
+      if (hasClientName()) {
+        hash = (37 * hash) + CLIENTNAME_FIELD_NUMBER;
+        hash = (53 * hash) + getClientName().hashCode();
+      }
+      hash = (29 * hash) + getUnknownFields().hashCode();
+      return hash;
+    }
+    
+    public static org.apache.hadoop.hdfs.protocol.proto.DataTransferProtos.ClientOperationHeaderProto parseFrom(
+        com.google.protobuf.ByteString data)
+        throws com.google.protobuf.InvalidProtocolBufferException {
+      return newBuilder().mergeFrom(data).buildParsed();
+    }
+    public static org.apache.hadoop.hdfs.protocol.proto.DataTransferProtos.ClientOperationHeaderProto parseFrom(
+        com.google.protobuf.ByteString data,
+        com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+        throws com.google.protobuf.InvalidProtocolBufferException {
+      return newBuilder().mergeFrom(data, extensionRegistry)
+               .buildParsed();
+    }
+    public static org.apache.hadoop.hdfs.protocol.proto.DataTransferProtos.ClientOperationHeaderProto parseFrom(byte[] data)
+        throws com.google.protobuf.InvalidProtocolBufferException {
+      return newBuilder().mergeFrom(data).buildParsed();
+    }
+    public static org.apache.hadoop.hdfs.protocol.proto.DataTransferProtos.ClientOperationHeaderProto parseFrom(
+        byte[] data,
+        com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+        throws com.google.protobuf.InvalidProtocolBufferException {
+      return newBuilder().mergeFrom(data, extensionRegistry)
+               .buildParsed();
+    }
+    public static org.apache.hadoop.hdfs.protocol.proto.DataTransferProtos.ClientOperationHeaderProto parseFrom(java.io.InputStream input)
+        throws java.io.IOException {
+      return newBuilder().mergeFrom(input).buildParsed();
+    }
+    public static org.apache.hadoop.hdfs.protocol.proto.DataTransferProtos.ClientOperationHeaderProto parseFrom(
+        java.io.InputStream input,
+        com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+        throws java.io.IOException {
+      return newBuilder().mergeFrom(input, extensionRegistry)
+               .buildParsed();
+    }
+    public static org.apache.hadoop.hdfs.protocol.proto.DataTransferProtos.ClientOperationHeaderProto parseDelimitedFrom(java.io.InputStream input)
+        throws java.io.IOException {
+      Builder builder = newBuilder();
+      if (builder.mergeDelimitedFrom(input)) {
+        return builder.buildParsed();
+      } else {
+        return null;
+      }
+    }
+    public static org.apache.hadoop.hdfs.protocol.proto.DataTransferProtos.ClientOperationHeaderProto parseDelimitedFrom(
+        java.io.InputStream input,
+        com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+        throws java.io.IOException {
+      Builder builder = newBuilder();
+      if (builder.mergeDelimitedFrom(input, extensionRegistry)) {
+        return builder.buildParsed();
+      } else {
+        return null;
+      }
+    }
+    public static org.apache.hadoop.hdfs.protocol.proto.DataTransferProtos.ClientOperationHeaderProto parseFrom(
+        com.google.protobuf.CodedInputStream input)
+        throws java.io.IOException {
+      return newBuilder().mergeFrom(input).buildParsed();
+    }
+    public static org.apache.hadoop.hdfs.protocol.proto.DataTransferProtos.ClientOperationHeaderProto parseFrom(
+        com.google.protobuf.CodedInputStream input,
+        com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+        throws java.io.IOException {
+      return newBuilder().mergeFrom(input, extensionRegistry)
+               .buildParsed();
+    }
+    
+    public static Builder newBuilder() { return Builder.create(); }
+    public Builder newBuilderForType() { return newBuilder(); }
+    public static Builder newBuilder(org.apache.hadoop.hdfs.protocol.proto.DataTransferProtos.ClientOperationHeaderProto prototype) {
+      return newBuilder().mergeFrom(prototype);
+    }
+    public Builder toBuilder() { return newBuilder(this); }
+    
+    @java.lang.Override
+    protected Builder newBuilderForType(
+        com.google.protobuf.GeneratedMessage.BuilderParent parent) {
+      Builder builder = new Builder(parent);
+      return builder;
+    }
+    public static final class Builder extends
+        com.google.protobuf.GeneratedMessage.Builder<Builder>
+       implements org.apache.hadoop.hdfs.protocol.proto.DataTransferProtos.ClientOperationHeaderProtoOrBuilder {
+      public static final com.google.protobuf.Descriptors.Descriptor
+          getDescriptor() {
+        return org.apache.hadoop.hdfs.protocol.proto.DataTransferProtos.internal_static_ClientOperationHeaderProto_descriptor;
+      }
+      
+      protected com.google.protobuf.GeneratedMessage.FieldAccessorTable
+          internalGetFieldAccessorTable() {
+        return org.apache.hadoop.hdfs.protocol.proto.DataTransferProtos.internal_static_ClientOperationHeaderProto_fieldAccessorTable;
+      }
+      
+      // Construct using org.apache.hadoop.hdfs.protocol.proto.DataTransferProtos.ClientOperationHeaderProto.newBuilder()
+      private Builder() {
+        maybeForceBuilderInitialization();
+      }
+      
+      private Builder(BuilderParent parent) {
+        super(parent);
+        maybeForceBuilderInitialization();
+      }
+      private void maybeForceBuilderInitialization() {
+        if (com.google.protobuf.GeneratedMessage.alwaysUseFieldBuilders) {
+          getBaseHeaderFieldBuilder();
+        }
+      }
+      private static Builder create() {
+        return new Builder();
+      }
+      
+      public Builder clear() {
+        super.clear();
+        if (baseHeaderBuilder_ == null) {
+          baseHeader_ = org.apache.hadoop.hdfs.protocol.proto.DataTransferProtos.BaseHeaderProto.getDefaultInstance();
+        } else {
+          baseHeaderBuilder_.clear();
+        }
+        bitField0_ = (bitField0_ & ~0x00000001);
+        clientName_ = "";
+        bitField0_ = (bitField0_ & ~0x00000002);
+        return this;
+      }
+      
+      public Builder clone() {
+        return create().mergeFrom(buildPartial());
+      }
+      
+      public com.google.protobuf.Descriptors.Descriptor
+          getDescriptorForType() {
+        return org.apache.hadoop.hdfs.protocol.proto.DataTransferProtos.ClientOperationHeaderProto.getDescriptor();
+      }
+      
+      public org.apache.hadoop.hdfs.protocol.proto.DataTransferProtos.ClientOperationHeaderProto getDefaultInstanceForType() {
+        return org.apache.hadoop.hdfs.protocol.proto.DataTransferProtos.ClientOperationHeaderProto.getDefaultInstance();
+      }
+      
+      public org.apache.hadoop.hdfs.protocol.proto.DataTransferProtos.ClientOperationHeaderProto build() {
+        org.apache.hadoop.hdfs.protocol.proto.DataTransferProtos.ClientOperationHeaderProto result = buildPartial();
+        if (!result.isInitialized()) {
+          throw newUninitializedMessageException(result);
+        }
+        return result;
+      }
+      
+      private org.apache.hadoop.hdfs.protocol.proto.DataTransferProtos.ClientOperationHeaderProto buildParsed()
+          throws com.google.protobuf.InvalidProtocolBufferException {
+        org.apache.hadoop.hdfs.protocol.proto.DataTransferProtos.ClientOperationHeaderProto result = buildPartial();
+        if (!result.isInitialized()) {
+          throw newUninitializedMessageException(
+            result).asInvalidProtocolBufferException();
+        }
+        return result;
+      }
+      
+      public org.apache.hadoop.hdfs.protocol.proto.DataTransferProtos.ClientOperationHeaderProto buildPartial() {
+        org.apache.hadoop.hdfs.protocol.proto.DataTransferProtos.ClientOperationHeaderProto result = new org.apache.hadoop.hdfs.protocol.proto.DataTransferProtos.ClientOperationHeaderProto(this);
+        int from_bitField0_ = bitField0_;
+        int to_bitField0_ = 0;
+        if (((from_bitField0_ & 0x00000001) == 0x00000001)) {
+          to_bitField0_ |= 0x00000001;
+        }
+        if (baseHeaderBuilder_ == null) {
+          result.baseHeader_ = baseHeader_;
+        } else {
+          result.baseHeader_ = baseHeaderBuilder_.build();
+        }
+        if (((from_bitField0_ & 0x00000002) == 0x00000002)) {
+          to_bitField0_ |= 0x00000002;
+        }
+        result.clientName_ = clientName_;
+        result.bitField0_ = to_bitField0_;
+        onBuilt();
+        return result;
+      }
+      
+      public Builder mergeFrom(com.google.protobuf.Message other) {
+        if (other instanceof org.apache.hadoop.hdfs.protocol.proto.DataTransferProtos.ClientOperationHeaderProto) {
+          return mergeFrom((org.apache.hadoop.hdfs.protocol.proto.DataTransferProtos.ClientOperationHeaderProto)other);
+        } else {
+          super.mergeFrom(other);
+          return this;
+        }
+      }
+      
+      public Builder mergeFrom(org.apache.hadoop.hdfs.protocol.proto.DataTransferProtos.ClientOperationHeaderProto other) {
+        if (other == org.apache.hadoop.hdfs.protocol.proto.DataTransferProtos.ClientOperationHeaderProto.getDefaultInstance()) return this;
+        if (other.hasBaseHeader()) {
+          mergeBaseHeader(other.getBaseHeader());
+        }
+        if (other.hasClientName()) {
+          setClientName(other.getClientName());
+        }
+        this.mergeUnknownFields(other.getUnknownFields());
+        return this;
+      }
+      
+      public final boolean isInitialized() {
+        if (!hasBaseHeader()) {
+          
+          return false;
+        }
+        if (!hasClientName()) {
+          
+          return false;
+        }
+        if (!getBaseHeader().isInitialized()) {
+          
+          return false;
+        }
+        return true;
+      }
+      
+      public Builder mergeFrom(
+          com.google.protobuf.CodedInputStream input,
+          com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+          throws java.io.IOException {
+        com.google.protobuf.UnknownFieldSet.Builder unknownFields =
+          com.google.protobuf.UnknownFieldSet.newBuilder(
+            this.getUnknownFields());
+        while (true) {
+          int tag = input.readTag();
+          switch (tag) {
+            case 0:
+              this.setUnknownFields(unknownFields.build());
+              onChanged();
+              return this;
+            default: {
+              if (!parseUnknownField(input, unknownFields,
+                                     extensionRegistry, tag)) {
+                this.setUnknownFields(unknownFields.build());
+                onChanged();
+                return this;
+              }
+              break;
+            }
+            case 10: {
+              org.apache.hadoop.hdfs.protocol.proto.DataTransferProtos.BaseHeaderProto.Builder subBuilder = org.apache.hadoop.hdfs.protocol.proto.DataTransferProtos.BaseHeaderProto.newBuilder();
+              if (hasBaseHeader()) {
+                subBuilder.mergeFrom(getBaseHeader());
+              }
+              input.readMessage(subBuilder, extensionRegistry);
+              setBaseHeader(subBuilder.buildPartial());
+              break;
+            }
+            case 18: {
+              bitField0_ |= 0x00000002;
+              clientName_ = input.readBytes();
+              break;
+            }
+          }
+        }
+      }
+      
+      private int bitField0_;
+      
+      // required .BaseHeaderProto baseHeader = 1;
+      private org.apache.hadoop.hdfs.protocol.proto.DataTransferProtos.BaseHeaderProto baseHeader_ = org.apache.hadoop.hdfs.protocol.proto.DataTransferProtos.BaseHeaderProto.getDefaultInstance();
+      private com.google.protobuf.SingleFieldBuilder<
+          org.apache.hadoop.hdfs.protocol.proto.DataTransferProtos.BaseHeaderProto, org.apache.hadoop.hdfs.protocol.proto.DataTransferProtos.BaseHeaderProto.Builder, org.apache.hadoop.hdfs.protocol.proto.DataTransferProtos.BaseHeaderProtoOrBuilder> baseHeaderBuilder_;
+      public boolean hasBaseHeader() {
+        return ((bitField0_ & 0x00000001) == 0x00000001);
+      }
+      public org.apache.hadoop.hdfs.protocol.proto.DataTransferProtos.BaseHeaderProto getBaseHeader() {
+        if (baseHeaderBuilder_ == null) {
+          return baseHeader_;
+        } else {
+          return baseHeaderBuilder_.getMessage();
+        }
+      }
+      public Builder setBaseHeader(org.apache.hadoop.hdfs.protocol.proto.DataTransferProtos.BaseHeaderProto value) {
+        if (baseHeaderBuilder_ == null) {
+          if (value == null) {
+            throw new NullPointerException();
+          }
+          baseHeader_ = value;
+          onChanged();
+        } else {
+          baseHeaderBuilder_.setMessage(value);
+        }
+        bitField0_ |= 0x00000001;
+        return this;
+      }
+      public Builder setBaseHeader(
+          org.apache.hadoop.hdfs.protocol.proto.DataTransferProtos.BaseHeaderProto.Builder builderForValue) {
+        if (baseHeaderBuilder_ == null) {
+          baseHeader_ = builderForValue.build();
+          onChanged();
+        } else {
+          baseHeaderBuilder_.setMessage(builderForValue.build());
+        }
+        bitField0_ |= 0x00000001;
+        return this;
+      }
+      public Builder mergeBaseHeader(org.apache.hadoop.hdfs.protocol.proto.DataTransferProtos.BaseHeaderProto value) {
+        if (baseHeaderBuilder_ == null) {
+          if (((bitField0_ & 0x00000001) == 0x00000001) &&
+              baseHeader_ != org.apache.hadoop.hdfs.protocol.proto.DataTransferProtos.BaseHeaderProto.getDefaultInstance()) {
+            baseHeader_ =
+              org.apache.hadoop.hdfs.protocol.proto.DataTransferProtos.BaseHeaderProto.newBuilder(baseHeader_).mergeFrom(value).buildPartial();
+          } else {
+            baseHeader_ = value;
+          }
+          onChanged();
+        } else {
+          baseHeaderBuilder_.mergeFrom(value);
+        }
+        bitField0_ |= 0x00000001;
+        return this;
+      }
+      public Builder clearBaseHeader() {
+        if (baseHeaderBuilder_ == null) {
+          baseHeader_ = org.apache.hadoop.hdfs.protocol.proto.DataTransferProtos.BaseHeaderProto.getDefaultInstance();
+          onChanged();
+        } else {
+          baseHeaderBuilder_.clear();
+        }
+        bitField0_ = (bitField0_ & ~0x00000001);
+        return this;
+      }
+      public org.apache.hadoop.hdfs.protocol.proto.DataTransferProtos.BaseHeaderProto.Builder getBaseHeaderBuilder() {
+        bitField0_ |= 0x00000001;
+        onChanged();
+        return getBaseHeaderFieldBuilder().getBuilder();
+      }
+      public org.apache.hadoop.hdfs.protocol.proto.DataTransferProtos.BaseHeaderProtoOrBuilder getBaseHeaderOrBuilder() {
+        if (baseHeaderBuilder_ != null) {
+          return baseHeaderBuilder_.getMessageOrBuilder();
+        } else {
+          return baseHeader_;
+        }
+      }
+      private com.google.protobuf.SingleFieldBuilder<
+          org.apache.hadoop.hdfs.protocol.proto.DataTransferProtos.BaseHeaderProto, org.apache.hadoop.hdfs.protocol.proto.DataTransferProtos.BaseHeaderProto.Builder, org.apache.hadoop.hdfs.protocol.proto.DataTransferProtos.BaseHeaderProtoOrBuilder> 
+          getBaseHeaderFieldBuilder() {
+        if (baseHeaderBuilder_ == null) {
+          baseHeaderBuilder_ = new com.google.protobuf.SingleFieldBuilder<
+              org.apache.hadoop.hdfs.protocol.proto.DataTransferProtos.BaseHeaderProto, org.apache.hadoop.hdfs.protocol.proto.DataTransferProtos.BaseHeaderProto.Builder, org.apache.hadoop.hdfs.protocol.proto.DataTransferProtos.BaseHeaderProtoOrBuilder>(
+                  baseHeader_,
+                  getParentForChildren(),
+                  isClean());
+          baseHeader_ = null;
+        }
+        return baseHeaderBuilder_;
+      }
+      
+      // required string clientName = 2;
+      private java.lang.Object clientName_ = "";
+      public boolean hasClientName() {
+        return ((bitField0_ & 0x00000002) == 0x00000002);
+      }
+      public String getClientName() {
+        java.lang.Object ref = clientName_;
+        if (!(ref instanceof String)) {
+          String s = ((com.google.protobuf.ByteString) ref).toStringUtf8();
+          clientName_ = s;
+          return s;
+        } else {
+          return (String) ref;
+        }
+      }
+      public Builder setClientName(String value) {
+        if (value == null) {
+    throw new NullPointerException();
+  }
+  bitField0_ |= 0x00000002;
+        clientName_ = value;
+        onChanged();
+        return this;
+      }
+      public Builder clearClientName() {
+        bitField0_ = (bitField0_ & ~0x00000002);
+        clientName_ = getDefaultInstance().getClientName();
+        onChanged();
+        return this;
+      }
+      void setClientName(com.google.protobuf.ByteString value) {
+        bitField0_ |= 0x00000002;
+        clientName_ = value;
+        onChanged();
+      }
+      
+      // @@protoc_insertion_point(builder_scope:ClientOperationHeaderProto)
+    }
+    
+    static {
+      defaultInstance = new ClientOperationHeaderProto(true);
+      defaultInstance.initFields();
+    }
+    
+    // @@protoc_insertion_point(class_scope:ClientOperationHeaderProto)
+  }
+  
+  public interface OpReadBlockProtoOrBuilder
+      extends com.google.protobuf.MessageOrBuilder {
+    
+    // required .ClientOperationHeaderProto header = 1;
+    boolean hasHeader();
+    org.apache.hadoop.hdfs.protocol.proto.DataTransferProtos.ClientOperationHeaderProto getHeader();
+    org.apache.hadoop.hdfs.protocol.proto.DataTransferProtos.ClientOperationHeaderProtoOrBuilder getHeaderOrBuilder();
+    
+    // required uint64 offset = 2;
+    boolean hasOffset();
+    long getOffset();
+    
+    // required uint64 len = 3;
+    boolean hasLen();
+    long getLen();
+  }
+  public static final class OpReadBlockProto extends
+      com.google.protobuf.GeneratedMessage
+      implements OpReadBlockProtoOrBuilder {
+    // Use OpReadBlockProto.newBuilder() to construct.
+    private OpReadBlockProto(Builder builder) {
+      super(builder);
+    }
+    private OpReadBlockProto(boolean noInit) {}
+    
+    private static final OpReadBlockProto defaultInstance;
+    public static OpReadBlockProto getDefaultInstance() {
+      return defaultInstance;
+    }
+    
+    public OpReadBlockProto getDefaultInstanceForType() {
+      return defaultInstance;
+    }
+    
+    public static final com.google.protobuf.Descriptors.Descriptor
+        getDescriptor() {
+      return org.apache.hadoop.hdfs.protocol.proto.DataTransferProtos.internal_static_OpReadBlockProto_descriptor;
+    }
+    
+    protected com.google.protobuf.GeneratedMessage.FieldAccessorTable
+        internalGetFieldAccessorTable() {
+      return org.apache.hadoop.hdfs.protocol.proto.DataTransferProtos.internal_static_OpReadBlockProto_fieldAccessorTable;
+    }
+    
+    private int bitField0_;
+    // required .ClientOperationHeaderProto header = 1;
+    public static final int HEADER_FIELD_NUMBER = 1;
+    private org.apache.hadoop.hdfs.protocol.proto.DataTransferProtos.ClientOperationHeaderProto header_;
+    public boolean hasHeader() {
+      return ((bitField0_ & 0x00000001) == 0x00000001);
+    }
+    public org.apache.hadoop.hdfs.protocol.proto.DataTransferProtos.ClientOperationHeaderProto getHeader() {
+      return header_;
+    }
+    public org.apache.hadoop.hdfs.protocol.proto.DataTransferProtos.ClientOperationHeaderProtoOrBuilder getHeaderOrBuilder() {
+      return header_;
+    }
+    
+    // required uint64 offset = 2;
+    public static final int OFFSET_FIELD_NUMBER = 2;
+    private long offset_;
+    public boolean hasOffset() {
+      return ((bitField0_ & 0x00000002) == 0x00000002);
+    }
+    public long getOffset() {
+      return offset_;
+    }
+    
+    // required uint64 len = 3;
+    public static final int LEN_FIELD_NUMBER = 3;
+    private long len_;
+    public boolean hasLen() {
+      return ((bitField0_ & 0x00000004) == 0x00000004);
+    }
+    public long getLen() {
+      return len_;
+    }
+    
+    private void initFields() {
+      header_ = org.apache.hadoop.hdfs.protocol.proto.DataTransferProtos.ClientOperationHeaderProto.getDefaultInstance();
+      offset_ = 0L;
+      len_ = 0L;
+    }
+    private byte memoizedIsInitialized = -1;
+    public final boolean isInitialized() {
+      byte isInitialized = memoizedIsInitialized;
+      if (isInitialized != -1) return isInitialized == 1;
+      
+      if (!hasHeader()) {
+        memoizedIsInitialized = 0;
+        return false;
+      }
+      if (!hasOffset()) {
+        memoizedIsInitialized = 0;
+        return false;
+      }
+      if (!hasLen()) {
+        memoizedIsInitialized = 0;
+        return false;
+      }
+      if (!getHeader().isInitialized()) {
+        memoizedIsInitialized = 0;
+        return false;
+      }
+      memoizedIsInitialized = 1;
+      return true;
+    }
+    
+    public void writeTo(com.google.protobuf.CodedOutputStream output)
+                        throws java.io.IOException {
+      getSerializedSize();
+      if (((bitField0_ & 0x00000001) == 0x00000001)) {
+        output.writeMessage(1, header_);
+      }
+      if (((bitField0_ & 0x00000002) == 0x00000002)) {
+        output.writeUInt64(2, offset_);
+      }
+      if (((bitField0_ & 0x00000004) == 0x00000004)) {
+        output.writeUInt64(3, len_);
+      }
+      getUnknownFields().writeTo(output);
+    }
+    
+    private int memoizedSerializedSize = -1;
+    public int getSerializedSize() {
+      int size = memoizedSerializedSize;
+      if (size != -1) return size;
+    
+      size = 0;
+      if (((bitField0_ & 0x00000001) == 0x00000001)) {
+        size += com.google.protobuf.CodedOutputStream
+          .computeMessageSize(1, header_);
+      }
+      if (((bitField0_ & 0x00000002) == 0x00000002)) {
+        size += com.google.protobuf.CodedOutputStream
+          .computeUInt64Size(2, offset_);
+      }
+      if (((bitField0_ & 0x00000004) == 0x00000004)) {
+        size += com.google.protobuf.CodedOutputStream
+          .computeUInt64Size(3, len_);
+      }
+      size += getUnknownFields().getSerializedSize();
+      memoizedSerializedSize = size;
+      return size;
+    }
+    
+    private static final long serialVersionUID = 0L;
+    @java.lang.Override
+    protected java.lang.Object writeReplace()
+        throws java.io.ObjectStreamException {
+      return super.writeReplace();
+    }
+    
+    @java.lang.Override
+    public boolean equals(final java.lang.Object obj) {
+      if (obj == this) {
+       return true;
+      }
+      if (!(obj instanceof org.apache.hadoop.hdfs.protocol.proto.DataTransferProtos.OpReadBlockProto)) {
+        return super.equals(obj);
+      }
+      org.apache.hadoop.hdfs.protocol.proto.DataTransferProtos.OpReadBlockProto other = (org.apache.hadoop.hdfs.protocol.proto.DataTransferProtos.OpReadBlockProto) obj;
+      
+      boolean result = true;
+      result = result && (hasHeader() == other.hasHeader());
+      if (hasHeader()) {
+        result = result && getHeader()
+            .equals(other.getHeader());
+      }
+      result = result && (hasOffset() == other.hasOffset());
+      if (hasOffset()) {
+        result = result && (getOffset()
+            == other.getOffset());
+      }
+      result = result && (hasLen() == other.hasLen());
+      if (hasLen()) {
+        result = result && (getLen()
+            == other.getLen());
+      }
+      result = result &&
+          getUnknownFields().equals(other.getUnknownFields());
+      return result;
+    }
+    
+    @java.lang.Override
+    public int hashCode() {
+      int hash = 41;
+      hash = (19 * hash) + getDescriptorForType().hashCode();
+      if (hasHeader()) {
+        hash = (37 * hash) + HEADER_FIELD_NUMBER;
+        hash = (53 * hash) + getHeader().hashCode();
+      }
+      if (hasOffset()) {
+        hash = (37 * hash) + OFFSET_FIELD_NUMBER;
+        hash = (53 * hash) + hashLong(getOffset());
+      }
+      if (hasLen()) {
+        hash = (37 * hash) + LEN_FIELD_NUMBER;
+        hash = (53 * hash) + hashLong(getLen());
+      }
+      hash = (29 * hash) + getUnknownFields().hashCode();
+      return hash;
+    }
+    
+    public static org.apache.hadoop.hdfs.protocol.proto.DataTransferProtos.OpReadBlockProto parseFrom(
+        com.google.protobuf.ByteString data)
+        throws com.google.protobuf.InvalidProtocolBufferException {
+      return newBuilder().mergeFrom(data).buildParsed();
+    }
+    public static org.apache.hadoop.hdfs.protocol.proto.DataTransferProtos.OpReadBlockProto parseFrom(
+        com.google.protobuf.ByteString data,
+        com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+        throws com.google.protobuf.InvalidProtocolBufferException {
+      return newBuilder().mergeFrom(data, extensionRegistry)
+               .buildParsed();
+    }
+    public static org.apache.hadoop.hdfs.protocol.proto.DataTransferProtos.OpReadBlockProto parseFrom(byte[] data)
+        throws com.google.protobuf.InvalidProtocolBufferException {
+      return newBuilder().mergeFrom(data).buildParsed();
+    }
+    public static org.apache.hadoop.hdfs.protocol.proto.DataTransferProtos.OpReadBlockProto parseFrom(
+        byte[] data,
+        com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+        throws com.google.protobuf.InvalidProtocolBufferException {
+      return newBuilder().mergeFrom(data, extensionRegistry)
+               .buildParsed();
+    }
+    public static org.apache.hadoop.hdfs.protocol.proto.DataTransferProtos.OpReadBlockProto parseFrom(java.io.InputStream input)
+        throws java.io.IOException {
+      return newBuilder().mergeFrom(input).buildParsed();
+    }
+    public static org.apache.hadoop.hdfs.protocol.proto.DataTransferProtos.OpReadBlockProto parseFrom(
+        java.io.InputStream input,
+        com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+        throws java.io.IOException {
+      return newBuilder().mergeFrom(input, extensionRegistry)
+               .buildParsed();
+    }
+    public static org.apache.hadoop.hdfs.protocol.proto.DataTransferProtos.OpReadBlockProto parseDelimitedFrom(java.io.InputStream input)
+        throws java.io.IOException {
+      Builder builder = newBuilder();
+      if (builder.mergeDelimitedFrom(input)) {
+        return builder.buildParsed();
+      } else {
+        return null;
+      }
+    }
+    public static org.apache.hadoop.hdfs.protocol.proto.DataTransferProtos.OpReadBlockProto parseDelimitedFrom(
+        java.io.InputStream input,
+        com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+        throws java.io.IOException {
+      Builder builder = newBuilder();
+      if (builder.mergeDelimitedFrom(input, extensionRegistry)) {
+        return builder.buildParsed();
+      } else {
+        return null;
+      }
+    }
+    public static org.apache.hadoop.hdfs.protocol.proto.DataTransferProtos.OpReadBlockProto parseFrom(
+        com.google.protobuf.CodedInputStream input)
+        throws java.io.IOException {
+      return newBuilder().mergeFrom(input).buildParsed();
+    }
+    public static org.apache.hadoop.hdfs.protocol.proto.DataTransferProtos.OpReadBlockProto parseFrom(
+        com.google.protobuf.CodedInputStream input,
+        com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+        throws java.io.IOException {
+      return newBuilder().mergeFrom(input, extensionRegistry)
+               .buildParsed();
+    }
+    
+    public static Builder newBuilder() { return Builder.create(); }
+    public Builder newBuilderForType() { return newBuilder(); }
+    public static Builder newBuilder(org.apache.hadoop.hdfs.protocol.proto.DataTransferProtos.OpReadBlockProto prototype) {
+      return newBuilder().mergeFrom(prototype);
+    }
+    public Builder toBuilder() { return newBuilder(this); }
+    
+    @java.lang.Override
+    protected Builder newBuilderForType(
+        com.google.protobuf.GeneratedMessage.BuilderParent parent) {
+      Builder builder = new Builder(parent);
+      return builder;
+    }
+    public static final class Builder extends
+        com.google.protobuf.GeneratedMessage.Builder<Builder>
+       implements org.apache.hadoop.hdfs.protocol.proto.DataTransferProtos.OpReadBlockProtoOrBuilder {
+      public static final com.google.protobuf.Descriptors.Descriptor
+          getDescriptor() {
+        return org.apache.hadoop.hdfs.protocol.proto.DataTransferProtos.internal_static_OpReadBlockProto_descriptor;
+      }
+      
+      protected com.google.protobuf.GeneratedMessage.FieldAccessorTable
+          internalGetFieldAccessorTable() {
+        return org.apache.hadoop.hdfs.protocol.proto.DataTransferProtos.internal_static_OpReadBlockProto_fieldAccessorTable;
+      }
+      
+      // Construct using org.apache.hadoop.hdfs.protocol.proto.DataTransferProtos.OpReadBlockProto.newBuilder()
+      private Builder() {
+        maybeForceBuilderInitialization();
+      }
+      
+      private Builder(BuilderParent parent) {
+        super(parent);
+        maybeForceBuilderInitialization();
+      }
+      private void maybeForceBuilderInitialization() {
+        if (com.google.protobuf.GeneratedMessage.alwaysUseFieldBuilders) {
+          getHeaderFieldBuilder();
+        }
+      }
+      private static Builder create() {
+        return new Builder();
+      }
+      
+      public Builder clear() {
+        super.clear();
+        if (headerBuilder_ == null) {
+          header_ = org.apache.hadoop.hdfs.protocol.proto.DataTransferProtos.ClientOperationHeaderProto.getDefaultInstance();
+        } else {
+          headerBuilder_.clear();
+        }
+        bitField0_ = (bitField0_ & ~0x00000001);
+        offset_ = 0L;
+        bitField0_ = (bitField0_ & ~0x00000002);
+        len_ = 0L;
+        bitField0_ = (bitField0_ & ~0x00000004);
+        return this;
+      }
+      
+      public Builder clone() {
+        return create().mergeFrom(buildPartial());
+      }
+      
+      public com.google.protobuf.Descriptors.Descriptor
+          getDescriptorForType() {
+        return org.apache.hadoop.hdfs.protocol.proto.DataTransferProtos.OpReadBlockProto.getDescriptor();
+      }
+      
+      public org.apache.hadoop.hdfs.protocol.proto.DataTransferProtos.OpReadBlockProto getDefaultInstanceForType() {
+        return org.apache.hadoop.hdfs.protocol.proto.DataTransferProtos.OpReadBlockProto.getDefaultInstance();
+      }
+      
+      public org.apache.hadoop.hdfs.protocol.proto.DataTransferProtos.OpReadBlockProto build() {
+        org.apache.hadoop.hdfs.protocol.proto.DataTransferProtos.OpReadBlockProto result = buildPartial();
+        if (!result.isInitialized()) {
+          throw newUninitializedMessageException(result);
+        }
+        return result;
+      }
+      
+      private org.apache.hadoop.hdfs.protocol.proto.DataTransferProtos.OpReadBlockProto buildParsed()
+          throws com.google.protobuf.InvalidProtocolBufferException {
+        org.apache.hadoop.hdfs.protocol.proto.DataTransferProtos.OpReadBlockProto result = buildPartial();
+        if (!result.isInitialized()) {
+          throw newUninitializedMessageException(
+            result).asInvalidProtocolBufferException();
+        }
+        return result;
+      }
+      
+      public org.apache.hadoop.hdfs.protocol.proto.DataTransferProtos.OpReadBlockProto buildPartial() {
+        org.apache.hadoop.hdfs.protocol.proto.DataTransferProtos.OpReadBlockProto result = new org.apache.hadoop.hdfs.protocol.proto.DataTransferProtos.OpReadBlockProto(this);
+        int from_bitField0_ = bitField0_;
+        int to_bitField0_ = 0;
+        if (((from_bitField0_ & 0x00000001) == 0x00000001)) {
+          to_bitField0_ |= 0x00000001;
+        }
+        if (headerBuilder_ == null) {
+          result.header_ = header_;
+        } else {
+          result.header_ = headerBuilder_.build();
+        }
+        if (((from_bitField0_ & 0x00000002) == 0x00000002)) {
+          to_bitField0_ |= 0x00000002;
+        }
+        result.offset_ = offset_;
+        if (((from_bitField0_ & 0x00000004) == 0x00000004)) {
+          to_bitField0_ |= 0x00000004;
+        }
+        result.len_ = len_;
+        result.bitField0_ = to_bitField0_;
+        onBuilt();
+        return result;
+      }
+      
+      public Builder mergeFrom(com.google.protobuf.Message other) {
+        if (other instanceof org.apache.hadoop.hdfs.protocol.proto.DataTransferProtos.OpReadBlockProto) {
+          return mergeFrom((org.apache.hadoop.hdfs.protocol.proto.DataTransferProtos.OpReadBlockProto)other);
+        } else {
+          super.mergeFrom(other);
+          return this;
+        }
+      }
+      
+      public Builder mergeFrom(org.apache.hadoop.hdfs.protocol.proto.DataTransferProtos.OpReadBlockProto other) {
+        if (other == org.apache.hadoop.hdfs.protocol.proto.DataTransferProtos.OpReadBlockProto.getDefaultInstance()) return this;
+        if (other.hasHeader()) {
+          mergeHeader(other.getHeader());
+        }
+        if (other.hasOffset()) {
+          setOffset(other.getOffset());
+        }
+        if (other.hasLen()) {
+          setLen(other.getLen());
+        }
+        this.mergeUnknownFields(other.getUnknownFields());
+        return this;
+      }
+      
+      public final boolean isInitialized() {
+        if (!hasHeader()) {
+          
+          return false;
+        }
+        if (!hasOffset()) {
+          
+          return false;
+        }
+        if (!hasLen()) {
+          
+          return false;
+        }
+        if (!getHeader().isInitialized()) {
+          
+          return false;
+        }
+        return true;
+      }
+      
+      public Builder mergeFrom(
+          com.google.protobuf.CodedInputStream input,
+          com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+          throws java.io.IOException {
+        com.google.protobuf.UnknownFieldSet.Builder unknownFields =
+          com.google.protobuf.UnknownFieldSet.newBuilder(
+            this.getUnknownFields());
+        while (true) {
+          int tag = input.readTag();
+          switch (tag) {
+            case 0:
+              this.setUnknownFields(unknownFields.build());
+              onChanged();
+              return this;
+            default: {
+              if (!parseUnknownField(input, unknownFields,
+                                     extensionRegistry, tag)) {
+                this.setUnknownFields(unknownFields.build());
+                onChanged();
+                return this;
+              }
+              break;
+            }
+            case 10: {
+              org.apache.hadoop.hdfs.protocol.proto.DataTransferProtos.ClientOperationHeaderProto.Builder subBuilder = org.apache.hadoop.hdfs.protocol.proto.DataTransferProtos.ClientOperationHeaderProto.newBuilder();
+              if (hasHeader()) {
+                subBuilder.mergeFrom(getHeader());
+              }
+              input.readMessage(subBuilder, extensionRegistry);
+              setHeader(subBuilder.buildPartial());
+              break;
+            }
+            case 16: {
+              bitField0_ |= 0x00000002;
+              offset_ = input.readUInt64();
+              break;
+            }
+            case 24: {
+              bitField0_ |= 0x00000004;
+              len_ = input.readUInt64();
+              break;
+            }
+          }
+        }
+      }
+      
+      private int bitField0_;
+      
+      // required .ClientOperationHeaderProto header = 1;
+      private org.apache.hadoop.hdfs.protocol.proto.DataTransferProtos.ClientOperationHeaderProto header_ = org.apache.hadoop.hdfs.protocol.proto.DataTransferProtos.ClientOperationHeaderProto.getDefaultInstance();
+      private com.google.protobuf.SingleFieldBuilder<
+          org.apache.hadoop.hdfs.protocol.proto.DataTransferProtos.ClientOperationHeaderProto, org.apache.hadoop.hdfs.protocol.proto.DataTransferProtos.ClientOperationHeaderProto.Builder, org.apache.hadoop.hdfs.protocol.proto.DataTransferProtos.ClientOperationHeaderProtoOrBuilder> headerBuilder_;
+      public boolean hasHeader() {
+        return ((bitField0_ & 0x00000001) == 0x00000001);
+      }
+      public org.apache.hadoop.hdfs.protocol.proto.DataTransferProtos.ClientOperationHeaderProto getHeader() {
+        if (headerBuilder_ == null) {
+          return header_;
+        } else {
+          return headerBuilder_.getMessage();
+        }
+      }
+      public Builder setHeader(org.apache.hadoop.hdfs.protocol.proto.DataTransferProtos.ClientOperationHeaderProto value) {
+        if (headerBuilder_ == null) {
+          if (value == null) {
+            throw new NullPointerException();
+          }
+          header_ = value;
+          onChanged();
+        } else {
+          headerBuilder_.setMessage(value);
+        }
+        bitField0_ |= 0x00000001;
+        return this;
+      }
+      public Builder setHeader(
+          org.apache.hadoop.hdfs.protocol.proto.DataTransferProtos.ClientOperationHeaderProto.Builder builderForValue) {
+        if (headerBuilder_ == null) {
+          header_ = builderForValue.build();
+          onChanged();
+        } else {
+          headerBuilder_.setMessage(builderForValue.build());
+        }
+        bitField0_ |= 0x00000001;
+        return this;
+      }
+      public Builder mergeHeader(org.apache.hadoop.hdfs.protocol.proto.DataTransferProtos.ClientOperationHeaderProto value) {
+        if (headerBuilder_ == null) {
+          if (((bitField0_ & 0x00000001) == 0x00000001) &&
+              header_ != org.apache.hadoop.hdfs.protocol.proto.DataTransferProtos.ClientOperationHeaderProto.getDefaultInstance()) {
+            header_ =
+              org.apache.hadoop.hdfs.protocol.proto.DataTransferProtos.ClientOperationHeaderProto.newBuilder(header_).mergeFrom(value).buildPartial();
+          } else {
+            header_ = value;
+          }
+          onChanged();
+        } else {
+          headerBuilder_.mergeFrom(value);
+        }
+        bitField0_ |= 0x00000001;
+        return this;
+      }
+      public Builder clearHeader() {
+        if (headerBuilder_ == null) {
+          header_ = org.apache.hadoop.hdfs.protocol.proto.DataTransferProtos.ClientOperationHeaderProto.getDefaultInstance();
+          onChanged();
+        } else {
+          headerBuilder_.clear();
+        }
+        bitField0_ = (bitField0_ & ~0x00000001);
+        return this;
+      }
+      public org.apache.hadoop.hdfs.protocol.proto.DataTransferProtos.ClientOperationHeaderProto.Builder getHeaderBuilder() {
+        bitField0_ |= 0x00000001;
+        onChanged();
+        return getHeaderFieldBuilder().getBuilder();
+      }
+      public org.apache.hadoop.hdfs.protocol.proto.DataTransferProtos.ClientOperationHeaderProtoOrBuilder getHeaderOrBuilder() {
+        if (headerBuilder_ != null) {
+          return headerBuilder_.getMessageOrBuilder();
+        } else {
+          return header_;
+        }
+      }
+      private com.google.protobuf.SingleFieldBuilder<
+          org.apache.hadoop.hdfs.protocol.proto.DataTransferProtos.ClientOperationHeaderProto, org.apache.hadoop.hdfs.protocol.proto.DataTransferProtos.ClientOperationHeaderProto.Builder, org.apache.hadoop.hdfs.protocol.proto.DataTransferProtos.ClientOperationHeaderProtoOrBuilder> 
+          getHeaderFieldBuilder() {
+        if (headerBuilder_ == null) {
+          headerBuilder_ = new com.google.protobuf.SingleFieldBuilder<
+              org.apache.hadoop.hdfs.protocol.proto.DataTransferProtos.ClientOperationHeaderProto, org.apache.hadoop.hdfs.protocol.proto.DataTransferProtos.ClientOperationHeaderProto.Builder, org.apache.hadoop.hdfs.protocol.proto.DataTransferProtos.ClientOperationHeaderProtoOrBuilder>(
+                  header_,
+                  getParentForChildren(),
+                  isClean());
+          header_ = null;
+        }
+        return headerBuilder_;
+      }
+      
+      // required uint64 offset = 2;
+      private long offset_ ;
+      public boolean hasOffset() {
+        return ((bitField0_ & 0x00000002) == 0x00000002);
+      }
+      public long getOffset() {
+        return offset_;
+      }
+      public Builder setOffset(long value) {
+        bitField0_ |= 0x00000002;
+        offset_ = value;
+        onChanged();
+        return this;
+      }
+      public Builder clearOffset() {
+        bitField0_ = (bitField0_ & ~0x00000002);
+        offset_ = 0L;
+        onChanged();
+        return this;
+      }
+      
+      // required uint64 len = 3;
+      private long len_ ;
+      public boolean hasLen() {
+        return ((bitField0_ & 0x00000004) == 0x00000004);
+      }
+      public long getLen() {
+        return len_;
+      }
+      public Builder setLen(long value) {
+        bitField0_ |= 0x00000004;
+        len_ = value;
+        onChanged();
+        return this;
+      }
+      public Builder clearLen() {
+        bitField0_ = (bitField0_ & ~0x00000004);
+        len_ = 0L;
+        onChanged();
+        return this;
+      }
+      
+      // @@protoc_insertion_point(builder_scope:OpReadBlockProto)
+    }
+    
+    static {
+      defaultInstance = new OpReadBlockProto(true);
+      defaultInstance.initFields();
+    }
+    
+    // @@protoc_insertion_point(class_scope:OpReadBlockProto)
+  }
+  
+  public interface OpWriteBlockProtoOrBuilder
+      extends com.google.protobuf.MessageOrBuilder {
+    
+    // required .ClientOperationHeaderProto header = 1;
+    boolean hasHeader();
+    org.apache.hadoop.hdfs.protocol.proto.DataTransferProtos.ClientOperationHeaderProto getHeader();
+    org.apache.hadoop.hdfs.protocol.proto.DataTransferProtos.ClientOperationHeaderProtoOrBuilder getHeaderOrBuilder();
+    
+    // repeated .DatanodeInfoProto targets = 2;
+    java.util.List<org.apache.hadoop.hdfs.protocol.proto.HdfsProtos.DatanodeInfoProto> 
+        getTargetsList();
+    org.apache.hadoop.hdfs.protocol.proto.HdfsProtos.DatanodeInfoProto getTargets(int index);
+    int getTargetsCount();
+    java.util.List<? extends org.apache.hadoop.hdfs.protocol.proto.HdfsProtos.DatanodeInfoProtoOrBuilder> 
+        getTargetsOrBuilderList();
+    org.apache.hadoop.hdfs.protocol.proto.HdfsProtos.DatanodeInfoProtoOrBuilder getTargetsOrBuilder(
+        int index);
+    
+    // optional .DatanodeInfoProto source = 3;
+    boolean hasSource();
+    org.apache.hadoop.hdfs.protocol.proto.HdfsProtos.DatanodeInfoProto getSource();
+    org.apache.hadoop.hdfs.protocol.proto.HdfsProtos.DatanodeInfoProtoOrBuilder getSourceOrBuilder();
+    
+    // required .OpWriteBlockProto.BlockConstructionStage stage = 4;
+    boolean hasStage();
+    org.apache.hadoop.hdfs.protocol.proto.DataTransferProtos.OpWriteBlockProto.BlockConstructionStage getStage();
+    
+    // required uint32 pipelineSize = 5;
+    boolean hasPipelineSize();
+    int getPipelineSize();
+    
+    // required uint64 minBytesRcvd = 6;
+    boolean hasMinBytesRcvd();
+    long getMinBytesRcvd();
+    
+    // required uint64 maxBytesRcvd = 7;
+    boolean hasMaxBytesRcvd();
+    long getMaxBytesRcvd();
+    
+    // required uint64 latestGenerationStamp = 8;
+    boolean hasLatestGenerationStamp();
+    long getLatestGenerationStamp();
+  }
+  public static final class OpWriteBlockProto extends
+      com.google.protobuf.GeneratedMessage
+      implements OpWriteBlockProtoOrBuilder {
+    // Use OpWriteBlockProto.newBuilder() to construct.
+    private OpWriteBlockProto(Builder builder) {
+      super(builder);
+    }
+    private OpWriteBlockProto(boolean noInit) {}
+    
+    private static final OpWriteBlockProto defaultInstance;
+    public static OpWriteBlockProto getDefaultInstance() {
+      return defaultInstance;
+    }
+    
+    public OpWriteBlockProto getDefaultInstanceForType() {
+      return defaultInstance;
+    }
+    
+    public static final com.google.protobuf.Descriptors.Descriptor
+        getDescriptor() {
+      return org.apache.hadoop.hdfs.protocol.proto.DataTransferProtos.internal_static_OpWriteBlockProto_descriptor;
+    }
+    
+    protected com.google.protobuf.GeneratedMessage.FieldAccessorTable
+        internalGetFieldAccessorTable() {
+      return org.apache.hadoop.hdfs.protocol.proto.DataTransferProtos.internal_static_OpWriteBlockProto_fieldAccessorTable;
+    }
+    
+    public enum BlockConstructionStage
+        implements com.google.protobuf.ProtocolMessageEnum {
+      PIPELINE_SETUP_APPEND(0, 0),
+      PIPELINE_SETUP_APPEND_RECOVERY(1, 1),
+      DATA_STREAMING(2, 2),
+      PIPELINE_SETUP_STREAMING_RECOVERY(3, 3),
+      PIPELINE_CLOSE(4, 4),
+      PIPELINE_CLOSE_RECOVERY(5, 5),
+      PIPELINE_SETUP_CREATE(6, 6),
+      TRANSFER_RBW(7, 7),
+      TRANSFER_FINALIZED(8, 8),
+      ;
+      
+      public static final int PIPELINE_SETUP_APPEND_VALUE = 0;
+      public static final int PIPELINE_SETUP_APPEND_RECOVERY_VALUE = 1;
+      public static final int DATA_STREAMING_VALUE = 2;
+      public static final int PIPELINE_SETUP_STREAMING_RECOVERY_VALUE = 3;
+      public static final int PIPELINE_CLOSE_VALUE = 4;
+      public static final int PIPELINE_CLOSE_RECOVERY_VALUE = 5;
+      public static final int PIPELINE_SETUP_CREATE_VALUE = 6;
+      public static final int TRANSFER_RBW_VALUE = 7;
+      public static final int TRANSFER_FINALIZED_VALUE = 8;
+      
+      
+      public final int getNumber() { return value; }
+      
+      public static BlockConstructionStage valueOf(int value) {
+        switch (value) {
+          case 0: return PIPELINE_SETUP_APPEND;
+          case 1: return PIPELINE_SETUP_APPEND_RECOVERY;
+          case 2: return DATA_STREAMING;
+          case 3: return PIPELINE_SETUP_STREAMING_RECOVERY;
+          case 4: return PIPELINE_CLOSE;
+          case 5: return PIPELINE_CLOSE_RECOVERY;
+          case 6: return PIPELINE_SETUP_CREATE;
+          case 7: return TRANSFER_RBW;
+          case 8: return TRANSFER_FINALIZED;
+          default: return null;
+        }
+      }
+      
+      public static com.google.protobuf.Internal.EnumLiteMap<BlockConstructionStage>
+          internalGetValueMap() {
+        return internalValueMap;
+      }
+      private static com.google.protobuf.Internal.EnumLiteMap<BlockConstructionStage>
+          internalValueMap =
+            new com.google.protobuf.Internal.EnumLiteMap<BlockConstructionStage>() {
+              public BlockConstructionStage findValueByNumber(int number) {
+                return BlockConstructionStage.valueOf(number);
+              }
+            };
+      
+      public final com.google.protobuf.Descriptors.EnumValueDescriptor
+          getValueDescriptor() {
+        return getDescriptor().getValues().get(index);
+      }
+      public final com.google.protobuf.Descriptors.EnumDescriptor
+          getDescriptorForType() {
+        return getDescriptor();
+      }
+      public static final com.google.protobuf.Descriptors.EnumDescriptor
+          getDescriptor() {
+        return org.apache.hadoop.hdfs.protocol.proto.DataTransferProtos.OpWriteBlockProto.getDescriptor().getEnumTypes().get(0);
+      }
+      
+      private static final BlockConstructionStage[] VALUES = {
+        PIPELINE_SETUP_APPEND, PIPELINE_SETUP_APPEND_RECOVERY, DATA_STREAMING, PIPELINE_SETUP_STREAMING_RECOVERY, PIPELINE_CLOSE, PIPELINE_CLOSE_RECOVERY, PIPELINE_SETUP_CREATE, TRANSFER_RBW, TRANSFER_FINALIZED, 
+      };
+      
+      public static BlockConstructionStage valueOf(
+          com.google.protobuf.Descriptors.EnumValueDescriptor desc) {
+        if (desc.getType() != getDescriptor()) {
+          throw new java.lang.IllegalArgumentException(
+            "EnumValueDescriptor is not for this type.");
+        }
+        return VALUES[desc.getIndex()];
+      }
+      
+      private final int index;
+      private final int value;
+      
+      private BlockConstructionStage(int index, int value) {
+        this.index = index;
+        this.value = value;
+      }
+      
+      // @@protoc_insertion_point(enum_scope:OpWriteBlockProto.BlockConstructionStage)
+    }
+    
+    private int bitField0_;
+    // required .ClientOperationHeaderProto header = 1;
+    public static final int HEADER_FIELD_NUMBER = 1;
+    private org.apache.hadoop.hdfs.protocol.proto.DataTransferProtos.ClientOperationHeaderProto header_;
+    public boolean hasHeader() {
+      return ((bitField0_ & 0x00000001) == 0x00000001);
+    }
+    public org.apache.hadoop.hdfs.protocol.proto.DataTransferProtos.ClientOperationHeaderProto getHeader() {
+      return header_;
+    }
+    public org.apache.hadoop.hdfs.protocol.proto.DataTransferProtos.ClientOperationHeaderProtoOrBuilder getHeaderOrBuilder() {
+      return header_;
+    }
+    
+    // repeated .DatanodeInfoProto targets = 2;
+    public static final int TARGETS_FIELD_NUMBER = 2;
+    private java.util.List<org.apache.hadoop.hdfs.protocol.proto.HdfsProtos.DatanodeInfoProto> targets_;
+    public java.util.List<org.apache.hadoop.hdfs.protocol.proto.HdfsProtos.DatanodeInfoProto> getTargetsList() {
+      return targets_;
+    }
+    public java.util.List<? extends org.apache.hadoop.hdfs.protocol.proto.HdfsProtos.DatanodeInfoProtoOrBuilder> 
+        getTargetsOrBuilderList() {
+      return targets_;
+    }
+    public int getTargetsCount() {
+      return targets_.size();
+    }
+    public org.apache.hadoop.hdfs.protocol.proto.HdfsProtos.DatanodeInfoProto getTargets(int index) {
+      return targets_.get(index);
+    }
+    public org.apache.hadoop.hdfs.protocol.proto.HdfsProtos.DatanodeInfoProtoOrBuilder getTargetsOrBuilder(
+        int index) {
+      return targets_.get(index);
+    }
+    
+    // optional .DatanodeInfoProto source = 3;
+    public static final int SOURCE_FIELD_NUMBER = 3;
+    private org.apache.hadoop.hdfs.protocol.proto.HdfsProtos.DatanodeInfoProto source_;
+    public boolean hasSource() {
+      return ((bitField0_ & 0x00000002) == 0x00000002);
+    }
+    public org.apache.hadoop.hdfs.protocol.proto.HdfsProtos.DatanodeInfoProto getSource() {
+      return source_;
+    }
+    public org.apache.hadoop.hdfs.protocol.proto.HdfsProtos.DatanodeInfoProtoOrBuilder getSourceOrBuilder() {
+      return source_;
+    }
+    
+    // required .OpWriteBlockProto.BlockConstructionStage stage = 4;
+    public static final int STAGE_FIELD_NUMBER = 4;
+    private org.apache.hadoop.hdfs.protocol.proto.DataTransferProtos.OpWriteBlockProto.BlockConstructionStage stage_;
+    public boolean hasStage() {
+      return ((bitField0_ & 0x00000004) == 0x00000004);
+    }
+    public org.apache.hadoop.hdfs.protocol.proto.DataTransferProtos.OpWriteBlockProto.BlockConstructionStage getStage() {
+      return stage_;
+    }
+    
+    // required uint32 pipelineSize = 5;
+    public static final int PIPELINESIZE_FIELD_NUMBER = 5;
+    private int pipelineSize_;
+    public boolean hasPipelineSize() {
+      return ((bitField0_ & 0x00000008) == 0x00000008);
+    }
+    public int getPipelineSize() {
+      return pipelineSize_;
+    }
+    
+    // required uint64 minBytesRcvd = 6;
+    public static final int MINBYTESRCVD_FIELD_NUMBER = 6;
+    private long minBytesRcvd_;
+    public boolean hasMinBytesRcvd() {
+      return ((bitField0_ & 0x00000010) == 0x00000010);
+    }
+    public long getMinBytesRcvd() {
+      return minBytesRcvd_;
+    }
+    
+    // required uint64 maxBytesRcvd = 7;
+    public static final int MAXBYTESRCVD_FIELD_NUMBER = 7;
+    private long maxBytesRcvd_;
+    public boolean hasMaxBytesRcvd() {
+      return ((bitField0_ & 0x00000020) == 0x00000020);
+    }
+    public long getMaxBytesRcvd() {
+      return maxBytesRcvd_;
+    }
+    
+    // required uint64 latestGenerationStamp = 8;
+    public static final int LATESTGENERATIONSTAMP_FIELD_NUMBER = 8;
+    private long latestGenerationStamp_;
+    public boolean hasLatestGenerationStamp() {
+      return ((bitField0_ & 0x00000040) == 0x00000040);
+    }
+    public long getLatestGenerationStamp() {
+      return latestGenerationStamp_;
+    }
+    
+    private void initFields() {
+      header_ = org.apache.hadoop.hdfs.protocol.proto.DataTransferProtos.ClientOperationHeaderProto.getDefaultInstance();
+      targets_ = java.util.Collections.emptyList();
+      source_ = org.apache.hadoop.hdfs.protocol.proto.HdfsProtos.DatanodeInfoProto.getDefaultInstance();
+      stage_ = org.apache.hadoop.hdfs.protocol.proto.DataTransferProtos.OpWriteBlockProto.BlockConstructionStage.PIPELINE_SETUP_APPEND;
+      pipelineSize_ = 0;
+      minBytesRcvd_ = 0L;
+      maxBytesRcvd_ = 0L;
+      latestGenerationStamp_ = 0L;
+    }
+    private byte memoizedIsInitialized = -1;
+    public final boolean isInitialized() {
+      byte isInitialized = memoizedIsInitialized;
+      if (isInitialized != -1) return isInitialized == 1;
+      
+      if (!hasHeader()) {
+        memoizedIsInitialized = 0;
+        return false;
+      }
+      if (!hasStage()) {
+        memoizedIsInitialized = 0;
+        return false;
+      }
+      if (!hasPipelineSize()) {
+        memoizedIsInitialized = 0;
+        return false;
+      }
+      if (!hasMinBytesRcvd()) {
+        memoizedIsInitialized = 0;
+        return false;
+      }
+      if (!hasMaxBytesRcvd()) {
+        memoizedIsInitialized = 0;
+        return false;
+      }
+      if (!hasLatestGenerationStamp()) {
+        memoizedIsInitialized = 0;
+        return false;
+      }
+      if (!getHeader().isInitialized()) {
+        memoizedIsInitialized = 0;
+        return false;
+      }
+      for (int i = 0; i < getTargetsCount(); i++) {
+        if (!getTargets(i).isInitialized()) {
+          memoizedIsInitialized = 0;
+          return false;
+        }
+      }
+      if (hasSource()) {
+        if (!getSource().isInitialized()) {
+          memoizedIsInitialized = 0;
+          return false;
+        }
+      }
+      memoizedIsInitialized = 1;
+      return true;
+    }
+    
+    public void writeTo(com.google.protobuf.CodedOutputStream output)
+                        throws java.io.IOException {
+      getSerializedSize();
+      if (((bitField0_ & 0x00000001) == 0x00000001)) {
+        output.writeMessage(1, header_);
+      }
+      for (int i = 0; i < targets_.size(); i++) {
+        output.writeMessage(2, targets_.get(i));
+      }
+      if (((bitField0_ & 0x00000002) == 0x00000002)) {
+        output.writeMessage(3, source_);
+      }
+      if (((bitField0_ & 0x00000004) == 0x00000004)) {
+        output.writeEnum(4, stage_.getNumber());
+      }
+      if (((bitField0_ & 0x00000008) == 0x00000008)) {
+        output.writeUInt32(5, pipelineSize_);
+      }
+      if (((bitField0_ & 0x00000010) == 0x00000010)) {
+        output.writeUInt64(6, minBytesRcvd_);
+      }
+      if (((bitField0_ & 0x00000020) == 0x00000020)) {
+        output.writeUInt64(7, maxBytesRcvd_);
+      }
+      if (((bitField0_ & 0x00000040) == 0x00000040)) {
+        output.writeUInt64(8, latestGenerationStamp_);
+      }
+      getUnknownFields().writeTo(output);
+    }
+    
+    private int memoizedSerializedSize = -1;
+    public int getSerializedSize() {
+      int size = memoizedSerializedSize;
+      if (size != -1) return size;
+    
+      size = 0;
+      if (((bitField0_ & 0x00000001) == 0x00000001)) {
+        size += com.google.protobuf.CodedOutputStream
+          .computeMessageSize(1, header_);
+      }
+      for (int i = 0; i < targets_.size(); i++) {
+        size += com.google.protobuf.CodedOutputStream
+          .computeMessageSize(2, targets_.get(i));
+      }
+      if (((bitField0_ & 0x00000002) == 0x00000002)) {
+        size += com.google.protobuf.CodedOutputStream
+          .computeMessageSize(3, source_);
+      }
+      if (((bitField0_ & 0x00000004) == 0x00000004)) {
+        size += com.google.protobuf.CodedOutputStream
+          .computeEnumSize(4, stage_.getNumber());
+      }
+      if (((bitField0_ & 0x00000008) == 0x00000008)) {
+        size += com.google.protobuf.CodedOutputStream
+          .computeUInt32Size(5, pipelineSize_);
+      }
+      if (((bitField0_ & 0x00000010) == 0x00000010)) {
+        size += com.google.protobuf.CodedOutputStream
+          .computeUInt64Size(6, minBytesRcvd_);
+      }
+      if (((bitField0_ & 0x00000020) == 0x00000020)) {
+        size += com.google.protobuf.CodedOutputStream
+          .computeUInt64Size(7, maxBytesRcvd_);
+      }
+      if (((bitField0_ & 0x00000040) == 0x00000040)) {
+        size += com.google.protobuf.CodedOutputStream
+          .computeUInt64Size(8, latestGenerationStamp_);
+      }
+      size += getUnknownFields().getSerializedSize();
+      memoizedSerializedSize = size;
+      return size;
+    }
+    
+    private static final long serialVersionUID = 0L;
+    @java.lang.Override
+    protected java.lang.Object writeReplace()
+        throws java.io.ObjectStreamException {
+      return super.writeReplace();
+    }
+    
+    @java.lang.Override
+    public boolean equals(final java.lang.Object obj) {
+      if (obj == this) {
+       return true;
+      }
+      if (!(obj instanceof org.apache.hadoop.hdfs.protocol.proto.DataTransferProtos.OpWriteBlockProto)) {
+        return super.equals(obj);
+      }
+      org.apache.hadoop.hdfs.protocol.proto.DataTransferProtos.OpWriteBlockProto other = (org.apache.hadoop.hdfs.protocol.proto.DataTransferProtos.OpWriteBlockProto) obj;
+      
+      boolean result = true;
+      result = result && (hasHeader() == other.hasHeader());
+      if (hasHeader()) {
+        result = result && getHeader()
+            .equals(other.getHeader());
+      }
+      result = result && getTargetsList()
+          .equals(other.getTargetsList());
+      result = result && (hasSource() == other.hasSource());
+      if (hasSource()) {
+        result = result && getSource()
+            .equals(other.getSource());
+      }
+      result = result && (hasStage() == other.hasStage());
+      if (hasStage()) {
+        result = result &&
+            (getStage() == other.getStage());
+      }
+      result = result && (hasPipelineSize() == other.hasPipelineSize());
+      if (hasPipelineSize()) {
+        result = result && (getPipelineSize()
+            == other.getPipelineSize());
+      }
+      result = result && (hasMinBytesRcvd() == other.hasMinBytesRcvd());
+      if (hasMinBytesRcvd()) {
+        result = result && (getMinBytesRcvd()
+            == other.getMinBytesRcvd());
+      }
+      result = result && (hasMaxBytesRcvd() == other.hasMaxBytesRcvd());
+      if (hasMaxBytesRcvd()) {
+        result = result && (getMaxBytesRcvd()
+            == other.getMaxBytesRcvd());
+      }
+      result = result && (hasLatestGenerationStamp() == other.hasLatestGenerationStamp());
+      if (hasLatestGenerationStamp()) {
+        result = result && (getLatestGenerationStamp()
+            == other.getLatestGenerationStamp());
+      }
+      result = result &&
+          getUnknownFields().equals(other.getUnknownFields());
+      return result;
+    }
+    
+    @java.lang.Override
+    public int hashCode() {
+      int hash = 41;
+      hash = (19 * hash) + getDescriptorForType().hashCode();
+      if (hasHeader()) {
+        hash = (37 * hash) + HEADER_FIELD_NUMBER;
+        hash = (53 * hash) + getHeader().hashCode();
+      }
+      if (getTargetsCount() > 0) {
+        hash = (37 * hash) + TARGETS_FIELD_NUMBER;
+        hash = (53 * hash) + getTargetsList().hashCode();
+      }
+      if (hasSource()) {
+        hash = (37 * hash) + SOURCE_FIELD_NUMBER;
+        hash = (53 * hash) + getSource().hashCode();
+      }
+      if (hasStage()) {
+        hash = (37 * hash) + STAGE_FIELD_NUMBER;
+        hash = (53 * hash) + hashEnum(getStage());
+      }
+      if (hasPipelineSize()) {
+        hash = (37 * hash) + PIPELINESIZE_FIELD_NUMBER;
+        hash = (53 * hash) + getPipelineSize();
+      }
+      if (hasMinBytesRcvd()) {
+        hash = (37 * hash) + MINBYTESRCVD_FIELD_NUMBER;
+        hash = (53 * hash) + hashLong(getMinBytesRcvd());
+      }
+      if (hasMaxBytesRcvd()) {
+        hash = (37 * hash) + MAXBYTESRCVD_FIELD_NUMBER;
+        hash = (53 * hash) + hashLong(getMaxBytesRcvd());
+      }
+      if (hasLatestGenerationStamp()) {
+        hash = (37 * hash) + LATESTGENERATIONSTAMP_FIELD_NUMBER;
+        hash = (53 * hash) + hashLong(getLatestGenerationStamp());
+      }
+      hash = (29 * hash) + getUnknownFields().hashCode();
+      return hash;
+    }
+    
+    public static org.apache.hadoop.hdfs.protocol.proto.DataTransferProtos.OpWriteBlockProto parseFrom(
+        com.google.protobuf.ByteString data)
+        throws com.google.protobuf.InvalidProtocolBufferException {
+      return newBuilder().mergeFrom(data).buildParsed();
+    }
+    public static org.apache.hadoop.hdfs.protocol.proto.DataTransferProtos.OpWriteBlockProto parseFrom(
+        com.google.protobuf.ByteString data,
+        com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+        throws com.google.protobuf.InvalidProtocolBufferException {
+      return newBuilder().mergeFrom(data, extensionRegistry)
+               .buildParsed();
+    }
+    public static org.apache.hadoop.hdfs.protocol.proto.DataTransferProtos.OpWriteBlockProto parseFrom(byte[] data)
+        throws com.google.protobuf.InvalidProtocolBufferException {
+      return newBuilder().mergeFrom(data).buildParsed();
+    }
+    public static org.apache.hadoop.hdfs.protocol.proto.DataTransferProtos.OpWriteBlockProto parseFrom(
+        byte[] data,
+        com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+        throws com.google.protobuf.InvalidProtocolBufferException {
+      return newBuilder().mergeFrom(data, extensionRegistry)
+               .buildParsed();
+    }
+    public static org.apache.hadoop.hdfs.protocol.proto.DataTransferProtos.OpWriteBlockProto parseFrom(java.io.InputStream input)
+        throws java.io.IOException {
+      return newBuilder().mergeFrom(input).buildParsed();
+    }
+    public static org.apache.hadoop.hdfs.protocol.proto.DataTransferProtos.OpWriteBlockProto parseFrom(
+        java.io.InputStream input,
+        com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+        throws java.io.IOException {
+      return newBuilder().mergeFrom(input, extensionRegistry)
+               .buildParsed();
+    }
+    public static org.apache.hadoop.hdfs.protocol.proto.DataTransferProtos.OpWriteBlockProto parseDelimitedFrom(java.io.InputStream input)
+        throws java.io.IOException {
+      Builder builder = newBuilder();
+      if (builder.mergeDelimitedFrom(input)) {
+        return builder.buildParsed();
+      } else {
+        return null;
+      }
+    }
+    public static org.apache.hadoop.hdfs.protocol.proto.DataTransferProtos.OpWriteBlockProto parseDelimitedFrom(
+        java.io.InputStream input,
+        com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+        throws java.io.IOException {
+      Builder builder = newBuilder();
+      if (builder.mergeDelimitedFrom(input, extensionRegistry)) {
+        return builder.buildParsed();
+      } else {
+        return null;
+      }
+    }
+    public static org.apache.hadoop.hdfs.protocol.proto.DataTransferProtos.OpWriteBlockProto parseFrom(
+        com.google.protobuf.CodedInputStream input)
+        throws java.io.IOException {
+      return newBuilder().mergeFrom(input).buildParsed();
+    }
+    public static org.apache.hadoop.hdfs.protocol.proto.DataTransferProtos.OpWriteBlockProto parseFrom(
+        com.google.protobuf.CodedInputStream input,
+        com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+        throws java.io.IOException {
+      return newBuilder().mergeFrom(input, extensionRegistry)
+               .buildParsed();
+    }
+    
+    public static Builder newBuilder() { return Builder.create(); }
+    public Builder newBuilderForType() { return newBuilder(); }
+    public static Builder newBuilder(org.apache.hadoop.hdfs.protocol.proto.DataTransferProtos.OpWriteBlockProto prototype) {
+      return newBuilder().mergeFrom(prototype);
+    }
+    public Builder toBuilder() { return newBuilder(this); }
+    
+    @java.lang.Override
+    protected Builder newBuilderForType(
+        com.google.protobuf.GeneratedMessage.BuilderParent parent) {
+      Builder builder = new Builder(parent);
+      return builder;
+    }
+    public static final class Builder extends
+        com.google.protobuf.GeneratedMessage.Builder<Builder>
+       implements org.apache.hadoop.hdfs.protocol.proto.DataTransferProtos.OpWriteBlockProtoOrBuilder {
+      public static final com.google.protobuf.Descriptors.Descriptor
+          getDescriptor() {
+        return org.apache.hadoop.hdfs.protocol.proto.DataTransferProtos.internal_static_OpWriteBlockProto_descriptor;
+      }
+      
+      protected com.google.protobuf.GeneratedMessage.FieldAccessorTable
+          internalGetFieldAccessorTable() {
+        return org.apache.hadoop.hdfs.protocol.proto.DataTransferProtos.internal_static_OpWriteBlockProto_fieldAccessorTable;
+      }
+      
+      // Construct using org.apache.hadoop.hdfs.protocol.proto.DataTransferProtos.OpWriteBlockProto.newBuilder()
+      private Builder() {
+        maybeForceBuilderInitialization();
+      }
+      
+      private Builder(BuilderParent parent) {
+        super(parent);
+        maybeForceBuilderInitialization();
+      }
+      private void maybeForceBuilderInitialization() {
+        if (com.google.protobuf.GeneratedMessage.alwaysUseFieldBuilders) {
+          getHeaderFieldBuilder();
+          getTargetsFieldBuilder();
+          getSourceFieldBuilder();
+        }
+      }
+      private static Builder create() {
+        return new Builder();
+      }
+      
+      public Builder clear() {
+        super.clear();
+        if (headerBuilder_ == null) {
+          header_ = org.apache.hadoop.hdfs.protocol.proto.DataTransferProtos.ClientOperationHeaderProto.getDefaultInstance();
+        } else {
+          headerBuilder_.clear();
+        }
+        bitField0_ = (bitField0_ & ~0x00000001);
+        if (targetsBuilder_ == null) {
+          targets_ = java.util.Collections.emptyList();
+          bitField0_ = (bitField0_ & ~0x00000002);
+        } else {
+          targetsBuilder_.clear();
+        }
+        if (sourceBuilder_ == null) {
+          source_ = org.apache.hadoop.hdfs.protocol.proto.HdfsProtos.DatanodeInfoProto.getDefaultInstance();
+        } else {
+          sourceBuilder_.clear();
+        }
+        bitField0_ = (bitField0_ & ~0x00000004);
+        stage_ = org.apache.hadoop.hdfs.protocol.proto.DataTransferProtos.OpWriteBlockProto.BlockConstructionStage.PIPELINE_SETUP_APPEND;
+        bitField0_ = (bitField0_ & ~0x00000008);
+        pipelineSize_ = 0;
+        bitField0_ = (bitField0_ & ~0x00000010);
+        minBytesRcvd_ = 0L;
+        bitField0_ = (bitField0_ & ~0x00000020);
+        maxBytesRcvd_ = 0L;
+        bitField0_ = (bitField0_ & ~0x00000040);
+        latestGenerationStamp_ = 0L;
+        bitField0_ = (bitField0_ & ~0x00000080);
+        return this;
+      }
+      
+      public Builder clone() {
+        return create().mergeFrom(buildPartial());
+      }
+      
+      public com.google.protobuf.Descriptors.Descriptor
+          getDescriptorForType() {
+        return org.apache.hadoop.hdfs.protocol.proto.DataTransferProtos.OpWriteBlockProto.getDescriptor();
+      }
+      
+      public org.apache.hadoop.hdfs.protocol.proto.DataTransferProtos.OpWriteBlockProto getDefaultInstanceForType() {
+        return org.apache.hadoop.hdfs.protocol.proto.DataTransferProtos.OpWriteBlockProto.getDefaultInstance();
+      }
+      
+      public org.apache.hadoop.hdfs.protocol.proto.DataTransferProtos.OpWriteBlockProto build() {
+        org.apache.hadoop.hdfs.protocol.proto.DataTransferProtos.OpWriteBlockProto result = buildPartial();
+        if (!result.isInitialized()) {
+          throw newUninitializedMessageException(result);
+        }
+        return result;
+      }
+      
+      private org.apache.hadoop.hdfs.protocol.proto.DataTransferProtos.OpWriteBlockProto buildParsed()
+          throws com.google.protobuf.InvalidProtocolBufferException {
+        org.apache.hadoop.hdfs.protocol.proto.DataTransferProtos.OpWriteBlockProto result = buildPartial();
+        if (!result.isInitialized()) {
+          throw newUninitializedMessageException(
+            result).asInvalidProtocolBufferException();
+        }
+        return result;
+      }
+      
+      public org.apache.hadoop.hdfs.protocol.proto.DataTransferProtos.OpWriteBlockProto buildPartial() {
+        org.apache.hadoop.hdfs.protocol.proto.DataTransferProtos.OpWriteBlockProto result = new org.apache.hadoop.hdfs.protocol.proto.DataTransferProtos.OpWriteBlockProto(this);
+        int from_bitField0_ = bitField0_;
+        int to_bitField0_ = 0;
+        if (((from_bitField0_ & 0x00000001) == 0x00000001)) {
+          to_bitField0_ |= 0x00000001;
+        }
+        if (headerBuilder_ == null) {
+          result.header_ = header_;
+        } else {
+          result.header_ = headerBuilder_.build();
+        }
+        if (targetsBuilder_ == null) {
+          if (((bitField0_ & 0x00000002) == 0x00000002)) {
+            targets_ = java.util.Collections.unmodifiableList(targets_);
+            bitField0_ = (bitField0_ & ~0x00000002);
+          }
+          result.targets_ = targets_;
+        } else {
+          result.targets_ = targetsBuilder_.build();
+        }
+        if (((from_bitField0_ & 0x00000004) == 0x00000004)) {
+          to_bitField0_ |= 0x00000002;
+        }
+        if (sourceBuilder_ == null) {
+          result.source_ = source_;
+        } else {
+          result.source_ = sourceBuilder_.build();
+        }
+        if (((from_bitField0_ & 0x00000008) == 0x00000008)) {
+          to_bitField0_ |= 0x00000004;
+        }
+        result.stage_ = stage_;
+        if (((from_bitField0_ & 0x00000010) == 0x00000010)) {
+          to_bitField0_ |= 0x00000008;
+        }
+        result.pipelineSize_ = pipelineSize_;
+        if (((from_bitField0_ & 0x00000020) == 0x00000020)) {
+          to_bitField0_ |= 0x00000010;
+        }
+        result.minBytesRcvd_ = minBytesRcvd_;
+        if (((from_bitField0_ & 0x00000040) == 0x00000040)) {
+          to_bitField0_ |= 0x00000020;
+        }
+        result.maxBytesRcvd_ = maxBytesRcvd_;
+        if (((from_bitField0_ & 0x00000080) == 0x00000080)) {
+          to_bitField0_ |= 0x00000040;
+        }
+        result.latestGenerationStamp_ = latestGenerationStamp_;
+        result.bitField0_ = to_bitField0_;
+        onBuilt();
+        return result;
+      }
+      
+      public Builder mergeFrom(com.google.protobuf.Message other) {
+        if (other instanceof org.apache.hadoop.hdfs.protocol.proto.DataTransferProtos.OpWriteBlockProto) {
+          return mergeFrom((org.apache.hadoop.hdfs.protocol.proto.DataTransferProtos.OpWriteBlockProto)other);
+        } else {
+          super.mergeFrom(other);
+          return this;
+        }
+      }
+      
+      public Builder mergeFrom(org.apache.hadoop.hdfs.protocol.proto.DataTransferProtos.OpWriteBlockProto other) {
+        if (other == org.apache.hadoop.hdfs.protocol.proto.DataTransferProtos.OpWriteBlockProto.getDefaultInstance()) return this;
+        if (other.hasHeader()) {
+          mergeHeader(other.getHeader());
+        }
+        if (targetsBuilder_ == null) {
+          if (!other.targets_.isEmpty()) {
+            if (targets_.isEmpty()) {
+              targets_ = other.targets_;
+              bitField0_ = (bitField0_ & ~0x00000002);
+            } else {
+              ensureTargetsIsMutable();
+              targets_.addAll(other.targets_);
+            }
+            onChanged();
+          }
+        } else {
+          if (!other.targets_.isEmpty()) {
+            if (targetsBuilder_.isEmpty()) {
+              targetsBuilder_.dispose();
+              targetsBuilder_ = null;
+              targets_ = other.targets_;
+              bitField0_ = (bitField0_ & ~0x00000002);
+              targetsBuilder_ = 
+                com.google.protobuf.GeneratedMessage.alwaysUseFieldBuilders ?
+                   getTargetsFieldBuilder() : null;
+            } else {
+              targetsBuilder_.addAllMessages(other.targets_);
+            }
+          }
+        }
+        if (other.hasSource()) {
+          mergeSource(other.getSource());
+        }
+        if (other.hasStage()) {
+          setStage(other.getStage());
+        }
+        if (other.hasPipelineSize()) {
+          setPipelineSize(other.getPipelineSize());
+        }
+        if (other.hasMinBytesRcvd()) {
+          setMinBytesRcvd(other.getMinBytesRcvd());
+        }
+        if (other.hasMaxBytesRcvd()) {
+          setMaxBytesRcvd(other.getMaxBytesRcvd());
+        }
+        if (other.hasLatestGenerationStamp()) {
+          setLatestGenerationStamp(other.getLatestGenerationStamp());
+        }
+        this.mergeUnknownFields(other.getUnknownFields());
+        return this;
+      }
+      
+      public final boolean isInitialized() {
+        if (!hasHeader()) {
+          
+          return false;
+        }
+        if (!hasStage()) {
+          
+          return false;
+        }
+        if (!hasPipelineSize()) {
+          
+          return false;
+        }
+        if (!hasMinBytesRcvd()) {
+          
+          return false;
+        }
+        if (!hasMaxBytesRcvd()) {
+          
+          return false;
+        }
+        if (!hasLatestGenerationStamp()) {
+          
+          return false;
+        }
+        if (!getHeader().isInitialized()) {
+          
+          return false;
+        }
+        for (int i = 0; i < getTargetsCount(); i++) {
+          if (!getTargets(i).isInitialized()) {
+            
+            return false;
+          }
+        }
+        if (hasSource()) {
+          if (!getSource().isInitialized()) {
+            
+            return false;
+          }
+        }
+        return true;
+      }
+      
+      public Builder mergeFrom(
+          com.google.protobuf.CodedInputStream input,
+          com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+          throws java.io.IOException {
+        com.google.protobuf.UnknownFieldSet.Builder unknownFields =
+          com.google.protobuf.UnknownFieldSet.newBuilder(
+            this.getUnknownFields());
+        while (true) {
+          int tag = input.readTag();
+          switch (tag) {
+            case 0:
+              this.setUnknownFields(unknownFields.build());
+              onChanged();
+              return this;
+            default: {
+              if (!parseUnknownField(input, unknownFields,
+                                     extensionRegistry, tag)) {
+                this.setUnknownFields(unknownFields.build());
+                onChanged();
+                return this;
+              }
+              break;
+            }
+            case 10: {
+              org.apache.hadoop.hdfs.protocol.proto.DataTransferProtos.ClientOperationHeaderProto.Builder subBuilder = org.apache.hadoop.hdfs.protocol.proto.DataTransferProtos.ClientOperationHeaderProto.newBuilder();
+              if (hasHeader()) {
+                subBuilder.mergeFrom(getHeader());
+              }
+              input.readMessage(subBuilder, extensionRegistry);
+              setHeader(subBuilder.buildPartial());
+              break;
+            }
+            case 18: {
+              org.apache.hadoop.hdfs.protocol.proto.HdfsProtos.DatanodeInfoProto.Builder subBuilder = org.apache.hadoop.hdfs.protocol.proto.HdfsProtos.DatanodeInfoProto.newBuilder();
+              input.readMessage(subBuilder, extensionRegistry);
+              addTargets(subBuilder.buildPartial());
+              break;
+            }
+            case 26: {
+              org.apache.hadoop.hdfs.protocol.proto.HdfsProtos.DatanodeInfoProto.Builder subBuilder = org.apache.hadoop.hdfs.protocol.proto.HdfsProtos.DatanodeInfoProto.newBuilder();
+              if (hasSource()) {
+                subBuilder.mergeFrom(getSource());
+              }
+              input.readMessage(subBuilder, extensionRegistry);
+              setSource(subBuilder.buildPartial());
+              break;
+            }
+            case 32: {
+              int rawValue = input.readEnum();
+              org.apache.hadoop.hdfs.protocol.proto.DataTransferProtos.OpWriteBlockProto.BlockConstructionStage value = org.apache.hadoop.hdfs.protocol.proto.DataTransferProtos.OpWriteBlockProto.BlockConstructionStage.valueOf(rawValue);
+              if (value == null) {
+                unknownFields.mergeVarintField(4, rawValue);
+              } else {
+                bitField0_ |= 0x00000008;
+                stage_ = value;
+              }
+              break;
+            }
+            case 40: {
+              bitField0_ |= 0x00000010;
+              pipelineSize_ = input.readUInt32();
+              break;
+            }
+            case 48: {
+              bitField0_ |= 0x00000020;
+              minBytesRcvd_ = input.readUInt64();
+              break;
+            }
+            case 56: {
+              bitField0_ |= 0x00000040;
+              maxBytesRcvd_ = input.readUInt64();
+              break;
+            }
+            case 64: {
+              bitField0_ |= 0x00000080;
+              latestGenerationStamp_ = input.readUInt64();
+              break;
+            }
+          }
+        }
+      }
+      
+      private int bitField0_;
+      
+      // required .ClientOperationHeaderProto header = 1;
+      private org.apache.hadoop.hdfs.protocol.proto.DataTransferProtos.ClientOperationHeaderProto header_ = org.apache.hadoop.hdfs.protocol.proto.DataTransferProtos.ClientOperationHeaderProto.getDefaultInstance();
+      private com.google.protobuf.SingleFieldBuilder<
+          org.apache.hadoop.hdfs.protocol.proto.DataTransferProtos.ClientOperationHeaderProto, org.apache.hadoop.hdfs.protocol.proto.DataTransferProtos.ClientOperationHeaderProto.Builder, org.apache.hadoop.hdfs.protocol.proto.DataTransferProtos.ClientOperationHeaderProtoOrBuilder> headerBuilder_;
+      public boolean hasHeader() {
+        return ((bitField0_ & 0x00000001) == 0x00000001);
+      }
+      public org.apache.hadoop.hdfs.protocol.proto.DataTransferProtos.ClientOperationHeaderProto getHeader() {
+        if (headerBuilder_ == null) {
+          return header_;
+        } else {
+          return headerBuilder_.getMessage();
+        }
+      }
+      public Builder setHeader(org.apache.hadoop.hdfs.protocol.proto.DataTransferProtos.ClientOperationHeaderProto value) {
+        if (headerBuilder_ == null) {
+          if (value == null) {
+            throw new NullPointerException();
+          }
+          header_ = value;
+          onChanged();
+        } else {
+          headerBuilder_.setMessage(value);
+        }
+        bitField0_ |= 0x00000001;
+        return this;
+      }
+      public Builder setHeader(
+          org.apache.hadoop.hdfs.protocol.proto.DataTransferProtos.ClientOperationHeaderProto.Builder builderForValue) {
+        if (headerBuilder_ == null) {
+          header_ = builderForValue.build();
+          onChanged();
+        } else {
+          headerBuilder_.setMessage(builderForValue.build());
+        }
+        bitField0_ |= 0x00000001;
+        return this;
+      }
+      public Builder mergeHeader(org.apache.hadoop.hdfs.protocol.proto.DataTransferProtos.ClientOperationHeaderProto value) {
+        if (headerBuilder_ == null) {
+          if (((bitField0_ & 0x00000001) == 0x00000001) &&
+              header_ != org.apache.hadoop.hdfs.protocol.proto.DataTransferProtos.ClientOperationHeaderProto.getDefaultInstance()) {
+            header_ =
+              org.apache.hadoop.hdfs.protocol.proto.DataTransferProtos.ClientOperationHeaderProto.newBuilder(header_).mergeFrom(value).buildPartial();
+          } else {
+            header_ = value;
+          }
+          onChanged();
+        } else {
+          headerBuilder_.mergeFrom(value);
+        }
+        bitField0_ |= 0x00000001;
+        return this;
+      }
+      public Builder clearHeader() {
+        if (headerBuilder_ == null) {
+          header_ = org.apache.hadoop.hdfs.protocol.proto.DataTransferProtos.ClientOperationHeaderProto.getDefaultInstance();
+          onChanged();
+        } else {
+          headerBuilder_.clear();
+        }
+        bitField0_ = (bitField0_ & ~0x00000001);
+        return this;
+      }
+      public org.apache.hadoop.hdfs.protocol.proto.DataTransferProtos.ClientOperationHeaderProto.Builder getHeaderBuilder() {
+        bitField0_ |= 0x00000001;
+        onChanged();
+        return getHeaderFieldBuilder().getBuilder();
+      }
+      public org.apache.hadoop.hdfs.protocol.proto.DataTransferProtos.ClientOperationHeaderProtoOrBuilder getHeaderOrBuilder() {
+        if (headerBuilder_ != null) {
+          return headerBuilder_.getMessageOrBuilder();
+        } else {
+          return header_;
+        }
+      }
+      private com.google.protobuf.SingleFieldBuilder<
+          org.apache.hadoop.hdfs.protocol.proto.DataTransferProtos.ClientOperationHeaderProto, org.apache.hadoop.hdfs.protocol.proto.DataTransferProtos.ClientOperationHeaderProto.Builder, org.apache.hadoop.hdfs.protocol.proto.DataTransferProtos.ClientOperationHeaderProtoOrBuilder> 
+          getHeaderFieldBuilder() {
+        if (headerBuilder_ == null) {
+          headerBuilder_ = new com.google.protobuf.SingleFieldBuilder<
+              org.apache.hadoop.hdfs.protocol.proto.DataTransferProtos.ClientOperationHeaderProto, org.apache.hadoop.hdfs.protocol.proto.DataTransferProtos.ClientOperationHeaderProto.Builder, org.apache.hadoop.hdfs.protocol.proto.DataTransferProtos.ClientOperationHeaderProtoOrBuilder>(
+                  header_,
+                  getParentForChildren(),
+                  isClean());
+          header_ = null;
+        }
+        return headerBuilder_;
+      }
+      
+      // repeated .DatanodeInfoProto targets = 2;
+      private java.util.List<org.apache.hadoop.hdfs.protocol.proto.HdfsProtos.DatanodeInfoProto> targets_ =
+        java.util.Collections.emptyList();
+      private void ensureTargetsIsMutable() {
+        if (!((bitField0_ & 0x00000002) == 0x00000002)) {
+          targets_ = new java.util.ArrayList<org.apache.hadoop.hdfs.protocol.proto.HdfsProtos.DatanodeInfoProto>(targets_);
+          bitField0_ |= 0x00000002;
+         }
+      }
+      
+      private com.google.protobuf.RepeatedFieldBuilder<
+          org.apache.hadoop.hdfs.protocol.proto.HdfsProtos.DatanodeInfoProto, org.apache.hadoop.hdfs.protocol.proto.HdfsProtos.DatanodeInfoProto.Builder, org.apache.hadoop.hdfs.protocol.proto.HdfsProtos.DatanodeInfoProtoOrBuilder> targetsBuilder_;
+      
+      public java.util.List<org.apache.hadoop.hdfs.protocol.proto.HdfsProtos.DatanodeInfoProto> getTargetsList() {
+        if (targetsBuilder_ == null) {
+          return java.util.Collections.unmodifiableList(targets_);
+        } else {
+          return targetsBuilder_.getMessageList();
+        }
+      }
+      public int getTargetsCount() {
+        if (targetsBuilder_ == null) {
+          return targets_.size();
+        } else {
+          return targetsBuilder_.getCount();
+        }
+      }
+      public org.apache.hadoop.hdfs.protocol.proto.HdfsProtos.DatanodeInfoProto getTargets(int index) {
+        if (targetsBuilder_ == null) {
+          return targets_.get(index);
+        } else {
+          return targetsBuilder_.getMessage(index);
+        }
+      }
+      public Builder setTargets(
+          int index, org.apache.hadoop.hdfs.protocol.proto.HdfsProtos.DatanodeInfoProto value) {
+        if (targetsBuilder_ == null) {
+          if (value == null) {
+            throw new NullPointerException();
+          }
+          ensureTargetsIsMutable();
+          targets_.set(index, value);
+          onChanged();
+        } else {
+          targetsBuilder_.setMessage(index, value);
+        }
+        return this;
+      }
+      public Builder setTargets(
+          int index, org.apache.hadoop.hdfs.protocol.proto.HdfsProtos.DatanodeInfoProto.Builder builderForValue) {
+        if (targetsBuilder_ == null) {
+          ensureTargetsIsMutable();
+          targets_.set(index, builderForValue.build());
+          onChanged();
+        } else {
+          targetsBuilder_.setMessage(index, builderForValue.build());
+        }
+        return this;
+      }
+      public Builder addTargets(org.apache.hadoop.hdfs.protocol.proto.HdfsProtos.DatanodeInfoProto value) {
+        if (targetsBuilder_ == null) {
+          if (value == null) {
+            throw new NullPointerException();
+          }
+          ensureTargetsIsMutable();
+          targets_.add(value);
+          onChanged();
+        } else {
+          targetsBuilder_.addMessage(value);
+        }
+        return this;
+      }
+      public Builder addTargets(
+          int index, org.apache.hadoop.hdfs.protocol.proto.HdfsProtos.DatanodeInfoProto value) {
+        if (targetsBuilder_ == null) {
+          if (value == null) {
+            throw new NullPointerException();
+          }
+          ensureTargetsIsMutable();
+          targets_.add(index, value);
+          onChanged();
+        } else {
+          targetsBuilder_.addMessage(index, value);
+        }
+        return this;
+      }
+      public Builder addTargets(
+          org.apache.hadoop.hdfs.protocol.proto.HdfsProtos.DatanodeInfoProto.Builder builderForValue) {
+        if (targetsBuilder_ == null) {
+          ensureTargetsIsMutable();
+          targets_.add(builderForValue.build());
+          onChanged();
+        } else {
+          targetsBuilder_.addMessage(builderForValue.build());
+        }
+        return this;
+      }
+      public Builder addTargets(
+          int index, org.apache.hadoop.hdfs.protocol.proto.HdfsProtos.DatanodeInfoProto.Builder builderForValue) {
+        if (targetsBuilder_ == null) {
+          ensureTargetsIsMutable();
+          targets_.add(index, builderForValue.build());
+          onChanged();
+        } else {
+          targetsBuilder_.addMessage(index, builderForValue.build());
+        }
+        return this;
+      }
+      public Builder addAllTargets(
+          java.lang.Iterable<? extends org.apache.hadoop.hdfs.protocol.proto.HdfsProtos.DatanodeInfoProto> values) {
+        if (targetsBuilder_ == null) {
+          ensureTargetsIsMutable();
+          super.addAll(values, targets_);
+          onChanged();
+        } else {
+          targetsBuilder_.addAllMessages(values);
+        }
+        return this;
+      }
+      public Builder clearTargets() {
+        if (targetsBuilder_ == null) {
+          targets_ = java.util.Collections.emptyList();
+          bitField0_ = (bitField0_ & ~0x00000002);
+          onChanged();
+        } else {
+          targetsBuilder_.clear();
+        }
+        return this;
+      }
+      public Builder removeTargets(int index) {
+        if (targetsBuilder_ == null) {
+          ensureTargetsIsMutable();
+          targets_.remove(index);
+          onChanged();
+        } else {
+          targetsBuilder_.remove(index);
+        }
+        return this;
+      }
+      public org.apache.hadoop.hdfs.protocol.proto.HdfsProtos.DatanodeInfoProto.Builder getTargetsBuilder(
+          int index) {
+        return getTargetsFieldBuilder().getBuilder(index);
+      }
+      public org.apache.hadoop.hdfs.protocol.proto.HdfsProtos.DatanodeInfoProtoOrBuilder getTargetsOrBuilder(
+          int index) {
+        if (targetsBuilder_ == null) {
+          return targets_.get(index);  } else {
+          return targetsBuilder_.getMessageOrBuilder(index);
+        }
+      }
+      public java.util.List<? extends org.apache.hadoop.hdfs.protocol.proto.HdfsProtos.DatanodeInfoProtoOrBuilder> 
+           getTargetsOrBuilderList() {
+        if (targetsBuilder_ != null) {
+          return targetsBuilder_.getMessageOrBuilderList();
+        } else {
+          return java.util.Collections.unmodifiableList(targets_);
+        }
+      }
+      public org.apache.hadoop.hdfs.protocol.proto.HdfsProtos.DatanodeInfoProto.Builder addTargetsBuilder() {
+        return getTargetsFieldBuilder().addBuilder(
+            org.apache.hadoop.hdfs.protocol.proto.HdfsProtos.DatanodeInfoProto.getDefaultInstance());
+      }
+      public org.apache.hadoop.hdfs.protocol.proto.HdfsProtos.DatanodeInfoProto.Builder addTargetsBuilder(
+          int index) {
+        return getTargetsFieldBuilder().addBuilder(
+            index, org.apache.hadoop.hdfs.protocol.proto.HdfsProtos.DatanodeInfoProto.getDefaultInstance());
+      }
+      public java.util.List<org.apache.hadoop.hdfs.protocol.proto.HdfsProtos.DatanodeInfoProto.Builder> 
+           getTargetsBuilderList() {
+        return getTargetsFieldBuilder().getBuilderList();
+      }
+      private com.google.protobuf.RepeatedFieldBuilder<
+          org.apache.hadoop.hdfs.protocol.proto.HdfsProtos.DatanodeInfoProto, org.apache.hadoop.hdfs.protocol.proto.HdfsProtos.DatanodeInfoProto.Builder, org.apache.hadoop.hdfs.protocol.proto.HdfsProtos.DatanodeInfoProtoOrBuilder> 
+          getTargetsFieldBuilder() {
+        if (targetsBuilder_ == null) {
+          targetsBuilder_ = new com.google.protobuf.RepeatedFieldBuilder<
+              org.apache.hadoop.hdfs.protocol.proto.HdfsProtos.DatanodeInfoProto, org.apache.hadoop.hdfs.protocol.proto.HdfsProtos.DatanodeInfoProto.Builder, org.apache.hadoop.hdfs.protocol.proto.HdfsProtos.DatanodeInfoProtoOrBuilder>(
+                  targets_,
+                  ((bitField0_ & 0x00000002) == 0x00000002),
+                  getParentForChildren(),
+                  isClean());
+          targets_ = null;
+        }
+        return targetsBuilder_;
+      }
+      
+      // optional .DatanodeInfoProto source = 3;
+      private org.apache.hadoop.hdfs.protocol.proto.HdfsProtos.DatanodeInfoProto source_ = org.apache.hadoop.hdfs.protocol.proto.HdfsProtos.DatanodeInfoProto.getDefaultInstance();
+      private com.google.protobuf.SingleFieldBuilder<
+          org.apache.hadoop.hdfs.protocol.proto.HdfsProtos.DatanodeInfoProto, org.apache.hadoop.hdfs.protocol.proto.HdfsProtos.DatanodeInfoProto.Builder, org.apache.hadoop.hdfs.protocol.proto.HdfsProtos.DatanodeInfoProtoOrBuilder> sourceBuilder_;
+      public boolean hasSource() {
+        return ((bitField0_ & 0x00000004) == 0x00000004);
+      }
+      public org.apache.hadoop.hdfs.protocol.proto.HdfsProtos.DatanodeInfoProto getSource() {
+        if (sourceBuilder_ == null) {
+          return source_;
+        } else {
+          return sourceBuilder_.getMessage();
+        }
+      }
+      public Builder setSource(org.apache.hadoop.hdfs.protocol.proto.HdfsProtos.DatanodeInfoProto value) {
+        if (sourceBuilder_ == null) {
+          if (value == null) {
+            throw new NullPointerException();
+          }
+          source_ = value;
+          onChanged();
+        } else {
+          sourceBuilder_.setMessage(value);
+        }
+        bitField0_ |= 0x00000004;
+        return this;
+      }
+      public Builder setSource(
+          org.apache.hadoop.hdfs.protocol.proto.HdfsProtos.DatanodeInfoProto.Builder builderForValue) {
+        if (sourceBuilder_ == null) {
+          source_ = builderForValue.build();
+          onChanged();
+        } else {
+          sourceBuilder_.setMessage(builderForValue.build());
+        }
+        bitField0_ |= 0x00000004;
+        return this;
+      }
+      public Builder mergeSource(org.apache.hadoop.hdfs.protocol.proto.HdfsProtos.DatanodeInfoProto value) {
+        if (sourceBuilder_ == null) {
+          if (((bitField0_ & 0x00000004) == 0x00000004) &&
+              source_ != org.apache.hadoop.hdfs.protocol.proto.HdfsProtos.DatanodeInfoProto.getDefaultInstance()) {
+            source_ =
+              org.apache.hadoop.hdfs.protocol.proto.HdfsProtos.DatanodeInfoProto.newBuilder(source_).mergeFrom(value).buildPartial();
+          } else {
+            source_ = value;
+          }
+          onChanged();
+        } else {
+          sourceBuilder_.mergeFrom(value);
+        }
+        bitField0_ |= 0x00000004;
+        return this;
+      }
+      public Builder clearSource() {
+        if (sourceBuilder_ == null) {
+          source_ = org.apache.hadoop.hdfs.protocol.proto.HdfsProtos.DatanodeInfoProto.getDefaultInstance();
+          onChanged();
+        } else {
+          sourceBuilder_.clear();
+        }
+        bitField0_ = (bitField0_ & ~0x00000004);
+        return this;
+      }
+      public org.apache.hadoop.hdfs.protocol.proto.HdfsProtos.DatanodeInfoProto.Builder getSourceBuilder() {
+        bitField0_ |= 0x00000004;
+        onChanged();
+        return getSourceFieldBuilder().getBuilder();
+      }
+      public org.apache.hadoop.hdfs.protocol.proto.HdfsProtos.DatanodeInfoProtoOrBuilder getSourceOrBuilder() {
+        if (sourceBuilder_ != null) {
+          return sourceBuilder_.getMessageOrBuilder();
+        } else {
+          return source_;
+        }
+      }
+      private com.google.protobuf.SingleFieldBuilder<
+          org.apache.hadoop.hdfs.protocol.proto.HdfsProtos.DatanodeInfoProto, org.apache.hadoop.hdfs.protocol.proto.HdfsProtos.DatanodeInfoProto.Builder, org.apache.hadoop.hdfs.protocol.proto.HdfsProtos.DatanodeInfoProtoOrBuilder> 
+          getSourceFieldBuilder() {
+        if (sourceBuilder_ == null) {
+          sourceBuilder_ = new com.google.protobuf.SingleFieldBuilder<
+              org.apache.hadoop.hdfs.protocol.proto.HdfsProtos.DatanodeInfoProto, org.apache.hadoop.hdfs.protocol.proto.HdfsProtos.DatanodeInfoProto.Builder, org.apache.hadoop.hdfs.protocol.proto.HdfsProtos.DatanodeInfoProtoOrBuilder>(
+                  source_,
+                  getParentForChildren(),
+                  isClean());
+          source_ = null;
+        }
+        return sourceBuilder_;
+      }
+      
+      // required .OpWriteBlockProto.BlockConstructionStage stage = 4;
+      private org.apache.hadoop.hdfs.protocol.proto.DataTransferProtos.OpWriteBlockProto.BlockConstructionStage stage_ = org.apache.hadoop.hdfs.protocol.proto.DataTransferProtos.OpWriteBlockProto.BlockConstructionStage.PIPELINE_SETUP_APPEND;
+      public boolean hasStage() {
+        return ((bitField0_ & 0x00000008) == 0x00000008);
+      }
+      public org.apache.hadoop.hdfs.protocol.proto.DataTransferProtos.OpWriteBlockProto.BlockConstructionStage getStage() {
+        return stage_;
+      }
+      public Builder setStage(org.apache.hadoop.hdfs.protocol.proto.DataTransferProtos.OpWriteBlockProto.BlockConstructionStage value) {
+        if (value == null) {
+          throw new NullPointerException();
+        }
+        bitField0_ |= 0x00000008;
+        stage_ = value;
+        onChanged();
+        return this;
+      }
+      public Builder clearStage() {
+        bitField0_ = (bitField0_ & ~0x00000008);
+        stage_ = org.apache.hadoop.hdfs.protocol.proto.DataTransferProtos.OpWriteBlockProto.BlockConstructionStage.PIPELINE_SETUP_APPEND;
+        onChanged();
+        return this;
+      }
+      
+      // required uint32 pipelineSize = 5;
+      private int pipelineSize_ ;
+      public boolean hasPipelineSize() {
+        return ((bitField0_ & 0x00000010) == 0x00000010);
+      }
+      public int getPipelineSize() {
+        return pipelineSize_;
+      }
+      public Builder setPipelineSize(int value) {
+        bitField0_ |= 0x00000010;
+        pipelineSize_ = value;
+        onChanged();
+        return this;
+      }
+      public Builder clearPipelineSize() {
+        bitField0_ = (bitField0_ & ~0x00000010);
+        pipelineSize_ = 0;
+        onChanged();
+        return this;
+      }
+      
+      // required uint64 minBytesRcvd = 6;
+      private long minBytesRcvd_ ;
+      public boolean hasMinBytesRcvd() {
+        return ((bitField0_ & 0x00000020) == 0x00000020);
+      }
+      public long getMinBytesRcvd() {
+        return minBytesRcvd_;
+      }
+      public Builder setMinBytesRcvd(long value) {
+        bitField0_ |= 0x00000020;
+        minBytesRcvd_ = value;
+        onChanged();
+        return this;
+      }
+      public Builder clearMinBytesRcvd() {
+        bitField0_ = (bitField0_ & ~0x00000020);
+        minBytesRcvd_ = 0L;
+        onChanged();
+        return this;
+      }
+      
+      // required uint64 maxBytesRcvd = 7;
+      private long maxBytesRcvd_ ;
+      public boolean hasMaxBytesRcvd() {
+        return ((bitField0_ & 0x00000040) == 0x00000040);
+      }
+      public long getMaxBytesRcvd() {
+        return maxBytesRcvd_;
+      }
+      public Builder setMaxBytesRcvd(long value) {
+        bitField0_ |= 0x00000040;
+        maxBytesRcvd_ = value;
+        onChanged();
+        return this;
+      }
+      public Builder clearMaxBytesRcvd() {
+        bitField0_ = (bitField0_ & ~0x00000040);
+        maxBytesRcvd_ = 0L;
+        onChanged();
+        return this;
+      }
+      
+      // required uint64 latestGenerationStamp = 8;
+      private long latestGenerationStamp_ ;
+      public boolean hasLatestGenerationStamp() {
+        return ((bitField0_ & 0x00000080) == 0x00000080);
+      }
+      public long getLatestGenerationStamp() {
+        return latestGenerationStamp_;
+      }
+      public Builder setLatestGenerationStamp(long value) {
+        bitField0_ |= 0x00000080;
+        latestGenerationStamp_ = value;
+        onChanged();
+        return this;
+      }
+      public Builder clearLatestGenerationStamp() {
+        bitField0_ = (bitField0_ & ~0x00000080);
+        latestGenerationStamp_ = 0L;
+        onChanged();
+        return this;
+      }
+      
+      // @@protoc_insertion_point(builder_scope:OpWriteBlockProto)
+    }
+    
+    static {
+      defaultInstance = new OpWriteBlockProto(true);
+      defaultInstance.initFields();
+    }
+    
+    // @@protoc_insertion_point(class_scope:OpWriteBlockProto)
+  }
+  
+  public interface OpTransferBlockProtoOrBuilder
+      extends com.google.protobuf.MessageOrBuilder {
+    
+    // required .ClientOperationHeaderProto header = 1;
+    boolean hasHeader();
+    org.apache.hadoop.hdfs.protocol.proto.DataTransferProtos.ClientOperationHeaderProto getHeader();
+    org.apache.hadoop.hdfs.protocol.proto.DataTransferProtos.ClientOperationHeaderProtoOrBuilder getHeaderOrBuilder();
+    
+    // repeated .DatanodeInfoProto targets = 2;
+    java.util.List<org.apache.hadoop.hdfs.protocol.proto.HdfsProtos.DatanodeInfoProto> 
+        getTargetsList();
+    org.apache.hadoop.hdfs.protocol.proto.HdfsProtos.DatanodeInfoProto getTargets(int index);
+    int getTargetsCount();
+    java.util.List<? extends org.apache.hadoop.hdfs.protocol.proto.HdfsProtos.DatanodeInfoProtoOrBuilder> 
+        getTargetsOrBuilderList();
+    org.apache.hadoop.hdfs.protocol.proto.HdfsProtos.DatanodeInfoProtoOrBuilder getTargetsOrBuilder(
+        int index);
+  }
+  public static final class OpTransferBlockProto extends
+      com.google.protobuf.GeneratedMessage
+      implements OpTransferBlockProtoOrBuilder {
+    // Use OpTransferBlockProto.newBuilder() to construct.
+    private OpTransferBlockProto(Builder builder) {
+      super(builder);
+    }
+    private OpTransferBlockProto(boolean noInit) {}
+    
+    private static final OpTransferBlockProto defaultInstance;
+    public static OpTransferBlockProto getDefaultInstance() {
+      return defaultInstance;
+    }
+    
+    public OpTransferBlockProto getDefaultInstanceForType() {
+      return defaultInstance;
+    }
+    
+    public static final com.google.protobuf.Descriptors.Descriptor
+        getDescriptor() {
+      return org.apache.hadoop.hdfs.protocol.proto.DataTransferProtos.internal_static_OpTransferBlockProto_descriptor;
+    }
+    
+    protected com.google.protobuf.GeneratedMessage.FieldAccessorTable
+        internalGetFieldAccessorTable() {
+      return org.apache.hadoop.hdfs.protocol.proto.DataTransferProtos.internal_static_OpTransferBlockProto_fieldAccessorTable;
+    }
+    
+    private int bitField0_;
+    // required .ClientOperationHeaderProto header = 1;
+    public static final int HEADER_FIELD_NUMBER = 1;
+    private org.apache.hadoop.hdfs.protocol.proto.DataTransferProtos.ClientOperationHeaderProto header_;
+    public boolean hasHeader() {
+      return ((bitField0_ & 0x00000001) == 0x00000001);
+    }
+    public org.apache.hadoop.hdfs.protocol.proto.DataTransferProtos.ClientOperationHeaderProto getHeader() {
+      return header_;
+    }
+    public org.apache.hadoop.hdfs.protocol.proto.DataTransferProtos.ClientOperationHeaderProtoOrBuilder getHeaderOrBuilder() {
+      return header_;
+    }
+    
+    // repeated .DatanodeInfoProto targets = 2;
+    public static final int TARGETS_FIELD_NUMBER = 2;
+    private java.util.List<org.apache.hadoop.hdfs.protocol.proto.HdfsProtos.DatanodeInfoProto> targets_;
+    public java.util.List<org.apache.hadoop.hdfs.protocol.proto.HdfsProtos.DatanodeInfoProto> getTargetsList() {
+      return targets_;
+    }
+    public java.util.List<? extends org.apache.hadoop.hdfs.protocol.proto.HdfsProtos.DatanodeInfoProtoOrBuilder> 
+        getTargetsOrBuilderList() {
+      return targets_;
+    }
+    public int getTargetsCount() {
+      return targets_.size();
+    }
+    public org.apache.hadoop.hdfs.protocol.proto.HdfsProtos.DatanodeInfoProto getTargets(int index) {
+      return targets_.get(index);
+    }
+    public org.apache.hadoop.hdfs.protocol.proto.HdfsProtos.DatanodeInfoProtoOrBuilder getTargetsOrBuilder(
+        int index) {
+      return targets_.get(index);
+    }
+    
+    private void initFields() {
+      header_ = org.apache.hadoop.hdfs.protocol.proto.DataTransferProtos.ClientOperationHeaderProto.getDefaultInstance();
+      targets_ = java.util.Collections.emptyList();
+    }
+    private byte memoizedIsInitialized = -1;
+    public final boolean isInitialized() {
+      byte isInitialized = memoizedIsInitialized;
+      if (isInitialized != -1) return isInitialized == 1;
+      
+      if (!hasHeader()) {
+        memoizedIsInitialized = 0;
+        return false;
+      }
+      if (!getHeader().isInitialized()) {
+        memoizedIsInitialized = 0;
+        return false;
+      }
+      for (int i = 0; i < getTargetsCount(); i++) {
+        if (!getTargets(i).isInitialized()) {
+          memoizedIsInitialized = 0;
+          return false;
+        }
+      }
+      memoizedIsInitialized = 1;
+      return true;
+    }
+    
+    public void writeTo(com.google.protobuf.CodedOutputStream output)
+                        throws java.io.IOException {
+      getSerializedSize();
+      if (((bitField0_ & 0x00000001) == 0x00000001)) {
+        output.writeMessage(1, header_);
+      }
+      for (int i = 0; i < targets_.size(); i++) {
+        output.writeMessage(2, targets_.get(i));
+      }
+      getUnknownFields().writeTo(output);
+    }
+    
+    private int memoizedSerializedSize = -1;
+    public int getSerializedSize() {
+      int size = memoizedSerializedSize;
+      if (size != -1) return size;
+    
+      size = 0;
+      if (((bitField0_ & 0x00000001) == 0x00000001)) {
+        size += com.google.protobuf.CodedOutputStream
+          .computeMessageSize(1, header_);
+      }
+      for (int i = 0; i < targets_.size(); i++) {
+        size += com.google.protobuf.CodedOutputStream
+          .computeMessageSize(2, targets_.get(i));
+      }
+      size += getUnknownFields().getSerializedSize();
+      memoizedSerializedSize = size;
+      return size;
+    }
+    
+    private static final long serialVersionUID = 0L;
+    @java.lang.Override
+    protected java.lang.Object writeReplace()
+        throws java.io.ObjectStreamException {
+      return super.writeReplace();
+    }
+    
+    @java.lang.Override
+    public boolean equals(final java.lang.Object obj) {
+      if (obj == this) {
+       return true;
+      }
+      if (!(obj instanceof org.apache.hadoop.hdfs.protocol.proto.DataTransferProtos.OpTransferBlockProto)) {
+        return super.equals(obj);
+      }
+      org.apache.hadoop.hdfs.protocol.proto.DataTransferProtos.OpTransferBlockProto other = (org.apache.hadoop.hdfs.protocol.proto.DataTransferProtos.OpTransferBlockProto) obj;
+      
+      boolean result = true;
+      result = result && (hasHeader() == other.hasHeader());
+      if (hasHeader()) {
+        result = result && getHeader()
+            .equals(other.getHeader());
+      }
+      result = result && getTargetsList()
+          .equals(other.getTargetsList());
+      result = result &&
+          getUnknownFields().equals(other.getUnknownFields());
+      return result;
+    }
+    
+    @java.lang.Override
+    public int hashCode() {
+      int hash = 41;
+      hash = (19 * hash) + getDescriptorForType().hashCode();
+      if (hasHeader()) {
+        hash = (37 * hash) + HEADER_FIELD_NUMBER;
+        hash = (53 * hash) + getHeader().hashCode();
+      }
+      if (getTargetsCount() > 0) {
+        hash = (37 * hash) + TARGETS_FIELD_NUMBER;
+        hash = (53 * hash) + getTargetsList().hashCode();
+      }
+      hash = (29 * hash) + getUnknownFields().hashCode();
+      return hash;
+    }
+    
+    public static org.apache.hadoop.hdfs.protocol.proto.DataTransferProtos.OpTransferBlockProto parseFrom(
+        com.google.protobuf.ByteString data)
+        throws com.google.protobuf.InvalidProtocolBufferException {
+      return newBuilder().mergeFrom(data).buildParsed();
+    }
+    public static org.apache.hadoop.hdfs.protocol.proto.DataTransferProtos.OpTransferBlockProto parseFrom(
+        com.google.protobuf.ByteString data,
+        com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+        throws com.google.protobuf.InvalidProtocolBufferException {
+      return newBuilder().mergeFrom(data, extensionRegistry)
+               .buildParsed();
+    }
+    public static org.apache.hadoop.hdfs.protocol.proto.DataTransferProtos.OpTransferBlockProto parseFrom(byte[] data)
+        throws com.google.protobuf.InvalidProtocolBufferException {
+      return newBuilder().mergeFrom(data).buildParsed();
+    }
+    public static org.apache.hadoop.hdfs.protocol.proto.DataTransferProtos.OpTransferBlockProto parseFrom(
+        byte[] data,
+        com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+        throws com.google.protobuf.InvalidProtocolBufferException {
+      return newBuilder().mergeFrom(data, extensionRegistry)
+               .buildParsed();
+    }
+    public static org.apache.hadoop.hdfs.protocol.proto.DataTransferProtos.OpTransferBlockProto parseFrom(java.io.InputStream input)
+        throws java.io.IOException {
+      return newBuilder().mergeFrom(input).buildParsed();
+    }
+    public static org.apache.hadoop.hdfs.protocol.proto.DataTransferProtos.OpTransferBlockProto parseFrom(
+        java.io.InputStream input,
+        com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+        throws java.io.IOException {
+      return newBuilder().mergeFrom(input, extensionRegistry)
+               .buildParsed();
+    }
+    public static org.apache.hadoop.hdfs.protocol.proto.DataTransferProtos.OpTransferBlockProto parseDelimitedFrom(java.io.InputStream input)
+        throws java.io.IOException {
+      Builder builder = newBuilder();
+      if (builder.mergeDelimitedFrom(input)) {
+        return builder.buildParsed();
+      } else {
+        return null;
+      }
+    }
+    public static org.apache.hadoop.hdfs.protocol.proto.DataTransferProtos.OpTransferBlockProto parseDelimitedFrom(
+        java.io.InputStream input,
+        com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+        throws java.io.IOException {
+      Builder builder = newBuilder();
+      if (builder.mergeDelimitedFrom(input, extensionRegistry)) {
+        return builder.buildParsed();
+      } else {
+        return null;
+      }
+    }
+    public static org.apache.hadoop.hdfs.protocol.proto.DataTransferProtos.OpTransferBlockProto parseFrom(
+        com.google.protobuf.CodedInputStream input)
+        throws java.io.IOException {
+      return newBuilder().mergeFrom(input).buildParsed();
+    }
+    public static org.apache.hadoop.hdfs.protocol.proto.DataTransferProtos.OpTransferBlockProto parseFrom(
+        com.google.protobuf.CodedInputStream input,
+        com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+        throws java.io.IOException {
+      return newBuilder().mergeFrom(input, extensionRegistry)
+               .buildParsed();
+    }
+    
+    public static Builder newBuilder() { return Builder.create(); }
+    public Builder newBuilderForType() { return newBuilder(); }
+    public static Builder newBuilder(org.apache.hadoop.hdfs.protocol.proto.DataTransferProtos.OpTransferBlockProto prototype) {
+      return newBuilder().mergeFrom(prototype);
+    }
+    public Builder toBuilder() { return newBuilder(this); }
+    
+    @java.lang.Override
+    protected Builder newBuilderForType(
+        com.google.protobuf.GeneratedMessage.BuilderParent parent) {
+      Builder builder = new Builder(parent);
+      return builder;
+    }
+    public static final class Builder extends
+        com.google.protobuf.GeneratedMessage.Builder<Builder>
+       implements org.apache.hadoop.hdfs.protocol.proto.DataTransferProtos.OpTransferBlockProtoOrBuilder {
+      public static final com.google.protobuf.Descriptors.Descriptor
+          getDescriptor() {
+        return org.apache.hadoop.hdfs.protocol.proto.DataTransferProtos.internal_static_OpTransferBlockProto_descriptor;
+      }
+      
+      protected com.google.protobuf.GeneratedMessage.FieldAccessorTable
+          internalGetFieldAccessorTable() {
+        return org.apache.hadoop.hdfs.protocol.proto.DataTransferProtos.internal_static_OpTransferBlockProto_fieldAccessorTable;
+      }
+      
+      // Construct using org.apache.hadoop.hdfs.protocol.proto.DataTransferProtos.OpTransferBlockProto.newBuilder()
+      private Builder() {
+        maybeForceBuilderInitialization();
+      }
+      
+      private Builder(BuilderParent parent) {
+        super(parent);
+        maybeForceBuilderInitialization();
+      }
+      private void maybeForceBuilderInitialization() {
+        if (com.google.protobuf.GeneratedMessage.alwaysUseFieldBuilders) {
+          getHeaderFieldBuilder();
+          getTargetsFieldBuilder();
+        }
+      }
+      private static Builder create() {
+        return new Builder();
+      }
+      
+      public Builder clear() {
+        super.clear();
+        if (headerBuilder_ == null) {
+          header_ = org.apache.hadoop.hdfs.protocol.proto.DataTransferProtos.ClientOperationHeaderProto.getDefaultInstance();
+        } else {
+          headerBuilder_.clear();
+        }
+        bitField0_ = (bitField0_ & ~0x00000001);
+        if (targetsBuilder_ == null) {
+          targets_ = java.util.Collections.emptyList();
+          bitField0_ = (bitField0_ & ~0x00000002);
+        } else {
+          targetsBuilder_.clear();
+        }
+        return this;
+      }
+      
+      public Builder clone() {
+        return create().mergeFrom(buildPartial());
+      }
+      
+      public com.google.protobuf.Descriptors.Descriptor
+          getDescriptorForType() {
+        return org.apache.hadoop.hdfs.protocol.proto.DataTransferProtos.OpTransferBlockProto.getDescriptor();
+      }
+      
+      public org.apache.hadoop.hdfs.protocol.proto.DataTransferProtos.OpTransferBlockProto getDefaultInstanceForType() {
+        return org.apache.hadoop.hdfs.protocol.proto.DataTransferProtos.OpTransferBlockProto.getDefaultInstance();
+      }
+      
+      public org.apache.hadoop.hdfs.protocol.proto.DataTransferProtos.OpTransferBlockProto build() {
+        org.apache.hadoop.hdfs.protocol.proto.DataTransferProtos.OpTransferBlockProto result = buildPartial();
+        if (!result.isInitialized()) {
+          throw newUninitializedMessageException(result);
+        }
+        return result;
+      }
+      
+      private org.apache.hadoop.hdfs.protocol.proto.DataTransferProtos.OpTransferBlockProto buildParsed()
+          throws com.google.protobuf.InvalidProtocolBufferException {
+        org.apache.hadoop.hdfs.protocol.proto.DataTransferProtos.OpTransferBlockProto result = buildPartial();
+        if (!result.isInitialized()) {
+          throw newUninitializedMessageException(
+            result).asInvalidProtocolBufferException();
+        }
+        return result;
+      }
+      
+      public org.apache.hadoop.hdfs.protocol.proto.DataTransferProtos.OpTransferBlockProto buildPartial() {
+        org.apache.hadoop.hdfs.protocol.proto.DataTransferProtos.OpTransferBlockProto result = new org.apache.hadoop.hdfs.protocol.proto.DataTransferProtos.OpTransferBlockProto(this);
+        int from_bitField0_ = bitField0_;
+        int to_bitField0_ = 0;
+        if (((from_bitField0_ & 0x00000001) == 0x00000001)) {
+          to_bitField0_ |= 0x00000001;
+        }
+        if (headerBuilder_ == null) {
+          result.header_ = header_;
+        } else {
+          result.header_ = headerBuilder_.build();
+        }
+        if (targetsBuilder_ == null) {
+          if (((bitField0_ & 0x00000002) == 0x00000002)) {
+            targets_ = java.util.Collections.unmodifiableList(targets_);
+            bitField0_ = (bitField0_ & ~0x00000002);
+          }
+          result.targets_ = targets_;
+        } else {
+          result.targets_ = targetsBuilder_.build();
+        }
+        result.bitField0_ = to_bitField0_;
+        onBuilt();
+        return result;
+      }
+      
+      public Builder mergeFrom(com.google.protobuf.Message other) {
+        if (other instanceof org.apache.hadoop.hdfs.protocol.proto.DataTransferProtos.OpTransferBlockProto) {
+          return mergeFrom((org.apache.hadoop.hdfs.protocol.proto.DataTransferProtos.OpTransferBlockProto)other);
+        } else {
+          super.mergeFrom(other);
+          return this;
+        }
+      }
+      
+      public Builder mergeFrom(org.apache.hadoop.hdfs.protocol.proto.DataTransferProtos.OpTransferBlockProto other) {
+        if (other == org.apache.hadoop.hdfs.protocol.proto.DataTransferProtos.OpTransferBlockProto.getDefaultInstance()) return this;
+        if (other.hasHeader()) {
+          mergeHeader(other.getHeader());
+        }
+        if (targetsBuilder_ == null) {
+          if (!other.targets_.isEmpty()) {
+            if (targets_.isEmpty()) {
+              targets_ = other.targets_;
+              bitField0_ = (bitField0_ & ~0x00000002);
+            } else {
+              ensureTargetsIsMutable();
+              targets_.addAll(other.targets_);
+            }
+            onChanged();
+          }
+        } else {
+          if (!other.targets_.isEmpty()) {
+            if (targetsBuilder_.isEmpty()) {
+              targetsBuilder_.dispose();
+              targetsBuilder_ = null;
+              targets_ = other.targets_;
+              bitField0_ = (bitField0_ & ~0x00000002);
+              targetsBuilder_ = 
+                com.google.protobuf.GeneratedMessage.alwaysUseFieldBuilders ?
+                   getTargetsFieldBuilder() : null;
+            } else {
+              targetsBuilder_.addAllMessages(other.targets_);
+            }
+          }
+        }
+        this.mergeUnknownFields(other.getUnknownFields());
+        return this;
+      }
+      
+      public final boolean isInitialized() {
+        if (!hasHeader()) {
+          
+          return false;
+        }
+        if (!getHeader().isInitialized()) {
+          
+          return false;
+        }
+        for (int i = 0; i < getTargetsCount(); i++) {
+          if (!getTargets(i).isInitialized()) {
+            
+            return false;
+          }
+        }
+        return true;
+      }
+      
+      public Builder mergeFrom(
+          com.google.protobuf.CodedInputStream input,
+          com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+          throws java.io.IOException {
+        com.google.protobuf.UnknownFieldSet.Builder unknownFields =
+          com.google.protobuf.UnknownFieldSet.newBuilder(
+            this.getUnknownFields());
+        while (true) {
+          int tag = input.readTag();
+          switch (tag) {
+            case 0:
+              this.setUnknownFields(unknownFields.build());
+              onChanged();
+              return this;
+            default: {
+              if (!parseUnknownField(input, unknownFields,
+                                     extensionRegistry, tag)) {
+                this.setUnknownFields(unknownFields.build());
+                onChanged();
+                return this;
+              }
+              break;
+            }
+            case 10: {
+              org.apache.hadoop.hdfs.protocol.proto.DataTransferProtos.ClientOperationHeaderProto.Builder subBuilder = org.apache.hadoop.hdfs.protocol.proto.DataTransferProtos.ClientOperationHeaderProto.newBuilder();
+              if (hasHeader()) {
+                subBuilder.mergeFrom(getHeader());
+              }
+              input.readMessage(subBuilder, extensionRegistry);
+              setHeader(subBuilder.buildPartial());
+              break;
+            }
+            case 18: {
+              org.apache.hadoop.hdfs.protocol.proto.HdfsProtos.DatanodeInfoProto.Builder subBuilder = org.apache.hadoop.hdfs.protocol.proto.HdfsProtos.DatanodeInfoProto.newBuilder();
+              input.readMessage(subBuilder, extensionRegistry);
+              addTargets(subBuilder.buildPartial());
+              break;
+            }
+          }
+        }
+      }
+      
+      private int bitField0_;
+      
+      // required .ClientOperationHeaderProto header = 1;
+      private org.apache.hadoop.hdfs.protocol.proto.DataTransferProtos.ClientOperationHeaderProto header_ = org.apache.hadoop.hdfs.protocol.proto.DataTransferProtos.ClientOperationHeaderProto.getDefaultInstance();
+      private com.google.protobuf.SingleFieldBuilder<
+          org.apache.hadoop.hdfs.protocol.proto.DataTransferProtos.ClientOperationHeaderProto, org.apache.hadoop.hdfs.protocol.proto.DataTransferProtos.ClientOperationHeaderProto.Builder, org.apache.hadoop.hdfs.protocol.proto.DataTransferProtos.ClientOperationHeaderProtoOrBuilder> headerBuilder_;
+      public boolean hasHeader() {
+        return ((bitField0_ & 0x00000001) == 0x00000001);
+      }
+      public org.apache.hadoop.hdfs.protocol.proto.DataTransferProtos.ClientOperationHeaderProto getHeader() {
+        if (headerBuilder_ == null) {
+          return header_;
+        } else {
+          return headerBuilder_.getMessage();
+        }
+      }
+      public Builder setHeader(org.apache.hadoop.hdfs.protocol.proto.DataTransferProtos.ClientOperationHeaderProto value) {
+        if (headerBuilder_ == null) {
+          if (value == null) {
+            throw new NullPointerException();
+          }
+          header_ = value;
+          onChanged();
+        } else {
+          headerBuilder_.setMessage(value);
+        }
+        bitField0_ |= 0x00000001;
+        return this;
+      }
+      public Builder setHeader(
+          org.apache.hadoop.hdfs.protocol.proto.DataTransferProtos.ClientOperationHeaderProto.Builder builderForValue) {
+        if (headerBuilder_ == null) {
+          header_ = builderForValue.build();
+          onChanged();
+        } else {
+          headerBuilder_.setMessage(builderForValue.build());
+        }
+        bitField0_ |= 0x00000001;
+        return this;
+      }
+      public Builder mergeHeader(org.apache.hadoop.hdfs.protocol.proto.DataTransferProtos.ClientOperationHeaderProto value) {
+        if (headerBuilder_ == null) {
+          if (((bitField0_ & 0x00000001) == 0x00000001) &&
+              header_ != org.apache.hadoop.hdfs.protocol.proto.DataTransferProtos.ClientOperationHeaderProto.getDefaultInstance()) {
+            header_ =
+              org.apache.hadoop.hdfs.protocol.proto.DataTransferProtos.ClientOperationHeaderProto.newBuilder(header_).mergeFrom(value).buildPartial();
+          } else {
+            header_ = value;
+          }
+          onChanged();
+        } else {
+          headerBuilder_.mergeFrom(value);
+        }
+        bitField0_ |= 0x00000001;
+        return this;
+      }
+      public Builder clearHeader() {
+        if (headerBuilder_ == null) {
+          header_ = org.apache.hadoop.hdfs.protocol.proto.DataTransferProtos.ClientOperationHeaderProto.getDefaultInstance();
+          onChanged();
+        } else {
+          headerBuilder_.clear();
+        }
+        bitField0_ = (bitField0_ & ~0x00000001);
+        return this;
+      }
+      public org.apache.hadoop.hdfs.protocol.proto.DataTransferProtos.ClientOperationHeaderProto.Builder getHeaderBuilder() {
+        bitField0_ |= 0x00000001;
+        onChanged();
+        return getHeaderFieldBuilder().getBuilder();
+      }
+      public org.apache.hadoop.hdfs.protocol.proto.DataTransferProtos.ClientOperationHeaderProtoOrBuilder getHeaderOrBuilder() {
+        if (headerBuilder_ != null) {
+          return headerBuilder_.getMessageOrBuilder();
+        } else {
+          return header_;
+        }
+      }
+      private com.google.protobuf.SingleFieldBuilder<
+          org.apache.hadoop.hdfs.protocol.proto.DataTransferProtos.ClientOperationHeaderProto, org.apache.hadoop.hdfs.protocol.proto.DataTransferProtos.ClientOperationHeaderProto.Builder, org.apache.hadoop.hdfs.protocol.proto.DataTransferProtos.ClientOperationHeaderProtoOrBuilder> 
+          getHeaderFieldBuilder() {
+        if (headerBuilder_ == null) {
+          headerBuilder_ = new com.google.protobuf.SingleFieldBuilder<
+              org.apache.hadoop.hdfs.protocol.proto.DataTransferProtos.ClientOperationHeaderProto, org.apache.hadoop.hdfs.protocol.proto.DataTransferProtos.ClientOperationHeaderProto.Builder, org.apache.hadoop.hdfs.protocol.proto.DataTransferProtos.ClientOperationHeaderProtoOrBuilder>(
+                  header_,
+                  getParentForChildren(),
+                  isClean());
+          header_ = null;
+        }
+        return headerBuilder_;
+      }
+      
+      // repeated .DatanodeInfoProto targets = 2;
+      private java.util.List<org.apache.hadoop.hdfs.protocol.proto.HdfsProtos.DatanodeInfoProto> targets_ =
+        java.util.Collections.emptyList();
+      private void ensureTargetsIsMutable() {
+        if (!((bitField0_ & 0x00000002) == 0x00000002)) {
+          targets_ = new java.util.ArrayList<org.apache.hadoop.hdfs.protocol.proto.HdfsProtos.DatanodeInfoProto>(targets_);
+          bitField0_ |= 0x00000002;
+         }
+      }
+      
+      private com.google.protobuf.RepeatedFieldBuilder<
+          org.apache.hadoop.hdfs.protocol.proto.HdfsProtos.DatanodeInfoProto, org.apache.hadoop.hdfs.protocol.proto.HdfsProtos.DatanodeInfoProto.Builder, org.apache.hadoop.hdfs.protocol.proto.HdfsProtos.DatanodeInfoProtoOrBuilder> targetsBuilder_;
+      
+      public java.util.List<org.apache.hadoop.hdfs.protocol.proto.HdfsProtos.DatanodeInfoProto> getTargetsList() {
+        if (targetsBuilder_ == null) {
+          return java.util.Collections.unmodifiableList(targets_);
+        } else {
+          return targetsBuilder_.getMessageList();
+        }
+      }
+      public int getTargetsCount() {
+        if (targetsBuilder_ == null) {
+          return targets_.size();
+        } else {
+          return targetsBuilder_.getCount();
+        }
+      }
+      public org.apache.hadoop.hdfs.protocol.proto.HdfsProtos.DatanodeInfoProto getTargets(int index) {
+        if (targetsBuilder_ == null) {
+          return targets_.get(index);
+        } else {
+          return targetsBuilder_.getMessage(index);
+        }
+      }
+      public Builder setTargets(
+          int index, org.apache.hadoop.hdfs.protocol.proto.HdfsProtos.DatanodeInfoProto value) {
+        if (targetsBuilder_ == null) {
+          if (value == null) {
+            throw new NullPointerException();
+          }
+          ensureTargetsIsMutable();
+          targets_.set(index, value);
+          onChanged();
+        } else {
+          targetsBuilder_.setMessage(index, value);
+        }
+        return this;
+      }
+      public Builder setTargets(
+          int index, org.apache.hadoop.hdfs.protocol.proto.HdfsProtos.DatanodeInfoProto.Builder builderForValue) {
+        if (targetsBuilder_ == null) {
+          ensureTargetsIsMutable();
+          targets_.set(index, builderForValue.build());
+          onChanged();
+        } else {
+          targetsBuilder_.setMessage(index, builderForValue.build());
+        }
+        return this;
+      }
+      public Builder addTargets(org.apache.hadoop.hdfs.protocol.proto.HdfsProtos.DatanodeInfoProto value) {
+        if (targetsBuilder_ == null) {
+          if (value == null) {
+            throw new NullPointerException();
+          }
+          ensureTargetsIsMutable();
+          targets_.add(value);
+          onChanged();
+        } else {
+          targetsBuilder_.addMessage(value);
+        }
+        return this;
+      }
+      public Builder addTargets(
+          int index, org.apache.hadoop.hdfs.protocol.proto.HdfsProtos.DatanodeInfoProto value) {
+        if (targetsBuilder_ == null) {
+          if (value == null) {
+            throw new NullPointerException();
+          }
+          ensureTargetsIsMutable();
+          targets_.add(index, value);
+          onChanged();
+        } else {
+          targetsBuilder_.addMessage(index, value);
+        }
+        return this;
+      }
+      public Builder addTargets(
+          org.apache.hadoop.hdfs.protocol.proto.HdfsProtos.DatanodeInfoProto.Builder builderForValue) {
+        if (targetsBuilder_ == null) {
+          ensureTargetsIsMutable();
+          targets_.add(builderForValue.build());
+          onChanged();
+        } else {
+          targetsBuilder_.addMessage(builderForValue.build());
+        }
+        return this;
+      }
+      public Builder addTargets(
+          int index, org.apache.hadoop.hdfs.protocol.proto.HdfsProtos.DatanodeInfoProto.Builder builderForValue) {
+        if (targetsBuilder_ == null) {
+          ensureTargetsIsMutable();
+          targets_.add(index, builderForValue.build());
+          onChanged();
+        } else {
+          targetsBuilder_.addMessage(index, builderForValue.build());
+        }
+        return this;
+      }
+      public Builder addAllTargets(
+          java.lang.Iterable<? extends org.apache.hadoop.hdfs.protocol.proto.HdfsProtos.DatanodeInfoProto> values) {
+        if (targetsBuilder_ == null) {
+          ensureTargetsIsMutable();
+          super.addAll(values, targets_);
+          onChanged();
+        } else {
+          targetsBuilder_.addAllMessages(values);
+        }
+        return this;
+      }
+      public Builder clearTargets() {
+        if (targetsBuilder_ == null) {
+          targets_ = java.util.Collections.emptyList();
+          bitField0_ = (bitField0_ & ~0x00000002);
+          onChanged();
+        } else {
+          targetsBuilder_.clear();
+        }
+        return this;
+      }
+      public Builder removeTargets(int index) {
+        if (targetsBuilder_ == null) {
+          ensureTargetsIsMutable();
+          targets_.remove(index);
+          onChanged();
+        } else {
+          targetsBuilder_.remove(index);
+        }
+        return this;
+      }
+      public org.apache.hadoop.hdfs.protocol.proto.HdfsProtos.DatanodeInfoProto.Builder getTargetsBuilder(
+          int index) {
+        return getTargetsFieldBuilder().getBuilder(index);
+      }
+      public org.apache.hadoop.hdfs.protocol.proto.HdfsProtos.DatanodeInfoProtoOrBuilder getTargetsOrBuilder(
+          int index) {
+        if (targetsBuilder_ == null) {
+          return targets_.get(index);  } else {
+          return targetsBuilder_.getMessageOrBuilder(index);
+        }
+      }
+      public java.util.List<? extends org.apache.hadoop.hdfs.protocol.proto.HdfsProtos.DatanodeInfoProtoOrBuilder> 
+           getTargetsOrBuilderList() {
+        if (targetsBuilder_ != null) {
+          return targetsBuilder_.getMessageOrBuilderList();
+        } else {
+          return java.util.Collections.unmodifiableList(targets_);
+        }
+      }
+      public org.apache.hadoop.hdfs.protocol.proto.HdfsProtos.DatanodeInfoProto.Builder addTargetsBuilder() {
+        return getTargetsFieldBuilder().addBuilder(
+            org.apache.hadoop.hdfs.protocol.proto.HdfsProtos.DatanodeInfoProto.getDefaultInstance());
+      }
+      public org.apache.hadoop.hdfs.protocol.proto.HdfsProtos.DatanodeInfoProto.Builder addTargetsBuilder(
+          int index) {
+        return getTargetsFieldBuilder().addBuilder(
+            index, org.apache.hadoop.hdfs.protocol.proto.HdfsProtos.DatanodeInfoProto.getDefaultInstance());
+      }
+      public java.util.List<org.apache.hadoop.hdfs.protocol.proto.HdfsProtos.DatanodeInfoProto.Builder> 
+           getTargetsBuilderList() {
+        return getTargetsFieldBuilder().getBuilderList();
+      }
+      private com.google.protobuf.RepeatedFieldBuilder<
+          org.apache.hadoop.hdfs.protocol.proto.HdfsProtos.DatanodeInfoProto, org.apache.hadoop.hdfs.protocol.proto.HdfsProtos.DatanodeInfoProto.Builder, org.apache.hadoop.hdfs.protocol.proto.HdfsProtos.DatanodeInfoProtoOrBuilder> 
+          getTargetsFieldBuilder() {
+        if (targetsBuilder_ == null) {
+          targetsBuilder_ = new com.google.protobuf.RepeatedFieldBuilder<
+              org.apache.hadoop.hdfs.protocol.proto.HdfsProtos.DatanodeInfoProto, org.apache.hadoop.hdfs.protocol.proto.HdfsProtos.DatanodeInfoProto.Builder, org.apache.hadoop.hdfs.protocol.proto.HdfsProtos.DatanodeInfoProtoOrBuilder>(
+                  targets_,
+                  ((bitField0_ & 0x00000002) == 0x00000002),
+                  getParentForChildren(),
+                  isClean());
+          targets_ = null;
+        }
+        return targetsBuilder_;
+      }
+      
+      // @@protoc_insertion_point(builder_scope:OpTransferBlockProto)
+    }
+    
+    static {
+      defaultInstance = new OpTransferBlockProto(true);
+      defaultInstance.initFields();
+    }
+    
+    // @@protoc_insertion_point(class_scope:OpTransferBlockProto)
+  }
+  
+  public interface OpReplaceBlockProtoOrBuilder
+      extends com.google.protobuf.MessageOrBuilder {
+    
+    // required .BaseHeaderProto header = 1;
+    boolean hasHeader();
+    org.apache.hadoop.hdfs.protocol.proto.DataTransferProtos.BaseHeaderProto getHeader();
+    org.apache.hadoop.hdfs.protocol.proto.DataTransferProtos.BaseHeaderProtoOrBuilder getHeaderOrBuilder();
+    
+    // required string delHint = 2;
+    boolean hasDelHint();
+    String getDelHint();
+    
+    // required .DatanodeInfoProto source = 3;
+    boolean hasSource();
+    org.apache.hadoop.hdfs.protocol.proto.HdfsProtos.DatanodeInfoProto getSource();
+    org.apache.hadoop.hdfs.protocol.proto.HdfsProtos.DatanodeInfoProtoOrBuilder getSourceOrBuilder();
+  }
+  public static final class OpReplaceBlockProto extends
+      com.google.protobuf.GeneratedMessage
+      implements OpReplaceBlockProtoOrBuilder {
+    // Use OpReplaceBlockProto.newBuilder() to construct.
+    private OpReplaceBlockProto(Builder builder) {
+      super(builder);
+    }
+    private OpReplaceBlockProto(boolean noInit) {}
+    
+    private static final OpReplaceBlockProto defaultInstance;
+    public static OpReplaceBlockProto getDefaultInstance() {
+      return defaultInstance;
+    }
+    
+    public OpReplaceBlockProto getDefaultInstanceForType() {
+      return defaultInstance;
+    }
+    
+    public static final com.google.protobuf.Descriptors.Descriptor
+        getDescriptor() {
+      return org.apache.hadoop.hdfs.protocol.proto.DataTransferProtos.internal_static_OpReplaceBlockProto_descriptor;
+    }
+    
+    protected com.google.protobuf.GeneratedMessage.FieldAccessorTable
+        internalGetFieldAccessorTable() {
+      return org.apache.hadoop.hdfs.protocol.proto.DataTransferProtos.internal_static_OpReplaceBlockProto_fieldAccessorTable;
+    }
+    
+    private int bitField0_;
+    // required .BaseHeaderProto header = 1;
+    public static final int HEADER_FIELD_NUMBER = 1;
+    private org.apache.hadoop.hdfs.protocol.proto.DataTransferProtos.BaseHeaderProto header_;
+    public boolean hasHeader() {
+      return ((bitField0_ & 0x00000001) == 0x00000001);
+    }
+    public org.apache.hadoop.hdfs.protocol.proto.DataTransferProtos.BaseHeaderProto getHeader() {
+      return header_;
+    }
+    public org.apache.hadoop.hdfs.protocol.proto.DataTransferProtos.BaseHeaderProtoOrBuilder getHeaderOrBuilder() {
+      return header_;
+    }
+    
+    // required string delHint = 2;
+    public static final int DELHINT_FIELD_NUMBER = 2;
+    private java.lang.Object delHint_;
+    public boolean hasDelHint() {
+      return ((bitField0_ & 0x00000002) == 0x00000002);
+    }
+    public String getDelHint() {
+      java.lang.Object ref = delHint_;
+      if (ref instanceof String) {
+        return (String) ref;
+      } else {
+        com.google.protobuf.ByteString bs = 
+            (com.google.protobuf.ByteString) ref;
+        String s = bs.toStringUtf8();
+        if (com.google.protobuf.Internal.isValidUtf8(bs)) {
+          delHint_ = s;
+        }
+        return s;
+      }
+    }
+    private com.google.protobuf.ByteString getDelHintBytes() {
+      java.lang.Object ref = delHint_;
+      if (ref instanceof String) {
+        com.google.protobuf.ByteString b = 
+            com.google.protobuf.ByteString.copyFromUtf8((String) ref);
+        delHint_ = b;
+        return b;
+      } else {
+        return (com.google.protobuf.ByteString) ref;
+      }
+    }
+    
+    // required .DatanodeInfoProto source = 3;
+    public static final int SOURCE_FIELD_NUMBER = 3;
+    private org.apache.hadoop.hdfs.protocol.proto.HdfsProtos.DatanodeInfoProto source_;
+    public boolean hasSource() {
+      return ((bitField0_ & 0x00000004) == 0x00000004);
+    }
+    public org.apache.hadoop.hdfs.protocol.proto.HdfsProtos.DatanodeInfoProto getSource() {
+      return source_;
+    }
+    public org.apache.hadoop.hdfs.protocol.proto.HdfsProtos.DatanodeInfoProtoOrBuilder getSourceOrBuilder() {
+      return source_;
+    }
+    
+    private void initFields() {
+      header_ = org.apache.hadoop.hdfs.protocol.proto.DataTransferProtos.BaseHeaderProto.getDefaultInstance();
+      delHint_ = "";
+      source_ = org.apache.hadoop.hdfs.protocol.proto.HdfsProtos.DatanodeInfoProto.getDefaultInstance();
+    }
+    private byte memoizedIsInitialized = -1;
+    public final boolean isInitialized() {
+      byte isInitialized = memoizedIsInitialized;
+      if (isInitialized != -1) return isInitialized == 1;
+      
+      if (!hasHeader()) {
+        memoizedIsInitialized = 0;
+        return false;
+      }
+      if (!hasDelHint()) {
+        memoizedIsInitialized = 0;
+        return false;
+      }
+      if (!hasSource()) {
+        memoizedIsInitialized = 0;
+        return false;
+      }
+      if (!getHeader().isInitialized()) {
+        memoizedIsInitialized = 0;
+        return false;
+      }
+      if (!getSource().isInitialized()) {
+        memoizedIsInitialized = 0;
+        return false;
+      }
+      memoizedIsInitialized = 1;
+      return true;
+    }
+    
+    public void writeTo(com.google.protobuf.CodedOutputStream output)
+                        throws java.io.IOException {
+      getSerializedSize();
+      if (((bitField0_ & 0x00000001) == 0x00000001)) {
+        output.writeMessage(1, header_);
+      }
+      if (((bitField0_ & 0x00000002) == 0x00000002)) {
+        output.writeBytes(2, getDelHintBytes());
+      }
+      if (((bitField0_ & 0x00000004) == 0x00000004)) {
+        output.writeMessage(3, source_);
+      }
+      getUnknownFields().writeTo(output);
+    }
+    
+    private int memoizedSerializedSize = -1;
+    public int getSerializedSize() {
+      int size = memoizedSerializedSize;
+      if (size != -1) return size;
+    
+      size = 0;
+      if (((bitField0_ & 0x00000001) == 0x00000001)) {
+        size += com.google.protobuf.CodedOutputStream
+          .computeMessageSize(1, header_);
+      }
+      if (((bitField0_ & 0x00000002) == 0x00000002)) {
+        size += com.google.protobuf.CodedOutputStream
+          .computeBytesSize(2, getDelHintBytes());
+      }
+      if (((bitField0_ & 0x00000004) == 0x00000004)) {
+        size += com.google.protobuf.CodedOutputStream
+          .computeMessageSize(3, source_);
+      }
+      size += getUnknownFields().getSerializedSize();
+      memoizedSerializedSize = size;
+      return size;
+    }
+    
+    private static final long serialVersionUID = 0L;
+    @java.lang.Override
+    protected java.lang.Object writeReplace()
+        throws java.io.ObjectStreamException {
+      return super.writeReplace();
+    }
+    
+    @java.lang.Override
+    public boolean equals(final java.lang.Object obj) {
+      if (obj == this) {
+       return true;
+      }
+      if (!(obj instanceof org.apache.hadoop.hdfs.protocol.proto.DataTransferProtos.OpReplaceBlockProto)) {
+        return super.equals(obj);
+      }
+      org.apache.hadoop.hdfs.protocol.proto.DataTransferProtos.OpReplaceBlockProto other = (org.apache.hadoop.hdfs.protocol.proto.DataTransferProtos.OpReplaceBlockProto) obj;
+      
+      boolean result = true;
+      result = result && (hasHeader() == other.hasHeader());
+      if (hasHeader()) {
+        result = result && getHeader()
+            .equals(other.getHeader());
+      }
+      result = result && (hasDelHint() == other.hasDelHint());
+      if (hasDelHint()) {
+        result = result && getDelHint()
+            .equals(other.getDelHint());
+      }
+      result = result && (hasSource() == other.hasSource());
+      if (hasSource()) {
+        result = result && getSource()
+            .equals(other.getSource());
+      }
+      result = result &&
+          getUnknownFields().equals(other.getUnknownFields());
+      return result;
+    }
+    
+    @java.lang.Override
+    public int hashCode() {
+      int hash = 41;
+      hash = (19 * hash) + getDescriptorForType().hashCode();
+      if (hasHeader()) {
+        hash = (37 * hash) + HEADER_FIELD_NUMBER;
+        hash = (53 * hash) + getHeader().hashCode();
+      }
+      if (hasDelHint()) {
+        hash = (37 * hash) + DELHINT_FIELD_NUMBER;
+        hash = (53 * hash) + getDelHint().hashCode();
+      }
+      if (hasSource()) {
+        hash = (37 * hash) + SOURCE_FIELD_NUMBER;
+        hash = (53 * hash) + getSource().hashCode();
+      }
+      hash = (29 * hash) + getUnknownFields().hashCode();
+      return hash;
+    }
+    
+    public static org.apache.hadoop.hdfs.protocol.proto.DataTransferProtos.OpReplaceBlockProto parseFrom(
+        com.google.protobuf.ByteString data)
+        throws com.google.protobuf.InvalidProtocolBufferException {
+      return newBuilder().mergeFrom(data).buildParsed();
+    }
+    public static org.apache.hadoop.hdfs.protocol.proto.DataTransferProtos.OpReplaceBlockProto parseFrom(
+        com.google.protobuf.ByteString data,
+        com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+        throws com.google.protobuf.InvalidProtocolBufferException {
+      return newBuilder().mergeFrom(data, extensionRegistry)
+               .buildParsed();
+    }
+    public static org.apache.hadoop.hdfs.protocol.proto.DataTransferProtos.OpReplaceBlockProto parseFrom(byte[] data)
+        throws com.google.protobuf.InvalidProtocolBufferException {
+      return newBuilder().mergeFrom(data).buildParsed();
+    }
+    public static org.apache.hadoop.hdfs.protocol.proto.DataTransferProtos.OpReplaceBlockProto parseFrom(
+        byte[] data,
+        com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+        throws com.google.protobuf.InvalidProtocolBufferException {
+      return newBuilder().mergeFrom(data, extensionRegistry)
+               .buildParsed();
+    }
+    public static org.apache.hadoop.hdfs.protocol.proto.DataTransferProtos.OpReplaceBlockProto parseFrom(java.io.InputStream input)
+        throws java.io.IOException {
+      return newBuilder().mergeFrom(input).buildParsed();
+    }
+    public static org.apache.hadoop.hdfs.protocol.proto.DataTransferProtos.OpReplaceBlockProto parseFrom(
+        java.io.InputStream input,
+        com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+        throws java.io.IOException {
+      return newBuilder().mergeFrom(input, extensionRegistry)
+               .buildParsed();
+    }
+    public static org.apache.hadoop.hdfs.protocol.proto.DataTransferProtos.OpReplaceBlockProto parseDelimitedFrom(java.io.InputStream input)
+        throws java.io.IOException {
+      Builder builder = newBuilder();
+      if (builder.mergeDelimitedFrom(input)) {
+        return builder.buildParsed();
+      } else {
+        return null;
+      }
+    }
+    public static org.apache.hadoop.hdfs.protocol.proto.DataTransferProtos.OpReplaceBlockProto parseDelimitedFrom(
+        java.io.InputStream input,
+        com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+        throws java.io.IOException {
+      Builder builder = newBuilder();
+      if (builder.mergeDelimitedFrom(input, extensionRegistry)) {
+        return builder.buildParsed();
+      } else {
+        return null;
+      }
+    }
+    public static org.apache.hadoop.hdfs.protocol.proto.DataTransferProtos.OpReplaceBlockProto parseFrom(
+        com.google.protobuf.CodedInputStream input)
+        throws java.io.IOException {
+      return newBuilder().mergeFrom(input).buildParsed();
+    }
+    public static org.apache.hadoop.hdfs.protocol.proto.DataTransferProtos.OpReplaceBlockProto parseFrom(
+        com.google.protobuf.CodedInputStream input,
+        com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+        throws java.io.IOException {
+      return newBuilder().mergeFrom(input, extensionRegistry)
+               .buildParsed();
+    }
+    
+    public static Builder newBuilder() { return Builder.create(); }
+    public Builder newBuilderForType() { return newBuilder(); }
+    public static Builder newBuilder(org.apache.hadoop.hdfs.protocol.proto.DataTransferProtos.OpReplaceBlockProto prototype) {
+      return newBuilder().mergeFrom(prototype);
+    }
+    public Builder toBuilder() { return newBuilder(this); }
+    
+    @java.lang.Override
+    protected Builder newBuilderForType(
+        com.google.protobuf.GeneratedMessage.BuilderParent parent) {
+      Builder builder = new Builder(parent);
+      return builder;
+    }
+    public static final class Builder extends
+        com.google.protobuf.GeneratedMessage.Builder<Builder>
+       implements org.apache.hadoop.hdfs.protocol.proto.DataTransferProtos.OpReplaceBlockProtoOrBuilder {
+      public static final com.google.protobuf.Descriptors.Descriptor
+          getDescriptor() {
+        return org.apache.hadoop.hdfs.protocol.proto.DataTransferProtos.internal_static_OpReplaceBlockProto_descriptor;
+      }
+      
+      protected com.google.protobuf.GeneratedMessage.FieldAccessorTable
+          internalGetFieldAccessorTable() {
+        return org.apache.hadoop.hdfs.protocol.proto.DataTransferProtos.internal_static_OpReplaceBlockProto_fieldAccessorTable;
+      }
+      
+      // Construct using org.apache.hadoop.hdfs.protocol.proto.DataTransferProtos.OpReplaceBlockProto.newBuilder()
+      private Builder() {
+        maybeForceBuilderInitialization();
+      }
+      
+      private Builder(BuilderParent parent) {
+        super(parent);
+        maybeForceBuilderInitialization();
+      }
+      private void maybeForceBuilderInitialization() {
+        if (com.google.protobuf.GeneratedMessage.alwaysUseFieldBuilders) {
+          getHeaderFieldBuilder();
+          getSourceFieldBuilder();
+        }
+      }
+      private static Builder create() {
+        return new Builder();
+      }
+      
+      public Builder clear() {
+        super.clear();
+        if (headerBuilder_ == null) {
+          header_ = org.apache.hadoop.hdfs.protocol.proto.DataTransferProtos.BaseHeaderProto.getDefaultInstance();
+        } else {
+          headerBuilder_.clear();
+        }
+        bitField0_ = (bitField0_ & ~0x00000001);
+        delHint_ = "";
+        bitField0_ = (bitField0_ & ~0x00000002);
+        if (sourceBuilder_ == null) {
+          source_ = org.apache.hadoop.hdfs.protocol.proto.HdfsProtos.DatanodeInfoProto.getDefaultInstance();
+        } else {
+          sourceBuilder_.clear();
+        }
+        bitField0_ = (bitField0_ & ~0x00000004);
+        return this;
+      }
+      
+      public Builder clone() {
+        return create().mergeFrom(buildPartial());
+      }
+      
+      public com.google.protobuf.Descriptors.Descriptor
+          getDescriptorForType() {
+        return org.apache.hadoop.hdfs.protocol.proto.DataTransferProtos.OpReplaceBlockProto.getDescriptor();
+      }
+      
+      public org.apache.hadoop.hdfs.protocol.proto.DataTransferProtos.OpReplaceBlockProto getDefaultInstanceForType() {
+        return org.apache.hadoop.hdfs.protocol.proto.DataTransferProtos.OpReplaceBlockProto.getDefaultInstance();
+      }
+      
+      public org.apache.hadoop.hdfs.protocol.proto.DataTransferProtos.OpReplaceBlockProto build() {
+        org.apache.hadoop.hdfs.protocol.proto.DataTransferProtos.OpReplaceBlockProto result = buildPartial();
+        if (!result.isInitialized()) {
+          throw newUninitializedMessageException(result);
+        }
+        return result;
+      }
+      
+      private org.apache.hadoop.hdfs.protocol.proto.DataTransferProtos.OpReplaceBlockProto buildParsed()
+          throws com.google.protobuf.InvalidProtocolBufferException {
+        org.apache.hadoop.hdfs.protocol.proto.DataTransferProtos.OpReplaceBlockProto result = buildPartial();
+        if (!result.isInitialized()) {
+          throw newUninitializedMessageException(
+            result).asInvalidProtocolBufferException();
+        }
+        return result;
+      }
+      
+      public org.apache.hadoop.hdfs.protocol.proto.DataTransferProtos.OpReplaceBlockProto buildPartial() {
+        org.apache.hadoop.hdfs.protocol.proto.DataTransferProtos.OpReplaceBlockProto result = new org.apache.hadoop.hdfs.protocol.proto.DataTransferProtos.OpReplaceBlockProto(this);
+        int from_bitField0_ = bitField0_;
+        int to_bitField0_ = 0;
+        if (((from_bitField0_ & 0x00000001) == 0x00000001)) {
+          to_bitField0_ |= 0x00000001;
+        }
+        if (headerBuilder_ == null) {
+          result.header_ = header_;
+        } else {
+          result.header_ = headerBuilder_.build();
+        }
+        if (((from_bitField0_ & 0x00000002) == 0x00000002)) {
+          to_bitField0_ |= 0x00000002;
+        }
+        result.delHint_ = delHint_;
+        if (((from_bitField0_ & 0x00000004) == 0x00000004)) {
+          to_bitField0_ |= 0x00000004;
+        }
+        if (sourceBuilder_ == null) {
+          result.source_ = source_;
+        } else {
+          result.source_ = sourceBuilder_.build();
+        }
+        result.bitField0_ = to_bitField0_;
+        onBuilt();
+        return result;
+      }
+      
+      public Builder mergeFrom(com.google.protobuf.Message other) {
+        if (other instanceof org.apache.hadoop.hdfs.protocol.proto.DataTransferProtos.OpReplaceBlockProto) {
+          return mergeFrom((org.apache.hadoop.hdfs.protocol.proto.DataTransferProtos.OpReplaceBlockProto)other);
+        } else {
+          super.mergeFrom(other);
+          return this;
+        }
+      }
+      
+      public Builder mergeFrom(org.apache.hadoop.hdfs.protocol.proto.DataTransferProtos.OpReplaceBlockProto other) {
+        if (other == org.apache.hadoop.hdfs.protocol.proto.DataTransferProtos.OpReplaceBlockProto.getDefaultInstance()) return this;
+        if (other.hasHeader()) {
+          mergeHeader(other.getHeader());
+        }
+        if (other.hasDelHint()) {
+          setDelHint(other.getDelHint());
+        }
+        if (other.hasSource()) {
+          mergeSource(other.getSource());
+        }
+        this.mergeUnknownFields(other.getUnknownFields());
+        return this;
+      }
+      
+      public final boolean isInitialized() {
+        if (!hasHeader()) {
+          
+          return false;
+        }
+        if (!hasDelHint()) {
+          
+          return false;
+        }
+        if (!hasSource()) {
+          
+          return false;
+        }
+        if (!getHeader().isInitialized()) {
+          
+          return false;
+        }
+        if (!getSource().isInitialized()) {
+          
+          return false;
+        }
+        return true;
+      }
+      
+      public Builder mergeFrom(
+          com.google.protobuf.CodedInputStream input,
+          com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+          throws java.io.IOException {
+        com.google.protobuf.UnknownFieldSet.Builder unknownFields =
+          com.google.protobuf.UnknownFieldSet.newBuilder(
+            this.getUnknownFields());
+        while (true) {
+          int tag = input.readTag();
+          switch (tag) {
+            case 0:
+              this.setUnknownFields(unknownFields.build());
+              onChanged();
+              return this;
+            default: {
+              if (!parseUnknownField(input, unknownFields,
+                                     extensionRegistry, tag)) {
+                this.setUnknownFields(unknownFields.build());
+                onChanged();
+                return this;
+              }
+              break;
+            }
+            case 10: {
+              org.apache.hadoop.hdfs.protocol.proto.DataTransferProtos.BaseHeaderProto.Builder subBuilder = org.apache.hadoop.hdfs.protocol.proto.DataTransferProtos.BaseHeaderProto.newBuilder();
+              if (hasHeader()) {
+                subBuilder.mergeFrom(getHeader());
+              }
+              input.readMessage(subBuilder, extensionRegistry);
+              setHeader(subBuilder.buildPartial());
+              break;
+            }
+            case 18: {
+              bitField0_ |= 0x00000002;
+              delHint_ = input.readBytes();
+              break;
+            }
+            case 26: {
+              org.apache.hadoop.hdfs.protocol.proto.HdfsProtos.DatanodeInfoProto.Builder subBuilder = org.apache.hadoop.hdfs.protocol.proto.HdfsProtos.DatanodeInfoProto.newBuilder();
+              if (hasSource()) {
+                subBuilder.mergeFrom(getSource());
+              }
+              input.readMessage(subBuilder, extensionRegistry);
+              setSource(subBuilder.buildPartial());
+              break;
+            }
+          }
+        }
+      }
+      
+      private int bitField0_;
+      
+      // required .BaseHeaderProto header = 1;
+      private org.apache.hadoop.hdfs.protocol.proto.DataTransferProtos.BaseHeaderProto header_ = org.apache.hadoop.hdfs.protocol.proto.DataTransferProtos.BaseHeaderProto.getDefaultInstance();
+      private com.google.protobuf.SingleFieldBuilder<
+          org.apache.hadoop.hdfs.protocol.proto.DataTransferProtos.BaseHeaderProto, org.apache.hadoop.hdfs.protocol.proto.DataTransferProtos.BaseHeaderProto.Builder, org.apache.hadoop.hdfs.protocol.proto.DataTransferProtos.BaseHeaderProtoOrBuilder> headerBuilder_;
+      public boolean hasHeader() {
+        return ((bitField0_ & 0x00000001) == 0x00000001);
+      }
+      public org.apache.hadoop.hdfs.protocol.proto.DataTransferProtos.BaseHeaderProto getHeader() {
+        if (headerBuilder_ == null) {
+          return header_;
+        } else {
+          return headerBuilder_.getMessage();
+        }
+      }
+      public Builder setHeader(org.apache.hadoop.hdfs.protocol.proto.DataTransferProtos.BaseHeaderProto value) {
+        if (headerBuilder_ == null) {
+          if (value == null) {
+            throw new NullPointerException();
+          }
+          header_ = value;
+          onChanged();
+        } else {
+          headerBuilder_.setMessage(value);
+        }
+        bitField0_ |= 0x00000001;
+        return this;
+      }
+      public Builder setHeader(
+          org.apache.hadoop.hdfs.protocol.proto.DataTransferProtos.BaseHeaderProto.Builder builderForValue) {
+        if (headerBuilder_ == null) {
+          header_ = builderForValue.build();
+          onChanged();
+        } else {
+          headerBuilder_.setMessage(builderForValue.build());
+        }
+        bitField0_ |= 0x00000001;
+        return this;
+      }
+      public Builder mergeHeader(org.apache.hadoop.hdfs.protocol.proto.DataTransferProtos.BaseHeaderProto value) {
+        if (headerBuilder_ == null) {
+          if (((bitField0_ & 0x00000001) == 0x00000001) &&
+              header_ != org.apache.hadoop.hdfs.protocol.proto.DataTransferProtos.BaseHeaderProto.getDefaultInstance()) {
+            header_ =
+              org.apache.hadoop.hdfs.protocol.proto.DataTransferProtos.BaseHeaderProto.newBuilder(header_).mergeFrom(value).buildPartial();
+          } else {
+            header_ = value;
+          }
+          onChanged();
+        } else {
+          headerBuilder_.mergeFrom(value);
+        }
+        bitField0_ |= 0x00000001;
+        return this;
+      }
+      public Builder clearHeader() {
+        if (headerBuilder_ == null) {
+          header_ = org.apache.hadoop.hdfs.protocol.proto.DataTransferProtos.BaseHeaderProto.getDefaultInstance();
+          onChanged();
+        } else {
+          headerBuilder_.clear();
+        }
+        bitField0_ = (bitField0_ & ~0x00000001);
+        return this;
+      }
+      public org.apache.hadoop.hdfs.protocol.proto.DataTransferProtos.BaseHeaderProto.Builder getHeaderBuilder() {
+        bitField0_ |= 0x00000001;
+        onChanged();
+        return getHeaderFieldBuilder().getBuilder();
+      }
+      public org.apache.hadoop.hdfs.protocol.proto.DataTransferProtos.BaseHeaderProtoOrBuilder getHeaderOrBuilder() {
+        if (headerBuilder_ != null) {
+          return headerBuilder_.getMessageOrBuilder();
+        } else {
+          return header_;
+        }
+      }
+      private com.google.protobuf.SingleFieldBuilder<
+          org.apache.hadoop.hdfs.protocol.proto.DataTransferProtos.BaseHeaderProto, org.apache.hadoop.hdfs.protocol.proto.DataTransferProtos.BaseHeaderProto.Builder, org.apache.hadoop.hdfs.protocol.proto.DataTransferProtos.BaseHeaderProtoOrBuilder> 
+          getHeaderFieldBuilder() {
+        if (headerBuilder_ == null) {
+          headerBuilder_ = new com.google.protobuf.SingleFieldBuilder<
+              org.apache.hadoop.hdfs.protocol.proto.DataTransferProtos.BaseHeaderProto, org.apache.hadoop.hdfs.protocol.proto.DataTransferProtos.BaseHeaderProto.Builder, org.apache.hadoop.hdfs.protocol.proto.DataTransferProtos.BaseHeaderProtoOrBuilder>(
+                  header_,
+                  getParentForChildren(),
+                  isClean());
+          header_ = null;
+        }
+        return headerBuilder_;
+      }
+      
+      // required string delHint = 2;
+      private java.lang.Object delHint_ = "";
+      public boolean hasDelHint() {
+        return ((bitField0_ & 0x00000002) == 0x00000002);
+      }
+      public String getDelHint() {
+        java.lang.Object ref = delHint_;
+        if (!(ref instanceof String)) {
+          String s = ((com.google.protobuf.ByteString) ref).toStringUtf8();
+          delHint_ = s;
+          return s;
+        } else {
+          return (String) ref;
+        }
+      }
+      public Builder setDelHint(String value) {
+        if (value == null) {
+    throw new NullPointerException();
+  }
+  bitField0_ |= 0x00000002;
+        delHint_ = value;
+        onChanged();
+        return this;
+      }
+      public Builder clearDelHint() {
+        bitField0_ = (bitField0_ & ~0x00000002);
+        delHint_ = getDefaultInstance().getDelHint();
+        onChanged();
+        return this;
+      }
+      void setDelHint(com.google.protobuf.ByteString value) {
+        bitField0_ |= 0x00000002;
+        delHint_ = value;
+        onChanged();
+      }
+      
+      // required .DatanodeInfoProto source = 3;
+      private org.apache.hadoop.hdfs.protocol.proto.HdfsProtos.DatanodeInfoProto source_ = org.apache.hadoop.hdfs.protocol.proto.HdfsProtos.DatanodeInfoProto.getDefaultInstance();
+      private com.google.protobuf.SingleFieldBuilder<
+          org.apache.hadoop.hdfs.protocol.proto.HdfsProtos.DatanodeInfoProto, org.apache.hadoop.hdfs.protocol.proto.HdfsProtos.DatanodeInfoProto.Builder, org.apache.hadoop.hdfs.protocol.proto.HdfsProtos.DatanodeInfoProtoOrBuilder> sourceBuilder_;
+      public boolean hasSource() {
+        return ((bitField0_ & 0x00000004) == 0x00000004);
+      }
+      public org.apache.hadoop.hdfs.protocol.proto.HdfsProtos.DatanodeInfoProto getSource() {
+        if (sourceBuilder_ == null) {
+          return source_;
+        } else {
+          return sourceBuilder_.getMessage();
+        }
+      }
+      public Builder setSource(org.apache.hadoop.hdfs.protocol.proto.HdfsProtos.DatanodeInfoProto value) {
+        if (sourceBuilder_ == null) {
+          if (value == null) {
+            throw new NullPointerException();
+          }
+          source_ = value;
+          onChanged();
+        } else {
+          sourceBuilder_.setMessage(value);
+        }
+        bitField0_ |= 0x00000004;
+        return this;
+      }
+      public Builder setSource(
+          org.apache.hadoop.hdfs.protocol.proto.HdfsProtos.DatanodeInfoProto.Builder builderForValue) {
+        if (sourceBuilder_ == null) {
+          source_ = builderForValue.build();
+          onChanged();
+        } else {
+          sourceBuilder_.setMessage(builderForValue.build());
+        }
+        bitField0_ |= 0x00000004;
+        return this;
+      }
+      public Builder mergeSource(org.apache.hadoop.hdfs.protocol.proto.HdfsProtos.DatanodeInfoProto value) {
+        if (sourceBuilder_ == null) {
+          if (((bitField0_ & 0x00000004) == 0x00000004) &&
+              source_ != org.apache.hadoop.hdfs.protocol.proto.HdfsProtos.DatanodeInfoProto.getDefaultInstance()) {
+            source_ =
+              org.apache.hadoop.hdfs.protocol.proto.HdfsProtos.DatanodeInfoProto.newBuilder(source_).mergeFrom(value).buildPartial();
+          } else {
+            source_ = value;
+          }
+          onChanged();
+        } else {
+          sourceBuilder_.mergeFrom(value);
+        }
+        bitField0_ |= 0x00000004;
+        return this;
+      }
+      public Builder clearSource() {
+        if (sourceBuilder_ == null) {
+          source_ = org.apache.hadoop.hdfs.protocol.proto.HdfsProtos.DatanodeInfoProto.getDefaultInstance();
+          onChanged();
+        } else {
+          sourceBuilder_.clear();
+        }
+        bitField0_ = (bitField0_ & ~0x00000004);
+        return this;
+      }
+      public org.apache.hadoop.hdfs.protocol.proto.HdfsProtos.DatanodeInfoProto.Builder getSourceBuilder() {
+        bitField0_ |= 0x00000004;
+        onChanged();
+        return getSourceFieldBuilder().getBuilder();
+      }
+      public org.apache.hadoop.hdfs.protocol.proto.HdfsProtos.DatanodeInfoProtoOrBuilder getSourceOrBuilder() {
+        if (sourceBuilder_ != null) {
+          return sourceBuilder_.getMessageOrBuilder();
+        } else {
+          return source_;
+        }
+      }
+      private com.google.protobuf.SingleFieldBuilder<
+          org.apache.hadoop.hdfs.protocol.proto.HdfsProtos.DatanodeInfoProto, org.apache.hadoop.hdfs.protocol.proto.HdfsProtos.DatanodeInfoProto.Builder, org.apache.hadoop.hdfs.protocol.proto.HdfsProtos.DatanodeInfoProtoOrBuilder> 
+          getSourceFieldBuilder() {
+        if (sourceBuilder_ == null) {
+          sourceBuilder_ = new com.google.protobuf.SingleFieldBuilder<
+              org.apache.hadoop.hdfs.protocol.proto.HdfsProtos.DatanodeInfoProto, org.apache.hadoop.hdfs.protocol.proto.HdfsProtos.DatanodeInfoProto.Builder, org.apache.hadoop.hdfs.protocol.proto.HdfsProtos.DatanodeInfoProtoOrBuilder>(
+                  source_,
+                  getParentForChildren(),
+                  isClean());
+          source_ = null;
+        }
+        return sourceBuilder_;
+      }
+      
+      // @@protoc_insertion_point(builder_scope:OpReplaceBlockProto)
+    }
+    
+    static {
+      defaultInstance = new OpReplaceBlockProto(true);
+      defaultInstance.initFields();
+    }
+    
+    // @@protoc_insertion_point(class_scope:OpReplaceBlockProto)
+  }
+  
+  public interface OpCopyBlockProtoOrBuilder
+      extends com.google.protobuf.MessageOrBuilder {
+    
+    // required .BaseHeaderProto header = 1;
+    boolean hasHeader();
+    org.apache.hadoop.hdfs.protocol.proto.DataTransferProtos.BaseHeaderProto getHeader();
+    org.apache.hadoop.hdfs.protocol.proto.DataTransferProtos.BaseHeaderProtoOrBuilder getHeaderOrBuilder();
+  }
+  public static final class OpCopyBlockProto extends
+      com.google.protobuf.GeneratedMessage
+      implements OpCopyBlockProtoOrBuilder {
+    // Use OpCopyBlockProto.newBuilder() to construct.
+    private OpCopyBlockProto(Builder builder) {
+      super(builder);
+    }
+    private OpCopyBlockProto(boolean noInit) {}
+    
+    private static final OpCopyBlockProto defaultInstance;
+    public static OpCopyBlockProto getDefaultInstance() {
+      return defaultInstance;
+    }
+    
+    public OpCopyBlockProto getDefaultInstanceForType() {
+      return defaultInstance;
+    }
+    
+    public static final com.google.protobuf.Descriptors.Descriptor
+        getDescriptor() {
+      return org.apache.hadoop.hdfs.protocol.proto.DataTransferProtos.internal_static_OpCopyBlockProto_descriptor;
+    }
+    
+    protected com.google.protobuf.GeneratedMessage.FieldAccessorTable
+        internalGetFieldAccessorTable() {
+      return org.apache.hadoop.hdfs.protocol.proto.DataTransferProtos.internal_static_OpCopyBlockProto_fieldAccessorTable;
+    }
+    
+    private int bitField0_;
+    // required .BaseHeaderProto header = 1;
+    public static final int HEADER_FIELD_NUMBER = 1;
+    private org.apache.hadoop.hdfs.protocol.proto.DataTransferProtos.BaseHeaderProto header_;
+    public boolean hasHeader() {
+      return ((bitField0_ & 0x00000001) == 0x00000001);
+    }
+    public org.apache.hadoop.hdfs.protocol.proto.DataTransferProtos.BaseHeaderProto getHeader() {
+      return header_;
+    }
+    public org.apache.hadoop.hdfs.protocol.proto.DataTransferProtos.BaseHeaderProtoOrBuilder getHeaderOrBuilder() {
+      return header_;
+    }
+    
+    private void initFields() {
+      header_ = org.apache.hadoop.hdfs.protocol.proto.DataTransferProtos.BaseHeaderProto.getDefaultInstance();
+    }
+    private byte memoizedIsInitialized = -1;
+    public final boolean isInitialized() {
+      byte isInitialized = memoizedIsInitialized;
+      if (isInitialized != -1) return isInitialized == 1;
+      
+      if (!hasHeader()) {
+        memoizedIsInitialized = 0;
+        return false;
+      }
+      if (!getHeader().isInitialized()) {
+        memoizedIsInitialized = 0;
+        return false;
+      }
+      memoizedIsInitialized = 1;
+      return true;
+    }
+    
+    public void writeTo(com.google.protobuf.CodedOutputStream output)
+                        throws java.io.IOException {
+      getSerializedSize();
+      if (((bitField0_ & 0x00000001) == 0x00000001)) {
+        output.writeMessage(1, header_);
+      }
+      getUnknownFields().writeTo(output);
+    }
+    
+    private int memoizedSerializedSize = -1;
+    public int getSerializedSize() {
+      int size = memoizedSerializedSize;
+      if (size != -1) return size;
+    
+      size = 0;
+      if (((bitField0_ & 0x00000001) == 0x00000001)) {
+        size += com.google.protobuf.CodedOutputStream
+          .computeMessageSize(1, header_);
+      }
+      size += getUnknownFields().getSerializedSize();
+      memoizedSerializedSize = size;
+      return size;
+    }
+    
+    private static final long serialVersionUID = 0L;
+    @java.lang.Override
+    protected java.lang.Object writeReplace()
+        throws java.io.ObjectStreamException {
+      return super.writeReplace();
+    }
+    
+    @java.lang.Override
+    public boolean equals(final java.lang.Object obj) {
+      if (obj == this) {
+       return true;
+      }
+      if (!(obj instanceof org.apache.hadoop.hdfs.protocol.proto.DataTransferProtos.OpCopyBlockProto)) {
+        return super.equals(obj);
+      }
+      org.apache.hadoop.hdfs.protocol.proto.DataTransferProtos.OpCopyBlockProto other = (org.apache.hadoop.hdfs.protocol.proto.DataTransferProtos.OpCopyBlockProto) obj;
+      
+      boolean result = true;
+      result = result && (hasHeader() == other.hasHeader());
+      if (hasHeader()) {
+        result = result && getHeader()
+            .equals(other.getHeader());
+      }
+      result = result &&
+          getUnknownFields().equals(other.getUnknownFields());
+      return result;
+    }
+    
+    @java.lang.Override
+    public int hashCode() {
+      int hash = 41;
+      hash = (19 * hash) + getDescriptorForType().hashCode();
+      if (hasHeader()) {
+        hash = (37 * hash) + HEADER_FIELD_NUMBER;
+        hash = (53 * hash) + getHeader().hashCode();
+      }
+      hash = (29 * hash) + getUnknownFields().hashCode();
+      return hash;
+    }
+    
+    public static org.apache.hadoop.hdfs.protocol.proto.DataTransferProtos.OpCopyBlockProto parseFrom(
+        com.google.protobuf.ByteString data)
+        throws com.google.protobuf.InvalidProtocolBufferException {
+      return newBuilder().mergeFrom(data).buildParsed();
+    }
+    public static org.apache.hadoop.hdfs.protocol.proto.DataTransferProtos.OpCopyBlockProto parseFrom(
+        com.google.protobuf.ByteString data,
+        com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+        throws com.google.protobuf.InvalidProtocolBufferException {
+      return newBuilder().mergeFrom(data, extensionRegistry)
+               .buildParsed();
+    }
+    public static org.apache.hadoop.hdfs.protocol.proto.DataTransferProtos.OpCopyBlockProto parseFrom(byte[] data)
+        throws com.google.protobuf.InvalidProtocolBufferException {
+      return newBuilder().mergeFrom(data).buildParsed();
+    }
+    public static org.apache.hadoop.hdfs.protocol.proto.DataTransferProtos.OpCopyBlockProto parseFrom(
+        byte[] data,
+        com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+        throws com.google.protobuf.InvalidProtocolBufferException {
+      return newBuilder().mergeFrom(data, extensionRegistry)
+               .buildParsed();
+    }
+    public static org.apache.hadoop.hdfs.protocol.proto.DataTransferProtos.OpCopyBlockProto parseFrom(java.io.InputStream input)
+        throws java.io.IOException {
+      return newBuilder().mergeFrom(input).buildParsed();
+    }
+    public static org.apache.hadoop.hdfs.protocol.proto.DataTransferProtos.OpCopyBlockProto parseFrom(
+        java.io.InputStream input,
+        com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+        throws java.io.IOException {
+      return newBuilder().mergeFrom(input, extensionRegistry)
+               .buildParsed();
+    }
+    public static org.apache.hadoop.hdfs.protocol.proto.DataTransferProtos.OpCopyBlockProto parseDelimitedFrom(java.io.InputStream input)
+        throws java.io.IOException {
+      Builder builder = newBuilder();
+      if (builder.mergeDelimitedFrom(input)) {
+        return builder.buildParsed();
+      } else {
+        return null;
+      }
+    }
+    public static org.apache.hadoop.hdfs.protocol.proto.DataTransferProtos.OpCopyBlockProto parseDelimitedFrom(
+        java.io.InputStream input,
+        com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+        throws java.io.IOException {
+      Builder builder = newBuilder();
+      if (builder.mergeDelimitedFrom(input, extensionRegistry)) {
+        return builder.buildParsed();
+      } else {
+        return null;
+      }
+    }
+    public static org.apache.hadoop.hdfs.protocol.proto.DataTransferProtos.OpCopyBlockProto parseFrom(
+        com.google.protobuf.CodedInputStream input)
+        throws java.io.IOException {
+      return newBuilder().mergeFrom(input).buildParsed();
+    }
+    public static org.apache.hadoop.hdfs.protocol.proto.DataTransferProtos.OpCopyBlockProto parseFrom(
+        com.google.protobuf.CodedInputStream input,
+        com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+        throws java.io.IOException {
+      return newBuilder().mergeFrom(input, extensionRegistry)
+               .buildParsed();
+    }
+    
+    public static Builder newBuilder() { return Builder.create(); }
+    public Builder newBuilderForType() { return newBuilder(); }
+    public static Builder newBuilder(org.apache.hadoop.hdfs.protocol.proto.DataTransferProtos.OpCopyBlockProto prototype) {
+      return newBuilder().mergeFrom(prototype);
+    }
+    public Builder toBuilder() { return newBuilder(this); }
+    
+    @java.lang.Override
+    protected Builder newBuilderForType(
+        com.google.protobuf.GeneratedMessage.BuilderParent parent) {
+      Builder builder = new Builder(parent);
+      return builder;
+    }
+    public static final class Builder extends
+        com.google.protobuf.GeneratedMessage.Builder<Builder>
+       implements org.apache.hadoop.hdfs.protocol.proto.DataTransferProtos.OpCopyBlockProtoOrBuilder {
+      public static final com.google.protobuf.Descriptors.Descriptor
+          getDescriptor() {
+        return org.apache.hadoop.hdfs.protocol.proto.DataTransferProtos.internal_static_OpCopyBlockProto_descriptor;
+      }
+      
+      protected com.google.protobuf.GeneratedMessage.FieldAccessorTable
+          internalGetFieldAccessorTable() {
+        return org.apache.hadoop.hdfs.protocol.proto.DataTransferProtos.internal_static_OpCopyBlockProto_fieldAccessorTable;
+      }
+      
+      // Construct using org.apache.hadoop.hdfs.protocol.proto.DataTransferProtos.OpCopyBlockProto.newBuilder()
+      private Builder() {
+        maybeForceBuilderInitialization();
+      }
+      
+      private Builder(BuilderParent parent) {
+        super(parent);
+        maybeForceBuilderInitialization();
+      }
+      private void maybeForceBuilderInitialization() {
+        if (com.google.protobuf.GeneratedMessage.alwaysUseFieldBuilders) {
+          getHeaderFieldBuilder();
+        }
+      }
+      private static Builder create() {
+        return new Builder();
+      }
+      
+      public Builder clear() {
+        super.clear();
+        if (headerBuilder_ == null) {
+          header_ = org.apache.hadoop.hdfs.protocol.proto.DataTransferProtos.BaseHeaderProto.getDefaultInstance();
+        } else {
+          headerBuilder_.clear();
+        }
+        bitField0_ = (bitField0_ & ~0x00000001);
+        return this;
+      }
+      
+      public Builder clone() {
+        return create().mergeFrom(buildPartial());
+      }
+      
+      public com.google.protobuf.Descriptors.Descriptor
+          getDescriptorForType() {
+        return org.apache.hadoop.hdfs.protocol.proto.DataTransferProtos.OpCopyBlockProto.getDescriptor();
+      }
+      
+      public org.apache.hadoop.hdfs.protocol.proto.DataTransferProtos.OpCopyBlockProto getDefaultInstanceForType() {
+        return org.apache.hadoop.hdfs.protocol.proto.DataTransferProtos.OpCopyBlockProto.getDefaultInstance();
+      }
+      
+      public org.apache.hadoop.hdfs.protocol.proto.DataTransferProtos.OpCopyBlockProto build() {
+        org.apache.hadoop.hdfs.protocol.proto.DataTransferProtos.OpCopyBlockProto result = buildPartial();
+        if (!result.isInitialized()) {
+          throw newUninitializedMessageException(result);
+        }
+        return result;
+      }
+      
+      private org.apache.hadoop.hdfs.protocol.proto.DataTransferProtos.OpCopyBlockProto buildParsed()
+          throws com.google.protobuf.InvalidProtocolBufferException {
+        org.apache.hadoop.hdfs.protocol.proto.DataTransferProtos.OpCopyBlockProto result = buildPartial();
+        if (!result.isInitialized()) {
+          throw newUninitializedMessageException(
+            result).asInvalidProtocolBufferException();
+        }
+        return result;
+      }
+      
+      public org.apache.hadoop.hdfs.protocol.proto.DataTransferProtos.OpCopyBlockProto buildPartial() {
+        org.apache.hadoop.hdfs.protocol.proto.DataTransferProtos.OpCopyBlockProto result = new org.apache.hadoop.hdfs.protocol.proto.DataTransferProtos.OpCopyBlockProto(this);
+        int from_bitField0_ = bitField0_;
+        int to_bitField0_ = 0;
+        if (((from_bitField0_ & 0x00000001) == 0x00000001)) {
+          to_bitField0_ |= 0x00000001;
+        }
+        if (headerBuilder_ == null) {
+          result.header_ = header_;
+        } else {
+          result.header_ = headerBuilder_.build();
+        }
+        result.bitField0_ = to_bitField0_;
+        onBuilt();
+        return result;
+      }
+      
+      public Builder mergeFrom(com.google.protobuf.Message other) {
+        if (other instanceof org.apache.hadoop.hdfs.protocol.proto.DataTransferProtos.OpCopyBlockProto) {
+          return mergeFrom((org.apache.hadoop.hdfs.protocol.proto.DataTransferProtos.OpCopyBlockProto)other);
+        } else {
+          super.mergeFrom(other);
+          return this;
+        }
+      }
+      
+      public Builder mergeFrom(org.apache.hadoop.hdfs.protocol.proto.DataTransferProtos.OpCopyBlockProto other) {
+        if (other == org.apache.hadoop.hdfs.protocol.proto.DataTransferProtos.OpCopyBlockProto.getDefaultInstance()) return this;
+        if (other.hasHeader()) {
+          mergeHeader(other.getHeader());
+        }
+        this.mergeUnknownFields(other.getUnknownFields());
+        return this;
+      }
+      
+      public final boolean isInitialized() {
+        if (!hasHeader()) {
+          
+          return false;
+        }
+        if (!getHeader().isInitialized()) {
+          
+          return false;
+        }
+        return true;
+      }
+      
+      public Builder mergeFrom(
+          com.google.protobuf.CodedInputStream input,
+          com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+          throws java.io.IOException {
+        com.google.protobuf.UnknownFieldSet.Builder unknownFields =
+          com.google.protobuf.UnknownFieldSet.newBuilder(
+            this.getUnknownFields());
+        while (true) {
+          int tag = input.readTag();
+          switch (tag) {
+            case 0:
+              this.setUnknownFields(unknownFields.build());
+              onChanged();
+              return this;
+            default: {
+              if (!parseUnknownField(input, unknownFields,
+                                     extensionRegistry, tag)) {
+                this.setUnknownFields(unknownFields.build());
+                onChanged();
+                return this;
+              }
+              break;
+            }
+            case 10: {
+              org.apache.hadoop.hdfs.protocol.proto.DataTransferProtos.BaseHeaderProto.Builder subBuilder = org.apache.hadoop.hdfs.protocol.proto.DataTransferProtos.BaseHeaderProto.newBuilder();
+              if (hasHeader()) {
+                subBuilder.mergeFrom(getHeader());
+              }
+              input.readMessage(subBuilder, extensionRegistry);
+              setHeader(subBuilder.buildPartial());
+              break;
+            }
+          }
+        }
+      }
+      
+      private int bitField0_;
+      
+      // required .BaseHeaderProto header = 1;
+      private org.apache.hadoop.hdfs.protocol.proto.DataTransferProtos.BaseHeaderProto header_ = org.apache.hadoop.hdfs.protocol.proto.DataTransferProtos.BaseHeaderProto.getDefaultInstance();
+      private com.google.protobuf.SingleFieldBuilder<
+          org.apache.hadoop.hdfs.protocol.proto.DataTransferProtos.BaseHeaderProto, org.apache.hadoop.hdfs.protocol.proto.DataTransferProtos.BaseHeaderProto.Builder, org.apache.hadoop.hdfs.protocol.proto.DataTransferProtos.BaseHeaderProtoOrBuilder> headerBuilder_;
+      public boolean hasHeader() {
+        return ((bitField0_ & 0x00000001) == 0x00000001);
+      }
+      public org.apache.hadoop.hdfs.protocol.proto.DataTransferProtos.BaseHeaderProto getHeader() {
+        if (headerBuilder_ == null) {
+          return header_;
+        } else {
+          return headerBuilder_.getMessage();
+        }
+      }
+      public Builder setHeader(org.apache.hadoop.hdfs.protocol.proto.DataTransferProtos.BaseHeaderProto value) {
+        if (headerBuilder_ == null) {
+          if (value == null) {
+            throw new NullPointerException();
+          }
+          header_ = value;
+          onChanged();
+        } else {
+          headerBuilder_.setMessage(value);
+        }
+        bitField0_ |= 0x00000001;
+        return this;
+      }
+      public Builder setHeader(
+          org.apache.hadoop.hdfs.protocol.proto.DataTransferProtos.BaseHeaderProto.Builder builderForValue) {
+        if (headerBuilder_ == null) {
+          header_ = builderForValue.build();
+          onChanged();
+        } else {
+          headerBuilder_.setMessage(builderForValue.build());
+        }
+        bitField0_ |= 0x00000001;
+        return this;
+      }
+      public Builder mergeHeader(org.apache.hadoop.hdfs.protocol.proto.DataTransferProtos.BaseHeaderProto value) {
+        if (headerBuilder_ == null) {
+          if (((bitField0_ & 0x00000001) == 0x00000001) &&
+              header_ != org.apache.hadoop.hdfs.protocol.proto.DataTransferProtos.BaseHeaderProto.getDefaultInstance()) {
+            header_ =
+              org.apache.hadoop.hdfs.protocol.proto.DataTransferProtos.BaseHeaderProto.newBuilder(header_).mergeFrom(value).buildPartial();
+          } else {
+            header_ = value;
+          }
+          onChanged();
+        } else {
+          headerBuilder_.mergeFrom(value);
+        }
+        bitField0_ |= 0x00000001;
+        return this;
+      }
+      public Builder clearHeader() {
+        if (headerBuilder_ == null) {
+          header_ = org.apache.hadoop.hdfs.protocol.proto.DataTransferProtos.BaseHeaderProto.getDefaultInstance();
+          onChanged();
+        } else {
+          headerBuilder_.clear();
+        }
+        bitField0_ = (bitField0_ & ~0x00000001);
+        return this;
+      }
+      public org.apache.hadoop.hdfs.protocol.proto.DataTransferProtos.BaseHeaderProto.Builder getHeaderBuilder() {
+        bitField0_ |= 0x00000001;
+        onChanged();
+        return getHeaderFieldBuilder().getBuilder();
+      }
+      public org.apache.hadoop.hdfs.protocol.proto.DataTransferProtos.BaseHeaderProtoOrBuilder getHeaderOrBuilder() {
+        if (headerBuilder_ != null) {
+          return headerBuilder_.getMessageOrBuilder();
+        } else {
+          return header_;
+        }
+      }
+      private com.google.protobuf.SingleFieldBuilder<
+          org.apache.hadoop.hdfs.protocol.proto.DataTransferProtos.BaseHeaderProto, org.apache.hadoop.hdfs.protocol.proto.DataTransferProtos.BaseHeaderProto.Builder, org.apache.hadoop.hdfs.protocol.proto.DataTransferProtos.BaseHeaderProtoOrBuilder> 
+          getHeaderFieldBuilder() {
+        if (headerBuilder_ == null) {
+          headerBuilder_ = new com.google.protobuf.SingleFieldBuilder<
+              org.apache.hadoop.hdfs.protocol.proto.DataTransferProtos.BaseHeaderProto, org.apache.hadoop.hdfs.protocol.proto.DataTransferProtos.BaseHeaderProto.Builder, org.apache.hadoop.hdfs.protocol.proto.DataTransferProtos.BaseHeaderProtoOrBuilder>(
+                  header_,
+                  getParentForChildren(),
+                  isClean());
+          header_ = null;
+        }
+        return headerBuilder_;
+      }
+      
+      // @@protoc_insertion_point(builder_scope:OpCopyBlockProto)
+    }
+    
+    static {
+      defaultInstance = new OpCopyBlockProto(true);
+      defaultInstance.initFields();
+    }
+    
+    // @@protoc_insertion_point(class_scope:OpCopyBlockProto)
+  }
+  
+  public interface OpBlockChecksumProtoOrBuilder
+      extends com.google.protobuf.MessageOrBuilder {
+    
+    // required .BaseHeaderProto header = 1;
+    boolean hasHeader();
+    org.apache.hadoop.hdfs.protocol.proto.DataTransferProtos.BaseHeaderProto getHeader();
+    org.apache.hadoop.hdfs.protocol.proto.DataTransferProtos.BaseHeaderProtoOrBuilder getHeaderOrBuilder();
+  }
+  public static final class OpBlockChecksumProto extends
+      com.google.protobuf.GeneratedMessage
+      implements OpBlockChecksumProtoOrBuilder {
+    // Use OpBlockChecksumProto.newBuilder() to construct.
+    private OpBlockChecksumProto(Builder builder) {
+      super(builder);
+    }
+    private OpBlockChecksumProto(boolean noInit) {}
+    
+    private static final OpBlockChecksumProto defaultInstance;
+    public static OpBlockChecksumProto getDefaultInstance() {
+      return defaultInstance;
+    }
+    
+    public OpBlockChecksumProto getDefaultInstanceForType() {
+      return defaultInstance;
+    }
+    
+    public static final com.google.protobuf.Descriptors.Descriptor
+        getDescriptor() {
+      return org.apache.hadoop.hdfs.protocol.proto.DataTransferProtos.internal_static_OpBlockChecksumProto_descriptor;
+    }
+    
+    protected com.google.protobuf.GeneratedMessage.FieldAccessorTable
+        internalGetFieldAccessorTable() {
+      return org.apache.hadoop.hdfs.protocol.proto.DataTransferProtos.internal_static_OpBlockChecksumProto_fieldAccessorTable;
+    }
+    
+    private int bitField0_;
+    // required .BaseHeaderProto header = 1;
+    public static final int HEADER_FIELD_NUMBER = 1;
+    private org.apache.hadoop.hdfs.protocol.proto.DataTransferProtos.BaseHeaderProto header_;
+    public boolean hasHeader() {
+      return ((bitField0_ & 0x00000001) == 0x00000001);
+    }
+    public org.apache.hadoop.hdfs.protocol.proto.DataTransferProtos.BaseHeaderProto getHeader() {
+      return header_;
+    }
+    public org.apache.hadoop.hdfs.protocol.proto.DataTransferProtos.BaseHeaderProtoOrBuilder getHeaderOrBuilder() {
+      return header_;
+    }
+    
+    private void initFields() {
+      header_ = org.apache.hadoop.hdfs.protocol.proto.DataTransferProtos.BaseHeaderProto.getDefaultInstance();
+    }
+    private byte memoizedIsInitialized = -1;
+    public final boolean isInitialized() {
+      byte isInitialized = memoizedIsInitialized;
+      if (isInitialized != -1) return isInitialized == 1;
+      
+      if (!hasHeader()) {
+        memoizedIsInitialized = 0;
+        return false;
+      }
+      if (!getHeader().isInitialized()) {
+        memoizedIsInitialized = 0;
+        return false;
+      }
+      memoizedIsInitialized = 1;
+      return true;
+    }
+    
+    public void writeTo(com.google.protobuf.CodedOutputStream output)
+                        throws java.io.IOException {
+      getSerializedSize();
+      if (((bitField0_ & 0x00000001) == 0x00000001)) {
+        output.writeMessage(1, header_);
+      }
+      getUnknownFields().writeTo(output);
+    }
+    
+    private int memoizedSerializedSize = -1;
+    public int getSerializedSize() {
+      int size = memoizedSerializedSize;
+      if (size != -1) return size;
+    
+      size = 0;
+      if (((bitField0_ & 0x00000001) == 0x00000001)) {
+        size += com.google.protobuf.CodedOutputStream
+          .computeMessageSize(1, header_);
+      }
+      size += getUnknownFields().getSerializedSize();
+      memoizedSerializedSize = size;
+      return size;
+    }
+    
+    private static final long serialVersionUID = 0L;
+    @java.lang.Override
+    protected java.lang.Object writeReplace()
+        throws java.io.ObjectStreamException {
+      return super.writeReplace();
+    }
+    
+    @java.lang.Override
+    public boolean equals(final java.lang.Object obj) {
+      if (obj == this) {
+       return true;
+      }
+      if (!(obj instanceof org.apache.hadoop.hdfs.protocol.proto.DataTransferProtos.OpBlockChecksumProto)) {
+        return super.equals(obj);
+      }
+      org.apache.hadoop.hdfs.protocol.proto.DataTransferProtos.OpBlockChecksumProto other = (org.apache.hadoop.hdfs.protocol.proto.DataTransferProtos.OpBlockChecksumProto) obj;
+      
+      boolean result = true;
+      result = result && (hasHeader() == other.hasHeader());
+      if (hasHeader()) {
+        result = result && getHeader()
+            .equals(other.getHeader());
+      }
+      result = result &&
+          getUnknownFields().equals(other.getUnknownFields());
+      return result;
+    }
+    
+    @java.lang.Override
+    public int hashCode() {
+      int hash = 41;
+      hash = (19 * hash) + getDescriptorForType().hashCode();
+      if (hasHeader()) {
+        hash = (37 * hash) + HEADER_FIELD_NUMBER;
+        hash = (53 * hash) + getHeader().hashCode();
+      }
+      hash = (29 * hash) + getUnknownFields().hashCode();
+      return hash;
+    }
+    
+    public static org.apache.hadoop.hdfs.protocol.proto.DataTransferProtos.OpBlockChecksumProto parseFrom(
+        com.google.protobuf.ByteString data)
+        throws com.google.protobuf.InvalidProtocolBufferException {
+      return newBuilder().mergeFrom(data).buildParsed();
+    }
+    public static org.apache.hadoop.hdfs.protocol.proto.DataTransferProtos.OpBlockChecksumProto parseFrom(
+        com.google.protobuf.ByteString data,
+        com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+        throws com.google.protobuf.InvalidProtocolBufferException {
+      return newBuilder().mergeFrom(data, extensionRegistry)
+               .buildParsed();
+    }
+    public static org.apache.hadoop.hdfs.protocol.proto.DataTransferProtos.OpBlockChecksumProto parseFrom(byte[] data)
+        throws com.google.protobuf.InvalidProtocolBufferException {
+      return newBuilder().mergeFrom(data).buildParsed();
+    }
+    public static org.apache.hadoop.hdfs.protocol.proto.DataTransferProtos.OpBlockChecksumProto parseFrom(
+        byte[] data,
+        com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+        throws com.google.protobuf.InvalidProtocolBufferException {
+      return newBuilder().mergeFrom(data, extensionRegistry)
+               .buildParsed();
+    }
+    public static org.apache.hadoop.hdfs.protocol.proto.DataTransferProtos.OpBlockChecksumProto parseFrom(java.io.InputStream input)
+        throws java.io.IOException {
+      return newBuilder().mergeFrom(input).buildParsed();
+    }
+    public static org.apache.hadoop.hdfs.protocol.proto.DataTransferProtos.OpBlockChecksumProto parseFrom(
+        java.io.InputStream input,
+        com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+        throws java.io.IOException {
+      return newBuilder().mergeFrom(input, extensionRegistry)
+               .buildParsed();
+    }
+    public static org.apache.hadoop.hdfs.protocol.proto.DataTransferProtos.OpBlockChecksumProto parseDelimitedFrom(java.io.InputStream input)
+        throws java.io.IOException {
+      Builder builder = newBuilder();
+      if (builder.mergeDelimitedFrom(input)) {
+        return builder.buildParsed();
+      } else {
+        return null;
+      }
+    }
+    public static org.apache.hadoop.hdfs.protocol.proto.DataTransferProtos.OpBlockChecksumProto parseDelimitedFrom(
+        java.io.InputStream input,
+        com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+        throws java.io.IOException {
+      Builder builder = newBuilder();
+      if (builder.mergeDelimitedFrom(input, extensionRegistry)) {
+        return builder.buildParsed();
+      } else {
+        return null;
+      }
+    }
+    public static org.apache.hadoop.hdfs.protocol.proto.DataTransferProtos.OpBlockChecksumProto parseFrom(
+        com.google.protobuf.CodedInputStream input)
+        throws java.io.IOException {
+      return newBuilder().mergeFrom(input).buildParsed();
+    }
+    public static org.apache.hadoop.hdfs.protocol.proto.DataTransferProtos.OpBlockChecksumProto parseFrom(
+        com.google.protobuf.CodedInputStream input,
+        com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+        throws java.io.IOException {
+      return newBuilder().mergeFrom(input, extensionRegistry)
+               .buildParsed();
+    }
+    
+    public static Builder newBuilder() { return Builder.create(); }
+    public Builder newBuilderForType() { return newBuilder(); }
+    public static Builder newBuilder(org.apache.hadoop.hdfs.protocol.proto.DataTransferProtos.OpBlockChecksumProto prototype) {
+      return newBuilder().mergeFrom(prototype);
+    }
+    public Builder toBuilder() { return newBuilder(this); }
+    
+    @java.lang.Override
+    protected Builder newBuilderForType(
+        com.google.protobuf.GeneratedMessage.BuilderParent parent) {
+      Builder builder = new Builder(parent);
+      return builder;
+    }
+    public static final class Builder extends
+        com.google.protobuf.GeneratedMessage.Builder<Builder>
+       implements org.apache.hadoop.hdfs.protocol.proto.DataTransferProtos.OpBlockChecksumProtoOrBuilder {
+      public static final com.google.protobuf.Descriptors.Descriptor
+          getDescriptor() {
+        return org.apache.hadoop.hdfs.protocol.proto.DataTransferProtos.internal_static_OpBlockChecksumProto_descriptor;
+      }
+      
+      protected com.google.protobuf.GeneratedMessage.FieldAccessorTable
+          internalGetFieldAccessorTable() {
+        return org.apache.hadoop.hdfs.protocol.proto.DataTransferProtos.internal_static_OpBlockChecksumProto_fieldAccessorTable;
+      }
+      
+      // Construct using org.apache.hadoop.hdfs.protocol.proto.DataTransferProtos.OpBlockChecksumProto.newBuilder()
+      private Builder() {
+        maybeForceBuilderInitialization();
+      }
+      
+      private Builder(BuilderParent parent) {
+        super(parent);
+        maybeForceBuilderInitialization();
+      }
+      private void maybeForceBuilderInitialization() {
+        if (com.google.protobuf.GeneratedMessage.alwaysUseFieldBuilders) {
+          getHeaderFieldBuilder();
+        }
+      }
+      private static Builder create() {
+        return new Builder();
+      }
+      
+      public Builder clear() {
+        super.clear();
+        if (headerBuilder_ == null) {
+          header_ = org.apache.hadoop.hdfs.protocol.proto.DataTransferProtos.BaseHeaderProto.getDefaultInstance();
+        } else {
+          headerBuilder_.clear();
+        }
+        bitField0_ = (bitField0_ & ~0x00000001);
+        return this;
+      }
+      
+      public Builder clone() {
+        return create().mergeFrom(buildPartial());
+      }
+      
+      public com.google.protobuf.Descriptors.Descriptor
+          getDescriptorForType() {
+        return org.apache.hadoop.hdfs.protocol.proto.DataTransferProtos.OpBlockChecksumProto.getDescriptor();
+      }
+      
+      public org.apache.hadoop.hdfs.protocol.proto.DataTransferProtos.OpBlockChecksumProto getDefaultInstanceForType() {
+        return org.apache.hadoop.hdfs.protocol.proto.DataTransferProtos.OpBlockChecksumProto.getDefaultInstance();
+      }
+      
+      public org.apache.hadoop.hdfs.protocol.proto.DataTransferProtos.OpBlockChecksumProto build() {
+        org.apache.hadoop.hdfs.protocol.proto.DataTransferProtos.OpBlockChecksumProto result = buildPartial();
+        if (!result.isInitialized()) {
+          throw newUninitializedMessageException(result);
+        }
+        return result;
+      }
+      
+      private org.apache.hadoop.hdfs.protocol.proto.DataTransferProtos.OpBlockChecksumProto buildParsed()
+          throws com.google.protobuf.InvalidProtocolBufferException {
+        org.apache.hadoop.hdfs.protocol.proto.DataTransferProtos.OpBlockChecksumProto result = buildPartial();
+        if (!result.isInitialized()) {
+          throw newUninitializedMessageException(
+            result).asInvalidProtocolBufferException();
+        }
+        return result;
+      }
+      
+      public org.apache.hadoop.hdfs.protocol.proto.DataTransferProtos.OpBlockChecksumProto buildPartial() {
+        org.apache.hadoop.hdfs.protocol.proto.DataTransferProtos.OpBlockChecksumProto result = new org.apache.hadoop.hdfs.protocol.proto.DataTransferProtos.OpBlockChecksumProto(this);
+        int from_bitField0_ = bitField0_;
+        int to_bitField0_ = 0;
+        if (((from_bitField0_ & 0x00000001) == 0x00000001)) {
+          to_bitField0_ |= 0x00000001;
+        }
+        if (headerBuilder_ == null) {
+          result.header_ = header_;
+        } else {
+          result.header_ = headerBuilder_.build();
+        }
+        result.bitField0_ = to_bitField0_;
+        onBuilt();
+        return result;
+      }
+      
+      public Builder mergeFrom(com.google.protobuf.Message other) {
+        if (other instanceof org.apache.hadoop.hdfs.protocol.proto.DataTransferProtos.OpBlockChecksumProto) {
+          return mergeFrom((org.apache.hadoop.hdfs.protocol.proto.DataTransferProtos.OpBlockChecksumProto)other);
+        } else {
+          super.mergeFrom(other);
+          return this;
+        }
+      }
+      
+      public Builder mergeFrom(org.apache.hadoop.hdfs.protocol.proto.DataTransferProtos.OpBlockChecksumProto other) {
+        if (other == org.apache.hadoop.hdfs.protocol.proto.DataTransferProtos.OpBlockChecksumProto.getDefaultInstance()) return this;
+        if (other.hasHeader()) {
+          mergeHeader(other.getHeader());
+        }
+        this.mergeUnknownFields(other.getUnknownFields());
+        return this;
+      }
+      
+      public final boolean isInitialized() {
+        if (!hasHeader()) {
+          
+          return false;
+        }
+        if (!getHeader().isInitialized()) {
+          
+          return false;
+        }
+        return true;
+      }
+      
+      public Builder mergeFrom(
+          com.google.protobuf.CodedInputStream input,
+          com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+          throws java.io.IOException {
+        com.google.protobuf.UnknownFieldSet.Builder unknownFields =
+          com.google.protobuf.UnknownFieldSet.newBuilder(
+            this.getUnknownFields());
+        while (true) {
+          int tag = input.readTag();
+          switch (tag) {
+            case 0:
+              this.setUnknownFields(unknownFields.build());
+              onChanged();
+              return this;
+            default: {
+              if (!parseUnknownField(input, unknownFields,
+                                     extensionRegistry, tag)) {
+                this.setUnknownFields(unknownFields.build());
+                onChanged();
+                return this;
+              }
+              break;
+            }
+            case 10: {
+              org.apache.hadoop.hdfs.protocol.proto.DataTransferProtos.BaseHeaderProto.Builder subBuilder = org.apache.hadoop.hdfs.protocol.proto.DataTransferProtos.BaseHeaderProto.newBuilder();
+              if (hasHeader()) {
+                subBuilder.mergeFrom(getHeader());
+              }
+              input.readMessage(subBuilder, extensionRegistry);
+              setHeader(subBuilder.buildPartial());
+              break;
+            }
+          }
+        }
+      }
+      
+      private int bitField0_;
+      
+      // required .BaseHeaderProto header = 1;
+      private org.apache.hadoop.hdfs.protocol.proto.DataTransferProtos.BaseHeaderProto header_ = org.apache.hadoop.hdfs.protocol.proto.DataTransferProtos.BaseHeaderProto.getDefaultInstance();
+      private com.google.protobuf.SingleFieldBuilder<
+          org.apache.hadoop.hdfs.protocol.proto.DataTransferProtos.BaseHeaderProto, org.apache.hadoop.hdfs.protocol.proto.DataTransferProtos.BaseHeaderProto.Builder, org.apache.hadoop.hdfs.protocol.proto.DataTransferProtos.BaseHeaderProtoOrBuilder> headerBuilder_;
+      public boolean hasHeader() {
+        return ((bitField0_ & 0x00000001) == 0x00000001);
+      }
+      public org.apache.hadoop.hdfs.protocol.proto.DataTransferProtos.BaseHeaderProto getHeader() {
+        if (headerBuilder_ == null) {
+          return header_;
+        } else {
+          return headerBuilder_.getMessage();
+        }
+      }
+      public Builder setHeader(org.apache.hadoop.hdfs.protocol.proto.DataTransferProtos.BaseHeaderProto value) {
+        if (headerBuilder_ == null) {
+          if (value == null) {
+            throw new NullPointerException();
+          }
+          header_ = value;
+          onChanged();
+        } else {
+          headerBuilder_.setMessage(value);
+        }
+        bitField0_ |= 0x00000001;
+        return this;
+      }
+      public Builder setHeader(
+          org.apache.hadoop.hdfs.protocol.proto.DataTransferProtos.BaseHeaderProto.Builder builderForValue) {
+        if (headerBuilder_ == null) {
+          header_ = builderForValue.build();
+          onChanged();
+        } else {
+          headerBuilder_.setMessage(builderForValue.build());
+        }
+        bitField0_ |= 0x00000001;
+        return this;
+      }
+      public Builder mergeHeader(org.apache.hadoop.hdfs.protocol.proto.DataTransferProtos.BaseHeaderProto value) {
+        if (headerBuilder_ == null) {
+          if (((bitField0_ & 0x00000001) == 0x00000001) &&
+              header_ != org.apache.hadoop.hdfs.protocol.proto.DataTransferProtos.BaseHeaderProto.getDefaultInstance()) {
+            header_ =
+              org.apache.hadoop.hdfs.protocol.proto.DataTransferProtos.BaseHeaderProto.newBuilder(header_).mergeFrom(value).buildPartial();
+          } else {
+            header_ = value;
+          }
+          onChanged();
+        } else {
+          headerBuilder_.mergeFrom(value);
+        }
+        bitField0_ |= 0x00000001;
+        return this;
+      }
+      public Builder clearHeader() {
+        if (headerBuilder_ == null) {
+          header_ = org.apache.hadoop.hdfs.protocol.proto.DataTransferProtos.BaseHeaderProto.getDefaultInstance();
+          onChanged();
+        } else {
+          headerBuilder_.clear();
+        }
+        bitField0_ = (bitField0_ & ~0x00000001);
+        return this;
+      }
+      public org.apache.hadoop.hdfs.protocol.proto.DataTransferProtos.BaseHeaderProto.Builder getHeaderBuilder() {
+        bitField0_ |= 0x00000001;
+        onChanged();
+        return getHeaderFieldBuilder().getBuilder();
+      }
+      public org.apache.hadoop.hdfs.protocol.proto.DataTransferProtos.BaseHeaderProtoOrBuilder getHeaderOrBuilder() {
+        if (headerBuilder_ != null) {
+          return headerBuilder_.getMessageOrBuilder();
+        } else {
+          return header_;
+        }
+      }
+      private com.google.protobuf.SingleFieldBuilder<
+          org.apache.hadoop.hdfs.protocol.proto.DataTransferProtos.BaseHeaderProto, org.apache.hadoop.hdfs.protocol.proto.DataTransferProtos.BaseHeaderProto.Builder, org.apache.hadoop.hdfs.protocol.proto.DataTransferProtos.BaseHeaderProtoOrBuilder> 
+          getHeaderFieldBuilder() {
+        if (headerBuilder_ == null) {
+          headerBuilder_ = new com.google.protobuf.SingleFieldBuilder<
+              org.apache.hadoop.hdfs.protocol.proto.DataTransferProtos.BaseHeaderProto, org.apache.hadoop.hdfs.protocol.proto.DataTransferProtos.BaseHeaderProto.Builder, org.apache.hadoop.hdfs.protocol.proto.DataTransferProtos.BaseHeaderProtoOrBuilder>(
+                  header_,
+                  getParentForChildren(),
+                  isClean());
+          header_ = null;
+        }
+        return headerBuilder_;
+      }
+      
+      // @@protoc_insertion_point(builder_scope:OpBlockChecksumProto)
+    }
+    
+    static {
+      defaultInstance = new OpBlockChecksumProto(true);
+      defaultInstance.initFields();
+    }
+    
+    // @@protoc_insertion_point(class_scope:OpBlockChecksumProto)
+  }
+  
+  public interface PacketHeaderProtoOrBuilder
+      extends com.google.protobuf.MessageOrBuilder {
+    
+    // required sfixed64 offsetInBlock = 1;
+    boolean hasOffsetInBlock();
+    long getOffsetInBlock();
+    
+    // required sfixed64 seqno = 2;
+    boolean hasSeqno();
+    long getSeqno();
+    
+    // required bool lastPacketInBlock = 3;
+    boolean hasLastPacketInBlock();
+    boolean getLastPacketInBlock();
+    
+    // required sfixed32 dataLen = 4;
+    boolean hasDataLen();
+    int getDataLen();
+  }
+  public static final class PacketHeaderProto extends
+      com.google.protobuf.GeneratedMessage
+      implements PacketHeaderProtoOrBuilder {
+    // Use PacketHeaderProto.newBuilder() to construct.
+    private PacketHeaderProto(Builder builder) {
+      super(builder);
+    }
+    private PacketHeaderProto(boolean noInit) {}
+    
+    private static final PacketHeaderProto defaultInstance;
+    public static PacketHeaderProto getDefaultInstance() {
+      return defaultInstance;
+    }
+    
+    public PacketHeaderProto getDefaultInstanceForType() {
+      return defaultInstance;
+    }
+    
+    public static final com.google.protobuf.Descriptors.Descriptor
+        getDescriptor() {
+      return org.apache.hadoop.hdfs.protocol.proto.DataTransferProtos.internal_static_PacketHeaderProto_descriptor;
+    }
+    
+    protected com.google.protobuf.GeneratedMessage.FieldAccessorTable
+        internalGetFieldAccessorTable() {
+      return org.apache.hadoop.hdfs.protocol.proto.DataTransferProtos.internal_static_PacketHeaderProto_fieldAccessorTable;
+    }
+    
+    private int bitField0_;
+    // required sfixed64 offsetInBlock = 1;
+    public static final int OFFSETINBLOCK_FIELD_NUMBER = 1;
+    private long offsetInBlock_;
+    public boolean hasOffsetInBlock() {
+      return ((bitField0_ & 0x00000001) == 0x00000001);
+    }
+    public long getOffsetInBlock() {
+      return offsetInBlock_;
+    }
+    
+    // required sfixed64 seqno = 2;
+    public static final int SEQNO_FIELD_NUMBER = 2;
+    private long seqno_;
+    public boolean hasSeqno() {
+      return ((bitField0_ & 0x00000002) == 0x00000002);
+    }
+    public long getSeqno() {
+      return seqno_;
+    }
+    
+    // required bool lastPacketInBlock = 3;
+    public static final int LASTPACKETINBLOCK_FIELD_NUMBER = 3;
+    private boolean lastPacketInBlock_;
+    public boolean hasLastPacketInBlock() {
+      return ((bitField0_ & 0x00000004) == 0x00000004);
+    }
+    public boolean getLastPacketInBlock() {
+      return lastPacketInBlock_;
+    }
+    
+    // required sfixed32 dataLen = 4;
+    public static final int DATALEN_FIELD_NUMBER = 4;
+    private int dataLen_;
+    public boolean hasDataLen() {
+      return ((bitField0_ & 0x00000008) == 0x00000008);
+    }
+    public int getDataLen() {
+      return dataLen_;
+    }
+    
+    private void initFields() {
+      offsetInBlock_ = 0L;
+      seqno_ = 0L;
+      lastPacketInBlock_ = false;
+      dataLen_ = 0;
+    }
+    private byte memoizedIsInitialized = -1;
+    public final boolean isInitialized() {
+      byte isInitialized = memoizedIsInitialized;
+      if (isInitialized != -1) return isInitialized == 1;
+      
+      if (!hasOffsetInBlock()) {
+        memoizedIsInitialized = 0;
+        return false;
+      }
+      if (!hasSeqno()) {
+        memoizedIsInitialized = 0;
+        return false;
+      }
+      if (!hasLastPacketInBlock()) {
+        memoizedIsInitialized = 0;
+        return false;
+      }
+      if (!hasDataLen()) {
+        memoizedIsInitialized = 0;
+        return false;
+      }
+      memoizedIsInitialized = 1;
+      return true;
+    }
+    
+    public void writeTo(com.google.protobuf.CodedOutputStream output)
+                        throws java.io.IOException {
+      getSerializedSize();
+      if (((bitField0_ & 0x00000001) == 0x00000001)) {
+        output.writeSFixed64(1, offsetInBlock_);
+      }
+      if (((bitField0_ & 0x00000002) == 0x00000002)) {
+        output.writeSFixed64(2, seqno_);
+      }
+      if (((bitField0_ & 0x00000004) == 0x00000004)) {
+        output.writeBool(3, lastPacketInBlock_);
+      }
+      if (((bitField0_ & 0x00000008) == 0x00000008)) {
+        output.writeSFixed32(4, dataLen_);
+      }
+      getUnknownFields().writeTo(output);
+    }
+    
+    private int memoizedSerializedSize = -1;
+    public int getSerializedSize() {
+      int size = memoizedSerializedSize;
+      if (size != -1) return size;
+    
+      size = 0;
+      if (((bitField0_ & 0x00000001) == 0x00000001)) {
+        size += com.google.protobuf.CodedOutputStream
+          .computeSFixed64Size(1, offsetInBlock_);
+      }
+      if (((bitField0_ & 0x00000002) == 0x00000002)) {
+        size += com.google.protobuf.CodedOutputStream
+          .computeSFixed64Size(2, seqno_);
+      }
+      if (((bitField0_ & 0x00000004) == 0x00000004)) {
+        size += com.google.protobuf.CodedOutputStream
+          .computeBoolSize(3, lastPacketInBlock_);
+      }
+      if (((bitField0_ & 0x00000008) == 0x00000008)) {
+        size += com.google.protobuf.CodedOutputStream
+          .computeSFixed32Size(4, dataLen_);
+      }
+      size += getUnknownFields().getSerializedSize();
+      memoizedSerializedSize = size;
+      return size;
+    }
+    
+    private static final long serialVersionUID = 0L;
+    @java.lang.Override
+    protected java.lang.Object writeReplace()
+        throws java.io.ObjectStreamException {
+      return super.writeReplace();
+    }
+    
+    @java.lang.Override
+    public boolean equals(final java.lang.Object obj) {
+      if (obj == this) {
+       return true;
+      }
+      if (!(obj instanceof org.apache.hadoop.hdfs.protocol.proto.DataTransferProtos.PacketHeaderProto)) {
+        return super.equals(obj);
+      }
+      org.apache.hadoop.hdfs.protocol.proto.DataTransferProtos.PacketHeaderProto other = (org.apache.hadoop.hdfs.protocol.proto.DataTransferProtos.PacketHeaderProto) obj;
+      
+      boolean result = true;
+      result = result && (hasOffsetInBlock() == other.hasOffsetInBlock());
+      if (hasOffsetInBlock()) {
+        result = result && (getOffsetInBlock()
+            == other.getOffsetInBlock());
+      }
+      result = result && (hasSeqno() == other.hasSeqno());
+      if (hasSeqno()) {
+        result = result && (getSeqno()
+            == other.getSeqno());
+      }
+      result = result && (hasLastPacketInBlock() == other.hasLastPacketInBlock());
+      if (hasLastPacketInBlock()) {
+        result = result && (getLastPacketInBlock()
+            == other.getLastPacketInBlock());
+      }
+      result = result && (hasDataLen() == other.hasDataLen());
+      if (hasDataLen()) {
+        result = result && (getDataLen()
+            == other.getDataLen());
+      }
+      result = result &&
+          getUnknownFields().equals(other.getUnknownFields());
+      return result;
+    }
+    
+    @java.lang.Override
+    public int hashCode() {
+      int hash = 41;
+      hash = (19 * hash) + getDescriptorForType().hashCode();
+      if (hasOffsetInBlock()) {
+        hash = (37 * hash) + OFFSETINBLOCK_FIELD_NUMBER;
+        hash = (53 * hash) + hashLong(getOffsetInBlock());
+      }
+      if (hasSeqno()) {
+        hash = (37 * hash) + SEQNO_FIELD_NUMBER;
+        hash = (53 * hash) + hashLong(getSeqno());
+      }
+      if (hasLastPacketInBlock()) {
+        hash = (37 * hash) + LASTPACKETINBLOCK_FIELD_NUMBER;
+        hash = (53 * hash) + hashBoolean(getLastPacketInBlock());
+      }
+      if (hasDataLen()) {
+        hash = (37 * hash) + DATALEN_FIELD_NUMBER;
+        hash = (53 * hash) + getDataLen();
+      }
+      hash = (29 * hash) + getUnknownFields().hashCode();
+      return hash;
+    }
+    
+    public static org.apache.hadoop.hdfs.protocol.proto.DataTransferProtos.PacketHeaderProto parseFrom(
+        com.google.protobuf.ByteString data)
+        throws com.google.protobuf.InvalidProtocolBufferException {
+      return newBuilder().mergeFrom(data).buildParsed();
+    }
+    public static org.apache.hadoop.hdfs.protocol.proto.DataTransferProtos.PacketHeaderProto parseFrom(
+        com.google.protobuf.ByteString data,
+        com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+        throws com.google.protobuf.InvalidProtocolBufferException {
+      return newBuilder().mergeFrom(data, extensionRegistry)
+               .buildParsed();
+    }
+    public static org.apache.hadoop.hdfs.protocol.proto.DataTransferProtos.PacketHeaderProto parseFrom(byte[] data)
+        throws com.google.protobuf.InvalidProtocolBufferException {
+      return newBuilder().mergeFrom(data).buildParsed();
+    }
+    public static org.apache.hadoop.hdfs.protocol.proto.DataTransferProtos.PacketHeaderProto parseFrom(
+        byte[] data,
+        com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+        throws com.google.protobuf.InvalidProtocolBufferException {
+      return newBuilder().mergeFrom(data, extensionRegistry)
+               .buildParsed();
+    }
+    public static org.apache.hadoop.hdfs.protocol.proto.DataTransferProtos.PacketHeaderProto parseFrom(java.io.InputStream input)
+        throws java.io.IOException {
+      return newBuilder().mergeFrom(input).buildParsed();
+    }
+    public static org.apache.hadoop.hdfs.protocol.proto.DataTransferProtos.PacketHeaderProto parseFrom(
+        java.io.InputStream input,
+        com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+        throws java.io.IOException {
+      return newBuilder().mergeFrom(input, extensionRegistry)
+               .buildParsed();
+    }
+    public static org.apache.hadoop.hdfs.protocol.proto.DataTransferProtos.PacketHeaderProto parseDelimitedFrom(java.io.InputStream input)
+        throws java.io.IOException {
+      Builder builder = newBuilder();
+      if (builder.mergeDelimitedFrom(input)) {
+        return builder.buildParsed();
+      } else {
+        return null;
+      }
+    }
+    public static org.apache.hadoop.hdfs.protocol.proto.DataTransferProtos.PacketHeaderProto parseDelimitedFrom(
+        java.io.InputStream input,
+        com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+        throws java.io.IOException {
+      Builder builder = newBuilder();
+      if (builder.mergeDelimitedFrom(input, extensionRegistry)) {
+        return builder.buildParsed();
+      } else {
+        return null;
+      }
+    }
+    public static org.apache.hadoop.hdfs.protocol.proto.DataTransferProtos.PacketHeaderProto parseFrom(
+        com.google.protobuf.CodedInputStream input)
+        throws java.io.IOException {
+      return newBuilder().mergeFrom(input).buildParsed();
+    }
+    public static org.apache.hadoop.hdfs.protocol.proto.DataTransferProtos.PacketHeaderProto parseFrom(
+        com.google.protobuf.CodedInputStream input,
+        com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+        throws java.io.IOException {
+      return newBuilder().mergeFrom(input, extensionRegistry)
+               .buildParsed();
+    }
+    
+    public static Builder newBuilder() { return Builder.create(); }
+    public Builder newBuilderForType() { return newBuilder(); }
+    public static Builder newBuilder(org.apache.hadoop.hdfs.protocol.proto.DataTransferProtos.PacketHeaderProto prototype) {
+      return newBuilder().mergeFrom(prototype);
+    }
+    public Builder toBuilder() { return newBuilder(this); }
+    
+    @java.lang.Override
+    protected Builder newBuilderForType(
+        com.google.protobuf.GeneratedMessage.BuilderParent parent) {
+      Builder builder = new Builder(parent);
+      return builder;
+    }
+    public static final class Builder extends
+        com.google.protobuf.GeneratedMessage.Builder<Builder>
+       implements org.apache.hadoop.hdfs.protocol.proto.DataTransferProtos.PacketHeaderProtoOrBuilder {
+      public static final com.google.protobuf.Descriptors.Descriptor
+          getDescriptor() {
+        return org.apache.hadoop.hdfs.protocol.proto.DataTransferProtos.internal_static_PacketHeaderProto_descriptor;
+      }
+      
+      protected com.google.protobuf.GeneratedMessage.FieldAccessorTable
+          internalGetFieldAccessorTable() {
+        return org.apache.hadoop.hdfs.protocol.proto.DataTransferProtos.internal_static_PacketHeaderProto_fieldAccessorTable;
+      }
+      
+      // Construct using org.apache.hadoop.hdfs.protocol.proto.DataTransferProtos.PacketHeaderProto.newBuilder()
+      private Builder() {
+        maybeForceBuilderInitialization();
+      }
+      
+      private Builder(BuilderParent parent) {
+        super(parent);
+        maybeForceBuilderInitialization();
+      }
+      private void maybeForceBuilderInitialization() {
+        if (com.google.protobuf.GeneratedMessage.alwaysUseFieldBuilders) {
+        }
+      }
+      private static Builder create() {
+        return new Builder();
+      }
+      
+      public Builder clear() {
+        super.clear();
+        offsetInBlock_ = 0L;
+        bitField0_ = (bitField0_ & ~0x00000001);
+        seqno_ = 0L;
+        bitField0_ = (bitField0_ & ~0x00000002);
+        lastPacketInBlock_ = false;
+        bitField0_ = (bitField0_ & ~0x00000004);
+        dataLen_ = 0;
+        bitField0_ = (bitField0_ & ~0x00000008);
+        return this;
+      }
+      
+      public Builder clone() {
+        return create().mergeFrom(buildPartial());
+      }
+      
+      public com.google.protobuf.Descriptors.Descriptor
+          getDescriptorForType() {
+        return org.apache.hadoop.hdfs.protocol.proto.DataTransferProtos.PacketHeaderProto.getDescriptor();
+      }
+      
+      public org.apache.hadoop.hdfs.protocol.proto.DataTransferProtos.PacketHeaderProto getDefaultInstanceForType() {
+        return org.apache.hadoop.hdfs.protocol.proto.DataTransferProtos.PacketHeaderProto.getDefaultInstance();
+      }
+      
+      public org.apache.hadoop.hdfs.protocol.proto.DataTransferProtos.PacketHeaderProto build() {
+        org.apache.hadoop.hdfs.protocol.proto.DataTransferProtos.PacketHeaderProto result = buildPartial();
+        if (!result.isInitialized()) {
+          throw newUninitializedMessageException(result);
+        }
+        return result;
+      }
+      
+      private org.apache.hadoop.hdfs.protocol.proto.DataTransferProtos.PacketHeaderProto buildParsed()
+          throws com.google.protobuf.InvalidProtocolBufferException {
+        org.apache.hadoop.hdfs.protocol.proto.DataTransferProtos.PacketHeaderProto result = buildPartial();
+        if (!result.isInitialized()) {
+          throw newUninitializedMessageException(
+            result).asInvalidProtocolBufferException();
+        }
+        return result;
+      }
+      
+      public org.apache.hadoop.hdfs.protocol.proto.DataTransferProtos.PacketHeaderProto buildPartial() {
+        org.apache.hadoop.hdfs.protocol.proto.DataTransferProtos.PacketHeaderProto result = new org.apache.hadoop.hdfs.protocol.proto.DataTransferProtos.PacketHeaderProto(this);
+        int from_bitField0_ = bitField0_;
+        int to_bitField0_ = 0;
+        if (((from_bitField0_ & 0x00000001) == 0x00000001)) {
+          to_bitField0_ |= 0x00000001;
+        }
+        result.offsetInBlock_ = offsetInBlock_;
+        if (((from_bitField0_ & 0x00000002) == 0x00000002)) {
+          to_bitField0_ |= 0x00000002;
+        }
+        result.seqno_ = seqno_;
+        if (((from_bitField0_ & 0x00000004) == 0x00000004)) {
+          to_bitField0_ |= 0x00000004;
+        }
+        result.lastPacketInBlock_ = lastPacketInBlock_;
+        if (((from_bitField0_ & 0x00000008) == 0x00000008)) {
+          to_bitField0_ |= 0x00000008;
+        }
+        result.dataLen_ = dataLen_;
+        result.bitField0_ = to_bitField0_;
+        onBuilt();
+        return result;
+      }
+      
+      public Builder mergeFrom(com.google.protobuf.Message other) {
+        if (other instanceof org.apache.hadoop.hdfs.protocol.proto.DataTransferProtos.PacketHeaderProto) {
+          return mergeFrom((org.apache.hadoop.hdfs.protocol.proto.DataTransferProtos.PacketHeaderProto)other);
+        } else {
+          super.mergeFrom(other);
+          return this;
+        }
+      }
+      
+      public Builder mergeFrom(org.apache.hadoop.hdfs.protocol.proto.DataTransferProtos.PacketHeaderProto other) {
+        if (other == org.apache.hadoop.hdfs.protocol.proto.DataTransferProtos.PacketHeaderProto.getDefaultInstance()) return this;
+        if (other.hasOffsetInBlock()) {
+          setOffsetInBlock(other.getOffsetInBlock());
+        }
+        if (other.hasSeqno()) {
+          setSeqno(other.getSeqno());
+        }
+        if (other.hasLastPacketInBlock()) {
+          setLastPacketInBlock(other.getLastPacketInBlock());
+        }
+        if (other.hasDataLen()) {
+          setDataLen(other.getDataLen());
+        }
+        this.mergeUnknownFields(other.getUnknownFields());
+        return this;
+      }
+      
+      public final boolean isInitialized() {
+        if (!hasOffsetInBlock()) {
+          
+          return false;
+        }
+        if (!hasSeqno()) {
+          
+          return false;
+        }
+        if (!hasLastPacketInBlock()) {
+          
+          return false;
+        }
+        if (!hasDataLen()) {
+          
+          return false;
+        }
+        return true;
+      }
+      
+      public Builder mergeFrom(
+          com.google.protobuf.CodedInputStream input,
+          com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+          throws java.io.IOException {
+        com.google.protobuf.UnknownFieldSet.Builder unknownFields =
+          com.google.protobuf.UnknownFieldSet.newBuilder(
+            this.getUnknownFields());
+        while (true) {
+          int tag = input.readTag();
+          switch (tag) {
+            case 0:
+              this.setUnknownFields(unknownFields.build());
+              onChanged();
+              return this;
+            default: {
+              if (!parseUnknownField(input, unknownFields,
+                                     extensionRegistry, tag)) {
+                this.setUnknownFields(unknownFields.build());
+                onChanged();
+                return this;
+              }
+              break;
+            }
+            case 9: {
+              bitField0_ |= 0x00000001;
+              offsetInBlock_ = input.readSFixed64();
+              break;
+            }
+            case 17: {
+              bitField0_ |= 0x00000002;
+              seqno_ = input.readSFixed64();
+              break;
+            }
+            case 24: {
+              bitField0_ |= 0x00000004;
+              lastPacketInBlock_ = input.readBool();
+              break;
+            }
+            case 37: {
+              bitField0_ |= 0x00000008;
+              dataLen_ = input.readSFixed32();
+              break;
+            }
+          }
+        }
+      }
+      
+      private int bitField0_;
+      
+      // required sfixed64 offsetInBlock = 1;
+      private long offsetInBlock_ ;
+      public boolean hasOffsetInBlock() {
+        return ((bitField0_ & 0x00000001) == 0x00000001);
+      }
+      public long getOffsetInBlock() {
+        return offsetInBlock_;
+      }
+      public Builder setOffsetInBlock(long value) {
+        bitField0_ |= 0x00000001;
+        offsetInBlock_ = value;
+        onChanged();
+        return this;
+      }
+      public Builder clearOffsetInBlock() {
+        bitField0_ = (bitField0_ & ~0x00000001);
+        offsetInBlock_ = 0L;
+        onChanged();
+        return this;
+      }
+      
+      // required sfixed64 seqno = 2;
+      private long seqno_ ;
+      public boolean hasSeqno() {
+        return ((bitField0_ & 0x00000002) == 0x00000002);
+      }
+      public long getSeqno() {
+        return seqno_;
+      }
+      public Builder setSeqno(long value) {
+        bitField0_ |= 0x00000002;
+        seqno_ = value;
+        onChanged();
+        return this;
+      }
+      public Builder clearSeqno() {
+        bitField0_ = (bitField0_ & ~0x00000002);
+        seqno_ = 0L;
+        onChanged();
+        return this;
+      }
+      
+      // required bool lastPacketInBlock = 3;
+      private boolean lastPacketInBlock_ ;
+      public boolean hasLastPacketInBlock() {
+        return ((bitField0_ & 0x00000004) == 0x00000004);
+      }
+      public boolean getLastPacketInBlock() {
+        return lastPacketInBlock_;
+      }
+      public Builder setLastPacketInBlock(boolean value) {
+        bitField0_ |= 0x00000004;
+        lastPacketInBlock_ = value;
+        onChanged();
+        return this;
+      }
+      public Builder clearLastPacketInBlock() {
+        bitField0_ = (bitField0_ & ~0x00000004);
+        lastPacketInBlock_ = false;
+        onChanged();
+        return this;
+      }
+      
+      // required sfixed32 dataLen = 4;
+      private int dataLen_ ;
+      public boolean hasDataLen() {
+        return ((bitField0_ & 0x00000008) == 0x00000008);
+      }
+      public int getDataLen() {
+        return dataLen_;
+      }
+      public Builder setDataLen(int value) {
+        bitField0_ |= 0x00000008;
+        dataLen_ = value;
+        onChanged();
+        return this;
+      }
+      public Builder clearDataLen() {
+        bitField0_ = (bitField0_ & ~0x00000008);
+        dataLen_ = 0;
+        onChanged();
+        return this;
+      }
+      
+      // @@protoc_insertion_point(builder_scope:PacketHeaderProto)
+    }
+    
+    static {
+      defaultInstance = new PacketHeaderProto(true);
+      defaultInstance.initFields();
+    }
+    
+    // @@protoc_insertion_point(class_scope:PacketHeaderProto)
+  }
+  
+  public interface PipelineAckProtoOrBuilder
+      extends com.google.protobuf.MessageOrBuilder {
+    
+    // required sint64 seqno = 1;
+    boolean hasSeqno();
+    long getSeqno();
+    
+    // repeated .Status status = 2;
+    java.util.List<org.apache.hadoop.hdfs.protocol.proto.DataTransferProtos.Status> getStatusList();
+    int getStatusCount();
+    org.apache.hadoop.hdfs.protocol.proto.DataTransferProtos.Status getStatus(int index);
+  }
+  public static final class PipelineAckProto extends
+      com.google.protobuf.GeneratedMessage
+      implements PipelineAckProtoOrBuilder {
+    // Use PipelineAckProto.newBuilder() to construct.
+    private PipelineAckProto(Builder builder) {
+      super(builder);
+    }
+    private PipelineAckProto(boolean noInit) {}
+    
+    private static final PipelineAckProto defaultInstance;
+    public static PipelineAckProto getDefaultInstance() {
+      return defaultInstance;
+    }
+    
+    public PipelineAckProto getDefaultInstanceForType() {
+      return defaultInstance;
+    }
+    
+    public static final com.google.protobuf.Descriptors.Descriptor
+        getDescriptor() {
+      return org.apache.hadoop.hdfs.protocol.proto.DataTransferProtos.internal_static_PipelineAckProto_descriptor;
+    }
+    
+    protected com.google.protobuf.GeneratedMessage.FieldAccessorTable
+        internalGetFieldAccessorTable() {
+      return org.apache.hadoop.hdfs.protocol.proto.DataTransferProtos.internal_static_PipelineAckProto_fieldAccessorTable;
+    }
+    
+    private int bitField0_;
+    // required sint64 seqno = 1;
+    public static final int SEQNO_FIELD_NUMBER = 1;
+    private long seqno_;
+    public boolean hasSeqno() {
+      return ((bitField0_ & 0x00000001) == 0x00000001);
+    }
+    public long getSeqno() {
+      return seqno_;
+    }
+    
+    // repeated .Status status = 2;
+    public static final int STATUS_FIELD_NUMBER = 2;
+    private java.util.List<org.apache.hadoop.hdfs.protocol.proto.DataTransferProtos.Status> status_;
+    public java.util.List<org.apache.hadoop.hdfs.protocol.proto.DataTransferProtos.Status> getStatusList() {
+      return status_;
+    }
+    public int getStatusCount() {
+      return status_.size();
+    }
+    public org.apache.hadoop.hdfs.protocol.proto.DataTransferProtos.Status getStatus(int index) {
+      return status_.get(index);
+    }
+    
+    private void initFields() {
+      seqno_ = 0L;
+      status_ = java.util.Collections.emptyList();
+    }
+    private byte memoizedIsInitialized = -1;
+    public final boolean isInitialized() {
+      byte isInitialized = memoizedIsInitialized;
+      if (isInitialized != -1) return isInitialized == 1;
+      
+      if (!hasSeqno()) {
+        memoizedIsInitialized = 0;
+        return false;
+      }
+      memoizedIsInitialized = 1;
+      return true;
+    }
+    
+    public void writeTo(com.google.protobuf.CodedOutputStream output)
+                        throws java.io.IOException {
+      getSerializedSize();
+      if (((bitField0_ & 0x00000001) == 0x00000001)) {
+        output.writeSInt64(1, seqno_);
+      }
+      for (int i = 0; i < status_.size(); i++) {
+        output.writeEnum(2, status_.get(i).getNumber());
+      }
+      getUnknownFields().writeTo(output);
+    }
+    
+    private int memoizedSerializedSize = -1;
+    public int getSerializedSize() {
+      int size = memoizedSerializedSize;
+      if (size != -1) return size;
+    
+      size = 0;
+      if (((bitField0_ & 0x00000001) == 0x00000001)) {
+        size += com.google.protobuf.CodedOutputStream
+          .computeSInt64Size(1, seqno_);
+      }
+      {
+        int dataSize = 0;
+        for (int i = 0; i < status_.size(); i++) {
+          dataSize += com.google.protobuf.CodedOutputStream
+            .computeEnumSizeNoTag(status_.get(i).getNumber());
+        }
+        size += dataSize;
+        size += 1 * status_.size();
+      }
+      size += getUnknownFields().getSerializedSize();
+      memoizedSerializedSize = size;
+      return size;
+    }
+    
+    private static final long serialVersionUID = 0L;
+    @java.lang.Override
+    protected java.lang.Object writeReplace()
+        throws java.io.ObjectStreamException {
+      return super.writeReplace();
+    }
+    
+    @java.lang.Override
+    public boolean equals(final java.lang.Object obj) {
+      if (obj == this) {
+       return true;
+      }
+      if (!(obj instanceof org.apache.hadoop.hdfs.protocol.proto.DataTransferProtos.PipelineAckProto)) {
+        return super.equals(obj);
+      }
+      org.apache.hadoop.hdfs.protocol.proto.DataTransferProtos.PipelineAckProto other = (org.apache.hadoop.hdfs.protocol.proto.DataTransferProtos.PipelineAckProto) obj;
+      
+      boolean result = true;
+      result = result && (hasSeqno() == other.hasSeqno());
+      if (hasSeqno()) {
+        result = result && (getSeqno()
+            == other.getSeqno());
+      }
+      result = result && getStatusList()
+          .equals(other.getStatusList());
+      result = result &&
+          getUnknownFields().equals(other.getUnknownFields());
+      return result;
+    }
+    
+    @java.lang.Override
+    public int hashCode() {
+      int hash = 41;
+      hash = (19 * hash) + getDescriptorForType().hashCode();
+      if (hasSeqno()) {
+        hash = (37 * hash) + SEQNO_FIELD_NUMBER;
+        hash = (53 * hash) + hashLong(getSeqno());
+      }
+      if (getStatusCount() > 0) {
+        hash = (37 * hash) + STATUS_FIELD_NUMBER;
+        hash = (53 * hash) + hashEnumList(getStatusList());
+      }
+      hash = (29 * hash) + getUnknownFields().hashCode();
+      return hash;
+    }
+    
+    public static org.apache.hadoop.hdfs.protocol.proto.DataTransferProtos.PipelineAckProto parseFrom(
+        com.google.protobuf.ByteString data)
+        throws com.google.protobuf.InvalidProtocolBufferException {
+      return newBuilder().mergeFrom(data).buildParsed();
+    }
+    public static org.apache.hadoop.hdfs.protocol.proto.DataTransferProtos.PipelineAckProto parseFrom(
+        com.google.protobuf.ByteString data,
+        com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+        throws com.google.protobuf.InvalidProtocolBufferException {
+      return newBuilder().mergeFrom(data, extensionRegistry)
+               .buildParsed();
+    }
+    public static org.apache.hadoop.hdfs.protocol.proto.DataTransferProtos.PipelineAckProto parseFrom(byte[] data)
+        throws com.google.protobuf.InvalidProtocolBufferException {
+      return newBuilder().mergeFrom(data).buildParsed();
+    }
+    public static org.apache.hadoop.hdfs.protocol.proto.DataTransferProtos.PipelineAckProto parseFrom(
+        byte[] data,
+        com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+        throws com.google.protobuf.InvalidProtocolBufferException {
+      return newBuilder().mergeFrom(data, extensionRegistry)
+               .buildParsed();
+    }
+    public static org.apache.hadoop.hdfs.protocol.proto.DataTransferProtos.PipelineAckProto parseFrom(java.io.InputStream input)
+        throws java.io.IOException {
+      return newBuilder().mergeFrom(input).buildParsed();
+    }
+    public static org.apache.hadoop.hdfs.protocol.proto.DataTransferProtos.PipelineAckProto parseFrom(
+        java.io.InputStream input,
+        com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+        throws java.io.IOException {
+      return newBuilder().mergeFrom(input, extensionRegistry)
+               .buildParsed();
+    }
+    public static org.apache.hadoop.hdfs.protocol.proto.DataTransferProtos.PipelineAckProto parseDelimitedFrom(java.io.InputStream input)
+        throws java.io.IOException {
+      Builder builder = newBuilder();
+      if (builder.mergeDelimitedFrom(input)) {
+        return builder.buildParsed();
+      } else {
+        return null;
+      }
+    }
+    public static org.apache.hadoop.hdfs.protocol.proto.DataTransferProtos.PipelineAckProto parseDelimitedFrom(
+        java.io.InputStream input,
+        com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+        throws java.io.IOException {
+      Builder builder = newBuilder();
+      if (builder.mergeDelimitedFrom(input, extensionRegistry)) {
+        return builder.buildParsed();
+      } else {
+        return null;
+      }
+    }
+    public static org.apache.hadoop.hdfs.protocol.proto.DataTransferProtos.PipelineAckProto parseFrom(
+        com.google.protobuf.CodedInputStream input)
+        throws java.io.IOException {
+      return newBuilder().mergeFrom(input).buildParsed();
+    }
+    public static org.apache.hadoop.hdfs.protocol.proto.DataTransferProtos.PipelineAckProto parseFrom(
+        com.google.protobuf.CodedInputStream input,
+        com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+        throws java.io.IOException {
+      return newBuilder().mergeFrom(input, extensionRegistry)
+               .buildParsed();
+    }
+    
+    public static Builder newBuilder() { return Builder.create(); }
+    public Builder newBuilderForType() { return newBuilder(); }
+    public static Builder newBuilder(org.apache.hadoop.hdfs.protocol.proto.DataTransferProtos.PipelineAckProto prototype) {
+      return newBuilder().mergeFrom(prototype);
+    }
+    public Builder toBuilder() { return newBuilder(this); }
+    
+    @java.lang.Override
+    protected Builder newBuilderForType(
+        com.google.protobuf.GeneratedMessage.BuilderParent parent) {
+      Builder builder = new Builder(parent);
+      return builder;
+    }
+    public static final class Builder extends
+        com.google.protobuf.GeneratedMessage.Builder<Builder>
+       implements org.apache.hadoop.hdfs.protocol.proto.DataTransferProtos.PipelineAckProtoOrBuilder {
+      public static final com.google.protobuf.Descriptors.Descriptor
+          getDescriptor() {
+        return org.apache.hadoop.hdfs.protocol.proto.DataTransferProtos.internal_static_PipelineAckProto_descriptor;
+      }
+      
+      protected com.google.protobuf.GeneratedMessage.FieldAccessorTable
+          internalGetFieldAccessorTable() {
+        return org.apache.hadoop.hdfs.protocol.proto.DataTransferProtos.internal_static_PipelineAckProto_fieldAccessorTable;
+      }
+      
+      // Construct using org.apache.hadoop.hdfs.protocol.proto.DataTransferProtos.PipelineAckProto.newBuilder()
+      private Builder() {
+        maybeForceBuilderInitialization();
+      }
+      
+      private Builder(BuilderParent parent) {
+        super(parent);
+        maybeForceBuilderInitialization();
+      }
+      private void maybeForceBuilderInitialization() {
+        if (com.google.protobuf.GeneratedMessage.alwaysUseFieldBuilders) {
+        }
+      }
+      private static Builder create() {
+        return new Builder();
+      }
+      
+      public Builder clear() {
+        super.clear();
+        seqno_ = 0L;
+        bitField0_ = (bitField0_ & ~0x00000001);
+        status_ = java.util.Collections.emptyList();
+        bitField0_ = (bitField0_ & ~0x00000002);
+        return this;
+      }
+      
+      public Builder clone() {
+        return create().mergeFrom(buildPartial());
+      }
+      
+      public com.google.protobuf.Descriptors.Descriptor
+          getDescriptorForType() {
+        return org.apache.hadoop.hdfs.protocol.proto.DataTransferProtos.PipelineAckProto.getDescriptor();
+      }
+      
+      public org.apache.hadoop.hdfs.protocol.proto.DataTransferProtos.PipelineAckProto getDefaultInstanceForType() {
+        return org.apache.hadoop.hdfs.protocol.proto.DataTransferProtos.PipelineAckProto.getDefaultInstance();
+      }
+      
+      public org.apache.hadoop.hdfs.protocol.proto.DataTransferProtos.PipelineAckProto build() {
+        org.apache.hadoop.hdfs.protocol.proto.DataTransferProtos.PipelineAckProto result = buildPartial();
+        if (!result.isInitialized()) {
+          throw newUninitializedMessageException(result);
+        }
+        return result;
+      }
+      
+      private org.apache.hadoop.hdfs.protocol.proto.DataTransferProtos.PipelineAckProto buildParsed()
+          throws com.google.protobuf.InvalidProtocolBufferException {
+        org.apache.hadoop.hdfs.protocol.proto.DataTransferProtos.PipelineAckProto result = buildPartial();
+        if (!result.isInitialized()) {
+          throw newUninitializedMessageException(
+            result).asInvalidProtocolBufferException();
+        }
+        return result;
+      }
+      
+      public org.apache.hadoop.hdfs.protocol.proto.DataTransferProtos.PipelineAckProto buildPartial() {
+        org.apache.hadoop.hdfs.protocol.proto.DataTransferProtos.PipelineAckProto result = new org.apache.hadoop.hdfs.protocol.proto.DataTransferProtos.PipelineAckProto(this);
+        int from_bitField0_ = bitField0_;
+        int to_bitField0_ = 0;
+        if (((from_bitField0_ & 0x00000001) == 0x00000001)) {
+          to_bitField0_ |= 0x00000001;
+        }
+        result.seqno_ = seqno_;
+        if (((bitField0_ & 0x00000002) == 0x00000002)) {
+          status_ = java.util.Collections.unmodifiableList(status_);
+          bitField0_ = (bitField0_ & ~0x00000002);
+        }
+        result.status_ = status_;
+        result.bitField0_ = to_bitField0_;
+        onBuilt();
+        return result;
+      }
+      
+      public Builder mergeFrom(com.google.protobuf.Message other) {
+        if (other instanceof org.apache.hadoop.hdfs.protocol.proto.DataTransferProtos.PipelineAckProto) {
+          return mergeFrom((org.apache.hadoop.hdfs.protocol.proto.DataTransferProtos.PipelineAckProto)other);
+        } else {
+          super.mergeFrom(other);
+          return this;
+        }
+      }
+      
+      public Builder mergeFrom(org.apache.hadoop.hdfs.protocol.proto.DataTransferProtos.PipelineAckProto other) {
+        if (other == org.apache.hadoop.hdfs.protocol.proto.DataTransferProtos.PipelineAckProto.getDefaultInstance()) return this;
+        if (other.hasSeqno()) {
+          setSeqno(other.getSeqno());
+        }
+        if (!other.status_.isEmpty()) {
+          if (status_.isEmpty()) {
+            status_ = other.status_;
+            bitField0_ = (bitField0_ & ~0x00000002);
+          } else {
+            ensureStatusIsMutable();
+            status_.addAll(other.status_);
+          }
+          onChanged();
+        }
+        this.mergeUnknownFields(other.getUnknownFields());
+        return this;
+      }
+      
+      public final boolean isInitialized() {
+        if (!hasSeqno()) {
+          
+          return false;
+        }
+        return true;
+      }
+      
+      public Builder mergeFrom(
+          com.google.protobuf.CodedInputStream input,
+          com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+          throws java.io.IOException {
+        com.google.protobuf.UnknownFieldSet.Builder unknownFields =
+          com.google.protobuf.UnknownFieldSet.newBuilder(
+            this.getUnknownFields());
+        while (true) {
+          int tag = input.readTag();
+          switch (tag) {
+            case 0:
+              this.setUnknownFields(unknownFields.build());
+              onChanged();
+              return this;
+            default: {
+              if (!parseUnknownField(input, unknownFields,
+                                     extensionRegistry, tag)) {
+                this.setUnknownFields(unknownFields.build());
+                onChanged();
+                return this;
+              }
+              break;
+            }
+            case 8: {
+              bitField0_ |= 0x00000001;
+              seqno_ = input.readSInt64();
+              break;
+            }
+            case 16: {
+              int rawValue = input.readEnum();
+              org.apache.hadoop.hdfs.protocol.proto.DataTransferProtos.Status value = org.apache.hadoop.hdfs.protocol.proto.DataTransferProtos.Status.valueOf(rawValue);
+              if (value == null) {
+                unknownFields.mergeVarintField(2, rawValue);
+              } else {
+                addStatus(value);
+              }
+              break;
+            }
+            case 18: {
+              int length = input.readRawVarint32();
+              int oldLimit = input.pushLimit(length);
+              while(input.getBytesUntilLimit() > 0) {
+                int rawValue = input.readEnum();
+                org.apache.hadoop.hdfs.protocol.proto.DataTransferProtos.Status value = org.apache.hadoop.hdfs.protocol.proto.DataTransferProtos.Status.valueOf(rawValue);
+                if (value == null) {
+                  unknownFields.mergeVarintField(2, rawValue);
+                } else {
+                  addStatus(value);
+                }
+              }
+              input.popLimit(oldLimit);
+              break;
+            }
+          }
+        }
+      }
+      
+      private int bitField0_;
+      
+      // required sint64 seqno = 1;
+      private long seqno_ ;
+      public boolean hasSeqno() {
+        return ((bitField0_ & 0x00000001) == 0x00000001);
+      }
+      public long getSeqno() {
+        return seqno_;
+      }
+      public Builder setSeqno(long value) {
+        bitField0_ |= 0x00000001;
+        seqno_ = value;
+        onChanged();
+        return this;
+      }
+      public Builder clearSeqno() {
+        bitField0_ = (bitField0_ & ~0x00000001);
+        seqno_ = 0L;
+        onChanged();
+        return this;
+      }
+      
+      // repeated .Status status = 2;
+      private java.util.List<org.apache.hadoop.hdfs.protocol.proto.DataTransferProtos.Status> status_ =
+        java.util.Collections.emptyList();
+      private void ensureStatusIsMutable() {
+        if (!((bitField0_ & 0x00000002) == 0x00000002)) {
+          status_ = new java.util.ArrayList<org.apache.hadoop.hdfs.protocol.proto.DataTransferProtos.Status>(status_);
+          bitField0_ |= 0x00000002;
+        }
+      }
+      public java.util.List<org.apache.hadoop.hdfs.protocol.proto.DataTransferProtos.Status> getStatusList() {
+        return java.util.Collections.unmodifiableList(status_);
+      }
+      public int getStatusCount() {
+        return status_.size();
+      }
+      public org.apache.hadoop.hdfs.protocol.proto.DataTransferProtos.Status getStatus(int index) {
+        return status_.get(index);
+      }
+      public Builder setStatus(
+          int index, org.apache.hadoop.hdfs.protocol.proto.DataTransferProtos.Status value) {
+        if (value == null) {
+          throw new NullPointerException();
+        }
+        ensureStatusIsMutable();
+        status_.set(index, value);
+        onChanged();
+        return this;
+      }
+      public Builder addStatus(org.apache.hadoop.hdfs.protocol.proto.DataTransferProtos.Status value) {
+        if (value == null) {
+          throw new NullPointerException();
+        }
+        ensureStatusIsMutable();
+        status_.add(value);
+        onChanged();
+        return this;
+      }
+      public Builder addAllStatus(
+          java.lang.Iterable<? extends org.apache.hadoop.hdfs.protocol.proto.DataTransferProtos.Status> values) {
+        ensureStatusIsMutable();
+        super.addAll(values, status_);
+        onChanged();
+        return this;
+      }
+      public Builder clearStatus() {
+        status_ = java.util.Collections.emptyList();
+        bitField0_ = (bitField0_ & ~0x00000002);
+        onChanged();
+        return this;
+      }
+      
+      // @@protoc_insertion_point(builder_scope:PipelineAckProto)
+    }
+    
+    static {
+      defaultInstance = new PipelineAckProto(true);
+      defaultInstance.initFields();
+    }
+    
+    // @@protoc_insertion_point(class_scope:PipelineAckProto)
+  }
+  
+  public interface BlockOpResponseProtoOrBuilder
+      extends com.google.protobuf.MessageOrBuilder {
+    
+    // required .Status status = 1;
+    boolean hasStatus();
+    org.apache.hadoop.hdfs.protocol.proto.DataTransferProtos.Status getStatus();
+    
+    // optional string firstBadLink = 2;
+    boolean hasFirstBadLink();
+    String getFirstBadLink();
+    
+    // optional .OpBlockChecksumResponseProto checksumResponse = 3;
+    boolean hasChecksumResponse();
+    org.apache.hadoop.hdfs.protocol.proto.DataTransferProtos.OpBlockChecksumResponseProto getChecksumResponse();
+    org.apache.hadoop.hdfs.protocol.proto.DataTransferProtos.OpBlockChecksumResponseProtoOrBuilder getChecksumResponseOrBuilder();
+  }
+  public static final class BlockOpResponseProto extends
+      com.google.protobuf.GeneratedMessage
+      implements BlockOpResponseProtoOrBuilder {
+    // Use BlockOpResponseProto.newBuilder() to construct.
+    private BlockOpResponseProto(Builder builder) {
+      super(builder);
+    }
+    private BlockOpResponseProto(boolean noInit) {}
+    
+    private static final BlockOpResponseProto defaultInstance;
+    public static BlockOpResponseProto getDefaultInstance() {
+      return defaultInstance;
+    }
+    
+    public BlockOpResponseProto getDefaultInstanceForType() {
+      return defaultInstance;
+    }
+    
+    public static final com.google.protobuf.Descriptors.Descriptor
+        getDescriptor() {
+      return org.apache.hadoop.hdfs.protocol.proto.DataTransferProtos.internal_static_BlockOpResponseProto_descriptor;
+    }
+    
+    protected com.google.protobuf.GeneratedMessage.FieldAccessorTable
+        internalGetFieldAccessorTable() {
+      return org.apache.hadoop.hdfs.protocol.proto.DataTransferProtos.internal_static_BlockOpResponseProto_fieldAccessorTable;
+    }
+    
+    private int bitField0_;
+    // required .Status status = 1;
+    public static final int STATUS_FIELD_NUMBER = 1;
+    private org.apache.hadoop.hdfs.protocol.proto.DataTransferProtos.Status status_;
+    public boolean hasStatus() {
+      return ((bitField0_ & 0x00000001) == 0x00000001);
+    }
+    public org.apache.hadoop.hdfs.protocol.proto.DataTransferProtos.Status getStatus() {
+      return status_;
+    }
+    
+    // optional string firstBadLink = 2;
+    public static final int FIRSTBADLINK_FIELD_NUMBER = 2;
+    private java.lang.Object firstBadLink_;
+    public boolean hasFirstBadLink() {
+      return ((bitField0_ & 0x00000002) == 0x00000002);
+    }
+    public String getFirstBadLink() {
+      java.lang.Object ref = firstBadLink_;
+      if (ref instanceof String) {
+        return (String) ref;
+      } else {
+        com.google.protobuf.ByteString bs = 
+            (com.google.protobuf.ByteString) ref;
+        String s = bs.toStringUtf8();
+        if (com.google.protobuf.Internal.isValidUtf8(bs)) {
+          firstBadLink_ = s;
+        }
+        return s;
+      }
+    }
+    private com.google.protobuf.ByteString getFirstBadLinkBytes() {
+      java.lang.Object ref = firstBadLink_;
+      if (ref instanceof String) {
+        com.google.protobuf.ByteString b = 
+            com.google.protobuf.ByteString.copyFromUtf8((String) ref);
+        firstBadLink_ = b;
+        return b;
+      } else {
+        return (com.google.protobuf.ByteString) ref;
+      }
+    }
+    
+    // optional .OpBlockChecksumResponseProto checksumResponse = 3;
+    public static final int CHECKSUMRESPONSE_FIELD_NUMBER = 3;
+    private org.apache.hadoop.hdfs.protocol.proto.DataTransferProtos.OpBlockChecksumResponseProto checksumResponse_;
+    public boolean hasChecksumResponse() {
+      return ((bitField0_ & 0x00000004) == 0x00000004);
+    }
+    public org.apache.hadoop.hdfs.protocol.proto.DataTransferProtos.OpBlockChecksumResponseProto getChecksumResponse() {
+      return checksumResponse_;
+    }
+    public org.apache.hadoop.hdfs.protocol.proto.DataTransferProtos.OpBlockChecksumResponseProtoOrBuilder getChecksumResponseOrBuilder() {
+      return checksumResponse_;
+    }
+    
+    private void initFields() {
+      status_ = org.apache.hadoop.hdfs.protocol.proto.DataTransferProtos.Status.SUCCESS;
+      firstBadLink_ = "";
+      checksumResponse_ = org.apache.hadoop.hdfs.protocol.proto.DataTransferProtos.OpBlockChecksumResponseProto.getDefaultInstance();
+    }
+    private byte memoizedIsInitialized = -1;
+    public final boolean isInitialized() {
+      byte isInitialized = memoizedIsInitialized;
+      if (isInitialized != -1) return isInitialized == 1;
+      
+      if (!hasStatus()) {
+        memoizedIsInitialized = 0;
+        return false;
+      }
+      if (hasChecksumResponse()) {
+        if (!getChecksumResponse().isInitialized()) {
+          memoizedIsInitialized = 0;
+          return false;
+        }
+      }
+      memoizedIsInitialized = 1;
+      return true;
+    }
+    
+    public void writeTo(com.google.protobuf.CodedOutputStream output)
+                        throws java.io.IOException {
+      getSerializedSize();
+      if (((bitField0_ & 0x00000001) == 0x00000001)) {
+        output.writeEnum(1, status_.getNumber());
+      }
+      if (((bitField0_ & 0x00000002) == 0x00000002)) {
+        output.writeBytes(2, getFirstBadLinkBytes());
+      }
+      if (((bitField0_ & 0x00000004) == 0x00000004)) {
+        output.writeMessage(3, checksumResponse_);
+      }
+      getUnknownFields().writeTo(output);
+    }
+    
+    private int memoizedSerializedSize = -1;
+    public int getSerializedSize() {
+      int size = memoizedSerializedSize;
+      if (size != -1) return size;
+    
+      size = 0;
+      if (((bitField0_ & 0x00000001) == 0x00000001)) {
+        size += com.google.protobuf.CodedOutputStream
+          .computeEnumSize(1, status_.getNumber());
+      }
+      if (((bitField0_ & 0x00000002) == 0x00000002)) {
+        size += com.google.protobuf.CodedOutputStream
+          .computeBytesSize(2, getFirstBadLinkBytes());
+      }
+      if (((bitField0_ & 0x00000004) == 0x00000004)) {
+        size += com.google.protobuf.CodedOutputStream
+          .computeMessageSize(3, checksumResponse_);
+      }
+      size += getUnknownFields().getSerializedSize();
+      memoizedSerializedSize = size;
+      return size;
+    }
+    
+    private static final long serialVersionUID = 0L;
+    @java.lang.Override
+    protected java.lang.Object writeReplace()
+        throws java.io.ObjectStreamException {
+      return super.writeReplace();
+    }
+    
+    @java.lang.Override
+    public boolean equals(final java.lang.Object obj) {
+      if (obj == this) {
+       return true;
+      }
+      if (!(obj instanceof org.apache.hadoop.hdfs.protocol.proto.DataTransferProtos.BlockOpResponseProto)) {
+        return super.equals(obj);
+      }
+      org.apache.hadoop.hdfs.protocol.proto.DataTransferProtos.BlockOpResponseProto other = (org.apache.hadoop.hdfs.protocol.proto.DataTransferProtos.BlockOpResponseProto) obj;
+      
+      boolean result = true;
+      result = result && (hasStatus() == other.hasStatus());
+      if (hasStatus()) {
+        result = result &&
+            (getStatus() == other.getStatus());
+      }
+      result = result && (hasFirstBadLink() == other.hasFirstBadLink());
+      if (hasFirstBadLink()) {
+        result = result && getFirstBadLink()
+            .equals(other.getFirstBadLink());
+      }
+      result = result && (hasChecksumResponse() == other.hasChecksumResponse());
+      if (hasChecksumResponse()) {
+        result = result && getChecksumResponse()
+            .equals(other.getChecksumResponse());
+      }
+      result = result &&
+          getUnknownFields().equals(other.getUnknownFields());
+      return result;
+    }
+    
+    @java.lang.Override
+    public int hashCode() {
+      int hash = 41;
+      hash = (19 * hash) + getDescriptorForType().hashCode();
+      if (hasStatus()) {
+        hash = (37 * hash) + STATUS_FIELD_NUMBER;
+        hash = (53 * hash) + hashEnum(getStatus());
+      }
+      if (hasFirstBadLink()) {
+        hash = (37 * hash) + FIRSTBADLINK_FIELD_NUMBER;
+        hash = (53 * hash) + getFirstBadLink().hashCode();
+      }
+      if (hasChecksumResponse()) {
+        hash = (37 * hash) + CHECKSUMRESPONSE_FIELD_NUMBER;
+        hash = (53 * hash) + getChecksumResponse().hashCode();
+      }
+      hash = (29 * hash) + getUnknownFields().hashCode();
+      return hash;
+    }
+    
+    public static org.apache.hadoop.hdfs.protocol.proto.DataTransferProtos.BlockOpResponseProto parseFrom(
+        com.google.protobuf.ByteString data)
+        throws com.google.protobuf.InvalidProtocolBufferException {
+      return newBuilder().mergeFrom(data).buildParsed();
+    }
+    public static org.apache.hadoop.hdfs.protocol.proto.DataTransferProtos.BlockOpResponseProto parseFrom(
+        com.google.protobuf.ByteString data,
+        com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+        throws com.google.protobuf.InvalidProtocolBufferException {
+      return newBuilder().mergeFrom(data, extensionRegistry)
+               .buildParsed();
+    }
+    public static org.apache.hadoop.hdfs.protocol.proto.DataTransferProtos.BlockOpResponseProto parseFrom(byte[] data)
+        throws com.google.protobuf.InvalidProtocolBufferException {
+      return newBuilder().mergeFrom(data).buildParsed();
+    }
+    public static org.apache.hadoop.hdfs.protocol.proto.DataTransferProtos.BlockOpResponseProto parseFrom(
+        byte[] data,
+        com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+        throws com.google.protobuf.InvalidProtocolBufferException {
+      return newBuilder().mergeFrom(data, extensionRegistry)
+               .buildParsed();
+    }
+    public static org.apache.hadoop.hdfs.protocol.proto.DataTransferProtos.BlockOpResponseProto parseFrom(java.io.InputStream input)
+        throws java.io.IOException {
+      return newBuilder().mergeFrom(input).buildParsed();
+    }
+    public static org.apache.hadoop.hdfs.protocol.proto.DataTransferProtos.BlockOpResponseProto parseFrom(
+        java.io.InputStream input,
+        com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+        throws java.io.IOException {
+      return newBuilder().mergeFrom(input, extensionRegistry)
+               .buildParsed();
+    }
+    public static org.apache.hadoop.hdfs.protocol.proto.DataTransferProtos.BlockOpResponseProto parseDelimitedFrom(java.io.InputStream input)
+        throws java.io.IOException {
+      Builder builder = newBuilder();
+      if (builder.mergeDelimitedFrom(input)) {
+        return builder.buildParsed();
+      } else {
+        return null;
+      }
+    }
+    public static org.apache.hadoop.hdfs.protocol.proto.DataTransferProtos.BlockOpResponseProto parseDelimitedFrom(
+        java.io.InputStream input,
+        com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+        throws java.io.IOException {
+      Builder builder = newBuilder();
+      if (builder.mergeDelimitedFrom(input, extensionRegistry)) {
+        return builder.buildParsed();
+      } else {
+        return null;
+      }
+    }
+    public static org.apache.hadoop.hdfs.protocol.proto.DataTransferProtos.BlockOpResponseProto parseFrom(
+        com.google.protobuf.CodedInputStream input)
+        throws java.io.IOException {
+      return newBuilder().mergeFrom(input).buildParsed();
+    }
+    public static org.apache.hadoop.hdfs.protocol.proto.DataTransferProtos.BlockOpResponseProto parseFrom(
+        com.google.protobuf.CodedInputStream input,
+        com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+        throws java.io.IOException {
+      return newBuilder().mergeFrom(input, extensionRegistry)
+               .buildParsed();
+    }
+    
+    public static Builder newBuilder() { return Builder.create(); }
+    public Builder newBuilderForType() { return newBuilder(); }
+    public static Builder newBuilder(org.apache.hadoop.hdfs.protocol.proto.DataTransferProtos.BlockOpResponseProto prototype) {
+      return newBuilder().mergeFrom(prototype);
+    }
+    public Builder toBuilder() { return newBuilder(this); }
+    
+    @java.lang.Override
+    protected Builder newBuilderForType(
+        com.google.protobuf.GeneratedMessage.BuilderParent parent) {
+      Builder builder = new Builder(parent);
+      return builder;
+    }
+    public static final class Builder extends
+        com.google.protobuf.GeneratedMessage.Builder<Builder>
+       implements org.apache.hadoop.hdfs.protocol.proto.DataTransferProtos.BlockOpResponseProtoOrBuilder {
+      public static final com.google.protobuf.Descriptors.Descriptor
+          getDescriptor() {
+        return org.apache.hadoop.hdfs.protocol.proto.DataTransferProtos.internal_static_BlockOpResponseProto_descriptor;
+      }
+      
+      protected com.google.protobuf.GeneratedMessage.FieldAccessorTable
+          internalGetFieldAccessorTable() {
+        return org.apache.hadoop.hdfs.protocol.proto.DataTransferProtos.internal_static_BlockOpResponseProto_fieldAccessorTable;
+      }
+      
+      // Construct using org.apache.hadoop.hdfs.protocol.proto.DataTransferProtos.BlockOpResponseProto.newBuilder()
+      private Builder() {
+        maybeForceBuilderInitialization();
+      }
+      
+      private Builder(BuilderParent parent) {
+        super(parent);
+        maybeForceBuilderInitialization();
+      }
+      private void maybeForceBuilderInitialization() {
+        if (com.google.protobuf.GeneratedMessage.alwaysUseFieldBuilders) {
+          getChecksumResponseFieldBuilder();
+        }
+      }
+      private static Builder create() {
+        return new Builder();
+      }
+      
+      public Builder clear() {
+        super.clear();
+        status_ = org.apache.hadoop.hdfs.protocol.proto.DataTransferProtos.Status.SUCCESS;
+        bitField0_ = (bitField0_ & ~0x00000001);
+        firstBadLink_ = "";
+        bitField0_ = (bitField0_ & ~0x00000002);
+        if (checksumResponseBuilder_ == null) {
+          checksumResponse_ = org.apache.hadoop.hdfs.protocol.proto.DataTransferProtos.OpBlockChecksumResponseProto.getDefaultInstance();
+        } else {
+          checksumResponseBuilder_.clear();
+        }
+        bitField0_ = (bitField0_ & ~0x00000004);
+        return this;
+      }
+      
+      public Builder clone() {
+        return create().mergeFrom(buildPartial());
+      }
+      
+      public com.google.protobuf.Descriptors.Descriptor
+          getDescriptorForType() {
+        return org.apache.hadoop.hdfs.protocol.proto.DataTransferProtos.BlockOpResponseProto.getDescriptor();
+      }
+      
+      public org.apache.hadoop.hdfs.protocol.proto.DataTransferProtos.BlockOpResponseProto getDefaultInstanceForType() {
+        return org.apache.hadoop.hdfs.protocol.proto.DataTransferProtos.BlockOpResponseProto.getDefaultInstance();
+      }
+      
+      public org.apache.hadoop.hdfs.protocol.proto.DataTransferProtos.BlockOpResponseProto build() {
+        org.apache.hadoop.hdfs.protocol.proto.DataTransferProtos.BlockOpResponseProto result = buildPartial();
+        if (!result.isInitialized()) {
+          throw newUninitializedMessageException(result);
+        }
+        return result;
+      }
+      
+      private org.apache.hadoop.hdfs.protocol.proto.DataTransferProtos.BlockOpResponseProto buildParsed()
+          throws com.google.protobuf.InvalidProtocolBufferException {
+        org.apache.hadoop.hdfs.protocol.proto.DataTransferProtos.BlockOpResponseProto result = buildPartial();
+        if (!result.isInitialized()) {
+          throw newUninitializedMessageException(
+            result).asInvalidProtocolBufferException();
+        }
+        return result;
+      }
+      
+      public org.apache.hadoop.hdfs.protocol.proto.DataTransferProtos.BlockOpResponseProto buildPartial() {
+        org.apache.hadoop.hdfs.protocol.proto.DataTransferProtos.BlockOpResponseProto result = new org.apache.hadoop.hdfs.protocol.proto.DataTransferProtos.BlockOpResponseProto(this);
+        int from_bitField0_ = bitField0_;
+        int to_bitField0_ = 0;
+        if (((from_bitField0_ & 0x00000001) == 0x00000001)) {
+          to_bitField0_ |= 0x00000001;
+        }
+        result.status_ = status_;
+        if (((from_bitField0_ & 0x00000002) == 0x00000002)) {
+          to_bitField0_ |= 0x00000002;
+        }
+        result.firstBadLink_ = firstBadLink_;
+        if (((from_bitField0_ & 0x00000004) == 0x00000004)) {
+          to_bitField0_ |= 0x00000004;
+        }
+        if (checksumResponseBuilder_ == null) {
+          result.checksumResponse_ = checksumResponse_;
+        } else {
+          result.checksumResponse_ = checksumResponseBuilder_.build();
+        }
+        result.bitField0_ = to_bitField0_;
+        onBuilt();
+        return result;
+      }
+      
+      public Builder mergeFrom(com.google.protobuf.Message other) {
+        if (other instanceof org.apache.hadoop.hdfs.protocol.proto.DataTransferProtos.BlockOpResponseProto) {
+          return mergeFrom((org.apache.hadoop.hdfs.protocol.proto.DataTransferProtos.BlockOpResponseProto)other);
+        } else {
+          super.mergeFrom(other);
+          return this;
+        }
+      }
+      
+      public Builder mergeFrom(org.apache.hadoop.hdfs.protocol.proto.DataTransferProtos.BlockOpResponseProto other) {
+        if (other == org.apache.hadoop.hdfs.protocol.proto.DataTransferProtos.BlockOpResponseProto.getDefaultInstance()) return this;
+        if (other.hasStatus()) {
+          setStatus(other.getStatus());
+        }
+        if (other.hasFirstBadLink()) {
+          setFirstBadLink(other.getFirstBadLink());
+        }
+        if (other.hasChecksumResponse()) {
+          mergeChecksumResponse(other.getChecksumResponse());
+        }
+        this.mergeUnknownFields(other.getUnknownFields());
+        return this;
+      }
+      
+      public final boolean isInitialized() {
+        if (!hasStatus()) {
+          
+          return false;
+        }
+        if (hasChecksumResponse()) {
+          if (!getChecksumResponse().isInitialized()) {
+            
+            return false;
+          }
+        }
+        return true;
+      }
+      
+      public Builder mergeFrom(
+          com.google.protobuf.CodedInputStream input,
+          com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+          throws java.io.IOException {
+        com.google.protobuf.UnknownFieldSet.Builder unknownFields =
+          com.google.protobuf.UnknownFieldSet.newBuilder(
+            this.getUnknownFields());
+        while (true) {
+          int tag = input.readTag();
+          switch (tag) {
+            case 0:
+              this.setUnknownFields(unknownFields.build());
+              onChanged();
+              return this;
+            default: {
+              if (!parseUnknownField(input, unknownFields,
+                                     extensionRegistry, tag)) {
+                this.setUnknownFields(unknownFields.build());
+                onChanged();
+                return this;
+              }
+              break;
+            }
+            case 8: {
+              int rawValue = input.readEnum();
+              org.apache.hadoop.hdfs.protocol.proto.DataTransferProtos.Status value = org.apache.hadoop.hdfs.protocol.proto.DataTransferProtos.Status.valueOf(rawValue);
+              if (value == null) {
+                unknownFields.mergeVarintField(1, rawValue);
+              } else {
+                bitField0_ |= 0x00000001;
+                status_ = value;
+              }
+              break;
+            }
+            case 18: {
+              bitField0_ |= 0x00000002;
+              firstBadLink_ = input.readBytes();
+              break;
+            }
+            case 26: {
+              org.apache.hadoop.hdfs.protocol.proto.DataTransferProtos.OpBlockChecksumResponseProto.Builder subBuilder = org.apache.hadoop.hdfs.protocol.proto.DataTransferProtos.OpBlockChecksumResponseProto.newBuilder();
+              if (hasChecksumResponse()) {
+                subBuilder.mergeFrom(getChecksumResponse());
+              }
+              input.readMessage(subBuilder, extensionRegistry);
+              setChecksumResponse(subBuilder.buildPartial());
+              break;
+            }
+          }
+        }
+      }
+      
+      private int bitField0_;
+      
+      // required .Status status = 1;
+      private org.apache.hadoop.hdfs.protocol.proto.DataTransferProtos.Status status_ = org.apache.hadoop.hdfs.protocol.proto.DataTransferProtos.Status.SUCCESS;
+      public boolean hasStatus() {
+        return ((bitField0_ & 0x00000001) == 0x00000001);
+      }
+      public org.apache.hadoop.hdfs.protocol.proto.DataTransferProtos.Status getStatus() {
+        return status_;
+      }
+      public Builder setStatus(org.apache.hadoop.hdfs.protocol.proto.DataTransferProtos.Status value) {
+        if (value == null) {
+          throw new NullPointerException();
+        }
+        bitField0_ |= 0x00000001;
+        status_ = value;
+        onChanged();
+        return this;
+      }
+      public Builder clearStatus() {
+        bitField0_ = (bitField0_ & ~0x00000001);
+        status_ = org.apache.hadoop.hdfs.protocol.proto.DataTransferProtos.Status.SUCCESS;
+        onChanged();
+        return this;
+      }
+      
+      // optional string firstBadLink = 2;
+      private java.lang.Object firstBadLink_ = "";
+      public boolean hasFirstBadLink() {
+        return ((bitField0_ & 0x00000002) == 0x00000002);
+      }
+      public String getFirstBadLink() {
+        java.lang.Object ref = firstBadLink_;
+        if (!(ref instanceof String)) {
+          String s = ((com.google.protobuf.ByteString) ref).toStringUtf8();
+          firstBadLink_ = s;
+          return s;
+        } else {
+          return (String) ref;
+        }
+      }
+      public Builder setFirstBadLink(String value) {
+        if (value == null) {
+    throw new NullPointerException();
+  }
+  bitField0_ |= 0x00000002;
+        firstBadLink_ = value;
+        onChanged();
+        return this;
+      }
+      public Builder clearFirstBadLink() {
+        bitField0_ = (bitField0_ & ~0x00000002);
+        firstBadLink_ = getDefaultInstance().getFirstBadLink();
+        onChanged();
+        return this;
+      }
+      void setFirstBadLink(com.google.protobuf.ByteString value) {
+        bitField0_ |= 0x00000002;
+        firstBadLink_ = value;
+        onChanged();
+      }
+      
+      // optional .OpBlockChecksumResponseProto checksumResponse = 3;
+      private org.apache.hadoop.hdfs.protocol.proto.DataTransferProtos.OpBlockChecksumResponseProto checksumResponse_ = org.apache.hadoop.hdfs.protocol.proto.DataTransferProtos.OpBlockChecksumResponseProto.getDefaultInstance();
+      private com.google.protobuf.SingleFieldBuilder<
+          org.apache.hadoop.hdfs.protocol.proto.DataTransferProtos.OpBlockChecksumResponseProto, org.apache.hadoop.hdfs.protocol.proto.DataTransferProtos.OpBlockChecksumResponseProto.Builder, org.apache.hadoop.hdfs.protocol.proto.DataTransferProtos.OpBlockChecksumResponseProtoOrBuilder> checksumResponseBuilder_;
+      public boolean hasChecksumResponse() {
+        return ((bitField0_ & 0x00000004) == 0x00000004);
+      }
+      public org.apache.hadoop.hdfs.protocol.proto.DataTransferProtos.OpBlockChecksumResponseProto getChecksumResponse() {
+        if (checksumResponseBuilder_ == null) {
+          return checksumResponse_;
+        } else {
+          return checksumResponseBuilder_.getMessage();
+        }
+      }
+      public Builder setChecksumResponse(org.apache.hadoop.hdfs.protocol.proto.DataTransferProtos.OpBlockChecksumResponseProto value) {
+        if (checksumResponseBuilder_ == null) {
+          if (value == null) {
+            throw new NullPointerException();
+          }
+          checksumResponse_ = value;
+          onChanged();
+        } else {
+          checksumResponseBuilder_.setMessage(value);
+        }
+        bitField0_ |= 0x00000004;
+        return this;
+      }
+      public Builder setChecksumResponse(
+          org.apache.hadoop.hdfs.protocol.proto.DataTransferProtos.OpBlockChecksumResponseProto.Builder builderForValue) {
+        if (checksumResponseBuilder_ == null) {
+          checksumResponse_ = builderForValue.build();
+          onChanged();
+        } else {
+          checksumResponseBuilder_.setMessage(builderForValue.build());
+        }
+        bitField0_ |= 0x00000004;
+        return this;
+      }
+      public Builder mergeChecksumResponse(org.apache.hadoop.hdfs.protocol.proto.DataTransferProtos.OpBlockChecksumResponseProto value) {
+        if (checksumResponseBuilder_ == null) {
+          if (((bitField0_ & 0x00000004) == 0x00000004) &&
+              checksumResponse_ != org.apache.hadoop.hdfs.protocol.proto.DataTransferProtos.OpBlockChecksumResponseProto.getDefaultInstance()) {
+            checksumResponse_ =
+              org.apache.hadoop.hdfs.protocol.proto.DataTransferProtos.OpBlockChecksumResponseProto.newBuilder(checksumResponse_).mergeFrom(value).buildPartial();
+          } else {
+            checksumResponse_ = value;
+          }
+          onChanged();
+        } else {
+          checksumResponseBuilder_.mergeFrom(value);
+        }
+        bitField0_ |= 0x00000004;
+        return this;
+      }
+      public Builder clearChecksumResponse() {
+        if (checksumResponseBuilder_ == null) {
+          checksumResponse_ = org.apache.hadoop.hdfs.protocol.proto.DataTransferProtos.OpBlockChecksumResponseProto.getDefaultInstance();
+          onChanged();
+        } else {
+          checksumResponseBuilder_.clear();
+        }
+        bitField0_ = (bitField0_ & ~0x00000004);
+        return this;
+      }
+      public org.apache.hadoop.hdfs.protocol.proto.DataTransferProtos.OpBlockChecksumResponseProto.Builder getChecksumResponseBuilder() {
+        bitField0_ |= 0x00000004;
+        onChanged();
+        return getChecksumResponseFieldBuilder().getBuilder();
+      }
+      public org.apache.hadoop.hdfs.protocol.proto.DataTransferProtos.OpBlockChecksumResponseProtoOrBuilder getChecksumResponseOrBuilder() {
+        if (checksumResponseBuilder_ != null) {
+          return checksumResponseBuilder_.getMessageOrBuilder();
+        } else {
+          return checksumResponse_;
+        }
+      }
+      private com.google.protobuf.SingleFieldBuilder<
+          org.apache.hadoop.hdfs.protocol.proto.DataTransferProtos.OpBlockChecksumResponseProto, org.apache.hadoop.hdfs.protocol.proto.DataTransferProtos.OpBlockChecksumResponseProto.Builder, org.apache.hadoop.hdfs.protocol.proto.DataTransferProtos.OpBlockChecksumResponseProtoOrBuilder> 
+          getChecksumResponseFieldBuilder() {
+        if (checksumResponseBuilder_ == null) {
+          checksumResponseBuilder_ = new com.google.protobuf.SingleFieldBuilder<
+              org.apache.hadoop.hdfs.protocol.proto.DataTransferProtos.OpBlockChecksumResponseProto, org.apache.hadoop.hdfs.protocol.proto.DataTransferProtos.OpBlockChecksumResponseProto.Builder, org.apache.hadoop.hdfs.protocol.proto.DataTransferProtos.OpBlockChecksumResponseProtoOrBuilder>(
+                  checksumResponse_,
+                  getParentForChildren(),
+                  isClean());
+          checksumResponse_ = null;
+        }
+        return checksumResponseBuilder_;
+      }
+      
+      // @@protoc_insertion_point(builder_scope:BlockOpResponseProto)
+    }
+    
+    static {
+      defaultInstance = new BlockOpResponseProto(true);
+      defaultInstance.initFields();
+    }
+    
+    // @@protoc_insertion_point(class_scope:BlockOpResponseProto)
+  }
+  
+  public interface ClientReadStatusProtoOrBuilder
+      extends com.google.protobuf.MessageOrBuilder {
+    
+    // required .Status status = 1;
+    boolean hasStatus();
+    org.apache.hadoop.hdfs.protocol.proto.DataTransferProtos.Status getStatus();
+  }
+  public static final class ClientReadStatusProto extends
+      com.google.protobuf.GeneratedMessage
+      implements ClientReadStatusProtoOrBuilder {
+    // Use ClientReadStatusProto.newBuilder() to construct.
+    private ClientReadStatusProto(Builder builder) {
+      super(builder);
+    }
+    private ClientReadStatusProto(boolean noInit) {}
+    
+    private static final ClientReadStatusProto defaultInstance;
+    public static ClientReadStatusProto getDefaultInstance() {
+      return defaultInstance;
+    }
+    
+    public ClientReadStatusProto getDefaultInstanceForType() {
+      return defaultInstance;
+    }
+    
+    public static final com.google.protobuf.Descriptors.Descriptor
+        getDescriptor() {
+      return org.apache.hadoop.hdfs.protocol.proto.DataTransferProtos.internal_static_ClientReadStatusProto_descriptor;
+    }
+    
+    protected com.google.protobuf.GeneratedMessage.FieldAccessorTable
+        internalGetFieldAccessorTable() {
+      return org.apache.hadoop.hdfs.protocol.proto.DataTransferProtos.internal_static_ClientReadStatusProto_fieldAccessorTable;
+    }
+    
+    private int bitField0_;
+    // required .Status status = 1;
+    public static final int STATUS_FIELD_NUMBER = 1;
+    private org.apache.hadoop.hdfs.protocol.proto.DataTransferProtos.Status status_;
+    public boolean hasStatus() {
+      return ((bitField0_ & 0x00000001) == 0x00000001);
+    }
+    public org.apache.hadoop.hdfs.protocol.proto.DataTransferProtos.Status getStatus() {
+      return status_;
+    }
+    
+    private void initFields() {
+      status_ = org.apache.hadoop.hdfs.protocol.proto.DataTransferProtos.Status.SUCCESS;
+    }
+    private byte memoizedIsInitialized = -1;
+    public final boolean isInitialized() {
+      byte isInitialized = memoizedIsInitialized;
+      if (isInitialized != -1) return isInitialized == 1;
+      
+      if (!hasStatus()) {
+        memoizedIsInitialized = 0;
+        return false;
+      }
+      memoizedIsInitialized = 1;
+      return true;
+    }
+    
+    public void writeTo(com.google.protobuf.CodedOutputStream output)
+                        throws java.io.IOException {
+      getSerializedSize();
+      if (((bitField0_ & 0x00000001) == 0x00000001)) {
+        output.writeEnum(1, status_.getNumber());
+      }
+      getUnknownFields().writeTo(output);
+    }
+    
+    private int memoizedSerializedSize = -1;
+    public int getSerializedSize() {
+      int size = memoizedSerializedSize;
+      if (size != -1) return size;
+    
+      size = 0;
+      if (((bitField0_ & 0x00000001) == 0x00000001)) {
+        size += com.google.protobuf.CodedOutputStream
+          .computeEnumSize(1, status_.getNumber());
+      }
+      size += getUnknownFields().getSerializedSize();
+      memoizedSerializedSize = size;
+      return size;
+    }
+    
+    private static final long serialVersionUID = 0L;
+    @java.lang.Override
+    protected java.lang.Object writeReplace()
+        throws java.io.ObjectStreamException {
+      return super.writeReplace();
+    }
+    
+    @java.lang.Override
+    public boolean equals(final java.lang.Object obj) {
+      if (obj == this) {
+       return true;
+      }
+      if (!(obj instanceof org.apache.hadoop.hdfs.protocol.proto.DataTransferProtos.ClientReadStatusProto)) {
+        return super.equals(obj);
+      }
+      org.apache.hadoop.hdfs.protocol.proto.DataTransferProtos.ClientReadStatusProto other = (org.apache.hadoop.hdfs.protocol.proto.DataTransferProtos.ClientReadStatusProto) obj;
+      
+      boolean result = true;
+      result = result && (hasStatus() == other.hasStatus());
+      if (hasStatus()) {
+        result = result &&
+            (getStatus() == other.getStatus());
+      }
+      result = result &&
+          getUnknownFields().equals(other.getUnknownFields());
+      return result;
+    }
+    
+    @java.lang.Override
+    public int hashCode() {
+      int hash = 41;
+      hash = (19 * hash) + getDescriptorForType().hashCode();
+      if (hasStatus()) {
+        hash = (37 * hash) + STATUS_FIELD_NUMBER;
+        hash = (53 * hash) + hashEnum(getStatus());
+      }
+      hash = (29 * hash) + getUnknownFields().hashCode();
+      return hash;
+    }
+    
+    public static org.apache.hadoop.hdfs.protocol.proto.DataTransferProtos.ClientReadStatusProto parseFrom(
+        com.google.protobuf.ByteString data)
+        throws com.google.protobuf.InvalidProtocolBufferException {
+      return newBuilder().mergeFrom(data).buildParsed();
+    }
+    public static org.apache.hadoop.hdfs.protocol.proto.DataTransferProtos.ClientReadStatusProto parseFrom(
+        com.google.protobuf.ByteString data,
+        com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+        throws com.google.protobuf.InvalidProtocolBufferException {
+      return newBuilder().mergeFrom(data, extensionRegistry)
+               .buildParsed();
+    }
+    public static org.apache.hadoop.hdfs.protocol.proto.DataTransferProtos.ClientReadStatusProto parseFrom(byte[] data)
+        throws com.google.protobuf.InvalidProtocolBufferException {
+      return newBuilder().mergeFrom(data).buildParsed();
+    }
+    public static org.apache.hadoop.hdfs.protocol.proto.DataTransferProtos.ClientReadStatusProto parseFrom(
+        byte[] data,
+        com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+        throws com.google.protobuf.InvalidProtocolBufferException {
+      return newBuilder().mergeFrom(data, extensionRegistry)
+               .buildParsed();
+    }
+    public static org.apache.hadoop.hdfs.protocol.proto.DataTransferProtos.ClientReadStatusProto parseFrom(java.io.InputStream input)
+        throws java.io.IOException {
+      return newBuilder().mergeFrom(input).buildParsed();
+    }
+    public static org.apache.hadoop.hdfs.protocol.proto.DataTransferProtos.ClientReadStatusProto parseFrom(
+        java.io.InputStream input,
+        com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+        throws java.io.IOException {
+      return newBuilder().mergeFrom(input, extensionRegistry)
+               .buildParsed();
+    }
+    public static org.apache.hadoop.hdfs.protocol.proto.DataTransferProtos.ClientReadStatusProto parseDelimitedFrom(java.io.InputStream input)
+        throws java.io.IOException {
+      Builder builder = newBuilder();
+      if (builder.mergeDelimitedFrom(input)) {
+        return builder.buildParsed();
+      } else {
+        return null;
+      }
+    }
+    public static org.apache.hadoop.hdfs.protocol.proto.DataTransferProtos.ClientReadStatusProto parseDelimitedFrom(
+        java.io.InputStream input,
+        com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+        throws java.io.IOException {
+      Builder builder = newBuilder();
+      if (builder.mergeDelimitedFrom(input, extensionRegistry)) {
+        return builder.buildParsed();
+      } else {
+        return null;
+      }
+    }
+    public static org.apache.hadoop.hdfs.protocol.proto.DataTransferProtos.ClientReadStatusProto parseFrom(
+        com.google.protobuf.CodedInputStream input)
+        throws java.io.IOException {
+      return newBuilder().mergeFrom(input).buildParsed();
+    }
+    public static org.apache.hadoop.hdfs.protocol.proto.DataTransferProtos.ClientReadStatusProto parseFrom(
+        com.google.protobuf.CodedInputStream input,
+        com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+        throws java.io.IOException {
+      return newBuilder().mergeFrom(input, extensionRegistry)
+               .buildParsed();
+    }
+    
+    public static Builder newBuilder() { return Builder.create(); }
+    public Builder newBuilderForType() { return newBuilder(); }
+    public static Builder newBuilder(org.apache.hadoop.hdfs.protocol.proto.DataTransferProtos.ClientReadStatusProto prototype) {
+      return newBuilder().mergeFrom(prototype);
+    }
+    public Builder toBuilder() { return newBuilder(this); }
+    
+    @java.lang.Override
+    protected Builder newBuilderForType(
+        com.google.protobuf.GeneratedMessage.BuilderParent parent) {
+      Builder builder = new Builder(parent);
+      return builder;
+    }
+    public static final class Builder extends
+        com.google.protobuf.GeneratedMessage.Builder<Builder>
+       implements org.apache.hadoop.hdfs.protocol.proto.DataTransferProtos.ClientReadStatusProtoOrBuilder {
+      public static final com.google.protobuf.Descriptors.Descriptor
+          getDescriptor() {
+        return org.apache.hadoop.hdfs.protocol.proto.DataTransferProtos.internal_static_ClientReadStatusProto_descriptor;
+      }
+      
+      protected com.google.protobuf.GeneratedMessage.FieldAccessorTable
+          internalGetFieldAccessorTable() {
+        return org.apache.hadoop.hdfs.protocol.proto.DataTransferProtos.internal_static_ClientReadStatusProto_fieldAccessorTable;
+      }
+      
+      // Construct using org.apache.hadoop.hdfs.protocol.proto.DataTransferProtos.ClientReadStatusProto.newBuilder()
+      private Builder() {
+        maybeForceBuilderInitialization();
+      }
+      
+      private Builder(BuilderParent parent) {
+        super(parent);
+        maybeForceBuilderInitialization();
+      }
+      private void maybeForceBuilderInitialization() {
+        if (com.google.protobuf.GeneratedMessage.alwaysUseFieldBuilders) {
+        }
+      }
+      private static Builder create() {
+        return new Builder();
+      }
+      
+      public Builder clear() {
+        super.clear();
+        status_ = org.apache.hadoop.hdfs.protocol.proto.DataTransferProtos.Status.SUCCESS;
+        bitField0_ = (bitField0_ & ~0x00000001);
+        return this;
+      }
+      
+      public Builder clone() {
+        return create().mergeFrom(buildPartial());
+      }
+      
+      public com.google.protobuf.Descriptors.Descriptor
+          getDescriptorForType() {
+        return org.apache.hadoop.hdfs.protocol.proto.DataTransferProtos.ClientReadStatusProto.getDescriptor();
+      }
+      
+      public org.apache.hadoop.hdfs.protocol.proto.DataTransferProtos.ClientReadStatusProto getDefaultInstanceForType() {
+        return org.apache.hadoop.hdfs.protocol.proto.DataTransferProtos.ClientReadStatusProto.getDefaultInstance();
+      }
+      
+      public org.apache.hadoop.hdfs.protocol.proto.DataTransferProtos.ClientReadStatusProto build() {
+        org.apache.hadoop.hdfs.protocol.proto.DataTransferProtos.ClientReadStatusProto result = buildPartial();
+        if (!result.isInitialized()) {
+          throw newUninitializedMessageException(result);
+        }
+        return result;
+      }
+      
+      private org.apache.hadoop.hdfs.protocol.proto.DataTransferProtos.ClientReadStatusProto buildParsed()
+          throws com.google.protobuf.InvalidProtocolBufferException {
+        org.apache.hadoop.hdfs.protocol.proto.DataTransferProtos.ClientReadStatusProto result = buildPartial();
+        if (!result.isInitialized()) {
+          throw newUninitializedMessageException(
+            result).asInvalidProtocolBufferException();
+        }
+        return result;
+      }
+      
+      public org.apache.hadoop.hdfs.protocol.proto.DataTransferProtos.ClientReadStatusProto buildPartial() {
+        org.apache.hadoop.hdfs.protocol.proto.DataTransferProtos.ClientReadStatusProto result = new org.apache.hadoop.hdfs.protocol.proto.DataTransferProtos.ClientReadStatusProto(this);
+        int from_bitField0_ = bitField0_;
+        int to_bitField0_ = 0;
+        if (((from_bitField0_ & 0x00000001) == 0x00000001)) {
+          to_bitField0_ |= 0x00000001;
+        }
+        result.status_ = status_;
+        result.bitField0_ = to_bitField0_;
+        onBuilt();
+        return result;
+      }
+      
+      public Builder mergeFrom(com.google.protobuf.Message other) {
+        if (other instanceof org.apache.hadoop.hdfs.protocol.proto.DataTransferProtos.ClientReadStatusProto) {
+          return mergeFrom((org.apache.hadoop.hdfs.protocol.proto.DataTransferProtos.ClientReadStatusProto)other);
+        } else {
+          super.mergeFrom(other);
+          return this;
+        }
+      }
+      
+      public Builder mergeFrom(org.apache.hadoop.hdfs.protocol.proto.DataTransferProtos.ClientReadStatusProto other) {
+        if (other == org.apache.hadoop.hdfs.protocol.proto.DataTransferProtos.ClientReadStatusProto.getDefaultInstance()) return this;
+        if (other.hasStatus()) {
+          setStatus(other.getStatus());
+        }
+        this.mergeUnknownFields(other.getUnknownFields());
+        return this;
+      }
+      
+      public final boolean isInitialized() {
+        if (!hasStatus()) {
+          
+          return false;
+        }
+        return true;
+      }
+      
+      public Builder mergeFrom(
+          com.google.protobuf.CodedInputStream input,
+          com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+          throws java.io.IOException {
+        com.google.protobuf.UnknownFieldSet.Builder unknownFields =
+          com.google.protobuf.UnknownFieldSet.newBuilder(
+            this.getUnknownFields());
+        while (true) {
+          int tag = input.readTag();
+          switch (tag) {
+            case 0:
+              this.setUnknownFields(unknownFields.build());
+              onChanged();
+              return this;
+            default: {
+              if (!parseUnknownField(input, unknownFields,
+                                     extensionRegistry, tag)) {
+                this.setUnknownFields(unknownFields.build());
+                onChanged();
+                return this;
+              }
+              break;
+            }
+            case 8: {
+              int rawValue = input.readEnum();
+              org.apache.hadoop.hdfs.protocol.proto.DataTransferProtos.Status value = org.apache.hadoop.hdfs.protocol.proto.DataTransferProtos.Status.valueOf(rawValue);
+              if (value == null) {
+                unknownFields.mergeVarintField(1, rawValue);
+              } else {
+                bitField0_ |= 0x00000001;
+                status_ = value;
+              }
+              break;
+            }
+          }
+        }
+      }
+      
+      private int bitField0_;
+      
+      // required .Status status = 1;
+      private org.apache.hadoop.hdfs.protocol.proto.DataTransferProtos.Status status_ = org.apache.hadoop.hdfs.protocol.proto.DataTransferProtos.Status.SUCCESS;
+      public boolean hasStatus() {
+        return ((bitField0_ & 0x00000001) == 0x00000001);
+      }
+      public org.apache.hadoop.hdfs.protocol.proto.DataTransferProtos.Status getStatus() {
+        return status_;
+      }
+      public Builder setStatus(org.apache.hadoop.hdfs.protocol.proto.DataTransferProtos.Status value) {
+        if (value == null) {
+          throw new NullPointerException();
+        }
+        bitField0_ |= 0x00000001;
+        status_ = value;
+        onChanged();
+        return this;
+      }
+      public Builder clearStatus() {
+        bitField0_ = (bitField0_ & ~0x00000001);
+        status_ = org.apache.hadoop.hdfs.protocol.proto.DataTransferProtos.Status.SUCCESS;
+        onChanged();
+        return this;
+      }
+      
+      // @@protoc_insertion_point(builder_scope:ClientReadStatusProto)
+    }
+    
+    static {
+      defaultInstance = new ClientReadStatusProto(true);
+      defaultInstance.initFields();
+    }
+    
+    // @@protoc_insertion_point(class_scope:ClientReadStatusProto)
+  }
+  
+  public interface DNTransferAckProtoOrBuilder
+      extends com.google.protobuf.MessageOrBuilder {
+    
+    // required .Status status = 1;
+    boolean hasStatus();
+    org.apache.hadoop.hdfs.protocol.proto.DataTransferProtos.Status getStatus();
+  }
+  public static final class DNTransferAckProto extends
+      com.google.protobuf.GeneratedMessage
+      implements DNTransferAckProtoOrBuilder {
+    // Use DNTransferAckProto.newBuilder() to construct.
+    private DNTransferAckProto(Builder builder) {
+      super(builder);
+    }
+    private DNTransferAckProto(boolean noInit) {}
+    
+    private static final DNTransferAckProto defaultInstance;
+    public static DNTransferAckProto getDefaultInstance() {
+      return defaultInstance;
+    }
+    
+    public DNTransferAckProto getDefaultInstanceForType() {
+      return defaultInstance;
+    }
+    
+    public static final com.google.protobuf.Descriptors.Descriptor
+        getDescriptor() {
+      return org.apache.hadoop.hdfs.protocol.proto.DataTransferProtos.internal_static_DNTransferAckProto_descriptor;
+    }
+    
+    protected com.google.protobuf.GeneratedMessage.FieldAccessorTable
+        internalGetFieldAccessorTable() {
+      return org.apache.hadoop.hdfs.protocol.proto.DataTransferProtos.internal_static_DNTransferAckProto_fieldAccessorTable;
+    }
+    
+    private int bitField0_;
+    // required .Status status = 1;
+    public static final int STATUS_FIELD_NUMBER = 1;
+    private org.apache.hadoop.hdfs.protocol.proto.DataTransferProtos.Status status_;
+    public boolean hasStatus() {
+      return ((bitField0_ & 0x00000001) == 0x00000001);
+    }
+    public org.apache.hadoop.hdfs.protocol.proto.DataTransferProtos.Status getStatus() {
+      return status_;
+    }
+    
+    private void initFields() {
+      status_ = org.apache.hadoop.hdfs.protocol.proto.DataTransferProtos.Status.SUCCESS;
+    }
+    private byte memoizedIsInitialized = -1;
+    public final boolean isInitialized() {
+      byte isInitialized = memoizedIsInitialized;
+      if (isInitialized != -1) return isInitialized == 1;
+      
+      if (!hasStatus()) {
+        memoizedIsInitialized = 0;
+        return false;
+      }
+      memoizedIsInitialized = 1;
+      return true;
+    }
+    
+    public void writeTo(com.google.protobuf.CodedOutputStream output)
+                        throws java.io.IOException {
+      getSerializedSize();
+      if (((bitField0_ & 0x00000001) == 0x00000001)) {
+        output.writeEnum(1, status_.getNumber());
+      }
+      getUnknownFields().writeTo(output);
+    }
+    
+    private int memoizedSerializedSize = -1;
+    public int getSerializedSize() {
+      int size = memoizedSerializedSize;
+      if (size != -1) return size;
+    
+      size = 0;
+      if (((bitField0_ & 0x00000001) == 0x00000001)) {
+        size += com.google.protobuf.CodedOutputStream
+          .computeEnumSize(1, status_.getNumber());
+      }
+      size += getUnknownFields().getSerializedSize();
+      memoizedSerializedSize = size;
+      return size;
+    }
+    
+    private static final long serialVersionUID = 0L;
+    @java.lang.Override
+    protected java.lang.Object writeReplace()
+        throws java.io.ObjectStreamException {
+      return super.writeReplace();
+    }
+    
+    @java.lang.Override
+    public boolean equals(final java.lang.Object obj) {
+      if (obj == this) {
+       return true;
+      }
+      if (!(obj instanceof org.apache.hadoop.hdfs.protocol.proto.DataTransferProtos.DNTransferAckProto)) {
+        return super.equals(obj);
+      }
+      org.apache.hadoop.hdfs.protocol.proto.DataTransferProtos.DNTransferAckProto other = (org.apache.hadoop.hdfs.protocol.proto.DataTransferProtos.DNTransferAckProto) obj;
+      
+      boolean result = true;
+      result = result && (hasStatus() == other.hasStatus());
+      if (hasStatus()) {
+        result = result &&
+            (getStatus() == other.getStatus());
+      }
+      result = result &&
+          getUnknownFields().equals(other.getUnknownFields());
+      return result;
+    }
+    
+    @java.lang.Override
+    public int hashCode() {
+      int hash = 41;
+      hash = (19 * hash) + getDescriptorForType().hashCode();
+      if (hasStatus()) {
+        hash = (37 * hash) + STATUS_FIELD_NUMBER;
+        hash = (53 * hash) + hashEnum(getStatus());
+      }
+      hash = (29 * hash) + getUnknownFields().hashCode();
+      return hash;
+    }
+    
+    public static org.apache.hadoop.hdfs.protocol.proto.DataTransferProtos.DNTransferAckProto parseFrom(
+        com.google.protobuf.ByteString data)
+        throws com.google.protobuf.InvalidProtocolBufferException {
+      return newBuilder().mergeFrom(data).buildParsed();
+    }
+    public static org.apache.hadoop.hdfs.protocol.proto.DataTransferProtos.DNTransferAckProto parseFrom(
+        com.google.protobuf.ByteString data,
+        com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+        throws com.google.protobuf.InvalidProtocolBufferException {
+      return newBuilder().mergeFrom(data, extensionRegistry)
+               .buildParsed();
+    }
+    public static org.apache.hadoop.hdfs.protocol.proto.DataTransferProtos.DNTransferAckProto parseFrom(byte[] data)
+        throws com.google.protobuf.InvalidProtocolBufferException {
+      return newBuilder().mergeFrom(data).buildParsed();
+    }
+    public static org.apache.hadoop.hdfs.protocol.proto.DataTransferProtos.DNTransferAckProto parseFrom(
+        byte[] data,
+        com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+        throws com.google.protobuf.InvalidProtocolBufferException {
+      return newBuilder().mergeFrom(data, extensionRegistry)
+               .buildParsed();
+    }
+    public static org.apache.hadoop.hdfs.protocol.proto.DataTransferProtos.DNTransferAckProto parseFrom(java.io.InputStream input)
+        throws java.io.IOException {
+      return newBuilder().mergeFrom(input).buildParsed();
+    }
+    public static org.apache.hadoop.hdfs.protocol.proto.DataTransferProtos.DNTransferAckProto parseFrom(
+        java.io.InputStream input,
+        com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+        throws java.io.IOException {
+      return newBuilder().mergeFrom(input, extensionRegistry)
+               .buildParsed();
+    }
+    public static org.apache.hadoop.hdfs.protocol.proto.DataTransferProtos.DNTransferAckProto parseDelimitedFrom(java.io.InputStream input)
+        throws java.io.IOException {
+      Builder builder = newBuilder();
+      if (builder.mergeDelimitedFrom(input)) {
+        return builder.buildParsed();
+      } else {
+        return null;
+      }
+    }
+    public static org.apache.hadoop.hdfs.protocol.proto.DataTransferProtos.DNTransferAckProto parseDelimitedFrom(
+        java.io.InputStream input,
+        com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+        throws java.io.IOException {
+      Builder builder = newBuilder();
+      if (builder.mergeDelimitedFrom(input, extensionRegistry)) {
+        return builder.buildParsed();
+      } else {
+        return null;
+      }
+    }
+    public static org.apache.hadoop.hdfs.protocol.proto.DataTransferProtos.DNTransferAckProto parseFrom(
+        com.google.protobuf.CodedInputStream input)
+        throws java.io.IOException {
+      return newBuilder().mergeFrom(input).buildParsed();
+    }
+    public static org.apache.hadoop.hdfs.protocol.proto.DataTransferProtos.DNTransferAckProto parseFrom(
+        com.google.protobuf.CodedInputStream input,
+        com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+        throws java.io.IOException {
+      return newBuilder().mergeFrom(input, extensionRegistry)
+               .buildParsed();
+    }
+    
+    public static Builder newBuilder() { return Builder.create(); }
+    public Builder newBuilderForType() { return newBuilder(); }
+    public static Builder newBuilder(org.apache.hadoop.hdfs.protocol.proto.DataTransferProtos.DNTransferAckProto prototype) {
+      return newBuilder().mergeFrom(prototype);
+    }
+    public Builder toBuilder() { return newBuilder(this); }
+    
+    @java.lang.Override
+    protected Builder newBuilderForType(
+        com.google.protobuf.GeneratedMessage.BuilderParent parent) {
+      Builder builder = new Builder(parent);
+      return builder;
+    }
+    public static final class Builder extends
+        com.google.protobuf.GeneratedMessage.Builder<Builder>
+       implements org.apache.hadoop.hdfs.protocol.proto.DataTransferProtos.DNTransferAckProtoOrBuilder {
+      public static final com.google.protobuf.Descriptors.Descriptor
+          getDescriptor() {
+        return org.apache.hadoop.hdfs.protocol.proto.DataTransferProtos.internal_static_DNTransferAckProto_descriptor;
+      }
+      
+      protected com.google.protobuf.GeneratedMessage.FieldAccessorTable
+          internalGetFieldAccessorTable() {
+        return org.apache.hadoop.hdfs.protocol.proto.DataTransferProtos.internal_static_DNTransferAckProto_fieldAccessorTable;
+      }
+      
+      // Construct using org.apache.hadoop.hdfs.protocol.proto.DataTransferProtos.DNTransferAckProto.newBuilder()
+      private Builder() {
+        maybeForceBuilderInitialization();
+      }
+      
+      private Builder(BuilderParent parent) {
+        super(parent);
+        maybeForceBuilderInitialization();
+      }
+      private void maybeForceBuilderInitialization() {
+        if (com.google.protobuf.GeneratedMessage.alwaysUseFieldBuilders) {
+        }
+      }
+      private static Builder create() {
+        return new Builder();
+      }
+      
+      public Builder clear() {
+        super.clear();
+        status_ = org.apache.hadoop.hdfs.protocol.proto.DataTransferProtos.Status.SUCCESS;
+        bitField0_ = (bitField0_ & ~0x00000001);
+        return this;
+      }
+      
+      public Builder clone() {
+        return create().mergeFrom(buildPartial());
+      }
+      
+      public com.google.protobuf.Descriptors.Descriptor
+          getDescriptorForType() {
+        return org.apache.hadoop.hdfs.protocol.proto.DataTransferProtos.DNTransferAckProto.getDescriptor();
+      }
+      
+      public org.apache.hadoop.hdfs.protocol.proto.DataTransferProtos.DNTransferAckProto getDefaultInstanceForType() {
+        return org.apache.hadoop.hdfs.protocol.proto.DataTransferProtos.DNTransferAckProto.getDefaultInstance();
+      }
+      
+      public org.apache.hadoop.hdfs.protocol.proto.DataTransferProtos.DNTransferAckProto build() {
+        org.apache.hadoop.hdfs.protocol.proto.DataTransferProtos.DNTransferAckProto result = buildPartial();
+        if (!result.isInitialized()) {
+          throw newUninitializedMessageException(result);
+        }
+        return result;
+      }
+      
+      private org.apache.hadoop.hdfs.protocol.proto.DataTransferProtos.DNTransferAckProto buildParsed()
+          throws com.google.protobuf.InvalidProtocolBufferException {
+        org.apache.hadoop.hdfs.protocol.proto.DataTransferProtos.DNTransferAckProto result = buildPartial();
+        if (!result.isInitialized()) {
+          throw newUninitializedMessageException(
+            result).asInvalidProtocolBufferException();
+        }
+        return result;
+      }
+      
+      public org.apache.hadoop.hdfs.protocol.proto.DataTransferProtos.DNTransferAckProto buildPartial() {
+        org.apache.hadoop.hdfs.protocol.proto.DataTransferProtos.DNTransferAckProto result = new org.apache.hadoop.hdfs.protocol.proto.DataTransferProtos.DNTransferAckProto(this);
+        int from_bitField0_ = bitField0_;
+        int to_bitField0_ = 0;
+        if (((from_bitField0_ & 0x00000001) == 0x00000001)) {
+          to_bitField0_ |= 0x00000001;
+        }
+        result.status_ = status_;
+        result.bitField0_ = to_bitField0_;
+        onBuilt();
+        return result;
+      }
+      
+      public Builder mergeFrom(com.google.protobuf.Message other) {
+        if (other instanceof org.apache.hadoop.hdfs.protocol.proto.DataTransferProtos.DNTransferAckProto) {
+          return mergeFrom((org.apache.hadoop.hdfs.protocol.proto.DataTransferProtos.DNTransferAckProto)other);
+        } else {
+          super.mergeFrom(other);
+          return this;
+        }
+      }
+      
+      public Builder mergeFrom(org.apache.hadoop.hdfs.protocol.proto.DataTransferProtos.DNTransferAckProto other) {
+        if (other == org.apache.hadoop.hdfs.protocol.proto.DataTransferProtos.DNTransferAckProto.getDefaultInstance()) return this;
+        if (other.hasStatus()) {
+          setStatus(other.getStatus());
+        }
+        this.mergeUnknownFields(other.getUnknownFields());
+        return this;
+      }
+      
+      public final boolean isInitialized() {
+        if (!hasStatus()) {
+          
+          return false;
+        }
+        return true;
+      }
+      
+      public Builder mergeFrom(
+          com.google.protobuf.CodedInputStream input,
+          com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+          throws java.io.IOException {
+        com.google.protobuf.UnknownFieldSet.Builder unknownFields =
+          com.google.protobuf.UnknownFieldSet.newBuilder(
+            this.getUnknownFields());
+        while (true) {
+          int tag = input.readTag();
+          switch (tag) {
+            case 0:
+              this.setUnknownFields(unknownFields.build());
+              onChanged();
+              return this;
+            default: {
+              if (!parseUnknownField(input, unknownFields,
+                                     extensionRegistry, tag)) {
+                this.setUnknownFields(unknownFields.build());
+                onChanged();
+                return this;
+              }
+              break;
+            }
+            case 8: {
+              int rawValue = input.readEnum();
+              org.apache.hadoop.hdfs.protocol.proto.DataTransferProtos.Status value = org.apache.hadoop.hdfs.protocol.proto.DataTransferProtos.Status.valueOf(rawValue);
+              if (value == null) {
+                unknownFields.mergeVarintField(1, rawValue);
+              } else {
+                bitField0_ |= 0x00000001;
+                status_ = value;
+              }
+              break;
+            }
+          }
+        }
+      }
+      
+      private int bitField0_;
+      
+      // required .Status status = 1;
+      private org.apache.hadoop.hdfs.protocol.proto.DataTransferProtos.Status status_ = org.apache.hadoop.hdfs.protocol.proto.DataTransferProtos.Status.SUCCESS;
+      public boolean hasStatus() {
+        return ((bitField0_ & 0x00000001) == 0x00000001);
+      }
+      public org.apache.hadoop.hdfs.protocol.proto.DataTransferProtos.Status getStatus() {
+        return status_;
+      }
+      public Builder setStatus(org.apache.hadoop.hdfs.protocol.proto.DataTransferProtos.Status value) {
+        if (value == null) {
+          throw new NullPointerException();
+        }
+        bitField0_ |= 0x00000001;
+        status_ = value;
+        onChanged();
+        return this;
+      }
+      public Builder clearStatus() {
+        bitField0_ = (bitField0_ & ~0x00000001);
+        status_ = org.apache.hadoop.hdfs.protocol.proto.DataTransferProtos.Status.SUCCESS;
+        onChanged();
+        return this;
+      }
+      
+      // @@protoc_insertion_point(builder_scope:DNTransferAckProto)
+    }
+    
+    static {
+      defaultInstance = new DNTransferAckProto(true);
+      defaultInstance.initFields();
+    }
+    
+    // @@protoc_insertion_point(class_scope:DNTransferAckProto)
+  }
+  
+  public interface OpBlockChecksumResponseProtoOrBuilder
+      extends com.google.protobuf.MessageOrBuilder {
+    
+    // required uint32 bytesPerCrc = 1;
+    boolean hasBytesPerCrc();
+    int getBytesPerCrc();
+    
+    // required uint64 crcPerBlock = 2;
+    boolean hasCrcPerBlock();
+    long getCrcPerBlock();
+    
+    // required bytes md5 = 3;
+    boolean hasMd5();
+    com.google.protobuf.ByteString getMd5();
+  }
+  public static final class OpBlockChecksumResponseProto extends
+      com.google.protobuf.GeneratedMessage
+      implements OpBlockChecksumResponseProtoOrBuilder {
+    // Use OpBlockChecksumResponseProto.newBuilder() to construct.
+    private OpBlockChecksumResponseProto(Builder builder) {
+      super(builder);
+    }
+    private OpBlockChecksumResponseProto(boolean noInit) {}
+    
+    private static final OpBlockChecksumResponseProto defaultInstance;
+    public static OpBlockChecksumResponseProto getDefaultInstance() {
+      return defaultInstance;
+    }
+    
+    public OpBlockChecksumResponseProto getDefaultInstanceForType() {
+      return defaultInstance;
+    }
+    
+    public static final com.google.protobuf.Descriptors.Descriptor
+        getDescriptor() {
+      return org.apache.hadoop.hdfs.protocol.proto.DataTransferProtos.internal_static_OpBlockChecksumResponseProto_descriptor;
+    }
+    
+    protected com.google.protobuf.GeneratedMessage.FieldAccessorTable
+        internalGetFieldAccessorTable() {
+      return org.apache.hadoop.hdfs.protocol.proto.DataTransferProtos.internal_static_OpBlockChecksumResponseProto_fieldAccessorTable;
+    }
+    
+    private int bitField0_;
+    // required uint32 bytesPerCrc = 1;
+    public static final int BYTESPERCRC_FIELD_NUMBER = 1;
+    private int bytesPerCrc_;
+    public boolean hasBytesPerCrc() {
+      return ((bitField0_ & 0x00000001) == 0x00000001);
+    }
+    public int getBytesPerCrc() {
+      return bytesPerCrc_;
+    }
+    
+    // required uint64 crcPerBlock = 2;
+    public static final int CRCPERBLOCK_FIELD_NUMBER = 2;
+    private long crcPerBlock_;
+    public boolean hasCrcPerBlock() {
+      return ((bitField0_ & 0x00000002) == 0x00000002);
+    }
+    public long getCrcPerBlock() {
+      return crcPerBlock_;
+    }
+    
+    // required bytes md5 = 3;
+    public static final int MD5_FIELD_NUMBER = 3;
+    private com.google.protobuf.ByteString md5_;
+    public boolean hasMd5() {
+      return ((bitField0_ & 0x00000004) == 0x00000004);
+    }
+    public com.google.protobuf.ByteString getMd5() {
+      return md5_;
+    }
+    
+    private void initFields() {
+      bytesPerCrc_ = 0;
+      crcPerBlock_ = 0L;
+      md5_ = com.google.protobuf.ByteString.EMPTY;
+    }
+    private byte memoizedIsInitialized = -1;
+    public final boolean isInitialized() {
+      byte isInitialized = memoizedIsInitialized;
+      if (isInitialized != -1) return isInitialized == 1;
+      
+      if (!hasBytesPerCrc()) {
+        memoizedIsInitialized = 0;
+        return false;
+      }
+      if (!hasCrcPerBlock()) {
+        memoizedIsInitialized = 0;
+        return false;
+      }
+      if (!hasMd5()) {
+        memoizedIsInitialized = 0;
+        return false;
+      }
+      memoizedIsInitialized = 1;
+      return true;
+    }
+    
+    public void writeTo(com.google.protobuf.CodedOutputStream output)
+                        throws java.io.IOException {
+      getSerializedSize();
+      if (((bitField0_ & 0x00000001) == 0x00000001)) {
+        output.writeUInt32(1, bytesPerCrc_);
+      }
+      if (((bitField0_ & 0x00000002) == 0x00000002)) {
+        output.writeUInt64(2, crcPerBlock_);
+      }
+      if (((bitField0_ & 0x00000004) == 0x00000004)) {
+        output.writeBytes(3, md5_);
+      }
+      getUnknownFields().writeTo(output);
+    }
+    
+    private int memoizedSerializedSize = -1;
+    public int getSerializedSize() {
+      int size = memoizedSerializedSize;
+      if (size != -1) return size;
+    
+      size = 0;
+      if (((bitField0_ & 0x00000001) == 0x00000001)) {
+        size += com.google.protobuf.CodedOutputStream
+          .computeUInt32Size(1, bytesPerCrc_);
+      }
+      if (((bitField0_ & 0x00000002) == 0x00000002)) {
+        size += com.google.protobuf.CodedOutputStream
+          .computeUInt64Size(2, crcPerBlock_);
+      }
+      if (((bitField0_ & 0x00000004) == 0x00000004)) {
+        size += com.google.protobuf.CodedOutputStream
+          .computeBytesSize(3, md5_);
+      }
+      size += getUnknownFields().getSerializedSize();
+      memoizedSerializedSize = size;
+      return size;
+    }
+    
+    private static final long serialVersionUID = 0L;
+    @java.lang.Override
+    protected java.lang.Object writeReplace()
+        throws java.io.ObjectStreamException {
+      return super.writeReplace();
+    }
+    
+    @java.lang.Override
+    public boolean equals(final java.lang.Object obj) {
+      if (obj == this) {
+       return true;
+      }
+      if (!(obj instanceof org.apache.hadoop.hdfs.protocol.proto.DataTransferProtos.OpBlockChecksumResponseProto)) {
+        return super.equals(obj);
+      }
+      org.apache.hadoop.hdfs.protocol.proto.DataTransferProtos.OpBlockChecksumResponseProto other = (org.apache.hadoop.hdfs.protocol.proto.DataTransferProtos.OpBlockChecksumResponseProto) obj;
+      
+      boolean result = true;
+      result = result && (hasBytesPerCrc() == other.hasBytesPerCrc());
+      if (hasBytesPerCrc()) {
+        result = result && (getBytesPerCrc()
+            == other.getBytesPerCrc());
+      }
+      result = result && (hasCrcPerBlock() == other.hasCrcPerBlock());
+      if (hasCrcPerBlock()) {
+        result = result && (getCrcPerBlock()
+            == other.getCrcPerBlock());
+      }
+      result = result && (hasMd5() == other.hasMd5());
+      if (hasMd5()) {
+        result = result && getMd5()
+            .equals(other.getMd5());
+      }
+      result = result &&
+          getUnknownFields().equals(other.getUnknownFields());
+      return result;
+    }
+    
+    @java.lang.Override
+    public int hashCode() {
+      int hash = 41;
+      hash = (19 * hash) + getDescriptorForType().hashCode();
+      if (hasBytesPerCrc()) {
+        hash = (37 * hash) + BYTESPERCRC_FIELD_NUMBER;
+        hash = (53 * hash) + getBytesPerCrc();
+      }
+      if (hasCrcPerBlock()) {
+        hash = (37 * hash) + CRCPERBLOCK_FIELD_NUMBER;
+        hash = (53 * hash) + hashLong(getCrcPerBlock());
+      }
+      if (hasMd5()) {
+        hash = (37 * hash) + MD5_FIELD_NUMBER;
+        hash = (53 * hash) + getMd5().hashCode();
+      }
+      hash = (29 * hash) + getUnknownFields().hashCode();
+      return hash;
+    }
+    
+    public static org.apache.hadoop.hdfs.protocol.proto.DataTransferProtos.OpBlockChecksumResponseProto parseFrom(
+        com.google.protobuf.ByteString data)
+        throws com.google.protobuf.InvalidProtocolBufferException {
+      return newBuilder().mergeFrom(data).buildParsed();
+    }
+    public static org.apache.hadoop.hdfs.protocol.proto.DataTransferProtos.OpBlockChecksumResponseProto parseFrom(
+        com.google.protobuf.ByteString data,
+        com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+        throws com.google.protobuf.InvalidProtocolBufferException {
+      return newBuilder().mergeFrom(data, extensionRegistry)
+               .buildParsed();
+    }
+    public static org.apache.hadoop.hdfs.protocol.proto.DataTransferProtos.OpBlockChecksumResponseProto parseFrom(byte[] data)
+        throws com.google.protobuf.InvalidProtocolBufferException {
+      return newBuilder().mergeFrom(data).buildParsed();
+    }
+    public static org.apache.hadoop.hdfs.protocol.proto.DataTransferProtos.OpBlockChecksumResponseProto parseFrom(
+        byte[] data,
+        com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+        throws com.google.protobuf.InvalidProtocolBufferException {
+      return newBuilder().mergeFrom(data, extensionRegistry)
+               .buildParsed();
+    }
+    public static org.apache.hadoop.hdfs.protocol.proto.DataTransferProtos.OpBlockChecksumResponseProto parseFrom(java.io.InputStream input)
+        throws java.io.IOException {
+      return newBuilder().mergeFrom(input).buildParsed();
+    }
+    public static org.apache.hadoop.hdfs.protocol.proto.DataTransferProtos.OpBlockChecksumResponseProto parseFrom(
+        java.io.InputStream input,
+        com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+        throws java.io.IOException {
+      return newBuilder().mergeFrom(input, extensionRegistry)
+               .buildParsed();
+    }
+    public static org.apache.hadoop.hdfs.protocol.proto.DataTransferProtos.OpBlockChecksumResponseProto parseDelimitedFrom(java.io.InputStream input)
+        throws java.io.IOException {
+      Builder builder = newBuilder();
+      if (builder.mergeDelimitedFrom(input)) {
+        return builder.buildParsed();
+      } else {
+        return null;
+      }
+    }
+    public static org.apache.hadoop.hdfs.protocol.proto.DataTransferProtos.OpBlockChecksumResponseProto parseDelimitedFrom(
+        java.io.InputStream input,
+        com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+        throws java.io.IOException {
+      Builder builder = newBuilder();
+      if (builder.mergeDelimitedFrom(input, extensionRegistry)) {
+        return builder.buildParsed();
+      } else {
+        return null;
+      }
+    }
+    public static org.apache.hadoop.hdfs.protocol.proto.DataTransferProtos.OpBlockChecksumResponseProto parseFrom(
+        com.google.protobuf.CodedInputStream input)
+        throws java.io.IOException {
+      return newBuilder().mergeFrom(input).buildParsed();
+    }
+    public static org.apache.hadoop.hdfs.protocol.proto.DataTransferProtos.OpBlockChecksumResponseProto parseFrom(
+        com.google.protobuf.CodedInputStream input,
+        com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+        throws java.io.IOException {
+      return newBuilder().mergeFrom(input, extensionRegistry)
+               .buildParsed();
+    }
+    
+    public static Builder newBuilder() { return Builder.create(); }
+    public Builder newBuilderForType() { return newBuilder(); }
+    public static Builder newBuilder(org.apache.hadoop.hdfs.protocol.proto.DataTransferProtos.OpBlockChecksumResponseProto prototype) {
+      return newBuilder().mergeFrom(prototype);
+    }
+    public Builder toBuilder() { return newBuilder(this); }
+    
+    @java.lang.Override
+    protected Builder newBuilderForType(
+        com.google.protobuf.GeneratedMessage.BuilderParent parent) {
+      Builder builder = new Builder(parent);
+      return builder;
+    }
+    public static final class Builder extends
+        com.google.protobuf.GeneratedMessage.Builder<Builder>
+       implements org.apache.hadoop.hdfs.protocol.proto.DataTransferProtos.OpBlockChecksumResponseProtoOrBuilder {
+      public static final com.google.protobuf.Descriptors.Descriptor
+          getDescriptor() {
+        return org.apache.hadoop.hdfs.protocol.proto.DataTransferProtos.internal_static_OpBlockChecksumResponseProto_descriptor;
+      }
+      
+      protected com.google.protobuf.GeneratedMessage.FieldAccessorTable
+          internalGetFieldAccessorTable() {
+        return org.apache.hadoop.hdfs.protocol.proto.DataTransferProtos.internal_static_OpBlockChecksumResponseProto_fieldAccessorTable;
+      }
+      
+      // Construct using org.apache.hadoop.hdfs.protocol.proto.DataTransferProtos.OpBlockChecksumResponseProto.newBuilder()
+      private Builder() {
+        maybeForceBuilderInitialization();
+      }
+      
+      private Builder(BuilderParent parent) {
+        super(parent);
+        maybeForceBuilderInitialization();
+      }
+      private void maybeForceBuilderInitialization() {
+        if (com.google.protobuf.GeneratedMessage.alwaysUseFieldBuilders) {
+        }
+      }
+      private static Builder create() {
+        return new Builder();
+      }
+      
+      public Builder clear() {
+        super.clear();
+        bytesPerCrc_ = 0;
+        bitField0_ = (bitField0_ & ~0x00000001);
+        crcPerBlock_ = 0L;
+        bitField0_ = (bitField0_ & ~0x00000002);
+        md5_ = com.google.protobuf.ByteString.EMPTY;
+        bitField0_ = (bitField0_ & ~0x00000004);
+        return this;
+      }
+      
+      public Builder clone() {
+        return create().mergeFrom(buildPartial());
+      }
+      
+      public com.google.protobuf.Descriptors.Descriptor
+          getDescriptorForType() {
+        return org.apache.hadoop.hdfs.protocol.proto.DataTransferProtos.OpBlockChecksumResponseProto.getDescriptor();
+      }
+      
+      public org.apache.hadoop.hdfs.protocol.proto.DataTransferProtos.OpBlockChecksumResponseProto getDefaultInstanceForType() {
+        return org.apache.hadoop.hdfs.protocol.proto.DataTransferProtos.OpBlockChecksumResponseProto.getDefaultInstance();
+      }
+      
+      public org.apache.hadoop.hdfs.protocol.proto.DataTransferProtos.OpBlockChecksumResponseProto build() {
+        org.apache.hadoop.hdfs.protocol.proto.DataTransferProtos.OpBlockChecksumResponseProto result = buildPartial();
+        if (!result.isInitialized()) {
+          throw newUninitializedMessageException(result);
+        }
+        return result;
+      }
+      
+      private org.apache.hadoop.hdfs.protocol.proto.DataTransferProtos.OpBlockChecksumResponseProto buildParsed()
+          throws com.google.protobuf.InvalidProtocolBufferException {
+        org.apache.hadoop.hdfs.protocol.proto.DataTransferProtos.OpBlockChecksumResponseProto result = buildPartial();
+        if (!result.isInitialized()) {
+          throw newUninitializedMessageException(
+            result).asInvalidProtocolBufferException();
+        }
+        return result;
+      }
+      
+      public org.apache.hadoop.hdfs.protocol.proto.DataTransferProtos.OpBlockChecksumResponseProto buildPartial() {
+        org.apache.hadoop.hdfs.protocol.proto.DataTransferProtos.OpBlockChecksumResponseProto result = new org.apache.hadoop.hdfs.protocol.proto.DataTransferProtos.OpBlockChecksumResponseProto(this);
+        int from_bitField0_ = bitField0_;
+        int to_bitField0_ = 0;
+        if (((from_bitField0_ & 0x00000001) == 0x00000001)) {
+          to_bitField0_ |= 0x00000001;
+        }
+        result.bytesPerCrc_ = bytesPerCrc_;
+        if (((from_bitField0_ & 0x00000002) == 0x00000002)) {
+          to_bitField0_ |= 0x00000002;
+        }
+        result.crcPerBlock_ = crcPerBlock_;
+        if (((from_bitField0_ & 0x00000004) == 0x00000004)) {
+          to_bitField0_ |= 0x00000004;
+        }
+        result.md5_ = md5_;
+        result.bitField0_ = to_bitField0_;
+        onBuilt();
+        return result;
+      }
+      
+      public Builder mergeFrom(com.google.protobuf.Message other) {
+        if (other instanceof org.apache.hadoop.hdfs.protocol.proto.DataTransferProtos.OpBlockChecksumResponseProto) {
+          return mergeFrom((org.apache.hadoop.hdfs.protocol.proto.DataTransferProtos.OpBlockChecksumResponseProto)other);
+        } else {
+          super.mergeFrom(other);
+          return this;
+        }
+      }
+      
+      public Builder mergeFrom(org.apache.hadoop.hdfs.protocol.proto.DataTransferProtos.OpBlockChecksumResponseProto other) {
+        if (other == org.apache.hadoop.hdfs.protocol.proto.DataTransferProtos.OpBlockChecksumResponseProto.getDefaultInstance()) return this;
+        if (other.hasBytesPerCrc()) {
+          setBytesPerCrc(other.getBytesPerCrc());
+        }
+        if (other.hasCrcPerBlock()) {
+          setCrcPerBlock(other.getCrcPerBlock());
+        }
+        if (other.hasMd5()) {
+          setMd5(other.getMd5());
+        }
+        this.mergeUnknownFields(other.getUnknownFields());
+        return this;
+      }
+      
+      public final boolean isInitialized() {
+        if (!hasBytesPerCrc()) {
+          
+          return false;
+        }
+        if (!hasCrcPerBlock()) {
+          
+          return false;
+        }
+        if (!hasMd5()) {
+          
+          return false;
+        }
+        return true;
+      }
+      
+      public Builder mergeFrom(
+          com.google.protobuf.CodedInputStream input,
+          com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+          throws java.io.IOException {
+        com.google.protobuf.UnknownFieldSet.Builder unknownFields =
+          com.google.protobuf.UnknownFieldSet.newBuilder(
+            this.getUnknownFields());
+        while (true) {
+          int tag = input.readTag();
+          switch (tag) {
+            case 0:
+              this.setUnknownFields(unknownFields.build());
+              onChanged();
+              return this;
+            default: {
+              if (!parseUnknownField(input, unknownFields,
+                                     extensionRegistry, tag)) {
+                this.setUnknownFields(unknownFields.build());
+                onChanged();
+                return this;
+              }
+              break;
+            }
+            case 8: {
+              bitField0_ |= 0x00000001;
+              bytesPerCrc_ = input.readUInt32();
+              break;
+            }
+            case 16: {
+              bitField0_ |= 0x00000002;
+              crcPerBlock_ = input.readUInt64();
+              break;
+            }
+            case 26: {
+              bitField0_ |= 0x00000004;
+              md5_ = input.readBytes();
+              break;
+            }
+          }
+        }
+      }
+      
+      private int bitField0_;
+      
+      // required uint32 bytesPerCrc = 1;
+      private int bytesPerCrc_ ;
+      public boolean hasBytesPerCrc() {
+        return ((bitField0_ & 0x00000001) == 0x00000001);
+      }
+      public int getBytesPerCrc() {
+        return bytesPerCrc_;
+      }
+      public Builder setBytesPerCrc(int value) {
+        bitField0_ |= 0x00000001;
+        bytesPerCrc_ = value;
+        onChanged();
+        return this;
+      }
+      public Builder clearBytesPerCrc() {
+        bitField0_ = (bitField0_ & ~0x00000001);
+        bytesPerCrc_ = 0;
+        onChanged();
+        return this;
+      }
+      
+      // required uint64 crcPerBlock = 2;
+      private long crcPerBlock_ ;
+      public boolean hasCrcPerBlock() {
+        return ((bitField0_ & 0x00000002) == 0x00000002);
+      }
+      public long getCrcPerBlock() {
+        return crcPerBlock_;
+      }
+      public Builder setCrcPerBlock(long value) {
+        bitField0_ |= 0x00000002;
+        crcPerBlock_ = value;
+        onChanged();
+        return this;
+      }
+      public Builder clearCrcPerBlock() {
+        bitField0_ = (bitField0_ & ~0x00000002);
+        crcPerBlock_ = 0L;
+        onChanged();
+        return this;
+      }
+      
+      // required bytes md5 = 3;
+      private com.google.protobuf.ByteString md5_ = com.google.protobuf.ByteString.EMPTY;
+      public boolean hasMd5() {
+        return ((bitField0_ & 0x00000004) == 0x00000004);
+      }
+      public com.google.protobuf.ByteString getMd5() {
+        return md5_;
+      }
+      public Builder setMd5(com.google.protobuf.ByteString value) {
+        if (value == null) {
+    throw new NullPointerException();
+  }
+  bitField0_ |= 0x00000004;
+        md5_ = value;
+        onChanged();
+        return this;
+      }
+      public Builder clearMd5() {
+        bitField0_ = (bitField0_ & ~0x00000004);
+        md5_ = getDefaultInstance().getMd5();
+        onChanged();
+        return this;
+      }
+      
+      // @@protoc_insertion_point(builder_scope:OpBlockChecksumResponseProto)
+    }
+    
+    static {
+      defaultInstance = new OpBlockChecksumResponseProto(true);
+      defaultInstance.initFields();
+    }
+    
+    // @@protoc_insertion_point(class_scope:OpBlockChecksumResponseProto)
+  }
+  
+  private static com.google.protobuf.Descriptors.Descriptor
+    internal_static_BaseHeaderProto_descriptor;
+  private static
+    com.google.protobuf.GeneratedMessage.FieldAccessorTable
+      internal_static_BaseHeaderProto_fieldAccessorTable;
+  private static com.google.protobuf.Descriptors.Descriptor
+    internal_static_ClientOperationHeaderProto_descriptor;
+  private static
+    com.google.protobuf.GeneratedMessage.FieldAccessorTable
+      internal_static_ClientOperationHeaderProto_fieldAccessorTable;
+  private static com.google.protobuf.Descriptors.Descriptor
+    internal_static_OpReadBlockProto_descriptor;
+  private static
+    com.google.protobuf.GeneratedMessage.FieldAccessorTable
+      internal_static_OpReadBlockProto_fieldAccessorTable;
+  private static com.google.protobuf.Descriptors.Descriptor
+    internal_static_OpWriteBlockProto_descriptor;
+  private static
+    com.google.protobuf.GeneratedMessage.FieldAccessorTable
+      internal_static_OpWriteBlockProto_fieldAccessorTable;
+  private static com.google.protobuf.Descriptors.Descriptor
+    internal_static_OpTransferBlockProto_descriptor;
+  private static
+    com.google.protobuf.GeneratedMessage.FieldAccessorTable
+      internal_static_OpTransferBlockProto_fieldAccessorTable;
+  private static com.google.protobuf.Descriptors.Descriptor
+    internal_static_OpReplaceBlockProto_descriptor;
+  private static
+    com.google.protobuf.GeneratedMessage.FieldAccessorTable
+      internal_static_OpReplaceBlockProto_fieldAccessorTable;
+  private static com.google.protobuf.Descriptors.Descriptor
+    internal_static_OpCopyBlockProto_descriptor;
+  private static
+    com.google.protobuf.GeneratedMessage.FieldAccessorTable
+      internal_static_OpCopyBlockProto_fieldAccessorTable;
+  private static com.google.protobuf.Descriptors.Descriptor
+    internal_static_OpBlockChecksumProto_descriptor;
+  private static
+    com.google.protobuf.GeneratedMessage.FieldAccessorTable
+      internal_static_OpBlockChecksumProto_fieldAccessorTable;
+  private static com.google.protobuf.Descriptors.Descriptor
+    internal_static_PacketHeaderProto_descriptor;
+  private static
+    com.google.protobuf.GeneratedMessage.FieldAccessorTable
+      internal_static_PacketHeaderProto_fieldAccessorTable;
+  private static com.google.protobuf.Descriptors.Descriptor
+    internal_static_PipelineAckProto_descriptor;
+  private static
+    com.google.protobuf.GeneratedMessage.FieldAccessorTable
+      internal_static_PipelineAckProto_fieldAccessorTable;
+  private static com.google.protobuf.Descriptors.Descriptor
+    internal_static_BlockOpResponseProto_descriptor;
+  private static
+    com.google.protobuf.GeneratedMessage.FieldAccessorTable
+      internal_static_BlockOpResponseProto_fieldAccessorTable;
+  private static com.google.protobuf.Descriptors.Descriptor
+    internal_static_ClientReadStatusProto_descriptor;
+  private static
+    com.google.protobuf.GeneratedMessage.FieldAccessorTable
+      internal_static_ClientReadStatusProto_fieldAccessorTable;
+  private static com.google.protobuf.Descriptors.Descriptor
+    internal_static_DNTransferAckProto_descriptor;
+  private static
+    com.google.protobuf.GeneratedMessage.FieldAccessorTable
+      internal_static_DNTransferAckProto_fieldAccessorTable;
+  private static com.google.protobuf.Descriptors.Descriptor
+    internal_static_OpBlockChecksumResponseProto_descriptor;
+  private static
+    com.google.protobuf.GeneratedMessage.FieldAccessorTable
+      internal_static_OpBlockChecksumResponseProto_fieldAccessorTable;
+  
+  public static com.google.protobuf.Descriptors.FileDescriptor
+      getDescriptor() {
+    return descriptor;
+  }
+  private static com.google.protobuf.Descriptors.FileDescriptor
+      descriptor;
+  static {
+    java.lang.String[] descriptorData = {
+      "\n\022datatransfer.proto\032\nhdfs.proto\"`\n\017Base" +
+      "HeaderProto\022\"\n\005block\030\001 \002(\0132\023.ExtendedBlo" +
+      "ckProto\022)\n\005token\030\002 \001(\0132\032.BlockTokenIdent" +
+      "ifierProto\"V\n\032ClientOperationHeaderProto" +
+      "\022$\n\nbaseHeader\030\001 \002(\0132\020.BaseHeaderProto\022\022" +
+      "\n\nclientName\030\002 \002(\t\"\\\n\020OpReadBlockProto\022+" +
+      "\n\006header\030\001 \002(\0132\033.ClientOperationHeaderPr" +
+      "oto\022\016\n\006offset\030\002 \002(\004\022\013\n\003len\030\003 \002(\004\"\257\004\n\021OpW" +
+      "riteBlockProto\022+\n\006header\030\001 \002(\0132\033.ClientO" +
+      "perationHeaderProto\022#\n\007targets\030\002 \003(\0132\022.D",
+      "atanodeInfoProto\022\"\n\006source\030\003 \001(\0132\022.Datan" +
+      "odeInfoProto\0228\n\005stage\030\004 \002(\0162).OpWriteBlo" +
+      "ckProto.BlockConstructionStage\022\024\n\014pipeli" +
+      "neSize\030\005 \002(\r\022\024\n\014minBytesRcvd\030\006 \002(\004\022\024\n\014ma" +
+      "xBytesRcvd\030\007 \002(\004\022\035\n\025latestGenerationStam" +
+      "p\030\010 \002(\004\"\210\002\n\026BlockConstructionStage\022\031\n\025PI" +
+      "PELINE_SETUP_APPEND\020\000\022\"\n\036PIPELINE_SETUP_" +
+      "APPEND_RECOVERY\020\001\022\022\n\016DATA_STREAMING\020\002\022%\n" +
+      "!PIPELINE_SETUP_STREAMING_RECOVERY\020\003\022\022\n\016" +
+      "PIPELINE_CLOSE\020\004\022\033\n\027PIPELINE_CLOSE_RECOV",
+      "ERY\020\005\022\031\n\025PIPELINE_SETUP_CREATE\020\006\022\020\n\014TRAN" +
+      "SFER_RBW\020\007\022\026\n\022TRANSFER_FINALIZED\020\010\"h\n\024Op" +
+      "TransferBlockProto\022+\n\006header\030\001 \002(\0132\033.Cli" +
+      "entOperationHeaderProto\022#\n\007targets\030\002 \003(\013" +
+      "2\022.DatanodeInfoProto\"l\n\023OpReplaceBlockPr" +
+      "oto\022 \n\006header\030\001 \002(\0132\020.BaseHeaderProto\022\017\n" +
+      "\007delHint\030\002 \002(\t\022\"\n\006source\030\003 \002(\0132\022.Datanod" +
+      "eInfoProto\"4\n\020OpCopyBlockProto\022 \n\006header" +
+      "\030\001 \002(\0132\020.BaseHeaderProto\"8\n\024OpBlockCheck" +
+      "sumProto\022 \n\006header\030\001 \002(\0132\020.BaseHeaderPro",
+      "to\"e\n\021PacketHeaderProto\022\025\n\roffsetInBlock" +
+      "\030\001 \002(\020\022\r\n\005seqno\030\002 \002(\020\022\031\n\021lastPacketInBlo" +
+      "ck\030\003 \002(\010\022\017\n\007dataLen\030\004 \002(\017\":\n\020PipelineAck" +
+      "Proto\022\r\n\005seqno\030\001 \002(\022\022\027\n\006status\030\002 \003(\0162\007.S" +
+      "tatus\"~\n\024BlockOpResponseProto\022\027\n\006status\030" +
+      "\001 \002(\0162\007.Status\022\024\n\014firstBadLink\030\002 \001(\t\0227\n\020" +
+      "checksumResponse\030\003 \001(\0132\035.OpBlockChecksum" +
+      "ResponseProto\"0\n\025ClientReadStatusProto\022\027" +
+      "\n\006status\030\001 \002(\0162\007.Status\"-\n\022DNTransferAck" +
+      "Proto\022\027\n\006status\030\001 \002(\0162\007.Status\"U\n\034OpBloc",
+      "kChecksumResponseProto\022\023\n\013bytesPerCrc\030\001 " +
+      "\002(\r\022\023\n\013crcPerBlock\030\002 \002(\004\022\013\n\003md5\030\003 \002(\014*\202\001" +
+      "\n\006Status\022\013\n\007SUCCESS\020\000\022\t\n\005ERROR\020\001\022\022\n\016ERRO" +
+      "R_CHECKSUM\020\002\022\021\n\rERROR_INVALID\020\003\022\020\n\014ERROR" +
+      "_EXISTS\020\004\022\026\n\022ERROR_ACCESS_TOKEN\020\005\022\017\n\013CHE" +
+      "CKSUM_OK\020\006B>\n%org.apache.hadoop.hdfs.pro" +
+      "tocol.protoB\022DataTransferProtos\240\001\001"
+    };
+    com.google.protobuf.Descriptors.FileDescriptor.InternalDescriptorAssigner assigner =
+      new com.google.protobuf.Descriptors.FileDescriptor.InternalDescriptorAssigner() {
+        public com.google.protobuf.ExtensionRegistry assignDescriptors(
+            com.google.protobuf.Descriptors.FileDescriptor root) {
+          descriptor = root;
+          internal_static_BaseHeaderProto_descriptor =
+            getDescriptor().getMessageTypes().get(0);
+          internal_static_BaseHeaderProto_fieldAccessorTable = new
+            com.google.protobuf.GeneratedMessage.FieldAccessorTable(
+              internal_static_BaseHeaderProto_descriptor,
+              new java.lang.String[] { "Block", "Token", },
+              org.apache.hadoop.hdfs.protocol.proto.DataTransferProtos.BaseHeaderProto.class,
+              org.apache.hadoop.hdfs.protocol.proto.DataTransferProtos.BaseHeaderProto.Builder.class);
+          internal_static_ClientOperationHeaderProto_descriptor =
+            getDescriptor().getMessageTypes().get(1);
+          internal_static_ClientOperationHeaderProto_fieldAccessorTable = new
+            com.google.protobuf.GeneratedMessage.FieldAccessorTable(
+              internal_static_ClientOperationHeaderProto_descriptor,
+              new java.lang.String[] { "BaseHeader", "ClientName", },
+              org.apache.hadoop.hdfs.protocol.proto.DataTransferProtos.ClientOperationHeaderProto.class,
+              org.apache.hadoop.hdfs.protocol.proto.DataTransferProtos.ClientOperationHeaderProto.Builder.class);
+          internal_static_OpReadBlockProto_descriptor =
+            getDescriptor().getMessageTypes().get(2);
+          internal_static_OpReadBlockProto_fieldAccessorTable = new
+            com.google.protobuf.GeneratedMessage.FieldAccessorTable(
+              internal_static_OpReadBlockProto_descriptor,
+              new java.lang.String[] { "Header", "Offset", "Len", },
+              org.apache.hadoop.hdfs.protocol.proto.DataTransferProtos.OpReadBlockProto.class,
+              org.apache.hadoop.hdfs.protocol.proto.DataTransferProtos.OpReadBlockProto.Builder.class);
+          internal_static_OpWriteBlockProto_descriptor =
+            getDescriptor().getMessageTypes().get(3);
+          internal_static_OpWriteBlockProto_fieldAccessorTable = new
+            com.google.protobuf.GeneratedMessage.FieldAccessorTable(
+              internal_static_OpWriteBlockProto_descriptor,
+              new java.lang.String[] { "Header", "Targets", "Source", "Stage", "PipelineSize", "MinBytesRcvd", "MaxBytesRcvd", "LatestGenerationStamp", },
+              org.apache.hadoop.hdfs.protocol.proto.DataTransferProtos.OpWriteBlockProto.class,
+              org.apache.hadoop.hdfs.protocol.proto.DataTransferProtos.OpWriteBlockProto.Builder.class);
+          internal_static_OpTransferBlockProto_descriptor =
+            getDescriptor().getMessageTypes().get(4);
+          internal_static_OpTransferBlockProto_fieldAccessorTable = new
+            com.google.protobuf.GeneratedMessage.FieldAccessorTable(
+              internal_static_OpTransferBlockProto_descriptor,
+              new java.lang.String[] { "Header", "Targets", },
+              org.apache.hadoop.hdfs.protocol.proto.DataTransferProtos.OpTransferBlockProto.class,
+              org.apache.hadoop.hdfs.protocol.proto.DataTransferProtos.OpTransferBlockProto.Builder.class);
+          internal_static_OpReplaceBlockProto_descriptor =
+            getDescriptor().getMessageTypes().get(5);
+          internal_static_OpReplaceBlockProto_fieldAccessorTable = new
+            com.google.protobuf.GeneratedMessage.FieldAccessorTable(
+              internal_static_OpReplaceBlockProto_descriptor,
+              new java.lang.String[] { "Header", "DelHint", "Source", },
+              org.apache.hadoop.hdfs.protocol.proto.DataTransferProtos.OpReplaceBlockProto.class,
+              org.apache.hadoop.hdfs.protocol.proto.DataTransferProtos.OpReplaceBlockProto.Builder.class);
+          internal_static_OpCopyBlockProto_descriptor =
+            getDescriptor().getMessageTypes().get(6);
+          internal_static_OpCopyBlockProto_fieldAccessorTable = new
+            com.google.protobuf.GeneratedMessage.FieldAccessorTable(
+              internal_static_OpCopyBlockProto_descriptor,
+              new java.lang.String[] { "Header", },
+              org.apache.hadoop.hdfs.protocol.proto.DataTransferProtos.OpCopyBlockProto.class,
+              org.apache.hadoop.hdfs.protocol.proto.DataTransferProtos.OpCopyBlockProto.Builder.class);
+          internal_static_OpBlockChecksumProto_descriptor =
+            getDescriptor().getMessageTypes().get(7);
+          internal_static_OpBlockChecksumProto_fieldAccessorTable = new
+            com.google.protobuf.GeneratedMessage.FieldAccessorTable(
+              internal_static_OpBlockChecksumProto_descriptor,
+              new java.lang.String[] { "Header", },
+              org.apache.hadoop.hdfs.protocol.proto.DataTransferProtos.OpBlockChecksumProto.class,
+              org.apache.hadoop.hdfs.protocol.proto.DataTransferProtos.OpBlockChecksumProto.Builder.class);
+          internal_static_PacketHeaderProto_descriptor =
+            getDescriptor().getMessageTypes().get(8);
+          internal_static_PacketHeaderProto_fieldAccessorTable = new
+            com.google.protobuf.GeneratedMessage.FieldAccessorTable(
+              internal_static_PacketHeaderProto_descriptor,
+              new java.lang.String[] { "OffsetInBlock", "Seqno", "LastPacketInBlock", "DataLen", },
+              org.apache.hadoop.hdfs.protocol.proto.DataTransferProtos.PacketHeaderProto.class,
+              org.apache.hadoop.hdfs.protocol.proto.DataTransferProtos.PacketHeaderProto.Builder.class);
+          internal_static_PipelineAckProto_descriptor =
+            getDescriptor().getMessageTypes().get(9);
+          internal_static_PipelineAckProto_fieldAccessorTable = new
+            com.google.protobuf.GeneratedMessage.FieldAccessorTable(
+              internal_static_PipelineAckProto_descriptor,
+              new java.lang.String[] { "Seqno", "Status", },
+              org.apache.hadoop.hdfs.protocol.proto.DataTransferProtos.PipelineAckProto.class,
+              org.apache.hadoop.hdfs.protocol.proto.DataTransferProtos.PipelineAckProto.Builder.class);
+          internal_static_BlockOpResponseProto_descriptor =
+            getDescriptor().getMessageTypes().get(10);
+          internal_static_BlockOpResponseProto_fieldAccessorTable = new
+            com.google.protobuf.GeneratedMessage.FieldAccessorTable(
+              internal_static_BlockOpResponseProto_descriptor,
+              new java.lang.String[] { "Status", "FirstBadLink", "ChecksumResponse", },
+              org.apache.hadoop.hdfs.protocol.proto.DataTransferProtos.BlockOpResponseProto.class,
+              org.apache.hadoop.hdfs.protocol.proto.DataTransferProtos.BlockOpResponseProto.Builder.class);
+          internal_static_ClientReadStatusProto_descriptor =
+            getDescriptor().getMessageTypes().get(11);
+          internal_static_ClientReadStatusProto_fieldAccessorTable = new
+            com.google.protobuf.GeneratedMessage.FieldAccessorTable(
+              internal_static_ClientReadStatusProto_descriptor,
+              new java.lang.String[] { "Status", },
+              org.apache.hadoop.hdfs.protocol.proto.DataTransferProtos.ClientReadStatusProto.class,
+              org.apache.hadoop.hdfs.protocol.proto.DataTransferProtos.ClientReadStatusProto.Builder.class);
+          internal_static_DNTransferAckProto_descriptor =
+            getDescriptor().getMessageTypes().get(12);
+          internal_static_DNTransferAckProto_fieldAccessorTable = new
+            com.google.protobuf.GeneratedMessage.FieldAccessorTable(
+              internal_static_DNTransferAckProto_descriptor,
+              new java.lang.String[] { "Status", },
+              org.apache.hadoop.hdfs.protocol.proto.DataTransferProtos.DNTransferAckProto.class,
+              org.apache.hadoop.hdfs.protocol.proto.DataTransferProtos.DNTransferAckProto.Builder.class);
+          internal_static_OpBlockChecksumResponseProto_descriptor =
+            getDescriptor().getMessageTypes().get(13);
+          internal_static_OpBlockChecksumResponseProto_fieldAccessorTable = new
+            com.google.protobuf.GeneratedMessage.FieldAccessorTable(
+              internal_static_OpBlockChecksumResponseProto_descriptor,
+              new java.lang.String[] { "BytesPerCrc", "CrcPerBlock", "Md5", },
+              org.apache.hadoop.hdfs.protocol.proto.DataTransferProtos.OpBlockChecksumResponseProto.class,
+              org.apache.hadoop.hdfs.protocol.proto.DataTransferProtos.OpBlockChecksumResponseProto.Builder.class);
+          return null;
+        }
+      };
+    com.google.protobuf.Descriptors.FileDescriptor
+      .internalBuildGeneratedFileFrom(descriptorData,
+        new com.google.protobuf.Descriptors.FileDescriptor[] {
+          org.apache.hadoop.hdfs.protocol.proto.HdfsProtos.getDescriptor(),
+        }, assigner);
+  }
+  
+  // @@protoc_insertion_point(outer_class_scope)
+}
diff --git a/src/java/org/apache/hadoop/hdfs/protocol/proto/HdfsProtos.java b/src/java/org/apache/hadoop/hdfs/protocol/proto/HdfsProtos.java
new file mode 100644
index 0000000..a70fcc5
--- /dev/null
+++ b/src/java/org/apache/hadoop/hdfs/protocol/proto/HdfsProtos.java
@@ -0,0 +1,3247 @@
+// Generated by the protocol buffer compiler.  DO NOT EDIT!
+// source: hdfs.proto
+
+package org.apache.hadoop.hdfs.protocol.proto;
+
+public final class HdfsProtos {
+  private HdfsProtos() {}
+  public static void registerAllExtensions(
+      com.google.protobuf.ExtensionRegistry registry) {
+  }
+  public interface ExtendedBlockProtoOrBuilder
+      extends com.google.protobuf.MessageOrBuilder {
+    
+    // required string poolId = 1;
+    boolean hasPoolId();
+    String getPoolId();
+    
+    // required uint64 blockId = 2;
+    boolean hasBlockId();
+    long getBlockId();
+    
+    // required uint64 numBytes = 3;
+    boolean hasNumBytes();
+    long getNumBytes();
+    
+    // required uint64 generationStamp = 4;
+    boolean hasGenerationStamp();
+    long getGenerationStamp();
+  }
+  public static final class ExtendedBlockProto extends
+      com.google.protobuf.GeneratedMessage
+      implements ExtendedBlockProtoOrBuilder {
+    // Use ExtendedBlockProto.newBuilder() to construct.
+    private ExtendedBlockProto(Builder builder) {
+      super(builder);
+    }
+    private ExtendedBlockProto(boolean noInit) {}
+    
+    private static final ExtendedBlockProto defaultInstance;
+    public static ExtendedBlockProto getDefaultInstance() {
+      return defaultInstance;
+    }
+    
+    public ExtendedBlockProto getDefaultInstanceForType() {
+      return defaultInstance;
+    }
+    
+    public static final com.google.protobuf.Descriptors.Descriptor
+        getDescriptor() {
+      return org.apache.hadoop.hdfs.protocol.proto.HdfsProtos.internal_static_ExtendedBlockProto_descriptor;
+    }
+    
+    protected com.google.protobuf.GeneratedMessage.FieldAccessorTable
+        internalGetFieldAccessorTable() {
+      return org.apache.hadoop.hdfs.protocol.proto.HdfsProtos.internal_static_ExtendedBlockProto_fieldAccessorTable;
+    }
+    
+    private int bitField0_;
+    // required string poolId = 1;
+    public static final int POOLID_FIELD_NUMBER = 1;
+    private java.lang.Object poolId_;
+    public boolean hasPoolId() {
+      return ((bitField0_ & 0x00000001) == 0x00000001);
+    }
+    public String getPoolId() {
+      java.lang.Object ref = poolId_;
+      if (ref instanceof String) {
+        return (String) ref;
+      } else {
+        com.google.protobuf.ByteString bs = 
+            (com.google.protobuf.ByteString) ref;
+        String s = bs.toStringUtf8();
+        if (com.google.protobuf.Internal.isValidUtf8(bs)) {
+          poolId_ = s;
+        }
+        return s;
+      }
+    }
+    private com.google.protobuf.ByteString getPoolIdBytes() {
+      java.lang.Object ref = poolId_;
+      if (ref instanceof String) {
+        com.google.protobuf.ByteString b = 
+            com.google.protobuf.ByteString.copyFromUtf8((String) ref);
+        poolId_ = b;
+        return b;
+      } else {
+        return (com.google.protobuf.ByteString) ref;
+      }
+    }
+    
+    // required uint64 blockId = 2;
+    public static final int BLOCKID_FIELD_NUMBER = 2;
+    private long blockId_;
+    public boolean hasBlockId() {
+      return ((bitField0_ & 0x00000002) == 0x00000002);
+    }
+    public long getBlockId() {
+      return blockId_;
+    }
+    
+    // required uint64 numBytes = 3;
+    public static final int NUMBYTES_FIELD_NUMBER = 3;
+    private long numBytes_;
+    public boolean hasNumBytes() {
+      return ((bitField0_ & 0x00000004) == 0x00000004);
+    }
+    public long getNumBytes() {
+      return numBytes_;
+    }
+    
+    // required uint64 generationStamp = 4;
+    public static final int GENERATIONSTAMP_FIELD_NUMBER = 4;
+    private long generationStamp_;
+    public boolean hasGenerationStamp() {
+      return ((bitField0_ & 0x00000008) == 0x00000008);
+    }
+    public long getGenerationStamp() {
+      return generationStamp_;
+    }
+    
+    private void initFields() {
+      poolId_ = "";
+      blockId_ = 0L;
+      numBytes_ = 0L;
+      generationStamp_ = 0L;
+    }
+    private byte memoizedIsInitialized = -1;
+    public final boolean isInitialized() {
+      byte isInitialized = memoizedIsInitialized;
+      if (isInitialized != -1) return isInitialized == 1;
+      
+      if (!hasPoolId()) {
+        memoizedIsInitialized = 0;
+        return false;
+      }
+      if (!hasBlockId()) {
+        memoizedIsInitialized = 0;
+        return false;
+      }
+      if (!hasNumBytes()) {
+        memoizedIsInitialized = 0;
+        return false;
+      }
+      if (!hasGenerationStamp()) {
+        memoizedIsInitialized = 0;
+        return false;
+      }
+      memoizedIsInitialized = 1;
+      return true;
+    }
+    
+    public void writeTo(com.google.protobuf.CodedOutputStream output)
+                        throws java.io.IOException {
+      getSerializedSize();
+      if (((bitField0_ & 0x00000001) == 0x00000001)) {
+        output.writeBytes(1, getPoolIdBytes());
+      }
+      if (((bitField0_ & 0x00000002) == 0x00000002)) {
+        output.writeUInt64(2, blockId_);
+      }
+      if (((bitField0_ & 0x00000004) == 0x00000004)) {
+        output.writeUInt64(3, numBytes_);
+      }
+      if (((bitField0_ & 0x00000008) == 0x00000008)) {
+        output.writeUInt64(4, generationStamp_);
+      }
+      getUnknownFields().writeTo(output);
+    }
+    
+    private int memoizedSerializedSize = -1;
+    public int getSerializedSize() {
+      int size = memoizedSerializedSize;
+      if (size != -1) return size;
+    
+      size = 0;
+      if (((bitField0_ & 0x00000001) == 0x00000001)) {
+        size += com.google.protobuf.CodedOutputStream
+          .computeBytesSize(1, getPoolIdBytes());
+      }
+      if (((bitField0_ & 0x00000002) == 0x00000002)) {
+        size += com.google.protobuf.CodedOutputStream
+          .computeUInt64Size(2, blockId_);
+      }
+      if (((bitField0_ & 0x00000004) == 0x00000004)) {
+        size += com.google.protobuf.CodedOutputStream
+          .computeUInt64Size(3, numBytes_);
+      }
+      if (((bitField0_ & 0x00000008) == 0x00000008)) {
+        size += com.google.protobuf.CodedOutputStream
+          .computeUInt64Size(4, generationStamp_);
+      }
+      size += getUnknownFields().getSerializedSize();
+      memoizedSerializedSize = size;
+      return size;
+    }
+    
+    private static final long serialVersionUID = 0L;
+    @java.lang.Override
+    protected java.lang.Object writeReplace()
+        throws java.io.ObjectStreamException {
+      return super.writeReplace();
+    }
+    
+    @java.lang.Override
+    public boolean equals(final java.lang.Object obj) {
+      if (obj == this) {
+       return true;
+      }
+      if (!(obj instanceof org.apache.hadoop.hdfs.protocol.proto.HdfsProtos.ExtendedBlockProto)) {
+        return super.equals(obj);
+      }
+      org.apache.hadoop.hdfs.protocol.proto.HdfsProtos.ExtendedBlockProto other = (org.apache.hadoop.hdfs.protocol.proto.HdfsProtos.ExtendedBlockProto) obj;
+      
+      boolean result = true;
+      result = result && (hasPoolId() == other.hasPoolId());
+      if (hasPoolId()) {
+        result = result && getPoolId()
+            .equals(other.getPoolId());
+      }
+      result = result && (hasBlockId() == other.hasBlockId());
+      if (hasBlockId()) {
+        result = result && (getBlockId()
+            == other.getBlockId());
+      }
+      result = result && (hasNumBytes() == other.hasNumBytes());
+      if (hasNumBytes()) {
+        result = result && (getNumBytes()
+            == other.getNumBytes());
+      }
+      result = result && (hasGenerationStamp() == other.hasGenerationStamp());
+      if (hasGenerationStamp()) {
+        result = result && (getGenerationStamp()
+            == other.getGenerationStamp());
+      }
+      result = result &&
+          getUnknownFields().equals(other.getUnknownFields());
+      return result;
+    }
+    
+    @java.lang.Override
+    public int hashCode() {
+      int hash = 41;
+      hash = (19 * hash) + getDescriptorForType().hashCode();
+      if (hasPoolId()) {
+        hash = (37 * hash) + POOLID_FIELD_NUMBER;
+        hash = (53 * hash) + getPoolId().hashCode();
+      }
+      if (hasBlockId()) {
+        hash = (37 * hash) + BLOCKID_FIELD_NUMBER;
+        hash = (53 * hash) + hashLong(getBlockId());
+      }
+      if (hasNumBytes()) {
+        hash = (37 * hash) + NUMBYTES_FIELD_NUMBER;
+        hash = (53 * hash) + hashLong(getNumBytes());
+      }
+      if (hasGenerationStamp()) {
+        hash = (37 * hash) + GENERATIONSTAMP_FIELD_NUMBER;
+        hash = (53 * hash) + hashLong(getGenerationStamp());
+      }
+      hash = (29 * hash) + getUnknownFields().hashCode();
+      return hash;
+    }
+    
+    public static org.apache.hadoop.hdfs.protocol.proto.HdfsProtos.ExtendedBlockProto parseFrom(
+        com.google.protobuf.ByteString data)
+        throws com.google.protobuf.InvalidProtocolBufferException {
+      return newBuilder().mergeFrom(data).buildParsed();
+    }
+    public static org.apache.hadoop.hdfs.protocol.proto.HdfsProtos.ExtendedBlockProto parseFrom(
+        com.google.protobuf.ByteString data,
+        com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+        throws com.google.protobuf.InvalidProtocolBufferException {
+      return newBuilder().mergeFrom(data, extensionRegistry)
+               .buildParsed();
+    }
+    public static org.apache.hadoop.hdfs.protocol.proto.HdfsProtos.ExtendedBlockProto parseFrom(byte[] data)
+        throws com.google.protobuf.InvalidProtocolBufferException {
+      return newBuilder().mergeFrom(data).buildParsed();
+    }
+    public static org.apache.hadoop.hdfs.protocol.proto.HdfsProtos.ExtendedBlockProto parseFrom(
+        byte[] data,
+        com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+        throws com.google.protobuf.InvalidProtocolBufferException {
+      return newBuilder().mergeFrom(data, extensionRegistry)
+               .buildParsed();
+    }
+    public static org.apache.hadoop.hdfs.protocol.proto.HdfsProtos.ExtendedBlockProto parseFrom(java.io.InputStream input)
+        throws java.io.IOException {
+      return newBuilder().mergeFrom(input).buildParsed();
+    }
+    public static org.apache.hadoop.hdfs.protocol.proto.HdfsProtos.ExtendedBlockProto parseFrom(
+        java.io.InputStream input,
+        com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+        throws java.io.IOException {
+      return newBuilder().mergeFrom(input, extensionRegistry)
+               .buildParsed();
+    }
+    public static org.apache.hadoop.hdfs.protocol.proto.HdfsProtos.ExtendedBlockProto parseDelimitedFrom(java.io.InputStream input)
+        throws java.io.IOException {
+      Builder builder = newBuilder();
+      if (builder.mergeDelimitedFrom(input)) {
+        return builder.buildParsed();
+      } else {
+        return null;
+      }
+    }
+    public static org.apache.hadoop.hdfs.protocol.proto.HdfsProtos.ExtendedBlockProto parseDelimitedFrom(
+        java.io.InputStream input,
+        com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+        throws java.io.IOException {
+      Builder builder = newBuilder();
+      if (builder.mergeDelimitedFrom(input, extensionRegistry)) {
+        return builder.buildParsed();
+      } else {
+        return null;
+      }
+    }
+    public static org.apache.hadoop.hdfs.protocol.proto.HdfsProtos.ExtendedBlockProto parseFrom(
+        com.google.protobuf.CodedInputStream input)
+        throws java.io.IOException {
+      return newBuilder().mergeFrom(input).buildParsed();
+    }
+    public static org.apache.hadoop.hdfs.protocol.proto.HdfsProtos.ExtendedBlockProto parseFrom(
+        com.google.protobuf.CodedInputStream input,
+        com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+        throws java.io.IOException {
+      return newBuilder().mergeFrom(input, extensionRegistry)
+               .buildParsed();
+    }
+    
+    public static Builder newBuilder() { return Builder.create(); }
+    public Builder newBuilderForType() { return newBuilder(); }
+    public static Builder newBuilder(org.apache.hadoop.hdfs.protocol.proto.HdfsProtos.ExtendedBlockProto prototype) {
+      return newBuilder().mergeFrom(prototype);
+    }
+    public Builder toBuilder() { return newBuilder(this); }
+    
+    @java.lang.Override
+    protected Builder newBuilderForType(
+        com.google.protobuf.GeneratedMessage.BuilderParent parent) {
+      Builder builder = new Builder(parent);
+      return builder;
+    }
+    public static final class Builder extends
+        com.google.protobuf.GeneratedMessage.Builder<Builder>
+       implements org.apache.hadoop.hdfs.protocol.proto.HdfsProtos.ExtendedBlockProtoOrBuilder {
+      public static final com.google.protobuf.Descriptors.Descriptor
+          getDescriptor() {
+        return org.apache.hadoop.hdfs.protocol.proto.HdfsProtos.internal_static_ExtendedBlockProto_descriptor;
+      }
+      
+      protected com.google.protobuf.GeneratedMessage.FieldAccessorTable
+          internalGetFieldAccessorTable() {
+        return org.apache.hadoop.hdfs.protocol.proto.HdfsProtos.internal_static_ExtendedBlockProto_fieldAccessorTable;
+      }
+      
+      // Construct using org.apache.hadoop.hdfs.protocol.proto.HdfsProtos.ExtendedBlockProto.newBuilder()
+      private Builder() {
+        maybeForceBuilderInitialization();
+      }
+      
+      private Builder(BuilderParent parent) {
+        super(parent);
+        maybeForceBuilderInitialization();
+      }
+      private void maybeForceBuilderInitialization() {
+        if (com.google.protobuf.GeneratedMessage.alwaysUseFieldBuilders) {
+        }
+      }
+      private static Builder create() {
+        return new Builder();
+      }
+      
+      public Builder clear() {
+        super.clear();
+        poolId_ = "";
+        bitField0_ = (bitField0_ & ~0x00000001);
+        blockId_ = 0L;
+        bitField0_ = (bitField0_ & ~0x00000002);
+        numBytes_ = 0L;
+        bitField0_ = (bitField0_ & ~0x00000004);
+        generationStamp_ = 0L;
+        bitField0_ = (bitField0_ & ~0x00000008);
+        return this;
+      }
+      
+      public Builder clone() {
+        return create().mergeFrom(buildPartial());
+      }
+      
+      public com.google.protobuf.Descriptors.Descriptor
+          getDescriptorForType() {
+        return org.apache.hadoop.hdfs.protocol.proto.HdfsProtos.ExtendedBlockProto.getDescriptor();
+      }
+      
+      public org.apache.hadoop.hdfs.protocol.proto.HdfsProtos.ExtendedBlockProto getDefaultInstanceForType() {
+        return org.apache.hadoop.hdfs.protocol.proto.HdfsProtos.ExtendedBlockProto.getDefaultInstance();
+      }
+      
+      public org.apache.hadoop.hdfs.protocol.proto.HdfsProtos.ExtendedBlockProto build() {
+        org.apache.hadoop.hdfs.protocol.proto.HdfsProtos.ExtendedBlockProto result = buildPartial();
+        if (!result.isInitialized()) {
+          throw newUninitializedMessageException(result);
+        }
+        return result;
+      }
+      
+      private org.apache.hadoop.hdfs.protocol.proto.HdfsProtos.ExtendedBlockProto buildParsed()
+          throws com.google.protobuf.InvalidProtocolBufferException {
+        org.apache.hadoop.hdfs.protocol.proto.HdfsProtos.ExtendedBlockProto result = buildPartial();
+        if (!result.isInitialized()) {
+          throw newUninitializedMessageException(
+            result).asInvalidProtocolBufferException();
+        }
+        return result;
+      }
+      
+      public org.apache.hadoop.hdfs.protocol.proto.HdfsProtos.ExtendedBlockProto buildPartial() {
+        org.apache.hadoop.hdfs.protocol.proto.HdfsProtos.ExtendedBlockProto result = new org.apache.hadoop.hdfs.protocol.proto.HdfsProtos.ExtendedBlockProto(this);
+        int from_bitField0_ = bitField0_;
+        int to_bitField0_ = 0;
+        if (((from_bitField0_ & 0x00000001) == 0x00000001)) {
+          to_bitField0_ |= 0x00000001;
+        }
+        result.poolId_ = poolId_;
+        if (((from_bitField0_ & 0x00000002) == 0x00000002)) {
+          to_bitField0_ |= 0x00000002;
+        }
+        result.blockId_ = blockId_;
+        if (((from_bitField0_ & 0x00000004) == 0x00000004)) {
+          to_bitField0_ |= 0x00000004;
+        }
+        result.numBytes_ = numBytes_;
+        if (((from_bitField0_ & 0x00000008) == 0x00000008)) {
+          to_bitField0_ |= 0x00000008;
+        }
+        result.generationStamp_ = generationStamp_;
+        result.bitField0_ = to_bitField0_;
+        onBuilt();
+        return result;
+      }
+      
+      public Builder mergeFrom(com.google.protobuf.Message other) {
+        if (other instanceof org.apache.hadoop.hdfs.protocol.proto.HdfsProtos.ExtendedBlockProto) {
+          return mergeFrom((org.apache.hadoop.hdfs.protocol.proto.HdfsProtos.ExtendedBlockProto)other);
+        } else {
+          super.mergeFrom(other);
+          return this;
+        }
+      }
+      
+      public Builder mergeFrom(org.apache.hadoop.hdfs.protocol.proto.HdfsProtos.ExtendedBlockProto other) {
+        if (other == org.apache.hadoop.hdfs.protocol.proto.HdfsProtos.ExtendedBlockProto.getDefaultInstance()) return this;
+        if (other.hasPoolId()) {
+          setPoolId(other.getPoolId());
+        }
+        if (other.hasBlockId()) {
+          setBlockId(other.getBlockId());
+        }
+        if (other.hasNumBytes()) {
+          setNumBytes(other.getNumBytes());
+        }
+        if (other.hasGenerationStamp()) {
+          setGenerationStamp(other.getGenerationStamp());
+        }
+        this.mergeUnknownFields(other.getUnknownFields());
+        return this;
+      }
+      
+      public final boolean isInitialized() {
+        if (!hasPoolId()) {
+          
+          return false;
+        }
+        if (!hasBlockId()) {
+          
+          return false;
+        }
+        if (!hasNumBytes()) {
+          
+          return false;
+        }
+        if (!hasGenerationStamp()) {
+          
+          return false;
+        }
+        return true;
+      }
+      
+      public Builder mergeFrom(
+          com.google.protobuf.CodedInputStream input,
+          com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+          throws java.io.IOException {
+        com.google.protobuf.UnknownFieldSet.Builder unknownFields =
+          com.google.protobuf.UnknownFieldSet.newBuilder(
+            this.getUnknownFields());
+        while (true) {
+          int tag = input.readTag();
+          switch (tag) {
+            case 0:
+              this.setUnknownFields(unknownFields.build());
+              onChanged();
+              return this;
+            default: {
+              if (!parseUnknownField(input, unknownFields,
+                                     extensionRegistry, tag)) {
+                this.setUnknownFields(unknownFields.build());
+                onChanged();
+                return this;
+              }
+              break;
+            }
+            case 10: {
+              bitField0_ |= 0x00000001;
+              poolId_ = input.readBytes();
+              break;
+            }
+            case 16: {
+              bitField0_ |= 0x00000002;
+              blockId_ = input.readUInt64();
+              break;
+            }
+            case 24: {
+              bitField0_ |= 0x00000004;
+              numBytes_ = input.readUInt64();
+              break;
+            }
+            case 32: {
+              bitField0_ |= 0x00000008;
+              generationStamp_ = input.readUInt64();
+              break;
+            }
+          }
+        }
+      }
+      
+      private int bitField0_;
+      
+      // required string poolId = 1;
+      private java.lang.Object poolId_ = "";
+      public boolean hasPoolId() {
+        return ((bitField0_ & 0x00000001) == 0x00000001);
+      }
+      public String getPoolId() {
+        java.lang.Object ref = poolId_;
+        if (!(ref instanceof String)) {
+          String s = ((com.google.protobuf.ByteString) ref).toStringUtf8();
+          poolId_ = s;
+          return s;
+        } else {
+          return (String) ref;
+        }
+      }
+      public Builder setPoolId(String value) {
+        if (value == null) {
+    throw new NullPointerException();
+  }
+  bitField0_ |= 0x00000001;
+        poolId_ = value;
+        onChanged();
+        return this;
+      }
+      public Builder clearPoolId() {
+        bitField0_ = (bitField0_ & ~0x00000001);
+        poolId_ = getDefaultInstance().getPoolId();
+        onChanged();
+        return this;
+      }
+      void setPoolId(com.google.protobuf.ByteString value) {
+        bitField0_ |= 0x00000001;
+        poolId_ = value;
+        onChanged();
+      }
+      
+      // required uint64 blockId = 2;
+      private long blockId_ ;
+      public boolean hasBlockId() {
+        return ((bitField0_ & 0x00000002) == 0x00000002);
+      }
+      public long getBlockId() {
+        return blockId_;
+      }
+      public Builder setBlockId(long value) {
+        bitField0_ |= 0x00000002;
+        blockId_ = value;
+        onChanged();
+        return this;
+      }
+      public Builder clearBlockId() {
+        bitField0_ = (bitField0_ & ~0x00000002);
+        blockId_ = 0L;
+        onChanged();
+        return this;
+      }
+      
+      // required uint64 numBytes = 3;
+      private long numBytes_ ;
+      public boolean hasNumBytes() {
+        return ((bitField0_ & 0x00000004) == 0x00000004);
+      }
+      public long getNumBytes() {
+        return numBytes_;
+      }
+      public Builder setNumBytes(long value) {
+        bitField0_ |= 0x00000004;
+        numBytes_ = value;
+        onChanged();
+        return this;
+      }
+      public Builder clearNumBytes() {
+        bitField0_ = (bitField0_ & ~0x00000004);
+        numBytes_ = 0L;
+        onChanged();
+        return this;
+      }
+      
+      // required uint64 generationStamp = 4;
+      private long generationStamp_ ;
+      public boolean hasGenerationStamp() {
+        return ((bitField0_ & 0x00000008) == 0x00000008);
+      }
+      public long getGenerationStamp() {
+        return generationStamp_;
+      }
+      public Builder setGenerationStamp(long value) {
+        bitField0_ |= 0x00000008;
+        generationStamp_ = value;
+        onChanged();
+        return this;
+      }
+      public Builder clearGenerationStamp() {
+        bitField0_ = (bitField0_ & ~0x00000008);
+        generationStamp_ = 0L;
+        onChanged();
+        return this;
+      }
+      
+      // @@protoc_insertion_point(builder_scope:ExtendedBlockProto)
+    }
+    
+    static {
+      defaultInstance = new ExtendedBlockProto(true);
+      defaultInstance.initFields();
+    }
+    
+    // @@protoc_insertion_point(class_scope:ExtendedBlockProto)
+  }
+  
+  public interface BlockTokenIdentifierProtoOrBuilder
+      extends com.google.protobuf.MessageOrBuilder {
+    
+    // required bytes identifier = 1;
+    boolean hasIdentifier();
+    com.google.protobuf.ByteString getIdentifier();
+    
+    // required bytes password = 2;
+    boolean hasPassword();
+    com.google.protobuf.ByteString getPassword();
+    
+    // required string kind = 3;
+    boolean hasKind();
+    String getKind();
+    
+    // required string service = 4;
+    boolean hasService();
+    String getService();
+  }
+  public static final class BlockTokenIdentifierProto extends
+      com.google.protobuf.GeneratedMessage
+      implements BlockTokenIdentifierProtoOrBuilder {
+    // Use BlockTokenIdentifierProto.newBuilder() to construct.
+    private BlockTokenIdentifierProto(Builder builder) {
+      super(builder);
+    }
+    private BlockTokenIdentifierProto(boolean noInit) {}
+    
+    private static final BlockTokenIdentifierProto defaultInstance;
+    public static BlockTokenIdentifierProto getDefaultInstance() {
+      return defaultInstance;
+    }
+    
+    public BlockTokenIdentifierProto getDefaultInstanceForType() {
+      return defaultInstance;
+    }
+    
+    public static final com.google.protobuf.Descriptors.Descriptor
+        getDescriptor() {
+      return org.apache.hadoop.hdfs.protocol.proto.HdfsProtos.internal_static_BlockTokenIdentifierProto_descriptor;
+    }
+    
+    protected com.google.protobuf.GeneratedMessage.FieldAccessorTable
+        internalGetFieldAccessorTable() {
+      return org.apache.hadoop.hdfs.protocol.proto.HdfsProtos.internal_static_BlockTokenIdentifierProto_fieldAccessorTable;
+    }
+    
+    private int bitField0_;
+    // required bytes identifier = 1;
+    public static final int IDENTIFIER_FIELD_NUMBER = 1;
+    private com.google.protobuf.ByteString identifier_;
+    public boolean hasIdentifier() {
+      return ((bitField0_ & 0x00000001) == 0x00000001);
+    }
+    public com.google.protobuf.ByteString getIdentifier() {
+      return identifier_;
+    }
+    
+    // required bytes password = 2;
+    public static final int PASSWORD_FIELD_NUMBER = 2;
+    private com.google.protobuf.ByteString password_;
+    public boolean hasPassword() {
+      return ((bitField0_ & 0x00000002) == 0x00000002);
+    }
+    public com.google.protobuf.ByteString getPassword() {
+      return password_;
+    }
+    
+    // required string kind = 3;
+    public static final int KIND_FIELD_NUMBER = 3;
+    private java.lang.Object kind_;
+    public boolean hasKind() {
+      return ((bitField0_ & 0x00000004) == 0x00000004);
+    }
+    public String getKind() {
+      java.lang.Object ref = kind_;
+      if (ref instanceof String) {
+        return (String) ref;
+      } else {
+        com.google.protobuf.ByteString bs = 
+            (com.google.protobuf.ByteString) ref;
+        String s = bs.toStringUtf8();
+        if (com.google.protobuf.Internal.isValidUtf8(bs)) {
+          kind_ = s;
+        }
+        return s;
+      }
+    }
+    private com.google.protobuf.ByteString getKindBytes() {
+      java.lang.Object ref = kind_;
+      if (ref instanceof String) {
+        com.google.protobuf.ByteString b = 
+            com.google.protobuf.ByteString.copyFromUtf8((String) ref);
+        kind_ = b;
+        return b;
+      } else {
+        return (com.google.protobuf.ByteString) ref;
+      }
+    }
+    
+    // required string service = 4;
+    public static final int SERVICE_FIELD_NUMBER = 4;
+    private java.lang.Object service_;
+    public boolean hasService() {
+      return ((bitField0_ & 0x00000008) == 0x00000008);
+    }
+    public String getService() {
+      java.lang.Object ref = service_;
+      if (ref instanceof String) {
+        return (String) ref;
+      } else {
+        com.google.protobuf.ByteString bs = 
+            (com.google.protobuf.ByteString) ref;
+        String s = bs.toStringUtf8();
+        if (com.google.protobuf.Internal.isValidUtf8(bs)) {
+          service_ = s;
+        }
+        return s;
+      }
+    }
+    private com.google.protobuf.ByteString getServiceBytes() {
+      java.lang.Object ref = service_;
+      if (ref instanceof String) {
+        com.google.protobuf.ByteString b = 
+            com.google.protobuf.ByteString.copyFromUtf8((String) ref);
+        service_ = b;
+        return b;
+      } else {
+        return (com.google.protobuf.ByteString) ref;
+      }
+    }
+    
+    private void initFields() {
+      identifier_ = com.google.protobuf.ByteString.EMPTY;
+      password_ = com.google.protobuf.ByteString.EMPTY;
+      kind_ = "";
+      service_ = "";
+    }
+    private byte memoizedIsInitialized = -1;
+    public final boolean isInitialized() {
+      byte isInitialized = memoizedIsInitialized;
+      if (isInitialized != -1) return isInitialized == 1;
+      
+      if (!hasIdentifier()) {
+        memoizedIsInitialized = 0;
+        return false;
+      }
+      if (!hasPassword()) {
+        memoizedIsInitialized = 0;
+        return false;
+      }
+      if (!hasKind()) {
+        memoizedIsInitialized = 0;
+        return false;
+      }
+      if (!hasService()) {
+        memoizedIsInitialized = 0;
+        return false;
+      }
+      memoizedIsInitialized = 1;
+      return true;
+    }
+    
+    public void writeTo(com.google.protobuf.CodedOutputStream output)
+                        throws java.io.IOException {
+      getSerializedSize();
+      if (((bitField0_ & 0x00000001) == 0x00000001)) {
+        output.writeBytes(1, identifier_);
+      }
+      if (((bitField0_ & 0x00000002) == 0x00000002)) {
+        output.writeBytes(2, password_);
+      }
+      if (((bitField0_ & 0x00000004) == 0x00000004)) {
+        output.writeBytes(3, getKindBytes());
+      }
+      if (((bitField0_ & 0x00000008) == 0x00000008)) {
+        output.writeBytes(4, getServiceBytes());
+      }
+      getUnknownFields().writeTo(output);
+    }
+    
+    private int memoizedSerializedSize = -1;
+    public int getSerializedSize() {
+      int size = memoizedSerializedSize;
+      if (size != -1) return size;
+    
+      size = 0;
+      if (((bitField0_ & 0x00000001) == 0x00000001)) {
+        size += com.google.protobuf.CodedOutputStream
+          .computeBytesSize(1, identifier_);
+      }
+      if (((bitField0_ & 0x00000002) == 0x00000002)) {
+        size += com.google.protobuf.CodedOutputStream
+          .computeBytesSize(2, password_);
+      }
+      if (((bitField0_ & 0x00000004) == 0x00000004)) {
+        size += com.google.protobuf.CodedOutputStream
+          .computeBytesSize(3, getKindBytes());
+      }
+      if (((bitField0_ & 0x00000008) == 0x00000008)) {
+        size += com.google.protobuf.CodedOutputStream
+          .computeBytesSize(4, getServiceBytes());
+      }
+      size += getUnknownFields().getSerializedSize();
+      memoizedSerializedSize = size;
+      return size;
+    }
+    
+    private static final long serialVersionUID = 0L;
+    @java.lang.Override
+    protected java.lang.Object writeReplace()
+        throws java.io.ObjectStreamException {
+      return super.writeReplace();
+    }
+    
+    @java.lang.Override
+    public boolean equals(final java.lang.Object obj) {
+      if (obj == this) {
+       return true;
+      }
+      if (!(obj instanceof org.apache.hadoop.hdfs.protocol.proto.HdfsProtos.BlockTokenIdentifierProto)) {
+        return super.equals(obj);
+      }
+      org.apache.hadoop.hdfs.protocol.proto.HdfsProtos.BlockTokenIdentifierProto other = (org.apache.hadoop.hdfs.protocol.proto.HdfsProtos.BlockTokenIdentifierProto) obj;
+      
+      boolean result = true;
+      result = result && (hasIdentifier() == other.hasIdentifier());
+      if (hasIdentifier()) {
+        result = result && getIdentifier()
+            .equals(other.getIdentifier());
+      }
+      result = result && (hasPassword() == other.hasPassword());
+      if (hasPassword()) {
+        result = result && getPassword()
+            .equals(other.getPassword());
+      }
+      result = result && (hasKind() == other.hasKind());
+      if (hasKind()) {
+        result = result && getKind()
+            .equals(other.getKind());
+      }
+      result = result && (hasService() == other.hasService());
+      if (hasService()) {
+        result = result && getService()
+            .equals(other.getService());
+      }
+      result = result &&
+          getUnknownFields().equals(other.getUnknownFields());
+      return result;
+    }
+    
+    @java.lang.Override
+    public int hashCode() {
+      int hash = 41;
+      hash = (19 * hash) + getDescriptorForType().hashCode();
+      if (hasIdentifier()) {
+        hash = (37 * hash) + IDENTIFIER_FIELD_NUMBER;
+        hash = (53 * hash) + getIdentifier().hashCode();
+      }
+      if (hasPassword()) {
+        hash = (37 * hash) + PASSWORD_FIELD_NUMBER;
+        hash = (53 * hash) + getPassword().hashCode();
+      }
+      if (hasKind()) {
+        hash = (37 * hash) + KIND_FIELD_NUMBER;
+        hash = (53 * hash) + getKind().hashCode();
+      }
+      if (hasService()) {
+        hash = (37 * hash) + SERVICE_FIELD_NUMBER;
+        hash = (53 * hash) + getService().hashCode();
+      }
+      hash = (29 * hash) + getUnknownFields().hashCode();
+      return hash;
+    }
+    
+    public static org.apache.hadoop.hdfs.protocol.proto.HdfsProtos.BlockTokenIdentifierProto parseFrom(
+        com.google.protobuf.ByteString data)
+        throws com.google.protobuf.InvalidProtocolBufferException {
+      return newBuilder().mergeFrom(data).buildParsed();
+    }
+    public static org.apache.hadoop.hdfs.protocol.proto.HdfsProtos.BlockTokenIdentifierProto parseFrom(
+        com.google.protobuf.ByteString data,
+        com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+        throws com.google.protobuf.InvalidProtocolBufferException {
+      return newBuilder().mergeFrom(data, extensionRegistry)
+               .buildParsed();
+    }
+    public static org.apache.hadoop.hdfs.protocol.proto.HdfsProtos.BlockTokenIdentifierProto parseFrom(byte[] data)
+        throws com.google.protobuf.InvalidProtocolBufferException {
+      return newBuilder().mergeFrom(data).buildParsed();
+    }
+    public static org.apache.hadoop.hdfs.protocol.proto.HdfsProtos.BlockTokenIdentifierProto parseFrom(
+        byte[] data,
+        com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+        throws com.google.protobuf.InvalidProtocolBufferException {
+      return newBuilder().mergeFrom(data, extensionRegistry)
+               .buildParsed();
+    }
+    public static org.apache.hadoop.hdfs.protocol.proto.HdfsProtos.BlockTokenIdentifierProto parseFrom(java.io.InputStream input)
+        throws java.io.IOException {
+      return newBuilder().mergeFrom(input).buildParsed();
+    }
+    public static org.apache.hadoop.hdfs.protocol.proto.HdfsProtos.BlockTokenIdentifierProto parseFrom(
+        java.io.InputStream input,
+        com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+        throws java.io.IOException {
+      return newBuilder().mergeFrom(input, extensionRegistry)
+               .buildParsed();
+    }
+    public static org.apache.hadoop.hdfs.protocol.proto.HdfsProtos.BlockTokenIdentifierProto parseDelimitedFrom(java.io.InputStream input)
+        throws java.io.IOException {
+      Builder builder = newBuilder();
+      if (builder.mergeDelimitedFrom(input)) {
+        return builder.buildParsed();
+      } else {
+        return null;
+      }
+    }
+    public static org.apache.hadoop.hdfs.protocol.proto.HdfsProtos.BlockTokenIdentifierProto parseDelimitedFrom(
+        java.io.InputStream input,
+        com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+        throws java.io.IOException {
+      Builder builder = newBuilder();
+      if (builder.mergeDelimitedFrom(input, extensionRegistry)) {
+        return builder.buildParsed();
+      } else {
+        return null;
+      }
+    }
+    public static org.apache.hadoop.hdfs.protocol.proto.HdfsProtos.BlockTokenIdentifierProto parseFrom(
+        com.google.protobuf.CodedInputStream input)
+        throws java.io.IOException {
+      return newBuilder().mergeFrom(input).buildParsed();
+    }
+    public static org.apache.hadoop.hdfs.protocol.proto.HdfsProtos.BlockTokenIdentifierProto parseFrom(
+        com.google.protobuf.CodedInputStream input,
+        com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+        throws java.io.IOException {
+      return newBuilder().mergeFrom(input, extensionRegistry)
+               .buildParsed();
+    }
+    
+    public static Builder newBuilder() { return Builder.create(); }
+    public Builder newBuilderForType() { return newBuilder(); }
+    public static Builder newBuilder(org.apache.hadoop.hdfs.protocol.proto.HdfsProtos.BlockTokenIdentifierProto prototype) {
+      return newBuilder().mergeFrom(prototype);
+    }
+    public Builder toBuilder() { return newBuilder(this); }
+    
+    @java.lang.Override
+    protected Builder newBuilderForType(
+        com.google.protobuf.GeneratedMessage.BuilderParent parent) {
+      Builder builder = new Builder(parent);
+      return builder;
+    }
+    public static final class Builder extends
+        com.google.protobuf.GeneratedMessage.Builder<Builder>
+       implements org.apache.hadoop.hdfs.protocol.proto.HdfsProtos.BlockTokenIdentifierProtoOrBuilder {
+      public static final com.google.protobuf.Descriptors.Descriptor
+          getDescriptor() {
+        return org.apache.hadoop.hdfs.protocol.proto.HdfsProtos.internal_static_BlockTokenIdentifierProto_descriptor;
+      }
+      
+      protected com.google.protobuf.GeneratedMessage.FieldAccessorTable
+          internalGetFieldAccessorTable() {
+        return org.apache.hadoop.hdfs.protocol.proto.HdfsProtos.internal_static_BlockTokenIdentifierProto_fieldAccessorTable;
+      }
+      
+      // Construct using org.apache.hadoop.hdfs.protocol.proto.HdfsProtos.BlockTokenIdentifierProto.newBuilder()
+      private Builder() {
+        maybeForceBuilderInitialization();
+      }
+      
+      private Builder(BuilderParent parent) {
+        super(parent);
+        maybeForceBuilderInitialization();
+      }
+      private void maybeForceBuilderInitialization() {
+        if (com.google.protobuf.GeneratedMessage.alwaysUseFieldBuilders) {
+        }
+      }
+      private static Builder create() {
+        return new Builder();
+      }
+      
+      public Builder clear() {
+        super.clear();
+        identifier_ = com.google.protobuf.ByteString.EMPTY;
+        bitField0_ = (bitField0_ & ~0x00000001);
+        password_ = com.google.protobuf.ByteString.EMPTY;
+        bitField0_ = (bitField0_ & ~0x00000002);
+        kind_ = "";
+        bitField0_ = (bitField0_ & ~0x00000004);
+        service_ = "";
+        bitField0_ = (bitField0_ & ~0x00000008);
+        return this;
+      }
+      
+      public Builder clone() {
+        return create().mergeFrom(buildPartial());
+      }
+      
+      public com.google.protobuf.Descriptors.Descriptor
+          getDescriptorForType() {
+        return org.apache.hadoop.hdfs.protocol.proto.HdfsProtos.BlockTokenIdentifierProto.getDescriptor();
+      }
+      
+      public org.apache.hadoop.hdfs.protocol.proto.HdfsProtos.BlockTokenIdentifierProto getDefaultInstanceForType() {
+        return org.apache.hadoop.hdfs.protocol.proto.HdfsProtos.BlockTokenIdentifierProto.getDefaultInstance();
+      }
+      
+      public org.apache.hadoop.hdfs.protocol.proto.HdfsProtos.BlockTokenIdentifierProto build() {
+        org.apache.hadoop.hdfs.protocol.proto.HdfsProtos.BlockTokenIdentifierProto result = buildPartial();
+        if (!result.isInitialized()) {
+          throw newUninitializedMessageException(result);
+        }
+        return result;
+      }
+      
+      private org.apache.hadoop.hdfs.protocol.proto.HdfsProtos.BlockTokenIdentifierProto buildParsed()
+          throws com.google.protobuf.InvalidProtocolBufferException {
+        org.apache.hadoop.hdfs.protocol.proto.HdfsProtos.BlockTokenIdentifierProto result = buildPartial();
+        if (!result.isInitialized()) {
+          throw newUninitializedMessageException(
+            result).asInvalidProtocolBufferException();
+        }
+        return result;
+      }
+      
+      public org.apache.hadoop.hdfs.protocol.proto.HdfsProtos.BlockTokenIdentifierProto buildPartial() {
+        org.apache.hadoop.hdfs.protocol.proto.HdfsProtos.BlockTokenIdentifierProto result = new org.apache.hadoop.hdfs.protocol.proto.HdfsProtos.BlockTokenIdentifierProto(this);
+        int from_bitField0_ = bitField0_;
+        int to_bitField0_ = 0;
+        if (((from_bitField0_ & 0x00000001) == 0x00000001)) {
+          to_bitField0_ |= 0x00000001;
+        }
+        result.identifier_ = identifier_;
+        if (((from_bitField0_ & 0x00000002) == 0x00000002)) {
+          to_bitField0_ |= 0x00000002;
+        }
+        result.password_ = password_;
+        if (((from_bitField0_ & 0x00000004) == 0x00000004)) {
+          to_bitField0_ |= 0x00000004;
+        }
+        result.kind_ = kind_;
+        if (((from_bitField0_ & 0x00000008) == 0x00000008)) {
+          to_bitField0_ |= 0x00000008;
+        }
+        result.service_ = service_;
+        result.bitField0_ = to_bitField0_;
+        onBuilt();
+        return result;
+      }
+      
+      public Builder mergeFrom(com.google.protobuf.Message other) {
+        if (other instanceof org.apache.hadoop.hdfs.protocol.proto.HdfsProtos.BlockTokenIdentifierProto) {
+          return mergeFrom((org.apache.hadoop.hdfs.protocol.proto.HdfsProtos.BlockTokenIdentifierProto)other);
+        } else {
+          super.mergeFrom(other);
+          return this;
+        }
+      }
+      
+      public Builder mergeFrom(org.apache.hadoop.hdfs.protocol.proto.HdfsProtos.BlockTokenIdentifierProto other) {
+        if (other == org.apache.hadoop.hdfs.protocol.proto.HdfsProtos.BlockTokenIdentifierProto.getDefaultInstance()) return this;
+        if (other.hasIdentifier()) {
+          setIdentifier(other.getIdentifier());
+        }
+        if (other.hasPassword()) {
+          setPassword(other.getPassword());
+        }
+        if (other.hasKind()) {
+          setKind(other.getKind());
+        }
+        if (other.hasService()) {
+          setService(other.getService());
+        }
+        this.mergeUnknownFields(other.getUnknownFields());
+        return this;
+      }
+      
+      public final boolean isInitialized() {
+        if (!hasIdentifier()) {
+          
+          return false;
+        }
+        if (!hasPassword()) {
+          
+          return false;
+        }
+        if (!hasKind()) {
+          
+          return false;
+        }
+        if (!hasService()) {
+          
+          return false;
+        }
+        return true;
+      }
+      
+      public Builder mergeFrom(
+          com.google.protobuf.CodedInputStream input,
+          com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+          throws java.io.IOException {
+        com.google.protobuf.UnknownFieldSet.Builder unknownFields =
+          com.google.protobuf.UnknownFieldSet.newBuilder(
+            this.getUnknownFields());
+        while (true) {
+          int tag = input.readTag();
+          switch (tag) {
+            case 0:
+              this.setUnknownFields(unknownFields.build());
+              onChanged();
+              return this;
+            default: {
+              if (!parseUnknownField(input, unknownFields,
+                                     extensionRegistry, tag)) {
+                this.setUnknownFields(unknownFields.build());
+                onChanged();
+                return this;
+              }
+              break;
+            }
+            case 10: {
+              bitField0_ |= 0x00000001;
+              identifier_ = input.readBytes();
+              break;
+            }
+            case 18: {
+              bitField0_ |= 0x00000002;
+              password_ = input.readBytes();
+              break;
+            }
+            case 26: {
+              bitField0_ |= 0x00000004;
+              kind_ = input.readBytes();
+              break;
+            }
+            case 34: {
+              bitField0_ |= 0x00000008;
+              service_ = input.readBytes();
+              break;
+            }
+          }
+        }
+      }
+      
+      private int bitField0_;
+      
+      // required bytes identifier = 1;
+      private com.google.protobuf.ByteString identifier_ = com.google.protobuf.ByteString.EMPTY;
+      public boolean hasIdentifier() {
+        return ((bitField0_ & 0x00000001) == 0x00000001);
+      }
+      public com.google.protobuf.ByteString getIdentifier() {
+        return identifier_;
+      }
+      public Builder setIdentifier(com.google.protobuf.ByteString value) {
+        if (value == null) {
+    throw new NullPointerException();
+  }
+  bitField0_ |= 0x00000001;
+        identifier_ = value;
+        onChanged();
+        return this;
+      }
+      public Builder clearIdentifier() {
+        bitField0_ = (bitField0_ & ~0x00000001);
+        identifier_ = getDefaultInstance().getIdentifier();
+        onChanged();
+        return this;
+      }
+      
+      // required bytes password = 2;
+      private com.google.protobuf.ByteString password_ = com.google.protobuf.ByteString.EMPTY;
+      public boolean hasPassword() {
+        return ((bitField0_ & 0x00000002) == 0x00000002);
+      }
+      public com.google.protobuf.ByteString getPassword() {
+        return password_;
+      }
+      public Builder setPassword(com.google.protobuf.ByteString value) {
+        if (value == null) {
+    throw new NullPointerException();
+  }
+  bitField0_ |= 0x00000002;
+        password_ = value;
+        onChanged();
+        return this;
+      }
+      public Builder clearPassword() {
+        bitField0_ = (bitField0_ & ~0x00000002);
+        password_ = getDefaultInstance().getPassword();
+        onChanged();
+        return this;
+      }
+      
+      // required string kind = 3;
+      private java.lang.Object kind_ = "";
+      public boolean hasKind() {
+        return ((bitField0_ & 0x00000004) == 0x00000004);
+      }
+      public String getKind() {
+        java.lang.Object ref = kind_;
+        if (!(ref instanceof String)) {
+          String s = ((com.google.protobuf.ByteString) ref).toStringUtf8();
+          kind_ = s;
+          return s;
+        } else {
+          return (String) ref;
+        }
+      }
+      public Builder setKind(String value) {
+        if (value == null) {
+    throw new NullPointerException();
+  }
+  bitField0_ |= 0x00000004;
+        kind_ = value;
+        onChanged();
+        return this;
+      }
+      public Builder clearKind() {
+        bitField0_ = (bitField0_ & ~0x00000004);
+        kind_ = getDefaultInstance().getKind();
+        onChanged();
+        return this;
+      }
+      void setKind(com.google.protobuf.ByteString value) {
+        bitField0_ |= 0x00000004;
+        kind_ = value;
+        onChanged();
+      }
+      
+      // required string service = 4;
+      private java.lang.Object service_ = "";
+      public boolean hasService() {
+        return ((bitField0_ & 0x00000008) == 0x00000008);
+      }
+      public String getService() {
+        java.lang.Object ref = service_;
+        if (!(ref instanceof String)) {
+          String s = ((com.google.protobuf.ByteString) ref).toStringUtf8();
+          service_ = s;
+          return s;
+        } else {
+          return (String) ref;
+        }
+      }
+      public Builder setService(String value) {
+        if (value == null) {
+    throw new NullPointerException();
+  }
+  bitField0_ |= 0x00000008;
+        service_ = value;
+        onChanged();
+        return this;
+      }
+      public Builder clearService() {
+        bitField0_ = (bitField0_ & ~0x00000008);
+        service_ = getDefaultInstance().getService();
+        onChanged();
+        return this;
+      }
+      void setService(com.google.protobuf.ByteString value) {
+        bitField0_ |= 0x00000008;
+        service_ = value;
+        onChanged();
+      }
+      
+      // @@protoc_insertion_point(builder_scope:BlockTokenIdentifierProto)
+    }
+    
+    static {
+      defaultInstance = new BlockTokenIdentifierProto(true);
+      defaultInstance.initFields();
+    }
+    
+    // @@protoc_insertion_point(class_scope:BlockTokenIdentifierProto)
+  }
+  
+  public interface DatanodeIDProtoOrBuilder
+      extends com.google.protobuf.MessageOrBuilder {
+    
+    // required string name = 1;
+    boolean hasName();
+    String getName();
+    
+    // required string storageID = 2;
+    boolean hasStorageID();
+    String getStorageID();
+    
+    // required uint32 infoPort = 3;
+    boolean hasInfoPort();
+    int getInfoPort();
+  }
+  public static final class DatanodeIDProto extends
+      com.google.protobuf.GeneratedMessage
+      implements DatanodeIDProtoOrBuilder {
+    // Use DatanodeIDProto.newBuilder() to construct.
+    private DatanodeIDProto(Builder builder) {
+      super(builder);
+    }
+    private DatanodeIDProto(boolean noInit) {}
+    
+    private static final DatanodeIDProto defaultInstance;
+    public static DatanodeIDProto getDefaultInstance() {
+      return defaultInstance;
+    }
+    
+    public DatanodeIDProto getDefaultInstanceForType() {
+      return defaultInstance;
+    }
+    
+    public static final com.google.protobuf.Descriptors.Descriptor
+        getDescriptor() {
+      return org.apache.hadoop.hdfs.protocol.proto.HdfsProtos.internal_static_DatanodeIDProto_descriptor;
+    }
+    
+    protected com.google.protobuf.GeneratedMessage.FieldAccessorTable
+        internalGetFieldAccessorTable() {
+      return org.apache.hadoop.hdfs.protocol.proto.HdfsProtos.internal_static_DatanodeIDProto_fieldAccessorTable;
+    }
+    
+    private int bitField0_;
+    // required string name = 1;
+    public static final int NAME_FIELD_NUMBER = 1;
+    private java.lang.Object name_;
+    public boolean hasName() {
+      return ((bitField0_ & 0x00000001) == 0x00000001);
+    }
+    public String getName() {
+      java.lang.Object ref = name_;
+      if (ref instanceof String) {
+        return (String) ref;
+      } else {
+        com.google.protobuf.ByteString bs = 
+            (com.google.protobuf.ByteString) ref;
+        String s = bs.toStringUtf8();
+        if (com.google.protobuf.Internal.isValidUtf8(bs)) {
+          name_ = s;
+        }
+        return s;
+      }
+    }
+    private com.google.protobuf.ByteString getNameBytes() {
+      java.lang.Object ref = name_;
+      if (ref instanceof String) {
+        com.google.protobuf.ByteString b = 
+            com.google.protobuf.ByteString.copyFromUtf8((String) ref);
+        name_ = b;
+        return b;
+      } else {
+        return (com.google.protobuf.ByteString) ref;
+      }
+    }
+    
+    // required string storageID = 2;
+    public static final int STORAGEID_FIELD_NUMBER = 2;
+    private java.lang.Object storageID_;
+    public boolean hasStorageID() {
+      return ((bitField0_ & 0x00000002) == 0x00000002);
+    }
+    public String getStorageID() {
+      java.lang.Object ref = storageID_;
+      if (ref instanceof String) {
+        return (String) ref;
+      } else {
+        com.google.protobuf.ByteString bs = 
+            (com.google.protobuf.ByteString) ref;
+        String s = bs.toStringUtf8();
+        if (com.google.protobuf.Internal.isValidUtf8(bs)) {
+          storageID_ = s;
+        }
+        return s;
+      }
+    }
+    private com.google.protobuf.ByteString getStorageIDBytes() {
+      java.lang.Object ref = storageID_;
+      if (ref instanceof String) {
+        com.google.protobuf.ByteString b = 
+            com.google.protobuf.ByteString.copyFromUtf8((String) ref);
+        storageID_ = b;
+        return b;
+      } else {
+        return (com.google.protobuf.ByteString) ref;
+      }
+    }
+    
+    // required uint32 infoPort = 3;
+    public static final int INFOPORT_FIELD_NUMBER = 3;
+    private int infoPort_;
+    public boolean hasInfoPort() {
+      return ((bitField0_ & 0x00000004) == 0x00000004);
+    }
+    public int getInfoPort() {
+      return infoPort_;
+    }
+    
+    private void initFields() {
+      name_ = "";
+      storageID_ = "";
+      infoPort_ = 0;
+    }
+    private byte memoizedIsInitialized = -1;
+    public final boolean isInitialized() {
+      byte isInitialized = memoizedIsInitialized;
+      if (isInitialized != -1) return isInitialized == 1;
+      
+      if (!hasName()) {
+        memoizedIsInitialized = 0;
+        return false;
+      }
+      if (!hasStorageID()) {
+        memoizedIsInitialized = 0;
+        return false;
+      }
+      if (!hasInfoPort()) {
+        memoizedIsInitialized = 0;
+        return false;
+      }
+      memoizedIsInitialized = 1;
+      return true;
+    }
+    
+    public void writeTo(com.google.protobuf.CodedOutputStream output)
+                        throws java.io.IOException {
+      getSerializedSize();
+      if (((bitField0_ & 0x00000001) == 0x00000001)) {
+        output.writeBytes(1, getNameBytes());
+      }
+      if (((bitField0_ & 0x00000002) == 0x00000002)) {
+        output.writeBytes(2, getStorageIDBytes());
+      }
+      if (((bitField0_ & 0x00000004) == 0x00000004)) {
+        output.writeUInt32(3, infoPort_);
+      }
+      getUnknownFields().writeTo(output);
+    }
+    
+    private int memoizedSerializedSize = -1;
+    public int getSerializedSize() {
+      int size = memoizedSerializedSize;
+      if (size != -1) return size;
+    
+      size = 0;
+      if (((bitField0_ & 0x00000001) == 0x00000001)) {
+        size += com.google.protobuf.CodedOutputStream
+          .computeBytesSize(1, getNameBytes());
+      }
+      if (((bitField0_ & 0x00000002) == 0x00000002)) {
+        size += com.google.protobuf.CodedOutputStream
+          .computeBytesSize(2, getStorageIDBytes());
+      }
+      if (((bitField0_ & 0x00000004) == 0x00000004)) {
+        size += com.google.protobuf.CodedOutputStream
+          .computeUInt32Size(3, infoPort_);
+      }
+      size += getUnknownFields().getSerializedSize();
+      memoizedSerializedSize = size;
+      return size;
+    }
+    
+    private static final long serialVersionUID = 0L;
+    @java.lang.Override
+    protected java.lang.Object writeReplace()
+        throws java.io.ObjectStreamException {
+      return super.writeReplace();
+    }
+    
+    @java.lang.Override
+    public boolean equals(final java.lang.Object obj) {
+      if (obj == this) {
+       return true;
+      }
+      if (!(obj instanceof org.apache.hadoop.hdfs.protocol.proto.HdfsProtos.DatanodeIDProto)) {
+        return super.equals(obj);
+      }
+      org.apache.hadoop.hdfs.protocol.proto.HdfsProtos.DatanodeIDProto other = (org.apache.hadoop.hdfs.protocol.proto.HdfsProtos.DatanodeIDProto) obj;
+      
+      boolean result = true;
+      result = result && (hasName() == other.hasName());
+      if (hasName()) {
+        result = result && getName()
+            .equals(other.getName());
+      }
+      result = result && (hasStorageID() == other.hasStorageID());
+      if (hasStorageID()) {
+        result = result && getStorageID()
+            .equals(other.getStorageID());
+      }
+      result = result && (hasInfoPort() == other.hasInfoPort());
+      if (hasInfoPort()) {
+        result = result && (getInfoPort()
+            == other.getInfoPort());
+      }
+      result = result &&
+          getUnknownFields().equals(other.getUnknownFields());
+      return result;
+    }
+    
+    @java.lang.Override
+    public int hashCode() {
+      int hash = 41;
+      hash = (19 * hash) + getDescriptorForType().hashCode();
+      if (hasName()) {
+        hash = (37 * hash) + NAME_FIELD_NUMBER;
+        hash = (53 * hash) + getName().hashCode();
+      }
+      if (hasStorageID()) {
+        hash = (37 * hash) + STORAGEID_FIELD_NUMBER;
+        hash = (53 * hash) + getStorageID().hashCode();
+      }
+      if (hasInfoPort()) {
+        hash = (37 * hash) + INFOPORT_FIELD_NUMBER;
+        hash = (53 * hash) + getInfoPort();
+      }
+      hash = (29 * hash) + getUnknownFields().hashCode();
+      return hash;
+    }
+    
+    public static org.apache.hadoop.hdfs.protocol.proto.HdfsProtos.DatanodeIDProto parseFrom(
+        com.google.protobuf.ByteString data)
+        throws com.google.protobuf.InvalidProtocolBufferException {
+      return newBuilder().mergeFrom(data).buildParsed();
+    }
+    public static org.apache.hadoop.hdfs.protocol.proto.HdfsProtos.DatanodeIDProto parseFrom(
+        com.google.protobuf.ByteString data,
+        com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+        throws com.google.protobuf.InvalidProtocolBufferException {
+      return newBuilder().mergeFrom(data, extensionRegistry)
+               .buildParsed();
+    }
+    public static org.apache.hadoop.hdfs.protocol.proto.HdfsProtos.DatanodeIDProto parseFrom(byte[] data)
+        throws com.google.protobuf.InvalidProtocolBufferException {
+      return newBuilder().mergeFrom(data).buildParsed();
+    }
+    public static org.apache.hadoop.hdfs.protocol.proto.HdfsProtos.DatanodeIDProto parseFrom(
+        byte[] data,
+        com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+        throws com.google.protobuf.InvalidProtocolBufferException {
+      return newBuilder().mergeFrom(data, extensionRegistry)
+               .buildParsed();
+    }
+    public static org.apache.hadoop.hdfs.protocol.proto.HdfsProtos.DatanodeIDProto parseFrom(java.io.InputStream input)
+        throws java.io.IOException {
+      return newBuilder().mergeFrom(input).buildParsed();
+    }
+    public static org.apache.hadoop.hdfs.protocol.proto.HdfsProtos.DatanodeIDProto parseFrom(
+        java.io.InputStream input,
+        com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+        throws java.io.IOException {
+      return newBuilder().mergeFrom(input, extensionRegistry)
+               .buildParsed();
+    }
+    public static org.apache.hadoop.hdfs.protocol.proto.HdfsProtos.DatanodeIDProto parseDelimitedFrom(java.io.InputStream input)
+        throws java.io.IOException {
+      Builder builder = newBuilder();
+      if (builder.mergeDelimitedFrom(input)) {
+        return builder.buildParsed();
+      } else {
+        return null;
+      }
+    }
+    public static org.apache.hadoop.hdfs.protocol.proto.HdfsProtos.DatanodeIDProto parseDelimitedFrom(
+        java.io.InputStream input,
+        com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+        throws java.io.IOException {
+      Builder builder = newBuilder();
+      if (builder.mergeDelimitedFrom(input, extensionRegistry)) {
+        return builder.buildParsed();
+      } else {
+        return null;
+      }
+    }
+    public static org.apache.hadoop.hdfs.protocol.proto.HdfsProtos.DatanodeIDProto parseFrom(
+        com.google.protobuf.CodedInputStream input)
+        throws java.io.IOException {
+      return newBuilder().mergeFrom(input).buildParsed();
+    }
+    public static org.apache.hadoop.hdfs.protocol.proto.HdfsProtos.DatanodeIDProto parseFrom(
+        com.google.protobuf.CodedInputStream input,
+        com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+        throws java.io.IOException {
+      return newBuilder().mergeFrom(input, extensionRegistry)
+               .buildParsed();
+    }
+    
+    public static Builder newBuilder() { return Builder.create(); }
+    public Builder newBuilderForType() { return newBuilder(); }
+    public static Builder newBuilder(org.apache.hadoop.hdfs.protocol.proto.HdfsProtos.DatanodeIDProto prototype) {
+      return newBuilder().mergeFrom(prototype);
+    }
+    public Builder toBuilder() { return newBuilder(this); }
+    
+    @java.lang.Override
+    protected Builder newBuilderForType(
+        com.google.protobuf.GeneratedMessage.BuilderParent parent) {
+      Builder builder = new Builder(parent);
+      return builder;
+    }
+    public static final class Builder extends
+        com.google.protobuf.GeneratedMessage.Builder<Builder>
+       implements org.apache.hadoop.hdfs.protocol.proto.HdfsProtos.DatanodeIDProtoOrBuilder {
+      public static final com.google.protobuf.Descriptors.Descriptor
+          getDescriptor() {
+        return org.apache.hadoop.hdfs.protocol.proto.HdfsProtos.internal_static_DatanodeIDProto_descriptor;
+      }
+      
+      protected com.google.protobuf.GeneratedMessage.FieldAccessorTable
+          internalGetFieldAccessorTable() {
+        return org.apache.hadoop.hdfs.protocol.proto.HdfsProtos.internal_static_DatanodeIDProto_fieldAccessorTable;
+      }
+      
+      // Construct using org.apache.hadoop.hdfs.protocol.proto.HdfsProtos.DatanodeIDProto.newBuilder()
+      private Builder() {
+        maybeForceBuilderInitialization();
+      }
+      
+      private Builder(BuilderParent parent) {
+        super(parent);
+        maybeForceBuilderInitialization();
+      }
+      private void maybeForceBuilderInitialization() {
+        if (com.google.protobuf.GeneratedMessage.alwaysUseFieldBuilders) {
+        }
+      }
+      private static Builder create() {
+        return new Builder();
+      }
+      
+      public Builder clear() {
+        super.clear();
+        name_ = "";
+        bitField0_ = (bitField0_ & ~0x00000001);
+        storageID_ = "";
+        bitField0_ = (bitField0_ & ~0x00000002);
+        infoPort_ = 0;
+        bitField0_ = (bitField0_ & ~0x00000004);
+        return this;
+      }
+      
+      public Builder clone() {
+        return create().mergeFrom(buildPartial());
+      }
+      
+      public com.google.protobuf.Descriptors.Descriptor
+          getDescriptorForType() {
+        return org.apache.hadoop.hdfs.protocol.proto.HdfsProtos.DatanodeIDProto.getDescriptor();
+      }
+      
+      public org.apache.hadoop.hdfs.protocol.proto.HdfsProtos.DatanodeIDProto getDefaultInstanceForType() {
+        return org.apache.hadoop.hdfs.protocol.proto.HdfsProtos.DatanodeIDProto.getDefaultInstance();
+      }
+      
+      public org.apache.hadoop.hdfs.protocol.proto.HdfsProtos.DatanodeIDProto build() {
+        org.apache.hadoop.hdfs.protocol.proto.HdfsProtos.DatanodeIDProto result = buildPartial();
+        if (!result.isInitialized()) {
+          throw newUninitializedMessageException(result);
+        }
+        return result;
+      }
+      
+      private org.apache.hadoop.hdfs.protocol.proto.HdfsProtos.DatanodeIDProto buildParsed()
+          throws com.google.protobuf.InvalidProtocolBufferException {
+        org.apache.hadoop.hdfs.protocol.proto.HdfsProtos.DatanodeIDProto result = buildPartial();
+        if (!result.isInitialized()) {
+          throw newUninitializedMessageException(
+            result).asInvalidProtocolBufferException();
+        }
+        return result;
+      }
+      
+      public org.apache.hadoop.hdfs.protocol.proto.HdfsProtos.DatanodeIDProto buildPartial() {
+        org.apache.hadoop.hdfs.protocol.proto.HdfsProtos.DatanodeIDProto result = new org.apache.hadoop.hdfs.protocol.proto.HdfsProtos.DatanodeIDProto(this);
+        int from_bitField0_ = bitField0_;
+        int to_bitField0_ = 0;
+        if (((from_bitField0_ & 0x00000001) == 0x00000001)) {
+          to_bitField0_ |= 0x00000001;
+        }
+        result.name_ = name_;
+        if (((from_bitField0_ & 0x00000002) == 0x00000002)) {
+          to_bitField0_ |= 0x00000002;
+        }
+        result.storageID_ = storageID_;
+        if (((from_bitField0_ & 0x00000004) == 0x00000004)) {
+          to_bitField0_ |= 0x00000004;
+        }
+        result.infoPort_ = infoPort_;
+        result.bitField0_ = to_bitField0_;
+        onBuilt();
+        return result;
+      }
+      
+      public Builder mergeFrom(com.google.protobuf.Message other) {
+        if (other instanceof org.apache.hadoop.hdfs.protocol.proto.HdfsProtos.DatanodeIDProto) {
+          return mergeFrom((org.apache.hadoop.hdfs.protocol.proto.HdfsProtos.DatanodeIDProto)other);
+        } else {
+          super.mergeFrom(other);
+          return this;
+        }
+      }
+      
+      public Builder mergeFrom(org.apache.hadoop.hdfs.protocol.proto.HdfsProtos.DatanodeIDProto other) {
+        if (other == org.apache.hadoop.hdfs.protocol.proto.HdfsProtos.DatanodeIDProto.getDefaultInstance()) return this;
+        if (other.hasName()) {
+          setName(other.getName());
+        }
+        if (other.hasStorageID()) {
+          setStorageID(other.getStorageID());
+        }
+        if (other.hasInfoPort()) {
+          setInfoPort(other.getInfoPort());
+        }
+        this.mergeUnknownFields(other.getUnknownFields());
+        return this;
+      }
+      
+      public final boolean isInitialized() {
+        if (!hasName()) {
+          
+          return false;
+        }
+        if (!hasStorageID()) {
+          
+          return false;
+        }
+        if (!hasInfoPort()) {
+          
+          return false;
+        }
+        return true;
+      }
+      
+      public Builder mergeFrom(
+          com.google.protobuf.CodedInputStream input,
+          com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+          throws java.io.IOException {
+        com.google.protobuf.UnknownFieldSet.Builder unknownFields =
+          com.google.protobuf.UnknownFieldSet.newBuilder(
+            this.getUnknownFields());
+        while (true) {
+          int tag = input.readTag();
+          switch (tag) {
+            case 0:
+              this.setUnknownFields(unknownFields.build());
+              onChanged();
+              return this;
+            default: {
+              if (!parseUnknownField(input, unknownFields,
+                                     extensionRegistry, tag)) {
+                this.setUnknownFields(unknownFields.build());
+                onChanged();
+                return this;
+              }
+              break;
+            }
+            case 10: {
+              bitField0_ |= 0x00000001;
+              name_ = input.readBytes();
+              break;
+            }
+            case 18: {
+              bitField0_ |= 0x00000002;
+              storageID_ = input.readBytes();
+              break;
+            }
+            case 24: {
+              bitField0_ |= 0x00000004;
+              infoPort_ = input.readUInt32();
+              break;
+            }
+          }
+        }
+      }
+      
+      private int bitField0_;
+      
+      // required string name = 1;
+      private java.lang.Object name_ = "";
+      public boolean hasName() {
+        return ((bitField0_ & 0x00000001) == 0x00000001);
+      }
+      public String getName() {
+        java.lang.Object ref = name_;
+        if (!(ref instanceof String)) {
+          String s = ((com.google.protobuf.ByteString) ref).toStringUtf8();
+          name_ = s;
+          return s;
+        } else {
+          return (String) ref;
+        }
+      }
+      public Builder setName(String value) {
+        if (value == null) {
+    throw new NullPointerException();
+  }
+  bitField0_ |= 0x00000001;
+        name_ = value;
+        onChanged();
+        return this;
+      }
+      public Builder clearName() {
+        bitField0_ = (bitField0_ & ~0x00000001);
+        name_ = getDefaultInstance().getName();
+        onChanged();
+        return this;
+      }
+      void setName(com.google.protobuf.ByteString value) {
+        bitField0_ |= 0x00000001;
+        name_ = value;
+        onChanged();
+      }
+      
+      // required string storageID = 2;
+      private java.lang.Object storageID_ = "";
+      public boolean hasStorageID() {
+        return ((bitField0_ & 0x00000002) == 0x00000002);
+      }
+      public String getStorageID() {
+        java.lang.Object ref = storageID_;
+        if (!(ref instanceof String)) {
+          String s = ((com.google.protobuf.ByteString) ref).toStringUtf8();
+          storageID_ = s;
+          return s;
+        } else {
+          return (String) ref;
+        }
+      }
+      public Builder setStorageID(String value) {
+        if (value == null) {
+    throw new NullPointerException();
+  }
+  bitField0_ |= 0x00000002;
+        storageID_ = value;
+        onChanged();
+        return this;
+      }
+      public Builder clearStorageID() {
+        bitField0_ = (bitField0_ & ~0x00000002);
+        storageID_ = getDefaultInstance().getStorageID();
+        onChanged();
+        return this;
+      }
+      void setStorageID(com.google.protobuf.ByteString value) {
+        bitField0_ |= 0x00000002;
+        storageID_ = value;
+        onChanged();
+      }
+      
+      // required uint32 infoPort = 3;
+      private int infoPort_ ;
+      public boolean hasInfoPort() {
+        return ((bitField0_ & 0x00000004) == 0x00000004);
+      }
+      public int getInfoPort() {
+        return infoPort_;
+      }
+      public Builder setInfoPort(int value) {
+        bitField0_ |= 0x00000004;
+        infoPort_ = value;
+        onChanged();
+        return this;
+      }
+      public Builder clearInfoPort() {
+        bitField0_ = (bitField0_ & ~0x00000004);
+        infoPort_ = 0;
+        onChanged();
+        return this;
+      }
+      
+      // @@protoc_insertion_point(builder_scope:DatanodeIDProto)
+    }
+    
+    static {
+      defaultInstance = new DatanodeIDProto(true);
+      defaultInstance.initFields();
+    }
+    
+    // @@protoc_insertion_point(class_scope:DatanodeIDProto)
+  }
+  
+  public interface DatanodeInfoProtoOrBuilder
+      extends com.google.protobuf.MessageOrBuilder {
+    
+    // required .DatanodeIDProto id = 1;
+    boolean hasId();
+    org.apache.hadoop.hdfs.protocol.proto.HdfsProtos.DatanodeIDProto getId();
+    org.apache.hadoop.hdfs.protocol.proto.HdfsProtos.DatanodeIDProtoOrBuilder getIdOrBuilder();
+    
+    // optional uint64 capacity = 2;
+    boolean hasCapacity();
+    long getCapacity();
+    
+    // optional uint64 dfsUsed = 3;
+    boolean hasDfsUsed();
+    long getDfsUsed();
+    
+    // optional uint64 remaining = 4;
+    boolean hasRemaining();
+    long getRemaining();
+    
+    // optional uint64 blockPoolUsed = 5;
+    boolean hasBlockPoolUsed();
+    long getBlockPoolUsed();
+    
+    // optional uint64 lastUpdate = 6;
+    boolean hasLastUpdate();
+    long getLastUpdate();
+    
+    // optional uint32 xceiverCount = 7;
+    boolean hasXceiverCount();
+    int getXceiverCount();
+    
+    // optional string location = 8;
+    boolean hasLocation();
+    String getLocation();
+    
+    // optional string hostName = 9;
+    boolean hasHostName();
+    String getHostName();
+    
+    // optional .DatanodeInfoProto.AdminState adminState = 10;
+    boolean hasAdminState();
+    org.apache.hadoop.hdfs.protocol.proto.HdfsProtos.DatanodeInfoProto.AdminState getAdminState();
+  }
+  public static final class DatanodeInfoProto extends
+      com.google.protobuf.GeneratedMessage
+      implements DatanodeInfoProtoOrBuilder {
+    // Use DatanodeInfoProto.newBuilder() to construct.
+    private DatanodeInfoProto(Builder builder) {
+      super(builder);
+    }
+    private DatanodeInfoProto(boolean noInit) {}
+    
+    private static final DatanodeInfoProto defaultInstance;
+    public static DatanodeInfoProto getDefaultInstance() {
+      return defaultInstance;
+    }
+    
+    public DatanodeInfoProto getDefaultInstanceForType() {
+      return defaultInstance;
+    }
+    
+    public static final com.google.protobuf.Descriptors.Descriptor
+        getDescriptor() {
+      return org.apache.hadoop.hdfs.protocol.proto.HdfsProtos.internal_static_DatanodeInfoProto_descriptor;
+    }
+    
+    protected com.google.protobuf.GeneratedMessage.FieldAccessorTable
+        internalGetFieldAccessorTable() {
+      return org.apache.hadoop.hdfs.protocol.proto.HdfsProtos.internal_static_DatanodeInfoProto_fieldAccessorTable;
+    }
+    
+    public enum AdminState
+        implements com.google.protobuf.ProtocolMessageEnum {
+      NORMAL(0, 0),
+      DECOMMISSION_INPROGRESS(1, 1),
+      DECOMMISSIONED(2, 2),
+      ;
+      
+      public static final int NORMAL_VALUE = 0;
+      public static final int DECOMMISSION_INPROGRESS_VALUE = 1;
+      public static final int DECOMMISSIONED_VALUE = 2;
+      
+      
+      public final int getNumber() { return value; }
+      
+      public static AdminState valueOf(int value) {
+        switch (value) {
+          case 0: return NORMAL;
+          case 1: return DECOMMISSION_INPROGRESS;
+          case 2: return DECOMMISSIONED;
+          default: return null;
+        }
+      }
+      
+      public static com.google.protobuf.Internal.EnumLiteMap<AdminState>
+          internalGetValueMap() {
+        return internalValueMap;
+      }
+      private static com.google.protobuf.Internal.EnumLiteMap<AdminState>
+          internalValueMap =
+            new com.google.protobuf.Internal.EnumLiteMap<AdminState>() {
+              public AdminState findValueByNumber(int number) {
+                return AdminState.valueOf(number);
+              }
+            };
+      
+      public final com.google.protobuf.Descriptors.EnumValueDescriptor
+          getValueDescriptor() {
+        return getDescriptor().getValues().get(index);
+      }
+      public final com.google.protobuf.Descriptors.EnumDescriptor
+          getDescriptorForType() {
+        return getDescriptor();
+      }
+      public static final com.google.protobuf.Descriptors.EnumDescriptor
+          getDescriptor() {
+        return org.apache.hadoop.hdfs.protocol.proto.HdfsProtos.DatanodeInfoProto.getDescriptor().getEnumTypes().get(0);
+      }
+      
+      private static final AdminState[] VALUES = {
+        NORMAL, DECOMMISSION_INPROGRESS, DECOMMISSIONED, 
+      };
+      
+      public static AdminState valueOf(
+          com.google.protobuf.Descriptors.EnumValueDescriptor desc) {
+        if (desc.getType() != getDescriptor()) {
+          throw new java.lang.IllegalArgumentException(
+            "EnumValueDescriptor is not for this type.");
+        }
+        return VALUES[desc.getIndex()];
+      }
+      
+      private final int index;
+      private final int value;
+      
+      private AdminState(int index, int value) {
+        this.index = index;
+        this.value = value;
+      }
+      
+      // @@protoc_insertion_point(enum_scope:DatanodeInfoProto.AdminState)
+    }
+    
+    private int bitField0_;
+    // required .DatanodeIDProto id = 1;
+    public static final int ID_FIELD_NUMBER = 1;
+    private org.apache.hadoop.hdfs.protocol.proto.HdfsProtos.DatanodeIDProto id_;
+    public boolean hasId() {
+      return ((bitField0_ & 0x00000001) == 0x00000001);
+    }
+    public org.apache.hadoop.hdfs.protocol.proto.HdfsProtos.DatanodeIDProto getId() {
+      return id_;
+    }
+    public org.apache.hadoop.hdfs.protocol.proto.HdfsProtos.DatanodeIDProtoOrBuilder getIdOrBuilder() {
+      return id_;
+    }
+    
+    // optional uint64 capacity = 2;
+    public static final int CAPACITY_FIELD_NUMBER = 2;
+    private long capacity_;
+    public boolean hasCapacity() {
+      return ((bitField0_ & 0x00000002) == 0x00000002);
+    }
+    public long getCapacity() {
+      return capacity_;
+    }
+    
+    // optional uint64 dfsUsed = 3;
+    public static final int DFSUSED_FIELD_NUMBER = 3;
+    private long dfsUsed_;
+    public boolean hasDfsUsed() {
+      return ((bitField0_ & 0x00000004) == 0x00000004);
+    }
+    public long getDfsUsed() {
+      return dfsUsed_;
+    }
+    
+    // optional uint64 remaining = 4;
+    public static final int REMAINING_FIELD_NUMBER = 4;
+    private long remaining_;
+    public boolean hasRemaining() {
+      return ((bitField0_ & 0x00000008) == 0x00000008);
+    }
+    public long getRemaining() {
+      return remaining_;
+    }
+    
+    // optional uint64 blockPoolUsed = 5;
+    public static final int BLOCKPOOLUSED_FIELD_NUMBER = 5;
+    private long blockPoolUsed_;
+    public boolean hasBlockPoolUsed() {
+      return ((bitField0_ & 0x00000010) == 0x00000010);
+    }
+    public long getBlockPoolUsed() {
+      return blockPoolUsed_;
+    }
+    
+    // optional uint64 lastUpdate = 6;
+    public static final int LASTUPDATE_FIELD_NUMBER = 6;
+    private long lastUpdate_;
+    public boolean hasLastUpdate() {
+      return ((bitField0_ & 0x00000020) == 0x00000020);
+    }
+    public long getLastUpdate() {
+      return lastUpdate_;
+    }
+    
+    // optional uint32 xceiverCount = 7;
+    public static final int XCEIVERCOUNT_FIELD_NUMBER = 7;
+    private int xceiverCount_;
+    public boolean hasXceiverCount() {
+      return ((bitField0_ & 0x00000040) == 0x00000040);
+    }
+    public int getXceiverCount() {
+      return xceiverCount_;
+    }
+    
+    // optional string location = 8;
+    public static final int LOCATION_FIELD_NUMBER = 8;
+    private java.lang.Object location_;
+    public boolean hasLocation() {
+      return ((bitField0_ & 0x00000080) == 0x00000080);
+    }
+    public String getLocation() {
+      java.lang.Object ref = location_;
+      if (ref instanceof String) {
+        return (String) ref;
+      } else {
+        com.google.protobuf.ByteString bs = 
+            (com.google.protobuf.ByteString) ref;
+        String s = bs.toStringUtf8();
+        if (com.google.protobuf.Internal.isValidUtf8(bs)) {
+          location_ = s;
+        }
+        return s;
+      }
+    }
+    private com.google.protobuf.ByteString getLocationBytes() {
+      java.lang.Object ref = location_;
+      if (ref instanceof String) {
+        com.google.protobuf.ByteString b = 
+            com.google.protobuf.ByteString.copyFromUtf8((String) ref);
+        location_ = b;
+        return b;
+      } else {
+        return (com.google.protobuf.ByteString) ref;
+      }
+    }
+    
+    // optional string hostName = 9;
+    public static final int HOSTNAME_FIELD_NUMBER = 9;
+    private java.lang.Object hostName_;
+    public boolean hasHostName() {
+      return ((bitField0_ & 0x00000100) == 0x00000100);
+    }
+    public String getHostName() {
+      java.lang.Object ref = hostName_;
+      if (ref instanceof String) {
+        return (String) ref;
+      } else {
+        com.google.protobuf.ByteString bs = 
+            (com.google.protobuf.ByteString) ref;
+        String s = bs.toStringUtf8();
+        if (com.google.protobuf.Internal.isValidUtf8(bs)) {
+          hostName_ = s;
+        }
+        return s;
+      }
+    }
+    private com.google.protobuf.ByteString getHostNameBytes() {
+      java.lang.Object ref = hostName_;
+      if (ref instanceof String) {
+        com.google.protobuf.ByteString b = 
+            com.google.protobuf.ByteString.copyFromUtf8((String) ref);
+        hostName_ = b;
+        return b;
+      } else {
+        return (com.google.protobuf.ByteString) ref;
+      }
+    }
+    
+    // optional .DatanodeInfoProto.AdminState adminState = 10;
+    public static final int ADMINSTATE_FIELD_NUMBER = 10;
+    private org.apache.hadoop.hdfs.protocol.proto.HdfsProtos.DatanodeInfoProto.AdminState adminState_;
+    public boolean hasAdminState() {
+      return ((bitField0_ & 0x00000200) == 0x00000200);
+    }
+    public org.apache.hadoop.hdfs.protocol.proto.HdfsProtos.DatanodeInfoProto.AdminState getAdminState() {
+      return adminState_;
+    }
+    
+    private void initFields() {
+      id_ = org.apache.hadoop.hdfs.protocol.proto.HdfsProtos.DatanodeIDProto.getDefaultInstance();
+      capacity_ = 0L;
+      dfsUsed_ = 0L;
+      remaining_ = 0L;
+      blockPoolUsed_ = 0L;
+      lastUpdate_ = 0L;
+      xceiverCount_ = 0;
+      location_ = "";
+      hostName_ = "";
+      adminState_ = org.apache.hadoop.hdfs.protocol.proto.HdfsProtos.DatanodeInfoProto.AdminState.NORMAL;
+    }
+    private byte memoizedIsInitialized = -1;
+    public final boolean isInitialized() {
+      byte isInitialized = memoizedIsInitialized;
+      if (isInitialized != -1) return isInitialized == 1;
+      
+      if (!hasId()) {
+        memoizedIsInitialized = 0;
+        return false;
+      }
+      if (!getId().isInitialized()) {
+        memoizedIsInitialized = 0;
+        return false;
+      }
+      memoizedIsInitialized = 1;
+      return true;
+    }
+    
+    public void writeTo(com.google.protobuf.CodedOutputStream output)
+                        throws java.io.IOException {
+      getSerializedSize();
+      if (((bitField0_ & 0x00000001) == 0x00000001)) {
+        output.writeMessage(1, id_);
+      }
+      if (((bitField0_ & 0x00000002) == 0x00000002)) {
+        output.writeUInt64(2, capacity_);
+      }
+      if (((bitField0_ & 0x00000004) == 0x00000004)) {
+        output.writeUInt64(3, dfsUsed_);
+      }
+      if (((bitField0_ & 0x00000008) == 0x00000008)) {
+        output.writeUInt64(4, remaining_);
+      }
+      if (((bitField0_ & 0x00000010) == 0x00000010)) {
+        output.writeUInt64(5, blockPoolUsed_);
+      }
+      if (((bitField0_ & 0x00000020) == 0x00000020)) {
+        output.writeUInt64(6, lastUpdate_);
+      }
+      if (((bitField0_ & 0x00000040) == 0x00000040)) {
+        output.writeUInt32(7, xceiverCount_);
+      }
+      if (((bitField0_ & 0x00000080) == 0x00000080)) {
+        output.writeBytes(8, getLocationBytes());
+      }
+      if (((bitField0_ & 0x00000100) == 0x00000100)) {
+        output.writeBytes(9, getHostNameBytes());
+      }
+      if (((bitField0_ & 0x00000200) == 0x00000200)) {
+        output.writeEnum(10, adminState_.getNumber());
+      }
+      getUnknownFields().writeTo(output);
+    }
+    
+    private int memoizedSerializedSize = -1;
+    public int getSerializedSize() {
+      int size = memoizedSerializedSize;
+      if (size != -1) return size;
+    
+      size = 0;
+      if (((bitField0_ & 0x00000001) == 0x00000001)) {
+        size += com.google.protobuf.CodedOutputStream
+          .computeMessageSize(1, id_);
+      }
+      if (((bitField0_ & 0x00000002) == 0x00000002)) {
+        size += com.google.protobuf.CodedOutputStream
+          .computeUInt64Size(2, capacity_);
+      }
+      if (((bitField0_ & 0x00000004) == 0x00000004)) {
+        size += com.google.protobuf.CodedOutputStream
+          .computeUInt64Size(3, dfsUsed_);
+      }
+      if (((bitField0_ & 0x00000008) == 0x00000008)) {
+        size += com.google.protobuf.CodedOutputStream
+          .computeUInt64Size(4, remaining_);
+      }
+      if (((bitField0_ & 0x00000010) == 0x00000010)) {
+        size += com.google.protobuf.CodedOutputStream
+          .computeUInt64Size(5, blockPoolUsed_);
+      }
+      if (((bitField0_ & 0x00000020) == 0x00000020)) {
+        size += com.google.protobuf.CodedOutputStream
+          .computeUInt64Size(6, lastUpdate_);
+      }
+      if (((bitField0_ & 0x00000040) == 0x00000040)) {
+        size += com.google.protobuf.CodedOutputStream
+          .computeUInt32Size(7, xceiverCount_);
+      }
+      if (((bitField0_ & 0x00000080) == 0x00000080)) {
+        size += com.google.protobuf.CodedOutputStream
+          .computeBytesSize(8, getLocationBytes());
+      }
+      if (((bitField0_ & 0x00000100) == 0x00000100)) {
+        size += com.google.protobuf.CodedOutputStream
+          .computeBytesSize(9, getHostNameBytes());
+      }
+      if (((bitField0_ & 0x00000200) == 0x00000200)) {
+        size += com.google.protobuf.CodedOutputStream
+          .computeEnumSize(10, adminState_.getNumber());
+      }
+      size += getUnknownFields().getSerializedSize();
+      memoizedSerializedSize = size;
+      return size;
+    }
+    
+    private static final long serialVersionUID = 0L;
+    @java.lang.Override
+    protected java.lang.Object writeReplace()
+        throws java.io.ObjectStreamException {
+      return super.writeReplace();
+    }
+    
+    @java.lang.Override
+    public boolean equals(final java.lang.Object obj) {
+      if (obj == this) {
+       return true;
+      }
+      if (!(obj instanceof org.apache.hadoop.hdfs.protocol.proto.HdfsProtos.DatanodeInfoProto)) {
+        return super.equals(obj);
+      }
+      org.apache.hadoop.hdfs.protocol.proto.HdfsProtos.DatanodeInfoProto other = (org.apache.hadoop.hdfs.protocol.proto.HdfsProtos.DatanodeInfoProto) obj;
+      
+      boolean result = true;
+      result = result && (hasId() == other.hasId());
+      if (hasId()) {
+        result = result && getId()
+            .equals(other.getId());
+      }
+      result = result && (hasCapacity() == other.hasCapacity());
+      if (hasCapacity()) {
+        result = result && (getCapacity()
+            == other.getCapacity());
+      }
+      result = result && (hasDfsUsed() == other.hasDfsUsed());
+      if (hasDfsUsed()) {
+        result = result && (getDfsUsed()
+            == other.getDfsUsed());
+      }
+      result = result && (hasRemaining() == other.hasRemaining());
+      if (hasRemaining()) {
+        result = result && (getRemaining()
+            == other.getRemaining());
+      }
+      result = result && (hasBlockPoolUsed() == other.hasBlockPoolUsed());
+      if (hasBlockPoolUsed()) {
+        result = result && (getBlockPoolUsed()
+            == other.getBlockPoolUsed());
+      }
+      result = result && (hasLastUpdate() == other.hasLastUpdate());
+      if (hasLastUpdate()) {
+        result = result && (getLastUpdate()
+            == other.getLastUpdate());
+      }
+      result = result && (hasXceiverCount() == other.hasXceiverCount());
+      if (hasXceiverCount()) {
+        result = result && (getXceiverCount()
+            == other.getXceiverCount());
+      }
+      result = result && (hasLocation() == other.hasLocation());
+      if (hasLocation()) {
+        result = result && getLocation()
+            .equals(other.getLocation());
+      }
+      result = result && (hasHostName() == other.hasHostName());
+      if (hasHostName()) {
+        result = result && getHostName()
+            .equals(other.getHostName());
+      }
+      result = result && (hasAdminState() == other.hasAdminState());
+      if (hasAdminState()) {
+        result = result &&
+            (getAdminState() == other.getAdminState());
+      }
+      result = result &&
+          getUnknownFields().equals(other.getUnknownFields());
+      return result;
+    }
+    
+    @java.lang.Override
+    public int hashCode() {
+      int hash = 41;
+      hash = (19 * hash) + getDescriptorForType().hashCode();
+      if (hasId()) {
+        hash = (37 * hash) + ID_FIELD_NUMBER;
+        hash = (53 * hash) + getId().hashCode();
+      }
+      if (hasCapacity()) {
+        hash = (37 * hash) + CAPACITY_FIELD_NUMBER;
+        hash = (53 * hash) + hashLong(getCapacity());
+      }
+      if (hasDfsUsed()) {
+        hash = (37 * hash) + DFSUSED_FIELD_NUMBER;
+        hash = (53 * hash) + hashLong(getDfsUsed());
+      }
+      if (hasRemaining()) {
+        hash = (37 * hash) + REMAINING_FIELD_NUMBER;
+        hash = (53 * hash) + hashLong(getRemaining());
+      }
+      if (hasBlockPoolUsed()) {
+        hash = (37 * hash) + BLOCKPOOLUSED_FIELD_NUMBER;
+        hash = (53 * hash) + hashLong(getBlockPoolUsed());
+      }
+      if (hasLastUpdate()) {
+        hash = (37 * hash) + LASTUPDATE_FIELD_NUMBER;
+        hash = (53 * hash) + hashLong(getLastUpdate());
+      }
+      if (hasXceiverCount()) {
+        hash = (37 * hash) + XCEIVERCOUNT_FIELD_NUMBER;
+        hash = (53 * hash) + getXceiverCount();
+      }
+      if (hasLocation()) {
+        hash = (37 * hash) + LOCATION_FIELD_NUMBER;
+        hash = (53 * hash) + getLocation().hashCode();
+      }
+      if (hasHostName()) {
+        hash = (37 * hash) + HOSTNAME_FIELD_NUMBER;
+        hash = (53 * hash) + getHostName().hashCode();
+      }
+      if (hasAdminState()) {
+        hash = (37 * hash) + ADMINSTATE_FIELD_NUMBER;
+        hash = (53 * hash) + hashEnum(getAdminState());
+      }
+      hash = (29 * hash) + getUnknownFields().hashCode();
+      return hash;
+    }
+    
+    public static org.apache.hadoop.hdfs.protocol.proto.HdfsProtos.DatanodeInfoProto parseFrom(
+        com.google.protobuf.ByteString data)
+        throws com.google.protobuf.InvalidProtocolBufferException {
+      return newBuilder().mergeFrom(data).buildParsed();
+    }
+    public static org.apache.hadoop.hdfs.protocol.proto.HdfsProtos.DatanodeInfoProto parseFrom(
+        com.google.protobuf.ByteString data,
+        com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+        throws com.google.protobuf.InvalidProtocolBufferException {
+      return newBuilder().mergeFrom(data, extensionRegistry)
+               .buildParsed();
+    }
+    public static org.apache.hadoop.hdfs.protocol.proto.HdfsProtos.DatanodeInfoProto parseFrom(byte[] data)
+        throws com.google.protobuf.InvalidProtocolBufferException {
+      return newBuilder().mergeFrom(data).buildParsed();
+    }
+    public static org.apache.hadoop.hdfs.protocol.proto.HdfsProtos.DatanodeInfoProto parseFrom(
+        byte[] data,
+        com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+        throws com.google.protobuf.InvalidProtocolBufferException {
+      return newBuilder().mergeFrom(data, extensionRegistry)
+               .buildParsed();
+    }
+    public static org.apache.hadoop.hdfs.protocol.proto.HdfsProtos.DatanodeInfoProto parseFrom(java.io.InputStream input)
+        throws java.io.IOException {
+      return newBuilder().mergeFrom(input).buildParsed();
+    }
+    public static org.apache.hadoop.hdfs.protocol.proto.HdfsProtos.DatanodeInfoProto parseFrom(
+        java.io.InputStream input,
+        com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+        throws java.io.IOException {
+      return newBuilder().mergeFrom(input, extensionRegistry)
+               .buildParsed();
+    }
+    public static org.apache.hadoop.hdfs.protocol.proto.HdfsProtos.DatanodeInfoProto parseDelimitedFrom(java.io.InputStream input)
+        throws java.io.IOException {
+      Builder builder = newBuilder();
+      if (builder.mergeDelimitedFrom(input)) {
+        return builder.buildParsed();
+      } else {
+        return null;
+      }
+    }
+    public static org.apache.hadoop.hdfs.protocol.proto.HdfsProtos.DatanodeInfoProto parseDelimitedFrom(
+        java.io.InputStream input,
+        com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+        throws java.io.IOException {
+      Builder builder = newBuilder();
+      if (builder.mergeDelimitedFrom(input, extensionRegistry)) {
+        return builder.buildParsed();
+      } else {
+        return null;
+      }
+    }
+    public static org.apache.hadoop.hdfs.protocol.proto.HdfsProtos.DatanodeInfoProto parseFrom(
+        com.google.protobuf.CodedInputStream input)
+        throws java.io.IOException {
+      return newBuilder().mergeFrom(input).buildParsed();
+    }
+    public static org.apache.hadoop.hdfs.protocol.proto.HdfsProtos.DatanodeInfoProto parseFrom(
+        com.google.protobuf.CodedInputStream input,
+        com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+        throws java.io.IOException {
+      return newBuilder().mergeFrom(input, extensionRegistry)
+               .buildParsed();
+    }
+    
+    public static Builder newBuilder() { return Builder.create(); }
+    public Builder newBuilderForType() { return newBuilder(); }
+    public static Builder newBuilder(org.apache.hadoop.hdfs.protocol.proto.HdfsProtos.DatanodeInfoProto prototype) {
+      return newBuilder().mergeFrom(prototype);
+    }
+    public Builder toBuilder() { return newBuilder(this); }
+    
+    @java.lang.Override
+    protected Builder newBuilderForType(
+        com.google.protobuf.GeneratedMessage.BuilderParent parent) {
+      Builder builder = new Builder(parent);
+      return builder;
+    }
+    public static final class Builder extends
+        com.google.protobuf.GeneratedMessage.Builder<Builder>
+       implements org.apache.hadoop.hdfs.protocol.proto.HdfsProtos.DatanodeInfoProtoOrBuilder {
+      public static final com.google.protobuf.Descriptors.Descriptor
+          getDescriptor() {
+        return org.apache.hadoop.hdfs.protocol.proto.HdfsProtos.internal_static_DatanodeInfoProto_descriptor;
+      }
+      
+      protected com.google.protobuf.GeneratedMessage.FieldAccessorTable
+          internalGetFieldAccessorTable() {
+        return org.apache.hadoop.hdfs.protocol.proto.HdfsProtos.internal_static_DatanodeInfoProto_fieldAccessorTable;
+      }
+      
+      // Construct using org.apache.hadoop.hdfs.protocol.proto.HdfsProtos.DatanodeInfoProto.newBuilder()
+      private Builder() {
+        maybeForceBuilderInitialization();
+      }
+      
+      private Builder(BuilderParent parent) {
+        super(parent);
+        maybeForceBuilderInitialization();
+      }
+      private void maybeForceBuilderInitialization() {
+        if (com.google.protobuf.GeneratedMessage.alwaysUseFieldBuilders) {
+          getIdFieldBuilder();
+        }
+      }
+      private static Builder create() {
+        return new Builder();
+      }
+      
+      public Builder clear() {
+        super.clear();
+        if (idBuilder_ == null) {
+          id_ = org.apache.hadoop.hdfs.protocol.proto.HdfsProtos.DatanodeIDProto.getDefaultInstance();
+        } else {
+          idBuilder_.clear();
+        }
+        bitField0_ = (bitField0_ & ~0x00000001);
+        capacity_ = 0L;
+        bitField0_ = (bitField0_ & ~0x00000002);
+        dfsUsed_ = 0L;
+        bitField0_ = (bitField0_ & ~0x00000004);
+        remaining_ = 0L;
+        bitField0_ = (bitField0_ & ~0x00000008);
+        blockPoolUsed_ = 0L;
+        bitField0_ = (bitField0_ & ~0x00000010);
+        lastUpdate_ = 0L;
+        bitField0_ = (bitField0_ & ~0x00000020);
+        xceiverCount_ = 0;
+        bitField0_ = (bitField0_ & ~0x00000040);
+        location_ = "";
+        bitField0_ = (bitField0_ & ~0x00000080);
+        hostName_ = "";
+        bitField0_ = (bitField0_ & ~0x00000100);
+        adminState_ = org.apache.hadoop.hdfs.protocol.proto.HdfsProtos.DatanodeInfoProto.AdminState.NORMAL;
+        bitField0_ = (bitField0_ & ~0x00000200);
+        return this;
+      }
+      
+      public Builder clone() {
+        return create().mergeFrom(buildPartial());
+      }
+      
+      public com.google.protobuf.Descriptors.Descriptor
+          getDescriptorForType() {
+        return org.apache.hadoop.hdfs.protocol.proto.HdfsProtos.DatanodeInfoProto.getDescriptor();
+      }
+      
+      public org.apache.hadoop.hdfs.protocol.proto.HdfsProtos.DatanodeInfoProto getDefaultInstanceForType() {
+        return org.apache.hadoop.hdfs.protocol.proto.HdfsProtos.DatanodeInfoProto.getDefaultInstance();
+      }
+      
+      public org.apache.hadoop.hdfs.protocol.proto.HdfsProtos.DatanodeInfoProto build() {
+        org.apache.hadoop.hdfs.protocol.proto.HdfsProtos.DatanodeInfoProto result = buildPartial();
+        if (!result.isInitialized()) {
+          throw newUninitializedMessageException(result);
+        }
+        return result;
+      }
+      
+      private org.apache.hadoop.hdfs.protocol.proto.HdfsProtos.DatanodeInfoProto buildParsed()
+          throws com.google.protobuf.InvalidProtocolBufferException {
+        org.apache.hadoop.hdfs.protocol.proto.HdfsProtos.DatanodeInfoProto result = buildPartial();
+        if (!result.isInitialized()) {
+          throw newUninitializedMessageException(
+            result).asInvalidProtocolBufferException();
+        }
+        return result;
+      }
+      
+      public org.apache.hadoop.hdfs.protocol.proto.HdfsProtos.DatanodeInfoProto buildPartial() {
+        org.apache.hadoop.hdfs.protocol.proto.HdfsProtos.DatanodeInfoProto result = new org.apache.hadoop.hdfs.protocol.proto.HdfsProtos.DatanodeInfoProto(this);
+        int from_bitField0_ = bitField0_;
+        int to_bitField0_ = 0;
+        if (((from_bitField0_ & 0x00000001) == 0x00000001)) {
+          to_bitField0_ |= 0x00000001;
+        }
+        if (idBuilder_ == null) {
+          result.id_ = id_;
+        } else {
+          result.id_ = idBuilder_.build();
+        }
+        if (((from_bitField0_ & 0x00000002) == 0x00000002)) {
+          to_bitField0_ |= 0x00000002;
+        }
+        result.capacity_ = capacity_;
+        if (((from_bitField0_ & 0x00000004) == 0x00000004)) {
+          to_bitField0_ |= 0x00000004;
+        }
+        result.dfsUsed_ = dfsUsed_;
+        if (((from_bitField0_ & 0x00000008) == 0x00000008)) {
+          to_bitField0_ |= 0x00000008;
+        }
+        result.remaining_ = remaining_;
+        if (((from_bitField0_ & 0x00000010) == 0x00000010)) {
+          to_bitField0_ |= 0x00000010;
+        }
+        result.blockPoolUsed_ = blockPoolUsed_;
+        if (((from_bitField0_ & 0x00000020) == 0x00000020)) {
+          to_bitField0_ |= 0x00000020;
+        }
+        result.lastUpdate_ = lastUpdate_;
+        if (((from_bitField0_ & 0x00000040) == 0x00000040)) {
+          to_bitField0_ |= 0x00000040;
+        }
+        result.xceiverCount_ = xceiverCount_;
+        if (((from_bitField0_ & 0x00000080) == 0x00000080)) {
+          to_bitField0_ |= 0x00000080;
+        }
+        result.location_ = location_;
+        if (((from_bitField0_ & 0x00000100) == 0x00000100)) {
+          to_bitField0_ |= 0x00000100;
+        }
+        result.hostName_ = hostName_;
+        if (((from_bitField0_ & 0x00000200) == 0x00000200)) {
+          to_bitField0_ |= 0x00000200;
+        }
+        result.adminState_ = adminState_;
+        result.bitField0_ = to_bitField0_;
+        onBuilt();
+        return result;
+      }
+      
+      public Builder mergeFrom(com.google.protobuf.Message other) {
+        if (other instanceof org.apache.hadoop.hdfs.protocol.proto.HdfsProtos.DatanodeInfoProto) {
+          return mergeFrom((org.apache.hadoop.hdfs.protocol.proto.HdfsProtos.DatanodeInfoProto)other);
+        } else {
+          super.mergeFrom(other);
+          return this;
+        }
+      }
+      
+      public Builder mergeFrom(org.apache.hadoop.hdfs.protocol.proto.HdfsProtos.DatanodeInfoProto other) {
+        if (other == org.apache.hadoop.hdfs.protocol.proto.HdfsProtos.DatanodeInfoProto.getDefaultInstance()) return this;
+        if (other.hasId()) {
+          mergeId(other.getId());
+        }
+        if (other.hasCapacity()) {
+          setCapacity(other.getCapacity());
+        }
+        if (other.hasDfsUsed()) {
+          setDfsUsed(other.getDfsUsed());
+        }
+        if (other.hasRemaining()) {
+          setRemaining(other.getRemaining());
+        }
+        if (other.hasBlockPoolUsed()) {
+          setBlockPoolUsed(other.getBlockPoolUsed());
+        }
+        if (other.hasLastUpdate()) {
+          setLastUpdate(other.getLastUpdate());
+        }
+        if (other.hasXceiverCount()) {
+          setXceiverCount(other.getXceiverCount());
+        }
+        if (other.hasLocation()) {
+          setLocation(other.getLocation());
+        }
+        if (other.hasHostName()) {
+          setHostName(other.getHostName());
+        }
+        if (other.hasAdminState()) {
+          setAdminState(other.getAdminState());
+        }
+        this.mergeUnknownFields(other.getUnknownFields());
+        return this;
+      }
+      
+      public final boolean isInitialized() {
+        if (!hasId()) {
+          
+          return false;
+        }
+        if (!getId().isInitialized()) {
+          
+          return false;
+        }
+        return true;
+      }
+      
+      public Builder mergeFrom(
+          com.google.protobuf.CodedInputStream input,
+          com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+          throws java.io.IOException {
+        com.google.protobuf.UnknownFieldSet.Builder unknownFields =
+          com.google.protobuf.UnknownFieldSet.newBuilder(
+            this.getUnknownFields());
+        while (true) {
+          int tag = input.readTag();
+          switch (tag) {
+            case 0:
+              this.setUnknownFields(unknownFields.build());
+              onChanged();
+              return this;
+            default: {
+              if (!parseUnknownField(input, unknownFields,
+                                     extensionRegistry, tag)) {
+                this.setUnknownFields(unknownFields.build());
+                onChanged();
+                return this;
+              }
+              break;
+            }
+            case 10: {
+              org.apache.hadoop.hdfs.protocol.proto.HdfsProtos.DatanodeIDProto.Builder subBuilder = org.apache.hadoop.hdfs.protocol.proto.HdfsProtos.DatanodeIDProto.newBuilder();
+              if (hasId()) {
+                subBuilder.mergeFrom(getId());
+              }
+              input.readMessage(subBuilder, extensionRegistry);
+              setId(subBuilder.buildPartial());
+              break;
+            }
+            case 16: {
+              bitField0_ |= 0x00000002;
+              capacity_ = input.readUInt64();
+              break;
+            }
+            case 24: {
+              bitField0_ |= 0x00000004;
+              dfsUsed_ = input.readUInt64();
+              break;
+            }
+            case 32: {
+              bitField0_ |= 0x00000008;
+              remaining_ = input.readUInt64();
+              break;
+            }
+            case 40: {
+              bitField0_ |= 0x00000010;
+              blockPoolUsed_ = input.readUInt64();
+              break;
+            }
+            case 48: {
+              bitField0_ |= 0x00000020;
+              lastUpdate_ = input.readUInt64();
+              break;
+            }
+            case 56: {
+              bitField0_ |= 0x00000040;
+              xceiverCount_ = input.readUInt32();
+              break;
+            }
+            case 66: {
+              bitField0_ |= 0x00000080;
+              location_ = input.readBytes();
+              break;
+            }
+            case 74: {
+              bitField0_ |= 0x00000100;
+              hostName_ = input.readBytes();
+              break;
+            }
+            case 80: {
+              int rawValue = input.readEnum();
+              org.apache.hadoop.hdfs.protocol.proto.HdfsProtos.DatanodeInfoProto.AdminState value = org.apache.hadoop.hdfs.protocol.proto.HdfsProtos.DatanodeInfoProto.AdminState.valueOf(rawValue);
+              if (value == null) {
+                unknownFields.mergeVarintField(10, rawValue);
+              } else {
+                bitField0_ |= 0x00000200;
+                adminState_ = value;
+              }
+              break;
+            }
+          }
+        }
+      }
+      
+      private int bitField0_;
+      
+      // required .DatanodeIDProto id = 1;
+      private org.apache.hadoop.hdfs.protocol.proto.HdfsProtos.DatanodeIDProto id_ = org.apache.hadoop.hdfs.protocol.proto.HdfsProtos.DatanodeIDProto.getDefaultInstance();
+      private com.google.protobuf.SingleFieldBuilder<
+          org.apache.hadoop.hdfs.protocol.proto.HdfsProtos.DatanodeIDProto, org.apache.hadoop.hdfs.protocol.proto.HdfsProtos.DatanodeIDProto.Builder, org.apache.hadoop.hdfs.protocol.proto.HdfsProtos.DatanodeIDProtoOrBuilder> idBuilder_;
+      public boolean hasId() {
+        return ((bitField0_ & 0x00000001) == 0x00000001);
+      }
+      public org.apache.hadoop.hdfs.protocol.proto.HdfsProtos.DatanodeIDProto getId() {
+        if (idBuilder_ == null) {
+          return id_;
+        } else {
+          return idBuilder_.getMessage();
+        }
+      }
+      public Builder setId(org.apache.hadoop.hdfs.protocol.proto.HdfsProtos.DatanodeIDProto value) {
+        if (idBuilder_ == null) {
+          if (value == null) {
+            throw new NullPointerException();
+          }
+          id_ = value;
+          onChanged();
+        } else {
+          idBuilder_.setMessage(value);
+        }
+        bitField0_ |= 0x00000001;
+        return this;
+      }
+      public Builder setId(
+          org.apache.hadoop.hdfs.protocol.proto.HdfsProtos.DatanodeIDProto.Builder builderForValue) {
+        if (idBuilder_ == null) {
+          id_ = builderForValue.build();
+          onChanged();
+        } else {
+          idBuilder_.setMessage(builderForValue.build());
+        }
+        bitField0_ |= 0x00000001;
+        return this;
+      }
+      public Builder mergeId(org.apache.hadoop.hdfs.protocol.proto.HdfsProtos.DatanodeIDProto value) {
+        if (idBuilder_ == null) {
+          if (((bitField0_ & 0x00000001) == 0x00000001) &&
+              id_ != org.apache.hadoop.hdfs.protocol.proto.HdfsProtos.DatanodeIDProto.getDefaultInstance()) {
+            id_ =
+              org.apache.hadoop.hdfs.protocol.proto.HdfsProtos.DatanodeIDProto.newBuilder(id_).mergeFrom(value).buildPartial();
+          } else {
+            id_ = value;
+          }
+          onChanged();
+        } else {
+          idBuilder_.mergeFrom(value);
+        }
+        bitField0_ |= 0x00000001;
+        return this;
+      }
+      public Builder clearId() {
+        if (idBuilder_ == null) {
+          id_ = org.apache.hadoop.hdfs.protocol.proto.HdfsProtos.DatanodeIDProto.getDefaultInstance();
+          onChanged();
+        } else {
+          idBuilder_.clear();
+        }
+        bitField0_ = (bitField0_ & ~0x00000001);
+        return this;
+      }
+      public org.apache.hadoop.hdfs.protocol.proto.HdfsProtos.DatanodeIDProto.Builder getIdBuilder() {
+        bitField0_ |= 0x00000001;
+        onChanged();
+        return getIdFieldBuilder().getBuilder();
+      }
+      public org.apache.hadoop.hdfs.protocol.proto.HdfsProtos.DatanodeIDProtoOrBuilder getIdOrBuilder() {
+        if (idBuilder_ != null) {
+          return idBuilder_.getMessageOrBuilder();
+        } else {
+          return id_;
+        }
+      }
+      private com.google.protobuf.SingleFieldBuilder<
+          org.apache.hadoop.hdfs.protocol.proto.HdfsProtos.DatanodeIDProto, org.apache.hadoop.hdfs.protocol.proto.HdfsProtos.DatanodeIDProto.Builder, org.apache.hadoop.hdfs.protocol.proto.HdfsProtos.DatanodeIDProtoOrBuilder> 
+          getIdFieldBuilder() {
+        if (idBuilder_ == null) {
+          idBuilder_ = new com.google.protobuf.SingleFieldBuilder<
+              org.apache.hadoop.hdfs.protocol.proto.HdfsProtos.DatanodeIDProto, org.apache.hadoop.hdfs.protocol.proto.HdfsProtos.DatanodeIDProto.Builder, org.apache.hadoop.hdfs.protocol.proto.HdfsProtos.DatanodeIDProtoOrBuilder>(
+                  id_,
+                  getParentForChildren(),
+                  isClean());
+          id_ = null;
+        }
+        return idBuilder_;
+      }
+      
+      // optional uint64 capacity = 2;
+      private long capacity_ ;
+      public boolean hasCapacity() {
+        return ((bitField0_ & 0x00000002) == 0x00000002);
+      }
+      public long getCapacity() {
+        return capacity_;
+      }
+      public Builder setCapacity(long value) {
+        bitField0_ |= 0x00000002;
+        capacity_ = value;
+        onChanged();
+        return this;
+      }
+      public Builder clearCapacity() {
+        bitField0_ = (bitField0_ & ~0x00000002);
+        capacity_ = 0L;
+        onChanged();
+        return this;
+      }
+      
+      // optional uint64 dfsUsed = 3;
+      private long dfsUsed_ ;
+      public boolean hasDfsUsed() {
+        return ((bitField0_ & 0x00000004) == 0x00000004);
+      }
+      public long getDfsUsed() {
+        return dfsUsed_;
+      }
+      public Builder setDfsUsed(long value) {
+        bitField0_ |= 0x00000004;
+        dfsUsed_ = value;
+        onChanged();
+        return this;
+      }
+      public Builder clearDfsUsed() {
+        bitField0_ = (bitField0_ & ~0x00000004);
+        dfsUsed_ = 0L;
+        onChanged();
+        return this;
+      }
+      
+      // optional uint64 remaining = 4;
+      private long remaining_ ;
+      public boolean hasRemaining() {
+        return ((bitField0_ & 0x00000008) == 0x00000008);
+      }
+      public long getRemaining() {
+        return remaining_;
+      }
+      public Builder setRemaining(long value) {
+        bitField0_ |= 0x00000008;
+        remaining_ = value;
+        onChanged();
+        return this;
+      }
+      public Builder clearRemaining() {
+        bitField0_ = (bitField0_ & ~0x00000008);
+        remaining_ = 0L;
+        onChanged();
+        return this;
+      }
+      
+      // optional uint64 blockPoolUsed = 5;
+      private long blockPoolUsed_ ;
+      public boolean hasBlockPoolUsed() {
+        return ((bitField0_ & 0x00000010) == 0x00000010);
+      }
+      public long getBlockPoolUsed() {
+        return blockPoolUsed_;
+      }
+      public Builder setBlockPoolUsed(long value) {
+        bitField0_ |= 0x00000010;
+        blockPoolUsed_ = value;
+        onChanged();
+        return this;
+      }
+      public Builder clearBlockPoolUsed() {
+        bitField0_ = (bitField0_ & ~0x00000010);
+        blockPoolUsed_ = 0L;
+        onChanged();
+        return this;
+      }
+      
+      // optional uint64 lastUpdate = 6;
+      private long lastUpdate_ ;
+      public boolean hasLastUpdate() {
+        return ((bitField0_ & 0x00000020) == 0x00000020);
+      }
+      public long getLastUpdate() {
+        return lastUpdate_;
+      }
+      public Builder setLastUpdate(long value) {
+        bitField0_ |= 0x00000020;
+        lastUpdate_ = value;
+        onChanged();
+        return this;
+      }
+      public Builder clearLastUpdate() {
+        bitField0_ = (bitField0_ & ~0x00000020);
+        lastUpdate_ = 0L;
+        onChanged();
+        return this;
+      }
+      
+      // optional uint32 xceiverCount = 7;
+      private int xceiverCount_ ;
+      public boolean hasXceiverCount() {
+        return ((bitField0_ & 0x00000040) == 0x00000040);
+      }
+      public int getXceiverCount() {
+        return xceiverCount_;
+      }
+      public Builder setXceiverCount(int value) {
+        bitField0_ |= 0x00000040;
+        xceiverCount_ = value;
+        onChanged();
+        return this;
+      }
+      public Builder clearXceiverCount() {
+        bitField0_ = (bitField0_ & ~0x00000040);
+        xceiverCount_ = 0;
+        onChanged();
+        return this;
+      }
+      
+      // optional string location = 8;
+      private java.lang.Object location_ = "";
+      public boolean hasLocation() {
+        return ((bitField0_ & 0x00000080) == 0x00000080);
+      }
+      public String getLocation() {
+        java.lang.Object ref = location_;
+        if (!(ref instanceof String)) {
+          String s = ((com.google.protobuf.ByteString) ref).toStringUtf8();
+          location_ = s;
+          return s;
+        } else {
+          return (String) ref;
+        }
+      }
+      public Builder setLocation(String value) {
+        if (value == null) {
+    throw new NullPointerException();
+  }
+  bitField0_ |= 0x00000080;
+        location_ = value;
+        onChanged();
+        return this;
+      }
+      public Builder clearLocation() {
+        bitField0_ = (bitField0_ & ~0x00000080);
+        location_ = getDefaultInstance().getLocation();
+        onChanged();
+        return this;
+      }
+      void setLocation(com.google.protobuf.ByteString value) {
+        bitField0_ |= 0x00000080;
+        location_ = value;
+        onChanged();
+      }
+      
+      // optional string hostName = 9;
+      private java.lang.Object hostName_ = "";
+      public boolean hasHostName() {
+        return ((bitField0_ & 0x00000100) == 0x00000100);
+      }
+      public String getHostName() {
+        java.lang.Object ref = hostName_;
+        if (!(ref instanceof String)) {
+          String s = ((com.google.protobuf.ByteString) ref).toStringUtf8();
+          hostName_ = s;
+          return s;
+        } else {
+          return (String) ref;
+        }
+      }
+      public Builder setHostName(String value) {
+        if (value == null) {
+    throw new NullPointerException();
+  }
+  bitField0_ |= 0x00000100;
+        hostName_ = value;
+        onChanged();
+        return this;
+      }
+      public Builder clearHostName() {
+        bitField0_ = (bitField0_ & ~0x00000100);
+        hostName_ = getDefaultInstance().getHostName();
+        onChanged();
+        return this;
+      }
+      void setHostName(com.google.protobuf.ByteString value) {
+        bitField0_ |= 0x00000100;
+        hostName_ = value;
+        onChanged();
+      }
+      
+      // optional .DatanodeInfoProto.AdminState adminState = 10;
+      private org.apache.hadoop.hdfs.protocol.proto.HdfsProtos.DatanodeInfoProto.AdminState adminState_ = org.apache.hadoop.hdfs.protocol.proto.HdfsProtos.DatanodeInfoProto.AdminState.NORMAL;
+      public boolean hasAdminState() {
+        return ((bitField0_ & 0x00000200) == 0x00000200);
+      }
+      public org.apache.hadoop.hdfs.protocol.proto.HdfsProtos.DatanodeInfoProto.AdminState getAdminState() {
+        return adminState_;
+      }
+      public Builder setAdminState(org.apache.hadoop.hdfs.protocol.proto.HdfsProtos.DatanodeInfoProto.AdminState value) {
+        if (value == null) {
+          throw new NullPointerException();
+        }
+        bitField0_ |= 0x00000200;
+        adminState_ = value;
+        onChanged();
+        return this;
+      }
+      public Builder clearAdminState() {
+        bitField0_ = (bitField0_ & ~0x00000200);
+        adminState_ = org.apache.hadoop.hdfs.protocol.proto.HdfsProtos.DatanodeInfoProto.AdminState.NORMAL;
+        onChanged();
+        return this;
+      }
+      
+      // @@protoc_insertion_point(builder_scope:DatanodeInfoProto)
+    }
+    
+    static {
+      defaultInstance = new DatanodeInfoProto(true);
+      defaultInstance.initFields();
+    }
+    
+    // @@protoc_insertion_point(class_scope:DatanodeInfoProto)
+  }
+  
+  private static com.google.protobuf.Descriptors.Descriptor
+    internal_static_ExtendedBlockProto_descriptor;
+  private static
+    com.google.protobuf.GeneratedMessage.FieldAccessorTable
+      internal_static_ExtendedBlockProto_fieldAccessorTable;
+  private static com.google.protobuf.Descriptors.Descriptor
+    internal_static_BlockTokenIdentifierProto_descriptor;
+  private static
+    com.google.protobuf.GeneratedMessage.FieldAccessorTable
+      internal_static_BlockTokenIdentifierProto_fieldAccessorTable;
+  private static com.google.protobuf.Descriptors.Descriptor
+    internal_static_DatanodeIDProto_descriptor;
+  private static
+    com.google.protobuf.GeneratedMessage.FieldAccessorTable
+      internal_static_DatanodeIDProto_fieldAccessorTable;
+  private static com.google.protobuf.Descriptors.Descriptor
+    internal_static_DatanodeInfoProto_descriptor;
+  private static
+    com.google.protobuf.GeneratedMessage.FieldAccessorTable
+      internal_static_DatanodeInfoProto_fieldAccessorTable;
+  
+  public static com.google.protobuf.Descriptors.FileDescriptor
+      getDescriptor() {
+    return descriptor;
+  }
+  private static com.google.protobuf.Descriptors.FileDescriptor
+      descriptor;
+  static {
+    java.lang.String[] descriptorData = {
+      "\n\nhdfs.proto\"`\n\022ExtendedBlockProto\022\016\n\006po" +
+      "olId\030\001 \002(\t\022\017\n\007blockId\030\002 \002(\004\022\020\n\010numBytes\030" +
+      "\003 \002(\004\022\027\n\017generationStamp\030\004 \002(\004\"`\n\031BlockT" +
+      "okenIdentifierProto\022\022\n\nidentifier\030\001 \002(\014\022" +
+      "\020\n\010password\030\002 \002(\014\022\014\n\004kind\030\003 \002(\t\022\017\n\007servi" +
+      "ce\030\004 \002(\t\"D\n\017DatanodeIDProto\022\014\n\004name\030\001 \002(" +
+      "\t\022\021\n\tstorageID\030\002 \002(\t\022\020\n\010infoPort\030\003 \002(\r\"\312" +
+      "\002\n\021DatanodeInfoProto\022\034\n\002id\030\001 \002(\0132\020.Datan" +
+      "odeIDProto\022\020\n\010capacity\030\002 \001(\004\022\017\n\007dfsUsed\030" +
+      "\003 \001(\004\022\021\n\tremaining\030\004 \001(\004\022\025\n\rblockPoolUse",
+      "d\030\005 \001(\004\022\022\n\nlastUpdate\030\006 \001(\004\022\024\n\014xceiverCo" +
+      "unt\030\007 \001(\r\022\020\n\010location\030\010 \001(\t\022\020\n\010hostName\030" +
+      "\t \001(\t\0221\n\nadminState\030\n \001(\0162\035.DatanodeInfo" +
+      "Proto.AdminState\"I\n\nAdminState\022\n\n\006NORMAL" +
+      "\020\000\022\033\n\027DECOMMISSION_INPROGRESS\020\001\022\022\n\016DECOM" +
+      "MISSIONED\020\002B6\n%org.apache.hadoop.hdfs.pr" +
+      "otocol.protoB\nHdfsProtos\240\001\001"
+    };
+    com.google.protobuf.Descriptors.FileDescriptor.InternalDescriptorAssigner assigner =
+      new com.google.protobuf.Descriptors.FileDescriptor.InternalDescriptorAssigner() {
+        public com.google.protobuf.ExtensionRegistry assignDescriptors(
+            com.google.protobuf.Descriptors.FileDescriptor root) {
+          descriptor = root;
+          internal_static_ExtendedBlockProto_descriptor =
+            getDescriptor().getMessageTypes().get(0);
+          internal_static_ExtendedBlockProto_fieldAccessorTable = new
+            com.google.protobuf.GeneratedMessage.FieldAccessorTable(
+              internal_static_ExtendedBlockProto_descriptor,
+              new java.lang.String[] { "PoolId", "BlockId", "NumBytes", "GenerationStamp", },
+              org.apache.hadoop.hdfs.protocol.proto.HdfsProtos.ExtendedBlockProto.class,
+              org.apache.hadoop.hdfs.protocol.proto.HdfsProtos.ExtendedBlockProto.Builder.class);
+          internal_static_BlockTokenIdentifierProto_descriptor =
+            getDescriptor().getMessageTypes().get(1);
+          internal_static_BlockTokenIdentifierProto_fieldAccessorTable = new
+            com.google.protobuf.GeneratedMessage.FieldAccessorTable(
+              internal_static_BlockTokenIdentifierProto_descriptor,
+              new java.lang.String[] { "Identifier", "Password", "Kind", "Service", },
+              org.apache.hadoop.hdfs.protocol.proto.HdfsProtos.BlockTokenIdentifierProto.class,
+              org.apache.hadoop.hdfs.protocol.proto.HdfsProtos.BlockTokenIdentifierProto.Builder.class);
+          internal_static_DatanodeIDProto_descriptor =
+            getDescriptor().getMessageTypes().get(2);
+          internal_static_DatanodeIDProto_fieldAccessorTable = new
+            com.google.protobuf.GeneratedMessage.FieldAccessorTable(
+              internal_static_DatanodeIDProto_descriptor,
+              new java.lang.String[] { "Name", "StorageID", "InfoPort", },
+              org.apache.hadoop.hdfs.protocol.proto.HdfsProtos.DatanodeIDProto.class,
+              org.apache.hadoop.hdfs.protocol.proto.HdfsProtos.DatanodeIDProto.Builder.class);
+          internal_static_DatanodeInfoProto_descriptor =
+            getDescriptor().getMessageTypes().get(3);
+          internal_static_DatanodeInfoProto_fieldAccessorTable = new
+            com.google.protobuf.GeneratedMessage.FieldAccessorTable(
+              internal_static_DatanodeInfoProto_descriptor,
+              new java.lang.String[] { "Id", "Capacity", "DfsUsed", "Remaining", "BlockPoolUsed", "LastUpdate", "XceiverCount", "Location", "HostName", "AdminState", },
+              org.apache.hadoop.hdfs.protocol.proto.HdfsProtos.DatanodeInfoProto.class,
+              org.apache.hadoop.hdfs.protocol.proto.HdfsProtos.DatanodeInfoProto.Builder.class);
+          return null;
+        }
+      };
+    com.google.protobuf.Descriptors.FileDescriptor
+      .internalBuildGeneratedFileFrom(descriptorData,
+        new com.google.protobuf.Descriptors.FileDescriptor[] {
+        }, assigner);
+  }
+  
+  // @@protoc_insertion_point(outer_class_scope)
+}
diff --git a/src/java/org/apache/hadoop/hdfs/server/balancer/Balancer.java b/src/java/org/apache/hadoop/hdfs/server/balancer/Balancer.java
index 9081a19..efaa80d 100644
--- a/src/java/org/apache/hadoop/hdfs/server/balancer/Balancer.java
+++ b/src/java/org/apache/hadoop/hdfs/server/balancer/Balancer.java
@@ -17,8 +17,6 @@
  */
 package org.apache.hadoop.hdfs.server.balancer;
 
-import static org.apache.hadoop.hdfs.protocol.DataTransferProtocol.Status.ERROR_ACCESS_TOKEN;
-
 import java.io.BufferedInputStream;
 import java.io.BufferedOutputStream;
 import java.io.DataInputStream;
@@ -58,6 +56,9 @@
 import org.apache.hadoop.hdfs.protocol.ExtendedBlock;
 import org.apache.hadoop.hdfs.protocol.FSConstants;
 import org.apache.hadoop.hdfs.protocol.FSConstants.DatanodeReportType;
+import org.apache.hadoop.hdfs.protocol.HdfsProtoUtil;
+import org.apache.hadoop.hdfs.protocol.proto.DataTransferProtos.BlockOpResponseProto;
+import org.apache.hadoop.hdfs.protocol.proto.DataTransferProtos.Status;
 import org.apache.hadoop.hdfs.security.token.block.BlockTokenIdentifier;
 import org.apache.hadoop.hdfs.server.common.HdfsConstants;
 import org.apache.hadoop.hdfs.server.common.Util;
@@ -73,6 +74,8 @@
 import org.apache.hadoop.util.Tool;
 import org.apache.hadoop.util.ToolRunner;
 
+import static org.apache.hadoop.hdfs.protocol.HdfsProtoUtil.vintPrefixed;
+
 /** <p>The balancer is a tool that balances disk space usage on an HDFS cluster
  * when some datanodes become full or when new empty nodes join the cluster.
  * The tool is deployed as an application program that can be run by the 
@@ -352,9 +355,10 @@
     
     /* Receive a block copy response from the input stream */ 
     private void receiveResponse(DataInputStream in) throws IOException {
-      DataTransferProtocol.Status status = DataTransferProtocol.Status.read(in);
-      if (status != DataTransferProtocol.Status.SUCCESS) {
-        if (status == ERROR_ACCESS_TOKEN)
+      BlockOpResponseProto response = BlockOpResponseProto.parseFrom(
+          vintPrefixed(in));
+      if (response.getStatus() != Status.SUCCESS) {
+        if (response.getStatus() == Status.ERROR_ACCESS_TOKEN)
           throw new IOException("block move failed due to access token error");
         throw new IOException("block move is failed");
       }
diff --git a/src/java/org/apache/hadoop/hdfs/server/datanode/BlockReceiver.java b/src/java/org/apache/hadoop/hdfs/server/datanode/BlockReceiver.java
index 0ae3ef6..0956cc5 100644
--- a/src/java/org/apache/hadoop/hdfs/server/datanode/BlockReceiver.java
+++ b/src/java/org/apache/hadoop/hdfs/server/datanode/BlockReceiver.java
@@ -17,8 +17,6 @@
  */
 package org.apache.hadoop.hdfs.server.datanode;
 
-import static org.apache.hadoop.hdfs.protocol.DataTransferProtocol.Status.ERROR;
-import static org.apache.hadoop.hdfs.protocol.DataTransferProtocol.Status.SUCCESS;
 import static org.apache.hadoop.hdfs.server.datanode.DataNode.DN_CLIENTTRACE_FORMAT;
 
 import java.io.BufferedOutputStream;
@@ -39,11 +37,11 @@
 import org.apache.hadoop.hdfs.protocol.DataTransferProtocol.BlockConstructionStage;
 import org.apache.hadoop.hdfs.protocol.DataTransferProtocol.PacketHeader;
 import org.apache.hadoop.hdfs.protocol.DataTransferProtocol.PipelineAck;
-import org.apache.hadoop.hdfs.protocol.DataTransferProtocol.Status;
 import org.apache.hadoop.hdfs.protocol.DatanodeInfo;
 import org.apache.hadoop.hdfs.protocol.ExtendedBlock;
 import org.apache.hadoop.hdfs.protocol.FSConstants;
 import org.apache.hadoop.hdfs.protocol.LocatedBlock;
+import org.apache.hadoop.hdfs.protocol.proto.DataTransferProtos.Status;
 import org.apache.hadoop.hdfs.server.protocol.DatanodeProtocol;
 import org.apache.hadoop.hdfs.server.protocol.DatanodeRegistration;
 import org.apache.hadoop.hdfs.util.DataTransferThrottler;
@@ -996,13 +994,13 @@
             Status[] replies = null;
             if (mirrorError) { // ack read error
               replies = new Status[2];
-              replies[0] = SUCCESS;
-              replies[1] = ERROR;
+              replies[0] = Status.SUCCESS;
+              replies[1] = Status.ERROR;
             } else {
               short ackLen = type == PacketResponderType.LAST_IN_PIPELINE? 0
                   : ack.getNumOfReplies();
               replies = new Status[1+ackLen];
-              replies[0] = SUCCESS;
+              replies[0] = Status.SUCCESS;
               for (int i=0; i<ackLen; i++) {
                 replies[i+1] = ack.getReply(i);
               }
diff --git a/src/java/org/apache/hadoop/hdfs/server/datanode/DataNode.java b/src/java/org/apache/hadoop/hdfs/server/datanode/DataNode.java
index 51f0f16..09a87d4 100644
--- a/src/java/org/apache/hadoop/hdfs/server/datanode/DataNode.java
+++ b/src/java/org/apache/hadoop/hdfs/server/datanode/DataNode.java
@@ -19,8 +19,6 @@
 
 
 import static org.apache.hadoop.hdfs.DFSConfigKeys.*;
-import static org.apache.hadoop.hdfs.protocol.DataTransferProtocol.Status.ERROR_ACCESS_TOKEN;
-import static org.apache.hadoop.hdfs.protocol.DataTransferProtocol.Status.SUCCESS;
 import static org.apache.hadoop.hdfs.server.common.Util.now;
 
 import java.io.BufferedOutputStream;
@@ -75,8 +73,11 @@
 import org.apache.hadoop.hdfs.protocol.ExtendedBlock;
 import org.apache.hadoop.hdfs.protocol.FSConstants;
 import org.apache.hadoop.hdfs.protocol.LocatedBlock;
+import org.apache.hadoop.hdfs.protocol.HdfsProtoUtil;
 import org.apache.hadoop.hdfs.protocol.RecoveryInProgressException;
 import org.apache.hadoop.hdfs.protocol.UnregisteredNodeException;
+import org.apache.hadoop.hdfs.protocol.proto.DataTransferProtos.DNTransferAckProto;
+import org.apache.hadoop.hdfs.protocol.proto.DataTransferProtos.Status;
 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;
@@ -133,6 +134,7 @@
 import org.apache.hadoop.util.VersionInfo;
 import org.mortbay.util.ajax.JSON;
 
+
 /**********************************************************
  * DataNode is a class (and program) that stores a set of
  * blocks for a DFS deployment.  A single deployment can
@@ -1958,12 +1960,13 @@
         // read ack
         if (isClient) {
           in = new DataInputStream(NetUtils.getInputStream(sock));
-          final DataTransferProtocol.Status s = DataTransferProtocol.Status.read(in);
+          DNTransferAckProto closeAck = DNTransferAckProto.parseFrom(
+              HdfsProtoUtil.vintPrefixed(in));
           if (LOG.isDebugEnabled()) {
-            LOG.debug(getClass().getSimpleName() + ": close-ack=" + s);
+            LOG.debug(getClass().getSimpleName() + ": close-ack=" + closeAck);
           }
-          if (s != SUCCESS) {
-            if (s == ERROR_ACCESS_TOKEN) {
+          if (closeAck.getStatus() != Status.SUCCESS) {
+            if (closeAck.getStatus() == Status.ERROR_ACCESS_TOKEN) {
               throw new InvalidBlockTokenException(
                   "Got access token error for connect ack, targets="
                    + Arrays.asList(targets));
diff --git a/src/java/org/apache/hadoop/hdfs/server/datanode/DataXceiver.java b/src/java/org/apache/hadoop/hdfs/server/datanode/DataXceiver.java
index 6ba7806..8b5fc08 100644
--- a/src/java/org/apache/hadoop/hdfs/server/datanode/DataXceiver.java
+++ b/src/java/org/apache/hadoop/hdfs/server/datanode/DataXceiver.java
@@ -17,9 +17,8 @@
  */
 package org.apache.hadoop.hdfs.server.datanode;
 
-import static org.apache.hadoop.hdfs.protocol.DataTransferProtocol.Status.ERROR;
-import static org.apache.hadoop.hdfs.protocol.DataTransferProtocol.Status.ERROR_ACCESS_TOKEN;
-import static org.apache.hadoop.hdfs.protocol.DataTransferProtocol.Status.SUCCESS;
+import static org.apache.hadoop.hdfs.protocol.proto.DataTransferProtos.Status.*;
+import static org.apache.hadoop.hdfs.protocol.proto.DataTransferProtos.Status;
 import static org.apache.hadoop.hdfs.server.common.Util.now;
 import static org.apache.hadoop.hdfs.server.datanode.DataNode.DN_CLIENTTRACE_FORMAT;
 
@@ -44,6 +43,11 @@
 import org.apache.hadoop.hdfs.protocol.DatanodeInfo;
 import org.apache.hadoop.hdfs.protocol.ExtendedBlock;
 import org.apache.hadoop.hdfs.protocol.FSConstants;
+import org.apache.hadoop.hdfs.protocol.HdfsProtoUtil;
+import org.apache.hadoop.hdfs.protocol.proto.DataTransferProtos.BlockOpResponseProto;
+import org.apache.hadoop.hdfs.protocol.proto.DataTransferProtos.ClientReadStatusProto;
+import org.apache.hadoop.hdfs.protocol.proto.DataTransferProtos.OpBlockChecksumResponseProto;
+
 import org.apache.hadoop.hdfs.security.token.block.BlockTokenIdentifier;
 import org.apache.hadoop.hdfs.security.token.block.BlockTokenSecretManager;
 import org.apache.hadoop.hdfs.server.common.HdfsConstants;
@@ -51,7 +55,6 @@
 import org.apache.hadoop.hdfs.server.protocol.DatanodeRegistration;
 import org.apache.hadoop.io.IOUtils;
 import org.apache.hadoop.io.MD5Hash;
-import org.apache.hadoop.io.Text;
 import org.apache.hadoop.metrics2.lib.MutableCounterLong;
 import org.apache.hadoop.metrics2.lib.MutableRate;
 import org.apache.hadoop.net.NetUtils;
@@ -60,6 +63,9 @@
 import org.apache.hadoop.util.DataChecksum;
 import org.apache.hadoop.util.StringUtils;
 
+import com.google.protobuf.ByteString;
+
+
 /**
  * Thread for processing incoming/outgoing data stream.
  */
@@ -224,16 +230,19 @@
         sendResponse(s, ERROR, datanode.socketWriteTimeout);
         throw e;
       }
+      
+      // send op status
+      sendResponse(s, SUCCESS, datanode.socketWriteTimeout);
 
-      SUCCESS.write(out); // send op status
       long read = blockSender.sendBlock(out, baseStream, null); // send data
 
       if (blockSender.didSendEntireByteRange()) {
         // If we sent the entire range, then we should expect the client
         // to respond with a Status enum.
         try {
-          DataTransferProtocol.Status stat = DataTransferProtocol.Status.read(in);
-          if (stat == null) {
+          ClientReadStatusProto stat = ClientReadStatusProto.parseFrom(
+              HdfsProtoUtil.vintPrefixed(in));
+          if (!stat.hasStatus()) {
             LOG.warn("Client " + s.getInetAddress() + " did not send a valid status " +
                      "code after reading. Will close connection.");
             IOUtils.closeStream(out);
@@ -329,7 +338,7 @@
     BlockReceiver blockReceiver = null; // responsible for data handling
     String mirrorNode = null;           // the name:port of next target
     String firstBadLink = "";           // first datanode that failed in connection setup
-    DataTransferProtocol.Status mirrorInStatus = SUCCESS;
+    Status mirrorInStatus = SUCCESS;
     try {
       if (isDatanode || 
           stage != BlockConstructionStage.PIPELINE_CLOSE_RECOVERY) {
@@ -377,8 +386,10 @@
 
           // read connect ack (only for clients, not for replication req)
           if (isClient) {
-            mirrorInStatus = DataTransferProtocol.Status.read(mirrorIn);
-            firstBadLink = Text.readString(mirrorIn);
+            BlockOpResponseProto connectAck =
+              BlockOpResponseProto.parseFrom(HdfsProtoUtil.vintPrefixed(mirrorIn));
+            mirrorInStatus = connectAck.getStatus();
+            firstBadLink = connectAck.getFirstBadLink();
             if (LOG.isDebugEnabled() || mirrorInStatus != SUCCESS) {
               LOG.info("Datanode " + targets.length +
                        " got response for connect ack " +
@@ -389,8 +400,11 @@
 
         } catch (IOException e) {
           if (isClient) {
-            ERROR.write(replyOut);
-            Text.writeString(replyOut, mirrorNode);
+            BlockOpResponseProto.newBuilder()
+              .setStatus(ERROR)
+              .setFirstBadLink(mirrorNode)
+              .build()
+              .writeDelimitedTo(replyOut);
             replyOut.flush();
           }
           IOUtils.closeStream(mirrorOut);
@@ -417,8 +431,11 @@
                    " forwarding connect ack to upstream firstbadlink is " +
                    firstBadLink);
         }
-        mirrorInStatus.write(replyOut);
-        Text.writeString(replyOut, firstBadLink);
+        BlockOpResponseProto.newBuilder()
+          .setStatus(mirrorInStatus)
+          .setFirstBadLink(firstBadLink)
+          .build()
+          .writeDelimitedTo(replyOut);
         replyOut.flush();
       }
 
@@ -433,7 +450,7 @@
           if (LOG.isTraceEnabled()) {
             LOG.trace("TRANSFER: send close-ack");
           }
-          SUCCESS.write(replyOut);
+          writeResponse(SUCCESS, replyOut);
         }
       }
 
@@ -489,12 +506,12 @@
         NetUtils.getOutputStream(s, datanode.socketWriteTimeout));
     try {
       datanode.transferReplicaForPipelineRecovery(blk, targets, client);
-      SUCCESS.write(out);
+      writeResponse(Status.SUCCESS, out);
     } finally {
       IOUtils.closeStream(out);
     }
   }
-
+  
   /**
    * Get block checksum (MD5 of CRC32).
    */
@@ -530,10 +547,15 @@
       }
 
       //write reply
-      SUCCESS.write(out);
-      out.writeInt(bytesPerCRC);
-      out.writeLong(crcPerBlock);
-      md5.write(out);
+      BlockOpResponseProto.newBuilder()
+        .setStatus(SUCCESS)
+        .setChecksumResponse(OpBlockChecksumResponseProto.newBuilder()             
+          .setBytesPerCrc(bytesPerCRC)
+          .setCrcPerBlock(crcPerBlock)
+          .setMd5(ByteString.copyFrom(md5.getDigest()))
+          )
+        .build()
+        .writeDelimitedTo(out);
       out.flush();
     } finally {
       IOUtils.closeStream(out);
@@ -590,7 +612,7 @@
           baseStream, SMALL_BUFFER_SIZE));
 
       // send status first
-      SUCCESS.write(reply);
+      writeResponse(SUCCESS, reply);
       // send block content to the target
       long read = blockSender.sendBlock(reply, baseStream, 
                                         dataXceiverServer.balanceThrottler);
@@ -653,7 +675,7 @@
 
     Socket proxySock = null;
     DataOutputStream proxyOut = null;
-    DataTransferProtocol.Status opStatus = SUCCESS;
+    Status opStatus = SUCCESS;
     BlockReceiver blockReceiver = null;
     DataInputStream proxyReply = null;
     
@@ -676,10 +698,11 @@
       // receive the response from the proxy
       proxyReply = new DataInputStream(new BufferedInputStream(
           NetUtils.getInputStream(proxySock), BUFFER_SIZE));
-      final DataTransferProtocol.Status status
-          = DataTransferProtocol.Status.read(proxyReply);
-      if (status != SUCCESS) {
-        if (status == ERROR_ACCESS_TOKEN) {
+      BlockOpResponseProto copyResponse = BlockOpResponseProto.parseFrom(
+          HdfsProtoUtil.vintPrefixed(proxyReply));
+
+      if (copyResponse.getStatus() != SUCCESS) {
+        if (copyResponse.getStatus() == ERROR_ACCESS_TOKEN) {
           throw new IOException("Copy block " + block + " from "
               + proxySock.getRemoteSocketAddress()
               + " failed due to access token error");
@@ -748,13 +771,24 @@
    * @param opStatus status message to write
    * @param timeout send timeout
    **/
-  private void sendResponse(Socket s, DataTransferProtocol.Status opStatus,
+  private void sendResponse(Socket s, Status status,
       long timeout) throws IOException {
     DataOutputStream reply = 
       new DataOutputStream(NetUtils.getOutputStream(s, timeout));
-    opStatus.write(reply);
-    reply.flush();
+    
+    writeResponse(status, reply);
   }
+  
+  private void writeResponse(Status status, OutputStream out)
+  throws IOException {
+    BlockOpResponseProto response = BlockOpResponseProto.newBuilder()
+      .setStatus(status)
+      .build();
+    
+    response.writeDelimitedTo(out);
+    out.flush();
+  }
+  
 
   private void checkAccess(DataOutputStream out, final boolean reply, 
       final ExtendedBlock blk,
@@ -771,12 +805,15 @@
               out = new DataOutputStream(
                   NetUtils.getOutputStream(s, datanode.socketWriteTimeout));
             }
-            ERROR_ACCESS_TOKEN.write(out);
+            
+            BlockOpResponseProto.Builder resp = BlockOpResponseProto.newBuilder()
+              .setStatus(ERROR_ACCESS_TOKEN);
             if (mode == BlockTokenSecretManager.AccessMode.WRITE) {
               DatanodeRegistration dnR = 
                 datanode.getDNRegistrationForBP(blk.getBlockPoolId());
-              Text.writeString(out, dnR.getName());
+              resp.setFirstBadLink(dnR.getName());
             }
+            resp.build().writeDelimitedTo(out);
             out.flush();
           }
           LOG.warn("Block token verification failed: op=" + op
diff --git a/src/java/org/apache/hadoop/hdfs/util/ByteBufferOutputStream.java b/src/java/org/apache/hadoop/hdfs/util/ByteBufferOutputStream.java
new file mode 100644
index 0000000..31d4dcc
--- /dev/null
+++ b/src/java/org/apache/hadoop/hdfs/util/ByteBufferOutputStream.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.util;
+
+import java.io.IOException;
+import java.io.OutputStream;
+import java.nio.ByteBuffer;
+
+import org.apache.hadoop.classification.InterfaceAudience;
+import org.apache.hadoop.classification.InterfaceStability;
+
+/**
+ * OutputStream that writes into a {@link ByteBuffer}.
+ */
+@InterfaceAudience.Private
+@InterfaceStability.Stable
+public class ByteBufferOutputStream extends OutputStream {
+
+  private final ByteBuffer buf;
+
+  public ByteBufferOutputStream(ByteBuffer buf) {
+    this.buf = buf;
+  }
+
+  @Override
+  public void write(int b) throws IOException {
+    buf.put((byte)b);
+  }
+
+  @Override
+  public void write(byte[] b, int off, int len) throws IOException {
+    buf.put(b, off, len);
+  }
+}
\ No newline at end of file
diff --git a/src/java/org/apache/hadoop/hdfs/util/ExactSizeInputStream.java b/src/java/org/apache/hadoop/hdfs/util/ExactSizeInputStream.java
new file mode 100644
index 0000000..17365fb
--- /dev/null
+++ b/src/java/org/apache/hadoop/hdfs/util/ExactSizeInputStream.java
@@ -0,0 +1,125 @@
+/**
+ * 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.util;
+
+import java.io.EOFException;
+import java.io.FilterInputStream;
+import java.io.IOException;
+import java.io.InputStream;
+
+import org.apache.hadoop.classification.InterfaceAudience;
+import org.apache.hadoop.classification.InterfaceStability;
+
+import com.google.common.base.Preconditions;
+
+/**
+ * An InputStream implementations which reads from some other InputStream
+ * but expects an exact number of bytes. Any attempts to read past the
+ * specified number of bytes will return as if the end of the stream
+ * was reached. If the end of the underlying stream is reached prior to
+ * the specified number of bytes, an EOFException is thrown.
+ */
+@InterfaceAudience.Private
+@InterfaceStability.Evolving
+public class ExactSizeInputStream extends FilterInputStream {
+  private int remaining;
+
+  /**
+   * Construct an input stream that will read no more than
+   * 'numBytes' bytes.
+   * 
+   * If an EOF occurs on the underlying stream before numBytes
+   * bytes have been read, an EOFException will be thrown.
+   * 
+   * @param in the inputstream to wrap
+   * @param numBytes the number of bytes to read
+   */
+  public ExactSizeInputStream(InputStream in, int numBytes) {
+    super(in);
+    Preconditions.checkArgument(numBytes >= 0,
+        "Negative expected bytes: ", numBytes);
+    this.remaining = numBytes;
+  }
+
+  @Override
+  public int available() throws IOException {
+    return Math.min(super.available(), remaining);
+  }
+
+  @Override
+  public int read() throws IOException {
+    // EOF if we reached our limit
+    if (remaining <= 0) {
+      return -1;
+    }
+    final int result = super.read();
+    if (result >= 0) {
+      --remaining;
+    } else if (remaining > 0) {
+      // Underlying stream reached EOF but we haven't read the expected
+      // number of bytes.
+      throw new EOFException(
+          "Premature EOF. Expected " + remaining + "more bytes");
+    }
+    return result;
+  }
+
+  @Override
+  public int read(final byte[] b, final int off, int len)
+                  throws IOException {
+    if (remaining <= 0) {
+      return -1;
+    }
+    len = Math.min(len, remaining);
+    final int result = super.read(b, off, len);
+    if (result >= 0) {
+      remaining -= result;
+    } else if (remaining > 0) {
+      // Underlying stream reached EOF but we haven't read the expected
+      // number of bytes.
+      throw new EOFException(
+          "Premature EOF. Expected " + remaining + "more bytes");
+    }
+    return result;
+  }
+
+  @Override
+  public long skip(final long n) throws IOException {
+    final long result = super.skip(Math.min(n, remaining));
+    if (result > 0) {
+      remaining -= result;
+    } else if (remaining > 0) {
+      // Underlying stream reached EOF but we haven't read the expected
+      // number of bytes.
+      throw new EOFException(
+          "Premature EOF. Expected " + remaining + "more bytes");
+    }
+    return result;
+  }
+  
+  @Override
+  public boolean markSupported() {
+    return false;
+  }
+
+  @Override
+  public void mark(int readlimit) {
+    throw new UnsupportedOperationException();
+  }
+  
+}
\ No newline at end of file
diff --git a/src/proto/datatransfer.proto b/src/proto/datatransfer.proto
new file mode 100644
index 0000000..7c5f859
--- /dev/null
+++ b/src/proto/datatransfer.proto
@@ -0,0 +1,140 @@
+/**
+ * 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 to transfer data
+// to and from the datanode, as well as between datanodes.
+
+option java_package = "org.apache.hadoop.hdfs.protocol.proto";
+option java_outer_classname = "DataTransferProtos";
+option java_generate_equals_and_hash = true;
+
+import "hdfs.proto";
+
+message BaseHeaderProto {
+  required ExtendedBlockProto block = 1;
+  optional BlockTokenIdentifierProto token = 2;
+}
+
+message ClientOperationHeaderProto {
+  required BaseHeaderProto baseHeader = 1;
+  required string clientName = 2;
+}
+
+message OpReadBlockProto {
+  required ClientOperationHeaderProto header = 1;
+  required uint64 offset = 2;
+  required uint64 len = 3;
+}
+  
+message OpWriteBlockProto {
+  required ClientOperationHeaderProto header = 1;
+  repeated DatanodeInfoProto targets = 2;
+  optional DatanodeInfoProto source = 3;
+  enum BlockConstructionStage {
+    PIPELINE_SETUP_APPEND = 0;
+    // pipeline set up for failed PIPELINE_SETUP_APPEND recovery
+    PIPELINE_SETUP_APPEND_RECOVERY = 1;
+    // data streaming
+    DATA_STREAMING = 2;
+    // pipeline setup for failed data streaming recovery
+    PIPELINE_SETUP_STREAMING_RECOVERY = 3;
+    // close the block and pipeline
+    PIPELINE_CLOSE = 4;
+    // Recover a failed PIPELINE_CLOSE
+    PIPELINE_CLOSE_RECOVERY = 5;
+    // pipeline set up for block creation
+    PIPELINE_SETUP_CREATE = 6;
+    // transfer RBW for adding datanodes
+    TRANSFER_RBW = 7;
+    // transfer Finalized for adding datanodes
+    TRANSFER_FINALIZED = 8;
+  }
+  required BlockConstructionStage stage = 4;
+  required uint32 pipelineSize = 5;
+  required uint64 minBytesRcvd = 6;
+  required uint64 maxBytesRcvd = 7;
+  required uint64 latestGenerationStamp = 8;
+}
+  
+message OpTransferBlockProto {
+  required ClientOperationHeaderProto header = 1;
+  repeated DatanodeInfoProto targets = 2;
+}
+
+message OpReplaceBlockProto {
+  required BaseHeaderProto header = 1;
+  required string delHint = 2;
+  required DatanodeInfoProto source = 3;
+}
+
+message OpCopyBlockProto {
+  required BaseHeaderProto header = 1;
+}
+
+message OpBlockChecksumProto { 
+  required BaseHeaderProto header = 1;
+}
+
+
+message PacketHeaderProto {
+  // All fields must be fixed-length!
+  required sfixed64 offsetInBlock = 1;
+  required sfixed64 seqno = 2;
+  required bool lastPacketInBlock = 3;
+  required sfixed32 dataLen = 4;
+}
+
+enum Status {
+  SUCCESS = 0;
+  ERROR = 1;
+  ERROR_CHECKSUM = 2;
+  ERROR_INVALID = 3;
+  ERROR_EXISTS = 4;
+  ERROR_ACCESS_TOKEN = 5;
+  CHECKSUM_OK = 6;
+}
+
+message PipelineAckProto {
+  required sint64 seqno = 1;
+  repeated Status status = 2;
+}
+
+message BlockOpResponseProto {
+  required Status status = 1;
+
+  optional string firstBadLink = 2;
+  optional OpBlockChecksumResponseProto checksumResponse = 3;
+}
+
+/**
+ * Message sent from the client to the DN after reading the entire
+ * read request.
+ */
+message ClientReadStatusProto {
+  required Status status = 1;
+}
+
+message DNTransferAckProto {
+  required Status status = 1;
+}
+
+message OpBlockChecksumResponseProto {
+  required uint32 bytesPerCrc = 1;
+  required uint64 crcPerBlock = 2;
+  required bytes md5 = 3;
+}
diff --git a/src/proto/hdfs.proto b/src/proto/hdfs.proto
new file mode 100644
index 0000000..d11dbfa
--- /dev/null
+++ b/src/proto/hdfs.proto
@@ -0,0 +1,64 @@
+/**
+ * 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 = "HdfsProtos";
+option java_generate_equals_and_hash = true;
+
+message ExtendedBlockProto {
+  required string poolId = 1;
+  required uint64 blockId = 2;
+  required uint64 numBytes = 3;
+  required uint64 generationStamp = 4;
+}
+
+message BlockTokenIdentifierProto {
+  required bytes identifier = 1;
+  required bytes password = 2;
+  required string kind = 3;
+  required string service = 4;
+}
+
+message DatanodeIDProto {
+  required string name = 1;
+  required string storageID = 2;
+  required uint32 infoPort = 3;
+}
+
+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 string location = 8;
+  optional string hostName = 9;
+  enum AdminState {
+    NORMAL = 0;
+    DECOMMISSION_INPROGRESS = 1;
+    DECOMMISSIONED = 2;
+  }
+
+  optional AdminState adminState = 10;
+}
+
diff --git a/src/test/aop/org/apache/hadoop/hdfs/server/datanode/BlockReceiverAspects.aj b/src/test/aop/org/apache/hadoop/hdfs/server/datanode/BlockReceiverAspects.aj
index 874657d..c3bd393 100644
--- a/src/test/aop/org/apache/hadoop/hdfs/server/datanode/BlockReceiverAspects.aj
+++ b/src/test/aop/org/apache/hadoop/hdfs/server/datanode/BlockReceiverAspects.aj
@@ -17,9 +17,8 @@
  */
 package org.apache.hadoop.hdfs.server.datanode;
 
-import java.io.DataInput;
-import java.io.DataOutput;
 import java.io.IOException;
+import java.io.InputStream;
 import java.io.OutputStream;
 
 import org.apache.commons.logging.Log;
@@ -33,7 +32,6 @@
 import org.apache.hadoop.hdfs.PipelinesTestUtil.PipelinesTest;
 import org.apache.hadoop.hdfs.PipelinesTestUtil.NodeBytes;
 import org.apache.hadoop.hdfs.protocol.DatanodeID;
-import org.apache.hadoop.hdfs.protocol.DataTransferProtocol.Status;
 import org.apache.hadoop.hdfs.protocol.DataTransferProtocol.PipelineAck;
 import org.apache.hadoop.hdfs.server.protocol.DatanodeRegistration;
 import org.apache.hadoop.util.DiskChecker.DiskOutOfSpaceException;
@@ -83,7 +81,7 @@
   }
 
   pointcut afterDownstreamStatusRead(BlockReceiver.PacketResponder responder):
-    call(void PipelineAck.readFields(DataInput)) && this(responder);
+    call(void PipelineAck.readFields(InputStream)) && this(responder);
 
   after(BlockReceiver.PacketResponder responder)
       throws IOException: afterDownstreamStatusRead(responder) {
@@ -150,7 +148,7 @@
   }
   
   pointcut preventAckSending () :
-    call (void PipelineAck.write(DataOutput)) 
+    call (void PipelineAck.write(OutputStream)) 
     && within (PacketResponder);
 
   static int ackCounter = 0;
@@ -203,7 +201,7 @@
   }
 
   pointcut pipelineAck(BlockReceiver.PacketResponder packetresponder) :
-    call (void PipelineAck.readFields(DataInput))
+    call (void PipelineAck.readFields(InputStream))
       && this(packetresponder);
 
   after(BlockReceiver.PacketResponder packetresponder) throws IOException
diff --git a/src/test/aop/org/apache/hadoop/hdfs/server/datanode/DataTransferProtocolAspects.aj b/src/test/aop/org/apache/hadoop/hdfs/server/datanode/DataTransferProtocolAspects.aj
index f0b6acf..21ff0ae 100644
--- a/src/test/aop/org/apache/hadoop/hdfs/server/datanode/DataTransferProtocolAspects.aj
+++ b/src/test/aop/org/apache/hadoop/hdfs/server/datanode/DataTransferProtocolAspects.aj
@@ -19,15 +19,16 @@
 
 import java.io.DataInput;
 import java.io.DataInputStream;
+import java.io.InputStream;
 import java.io.IOException;
 
 import org.apache.commons.logging.Log;
 import org.apache.commons.logging.LogFactory;
 import org.apache.hadoop.fi.DataTransferTestUtil;
 import org.apache.hadoop.fi.DataTransferTestUtil.DataTransferTest;
+import org.apache.hadoop.hdfs.protocol.proto.DataTransferProtos.BlockOpResponseProto;
 import org.apache.hadoop.hdfs.protocol.DataTransferProtocol.Op;
 import org.apache.hadoop.hdfs.protocol.DataTransferProtocol.Receiver;
-import org.apache.hadoop.hdfs.protocol.DataTransferProtocol.Status;
 
 /** Aspect for DataTransferProtocol */
 public aspect DataTransferProtocolAspects {
@@ -53,9 +54,9 @@
   }
 
   pointcut statusRead(DataXceiver dataxceiver):
-    call(Status Status.read(DataInput)) && this(dataxceiver);
+    call(BlockOpResponseProto BlockOpResponseProto.parseFrom(InputStream)) && this(dataxceiver);
 
-  after(DataXceiver dataxceiver) returning(Status status
+  after(DataXceiver dataxceiver) returning(BlockOpResponseProto status
       ) throws IOException: statusRead(dataxceiver) {
     final DataNode d = dataxceiver.getDataNode();
     LOG.info("FI: statusRead " + status + ", datanode="
diff --git a/src/test/findbugsExcludeFile.xml b/src/test/findbugsExcludeFile.xml
index d72e50d..394c37e 100644
--- a/src/test/findbugsExcludeFile.xml
+++ b/src/test/findbugsExcludeFile.xml
@@ -3,6 +3,9 @@
        <Package name="org.apache.hadoop.record.compiler.generated" />
      </Match>
      <Match>
+       <Package name="org.apache.hadoop.hdfs.protocol.proto" />
+     </Match>
+     <Match>
        <Bug pattern="EI_EXPOSE_REP" />
      </Match>
      <Match>
diff --git a/src/test/hdfs/org/apache/hadoop/hdfs/DFSTestUtil.java b/src/test/hdfs/org/apache/hadoop/hdfs/DFSTestUtil.java
index f9248b8..57fdc43 100644
--- a/src/test/hdfs/org/apache/hadoop/hdfs/DFSTestUtil.java
+++ b/src/test/hdfs/org/apache/hadoop/hdfs/DFSTestUtil.java
@@ -57,6 +57,7 @@
 import org.apache.hadoop.hdfs.protocol.DatanodeInfo;
 import org.apache.hadoop.hdfs.protocol.LocatedBlock;
 import org.apache.hadoop.hdfs.protocol.LocatedBlocks;
+import org.apache.hadoop.hdfs.protocol.proto.DataTransferProtos.BlockOpResponseProto;
 import org.apache.hadoop.hdfs.security.token.block.BlockTokenIdentifier;
 import org.apache.hadoop.hdfs.server.common.HdfsConstants.StartupOption;
 import org.apache.hadoop.hdfs.server.datanode.DataNode;
@@ -66,7 +67,6 @@
 import org.apache.hadoop.hdfs.server.namenode.NameNode;
 import org.apache.hadoop.hdfs.server.namenode.NameNodeAdapter;
 import org.apache.hadoop.io.IOUtils;
-import org.apache.hadoop.ipc.RPC;
 import org.apache.hadoop.net.NetUtils;
 import org.apache.hadoop.security.ShellBasedUnixGroupsMapping;
 import org.apache.hadoop.security.UserGroupInformation;
@@ -662,7 +662,7 @@
   }
 
   /** For {@link TestTransferRbw} */
-  public static DataTransferProtocol.Status transferRbw(final ExtendedBlock b, 
+  public static BlockOpResponseProto transferRbw(final ExtendedBlock b, 
       final DFSClient dfsClient, final DatanodeInfo... datanodes) throws IOException {
     assertEquals(2, datanodes.length);
     final Socket s = DFSOutputStream.createSocketForPipeline(datanodes[0],
@@ -678,6 +678,6 @@
         new DatanodeInfo[]{datanodes[1]}, new Token<BlockTokenIdentifier>());
     out.flush();
 
-    return DataTransferProtocol.Status.read(in);
+    return BlockOpResponseProto.parseDelimitedFrom(in);
   }
 }
diff --git a/src/test/hdfs/org/apache/hadoop/hdfs/TestClientBlockVerification.java b/src/test/hdfs/org/apache/hadoop/hdfs/TestClientBlockVerification.java
index be6ac1c..639d7af 100644
--- a/src/test/hdfs/org/apache/hadoop/hdfs/TestClientBlockVerification.java
+++ b/src/test/hdfs/org/apache/hadoop/hdfs/TestClientBlockVerification.java
@@ -20,8 +20,8 @@
 
 import java.util.List;
 
-import org.apache.hadoop.hdfs.protocol.DataTransferProtocol.Status;
 import org.apache.hadoop.hdfs.protocol.LocatedBlock;
+import org.apache.hadoop.hdfs.protocol.proto.DataTransferProtos.Status;
 import org.apache.hadoop.fs.Path;
 
 import org.junit.Test;
diff --git a/src/test/hdfs/org/apache/hadoop/hdfs/TestDataTransferProtocol.java b/src/test/hdfs/org/apache/hadoop/hdfs/TestDataTransferProtocol.java
index 70338c9..63d1974 100644
--- a/src/test/hdfs/org/apache/hadoop/hdfs/TestDataTransferProtocol.java
+++ b/src/test/hdfs/org/apache/hadoop/hdfs/TestDataTransferProtocol.java
@@ -20,9 +20,6 @@
 import static org.apache.hadoop.hdfs.protocol.DataTransferProtocol.Op.WRITE_BLOCK;
 import static org.apache.hadoop.hdfs.protocol.DataTransferProtocol.PacketHeader;
 import static org.apache.hadoop.hdfs.protocol.DataTransferProtocol.PipelineAck;
-import static org.apache.hadoop.hdfs.protocol.DataTransferProtocol.Status;
-import static org.apache.hadoop.hdfs.protocol.DataTransferProtocol.Status.ERROR;
-import static org.apache.hadoop.hdfs.protocol.DataTransferProtocol.Status.SUCCESS;
 
 import java.io.ByteArrayInputStream;
 import java.io.ByteArrayOutputStream;
@@ -46,11 +43,15 @@
 import org.apache.hadoop.fs.FileSystem;
 import org.apache.hadoop.fs.Path;
 import org.apache.hadoop.hdfs.protocol.DataTransferProtocol;
+import org.apache.hadoop.hdfs.protocol.DataTransferProtocol.BlockConstructionStage;
 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.DataTransferProtocol.BlockConstructionStage;
 import org.apache.hadoop.hdfs.protocol.FSConstants.DatanodeReportType;
+import org.apache.hadoop.hdfs.protocol.proto.DataTransferProtos;
+import org.apache.hadoop.hdfs.protocol.proto.DataTransferProtos.BlockOpResponseProto;
+import org.apache.hadoop.hdfs.protocol.proto.DataTransferProtos.BlockOpResponseProto.Builder;
+import org.apache.hadoop.hdfs.protocol.proto.DataTransferProtos.Status;
 import org.apache.hadoop.hdfs.security.token.block.BlockTokenSecretManager;
 import org.apache.hadoop.hdfs.server.common.HdfsConstants;
 import org.apache.hadoop.hdfs.server.datanode.DataNodeTestUtils;
@@ -59,6 +60,7 @@
 import org.apache.hadoop.io.Text;
 import org.apache.hadoop.net.NetUtils;
 import org.apache.hadoop.util.DataChecksum;
+import org.apache.hadoop.util.StringUtils;
 import org.junit.Test;
 
 /**
@@ -93,6 +95,9 @@
       if ( testDescription != null ) {
         LOG.info("Testing : " + testDescription);
       }
+      LOG.info("Going to write:" +
+          StringUtils.byteToHexString(sendBuf.toByteArray()));
+      
       sock = new Socket();
       sock.connect(dnAddr, HdfsConstants.READ_TIMEOUT);
       sock.setSoTimeout(HdfsConstants.READ_TIMEOUT);
@@ -113,10 +118,11 @@
         }
         throw eof;
       }
-      for (int i=0; i<retBuf.length; i++) {
-        System.out.print(retBuf[i]);
-      }
-      System.out.println(":");
+
+      LOG.info("Received: " +
+          StringUtils.byteToHexString(retBuf));
+      LOG.info("Expected: " +
+          StringUtils.byteToHexString(recvBuf.toByteArray()));
       
       if (eofExpected) {
         throw new IOException("Did not recieve IOException when an exception " +
@@ -162,12 +168,22 @@
     sendOut.writeInt(0);           // zero checksum
 
     //ok finally write a block with 0 len
-    SUCCESS.write(recvOut);
-    Text.writeString(recvOut, "");
-    new PipelineAck(100, new Status[]{SUCCESS}).write(recvOut);
+    sendResponse(Status.SUCCESS, "", recvOut);
+    new PipelineAck(100, new Status[]{Status.SUCCESS}).write(recvOut);
     sendRecvData(description, false);
   }
   
+  private void sendResponse(Status status, String firstBadLink,
+      DataOutputStream out)
+  throws IOException {
+    Builder builder = BlockOpResponseProto.newBuilder().setStatus(status);
+    if (firstBadLink != null) {
+      builder.setFirstBadLink(firstBadLink);
+    }
+    builder.build()
+      .writeDelimitedTo(out);
+  }
+
   private void testWrite(ExtendedBlock block, BlockConstructionStage stage, long newGS,
       String description, Boolean eofExcepted) throws IOException {
     sendBuf.reset();
@@ -176,12 +192,11 @@
         stage, newGS, block.getNumBytes(), block.getNumBytes(), "cl", null,
         new DatanodeInfo[1], BlockTokenSecretManager.DUMMY_TOKEN);
     if (eofExcepted) {
-      ERROR.write(recvOut);
+      sendResponse(Status.ERROR, null, recvOut);
       sendRecvData(description, true);
     } else if (stage == BlockConstructionStage.PIPELINE_CLOSE_RECOVERY) {
       //ok finally write a block with 0 len
-      SUCCESS.write(recvOut);
-      Text.writeString(recvOut, ""); // first bad node
+      sendResponse(Status.SUCCESS, "", recvOut);
       sendRecvData(description, false);
     } else {
       writeZeroLengthPacket(block, description);
@@ -369,7 +384,7 @@
     // bad bytes per checksum
     sendOut.writeInt(-1-random.nextInt(oneMil));
     recvBuf.reset();
-    ERROR.write(recvOut);
+    sendResponse(Status.ERROR, null, recvOut);
     sendRecvData("wrong bytesPerChecksum while writing", true);
 
     sendBuf.reset();
@@ -389,9 +404,8 @@
       -1 - random.nextInt(oneMil)); // bad datalen
     hdr.write(sendOut);
 
-    SUCCESS.write(recvOut);
-    Text.writeString(recvOut, "");
-    new PipelineAck(100, new Status[]{ERROR}).write(recvOut);
+    sendResponse(Status.SUCCESS, "", recvOut);
+    new PipelineAck(100, new Status[]{Status.ERROR}).write(recvOut);
     sendRecvData("negative DATA_CHUNK len while writing block " + newBlockId, 
                  true);
 
@@ -415,9 +429,8 @@
     sendOut.writeInt(0);           // zero checksum
     sendOut.flush();
     //ok finally write a block with 0 len
-    SUCCESS.write(recvOut);
-    Text.writeString(recvOut, "");
-    new PipelineAck(100, new Status[]{SUCCESS}).write(recvOut);
+    sendResponse(Status.SUCCESS, "", recvOut);
+    new PipelineAck(100, new Status[]{Status.SUCCESS}).write(recvOut);
     sendRecvData("Writing a zero len block blockid " + newBlockId, false);
     
     /* Test OP_READ_BLOCK */
@@ -450,7 +463,7 @@
     
     // negative length is ok. Datanode assumes we want to read the whole block.
     recvBuf.reset();
-    SUCCESS.write(recvOut);    
+    sendResponse(Status.SUCCESS, null, recvOut);
     sendBuf.reset();
     DataTransferProtocol.Sender.opReadBlock(sendOut, blk, 0L, 
         -1 - random.nextInt(oneMil), "cl", BlockTokenSecretManager.DUMMY_TOKEN);
@@ -459,7 +472,7 @@
     
     // length is more than size of block.
     recvBuf.reset();
-    ERROR.write(recvOut);    
+    sendResponse(Status.ERROR, null, recvOut);
     sendBuf.reset();
     DataTransferProtocol.Sender.opReadBlock(sendOut, blk, 0L, 
         fileLen + 1, "cl", BlockTokenSecretManager.DUMMY_TOKEN);
diff --git a/src/test/hdfs/org/apache/hadoop/hdfs/server/datanode/TestBlockReplacement.java b/src/test/hdfs/org/apache/hadoop/hdfs/server/datanode/TestBlockReplacement.java
index 2a15af1..52ffa92 100644
--- a/src/test/hdfs/org/apache/hadoop/hdfs/server/datanode/TestBlockReplacement.java
+++ b/src/test/hdfs/org/apache/hadoop/hdfs/server/datanode/TestBlockReplacement.java
@@ -17,8 +17,6 @@
  */
 package org.apache.hadoop.hdfs.server.datanode;
 
-import static org.apache.hadoop.hdfs.protocol.DataTransferProtocol.Status.*;
-
 import java.io.DataInputStream;
 import java.io.DataOutputStream;
 import java.io.IOException;
@@ -46,6 +44,8 @@
 import org.apache.hadoop.hdfs.protocol.ExtendedBlock;
 import org.apache.hadoop.hdfs.protocol.LocatedBlock;
 import org.apache.hadoop.hdfs.protocol.FSConstants.DatanodeReportType;
+import org.apache.hadoop.hdfs.protocol.proto.DataTransferProtos.BlockOpResponseProto;
+import org.apache.hadoop.hdfs.protocol.proto.DataTransferProtos.Status;
 import org.apache.hadoop.hdfs.security.token.block.BlockTokenSecretManager;
 import org.apache.hadoop.hdfs.server.common.HdfsConstants;
 import org.apache.hadoop.hdfs.server.common.Util;
@@ -264,7 +264,9 @@
     // receiveResponse
     DataInputStream reply = new DataInputStream(sock.getInputStream());
 
-    return DataTransferProtocol.Status.read(reply) == SUCCESS;
+    BlockOpResponseProto proto =
+      BlockOpResponseProto.parseDelimitedFrom(reply);
+    return proto.getStatus() == Status.SUCCESS;
   }
 
   /**
diff --git a/src/test/hdfs/org/apache/hadoop/hdfs/server/datanode/TestTransferRbw.java b/src/test/hdfs/org/apache/hadoop/hdfs/server/datanode/TestTransferRbw.java
index 2550bb2..6bc92a1 100644
--- a/src/test/hdfs/org/apache/hadoop/hdfs/server/datanode/TestTransferRbw.java
+++ b/src/test/hdfs/org/apache/hadoop/hdfs/server/datanode/TestTransferRbw.java
@@ -33,6 +33,8 @@
 import org.apache.hadoop.hdfs.protocol.DatanodeInfo;
 import org.apache.hadoop.hdfs.protocol.ExtendedBlock;
 import org.apache.hadoop.hdfs.protocol.FSConstants.DatanodeReportType;
+import org.apache.hadoop.hdfs.protocol.proto.DataTransferProtos.BlockOpResponseProto;
+import org.apache.hadoop.hdfs.protocol.proto.DataTransferProtos.Status;
 import org.apache.hadoop.hdfs.server.common.HdfsConstants.ReplicaState;
 import org.apache.hadoop.hdfs.server.protocol.DatanodeRegistration;
 import org.apache.log4j.Level;
@@ -121,9 +123,9 @@
         //transfer RBW
         final ExtendedBlock b = new ExtendedBlock(bpid, oldrbw.getBlockId(), oldrbw.getBytesAcked(),
             oldrbw.getGenerationStamp());
-        final DataTransferProtocol.Status s = DFSTestUtil.transferRbw(
+        final BlockOpResponseProto s = DFSTestUtil.transferRbw(
             b, fs.getClient(), oldnodeinfo, newnodeinfo);
-        Assert.assertEquals(DataTransferProtocol.Status.SUCCESS, s);
+        Assert.assertEquals(Status.SUCCESS, s.getStatus());
       }
 
       //check new rbw
diff --git a/src/test/hdfs/org/apache/hadoop/hdfs/server/namenode/TestBlockTokenWithDFS.java b/src/test/hdfs/org/apache/hadoop/hdfs/server/namenode/TestBlockTokenWithDFS.java
index 1e0afab..717f887 100644
--- a/src/test/hdfs/org/apache/hadoop/hdfs/server/namenode/TestBlockTokenWithDFS.java
+++ b/src/test/hdfs/org/apache/hadoop/hdfs/server/namenode/TestBlockTokenWithDFS.java
@@ -43,6 +43,7 @@
 import org.apache.hadoop.fs.FileSystem;
 import org.apache.hadoop.fs.Path;
 import org.apache.hadoop.security.token.*;
+import org.apache.hadoop.util.StringUtils;
 import org.apache.log4j.Level;
 
 import junit.framework.TestCase;
@@ -142,7 +143,8 @@
             + "when it is expected to be valid", shouldSucceed);
         return;
       }
-      fail("OP_READ_BLOCK failed due to reasons other than access token");
+      fail("OP_READ_BLOCK failed due to reasons other than access token: "
+          + StringUtils.stringifyException(ex));
     } finally {
       if (s != null) {
         try {
diff --git a/src/test/hdfs/org/apache/hadoop/hdfs/util/TestExactSizeInputStream.java b/src/test/hdfs/org/apache/hadoop/hdfs/util/TestExactSizeInputStream.java
new file mode 100644
index 0000000..c64427f
--- /dev/null
+++ b/src/test/hdfs/org/apache/hadoop/hdfs/util/TestExactSizeInputStream.java
@@ -0,0 +1,128 @@
+/**
+ * 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.util;
+
+import static org.junit.Assert.*;
+
+import java.io.ByteArrayInputStream;
+import java.io.EOFException;
+import java.io.IOException;
+import java.io.InputStream;
+
+import org.apache.hadoop.hdfs.util.ExactSizeInputStream;
+import org.junit.Test;
+
+public class TestExactSizeInputStream {
+  @Test
+  public void testBasicsReadSingle() throws IOException {
+    ExactSizeInputStream s = new ExactSizeInputStream(byteStream("hello"), 3);
+    assertEquals(3, s.available());
+    
+    assertEquals((int)'h', s.read());
+    assertEquals((int)'e', s.read());
+    assertEquals((int)'l', s.read());
+    assertEquals(-1, s.read());
+    assertEquals(0, s.available());
+  }
+  
+  @Test
+  public void testBasicsReadArray() throws IOException {
+    ExactSizeInputStream s = new ExactSizeInputStream(byteStream("hello"), 3);
+    assertEquals(3, s.available());
+    
+    byte[] buf = new byte[10];
+    
+    assertEquals(2, s.read(buf, 0, 2));
+    assertEquals('h', buf[0]);
+    assertEquals('e', buf[1]);
+    
+    assertEquals(1, s.read(buf, 0, 2));
+    assertEquals('l', buf[0]);
+    
+    assertEquals(-1, s.read(buf, 0, 2));
+  }
+  
+  @Test
+  public void testBasicsSkip() throws IOException {
+    ExactSizeInputStream s = new ExactSizeInputStream(byteStream("hello"), 3);
+    assertEquals(3, s.available());
+    
+    assertEquals(2, s.skip(2));
+    assertEquals(1, s.skip(2));
+    assertEquals(0, s.skip(2));
+  }
+  
+  @Test
+  public void testReadNotEnough() throws IOException {
+    // Ask for 5 bytes, only has 2
+    ExactSizeInputStream s = new ExactSizeInputStream(byteStream("he"), 5);
+    assertEquals(2, s.available());
+    
+    assertEquals((int)'h', s.read());
+    assertEquals((int)'e', s.read());
+    try {
+      s.read();
+      fail("Read when should be out of data");
+    } catch (EOFException e) {
+      // expected
+    }
+  }
+  
+  @Test
+  public void testSkipNotEnough() throws IOException {
+    // Ask for 5 bytes, only has 2
+    ExactSizeInputStream s = new ExactSizeInputStream(byteStream("he"), 5);
+    assertEquals(2, s.skip(3));
+    try {
+      s.skip(1);
+      fail("Skip when should be out of data");
+    } catch (EOFException e) {
+      // expected
+    }
+  }
+  
+  @Test
+  public void testReadArrayNotEnough() throws IOException {
+    // Ask for 5 bytes, only has 2
+    ExactSizeInputStream s = new ExactSizeInputStream(byteStream("he"), 5);
+    byte[] buf = new byte[10];
+    assertEquals(2, s.read(buf, 0, 5));
+    try {
+      s.read(buf, 2, 3);
+      fail("Read buf when should be out of data");
+    } catch (EOFException e) {
+      // expected
+    }
+  }
+
+  @Test
+  public void testMark() throws IOException {
+    ExactSizeInputStream s = new ExactSizeInputStream(byteStream("he"), 5);
+    assertFalse(s.markSupported());
+    try {
+      s.mark(1);
+      fail("Mark should not succeed");
+    } catch (UnsupportedOperationException uoe) {
+      // expected
+    }
+  }
+  
+  private static InputStream byteStream(String data) {
+    return new ByteArrayInputStream(data.getBytes());
+  }
+}