ソースを参照

Merge trunk into HDFS-1073

git-svn-id: https://svn.apache.org/repos/asf/hadoop/common/branches/HDFS-1073@1138160 13f79535-47bb-0310-9956-ffa450edef68
Todd Lipcon 14 年 前
コミット
23d4e0fb9d
100 ファイル変更16075 行追加2609 行削除
  1. 68 6
      hdfs/CHANGES.txt
  2. 12 0
      hdfs/bin/hdfs
  3. 36 2
      hdfs/build.xml
  4. 1 0
      hdfs/ivy.xml
  5. 33 1
      hdfs/src/c++/libhdfs/hdfs.c
  6. 10 0
      hdfs/src/c++/libhdfs/hdfs.h
  7. 6 0
      hdfs/src/c++/libhdfs/hdfs_test.c
  8. 4 2
      hdfs/src/c++/libhdfs/tests/test-libhdfs.sh
  9. 4 1
      hdfs/src/contrib/build-contrib.xml
  10. 5 1
      hdfs/src/contrib/build.xml
  11. 16 21
      hdfs/src/contrib/fuse-dfs/build.xml
  12. 3 0
      hdfs/src/contrib/fuse-dfs/configure.ac
  13. 88 19
      hdfs/src/contrib/fuse-dfs/src/fuse_connect.c
  14. 3 7
      hdfs/src/contrib/fuse-dfs/src/fuse_connect.h
  15. 0 1
      hdfs/src/contrib/fuse-dfs/src/fuse_context_handle.h
  16. 34 44
      hdfs/src/contrib/fuse-dfs/src/fuse_dfs.c
  17. 0 5
      hdfs/src/contrib/fuse-dfs/src/fuse_dfs.h
  18. 2 2
      hdfs/src/contrib/fuse-dfs/src/fuse_dfs_wrapper.sh
  19. 2 16
      hdfs/src/contrib/fuse-dfs/src/fuse_impls_access.c
  20. 14 11
      hdfs/src/contrib/fuse-dfs/src/fuse_impls_chmod.c
  21. 26 23
      hdfs/src/contrib/fuse-dfs/src/fuse_impls_chown.c
  22. 14 12
      hdfs/src/contrib/fuse-dfs/src/fuse_impls_getattr.c
  23. 12 10
      hdfs/src/contrib/fuse-dfs/src/fuse_impls_mkdir.c
  24. 6 13
      hdfs/src/contrib/fuse-dfs/src/fuse_impls_open.c
  25. 13 16
      hdfs/src/contrib/fuse-dfs/src/fuse_impls_readdir.c
  26. 6 0
      hdfs/src/contrib/fuse-dfs/src/fuse_impls_release.c
  27. 11 5
      hdfs/src/contrib/fuse-dfs/src/fuse_impls_rename.c
  28. 15 10
      hdfs/src/contrib/fuse-dfs/src/fuse_impls_rmdir.c
  29. 5 27
      hdfs/src/contrib/fuse-dfs/src/fuse_impls_statfs.c
  30. 18 10
      hdfs/src/contrib/fuse-dfs/src/fuse_impls_truncate.c
  31. 12 8
      hdfs/src/contrib/fuse-dfs/src/fuse_impls_unlink.c
  32. 16 12
      hdfs/src/contrib/fuse-dfs/src/fuse_impls_utimens.c
  33. 10 21
      hdfs/src/contrib/fuse-dfs/src/fuse_init.c
  34. 5 6
      hdfs/src/contrib/fuse-dfs/src/fuse_options.c
  35. 4 15
      hdfs/src/contrib/fuse-dfs/src/fuse_stat_struct.c
  36. 3 11
      hdfs/src/contrib/fuse-dfs/src/fuse_users.c
  37. 3 4
      hdfs/src/docs/src/documentation/content/xdocs/hdfs_design.xml
  38. 27 26
      hdfs/src/java/org/apache/hadoop/hdfs/BlockReader.java
  39. 23 16
      hdfs/src/java/org/apache/hadoop/hdfs/DFSClient.java
  40. 26 18
      hdfs/src/java/org/apache/hadoop/hdfs/DFSOutputStream.java
  41. 3 4
      hdfs/src/java/org/apache/hadoop/hdfs/HftpFileSystem.java
  42. 0 1068
      hdfs/src/java/org/apache/hadoop/hdfs/protocol/DataTransferProtocol.java
  43. 5 0
      hdfs/src/java/org/apache/hadoop/hdfs/protocol/DatanodeInfo.java
  44. 166 0
      hdfs/src/java/org/apache/hadoop/hdfs/protocol/HdfsProtoUtil.java
  45. 24 0
      hdfs/src/java/org/apache/hadoop/hdfs/protocol/LayoutVersion.java
  46. 62 0
      hdfs/src/java/org/apache/hadoop/hdfs/protocol/datatransfer/BlockConstructionStage.java
  47. 69 0
      hdfs/src/java/org/apache/hadoop/hdfs/protocol/datatransfer/DataTransferProtoUtil.java
  48. 42 0
      hdfs/src/java/org/apache/hadoop/hdfs/protocol/datatransfer/DataTransferProtocol.java
  49. 62 0
      hdfs/src/java/org/apache/hadoop/hdfs/protocol/datatransfer/Op.java
  50. 158 0
      hdfs/src/java/org/apache/hadoop/hdfs/protocol/datatransfer/PacketHeader.java
  51. 106 0
      hdfs/src/java/org/apache/hadoop/hdfs/protocol/datatransfer/PipelineAck.java
  52. 201 0
      hdfs/src/java/org/apache/hadoop/hdfs/protocol/datatransfer/Receiver.java
  53. 117 0
      hdfs/src/java/org/apache/hadoop/hdfs/protocol/datatransfer/ReplaceDatanodeOnFailure.java
  54. 152 0
      hdfs/src/java/org/apache/hadoop/hdfs/protocol/datatransfer/Sender.java
  55. 9122 0
      hdfs/src/java/org/apache/hadoop/hdfs/protocol/proto/DataTransferProtos.java
  56. 3247 0
      hdfs/src/java/org/apache/hadoop/hdfs/protocol/proto/HdfsProtos.java
  57. 9 6
      hdfs/src/java/org/apache/hadoop/hdfs/server/balancer/Balancer.java
  58. 7 9
      hdfs/src/java/org/apache/hadoop/hdfs/server/datanode/BlockReceiver.java
  59. 1 1
      hdfs/src/java/org/apache/hadoop/hdfs/server/datanode/BlockSender.java
  60. 54 10
      hdfs/src/java/org/apache/hadoop/hdfs/server/datanode/DataNode.java
  61. 101 62
      hdfs/src/java/org/apache/hadoop/hdfs/server/datanode/DataXceiver.java
  62. 6 7
      hdfs/src/java/org/apache/hadoop/hdfs/server/datanode/DataXceiverServer.java
  63. 7 0
      hdfs/src/java/org/apache/hadoop/hdfs/server/datanode/DatanodeJspHelper.java
  64. 2 1
      hdfs/src/java/org/apache/hadoop/hdfs/server/datanode/FSDataset.java
  65. 1 1
      hdfs/src/java/org/apache/hadoop/hdfs/server/namenode/BackupNode.java
  66. 6 3
      hdfs/src/java/org/apache/hadoop/hdfs/server/namenode/BlockManager.java
  67. 391 383
      hdfs/src/java/org/apache/hadoop/hdfs/server/namenode/FSDirectory.java
  68. 10 3
      hdfs/src/java/org/apache/hadoop/hdfs/server/namenode/FSEditLogLoader.java
  69. 1 4
      hdfs/src/java/org/apache/hadoop/hdfs/server/namenode/FSEditLogOp.java
  70. 3 18
      hdfs/src/java/org/apache/hadoop/hdfs/server/namenode/FSImage.java
  71. 401 324
      hdfs/src/java/org/apache/hadoop/hdfs/server/namenode/FSNamesystem.java
  72. 1 0
      hdfs/src/java/org/apache/hadoop/hdfs/server/namenode/LeaseManager.java
  73. 34 0
      hdfs/src/java/org/apache/hadoop/hdfs/server/namenode/NNStorage.java
  74. 78 135
      hdfs/src/java/org/apache/hadoop/hdfs/server/namenode/NameNode.java
  75. 17 4
      hdfs/src/java/org/apache/hadoop/hdfs/server/namenode/NamenodeJspHelper.java
  76. 2 1
      hdfs/src/java/org/apache/hadoop/hdfs/tools/DelegationTokenFetcher.java
  77. 49 0
      hdfs/src/java/org/apache/hadoop/hdfs/util/ByteBufferOutputStream.java
  78. 125 0
      hdfs/src/java/org/apache/hadoop/hdfs/util/ExactSizeInputStream.java
  79. 140 0
      hdfs/src/proto/datatransfer.proto
  80. 64 0
      hdfs/src/proto/hdfs.proto
  81. 5 7
      hdfs/src/test/aop/org/apache/hadoop/hdfs/server/datanode/BlockReceiverAspects.aj
  82. 6 5
      hdfs/src/test/aop/org/apache/hadoop/hdfs/server/datanode/DataTransferProtocolAspects.aj
  83. 1 1
      hdfs/src/test/aop/org/apache/hadoop/hdfs/server/datanode/TestFiDataTransferProtocol.java
  84. 1 1
      hdfs/src/test/aop/org/apache/hadoop/hdfs/server/datanode/TestFiDataTransferProtocol2.java
  85. 3 0
      hdfs/src/test/findbugsExcludeFile.xml
  86. 1 1
      hdfs/src/test/hdfs/org/apache/hadoop/cli/TestHDFSCLI.java
  87. 17 8
      hdfs/src/test/hdfs/org/apache/hadoop/hdfs/DFSTestUtil.java
  88. 1 1
      hdfs/src/test/hdfs/org/apache/hadoop/hdfs/TestClientBlockVerification.java
  89. 51 49
      hdfs/src/test/hdfs/org/apache/hadoop/hdfs/TestDataTransferProtocol.java
  90. 4 2
      hdfs/src/test/hdfs/org/apache/hadoop/hdfs/TestDecommission.java
  91. 4 4
      hdfs/src/test/hdfs/org/apache/hadoop/hdfs/TestReplaceDatanodeOnFailure.java
  92. 147 61
      hdfs/src/test/hdfs/org/apache/hadoop/hdfs/TestSafeMode.java
  93. 9 7
      hdfs/src/test/hdfs/org/apache/hadoop/hdfs/server/datanode/TestBlockReplacement.java
  94. 91 0
      hdfs/src/test/hdfs/org/apache/hadoop/hdfs/server/datanode/TestDataNodeExit.java
  95. 52 3
      hdfs/src/test/hdfs/org/apache/hadoop/hdfs/server/datanode/TestDatanodeJsp.java
  96. 9 8
      hdfs/src/test/hdfs/org/apache/hadoop/hdfs/server/datanode/TestDiskError.java
  97. 5 3
      hdfs/src/test/hdfs/org/apache/hadoop/hdfs/server/datanode/TestTransferRbw.java
  98. 15 0
      hdfs/src/test/hdfs/org/apache/hadoop/hdfs/server/namenode/NameNodeAdapter.java
  99. 3 1
      hdfs/src/test/hdfs/org/apache/hadoop/hdfs/server/namenode/TestBlockTokenWithDFS.java
  100. 8 1
      hdfs/src/test/hdfs/org/apache/hadoop/hdfs/server/namenode/TestDeadDatanode.java

+ 68 - 6
hdfs/CHANGES.txt

@@ -11,6 +11,8 @@ Trunk (unreleased changes)
 
   NEW FEATURES
 
+    HDFS-1359. Add BlockPoolID to Block. (suresh)
+
     HDFS-1365. Federation: propose ClusterID and BlockPoolID format 
     (Tanping via boryas)
 
@@ -285,6 +287,11 @@ Trunk (unreleased changes)
     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)
+
+    HDFS-2055. Add hflush support to libhdfs. (Travis Crawford via eli)
+
   IMPROVEMENTS
 
     HDFS-1875. MiniDFSCluster hard-codes dfs.datanode.address to localhost
@@ -497,6 +504,21 @@ Trunk (unreleased changes)
     HDFS-2003. Separate FSEditLog reading logic from edit log memory state
     building logic. (Ivan Kelly via todd)
 
+    HDFS-2066. Create a package and individual class files for
+    DataTransferProtocol.  (szetszwo)
+
+    HADOOP-7106. Reorganize project SVN layout to "unsplit" the projects.
+    (todd, nigel)
+
+    HDFS-2046. Force entropy to come from non-true random for tests. (todd)
+
+    HDFS-2073. Add @Override annotation to NameNode. (suresh)
+
+    HDFS-420. Fuse-dfs should cache fs handles. (Brian Bockelman and eli)
+
+    HDFS-1568. Improve the log messages in DataXceiver.  (Joey Echeverria via
+    szetszwo)
+
   OPTIMIZATIONS
 
     HDFS-1458. Improve checkpoint performance by avoiding unnecessary image
@@ -507,11 +529,15 @@ Trunk (unreleased changes)
     HDFS-1826. NameNode should save image to name directories in parallel
     during upgrade. (Matt Foley via hairong)
 
-    HDFS-941. The DFS client should cache and reuse open sockets to datanodes
-    while performing reads. (bc Wong and Todd Lipcon via todd)
+    HDFS-2030. Improve usability of namenode -upgrade command.
+    (Bharath Mundlapudi via suresh)
+
+    HDFS-2056. Update fetchdt usage. (Tanping Wang via jitendra)
 
   BUG FIXES
 
+    HDFS-2061. Two minor bugs in BlockManager block report processing. (mattf)
+
     HDFS-1449. Fix test failures - ExtendedBlock must return 
     block file name in #getBlockName(). (suresh)
 
@@ -654,6 +680,8 @@ Trunk (unreleased changes)
     HDFS-1931. Update TestDFSShell for improved "du" shell command. (Daryn
     Sharp via todd)
 
+    HDFS-1439. HDFS Federation: Fix compilation error in TestFiHftp. (suresh)
+
     HDFS-1881. Federation: after taking snapshot the current directory 
     of datanode is empty. (Tanping Wang via suresh)
 
@@ -711,6 +739,29 @@ Trunk (unreleased changes)
     HDFS-1998. Federation: Make refresh-namenodes.sh refresh all the
     namenode. (Tanping Wang via suresh)
 
+    HDFS-2041. OP_CONCAT_DELETE doesn't properly restore modification time
+    of the concatenated file when edit logs are replayed. (todd)
+
+    HDFS-2063. libhdfs test is broken. (Eric Yang via eli)
+
+    HDFS-2067. Bump DATA_TRANSFER_VERSION constant in trunk after introduction
+    of protocol buffers in the protocol. (szetszwo via todd)
+
+    HDFS-2069. Incorrect default trash interval value in the docs.
+    (Harsh J Chouraria via eli)
+
+    HDFS-1942. Datanode must exist when all the block pool service threads
+    exit. (Bharath Mundlapudi via suresh)
+
+    HDFS-1656. Fixes an issue to do with fetching of delegation tokens in
+    HftpFileSystem. Contributed by Kan Zhang.
+
+    HDFS-1692. In secure mode, Datanode process doesn't exit when disks 
+    fail. (Bharath Mundlapudi via suresh)
+
+    HDFS-1734. 'Chunk size to view' option is not working in Name Node UI.
+    (Uma Maheswara Rao G via jitendra)
+
 Release 0.22.0 - Unreleased
 
   INCOMPATIBLE CHANGES
@@ -748,8 +799,6 @@ Release 0.22.0 - Unreleased
     HDFS piggyback block locations to each file status when listing a
     directory.  (hairong)
 
-    HDFS-1359. Add BlockPoolID to Block. (suresh)
-
     HDFS-1361. Add -fileStatus operation to NNThroughputBenchmark. (shv)
 
     HDFS-1435. Provide an option to store fsimage compressed. (hairong)
@@ -962,6 +1011,13 @@ Release 0.22.0 - Unreleased
     HDFS-1948  Forward port 'hdfs-1520 lightweight namenode operation to
     trigger lease recovery' (stack)
 
+    HDFS-1954. Improved corrupt files warning on NameNode web UI.
+    (Patrick Hunt via shv)
+
+    HDFS-1409. BackupNode registration throwing  
+    UnsupportedActionException("register") instead of "journal".
+    (Ching-Shen Chen via shv)
+
   OPTIMIZATIONS
 
     HDFS-1140. Speedup INode.getPathComponents. (Dmytro Molkov via shv)
@@ -980,6 +1036,9 @@ Release 0.22.0 - Unreleased
     HDFS-1434. Refactor Datanode#startDataNode method into smaller methods.
     (suresh)
 
+    HDFS-941. The DFS client should cache and reuse open sockets to datanodes
+    while performing reads. (bc Wong and Todd Lipcon via todd)
+
   BUG FIXES
 
     HDFS-1039. Adding test for  JspHelper.getUGI(jnp via boryas)
@@ -1177,6 +1236,11 @@ Release 0.22.0 - Unreleased
     HDFS-2039. TestNameNodeMetrics uses a bad test root path, preventing it
     from running inside Eclipse. (todd)
 
+    HDFS-988. saveNamespace race can corrupt the edits log. (eli)
+
+    HDFS-2071. Use of isConnected() in DataXceiver is invalid. (Kihwal Lee
+    via todd)
+
 Release 0.21.1 - Unreleased
     HDFS-1466. TestFcHdfsSymlink relies on /tmp/test not existing. (eli)
 
@@ -1372,8 +1436,6 @@ Release 0.21.0 - 2010-08-13
     error message on the screen when cat a directory or a 
     non-existent file. (hairong)
 
-    HDFS-1439. HDFS Federation: Fix compilation error in TestFiHftp. (suresh)
-
   NEW FEATURES
 
     HDFS-1134. Large-scale Automated Framework. (cos)

+ 12 - 0
hdfs/bin/hdfs

@@ -125,6 +125,18 @@ if [ -d "$HADOOP_HDFS_HOME/build/tools" ]; then
   CLASSPATH=${CLASSPATH}:$HADOOP_HDFS_HOME/build/tools
 fi
 
+if [ -d "$HADOOP_HDFS_HOME/build/ivy/lib/hadoop-hdfs/common" ]; then
+  for f in $HADOOP_HDFS_HOME/build/ivy/lib/hadoop-hdfs/common/*.jar; do
+    CLASSPATH=${CLASSPATH}:$f;
+  done
+fi
+
+if [ -d "$HADOOP_HDFS_HOME/build/ivy/lib/hadoop-hdfs/hdfs" ]; then
+  for f in $HADOOP_HDFS_HOME/build/ivy/lib/hadoop-hdfs/hdfs/*.jar; do
+    CLASSPATH=${CLASSPATH}:$f;
+  done
+fi
+
 # for releases, add core hdfs jar & webapps to CLASSPATH
 if [ -d "$HADOOP_PREFIX/share/hadoop/hdfs/webapps" ]; then
   CLASSPATH=${CLASSPATH}:$HADOOP_PREFIX/share/hadoop/hdfs

+ 36 - 2
hdfs/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}" />
 
@@ -644,6 +648,7 @@
         <sysproperty key="java.security.krb5.conf" value="${test.src.dir}/krb5.conf"/>
         <sysproperty key="hadoop.policy.file" value="hadoop-policy.xml"/>
         <sysproperty key="hdfs.rpc.engine" value="${test.hdfs.rpc.engine}"/>
+        <sysproperty key="java.security.egd" value="file:///dev/urandom" />
         <classpath refid="@{classpath}"/>
         <!-- Pass probability specifications to the spawn JVM -->
         <syspropertyset id="FaultProbabilityProperties">
@@ -923,7 +928,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 +950,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 +1374,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*" />
@@ -1449,6 +1460,7 @@
         <env key="JVM_ARCH" value="${jvm.arch}"/>
         <env key="LIBHDFS_BUILD_DIR" value="${build.c++.libhdfs}"/>
         <env key="HADOOP_PREFIX" value="${basedir}"/>
+        <env key="HADOOP_HDFS_HOME" value="${basedir}"/>
         <env key="HADOOP_CONF_DIR" value="${test.libhdfs.dir}/conf"/>
         <env key="HADOOP_LOG_DIR" value="${test.libhdfs.dir}/logs"/>
         <env key="LIBHDFS_TEST_DIR" value="${test.libhdfs.dir}"/>
@@ -1456,6 +1468,7 @@
         <env key="LIBHDFS_INSTALL_DIR" value="${install.c++}/lib"/>  
         <env key="LIB_DIR" value="${common.ivy.lib.dir}"/>
         <env key="CLOVER_JAR" value="${clover.jar}"/>
+        <env key="HADOOP_VERSION" value="${version}"/>
 		<arg value="test"/>
     </exec>
   </target>
@@ -1672,6 +1685,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}" />

+ 1 - 0
hdfs/ivy.xml

@@ -68,6 +68,7 @@
     <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.guava" name="guava" rev="${guava.version}" conf="hdfs->default" />
+    <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"/>

+ 33 - 1
hdfs/src/c++/libhdfs/hdfs.c

@@ -259,7 +259,7 @@ hdfsFS hdfsConnectAsUser(const char* host, tPort port, const char *user)
       sprintf(cURI, "hdfs://%s:%d", host, (int)(port));
       if (cURI == NULL) {
         fprintf (stderr, "Couldn't allocate an object of size %d",
-                 strlen(host) + 16);
+                 (int)(strlen(host) + 16));
         errno = EINTERNAL;			
         goto done;	
       }
@@ -1006,6 +1006,38 @@ int hdfsFlush(hdfsFS fs, hdfsFile f)
 
 
 
+int hdfsHFlush(hdfsFS fs, hdfsFile f)
+{
+    //Get the JNIEnv* corresponding to current thread
+    JNIEnv* env = getJNIEnv();
+    if (env == NULL) {
+      errno = EINTERNAL;
+      return -1;
+    }
+
+    //Parameters
+    jobject jOutputStream = (jobject)(f ? f->file : 0);
+
+    //Caught exception
+    jthrowable jExc = NULL;
+
+    //Sanity check
+    if (!f || f->type != OUTPUT) {
+        errno = EBADF;
+        return -1;
+    }
+
+    if (invokeMethod(env, NULL, &jExc, INSTANCE, jOutputStream,
+                     HADOOP_OSTRM, "hflush", "()V") != 0) {
+        errno = errnoFromException(jExc, env, HADOOP_OSTRM "::hflush");
+        return -1;
+    }
+
+    return 0;
+}
+
+
+
 int hdfsAvailable(hdfsFS fs, hdfsFile f)
 {
     // JAVA EQUIVALENT

+ 10 - 0
hdfs/src/c++/libhdfs/hdfs.h

@@ -239,6 +239,16 @@ extern  "C" {
     int hdfsFlush(hdfsFS fs, hdfsFile file);
 
 
+    /**
+     * hdfsHFlush - Flush out the data in client's user buffer. After the
+     * return of this call, new readers will see the data.
+     * @param fs configured filesystem handle
+     * @param file file handle
+     * @return 0 on success, -1 on error and sets errno
+     */
+    int hdfsHFlush(hdfsFS fs, hdfsFile file);
+
+
     /**
      * hdfsAvailable - Number of bytes that can be read from this
      * input stream without blocking.

+ 6 - 0
hdfs/src/c++/libhdfs/hdfs_test.c

@@ -95,6 +95,12 @@ int main(int argc, char **argv) {
         }
         fprintf(stderr, "Flushed %s successfully!\n", writePath); 
 
+        if (hdfsHFlush(fs, writeFile)) {
+            fprintf(stderr, "Failed to 'hflush' %s\n", writePath);
+            exit(-1);
+        }
+        fprintf(stderr, "HFlushed %s successfully!\n", writePath);
+
         hdfsCloseFile(fs, writeFile);
     }
 

+ 4 - 2
hdfs/src/c++/libhdfs/tests/test-libhdfs.sh

@@ -26,6 +26,8 @@
 # f) LIBHDFS_INSTALL_DIR
 # g) OS_NAME
 # h) CLOVER_JAR
+# i} HADOOP_VERSION
+# j) HADOOP_HDFS_HOME
 # All these are passed by build.xml.
 #
 
@@ -33,8 +35,8 @@ HDFS_TEST=hdfs_test
 HADOOP_LIB_DIR=$HADOOP_PREFIX/lib
 HADOOP_BIN_DIR=$HADOOP_PREFIX/bin
 
-COMMON_BUILD_DIR=$HADOOP_PREFIX/build/ivy/lib/Hadoop-Hdfs/common
-COMMON_JAR=$COMMON_BUILD_DIR/hadoop-common-0.22.0-SNAPSHOT.jar
+COMMON_BUILD_DIR=$HADOOP_PREFIX/build/ivy/lib/hadoop-hdfs/common
+COMMON_JAR=$COMMON_BUILD_DIR/hadoop-common-$HADOOP_VERSION.jar
 
 cat > $HADOOP_CONF_DIR/core-site.xml <<EOF
 <?xml version="1.0"?>

+ 4 - 1
hdfs/src/contrib/build-contrib.xml

@@ -43,6 +43,8 @@
   <property name="test.output" value="no"/>
   <property name="test.timeout" value="900000"/>
   <property name="build.dir" location="${hadoop.root}/build/contrib/${name}"/>
+  <property name="build.webapps.root.dir" value="${hadoop.root}/build/web"/>
+  <property name="build.webapps" value="${build.webapps.root.dir}/webapps"/>
   <property name="build.classes" location="${build.dir}/classes"/>
   <!-- NB: sun.arch.data.model is not supported on all platforms -->
   <property name="build.platform"
@@ -96,7 +98,8 @@
 
   <!-- the unit test classpath -->
   <path id="test.classpath">
-    <pathelement location="${build.test}" />
+    <pathelement location="${build.test}"/>
+    <pathelement location="${build.webapps.root.dir}"/>
     <pathelement location="${hadoop.root}/build/test/core/classes"/>
     <pathelement location="${hadoop.root}/build/test/hdfs/classes"/>
     <pathelement location="${hadoop.root}/build/test/mapred/classes"/>

+ 5 - 1
hdfs/src/contrib/build.xml

@@ -28,7 +28,7 @@
   <!-- ====================================================== -->
   <target name="compile">
     <subant target="compile">
-      <fileset dir="." includes="hdfsproxy/build.xml"/>
+      <fileset dir="." includes="*/build.xml"/>
     </subant>
   </target>
   
@@ -45,6 +45,10 @@
   <!-- Test all the contribs.                               -->
   <!-- ====================================================== -->
   <target name="test">
+    <subant target="test">
+      <fileset dir="." includes="fuse-dfs/build.xml"/>
+    </subant> 
+
     <!-- hdfsproxy tests failing due to HDFS-1666
     <subant target="test">
       <fileset dir="." includes="hdfsproxy/build.xml"/>

+ 16 - 21
hdfs/src/contrib/fuse-dfs/build.xml

@@ -29,11 +29,7 @@
     </fail>
   </target>
 
-  <target name="compile">
-    <condition property="perms" value="1" else="0">
-      <not> <isset property="libhdfs.noperms"/> </not>
-    </condition>
-
+  <target name="compile" if="fusedfs">
     <exec executable="autoreconf" dir="${basedir}" 
           searchpath="yes" failonerror="yes">
        <arg value="-if"/>
@@ -49,7 +45,18 @@
       <env key="HADOOP_PREFIX" value="${hadoop.root}"/>
       <env key="PACKAGE_VERSION" value="0.1.0"/>
       <env key="BUILD_PLATFORM" value="${build.platform}" />
-      <env key="PERMS" value="${perms}"/>
+    </exec>
+
+    <mkdir dir="${build.dir}"/>
+    <mkdir dir="${build.dir}/test"/>
+
+    <!-- Use exec since the copy task doesn't preserve attrs -->
+    <exec executable="cp" failonerror="true">
+      <arg line="${hadoop.root}/src/contrib/fuse-dfs/src/fuse_dfs ${build.dir}"/>
+    </exec>
+
+    <exec executable="cp" failonerror="true">
+      <arg line="${hadoop.root}/src/contrib/fuse-dfs/src/fuse_dfs_wrapper.sh ${build.dir}"/>
     </exec>
   </target>
 
@@ -57,8 +64,6 @@
   <target name="package" />
 
   <target name="compile-test" depends="ivy-retrieve-common, check-libhdfs-exists" if="fusedfs">
-    <mkdir dir="${build.dir}"/>
-    <mkdir dir="${build.dir}/test"/>
     <javac encoding="${build.encoding}"
 	   srcdir="${src.test}"
 	   includes="**/*.java"
@@ -66,28 +71,18 @@
 	   debug="${javac.debug}">
       <classpath refid="test.classpath"/>
     </javac>
-
-    <!-- Use exec since the copy task doesn't preserve attrs -->
-    <exec executable="cp" failonerror="true">
-      <arg line="${hadoop.root}/src/contrib/fuse-dfs/src/fuse_dfs ${build.dir}"/>
-    </exec>
-
-    <mkdir dir="${build.dir}/test"/>
-
-    <exec executable="cp" failonerror="true">
-      <arg line="${hadoop.root}/src/contrib/fuse-dfs/src/fuse_dfs_wrapper.sh ${build.dir}"/>
-    </exec>
   </target>
 
   <target name="test" depends="compile-test,check-libhdfs-exists" if="fusedfs">
-    <junit showoutput="${test.output}" fork="yes" printsummary="yes" errorProperty="tests.failed" haltonfailure="no" failureProperty="tests.failed">
+    <junit showoutput="${test.output}" fork="yes" printsummary="yes"
+           errorProperty="tests.failed" haltonfailure="no" failureProperty="tests.failed">
       <classpath refid="test.classpath"/>
       <sysproperty key="test.build.data" value="${build.test}/data"/>
       <sysproperty key="build.test" value="${build.test}"/>
       <sysproperty key="user.dir" value="${build.test}/data"/>
       <sysproperty key="hadoop.log.dir" value="${hadoop.log.dir}"/>
       <sysproperty key="test.src.dir" value="${test.src.dir}"/>
-      <formatter type="plain" />
+      <formatter type="${test.junit.output.format}" />
       <batchtest todir="${build.test}" unless="testcase">
         <fileset dir="${src.test}">
           <include name="**/Test*.java"/>

+ 3 - 0
hdfs/src/contrib/fuse-dfs/configure.ac

@@ -42,6 +42,9 @@ AC_PREFIX_DEFAULT([`pwd`])
  DEFS=""
 AC_SUBST([DEFS])
 
+# Need GNU source for multiple hashtables from glibc
+AC_GNU_SOURCE
+
 AC_FUNC_GETGROUPS
 AC_TYPE_GETGROUPS
 

+ 88 - 19
hdfs/src/contrib/fuse-dfs/src/fuse_connect.c

@@ -17,37 +17,106 @@
  */
 
 #include "hdfs.h"
-
 #include "fuse_dfs.h"
 #include "fuse_connect.h"
 #include "fuse_users.h" 
 
+#include <search.h>
 
-#if PERMS
+#define MAX_ELEMENTS (16 * 1024)
+static struct hsearch_data *fsTable = NULL;
+static pthread_mutex_t tableMutex = PTHREAD_MUTEX_INITIALIZER;
 
-/**
- * Connects to the NN as the current user/group according to FUSE
- *
+/*
+ * Allocate a hash table for fs handles. Returns 0 on success,
+ * -1 on failure.
  */
-hdfsFS doConnectAsUser(const char *hostname, int port) {
-  uid_t uid = fuse_get_context()->uid;
+int allocFsTable(void) {
+  assert(NULL == fsTable);
+  fsTable = calloc(1, sizeof(struct hsearch_data));
+  if (0 == hcreate_r(MAX_ELEMENTS, fsTable)) {
+    ERROR("Unable to initialize connection table");
+    return -1;
+  }
+  return 0;
+}
 
-  char *user = getUsername(uid);
-  if (NULL == user)
+/*
+ * Find a fs handle for the given key. Returns a fs handle, 
+ * or NULL if there is no fs for the given key.
+ */
+static hdfsFS findFs(char *key) {
+  ENTRY entry;
+  ENTRY *entryP = NULL;
+  entry.key = key;
+  if (0 == hsearch_r(entry, FIND, &entryP, fsTable)) {
     return NULL;
-  int numgroups = 0;
-  char **groups = getGroups(uid, &numgroups);
-  hdfsFS fs = hdfsConnectAsUser(hostname, port, user);
-  freeGroups(groups, numgroups);
-  if (user) 
-    free(user);
-  return fs;
+  }
+  assert(NULL != entryP->data);
+  return (hdfsFS)entryP->data;
 }
 
-#else
+/*
+ * Insert the given fs handle into the table.
+ * Returns 0 on success, -1 on failure.
+ */
+static int insertFs(char *key, hdfsFS fs) {
+  ENTRY entry;
+  ENTRY *entryP = NULL;
+  assert(NULL != fs);
+  entry.key = strdup(key);
+  if (entry.key == NULL) {
+    return -1;
+  }
+  entry.data = (void*)fs;
+  if (0 == hsearch_r(entry, ENTER, &entryP, fsTable)) {
+    return -1;
+  }
+  return 0;
+}
 
+/*
+ * Connect to the NN as the current user/group.
+ * Returns a fs handle on success, or NULL on failure.
+ */
 hdfsFS doConnectAsUser(const char *hostname, int port) {
-  return hdfsConnect(hostname, port);
+  uid_t uid = fuse_get_context()->uid;
+  char *user = getUsername(uid);
+  int ret;
+  hdfsFS fs = NULL;
+  if (NULL == user) {
+    goto done;
+  }
+
+  ret = pthread_mutex_lock(&tableMutex);
+  assert(0 == ret);
+
+  fs = findFs(user);
+  if (NULL == fs) {
+    fs = hdfsConnectAsUserNewInstance(hostname, port, user);
+    if (NULL == fs) {
+      ERROR("Unable to create fs for user %s", user);
+      goto done;
+    }
+    if (-1 == insertFs(user, fs)) {
+      ERROR("Unable to cache fs for user %s", user);
+    }
+  }
+
+done:
+  ret = pthread_mutex_unlock(&tableMutex);
+  assert(0 == ret);
+  if (user) {
+    free(user);
+  }
+  return fs;
 }
 
-#endif
+/*
+ * We currently cache a fs handle per-user in this module rather
+ * than use the FileSystem cache in the java client. Therefore
+ * we do not disconnect the fs handle here.
+ */
+int doDisconnect(hdfsFS fs) {
+  return 0;
+}

+ 3 - 7
hdfs/src/contrib/fuse-dfs/src/fuse_connect.h

@@ -21,12 +21,8 @@
 
 #include "fuse_dfs.h"
 
-/**
- * Connects to the NN as the current user/group according to FUSE.
- * Uses the fuse context to get the user name and groups.
- * (if hadoop pre 0.19, will ignore user and group).
- */
-
-hdfsFS doConnectAsUser(const char *hostname, int port) ;
+hdfsFS doConnectAsUser(const char *hostname, int port);
+int doDisconnect(hdfsFS fs);
+int allocFsTable(void);
 
 #endif

+ 0 - 1
hdfs/src/contrib/fuse-dfs/src/fuse_context_handle.h

@@ -33,7 +33,6 @@ typedef struct dfs_context_struct {
   int debug;
   char *nn_hostname;
   int nn_port;
-  hdfsFS fs;
   int read_only;
   int usetrash;
   int direct_io;

+ 34 - 44
hdfs/src/contrib/fuse-dfs/src/fuse_dfs.c

@@ -20,7 +20,7 @@
 #include "fuse_options.h"
 #include "fuse_impls.h"
 #include "fuse_init.h"
-
+#include "fuse_connect.h"
 
 int is_protected(const char *path) {
 
@@ -38,55 +38,49 @@ int is_protected(const char *path) {
 }
 
 static struct fuse_operations dfs_oper = {
-  .getattr	= dfs_getattr,
-  .access	= dfs_access,
-  .readdir	= dfs_readdir,
-  .destroy       = dfs_destroy,
-  .init         = dfs_init,
-  .open	        = dfs_open,
-  .read	        = dfs_read,
-  .symlink	= dfs_symlink,
-  .statfs	= dfs_statfs,
-  .mkdir	= dfs_mkdir,
-  .rmdir	= dfs_rmdir,
-  .rename	= dfs_rename,
-  .unlink       = dfs_unlink,
-  .release      = dfs_release,
-  .create       = dfs_create,
-  .write	= dfs_write,
-  .flush        = dfs_flush,
-  .mknod        = dfs_mknod,
-  .utimens      = dfs_utimens,
-  .chmod	= dfs_chmod,
-  .chown	= dfs_chown,
-  .truncate	= dfs_truncate,
+  .getattr  = dfs_getattr,
+  .access   = dfs_access,
+  .readdir  = dfs_readdir,
+  .destroy  = dfs_destroy,
+  .init     = dfs_init,
+  .open     = dfs_open,
+  .read     = dfs_read,
+  .symlink  = dfs_symlink,
+  .statfs   = dfs_statfs,
+  .mkdir    = dfs_mkdir,
+  .rmdir    = dfs_rmdir,
+  .rename   = dfs_rename,
+  .unlink   = dfs_unlink,
+  .release  = dfs_release,
+  .create   = dfs_create,
+  .write    = dfs_write,
+  .flush    = dfs_flush,
+  .mknod    = dfs_mknod,
+  .utimens  = dfs_utimens,
+  .chmod    = dfs_chmod,
+  .chown    = dfs_chown,
+  .truncate = dfs_truncate,
 };
 
-
 int main(int argc, char *argv[])
 {
-
   umask(0);
 
   extern const char *program;  
   program = argv[0];
   struct fuse_args args = FUSE_ARGS_INIT(argc, argv);
 
-  /* clear structure that holds our options */
   memset(&options, 0, sizeof(struct options));
 
-  // some defaults
   options.rdbuffer_size = 10*1024*1024; 
   options.attribute_timeout = 60; 
   options.entry_timeout = 60;
 
-  if (fuse_opt_parse(&args, &options, dfs_opts, dfs_options) == -1)
-    /** error parsing options */
+  if (-1 == fuse_opt_parse(&args, &options, dfs_opts, dfs_options)) {
     return -1;
+  }
 
-
-  // Some fuse options we set
-  if (! options.private) {
+  if (!options.private) {
     fuse_opt_add_arg(&args, "-oallow_other");
   }
 
@@ -109,28 +103,24 @@ int main(int argc, char *argv[])
     exit(0);
   }
 
-
-  // 
-  // Check we can connect to hdfs
-  // 
+  // Check connection as root
   if (options.initchecks == 1) {
-    hdfsFS temp;
-    if ((temp = hdfsConnect(options.server, options.port)) == NULL) {
+    hdfsFS tempFS = hdfsConnectAsUser(options.server, options.port, "root");
+    if (NULL == tempFS) {
       const char *cp = getenv("CLASSPATH");
       const char *ld = getenv("LD_LIBRARY_PATH");
       ERROR("FATAL: misconfiguration - cannot connect to HDFS");
       ERROR("LD_LIBRARY_PATH=%s",ld == NULL ? "NULL" : ld);
       ERROR("CLASSPATH=%s",cp == NULL ? "NULL" : cp);
-      exit(0);
+      exit(1);
+    }
+    if (doDisconnect(tempFS)) {
+      ERROR("FATAL: unable to disconnect from test filesystem.");
+      exit(1);
     }
   }
 
   int ret = fuse_main(args.argc, args.argv, &dfs_oper, NULL);
-
-  if (ret) printf("\n");
-
-  /** free arguments */
   fuse_opt_free_args(&args);
-
   return ret;
 }

+ 0 - 5
hdfs/src/contrib/fuse-dfs/src/fuse_dfs.h

@@ -35,11 +35,6 @@
 #include <config.h>
 #endif
 
-#ifdef linux
-/* For pread()/pwrite() */
-#define _XOPEN_SOURCE 500
-#endif
-
 #ifdef HAVE_SETXATTR
 #include <sys/xattr.h>
 #endif

+ 2 - 2
hdfs/src/contrib/fuse-dfs/src/fuse_dfs_wrapper.sh

@@ -29,7 +29,7 @@ export  JAVA_HOME=/usr/local/java
 fi
 
 if [ "$LD_LIBRARY_PATH" = "" ]; then
-export LD_LIBRARY_PATH=$JAVA_HOME/jre/lib/$OS_ARCH/server:/usr/local/share/hdfs/libhdfs/:/usr/local/lib
+export LD_LIBRARY_PATH=$JAVA_HOME/jre/lib/$OS_ARCH/server:/usr/local/lib
 fi
 
 # If dev build set paths accordingly
@@ -38,7 +38,7 @@ if [ -d $HADOOP_PREFIX/build ]; then
   for f in ${HADOOP_PREFIX}/build/*.jar ; do
     export CLASSPATH=$CLASSPATH:$f
   done
-  for f in $HADOOP_PREFIX/build/ivy/lib/Hadoop-Hdfs/common/*.jar ; do
+  for f in $HADOOP_PREFIX/build/ivy/lib/hadoop-hdfs/common/*.jar ; do
     export CLASSPATH=$CLASSPATH:$f
   done
   export PATH=$HADOOP_PREFIX/build/contrib/fuse-dfs:$PATH

+ 2 - 16
hdfs/src/contrib/fuse-dfs/src/fuse_impls_access.c

@@ -23,21 +23,7 @@
 int dfs_access(const char *path, int mask)
 {
   TRACE1("access", path)
-  // bugbug - I think we need the FileSystemAPI/libhdfs to expose this!
-  // retrieve dfs specific data
-  dfs_context *dfs = (dfs_context*)fuse_get_context()->private_data;
-
-  // check params and the context var
-  assert(dfs);
-  assert(path);
-
-  hdfsFS userFS;
-  if ((userFS = doConnectAsUser(dfs->nn_hostname,dfs->nn_port)) == NULL) {
-    ERROR("Could not connect to HDFS");
-    return -EIO;
-  }
-  //  return hdfsAccess(userFS, path, mask);
+  assert(path != NULL);
+  // TODO: HDFS-428
   return 0;
 }
-
-

+ 14 - 11
hdfs/src/contrib/fuse-dfs/src/fuse_impls_chmod.c

@@ -24,27 +24,30 @@
 int dfs_chmod(const char *path, mode_t mode)
 {
   TRACE1("chmod", path)
-
-#if PERMS
-  // retrieve dfs specific data
+  int ret = 0;
   dfs_context *dfs = (dfs_context*)fuse_get_context()->private_data;
 
-  // check params and the context var
   assert(path);
   assert(dfs);
   assert('/' == *path);
 
-  hdfsFS userFS;
-  // if not connected, try to connect and fail out if we can't.
-  if ((userFS = doConnectAsUser(dfs->nn_hostname,dfs->nn_port))== NULL) {
+  hdfsFS userFS = doConnectAsUser(dfs->nn_hostname, dfs->nn_port);
+  if (userFS == NULL) {
     ERROR("Could not connect to HDFS");
-    return -EIO;
+    ret = -EIO;
+    goto cleanup;
   }
 
   if (hdfsChmod(userFS, path, (short)mode)) {
     ERROR("Could not chmod %s to %d", path, (int)mode);
-    return -EIO;
+    ret = (errno > 0) ? -errno : -EIO;
+    goto cleanup;
+  }
+
+cleanup:
+  if (doDisconnect(userFS)) {
+    ret = -EIO;
   }
-#endif
-  return 0;
+
+  return ret;
 }

+ 26 - 23
hdfs/src/contrib/fuse-dfs/src/fuse_impls_chown.c

@@ -26,10 +26,9 @@
   TRACE1("chown", path)
 
   int ret = 0;
-
-#if PERMS
   char *user = NULL;
   char *group = NULL;
+  hdfsFS userFS = NULL;
 
   // retrieve dfs specific data
   dfs_context *dfs = (dfs_context*)fuse_get_context()->private_data;
@@ -43,36 +42,40 @@
   if (NULL == user) {
     ERROR("Could not lookup the user id string %d",(int)uid); 
     ret = -EIO;
+    goto cleanup;
   }
 
-  if (0 == ret) {
-    group = getGroup(gid);
-    if (group == NULL) {
-      ERROR("Could not lookup the group id string %d",(int)gid);
-      ret = -EIO;
-    } 
+  group = getGroup(gid);
+  if (group == NULL) {
+    ERROR("Could not lookup the group id string %d",(int)gid);
+    ret = -EIO;
+    goto cleanup;
+  } 
+
+  userFS = doConnectAsUser(dfs->nn_hostname, dfs->nn_port);
+  if (userFS == NULL) {
+    ERROR("Could not connect to HDFS");
+    ret = -EIO;
+    goto cleanup;
   }
 
-  hdfsFS userFS = NULL;
-  if (0 == ret) {
-    // if not connected, try to connect and fail out if we can't.
-    if ((userFS = doConnectAsUser(dfs->nn_hostname,dfs->nn_port))== NULL) {
-      ERROR("Could not connect to HDFS");
-      ret = -EIO;
-    }
+  if (hdfsChown(userFS, path, user, group)) {
+    ERROR("Could not chown %s to %d:%d", path, (int)uid, gid);
+    ret = (errno > 0) ? -errno : -EIO;
+    goto cleanup;
   }
 
-  if (0 == ret) {
-    if (hdfsChown(userFS, path, user, group)) {
-      ERROR("Could not chown %s to %d:%d", path, (int)uid, gid);
-      ret = -EIO;
-    }
+cleanup:
+  if (userFS && doDisconnect(userFS)) {
+    ret = -EIO;
   }
-  if (user) 
+  if (user) {
     free(user);
-  if (group)
+  }
+  if (group) {
     free(group);
-#endif
+  }
+
   return ret;
 
 }

+ 14 - 12
hdfs/src/contrib/fuse-dfs/src/fuse_impls_getattr.c

@@ -19,39 +19,36 @@
 #include "fuse_dfs.h"
 #include "fuse_impls.h"
 #include "fuse_stat_struct.h"
+#include "fuse_connect.h"
 
 int dfs_getattr(const char *path, struct stat *st)
 {
   TRACE1("getattr", path)
 
-  // retrieve dfs specific data
   dfs_context *dfs = (dfs_context*)fuse_get_context()->private_data;
 
-  // check params and the context var
   assert(dfs);
   assert(path);
   assert(st);
 
-  // if not connected, try to connect and fail out if we can't.
-  if (NULL == dfs->fs && 
-      NULL == (dfs->fs = hdfsConnect(dfs->nn_hostname,dfs->nn_port))) {
+  hdfsFS fs = doConnectAsUser(dfs->nn_hostname,dfs->nn_port);
+  if (NULL == fs) {
     ERROR("Could not connect to %s:%d", dfs->nn_hostname, dfs->nn_port);
     return -EIO;
   }
 
-  // call the dfs API to get the actual information
-  hdfsFileInfo *info = hdfsGetPathInfo(dfs->fs,path);
-
+  int ret = 0;
+  hdfsFileInfo *info = hdfsGetPathInfo(fs,path);
   if (NULL == info) {
-    return -ENOENT;
+    ret = -ENOENT;
+    goto cleanup;
   }
-
   fill_stat_structure(&info[0], st);
 
   // setup hard link info - for a file it is 1 else num entries in a dir + 2 (for . and ..)
   if (info[0].mKind == kObjectKindDirectory) {
     int numEntries = 0;
-    hdfsFileInfo *info = hdfsListDirectory(dfs->fs,path,&numEntries);
+    hdfsFileInfo *info = hdfsListDirectory(fs,path,&numEntries);
 
     if (info) {
       hdfsFreeFileInfo(info,numEntries);
@@ -65,5 +62,10 @@ int dfs_getattr(const char *path, struct stat *st)
   // free the info pointer
   hdfsFreeFileInfo(info,1);
 
-  return 0;
+cleanup:
+  if (doDisconnect(fs)) {
+    ERROR("Could not disconnect from filesystem");
+    ret = -EIO;
+  }
+  return ret;
 }

+ 12 - 10
hdfs/src/contrib/fuse-dfs/src/fuse_impls_mkdir.c

@@ -25,10 +25,8 @@ int dfs_mkdir(const char *path, mode_t mode)
 {
   TRACE1("mkdir", path)
 
-  // retrieve dfs specific data
   dfs_context *dfs = (dfs_context*)fuse_get_context()->private_data;
 
-  // check params and the context var
   assert(path);
   assert(dfs);
   assert('/' == *path);
@@ -43,25 +41,29 @@ int dfs_mkdir(const char *path, mode_t mode)
     return -EACCES;
   }
   
-  hdfsFS userFS;
-  // if not connected, try to connect and fail out if we can't.
-  if ((userFS = doConnectAsUser(dfs->nn_hostname,dfs->nn_port))== NULL) {
+  hdfsFS userFS = doConnectAsUser(dfs->nn_hostname, dfs->nn_port);
+  if (userFS == NULL) {
     ERROR("Could not connect");
     return -EIO;
   }
 
   // In theory the create and chmod should be atomic.
 
+  int ret = 0;
   if (hdfsCreateDirectory(userFS, path)) {
     ERROR("HDFS could not create directory %s", path);
-    return -EIO;
+    ret = (errno > 0) ? -errno : -EIO;
+    goto cleanup;
   }
 
-#if PERMS
   if (hdfsChmod(userFS, path, (short)mode)) {
     ERROR("Could not chmod %s to %d", path, (int)mode);
-    return -EIO;
+    ret = (errno > 0) ? -errno : -EIO;
+  }
+
+cleanup:
+  if (doDisconnect(userFS)) {
+    ret = -EIO;
   }
-#endif
-  return 0;
+  return ret;
 }

+ 6 - 13
hdfs/src/contrib/fuse-dfs/src/fuse_impls_open.c

@@ -39,19 +39,20 @@ int dfs_open(const char *path, struct fuse_file_info *fi)
   int flags = (fi->flags & 0x7FFF);
 
   // retrieve dfs specific data
-  dfs_fh *fh = (dfs_fh*)malloc(sizeof (dfs_fh));
+  dfs_fh *fh = (dfs_fh*)calloc(1, sizeof (dfs_fh));
   if (fh == NULL) {
     ERROR("Malloc of new file handle failed");
     return -EIO;
   }
 
-  if ((fh->fs = doConnectAsUser(dfs->nn_hostname,dfs->nn_port)) == NULL) {
+  fh->fs = doConnectAsUser(dfs->nn_hostname, dfs->nn_port);
+  if (fh->fs == NULL) {
     ERROR("Could not connect to dfs");
     return -EIO;
   }
 
   if (flags & O_RDWR) {
-    hdfsFileInfo *info = hdfsGetPathInfo(dfs->fs,path);
+    hdfsFileInfo *info = hdfsGetPathInfo(fh->fs,path);
     if (info == NULL) {
       // File does not exist (maybe?); interpret it as a O_WRONLY
       // If the actual error was something else, we'll get it again when
@@ -73,28 +74,20 @@ int dfs_open(const char *path, struct fuse_file_info *fi)
     return -errno;
   }
 
-  // 
-  // mutex needed for reads/writes
-  //
   pthread_mutex_init(&fh->mutex, NULL);
 
   if (fi->flags & O_WRONLY || fi->flags & O_CREAT) {
-    // write specific initialization
     fh->buf = NULL;
   } else  {
-    // read specific initialization
-
     assert(dfs->rdbuffer_size > 0);
-
-    if (NULL == (fh->buf = (char*)malloc(dfs->rdbuffer_size*sizeof (char)))) {
+    fh->buf = (char*)malloc(dfs->rdbuffer_size * sizeof(char));
+    if (NULL == fh->buf) {
       ERROR("Could not allocate memory for a read for file %s\n", path);
       ret = -EIO;
     }
-
     fh->buffersStartOffset = 0;
     fh->bufferSize = 0;
   }
-
   fi->fh = (uint64_t)fh;
 
   return ret;

+ 13 - 16
hdfs/src/contrib/fuse-dfs/src/fuse_impls_readdir.c

@@ -24,24 +24,15 @@
 int dfs_readdir(const char *path, void *buf, fuse_fill_dir_t filler,
                        off_t offset, struct fuse_file_info *fi)
 {
-  TRACE1("readdir",path)
-
-  (void) offset;
-  (void) fi;
-
-  // retrieve dfs specific data
+  TRACE1("readdir", path)
   dfs_context *dfs = (dfs_context*)fuse_get_context()->private_data;
 
-  // check params and the context var
   assert(dfs);
   assert(path);
   assert(buf);
 
-  int path_len = strlen(path);
-
-  hdfsFS userFS;
-  // if not connected, try to connect and fail out if we can't.
-  if ((userFS = doConnectAsUser(dfs->nn_hostname,dfs->nn_port))== NULL) {
+  hdfsFS userFS = doConnectAsUser(dfs->nn_hostname, dfs->nn_port);
+  if (userFS == NULL) {
     ERROR("Could not connect");
     return -EIO;
   }
@@ -50,16 +41,16 @@ int dfs_readdir(const char *path, void *buf, fuse_fill_dir_t filler,
   // component (HDFS-975) would save us from parsing it out below.
   int numEntries = 0;
   hdfsFileInfo *info = hdfsListDirectory(userFS, path, &numEntries);
-  userFS = NULL;
 
+  int ret = 0;
   // NULL means either the directory doesn't exist or maybe IO error.
   if (NULL == info) {
-    return -ENOENT;
+    ret = (errno > 0) ? -errno : -ENOENT;
+    goto cleanup;
   }
 
   int i ;
   for (i = 0; i < numEntries; i++) {
-
     if (NULL == info[i].mName) {
       ERROR("Path %s info[%d].mName is NULL", path, i);
       continue;
@@ -115,5 +106,11 @@ int dfs_readdir(const char *path, void *buf, fuse_fill_dir_t filler,
     }
   // free the info pointers
   hdfsFreeFileInfo(info,numEntries);
-  return 0;
+
+cleanup:
+  if (doDisconnect(userFS)) {
+    ret = -EIO;
+    ERROR("Failed to disconnect %d", errno);
+  }
+  return ret;
 }

+ 6 - 0
hdfs/src/contrib/fuse-dfs/src/fuse_impls_release.c

@@ -19,6 +19,7 @@
 #include "fuse_dfs.h"
 #include "fuse_impls.h"
 #include "fuse_file_handle.h"
+#include "fuse_connect.h"
 
 /**
  * This mutex is to protect releasing a file handle in case the user calls close in different threads
@@ -63,6 +64,11 @@ int dfs_release (const char *path, struct fuse_file_info *fi) {
     if (fh->buf != NULL) {
       free(fh->buf);
     }
+
+    if (doDisconnect(fh->fs)) {
+      ret = -EIO;
+    }
+
     // this is always created and initialized, so always destroy it. (see dfs_open)
     pthread_mutex_destroy(&fh->mutex);
 

+ 11 - 5
hdfs/src/contrib/fuse-dfs/src/fuse_impls_rename.c

@@ -46,17 +46,23 @@ int dfs_rename(const char *from, const char *to)
     return -EACCES;
   }
 
-  hdfsFS userFS;
-  // if not connected, try to connect and fail out if we can't.
-  if ((userFS = doConnectAsUser(dfs->nn_hostname,dfs->nn_port))== NULL) {
+  hdfsFS userFS = doConnectAsUser(dfs->nn_hostname, dfs->nn_port);
+  if (userFS == NULL) {
     ERROR("Could not connect");
     return -EIO;
   }
 
+  int ret = 0;
   if (hdfsRename(userFS, from, to)) {
     ERROR("Rename %s to %s failed", from, to);
-    return -EIO;
+    ret = (errno > 0) ? -errno : -EIO;
+    goto cleanup;
+  }
+
+cleanup:
+  if (doDisconnect(userFS)) {
+    ret = -EIO;
   }
+  return ret;
 
-  return 0;
 }

+ 15 - 10
hdfs/src/contrib/fuse-dfs/src/fuse_impls_rmdir.c

@@ -27,10 +27,8 @@ int dfs_rmdir(const char *path)
 {
   TRACE1("rmdir", path)
 
-  // retrieve dfs specific data
   dfs_context *dfs = (dfs_context*)fuse_get_context()->private_data;
 
-  // check params and the context var
   assert(path);
   assert(dfs);
   assert('/' == *path);
@@ -45,27 +43,34 @@ int dfs_rmdir(const char *path)
     return -EACCES;
   }
 
-  hdfsFS userFS;
-  // if not connected, try to connect and fail out if we can't.
-  if ((userFS = doConnectAsUser(dfs->nn_hostname,dfs->nn_port))== NULL) {
+  hdfsFS userFS = doConnectAsUser(dfs->nn_hostname, dfs->nn_port);
+  if (userFS == NULL) {
     ERROR("Could not connect");
     return -EIO;
   }
 
+  int ret = 0;
   int numEntries = 0;
   hdfsFileInfo *info = hdfsListDirectory(userFS,path,&numEntries);
 
-  // free the info pointers
-  hdfsFreeFileInfo(info,numEntries);
+  if (info) {
+    hdfsFreeFileInfo(info, numEntries);
+  }
 
   if (numEntries) {
-    return -ENOTEMPTY;
+    ret = -ENOTEMPTY;
+    goto cleanup;
   }
 
   if (hdfsDeleteWithTrash(userFS, path, dfs->usetrash)) {
     ERROR("Error trying to delete directory %s", path);
-    return -EIO;
+    ret = -EIO;
+    goto cleanup;
   }
 
-  return 0;
+cleanup:
+  if (doDisconnect(userFS)) {
+    ret = -EIO;
+  }
+  return ret;
 }

+ 5 - 27
hdfs/src/contrib/fuse-dfs/src/fuse_impls_statfs.c

@@ -25,20 +25,16 @@ int dfs_statfs(const char *path, struct statvfs *st)
 {
   TRACE1("statfs",path)
 
-  // retrieve dfs specific data
   dfs_context *dfs = (dfs_context*)fuse_get_context()->private_data;
 
-  // check params and the context var
   assert(path);
   assert(st);
   assert(dfs);
 
-  // init the stat structure
   memset(st,0,sizeof(struct statvfs));
 
-  hdfsFS userFS;
-  // if not connected, try to connect and fail out if we can't.
-  if ((userFS = doConnectAsUser(dfs->nn_hostname,dfs->nn_port))== NULL) {
+  hdfsFS userFS = doConnectAsUser(dfs->nn_hostname, dfs->nn_port);
+  if (userFS == NULL) {
     ERROR("Could not connect");
     return -EIO;
   }
@@ -47,32 +43,15 @@ int dfs_statfs(const char *path, struct statvfs *st)
   const tOffset used  = hdfsGetUsed(userFS);
   const tOffset bsize = hdfsGetDefaultBlockSize(userFS);
 
-  // fill in the statvfs structure
-
-  /* FOR REFERENCE:
-     struct statvfs {
-     unsigned long  f_bsize;    // file system block size
-     unsigned long  f_frsize;   // fragment size
-     fsblkcnt_t     f_blocks;   // size of fs in f_frsize units
-     fsblkcnt_t     f_bfree;    // # free blocks
-     fsblkcnt_t     f_bavail;   // # free blocks for non-root
-     fsfilcnt_t     f_files;    // # inodes
-     fsfilcnt_t     f_ffree;    // # free inodes
-     fsfilcnt_t     f_favail;   // # free inodes for non-root
-     unsigned long  f_fsid;     // file system id
-     unsigned long  f_flag;     / mount flags
-     unsigned long  f_namemax;  // maximum filename length
-     };
-  */
+  if (doDisconnect(userFS)) {
+    return -EIO;
+  }
 
   st->f_bsize   =  bsize;
   st->f_frsize  =  bsize;
-
   st->f_blocks  =  cap/bsize;
-
   st->f_bfree   =  (cap-used)/bsize;
   st->f_bavail  =  (cap-used)/bsize;
-
   st->f_files   =  1000;
   st->f_ffree   =  500;
   st->f_favail  =  500;
@@ -82,4 +61,3 @@ int dfs_statfs(const char *path, struct statvfs *st)
 
   return 0;
 }
-

+ 18 - 10
hdfs/src/contrib/fuse-dfs/src/fuse_impls_truncate.c

@@ -29,9 +29,6 @@
 int dfs_truncate(const char *path, off_t size)
 {
   TRACE1("truncate", path)
-  if (size != 0) {
-    return -ENOTSUP;
-  }
 
   dfs_context *dfs = (dfs_context*)fuse_get_context()->private_data;
 
@@ -39,16 +36,20 @@ int dfs_truncate(const char *path, off_t size)
   assert('/' == *path);
   assert(dfs);
 
+  if (size != 0) {
+    return -ENOTSUP;
+  }
+
   int ret = dfs_unlink(path);
   if (ret != 0) {
     return ret;
   }
 
-  hdfsFS userFS;
-  // if not connected, try to connect and fail out if we can't.
-  if ((userFS = doConnectAsUser(dfs->nn_hostname,dfs->nn_port)) == NULL) {
+  hdfsFS userFS = doConnectAsUser(dfs->nn_hostname, dfs->nn_port);
+  if (userFS == NULL) {
     ERROR("Could not connect");
-    return -EIO;
+    ret = -EIO;
+    goto cleanup;
   }
 
   int flags = O_WRONLY | O_CREAT;
@@ -56,12 +57,19 @@ int dfs_truncate(const char *path, off_t size)
   hdfsFile file;
   if ((file = (hdfsFile)hdfsOpenFile(userFS, path, flags,  0, 0, 0)) == NULL) {
     ERROR("Could not connect open file %s", path);
-    return -EIO;
+    ret = -EIO;
+    goto cleanup;
   }
 
   if (hdfsCloseFile(userFS, file) != 0) {
     ERROR("Could not close file %s", path);
-    return -EIO;
+    ret = -EIO;
+    goto cleanup;
+  }
+
+cleanup:
+  if (doDisconnect(userFS)) {
+    ret = -EIO;
   }
-  return 0;
+  return ret;
 }

+ 12 - 8
hdfs/src/contrib/fuse-dfs/src/fuse_impls_unlink.c

@@ -26,16 +26,15 @@ int dfs_unlink(const char *path)
 {
   TRACE1("unlink", path)
 
-  // retrieve dfs specific data
+  int ret = 0;
   dfs_context *dfs = (dfs_context*)fuse_get_context()->private_data;
 
-  // check params and the context var
   assert(path);
   assert(dfs);
   assert('/' == *path);
 
   if (is_protected(path)) {
-    ERROR("Trying to delete protected directory %s ", path);
+    ERROR("Trying to delete protected directory %s", path);
     return -EACCES;
   }
 
@@ -44,17 +43,22 @@ int dfs_unlink(const char *path)
     return -EACCES;
   }
 
-  hdfsFS userFS;
-  // if not connected, try to connect and fail out if we can't.
-  if ((userFS = doConnectAsUser(dfs->nn_hostname,dfs->nn_port))== NULL) {
+  hdfsFS userFS = doConnectAsUser(dfs->nn_hostname, dfs->nn_port);
+  if (userFS == NULL) {
     ERROR("Could not connect");
     return -EIO;
   }
 
   if (hdfsDeleteWithTrash(userFS, path, dfs->usetrash)) {
     ERROR("Could not delete file %s", path);
-    return -EIO;
+    ret = (errno > 0) ? -errno : -EIO;
+    goto cleanup;
+  }
+
+cleanup:
+  if (doDisconnect(userFS)) {
+    ret = -EIO;
   }
+  return ret;
 
-  return 0;
 }

+ 16 - 12
hdfs/src/contrib/fuse-dfs/src/fuse_impls_utimens.c

@@ -23,11 +23,9 @@
 int dfs_utimens(const char *path, const struct timespec ts[2])
 {
   TRACE1("utimens", path)
-#if PERMS
-  // retrieve dfs specific data
+  int ret = 0;
   dfs_context *dfs = (dfs_context*)fuse_get_context()->private_data;
 
-  // check params and the context var
   assert(path);
   assert(dfs);
   assert('/' == *path);
@@ -35,25 +33,31 @@ int dfs_utimens(const char *path, const struct timespec ts[2])
   time_t aTime = ts[0].tv_sec;
   time_t mTime = ts[1].tv_sec;
 
-  hdfsFS userFS;
-  // if not connected, try to connect and fail out if we can't.
-  if ((userFS = doConnectAsUser(dfs->nn_hostname,dfs->nn_port))== NULL) {
+  hdfsFS userFS = doConnectAsUser(dfs->nn_hostname, dfs->nn_port);
+  if (userFS == NULL) {
     ERROR("Could not connect");
     return -EIO;
   }
 
   if (hdfsUtime(userFS, path, mTime, aTime)) {
-    hdfsFileInfo *info = hdfsGetPathInfo(dfs->fs,path);
+    hdfsFileInfo *info = hdfsGetPathInfo(userFS, path);
     if (info == NULL) {
-      return -EIO;
+      ret = (errno > 0) ? -errno : -ENOENT;
+      goto cleanup;
     }
     // Silently ignore utimens failure for directories, otherwise 
     // some programs like tar will fail.
     if (info->mKind == kObjectKindDirectory) {
-      return 0;
+      ret = 0;
+    } else {
+      ret = (errno > 0) ? -errno : -EACCES;
     }
-    return -errno;
+    goto cleanup;
   }
-#endif  
-  return 0;
+
+cleanup:
+  if (doDisconnect(userFS)) {
+    ret = -EIO;
+  }
+  return ret;
 }

+ 10 - 21
hdfs/src/contrib/fuse-dfs/src/fuse_init.c

@@ -22,6 +22,7 @@
 #include "fuse_init.h"
 #include "fuse_options.h"
 #include "fuse_context_handle.h"
+#include "fuse_connect.h"
 
 // Hacked up function to basically do:
 //  protectedpaths = split(options.protected,':');
@@ -30,7 +31,6 @@ void init_protectedpaths(dfs_context *dfs) {
 
   char *tmp = options.protected;
 
-
   // handle degenerate case up front.
   if (tmp == NULL || 0 == *tmp) {
     dfs->protectedpaths = (char**)malloc(sizeof(char*));
@@ -43,7 +43,6 @@ void init_protectedpaths(dfs_context *dfs) {
     print_options();
   }
 
-
   int i = 0;
   while (tmp && (NULL != (tmp = index(tmp,':')))) {
     tmp++; // pass the ,
@@ -75,27 +74,15 @@ void init_protectedpaths(dfs_context *dfs) {
     j++;
   }
   dfs->protectedpaths[j] = NULL;
-
-  /*
-    j  = 0;
-    while (dfs->protectedpaths[j]) {
-    printf("dfs->protectedpaths[%d]=%s\n",j,dfs->protectedpaths[j]);
-    fflush(stdout);
-    j++;
-    }
-    exit(1);
-  */
 }
 
-void *dfs_init()
-{
 
+void *dfs_init(void) {
   //
   // Create a private struct of data we will pass to fuse here and which
   // will then be accessible on every call.
   //
-  dfs_context *dfs = (dfs_context*)malloc(sizeof (dfs_context));
-
+  dfs_context *dfs = (dfs_context*)malloc(sizeof(dfs_context));
   if (NULL == dfs) {
     ERROR("FATAL: could not malloc dfs_context");
     exit(1);
@@ -105,7 +92,6 @@ void *dfs_init()
   dfs->debug                 = options.debug;
   dfs->nn_hostname           = options.server;
   dfs->nn_port               = options.port;
-  dfs->fs                    = NULL;
   dfs->read_only             = options.read_only;
   dfs->usetrash              = options.usetrash;
   dfs->protectedpaths        = NULL;
@@ -121,14 +107,17 @@ void *dfs_init()
     DEBUG("dfs->rdbuffersize <= 0 = %ld", dfs->rdbuffer_size);
     dfs->rdbuffer_size = 32768;
   }
+
+  if (0 != allocFsTable()) {
+    ERROR("FATAL: could not allocate ");
+    exit(1);
+  }
+
   return (void*)dfs;
 }
 
 
-
-void dfs_destroy (void *ptr)
+void dfs_destroy(void *ptr)
 {
   TRACE("destroy")
-  dfs_context *dfs = (dfs_context*)ptr;
-  dfs->fs = NULL;
 }

+ 5 - 6
hdfs/src/contrib/fuse-dfs/src/fuse_options.c

@@ -107,10 +107,10 @@ int dfs_options(void *data, const char *arg, int key,  struct fuse_args *outargs
 
   switch (key) {
   case FUSE_OPT_KEY_OPT:
-    fprintf(stderr, "fuse-dfs ignoring option %s\n", arg);
+    INFO("Ignoring option %s", arg);
     return 1;
-  case  KEY_VERSION:
-    fprintf(stdout, "%s %s\n", program, _FUSE_DFS_VERSION);
+  case KEY_VERSION:
+    INFO("%s %s\n", program, _FUSE_DFS_VERSION);
     exit(0);
   case KEY_HELP:
     print_usage(program);
@@ -159,14 +159,13 @@ int dfs_options(void *data, const char *arg, int key,  struct fuse_args *outargs
       } else if (strcmp(arg,"rw") == 0) {
         options.read_only = 0;
       } else {
-        ERROR("fuse-dfs didn't recognize %s,%d\n",arg,key);
-        fuse_opt_add_arg(outargs,arg);
+        INFO("Adding FUSE arg %s", arg);
+        fuse_opt_add_arg(outargs, arg);
         return 0;
       }
     } else {
       options.port = tmp_port;
       options.server = strdup(tmp_server);
-      ERROR("port=%d,server=%s\n", options.port, options.server);
     }
   }
   }

+ 4 - 15
hdfs/src/contrib/fuse-dfs/src/fuse_stat_struct.c

@@ -25,18 +25,15 @@
 #include "fuse_stat_struct.h"
 #include "fuse_context_handle.h"
 
-#if PERMS
-/**
+/*
  * getpwuid and getgrgid return static structs so we safeguard the contents
  * while retrieving fields using the 2 structs below.
  * NOTE: if using both, always get the passwd struct firt!
  */
 extern pthread_mutex_t passwdstruct_mutex; 
 extern pthread_mutex_t groupstruct_mutex;
-#endif
-
 
-const int default_id       = 99; // nobody  - not configurable since soon uids in dfs, yeah!
+const int default_id = 99; // nobody  - not configurable since soon uids in dfs, yeah!
 const int blksize = 512;
 
 /**
@@ -55,7 +52,6 @@ int fill_stat_structure(hdfsFileInfo *info, struct stat *st)
   st->st_nlink = (info->mKind == kObjectKindDirectory) ? 0 : 1;
 
   uid_t owner_id = default_id;
-#if PERMS
   if (info->mOwner != NULL) {
     //
     // Critical section - protect from concurrent calls in different threads since
@@ -73,9 +69,9 @@ int fill_stat_structure(hdfsFileInfo *info, struct stat *st)
     pthread_mutex_unlock(&passwdstruct_mutex);
 
   } 
-#endif
+
   gid_t group_id = default_id;
-#if PERMS
+
   if (info->mGroup != NULL) {
     //
     // Critical section - protect from concurrent calls in different threads since
@@ -93,15 +89,12 @@ int fill_stat_structure(hdfsFileInfo *info, struct stat *st)
     pthread_mutex_unlock(&groupstruct_mutex);
 
   }
-#endif
 
   short perm = (info->mKind == kObjectKindDirectory) ? (S_IFDIR | 0777) :  (S_IFREG | 0666);
-#if PERMS
   if (info->mPermissions > 0) {
     perm = (info->mKind == kObjectKindDirectory) ? S_IFDIR:  S_IFREG ;
     perm |= info->mPermissions;
   }
-#endif
 
   // set stat metadata
   st->st_size     = (info->mKind == kObjectKindDirectory) ? 4096 : info->mSize;
@@ -110,11 +103,7 @@ int fill_stat_structure(hdfsFileInfo *info, struct stat *st)
   st->st_mode     = perm;
   st->st_uid      = owner_id;
   st->st_gid      = group_id;
-#if PERMS
   st->st_atime    = info->mLastAccess;
-#else
-  st->st_atime    = info->mLastMod;
-#endif
   st->st_mtime    = info->mLastMod;
   st->st_ctime    = info->mLastMod;
 

+ 3 - 11
hdfs/src/contrib/fuse-dfs/src/fuse_users.c

@@ -24,26 +24,19 @@
 
 #include "fuse_dfs.h"
 
-
-
-#if PERMS
-/**
+/*
  * getpwuid and getgrgid return static structs so we safeguard the contents
  * while retrieving fields using the 2 structs below.
  * NOTE: if using both, always get the passwd struct firt!
  */
 pthread_mutex_t passwdstruct_mutex = PTHREAD_MUTEX_INITIALIZER;
 pthread_mutex_t groupstruct_mutex = PTHREAD_MUTEX_INITIALIZER;
-#endif
 
-#if PERMS
-
-/**
+/*
  * Utility for getting the user making the fuse call in char * form
  * NOTE: if non-null return, the return must be freed by the caller.
  */
-char *getUsername(uid_t uid)
-{
+char *getUsername(uid_t uid) {
   //
   // Critical section - protect from concurrent calls in different threads.
   // since the struct below is static.
@@ -218,4 +211,3 @@ char ** getGroups(uid_t uid, int *num_groups)
 #endif
   return groupnames;
 }
-#endif

+ 3 - 4
hdfs/src/docs/src/documentation/content/xdocs/hdfs_design.xml

@@ -391,7 +391,7 @@
         <title> Replication Pipelining </title>
         <p>
         When a client is writing data to an HDFS file with a replication factor of 3, the NameNode retrieves a list of DataNodes using a replication target choosing algorithm.
-        This list contains the DataNodes that will host a replica of that block. The client then writes to the first DataNode. The first DataNode starts receiving the data in small portions (4 KB), 
+        This list contains the DataNodes that will host a replica of that block. The client then writes to the first DataNode. The first DataNode starts receiving the data in small portions (64 KB, configurable), 
         writes each portion to its local repository and transfers that portion to the second DataNode in the list. 
         The second DataNode, in turn starts receiving each portion of the data block, writes that portion to its 
         repository and then flushes that portion to the third DataNode. Finally, the third DataNode writes the 
@@ -498,9 +498,8 @@
         If a user wants to undelete a file that he/she has deleted, he/she can navigate the <code>/trash</code> 
         directory and retrieve the file. The <code>/trash</code> directory contains only the latest copy of the file 
         that was deleted. The <code>/trash</code> directory is just like any other directory with one special 
-        feature: HDFS applies specified policies to automatically delete files from this directory. The current 
-        default policy is to delete files from <code>/trash</code> that are more than 6 hours old. In the future, 
-        this policy will be configurable through a well defined interface.
+        feature: HDFS applies specified policies to automatically delete files from this directory.
+        By default, the trash feature is disabled. It can be enabled by setting the <em>fs.trash.interval</em> property in core-site.xml to a non-zero value (set as minutes of retention required). The property needs to exist on both client and server side configurations.
         </p>
       </section>
 

+ 27 - 26
hdfs/src/java/org/apache/hadoop/hdfs/BlockReader.java

@@ -17,9 +17,7 @@
  */
 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 static org.apache.hadoop.hdfs.protocol.HdfsProtoUtil.vintPrefixed;
 
 import java.io.BufferedInputStream;
 import java.io.BufferedOutputStream;
@@ -34,9 +32,12 @@ import java.nio.ByteBuffer;
 import org.apache.hadoop.classification.InterfaceAudience;
 import org.apache.hadoop.fs.FSInputChecker;
 import org.apache.hadoop.fs.Path;
-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.datatransfer.PacketHeader;
+import org.apache.hadoop.hdfs.protocol.datatransfer.Sender;
+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 +46,7 @@ import org.apache.hadoop.net.NetUtils;
 import org.apache.hadoop.security.token.Token;
 import org.apache.hadoop.util.DataChecksum;
 
+
 /** This is a wrapper around connection to datanode
  * and understands checksum, offset etc.
  *
@@ -138,9 +140,9 @@ public class BlockReader extends FSInputChecker {
     // 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 +315,13 @@ public class BlockReader extends FSInputChecker {
         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();
-
-      if (!lastPacketInBlock ||
-          dataLen != 0) {
+      PacketHeader hdr = new PacketHeader();
+      hdr.readFields(in);
+
+      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;
@@ -409,7 +404,7 @@ public class BlockReader extends FSInputChecker {
                                      String clientName)
                                      throws IOException {
     // in and out will be closed when sock is closed (by the caller)
-    DataTransferProtocol.Sender.opReadBlock(
+    Sender.opReadBlock(
         new DataOutputStream(new BufferedOutputStream(
             NetUtils.getOutputStream(sock,HdfsConstants.WRITE_TIMEOUT))),
         block, startOffset, len, clientName, blockToken);
@@ -422,9 +417,10 @@ public class BlockReader extends FSInputChecker {
         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 +495,16 @@ public class BlockReader extends FSInputChecker {
    * 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) {

+ 23 - 16
hdfs/src/java/org/apache/hadoop/hdfs/DFSClient.java

@@ -18,10 +18,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;
 import java.io.DataInputStream;
@@ -65,17 +61,23 @@ import org.apache.hadoop.hdfs.protocol.ClientDatanodeProtocol;
 import org.apache.hadoop.hdfs.protocol.ClientProtocol;
 import org.apache.hadoop.hdfs.protocol.CorruptFileBlocks;
 import org.apache.hadoop.hdfs.protocol.DSQuotaExceededException;
-import org.apache.hadoop.hdfs.protocol.DataTransferProtocol;
 import org.apache.hadoop.hdfs.protocol.DatanodeID;
 import org.apache.hadoop.hdfs.protocol.DatanodeInfo;
 import org.apache.hadoop.hdfs.protocol.DirectoryListing;
 import org.apache.hadoop.hdfs.protocol.ExtendedBlock;
 import org.apache.hadoop.hdfs.protocol.FSConstants;
 import org.apache.hadoop.hdfs.protocol.HdfsFileStatus;
+import org.apache.hadoop.hdfs.protocol.HdfsProtoUtil;
 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.UnresolvedPathException;
+import org.apache.hadoop.hdfs.protocol.datatransfer.Op;
+import org.apache.hadoop.hdfs.protocol.datatransfer.ReplaceDatanodeOnFailure;
+import org.apache.hadoop.hdfs.protocol.datatransfer.Sender;
+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;
@@ -131,7 +133,7 @@ public class DFSClient implements FSConstants, java.io.Closeable {
   SocketFactory socketFactory;
   int socketTimeout;
   final int writePacketSize;
-  final DataTransferProtocol.ReplaceDatanodeOnFailure dtpReplaceDatanodeOnFailure;
+  final ReplaceDatanodeOnFailure dtpReplaceDatanodeOnFailure;
   final FileSystem.Statistics stats;
   final int hdfsTimeout;    // timeout value for a DFS operation.
   final LeaseRenewer leaserenewer;
@@ -265,7 +267,7 @@ public class DFSClient implements FSConstants, java.io.Closeable {
     this.writePacketSize = 
       conf.getInt(DFSConfigKeys.DFS_CLIENT_WRITE_PACKET_SIZE_KEY, 
                   DFSConfigKeys.DFS_CLIENT_WRITE_PACKET_SIZE_DEFAULT);
-    this.dtpReplaceDatanodeOnFailure = DataTransferProtocol.ReplaceDatanodeOnFailure.get(conf);
+    this.dtpReplaceDatanodeOnFailure = ReplaceDatanodeOnFailure.get(conf);
 
     // The hdfsTimeout is currently the same as the ipc timeout 
     this.hdfsTimeout = Client.getTimeout(conf);
@@ -1112,15 +1114,16 @@ public class DFSClient implements FSConstants, java.io.Closeable {
 
           if (LOG.isDebugEnabled()) {
             LOG.debug("write to " + datanodes[j].getName() + ": "
-                + BLOCK_CHECKSUM + ", block=" + block);
+                + Op.BLOCK_CHECKSUM + ", block=" + block);
           }
           // get block MD5
-          DataTransferProtocol.Sender.opBlockChecksum(out, block,
-              lb.getBlockToken());
+          Sender.opBlockChecksum(out, block, lb.getBlockToken());
+
+          final BlockOpResponseProto reply =
+            BlockOpResponseProto.parseFrom(HdfsProtoUtil.vintPrefixed(in));
 
-          final DataTransferProtocol.Status reply = DataTransferProtocol.Status.read(in);
-          if (reply != SUCCESS) {
-            if (reply == ERROR_ACCESS_TOKEN
+          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 +1141,12 @@ public class DFSClient implements FSConstants, java.io.Closeable {
                   + 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 +1156,14 @@ public class DFSClient implements FSConstants, java.io.Closeable {
           }
           
           //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;

+ 26 - 18
hdfs/src/java/org/apache/hadoop/hdfs/DFSOutputStream.java

@@ -17,8 +17,7 @@
  */
 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 java.io.BufferedOutputStream;
 import java.io.DataInputStream;
@@ -47,16 +46,20 @@ import org.apache.hadoop.fs.UnresolvedLinkException;
 import org.apache.hadoop.fs.permission.FsPermission;
 import org.apache.hadoop.hdfs.protocol.ClientProtocol;
 import org.apache.hadoop.hdfs.protocol.DSQuotaExceededException;
-import org.apache.hadoop.hdfs.protocol.DataTransferProtocol;
-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.DatanodeInfo;
 import org.apache.hadoop.hdfs.protocol.ExtendedBlock;
 import org.apache.hadoop.hdfs.protocol.HdfsFileStatus;
+import org.apache.hadoop.hdfs.protocol.HdfsProtoUtil;
 import org.apache.hadoop.hdfs.protocol.LocatedBlock;
 import org.apache.hadoop.hdfs.protocol.NSQuotaExceededException;
 import org.apache.hadoop.hdfs.protocol.UnresolvedPathException;
+import org.apache.hadoop.hdfs.protocol.datatransfer.BlockConstructionStage;
+import org.apache.hadoop.hdfs.protocol.datatransfer.DataTransferProtocol;
+import org.apache.hadoop.hdfs.protocol.datatransfer.PacketHeader;
+import org.apache.hadoop.hdfs.protocol.datatransfer.PipelineAck;
+import org.apache.hadoop.hdfs.protocol.datatransfer.Sender;
+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;
@@ -64,7 +67,6 @@ import org.apache.hadoop.hdfs.server.namenode.NotReplicatedYetException;
 import org.apache.hadoop.hdfs.server.namenode.SafeModeException;
 import org.apache.hadoop.io.EnumSetWritable;
 import org.apache.hadoop.io.IOUtils;
-import org.apache.hadoop.io.Text;
 import org.apache.hadoop.ipc.RemoteException;
 import org.apache.hadoop.net.NetUtils;
 import org.apache.hadoop.security.AccessControlException;
@@ -75,6 +77,8 @@ import org.apache.hadoop.util.Progressable;
 import org.apache.hadoop.util.PureJavaCrc32;
 import org.apache.hadoop.util.StringUtils;
 
+
+
 /****************************************************************
  * DFSOutputStream creates files from a stream of bytes.
  *
@@ -650,7 +654,7 @@ class DFSOutputStream extends FSOutputSummer implements Syncable {
             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 +
@@ -843,12 +847,14 @@ class DFSOutputStream extends FSOutputSummer implements Syncable {
             DataNode.SMALL_BUFFER_SIZE));
 
         //send the TRANSFER_BLOCK request
-        DataTransferProtocol.Sender.opTransferBlock(out, block,
+        Sender.opTransferBlock(out, block,
             dfsClient.clientName, targets, blockToken);
 
         //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 +996,7 @@ class DFSOutputStream extends FSOutputSummer implements Syncable {
     //
     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++) {
@@ -1015,7 +1021,7 @@ class DFSOutputStream extends FSOutputSummer implements Syncable {
         blockReplyStream = new DataInputStream(NetUtils.getInputStream(s));
 
         // send the request
-        DataTransferProtocol.Sender.opWriteBlock(out, block,
+        Sender.opWriteBlock(out, block,
             nodes.length, recoveryFlag ? stage.getRecoveryStage() : stage, newGS, 
             block.getNumBytes(), bytesSent, dfsClient.clientName, null, nodes,
             accessToken);
@@ -1023,10 +1029,13 @@ class DFSOutputStream extends FSOutputSummer implements Syncable {
         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);
@@ -1401,9 +1410,8 @@ class DFSOutputStream extends FSOutputSummer implements Syncable {
   }
   
   /**
-   * flushes out to all replicas of the block. 
-   * The data is in the buffers of the DNs 
-   * but not neccessary on the DN's OS buffers. 
+   * Flushes out to all replicas of the block. The data is in the buffers
+   * of the DNs but not necessarily in the DN's OS buffers.
    *
    * It is a synchronous operation. When it returns,
    * it guarantees that flushed data become visible to new readers. 

+ 3 - 4
hdfs/src/java/org/apache/hadoop/hdfs/HftpFileSystem.java

@@ -190,9 +190,6 @@ public class HftpFileSystem extends FileSystem {
         }
       }
       
-      //Renew TGT if needed
-      ugi.reloginFromKeytab();
-      
       //since we don't already have a token, go get one over https
       if (delegationToken == null) {
         delegationToken = 
@@ -204,8 +201,10 @@ public class HftpFileSystem extends FileSystem {
   
 
   @Override
-  public Token<?> getDelegationToken(final String renewer) throws IOException {
+  public synchronized Token<?> getDelegationToken(final String renewer) throws IOException {
     try {
+      //Renew TGT if needed
+      ugi.reloginFromKeytab();
       return ugi.doAs(new PrivilegedExceptionAction<Token<?>>() {
         public Token<?> run() throws IOException {
           Credentials c;

+ 0 - 1068
hdfs/src/java/org/apache/hadoop/hdfs/protocol/DataTransferProtocol.java

@@ -1,1068 +0,0 @@
-/**
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- *     http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-package org.apache.hadoop.hdfs.protocol;
-
-import java.io.DataInput;
-import java.io.DataInputStream;
-import java.io.DataOutput;
-import java.io.DataOutputStream;
-import java.io.IOException;
-import java.io.OutputStream;
-import java.nio.ByteBuffer;
-
-import org.apache.commons.logging.Log;
-import org.apache.commons.logging.LogFactory;
-import org.apache.hadoop.HadoopIllegalArgumentException;
-import org.apache.hadoop.classification.InterfaceAudience;
-import org.apache.hadoop.classification.InterfaceStability;
-import org.apache.hadoop.conf.Configuration;
-import org.apache.hadoop.hdfs.DFSConfigKeys;
-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.security.token.Token;
-
-/**
- * Transfer data to/from datanode using a streaming protocol.
- */
-@InterfaceAudience.Private
-@InterfaceStability.Evolving
-public interface DataTransferProtocol {
-  public static final Log LOG = LogFactory.getLog(DataTransferProtocol.class);
-  
-  /** Version for data transfers between clients and datanodes
-   * This should change when serialization of DatanodeInfo, not just
-   * when protocol changes. It is not very obvious. 
-   */
-  /*
-   * Version 25:
-   *    Encapsulate individual operation headers.
-   */
-  public static final int DATA_TRANSFER_VERSION = 25;
-
-  /** Operation */
-  public enum Op {
-    WRITE_BLOCK((byte)80),
-    READ_BLOCK((byte)81),
-    READ_METADATA((byte)82),
-    REPLACE_BLOCK((byte)83),
-    COPY_BLOCK((byte)84),
-    BLOCK_CHECKSUM((byte)85),
-    TRANSFER_BLOCK((byte)86);
-
-    /** The code for this operation. */
-    public final byte code;
-    
-    private Op(byte code) {
-      this.code = code;
-    }
-    
-    private static final int FIRST_CODE = values()[0].code;
-    /** Return the object represented by the code. */
-    private static Op valueOf(byte code) {
-      final int i = (code & 0xff) - FIRST_CODE;
-      return i < 0 || i >= values().length? null: values()[i];
-    }
-
-    /** Read from in */
-    public static Op read(DataInput in) throws IOException {
-      return valueOf(in.readByte());
-    }
-
-    /** Write to out */
-    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. 
-     * Changing this order will make getRecoveryStage not working.
-     */
-    // pipeline set up for block append
-    PIPELINE_SETUP_APPEND,
-    // pipeline set up for failed PIPELINE_SETUP_APPEND recovery
-    PIPELINE_SETUP_APPEND_RECOVERY,
-    // data streaming
-    DATA_STREAMING,
-    // pipeline setup for failed data streaming recovery
-    PIPELINE_SETUP_STREAMING_RECOVERY,
-    // close the block and pipeline
-    PIPELINE_CLOSE,
-    // Recover a failed PIPELINE_CLOSE
-    PIPELINE_CLOSE_RECOVERY,
-    // pipeline set up for block creation
-    PIPELINE_SETUP_CREATE,
-    // transfer RBW for adding datanodes
-    TRANSFER_RBW,
-    // transfer Finalized for adding datanodes
-    TRANSFER_FINALIZED;
-    
-    final static private byte RECOVERY_BIT = (byte)1;
-    
-    /**
-     * get the recovery stage of this stage
-     */
-    public BlockConstructionStage getRecoveryStage() {
-      if (this == PIPELINE_SETUP_CREATE) {
-        throw new IllegalArgumentException( "Unexpected blockStage " + this);
-      } else {
-        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
-  public static class Sender {
-    /** Initialize a operation. */
-    private static void op(final DataOutput out, final Op op
-        ) throws IOException {
-      out.writeShort(DataTransferProtocol.DATA_TRANSFER_VERSION);
-      op.write(out);
-    }
-
-    /** Send an operation request. */
-    private static void send(final DataOutputStream out, final Op opcode,
-        final Op.BaseHeader parameters) throws IOException {
-      op(out, opcode);
-      parameters.write(out);
-      out.flush();
-    }
-
-    /** Send OP_READ_BLOCK */
-    public static void opReadBlock(DataOutputStream out, ExtendedBlock blk,
-        long blockOffset, long blockLen, String clientName,
-        Token<BlockTokenIdentifier> blockToken)
-        throws IOException {
-      send(out, Op.READ_BLOCK, new Op.ReadBlockHeader(blk, blockToken,
-          clientName, blockOffset, blockLen));
-    }
-    
-    /** 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));
-    }
-
-    /** 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));
-    }
-
-    /** 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));
-    }
-
-    /** 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));
-    }
-
-    /** 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);
-      }
-    }
-  }
-
-  /** Receiver */
-  public static abstract class Receiver {
-    /** Read an Op.  It also checks protocol version. */
-    protected final Op readOp(DataInputStream in) throws IOException {
-      final short version = in.readShort();
-      if (version != DATA_TRANSFER_VERSION) {
-        throw new IOException( "Version Mismatch (Expected: " +
-            DataTransferProtocol.DATA_TRANSFER_VERSION  +
-            ", Received: " +  version + " )");
-      }
-      return Op.read(in);
-    }
-
-    /** Process op by the corresponding method. */
-    protected final void processOp(Op op, DataInputStream in
-        ) throws IOException {
-      switch(op) {
-      case READ_BLOCK:
-        opReadBlock(in);
-        break;
-      case WRITE_BLOCK:
-        opWriteBlock(in);
-        break;
-      case REPLACE_BLOCK:
-        opReplaceBlock(in);
-        break;
-      case COPY_BLOCK:
-        opCopyBlock(in);
-        break;
-      case BLOCK_CHECKSUM:
-        opBlockChecksum(in);
-        break;
-      case TRANSFER_BLOCK:
-        opTransferBlock(in);
-        break;
-      default:
-        throw new IOException("Unknown op " + op + " in data stream");
-      }
-    }
-
-    /** 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());
-    }
-
-    /**
-     * Abstract OP_READ_BLOCK method. Read a block.
-     */
-    protected abstract void opReadBlock(DataInputStream in, ExtendedBlock blk,
-        long offset, long length, String client,
-        Token<BlockTokenIdentifier> blockToken) throws IOException;
-    
-    /** 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());
-    }
-
-    /**
-     * Abstract OP_WRITE_BLOCK method. 
-     * Write a block.
-     */
-    protected abstract void opWriteBlock(DataInputStream in, ExtendedBlock blk,
-        int pipelineSize, BlockConstructionStage stage, long newGs,
-        long minBytesRcvd, long maxBytesRcvd, String client, DatanodeInfo src,
-        DatanodeInfo[] targets, Token<BlockTokenIdentifier> blockToken)
-        throws IOException;
-
-    /** 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());
-    }
-
-    /**
-     * Abstract {@link Op#TRANSFER_BLOCK} method.
-     * For {@link BlockConstructionStage#TRANSFER_RBW}
-     * or {@link BlockConstructionStage#TRANSFER_FINALIZED}.
-     */
-    protected abstract void opTransferBlock(DataInputStream in, ExtendedBlock blk,
-        String client, DatanodeInfo[] targets,
-        Token<BlockTokenIdentifier> blockToken)
-        throws IOException;
-
-    /** 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());
-    }
-
-    /**
-     * Abstract OP_REPLACE_BLOCK method.
-     * It is used for balancing purpose; send to a destination
-     */
-    protected abstract void opReplaceBlock(DataInputStream in,
-        ExtendedBlock blk, String delHint, DatanodeInfo src,
-        Token<BlockTokenIdentifier> blockToken) throws IOException;
-
-    /** 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());
-    }
-
-    /**
-     * Abstract OP_COPY_BLOCK method. It is used for balancing purpose; send to
-     * a proxy source.
-     */
-    protected abstract void opCopyBlock(DataInputStream in, ExtendedBlock blk,
-        Token<BlockTokenIdentifier> blockToken)
-        throws IOException;
-
-    /** 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());
-    }
-
-    /**
-     * Abstract OP_BLOCK_CHECKSUM method.
-     * Get the checksum of a block 
-     */
-    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 final static long UNKOWN_SEQNO = -2;
-
-    /** default constructor **/
-    public PipelineAck() {
-    }
-    
-    /**
-     * Constructor
-     * @param seqno sequence number
-     * @param replies an array of replies
-     */
-    public PipelineAck(long seqno, Status[] replies) {
-      this.seqno = seqno;
-      this.replies = replies;
-    }
-    
-    /**
-     * Get the sequence number
-     * @return the sequence number
-     */
-    public long getSeqno() {
-      return seqno;
-    }
-    
-    /**
-     * Get the number of replies
-     * @return the number of replies
-     */
-    public short getNumOfReplies() {
-      return (short)replies.length;
-    }
-    
-    /**
-     * get the ith reply
-     * @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];
-    }
-    
-    /**
-     * Check if this ack contains error status
-     * @return true if all statuses are SUCCESS
-     */
-    public boolean isSuccess() {
-      for (Status reply : replies) {
-        if (reply != Status.SUCCESS) {
-          return false;
-        }
-      }
-      return true;
-    }
-    
-    /**** 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);
-      }
-    }
-
-    @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);
-      }
-    }
-    
-    @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();
-    }
-  }
-
-  /**
-   * Header data for each packet that goes through the read/write pipelines.
-   */
-  public static class PacketHeader implements Writable {
-    /** 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 int packetLen;
-    private long offsetInBlock;
-    private long seqno;
-    private boolean lastPacketInBlock;
-    private int dataLen;
-
-    public PacketHeader() {
-    }
-
-    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;
-    }
-
-    public int getDataLen() {
-      return dataLen;
-    }
-
-    public boolean isLastPacketInBlock() {
-      return lastPacketInBlock;
-    }
-
-    public long getSeqno() {
-      return seqno;
-    }
-
-    public long getOffsetInBlock() {
-      return offsetInBlock;
-    }
-
-    public int getPacketLen() {
-      return packetLen;
-    }
-
-    @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();
-    }
-
-    @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();
-    }
-
-    @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);
-    }
-
-    /**
-     * Perform a sanity check on the packet, returning true if it is sane.
-     * @param lastSeqNo the previous sequence number received - we expect the current
-     * sequence number to be larger by 1.
-     */
-    public boolean sanityCheck(long lastSeqNo) {
-      // We should only have a non-positive data length for the last packet
-      if (dataLen <= 0 && lastPacketInBlock) return false;
-      // The last packet should not contain data
-      if (lastPacketInBlock && dataLen != 0) return false;
-      // Seqnos should always increase by 1 with each packet received
-      if (seqno != lastSeqNo + 1) return false;
-      return true;
-    }
-
-    @Override
-    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);
-    }
-
-    @Override
-    public int hashCode() {
-      return (int)seqno;
-    }
-  }
-
-  /**
-   * The setting of replace-datanode-on-failure feature.
-   */
-  public enum ReplaceDatanodeOnFailure {
-    /** The feature is disabled in the entire site. */
-    DISABLE,
-    /** Never add a new datanode. */
-    NEVER,
-    /**
-     * DEFAULT policy:
-     *   Let r be the replication number.
-     *   Let n be the number of existing datanodes.
-     *   Add a new datanode only if r >= 3 and either
-     *   (1) floor(r/2) >= n; or
-     *   (2) r > n and the block is hflushed/appended.
-     */
-    DEFAULT,
-    /** Always add a new datanode when an existing datanode is removed. */
-    ALWAYS;
-
-    /** Check if the feature is enabled. */
-    public void checkEnabled() {
-      if (this == DISABLE) {
-        throw new UnsupportedOperationException(
-            "This feature is disabled.  Please refer to "
-            + DFSConfigKeys.DFS_CLIENT_WRITE_REPLACE_DATANODE_ON_FAILURE_ENABLE_KEY
-            + " configuration property.");
-      }
-    }
-
-    /** Is the policy satisfied? */
-    public boolean satisfy(
-        final short replication, final DatanodeInfo[] existings,
-        final boolean isAppend, final boolean isHflushed) {
-      final int n = existings == null? 0: existings.length;
-      if (n == 0 || n >= replication) {
-        //don't need to add datanode for any policy.
-        return false;
-      } else if (this == DISABLE || this == NEVER) {
-        return false;
-      } else if (this == ALWAYS) {
-        return true;
-      } else {
-        //DEFAULT
-        if (replication < 3) {
-          return false;
-        } else {
-          if (n <= (replication/2)) {
-            return true;
-          } else {
-            return isAppend || isHflushed;
-          }
-        }
-      }
-    }
-
-    /** Get the setting from configuration. */
-    public static ReplaceDatanodeOnFailure get(final Configuration conf) {
-      final boolean enabled = conf.getBoolean(
-          DFSConfigKeys.DFS_CLIENT_WRITE_REPLACE_DATANODE_ON_FAILURE_ENABLE_KEY,
-          DFSConfigKeys.DFS_CLIENT_WRITE_REPLACE_DATANODE_ON_FAILURE_ENABLE_DEFAULT);
-      if (!enabled) {
-        return DISABLE;
-      }
-
-      final String policy = conf.get(
-          DFSConfigKeys.DFS_CLIENT_WRITE_REPLACE_DATANODE_ON_FAILURE_POLICY_KEY,
-          DFSConfigKeys.DFS_CLIENT_WRITE_REPLACE_DATANODE_ON_FAILURE_POLICY_DEFAULT);
-      for(int i = 1; i < values().length; i++) {
-        final ReplaceDatanodeOnFailure rdof = values()[i];
-        if (rdof.name().equalsIgnoreCase(policy)) {
-          return rdof;
-        }
-      }
-      throw new HadoopIllegalArgumentException("Illegal configuration value for "
-          + DFSConfigKeys.DFS_CLIENT_WRITE_REPLACE_DATANODE_ON_FAILURE_POLICY_KEY
-          + ": " + policy);
-    }
-
-    /** Write the setting to configuration. */
-    public void write(final Configuration conf) {
-      conf.setBoolean(
-          DFSConfigKeys.DFS_CLIENT_WRITE_REPLACE_DATANODE_ON_FAILURE_ENABLE_KEY,
-          this != DISABLE);
-      conf.set(
-          DFSConfigKeys.DFS_CLIENT_WRITE_REPLACE_DATANODE_ON_FAILURE_POLICY_KEY,
-          name());
-    }
-  }
-}

+ 5 - 0
hdfs/src/java/org/apache/hadoop/hdfs/protocol/DatanodeInfo.java

@@ -159,6 +159,11 @@ public class DatanodeInfo extends DatanodeID implements Node {
   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) { 

+ 166 - 0
hdfs/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);
+  }
+}

+ 24 - 0
hdfs/src/java/org/apache/hadoop/hdfs/protocol/LayoutVersion.java

@@ -108,6 +108,30 @@ public class LayoutVersion {
       this.ancestorLV = ancestorLV;
       this.description = description;
     }
+    
+    /** 
+     * Accessor method for feature layout version 
+     * @return int lv value
+     */
+    public int getLayoutVersion() {
+      return lv;
+    }
+
+    /** 
+     * Accessor method for feature ancestor layout version 
+     * @return int ancestor LV value
+     */
+    public int getAncestorLayoutVersion() {
+      return ancestorLV;
+    }
+
+    /** 
+     * Accessor method for feature description 
+     * @return String feature description 
+     */
+    public String getDescription() {
+      return description;
+    }
   }
   
   // Build layout version and corresponding feature matrix

+ 62 - 0
hdfs/src/java/org/apache/hadoop/hdfs/protocol/datatransfer/BlockConstructionStage.java

@@ -0,0 +1,62 @@
+/**
+ * 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.datatransfer;
+
+import org.apache.hadoop.classification.InterfaceAudience;
+import org.apache.hadoop.classification.InterfaceStability;
+
+/** Block Construction Stage */
+@InterfaceAudience.Private
+@InterfaceStability.Evolving
+public enum BlockConstructionStage {
+  /** The enumerates are always listed as regular stage followed by the
+   * recovery stage. 
+   * Changing this order will make getRecoveryStage not working.
+   */
+  // pipeline set up for block append
+  PIPELINE_SETUP_APPEND,
+  // pipeline set up for failed PIPELINE_SETUP_APPEND recovery
+  PIPELINE_SETUP_APPEND_RECOVERY,
+  // data streaming
+  DATA_STREAMING,
+  // pipeline setup for failed data streaming recovery
+  PIPELINE_SETUP_STREAMING_RECOVERY,
+  // close the block and pipeline
+  PIPELINE_CLOSE,
+  // Recover a failed PIPELINE_CLOSE
+  PIPELINE_CLOSE_RECOVERY,
+  // pipeline set up for block creation
+  PIPELINE_SETUP_CREATE,
+  // transfer RBW for adding datanodes
+  TRANSFER_RBW,
+  // transfer Finalized for adding datanodes
+  TRANSFER_FINALIZED;
+  
+  final static private byte RECOVERY_BIT = (byte)1;
+  
+  /**
+   * get the recovery stage of this stage
+   */
+  public BlockConstructionStage getRecoveryStage() {
+    if (this == PIPELINE_SETUP_CREATE) {
+      throw new IllegalArgumentException( "Unexpected blockStage " + this);
+    } else {
+      return values()[ordinal()|RECOVERY_BIT];
+    }
+  }
+}    

+ 69 - 0
hdfs/src/java/org/apache/hadoop/hdfs/protocol/datatransfer/DataTransferProtoUtil.java

@@ -0,0 +1,69 @@
+/**
+ * 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.datatransfer;
+
+
+import org.apache.hadoop.classification.InterfaceAudience;
+import org.apache.hadoop.classification.InterfaceStability;
+import org.apache.hadoop.hdfs.protocol.ExtendedBlock;
+import org.apache.hadoop.hdfs.protocol.HdfsProtoUtil;
+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.
+ */
+@InterfaceAudience.Private
+@InterfaceStability.Evolving
+abstract class DataTransferProtoUtil {
+
+  static 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();
+  }
+}

+ 42 - 0
hdfs/src/java/org/apache/hadoop/hdfs/protocol/datatransfer/DataTransferProtocol.java

@@ -0,0 +1,42 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.hadoop.hdfs.protocol.datatransfer;
+
+import org.apache.commons.logging.Log;
+import org.apache.commons.logging.LogFactory;
+import org.apache.hadoop.classification.InterfaceAudience;
+import org.apache.hadoop.classification.InterfaceStability;
+
+/**
+ * Transfer data to/from datanode using a streaming protocol.
+ */
+@InterfaceAudience.Private
+@InterfaceStability.Evolving
+public interface DataTransferProtocol {
+  public static final Log LOG = LogFactory.getLog(DataTransferProtocol.class);
+  
+  /** Version for data transfers between clients and datanodes
+   * This should change when serialization of DatanodeInfo, not just
+   * when protocol changes. It is not very obvious. 
+   */
+  /*
+   * Version 27:
+   *    Move DataTransferProtocol and the inner classes to a package.
+   */
+  public static final int DATA_TRANSFER_VERSION = 27;
+}

+ 62 - 0
hdfs/src/java/org/apache/hadoop/hdfs/protocol/datatransfer/Op.java

@@ -0,0 +1,62 @@
+/**
+ * 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.datatransfer;
+
+import java.io.DataInput;
+import java.io.DataOutput;
+import java.io.IOException;
+
+import org.apache.hadoop.classification.InterfaceAudience;
+import org.apache.hadoop.classification.InterfaceStability;
+
+/** Operation */
+@InterfaceAudience.Private
+@InterfaceStability.Evolving
+public enum Op {
+  WRITE_BLOCK((byte)80),
+  READ_BLOCK((byte)81),
+  READ_METADATA((byte)82),
+  REPLACE_BLOCK((byte)83),
+  COPY_BLOCK((byte)84),
+  BLOCK_CHECKSUM((byte)85),
+  TRANSFER_BLOCK((byte)86);
+
+  /** The code for this operation. */
+  public final byte code;
+  
+  private Op(byte code) {
+    this.code = code;
+  }
+  
+  private static final int FIRST_CODE = values()[0].code;
+  /** Return the object represented by the code. */
+  private static Op valueOf(byte code) {
+    final int i = (code & 0xff) - FIRST_CODE;
+    return i < 0 || i >= values().length? null: values()[i];
+  }
+
+  /** Read from in */
+  public static Op read(DataInput in) throws IOException {
+    return valueOf(in.readByte());
+  }
+
+  /** Write to out */
+  public void write(DataOutput out) throws IOException {
+    out.write(code);
+  }
+}

+ 158 - 0
hdfs/src/java/org/apache/hadoop/hdfs/protocol/datatransfer/PacketHeader.java

@@ -0,0 +1,158 @@
+/**
+ * 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.datatransfer;
+
+import java.io.DataInputStream;
+import java.io.DataOutputStream;
+import java.io.IOException;
+import java.nio.ByteBuffer;
+
+import org.apache.hadoop.classification.InterfaceAudience;
+import org.apache.hadoop.classification.InterfaceStability;
+import org.apache.hadoop.hdfs.protocol.proto.DataTransferProtos.PacketHeaderProto;
+import org.apache.hadoop.hdfs.util.ByteBufferOutputStream;
+
+/**
+ * Header data for each packet that goes through the read/write pipelines.
+ */
+@InterfaceAudience.Private
+@InterfaceStability.Evolving
+public class PacketHeader {
+  /** Header size for a packet */
+  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 PacketHeaderProto proto;
+
+  public PacketHeader() {
+  }
+
+  public PacketHeader(int packetLen, long offsetInBlock, long seqno,
+                      boolean lastPacketInBlock, int dataLen) {
+    this.packetLen = packetLen;
+    proto = PacketHeaderProto.newBuilder()
+      .setOffsetInBlock(offsetInBlock)
+      .setSeqno(seqno)
+      .setLastPacketInBlock(lastPacketInBlock)
+      .setDataLen(dataLen)
+      .build();
+  }
+
+  public int getDataLen() {
+    return proto.getDataLen();
+  }
+
+  public boolean isLastPacketInBlock() {
+    return proto.getLastPacketInBlock();
+  }
+
+  public long getSeqno() {
+    return proto.getSeqno();
+  }
+
+  public long getOffsetInBlock() {
+    return proto.getOffsetInBlock();
+  }
+
+  public int getPacketLen() {
+    return packetLen;
+  }
+
+  @Override
+  public String toString() {
+    return "PacketHeader with packetLen=" + packetLen +
+      "Header data: " + 
+      proto.toString();
+  }
+  
+  public void readFields(ByteBuffer buf) throws IOException {
+    packetLen = 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);
+  }
+
+
+  /**
+   * Write the header into the buffer.
+   * This requires that PKT_HEADER_LEN bytes are available.
+   */
+  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);
+  }
+
+  /**
+   * Perform a sanity check on the packet, returning true if it is sane.
+   * @param lastSeqNo the previous sequence number received - we expect the current
+   * sequence number to be larger by 1.
+   */
+  public boolean sanityCheck(long lastSeqNo) {
+    // We should only have a non-positive data length for the last packet
+    if (proto.getDataLen() <= 0 && proto.getLastPacketInBlock()) return false;
+    // The last packet should not contain data
+    if (proto.getLastPacketInBlock() && proto.getDataLen() != 0) return false;
+    // Seqnos should always increase by 1 with each packet received
+    if (proto.getSeqno() != lastSeqNo + 1) return false;
+    return true;
+  }
+
+  @Override
+  public boolean equals(Object o) {
+    if (!(o instanceof PacketHeader)) return false;
+    PacketHeader other = (PacketHeader)o;
+    return this.proto.equals(other.proto);
+  }
+
+  @Override
+  public int hashCode() {
+    return (int)proto.getSeqno();
+  }
+}

+ 106 - 0
hdfs/src/java/org/apache/hadoop/hdfs/protocol/datatransfer/PipelineAck.java

@@ -0,0 +1,106 @@
+/**
+ * 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.datatransfer;
+
+import static org.apache.hadoop.hdfs.protocol.HdfsProtoUtil.vintPrefixed;
+
+import java.io.IOException;
+import java.io.InputStream;
+import java.io.OutputStream;
+import java.util.Arrays;
+
+import org.apache.hadoop.classification.InterfaceAudience;
+import org.apache.hadoop.classification.InterfaceStability;
+import org.apache.hadoop.hdfs.protocol.proto.DataTransferProtos;
+import org.apache.hadoop.hdfs.protocol.proto.DataTransferProtos.PipelineAckProto;
+import org.apache.hadoop.hdfs.protocol.proto.DataTransferProtos.Status;
+
+/** Pipeline Acknowledgment **/
+@InterfaceAudience.Private
+@InterfaceStability.Evolving
+public class PipelineAck {
+  PipelineAckProto proto;
+  public final static long UNKOWN_SEQNO = -2;
+
+  /** default constructor **/
+  public PipelineAck() {
+  }
+  
+  /**
+   * Constructor
+   * @param seqno sequence number
+   * @param replies an array of replies
+   */
+  public PipelineAck(long seqno, Status[] replies) {
+    proto = PipelineAckProto.newBuilder()
+      .setSeqno(seqno)
+      .addAllStatus(Arrays.asList(replies))
+      .build();
+  }
+  
+  /**
+   * Get the sequence number
+   * @return the sequence number
+   */
+  public long getSeqno() {
+    return proto.getSeqno();
+  }
+  
+  /**
+   * Get the number of replies
+   * @return the number of replies
+   */
+  public short getNumOfReplies() {
+    return (short)proto.getStatusCount();
+  }
+  
+  /**
+   * get the ith reply
+   * @return the the ith reply
+   */
+  public Status getReply(int i) {
+    return proto.getStatus(i);
+  }
+  
+  /**
+   * Check if this ack contains error status
+   * @return true if all statuses are SUCCESS
+   */
+  public boolean isSuccess() {
+    for (DataTransferProtos.Status reply : proto.getStatusList()) {
+      if (reply != DataTransferProtos.Status.SUCCESS) {
+        return false;
+      }
+    }
+    return true;
+  }
+  
+  /**** Writable interface ****/
+  public void readFields(InputStream in) throws IOException {
+    proto = PipelineAckProto.parseFrom(vintPrefixed(in));
+  }
+
+  public void write(OutputStream out) throws IOException {
+    proto.writeDelimitedTo(out);
+  }
+  
+  @Override //Object
+  public String toString() {
+    return proto.toString();
+  }
+}

+ 201 - 0
hdfs/src/java/org/apache/hadoop/hdfs/protocol/datatransfer/Receiver.java

@@ -0,0 +1,201 @@
+/**
+ * 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.datatransfer;
+
+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.vintPrefixed;
+import static org.apache.hadoop.hdfs.protocol.datatransfer.DataTransferProtoUtil.fromProto;
+
+import java.io.DataInputStream;
+import java.io.IOException;
+
+import org.apache.hadoop.classification.InterfaceAudience;
+import org.apache.hadoop.classification.InterfaceStability;
+import org.apache.hadoop.hdfs.protocol.DatanodeInfo;
+import org.apache.hadoop.hdfs.protocol.ExtendedBlock;
+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.security.token.block.BlockTokenIdentifier;
+import org.apache.hadoop.security.token.Token;
+
+/** Receiver */
+@InterfaceAudience.Private
+@InterfaceStability.Evolving
+public abstract class Receiver {
+  /** Read an Op.  It also checks protocol version. */
+  protected final Op readOp(DataInputStream in) throws IOException {
+    final short version = in.readShort();
+    if (version != DataTransferProtocol.DATA_TRANSFER_VERSION) {
+      throw new IOException( "Version Mismatch (Expected: " +
+          DataTransferProtocol.DATA_TRANSFER_VERSION  +
+          ", Received: " +  version + " )");
+    }
+    return Op.read(in);
+  }
+
+  /** Process op by the corresponding method. */
+  protected final void processOp(Op op, DataInputStream in
+      ) throws IOException {
+    switch(op) {
+    case READ_BLOCK:
+      opReadBlock(in);
+      break;
+    case WRITE_BLOCK:
+      opWriteBlock(in);
+      break;
+    case REPLACE_BLOCK:
+      opReplaceBlock(in);
+      break;
+    case COPY_BLOCK:
+      opCopyBlock(in);
+      break;
+    case BLOCK_CHECKSUM:
+      opBlockChecksum(in);
+      break;
+    case TRANSFER_BLOCK:
+      opTransferBlock(in);
+      break;
+    default:
+      throw new IOException("Unknown op " + op + " in data stream");
+    }
+  }
+
+  /** Receive OP_READ_BLOCK */
+  private void opReadBlock(DataInputStream in) throws IOException {
+    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.
+   */
+  protected abstract void opReadBlock(DataInputStream in, ExtendedBlock blk,
+      long offset, long length, String client,
+      Token<BlockTokenIdentifier> blockToken) throws IOException;
+  
+  /** Receive OP_WRITE_BLOCK */
+  private void opWriteBlock(DataInputStream in) throws IOException {
+    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()));
+  }
+
+  /**
+   * Abstract OP_WRITE_BLOCK method. 
+   * Write a block.
+   */
+  protected abstract void opWriteBlock(DataInputStream in, ExtendedBlock blk,
+      int pipelineSize, BlockConstructionStage stage, long newGs,
+      long minBytesRcvd, long maxBytesRcvd, String client, DatanodeInfo src,
+      DatanodeInfo[] targets, Token<BlockTokenIdentifier> blockToken)
+      throws IOException;
+
+  /** Receive {@link Op#TRANSFER_BLOCK} */
+  private void opTransferBlock(DataInputStream in) throws IOException {
+    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()));
+  }
+
+  /**
+   * Abstract {@link Op#TRANSFER_BLOCK} method.
+   * For {@link BlockConstructionStage#TRANSFER_RBW}
+   * or {@link BlockConstructionStage#TRANSFER_FINALIZED}.
+   */
+  protected abstract void opTransferBlock(DataInputStream in, ExtendedBlock blk,
+      String client, DatanodeInfo[] targets,
+      Token<BlockTokenIdentifier> blockToken)
+      throws IOException;
+
+  /** Receive OP_REPLACE_BLOCK */
+  private void opReplaceBlock(DataInputStream in) throws IOException {
+    OpReplaceBlockProto proto = OpReplaceBlockProto.parseFrom(vintPrefixed(in));
+
+    opReplaceBlock(in,
+        fromProto(proto.getHeader().getBlock()),
+        proto.getDelHint(),
+        fromProto(proto.getSource()),
+        fromProto(proto.getHeader().getToken()));
+  }
+
+  /**
+   * Abstract OP_REPLACE_BLOCK method.
+   * It is used for balancing purpose; send to a destination
+   */
+  protected abstract void opReplaceBlock(DataInputStream in,
+      ExtendedBlock blk, String delHint, DatanodeInfo src,
+      Token<BlockTokenIdentifier> blockToken) throws IOException;
+
+  /** Receive OP_COPY_BLOCK */
+  private void opCopyBlock(DataInputStream in) throws IOException {
+    OpCopyBlockProto proto = OpCopyBlockProto.parseFrom(vintPrefixed(in));
+    
+    opCopyBlock(in,
+        fromProto(proto.getHeader().getBlock()),
+        fromProto(proto.getHeader().getToken()));
+  }
+
+  /**
+   * Abstract OP_COPY_BLOCK method. It is used for balancing purpose; send to
+   * a proxy source.
+   */
+  protected abstract void opCopyBlock(DataInputStream in, ExtendedBlock blk,
+      Token<BlockTokenIdentifier> blockToken)
+      throws IOException;
+
+  /** Receive OP_BLOCK_CHECKSUM */
+  private void opBlockChecksum(DataInputStream in) throws IOException {
+    OpBlockChecksumProto proto = OpBlockChecksumProto.parseFrom(vintPrefixed(in));
+    
+    opBlockChecksum(in,
+        fromProto(proto.getHeader().getBlock()),
+        fromProto(proto.getHeader().getToken()));
+  }
+
+  /**
+   * Abstract OP_BLOCK_CHECKSUM method.
+   * Get the checksum of a block 
+   */
+  protected abstract void opBlockChecksum(DataInputStream in,
+      ExtendedBlock blk, Token<BlockTokenIdentifier> blockToken)
+      throws IOException;
+}

+ 117 - 0
hdfs/src/java/org/apache/hadoop/hdfs/protocol/datatransfer/ReplaceDatanodeOnFailure.java

@@ -0,0 +1,117 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.hadoop.hdfs.protocol.datatransfer;
+
+import org.apache.hadoop.HadoopIllegalArgumentException;
+import org.apache.hadoop.classification.InterfaceAudience;
+import org.apache.hadoop.classification.InterfaceStability;
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.hdfs.DFSConfigKeys;
+import org.apache.hadoop.hdfs.protocol.DatanodeInfo;
+
+/**
+ * The setting of replace-datanode-on-failure feature.
+ */
+@InterfaceAudience.Private
+@InterfaceStability.Evolving
+public enum ReplaceDatanodeOnFailure {
+  /** The feature is disabled in the entire site. */
+  DISABLE,
+  /** Never add a new datanode. */
+  NEVER,
+  /**
+   * DEFAULT policy:
+   *   Let r be the replication number.
+   *   Let n be the number of existing datanodes.
+   *   Add a new datanode only if r >= 3 and either
+   *   (1) floor(r/2) >= n; or
+   *   (2) r > n and the block is hflushed/appended.
+   */
+  DEFAULT,
+  /** Always add a new datanode when an existing datanode is removed. */
+  ALWAYS;
+
+  /** Check if the feature is enabled. */
+  public void checkEnabled() {
+    if (this == DISABLE) {
+      throw new UnsupportedOperationException(
+          "This feature is disabled.  Please refer to "
+          + DFSConfigKeys.DFS_CLIENT_WRITE_REPLACE_DATANODE_ON_FAILURE_ENABLE_KEY
+          + " configuration property.");
+    }
+  }
+
+  /** Is the policy satisfied? */
+  public boolean satisfy(
+      final short replication, final DatanodeInfo[] existings,
+      final boolean isAppend, final boolean isHflushed) {
+    final int n = existings == null? 0: existings.length;
+    if (n == 0 || n >= replication) {
+      //don't need to add datanode for any policy.
+      return false;
+    } else if (this == DISABLE || this == NEVER) {
+      return false;
+    } else if (this == ALWAYS) {
+      return true;
+    } else {
+      //DEFAULT
+      if (replication < 3) {
+        return false;
+      } else {
+        if (n <= (replication/2)) {
+          return true;
+        } else {
+          return isAppend || isHflushed;
+        }
+      }
+    }
+  }
+
+  /** Get the setting from configuration. */
+  public static ReplaceDatanodeOnFailure get(final Configuration conf) {
+    final boolean enabled = conf.getBoolean(
+        DFSConfigKeys.DFS_CLIENT_WRITE_REPLACE_DATANODE_ON_FAILURE_ENABLE_KEY,
+        DFSConfigKeys.DFS_CLIENT_WRITE_REPLACE_DATANODE_ON_FAILURE_ENABLE_DEFAULT);
+    if (!enabled) {
+      return DISABLE;
+    }
+
+    final String policy = conf.get(
+        DFSConfigKeys.DFS_CLIENT_WRITE_REPLACE_DATANODE_ON_FAILURE_POLICY_KEY,
+        DFSConfigKeys.DFS_CLIENT_WRITE_REPLACE_DATANODE_ON_FAILURE_POLICY_DEFAULT);
+    for(int i = 1; i < values().length; i++) {
+      final ReplaceDatanodeOnFailure rdof = values()[i];
+      if (rdof.name().equalsIgnoreCase(policy)) {
+        return rdof;
+      }
+    }
+    throw new HadoopIllegalArgumentException("Illegal configuration value for "
+        + DFSConfigKeys.DFS_CLIENT_WRITE_REPLACE_DATANODE_ON_FAILURE_POLICY_KEY
+        + ": " + policy);
+  }
+
+  /** Write the setting to configuration. */
+  public void write(final Configuration conf) {
+    conf.setBoolean(
+        DFSConfigKeys.DFS_CLIENT_WRITE_REPLACE_DATANODE_ON_FAILURE_ENABLE_KEY,
+        this != DISABLE);
+    conf.set(
+        DFSConfigKeys.DFS_CLIENT_WRITE_REPLACE_DATANODE_ON_FAILURE_POLICY_KEY,
+        name());
+  }
+}

+ 152 - 0
hdfs/src/java/org/apache/hadoop/hdfs/protocol/datatransfer/Sender.java

@@ -0,0 +1,152 @@
+/**
+ * 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.datatransfer;
+
+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.datatransfer.DataTransferProtoUtil.toProto;
+
+import java.io.DataOutput;
+import java.io.DataOutputStream;
+import java.io.IOException;
+
+import org.apache.hadoop.classification.InterfaceAudience;
+import org.apache.hadoop.classification.InterfaceStability;
+import org.apache.hadoop.hdfs.protocol.DatanodeInfo;
+import org.apache.hadoop.hdfs.protocol.ExtendedBlock;
+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.security.token.block.BlockTokenIdentifier;
+import org.apache.hadoop.security.token.Token;
+
+import com.google.protobuf.Message;
+
+/** Sender */
+@InterfaceAudience.Private
+@InterfaceStability.Evolving
+public class Sender {
+  /** Initialize a operation. */
+  private static void op(final DataOutput out, final Op op
+      ) throws IOException {
+    out.writeShort(DataTransferProtocol.DATA_TRANSFER_VERSION);
+    op.write(out);
+  }
+
+  private static void send(final DataOutputStream out, final Op opcode,
+      final Message proto) throws IOException {
+    op(out, opcode);
+    proto.writeDelimitedTo(out);
+    out.flush();
+  }
+
+  /** Send OP_READ_BLOCK */
+  public static void opReadBlock(DataOutputStream out, ExtendedBlock blk,
+      long blockOffset, long blockLen, String clientName,
+      Token<BlockTokenIdentifier> blockToken)
+      throws IOException {
+
+    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 {
+    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 {
+    
+    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 {
+    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 {
+    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 {
+    OpBlockChecksumProto proto = OpBlockChecksumProto.newBuilder()
+      .setHeader(DataTransferProtoUtil.buildBaseHeader(blk, blockToken))
+      .build();
+    
+    send(out, Op.BLOCK_CHECKSUM, proto);
+  }
+}

+ 9122 - 0
hdfs/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)
+}

+ 3247 - 0
hdfs/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)
+}

+ 9 - 6
hdfs/src/java/org/apache/hadoop/hdfs/server/balancer/Balancer.java

@@ -17,7 +17,7 @@
  */
 package org.apache.hadoop.hdfs.server.balancer;
 
-import static org.apache.hadoop.hdfs.protocol.DataTransferProtocol.Status.ERROR_ACCESS_TOKEN;
+import static org.apache.hadoop.hdfs.protocol.HdfsProtoUtil.vintPrefixed;
 
 import java.io.BufferedInputStream;
 import java.io.BufferedOutputStream;
@@ -53,11 +53,13 @@ import org.apache.hadoop.conf.Configured;
 import org.apache.hadoop.hdfs.DFSConfigKeys;
 import org.apache.hadoop.hdfs.DFSUtil;
 import org.apache.hadoop.hdfs.protocol.Block;
-import org.apache.hadoop.hdfs.protocol.DataTransferProtocol;
 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.FSConstants.DatanodeReportType;
+import org.apache.hadoop.hdfs.protocol.datatransfer.Sender;
+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;
@@ -346,15 +348,16 @@ public class Balancer {
     private void sendRequest(DataOutputStream out) throws IOException {
       final ExtendedBlock eb = new ExtendedBlock(nnc.blockpoolID, block.getBlock());
       final Token<BlockTokenIdentifier> accessToken = nnc.getAccessToken(eb);
-      DataTransferProtocol.Sender.opReplaceBlock(out, eb, source.getStorageID(), 
+      Sender.opReplaceBlock(out, eb, source.getStorageID(), 
           proxySource.getDatanode(), accessToken);
     }
     
     /* 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");
       }

+ 7 - 9
hdfs/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;
@@ -36,14 +34,14 @@ import java.util.zip.Checksum;
 
 import org.apache.commons.logging.Log;
 import org.apache.hadoop.fs.FSOutputSummer;
-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.datatransfer.BlockConstructionStage;
+import org.apache.hadoop.hdfs.protocol.datatransfer.PacketHeader;
+import org.apache.hadoop.hdfs.protocol.datatransfer.PipelineAck;
+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 @@ class BlockReceiver implements Closeable, FSConstants {
             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);
               }

+ 1 - 1
hdfs/src/java/org/apache/hadoop/hdfs/server/datanode/BlockSender.java

@@ -33,7 +33,7 @@ import org.apache.commons.logging.Log;
 import org.apache.hadoop.fs.ChecksumException;
 import org.apache.hadoop.hdfs.protocol.ExtendedBlock;
 import org.apache.hadoop.hdfs.protocol.FSConstants;
-import org.apache.hadoop.hdfs.protocol.DataTransferProtocol.PacketHeader;
+import org.apache.hadoop.hdfs.protocol.datatransfer.PacketHeader;
 import org.apache.hadoop.hdfs.util.DataTransferThrottler;
 import org.apache.hadoop.io.IOUtils;
 import org.apache.hadoop.net.SocketOutputStream;

+ 54 - 10
hdfs/src/java/org/apache/hadoop/hdfs/server/datanode/DataNode.java

@@ -18,9 +18,32 @@
 package org.apache.hadoop.hdfs.server.datanode;
 
 
-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.DFSConfigKeys.DFS_ADMIN;
+import static org.apache.hadoop.hdfs.DFSConfigKeys.DFS_BLOCKREPORT_INITIAL_DELAY_KEY;
+import static org.apache.hadoop.hdfs.DFSConfigKeys.DFS_BLOCKREPORT_INTERVAL_MSEC_KEY;
+import static org.apache.hadoop.hdfs.DFSConfigKeys.DFS_CLIENT_HTTPS_NEED_AUTH_DEFAULT;
+import static org.apache.hadoop.hdfs.DFSConfigKeys.DFS_CLIENT_HTTPS_NEED_AUTH_KEY;
+import static org.apache.hadoop.hdfs.DFSConfigKeys.DFS_CLIENT_SOCKET_TIMEOUT_KEY;
+import static org.apache.hadoop.hdfs.DFSConfigKeys.DFS_CLIENT_WRITE_PACKET_SIZE_DEFAULT;
+import static org.apache.hadoop.hdfs.DFSConfigKeys.DFS_CLIENT_WRITE_PACKET_SIZE_KEY;
+import static org.apache.hadoop.hdfs.DFSConfigKeys.DFS_DATANODE_DATA_DIR_KEY;
+import static org.apache.hadoop.hdfs.DFSConfigKeys.DFS_DATANODE_DATA_DIR_PERMISSION_DEFAULT;
+import static org.apache.hadoop.hdfs.DFSConfigKeys.DFS_DATANODE_DATA_DIR_PERMISSION_KEY;
+import static org.apache.hadoop.hdfs.DFSConfigKeys.DFS_DATANODE_DIRECTORYSCAN_INTERVAL_DEFAULT;
+import static org.apache.hadoop.hdfs.DFSConfigKeys.DFS_DATANODE_DIRECTORYSCAN_INTERVAL_KEY;
+import static org.apache.hadoop.hdfs.DFSConfigKeys.DFS_DATANODE_HANDLER_COUNT_DEFAULT;
+import static org.apache.hadoop.hdfs.DFSConfigKeys.DFS_DATANODE_HANDLER_COUNT_KEY;
+import static org.apache.hadoop.hdfs.DFSConfigKeys.DFS_DATANODE_HOST_NAME_KEY;
+import static org.apache.hadoop.hdfs.DFSConfigKeys.DFS_DATANODE_KEYTAB_FILE_KEY;
+import static org.apache.hadoop.hdfs.DFSConfigKeys.DFS_DATANODE_SCAN_PERIOD_HOURS_DEFAULT;
+import static org.apache.hadoop.hdfs.DFSConfigKeys.DFS_DATANODE_SCAN_PERIOD_HOURS_KEY;
+import static org.apache.hadoop.hdfs.DFSConfigKeys.DFS_DATANODE_SOCKET_WRITE_TIMEOUT_KEY;
+import static org.apache.hadoop.hdfs.DFSConfigKeys.DFS_DATANODE_STORAGEID_KEY;
+import static org.apache.hadoop.hdfs.DFSConfigKeys.DFS_DATANODE_SYNCONCLOSE_DEFAULT;
+import static org.apache.hadoop.hdfs.DFSConfigKeys.DFS_DATANODE_SYNCONCLOSE_KEY;
+import static org.apache.hadoop.hdfs.DFSConfigKeys.DFS_DATANODE_USER_NAME_KEY;
+import static org.apache.hadoop.hdfs.DFSConfigKeys.DFS_FEDERATION_NAMESERVICES;
+import static org.apache.hadoop.hdfs.DFSConfigKeys.DFS_HEARTBEAT_INTERVAL_KEY;
 import static org.apache.hadoop.hdfs.server.common.Util.now;
 
 import java.io.BufferedOutputStream;
@@ -68,15 +91,19 @@ import org.apache.hadoop.hdfs.HdfsConfiguration;
 import org.apache.hadoop.hdfs.protocol.Block;
 import org.apache.hadoop.hdfs.protocol.BlockListAsLongs;
 import org.apache.hadoop.hdfs.protocol.ClientDatanodeProtocol;
-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.FSConstants;
+import org.apache.hadoop.hdfs.protocol.HdfsProtoUtil;
 import org.apache.hadoop.hdfs.protocol.LocatedBlock;
 import org.apache.hadoop.hdfs.protocol.RecoveryInProgressException;
 import org.apache.hadoop.hdfs.protocol.UnregisteredNodeException;
+import org.apache.hadoop.hdfs.protocol.datatransfer.BlockConstructionStage;
+import org.apache.hadoop.hdfs.protocol.datatransfer.DataTransferProtocol;
+import org.apache.hadoop.hdfs.protocol.datatransfer.Sender;
+import org.apache.hadoop.hdfs.protocol.proto.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 +160,7 @@ import org.apache.hadoop.util.StringUtils;
 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
@@ -1397,6 +1425,10 @@ public class DataNode extends Configured
     return blockPoolManager.getAllNamenodeThreads();
   }
   
+  int getBpOsCount() {
+    return blockPoolManager.getAllNamenodeThreads().length;
+  }
+  
   /**
    * Initializes the {@link #data}. The initialization is done only once, when
    * handshake with the the first namenode is completed.
@@ -1945,7 +1977,7 @@ public class DataNode extends Configured
               EnumSet.of(BlockTokenSecretManager.AccessMode.WRITE));
         }
 
-        DataTransferProtocol.Sender.opWriteBlock(out,
+        Sender.opWriteBlock(out,
             b, 0, stage, 0, 0, 0, clientname, srcNode, targets, accessToken);
 
         // send data & checksum
@@ -1958,12 +1990,13 @@ public class DataNode extends Configured
         // 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));
@@ -2105,6 +2138,10 @@ public class DataNode extends Configured
     while (shouldRun) {
       try {
         blockPoolManager.joinAll();
+        if (blockPoolManager.getAllNamenodeThreads() != null
+            && blockPoolManager.getAllNamenodeThreads().length == 0) {
+          shouldRun = false;
+        }
         Thread.sleep(2000);
       } catch (InterruptedException ex) {
         LOG.warn("Received exception in Datanode#join: " + ex);
@@ -2243,6 +2280,13 @@ public class DataNode extends Configured
     } catch (Throwable e) {
       LOG.error(StringUtils.stringifyException(e));
       System.exit(-1);
+    } finally {
+      // We need to add System.exit here because either shutdown was called or
+      // some disk related conditions like volumes tolerated or volumes required
+      // condition was not met. Also, In secure mode, control will go to Jsvc
+      // and Datanode process hangs without System.exit.
+      LOG.warn("Exiting Datanode");
+      System.exit(0);
     }
   }
   

+ 101 - 62
hdfs/src/java/org/apache/hadoop/hdfs/server/datanode/DataXceiver.java

@@ -17,9 +17,9 @@
  */
 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.ERROR;
+import static org.apache.hadoop.hdfs.protocol.proto.DataTransferProtos.Status.ERROR_ACCESS_TOKEN;
+import static org.apache.hadoop.hdfs.protocol.proto.DataTransferProtos.Status.SUCCESS;
 import static org.apache.hadoop.hdfs.server.common.Util.now;
 import static org.apache.hadoop.hdfs.server.datanode.DataNode.DN_CLIENTTRACE_FORMAT;
 
@@ -39,11 +39,18 @@ import java.util.Arrays;
 
 import org.apache.commons.logging.Log;
 import org.apache.hadoop.hdfs.DFSConfigKeys;
-import org.apache.hadoop.hdfs.protocol.DataTransferProtocol;
-import org.apache.hadoop.hdfs.protocol.DataTransferProtocol.BlockConstructionStage;
 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.datatransfer.BlockConstructionStage;
+import org.apache.hadoop.hdfs.protocol.datatransfer.Op;
+import org.apache.hadoop.hdfs.protocol.datatransfer.Receiver;
+import org.apache.hadoop.hdfs.protocol.datatransfer.Sender;
+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.protocol.proto.DataTransferProtos.Status;
 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,20 +58,19 @@ import org.apache.hadoop.hdfs.server.datanode.FSDatasetInterface.MetaDataInputSt
 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;
 import org.apache.hadoop.security.token.SecretManager.InvalidToken;
 import org.apache.hadoop.security.token.Token;
 import org.apache.hadoop.util.DataChecksum;
 import org.apache.hadoop.util.StringUtils;
 
+import com.google.protobuf.ByteString;
+
+
 /**
  * Thread for processing incoming/outgoing data stream.
  */
-class DataXceiver extends DataTransferProtocol.Receiver
-    implements Runnable, FSConstants {
+class DataXceiver extends Receiver implements Runnable, FSConstants {
   public static final Log LOG = DataNode.LOG;
   static final Log ClientTraceLog = DataNode.ClientTraceLog;
   
@@ -123,6 +129,7 @@ class DataXceiver extends DataTransferProtocol.Receiver
 
     DataInputStream in=null; 
     int opsProcessed = 0;
+    Op op = null;
     try {
       in = new DataInputStream(
           new BufferedInputStream(NetUtils.getInputStream(s), 
@@ -133,7 +140,6 @@ class DataXceiver extends DataTransferProtocol.Receiver
       // This optimistic behaviour allows the other end to reuse connections.
       // Setting keepalive timeout to 0 disable this behavior.
       do {
-        DataTransferProtocol.Op op;
         try {
           if (opsProcessed != 0) {
             assert socketKeepaliveTimeout > 0;
@@ -172,10 +178,12 @@ class DataXceiver extends DataTransferProtocol.Receiver
         opStartTime = now();
         processOp(op, in);
         ++opsProcessed;
-      } while (s.isConnected() && socketKeepaliveTimeout > 0);
+      } while (!s.isClosed() && socketKeepaliveTimeout > 0);
     } catch (Throwable t) {
-      LOG.error(datanode.getMachineName() + ":DataXceiver, at " +
-        s.toString(), t);
+      LOG.error(datanode.getMachineName() + ":DataXceiver error processing " +
+                ((op == null) ? "unknown" : op.name()) + " operation " +
+                " src: " + remoteAddress +
+                " dest: " + localAddress, t);
     } finally {
       if (LOG.isDebugEnabled()) {
         LOG.debug(datanode.getMachineName() + ":Number of active connections is: "
@@ -200,8 +208,7 @@ class DataXceiver extends DataTransferProtocol.Receiver
     DataOutputStream out = new DataOutputStream(
                  new BufferedOutputStream(baseStream, SMALL_BUFFER_SIZE));
     checkAccess(out, true, block, blockToken,
-        DataTransferProtocol.Op.READ_BLOCK,
-        BlockTokenSecretManager.AccessMode.READ);
+        Op.READ_BLOCK, BlockTokenSecretManager.AccessMode.READ);
   
     // send the block
     BlockSender blockSender = null;
@@ -213,7 +220,7 @@ class DataXceiver extends DataTransferProtocol.Receiver
             "%d", "HDFS_READ", clientName, "%d",
             dnR.getStorageID(), block, "%d")
         : dnR + " Served block " + block + " to " +
-            s.getInetAddress();
+            remoteAddress;
 
     updateCurrentThreadName("Sending block " + block);
     try {
@@ -221,19 +228,23 @@ class DataXceiver extends DataTransferProtocol.Receiver
         blockSender = new BlockSender(block, startOffset, length,
             true, true, false, datanode, clientTraceFmt);
       } catch(IOException e) {
+        LOG.info("opReadBlock " + block + " received exception " + e);
         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);
@@ -248,6 +259,10 @@ class DataXceiver extends DataTransferProtocol.Receiver
       datanode.metrics.incrBytesRead((int) read);
       datanode.metrics.incrBlocksRead();
     } catch ( SocketException ignored ) {
+      if (LOG.isTraceEnabled()) {
+        LOG.trace(dnR + ":Ignoring exception while serving " + block + " to " +
+            remoteAddress, ignored);
+      }
       // Its ok for remote side to close the connection anytime.
       datanode.metrics.incrBlocksRead();
       IOUtils.closeStream(out);
@@ -257,7 +272,7 @@ class DataXceiver extends DataTransferProtocol.Receiver
        */
       LOG.warn(dnR +  ":Got exception while serving " + 
           block + " to " +
-                s.getInetAddress() + ":\n" + 
+                remoteAddress + ":\n" + 
                 StringUtils.stringifyException(ioe) );
       throw ioe;
     } finally {
@@ -320,8 +335,7 @@ class DataXceiver extends DataTransferProtocol.Receiver
             NetUtils.getOutputStream(s, datanode.socketWriteTimeout),
             SMALL_BUFFER_SIZE));
     checkAccess(replyOut, isClient, block, blockToken,
-        DataTransferProtocol.Op.WRITE_BLOCK,
-        BlockTokenSecretManager.AccessMode.WRITE);
+        Op.WRITE_BLOCK, BlockTokenSecretManager.AccessMode.WRITE);
 
     DataOutputStream mirrorOut = null;  // stream to next target
     DataInputStream mirrorIn = null;    // reply from next target
@@ -329,7 +343,7 @@ class DataXceiver extends DataTransferProtocol.Receiver
     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) {
@@ -366,7 +380,7 @@ class DataXceiver extends DataTransferProtocol.Receiver
                          SMALL_BUFFER_SIZE));
           mirrorIn = new DataInputStream(NetUtils.getInputStream(mirrorSock));
 
-          DataTransferProtocol.Sender.opWriteBlock(mirrorOut, originalBlock,
+          Sender.opWriteBlock(mirrorOut, originalBlock,
               pipelineSize, stage, newGs, minBytesRcvd, maxBytesRcvd, clientname,
               srcDataNode, targets, blockToken);
 
@@ -377,8 +391,10 @@ class DataXceiver extends DataTransferProtocol.Receiver
 
           // 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 +405,11 @@ class DataXceiver extends DataTransferProtocol.Receiver
 
         } 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);
@@ -400,6 +419,8 @@ class DataXceiver extends DataTransferProtocol.Receiver
           IOUtils.closeSocket(mirrorSock);
           mirrorSock = null;
           if (isClient) {
+            LOG.error(datanode + ":Exception transfering block " +
+                      block + " to mirror " + mirrorNode + ": " + e);
             throw e;
           } else {
             LOG.info(datanode + ":Exception transfering block " +
@@ -417,8 +438,11 @@ class DataXceiver extends DataTransferProtocol.Receiver
                    " 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 +457,7 @@ class DataXceiver extends DataTransferProtocol.Receiver
           if (LOG.isTraceEnabled()) {
             LOG.trace("TRANSFER: send close-ack");
           }
-          SUCCESS.write(replyOut);
+          writeResponse(SUCCESS, replyOut);
         }
       }
 
@@ -458,7 +482,7 @@ class DataXceiver extends DataTransferProtocol.Receiver
 
       
     } catch (IOException ioe) {
-      LOG.info("writeBlock " + block + " received exception " + ioe);
+      LOG.info("opWriteBlock " + block + " received exception " + ioe);
       throw ioe;
     } finally {
       // close all opened streams
@@ -480,21 +504,20 @@ class DataXceiver extends DataTransferProtocol.Receiver
       final DatanodeInfo[] targets,
       final Token<BlockTokenIdentifier> blockToken) throws IOException {
     checkAccess(null, true, blk, blockToken,
-        DataTransferProtocol.Op.TRANSFER_BLOCK,
-        BlockTokenSecretManager.AccessMode.COPY);
+        Op.TRANSFER_BLOCK, BlockTokenSecretManager.AccessMode.COPY);
 
-    updateCurrentThreadName(DataTransferProtocol.Op.TRANSFER_BLOCK + " " + blk);
+    updateCurrentThreadName(Op.TRANSFER_BLOCK + " " + blk);
 
     final DataOutputStream out = new DataOutputStream(
         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).
    */
@@ -504,8 +527,7 @@ class DataXceiver extends DataTransferProtocol.Receiver
     final DataOutputStream out = new DataOutputStream(
         NetUtils.getOutputStream(s, datanode.socketWriteTimeout));
     checkAccess(out, true, block, blockToken,
-        DataTransferProtocol.Op.BLOCK_CHECKSUM,
-        BlockTokenSecretManager.AccessMode.READ);
+        Op.BLOCK_CHECKSUM, BlockTokenSecretManager.AccessMode.READ);
     updateCurrentThreadName("Reading metadata for block " + block);
     final MetaDataInputStream metadataIn = 
       datanode.data.getMetaDataInputStream(block);
@@ -530,10 +552,15 @@ class DataXceiver extends DataTransferProtocol.Receiver
       }
 
       //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 +617,7 @@ class DataXceiver extends DataTransferProtocol.Receiver
           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);
@@ -601,6 +628,7 @@ class DataXceiver extends DataTransferProtocol.Receiver
       LOG.info("Copied block " + block + " to " + s.getRemoteSocketAddress());
     } catch (IOException ioe) {
       isOpSuccess = false;
+      LOG.info("opCopyBlock " + block + " received exception " + ioe);
       throw ioe;
     } finally {
       dataXceiverServer.balanceThrottler.release();
@@ -653,7 +681,7 @@ class DataXceiver extends DataTransferProtocol.Receiver
 
     Socket proxySock = null;
     DataOutputStream proxyOut = null;
-    DataTransferProtocol.Status opStatus = SUCCESS;
+    Status opStatus = SUCCESS;
     BlockReceiver blockReceiver = null;
     DataInputStream proxyReply = null;
     
@@ -671,15 +699,16 @@ class DataXceiver extends DataTransferProtocol.Receiver
                      new BufferedOutputStream(baseStream, SMALL_BUFFER_SIZE));
 
       /* send request to the proxy */
-      DataTransferProtocol.Sender.opCopyBlock(proxyOut, block, blockToken);
+      Sender.opCopyBlock(proxyOut, block, blockToken);
 
       // 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");
@@ -705,6 +734,7 @@ class DataXceiver extends DataTransferProtocol.Receiver
       
     } catch (IOException ioe) {
       opStatus = ERROR;
+      LOG.info("opReplaceBlock " + block + " received exception " + ioe);
       throw ioe;
     } finally {
       // receive the last byte that indicates the proxy released its thread resource
@@ -737,29 +767,35 @@ class DataXceiver extends DataTransferProtocol.Receiver
     return now() - opStartTime;
   }
 
-  private void updateCounter(MutableCounterLong localCounter,
-      MutableCounterLong remoteCounter) {
-    (isLocal? localCounter: remoteCounter).incr();
-  }
-
   /**
    * Utility function for sending a response.
    * @param s socket to write to
    * @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,
       final Token<BlockTokenIdentifier> t,
-      final DataTransferProtocol.Op op,
+      final Op op,
       final BlockTokenSecretManager.AccessMode mode) throws IOException {
     if (datanode.isBlockTokenEnabled) {
       try {
@@ -771,12 +807,15 @@ class DataXceiver extends DataTransferProtocol.Receiver
               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

+ 6 - 7
hdfs/src/java/org/apache/hadoop/hdfs/server/datanode/DataXceiverServer.java

@@ -47,7 +47,7 @@ class DataXceiverServer implements Runnable, FSConstants {
   
   ServerSocket ss;
   DataNode datanode;
-  // Record all sockets opend for data transfer
+  // Record all sockets opened for data transfer
   Map<Socket, Socket> childSockets = Collections.synchronizedMap(
                                        new HashMap<Socket, Socket>());
   
@@ -140,19 +140,18 @@ class DataXceiverServer implements Runnable, FSConstants {
       } catch (SocketTimeoutException ignored) {
         // wake up to see if should continue to run
       } catch (IOException ie) {
-        LOG.warn(datanode.getMachineName() + ":DataXceiveServer: " 
-                                + StringUtils.stringifyException(ie));
+        LOG.warn(datanode.getMachineName() + ":DataXceiveServer: ", ie);
       } catch (Throwable te) {
-        LOG.error(datanode.getMachineName() + ":DataXceiveServer: Exiting due to:" 
-                                 + StringUtils.stringifyException(te));
+        LOG.error(datanode.getMachineName()
+            + ":DataXceiveServer: Exiting due to: ", te);
         datanode.shouldRun = false;
       }
     }
     try {
       ss.close();
     } catch (IOException ie) {
-      LOG.warn(datanode.getMachineName() + ":DataXceiveServer: " 
-                              + StringUtils.stringifyException(ie));
+      LOG.warn(datanode.getMachineName()
+          + ":DataXceiveServer: Close exception due to: ", ie);
     }
   }
   

+ 7 - 0
hdfs/src/java/org/apache/hadoop/hdfs/server/datanode/DatanodeJspHelper.java

@@ -241,6 +241,11 @@ public class DatanodeJspHelper {
     }
     datanodePort = Integer.parseInt(datanodePortStr);
 
+    final Long genStamp = JspHelper.validateLong(req.getParameter("genstamp"));
+    if (genStamp == null) {
+      out.print("Invalid input (genstamp absent)");
+      return;
+    }
     String namenodeInfoPortStr = req.getParameter("namenodeInfoPort");
     int namenodeInfoPort = -1;
     if (namenodeInfoPortStr != null)
@@ -322,6 +327,8 @@ public class DatanodeJspHelper {
         + startOffset + "\">");
     out.print("<input type=\"hidden\" name=\"filename\" value=\"" + filename
         + "\">");
+    out.print("<input type=\"hidden\" name=\"genstamp\" value=\"" + genStamp
+        + "\">");
     out.print("<input type=\"hidden\" name=\"datanodePort\" value=\""
         + datanodePort + "\">");
     out.print("<input type=\"hidden\" name=\"namenodeInfoPort\" value=\""

+ 2 - 1
hdfs/src/java/org/apache/hadoop/hdfs/server/datanode/FSDataset.java

@@ -879,7 +879,8 @@ public class FSDataset implements FSConstants, FSDatasetInterface {
           if (removedVols == null) {
             removedVols = new ArrayList<FSVolume>(1);
           }
-          removedVols.add(volumeList.get(idx));
+          removedVols.add(fsv);
+          fsv.shutdown(); 
           volumeList.set(idx, null); // Remove the volume
           numFailedVolumes++;
         }

+ 1 - 1
hdfs/src/java/org/apache/hadoop/hdfs/server/namenode/BackupNode.java

@@ -189,7 +189,7 @@ public class BackupNode extends NameNode {
   @Override // NamenodeProtocol
   public NamenodeRegistration register(NamenodeRegistration registration
   ) throws IOException {
-    throw new UnsupportedActionException("journal");
+    throw new UnsupportedActionException("register");
   }
 
   @Override // NamenodeProtocol

+ 6 - 3
hdfs/src/java/org/apache/hadoop/hdfs/server/namenode/BlockManager.java

@@ -570,6 +570,7 @@ public class BlockManager {
    * dumps the contents of recentInvalidateSets
    */
   private void dumpRecentInvalidateSets(PrintWriter out) {
+    assert namesystem.hasWriteLock();
     int size = recentInvalidateSets.values().size();
     out.println("Metasave: Blocks " + pendingDeletionBlocksCount 
         + " waiting deletion from " + size + " datanodes.");
@@ -1258,7 +1259,7 @@ public class BlockManager {
     // Ignore replicas already scheduled to be removed from the DN
     if(belongsToInvalidates(dn.getStorageID(), block)) {
       assert storedBlock.findDatanode(dn) < 0 : "Block " + block
-        + " in recentInvalidatesSet should not appear in DN " + this;
+        + " in recentInvalidatesSet should not appear in DN " + dn;
       return storedBlock;
     }
 
@@ -1392,9 +1393,9 @@ public class BlockManager {
                                DatanodeDescriptor delNodeHint,
                                boolean logEveryBlock)
   throws IOException {
-    assert (block != null && namesystem.hasWriteLock());
+    assert block != null && namesystem.hasWriteLock();
     BlockInfo storedBlock;
-    if (block.getClass() == BlockInfoUnderConstruction.class) {
+    if (block instanceof BlockInfoUnderConstruction) {
       //refresh our copy in case the block got completed in another thread
       storedBlock = blocksMap.getStoredBlock(block);
     } else {
@@ -1571,6 +1572,7 @@ public class BlockManager {
    */
   void processOverReplicatedBlock(Block block, short replication,
       DatanodeDescriptor addedNode, DatanodeDescriptor delNodeHint) {
+    assert namesystem.hasWriteLock();
     if (addedNode == delNodeHint) {
       delNodeHint = null;
     }
@@ -1596,6 +1598,7 @@ public class BlockManager {
   }
 
   void addToExcessReplicate(DatanodeInfo dn, Block block) {
+    assert namesystem.hasWriteLock();
     Collection<Block> excessBlocks = excessReplicateMap.get(dn.getStorageID());
     if (excessBlocks == null) {
       excessBlocks = new TreeSet<Block>();

ファイルの差分が大きいため隠しています
+ 391 - 383
hdfs/src/java/org/apache/hadoop/hdfs/server/namenode/FSDirectory.java


+ 10 - 3
hdfs/src/java/org/apache/hadoop/hdfs/server/namenode/FSEditLogLoader.java

@@ -99,6 +99,9 @@ public class FSEditLogLoader {
         numOpRenewDelegationToken = 0, numOpCancelDelegationToken = 0, 
         numOpUpdateMasterKey = 0, numOpReassignLease = 0, numOpOther = 0;
 
+    fsNamesys.writeLock();
+    fsDir.writeLock();
+
     // Keep track of the file offsets of the last several opcodes.
     // This is handy when manually recovering corrupted edits files.
     PositionTrackingInputStream tracker = new PositionTrackingInputStream(in);
@@ -219,7 +222,8 @@ public class FSEditLogLoader {
             numOpConcatDelete++;
 
             ConcatDeleteOp concatDeleteOp = (ConcatDeleteOp)op;
-            fsDir.unprotectedConcat(concatDeleteOp.trg, concatDeleteOp.srcs);
+            fsDir.unprotectedConcat(concatDeleteOp.trg, concatDeleteOp.srcs,
+                concatDeleteOp.timestamp);
             break;
           }
           case OP_RENAME_OLD: {
@@ -228,7 +232,7 @@ public class FSEditLogLoader {
             HdfsFileStatus dinfo = fsDir.getFileInfo(renameOp.dst, false);
             fsDir.unprotectedRenameTo(renameOp.src, renameOp.dst,
                                       renameOp.timestamp);
-            fsNamesys.changeLease(renameOp.src, renameOp.dst, dinfo);
+            fsNamesys.unprotectedChangeLease(renameOp.src, renameOp.dst, dinfo);
             break;
           }
           case OP_DELETE: {
@@ -319,7 +323,7 @@ public class FSEditLogLoader {
             HdfsFileStatus dinfo = fsDir.getFileInfo(renameOp.dst, false);
             fsDir.unprotectedRenameTo(renameOp.src, renameOp.dst,
                                       renameOp.timestamp, renameOp.options);
-            fsNamesys.changeLease(renameOp.src, renameOp.dst, dinfo);
+            fsNamesys.unprotectedChangeLease(renameOp.src, renameOp.dst, dinfo);
             break;
           }
           case OP_GET_DELEGATION_TOKEN: {
@@ -409,6 +413,9 @@ public class FSEditLogLoader {
       String errorMessage = sb.toString();
       FSImage.LOG.error(errorMessage);
       throw new IOException(errorMessage, t);
+    } finally {
+      fsDir.writeUnlock();
+      fsNamesys.writeUnlock();
     }
     if (FSImage.LOG.isDebugEnabled()) {
       FSImage.LOG.debug("numOpAdd = " + numOpAdd + " numOpClose = " + numOpClose 

+ 1 - 4
hdfs/src/java/org/apache/hadoop/hdfs/server/namenode/FSEditLogOp.java

@@ -268,7 +268,6 @@ public abstract class FSEditLogOp {
     int length;
     String path;
     long timestamp;
-    long atime;
     PermissionStatus permissions;
 
     private MkdirOp() {
@@ -291,9 +290,7 @@ public abstract class FSEditLogOp {
       // However, currently this is not being updated/used because of
       // performance reasons.
       if (LayoutVersion.supports(Feature.FILE_ACCESS_TIME, logVersion)) {
-        this.atime = readLong(in);
-      } else {
-        this.atime = 0;
+        /*unused this.atime = */readLong(in);
       }
 
       if (logVersion <= -11) {

+ 3 - 18
hdfs/src/java/org/apache/hadoop/hdfs/server/namenode/FSImage.java

@@ -241,24 +241,9 @@ public class FSImage implements Closeable {
           + "Please restart NameNode with -upgrade option.");
     }
     
-    // Upgrade to federation requires -upgrade -clusterid <clusterID> option
-    if (startOpt == StartupOption.UPGRADE && 
-        !LayoutVersion.supports(Feature.FEDERATION, layoutVersion)) {
-      if (startOpt.getClusterId() == null) {
-        throw new IOException(
-            "\nFile system image contains an old layout version "
-                + layoutVersion + ".\nAn upgrade to version "
-                + FSConstants.LAYOUT_VERSION
-                + " is required.\nPlease restart NameNode with "
-                + "-upgrade -clusterid <clusterID> option.");
-      }
-      storage.setClusterID(startOpt.getClusterId());
-      
-      // Create new block pool Id
-      storage.setBlockPoolID(storage.newBlockPoolID());
-    }
-    
-    // check whether distributed upgrade is reguired and/or should be continued
+    storage.processStartupOptionsForUpgrade(startOpt, layoutVersion);
+
+    // check whether distributed upgrade is required and/or should be continued
     storage.verifyDistributedUpgradeProgress(startOpt);
 
     // 2. Format unformatted dirs.

ファイルの差分が大きいため隠しています
+ 401 - 324
hdfs/src/java/org/apache/hadoop/hdfs/server/namenode/FSNamesystem.java


+ 1 - 0
hdfs/src/java/org/apache/hadoop/hdfs/server/namenode/LeaseManager.java

@@ -391,6 +391,7 @@ public class LeaseManager {
 
   /** Check the leases beginning from the oldest. */
   private synchronized void checkLeases() {
+    assert fsnamesystem.hasWriteLock();
     for(; sortedLeases.size() > 0; ) {
       final Lease oldest = sortedLeases.first();
       if (!oldest.expiredHardLimit()) {

+ 34 - 0
hdfs/src/java/org/apache/hadoop/hdfs/server/namenode/NNStorage.java

@@ -868,6 +868,40 @@ public class NNStorage extends Storage implements Closeable {
     LOG.debug("at the end current list of storage dirs:" + lsd);
   }
   
+  /** 
+   * Processes the startup options for the clusterid and blockpoolid 
+   * for the upgrade. 
+   * @param startOpt Startup options 
+   * @param layoutVersion Layout version for the upgrade 
+   * @throws IOException
+   */
+  void processStartupOptionsForUpgrade(StartupOption startOpt, int layoutVersion)
+      throws IOException {
+    if (startOpt == StartupOption.UPGRADE) {
+      // If upgrade from a release that does not support federation,
+      // if clusterId is provided in the startupOptions use it.
+      // Else generate a new cluster ID      
+      if (!LayoutVersion.supports(Feature.FEDERATION, layoutVersion)) {
+        if (startOpt.getClusterId() == null) {
+          startOpt.setClusterId(newClusterID());
+        }
+        setClusterID(startOpt.getClusterId());
+        setBlockPoolID(newBlockPoolID());
+      } else {
+        // Upgrade from one version of federation to another supported
+        // version of federation doesn't require clusterID.
+        // Warn the user if the current clusterid didn't match with the input
+        // clusterid.
+        if (startOpt.getClusterId() != null
+            && !startOpt.getClusterId().equals(getClusterID())) {
+          LOG.warn("Clusterid mismatch - current clusterid: " + getClusterID()
+              + ", Ignoring given clusterid: " + startOpt.getClusterId());
+        }
+      }
+      LOG.info("Using clusterid: " + getClusterID());
+    }
+  }
+  
   /**
    * Generate new clusterID.
    * 

+ 78 - 135
hdfs/src/java/org/apache/hadoop/hdfs/server/namenode/NameNode.java

@@ -195,7 +195,7 @@ public class NameNode implements NamenodeProtocols, FSConstants {
   }
     
 
-  @Override
+  @Override // VersionedProtocol
   public ProtocolSignature getProtocolSignature(String protocol,
       long clientVersion, int clientMethodsHash) throws IOException {
     return ProtocolSignature.getProtocolSignature(
@@ -696,7 +696,7 @@ public class NameNode implements NamenodeProtocols, FSConstants {
     return namesystem.getBlocks(datanode, size); 
   }
 
-  /** {@inheritDoc} */
+  @Override // NamenodeProtocol
   public ExportedBlockKeys getBlockKeys() throws IOException {
     return namesystem.getBlockKeys();
   }
@@ -744,39 +744,34 @@ public class NameNode implements NamenodeProtocols, FSConstants {
     return namesystem.getEditLogSize();
   }
 
-  /*
-   * Active name-node cannot journal.
-   */
   @Override // NamenodeProtocol
   public void journal(NamenodeRegistration registration,
                       int jAction,
                       int length,
                       byte[] args) throws IOException {
+    // Active name-node cannot journal.
     throw new UnsupportedActionException("journal");
   }
 
-  /////////////////////////////////////////////////////
-  // ClientProtocol
-  /////////////////////////////////////////////////////
-  
+  @Override // ClientProtocol
   public Token<DelegationTokenIdentifier> getDelegationToken(Text renewer)
       throws IOException {
     return namesystem.getDelegationToken(renewer);
   }
 
-  @Override
+  @Override // ClientProtocol
   public long renewDelegationToken(Token<DelegationTokenIdentifier> token)
       throws InvalidToken, IOException {
     return namesystem.renewDelegationToken(token);
   }
 
-  @Override
+  @Override // ClientProtocol
   public void cancelDelegationToken(Token<DelegationTokenIdentifier> token)
       throws IOException {
     namesystem.cancelDelegationToken(token);
   }
   
-  /** {@inheritDoc} */
+  @Override // ClientProtocol
   public LocatedBlocks getBlockLocations(String src, 
                                           long offset, 
                                           long length) 
@@ -786,20 +781,12 @@ public class NameNode implements NamenodeProtocols, FSConstants {
                                         src, offset, length);
   }
   
-  private static String getClientMachine() {
-    String clientMachine = Server.getRemoteAddress();
-    if (clientMachine == null) {
-      clientMachine = "";
-    }
-    return clientMachine;
-  }
-
-  /** {@inheritDoc} */
+  @Override // ClientProtocol
   public FsServerDefaults getServerDefaults() throws IOException {
     return namesystem.getServerDefaults();
   }
 
-  /** {@inheritDoc} */
+  @Override // ClientProtocol
   public void create(String src, 
                      FsPermission masked,
                      String clientName, 
@@ -824,7 +811,7 @@ public class NameNode implements NamenodeProtocols, FSConstants {
     metrics.incrCreateFileOps();
   }
 
-  /** {@inheritDoc} */
+  @Override // ClientProtocol
   public LocatedBlock append(String src, String clientName) 
       throws IOException {
     String clientMachine = getClientMachine();
@@ -837,31 +824,31 @@ public class NameNode implements NamenodeProtocols, FSConstants {
     return info;
   }
 
-  /** {@inheritDoc} */
+  @Override // ClientProtocol
   public boolean recoverLease(String src, String clientName) throws IOException {
     String clientMachine = getClientMachine();
     return namesystem.recoverLease(src, clientName, clientMachine);
   }
 
-  /** {@inheritDoc} */
+  @Override // ClientProtocol
   public boolean setReplication(String src, short replication) 
     throws IOException {  
     return namesystem.setReplication(src, replication);
   }
     
-  /** {@inheritDoc} */
+  @Override // ClientProtocol
   public void setPermission(String src, FsPermission permissions)
       throws IOException {
     namesystem.setPermission(src, permissions);
   }
 
-  /** {@inheritDoc} */
+  @Override // ClientProtocol
   public void setOwner(String src, String username, String groupname)
       throws IOException {
     namesystem.setOwner(src, username, groupname);
   }
 
-  @Override
+  @Override // ClientProtocol
   public LocatedBlock addBlock(String src,
                                String clientName,
                                ExtendedBlock previous,
@@ -885,7 +872,7 @@ public class NameNode implements NamenodeProtocols, FSConstants {
     return locatedBlock;
   }
 
-  @Override
+  @Override // ClientProtocol
   public LocatedBlock getAdditionalDatanode(final String src, final ExtendedBlock blk,
       final DatanodeInfo[] existings, final DatanodeInfo[] excludes,
       final int numAdditionalNodes, final String clientName
@@ -926,7 +913,7 @@ public class NameNode implements NamenodeProtocols, FSConstants {
     }
   }
 
-  /** {@inheritDoc} */
+  @Override // ClientProtocol
   public boolean complete(String src, String clientName, ExtendedBlock last)
       throws IOException {
     if(stateChangeLog.isDebugEnabled()) {
@@ -942,6 +929,7 @@ public class NameNode implements NamenodeProtocols, FSConstants {
    * mark the block as corrupt.  In the future we might 
    * check the blocks are actually corrupt. 
    */
+  @Override
   public void reportBadBlocks(LocatedBlock[] blocks) throws IOException {
     stateChangeLog.info("*DIR* NameNode.reportBadBlocks");
     for (int i = 0; i < blocks.length; i++) {
@@ -954,22 +942,21 @@ public class NameNode implements NamenodeProtocols, FSConstants {
     }
   }
 
-  /** {@inheritDoc} */
-  @Override
+  @Override // ClientProtocol
   public LocatedBlock updateBlockForPipeline(ExtendedBlock block, String clientName)
       throws IOException {
     return namesystem.updateBlockForPipeline(block, clientName);
   }
 
 
-  @Override
+  @Override // ClientProtocol
   public void updatePipeline(String clientName, ExtendedBlock oldBlock,
       ExtendedBlock newBlock, DatanodeID[] newNodes)
       throws IOException {
     namesystem.updatePipeline(clientName, oldBlock, newBlock, newNodes);
   }
   
-  /** {@inheritDoc} */
+  @Override // DatanodeProtocol
   public void commitBlockSynchronization(ExtendedBlock block,
       long newgenerationstamp, long newlength,
       boolean closeFile, boolean deleteblock, DatanodeID[] newtargets)
@@ -978,14 +965,14 @@ public class NameNode implements NamenodeProtocols, FSConstants {
         newgenerationstamp, newlength, closeFile, deleteblock, newtargets);
   }
   
+  @Override // ClientProtocol
   public long getPreferredBlockSize(String filename) 
       throws IOException {
     return namesystem.getPreferredBlockSize(filename);
   }
     
-  /** {@inheritDoc} */
   @Deprecated
-  @Override
+  @Override // ClientProtocol
   public boolean rename(String src, String dst) throws IOException {
     if(stateChangeLog.isDebugEnabled()) {
       stateChangeLog.debug("*DIR* NameNode.rename: " + src + " to " + dst);
@@ -1001,15 +988,12 @@ public class NameNode implements NamenodeProtocols, FSConstants {
     return ret;
   }
   
-  /** 
-   * {@inheritDoc}
-   */
+  @Override // ClientProtocol
   public void concat(String trg, String[] src) throws IOException {
     namesystem.concat(trg, src);
   }
   
-  /** {@inheritDoc} */
-  @Override
+  @Override // ClientProtocol
   public void rename(String src, String dst, Options.Rename... options)
       throws IOException {
     if(stateChangeLog.isDebugEnabled()) {
@@ -1023,14 +1007,13 @@ public class NameNode implements NamenodeProtocols, FSConstants {
     metrics.incrFilesRenamed();
   }
 
-  /**
-   */
   @Deprecated
+  @Override // ClientProtocol
   public boolean delete(String src) throws IOException {
     return delete(src, true);
   }
 
-  /** {@inheritDoc} */
+  @Override // ClientProtocol
   public boolean delete(String src, boolean recursive) throws IOException {
     if (stateChangeLog.isDebugEnabled()) {
       stateChangeLog.debug("*DIR* Namenode.delete: src=" + src
@@ -1046,7 +1029,6 @@ public class NameNode implements NamenodeProtocols, FSConstants {
    * Check path length does not exceed maximum.  Returns true if
    * length and depth are okay.  Returns false if length is too long 
    * or depth is too great.
-   * 
    */
   private boolean checkPathLength(String src) {
     Path srcPath = new Path(src);
@@ -1054,7 +1036,7 @@ public class NameNode implements NamenodeProtocols, FSConstants {
             srcPath.depth() <= MAX_PATH_DEPTH);
   }
     
-  /** {@inheritDoc} */
+  @Override // ClientProtocol
   public boolean mkdirs(String src, FsPermission masked, boolean createParent)
       throws IOException {
     if(stateChangeLog.isDebugEnabled()) {
@@ -1069,15 +1051,12 @@ public class NameNode implements NamenodeProtocols, FSConstants {
             null, masked), createParent);
   }
 
-  /**
-   */
+  @Override // ClientProtocol
   public void renewLease(String clientName) throws IOException {
     namesystem.renewLease(clientName);        
   }
 
-  /**
-   */
-  @Override
+  @Override // ClientProtocol
   public DirectoryListing getListing(String src, byte[] startAfter,
       boolean needLocation)
   throws IOException {
@@ -1090,24 +1069,13 @@ public class NameNode implements NamenodeProtocols, FSConstants {
     return files;
   }
 
-  /**
-   * Get the file info for a specific file.
-   * @param src The string representation of the path to the file
-   * @return object containing information regarding the file
-   *         or null if file not found
-   */
+  @Override // ClientProtocol
   public HdfsFileStatus getFileInfo(String src)  throws IOException {
     metrics.incrFileInfoOps();
     return namesystem.getFileInfo(src, true);
   }
 
-  /**
-   * Get the file info for a specific file. If the path refers to a 
-   * symlink then the FileStatus of the symlink is returned.
-   * @param src The string representation of the path to the file
-   * @return object containing information regarding the file
-   *         or null if file not found
-   */
+  @Override // ClientProtocol
   public HdfsFileStatus getFileLinkInfo(String src) throws IOException { 
     metrics.incrFileInfoOps();
     return namesystem.getFileInfo(src, false);
@@ -1118,8 +1086,7 @@ public class NameNode implements NamenodeProtocols, FSConstants {
     return namesystem.getStats();
   }
 
-  /**
-   */
+  @Override // ClientProtocol
   public DatanodeInfo[] getDatanodeReport(DatanodeReportType type)
       throws IOException {
     DatanodeInfo results[] = namesystem.datanodeReport(type);
@@ -1129,7 +1096,7 @@ public class NameNode implements NamenodeProtocols, FSConstants {
     return results;
   }
     
-  @Override
+  @Override // ClientProtocol
   public boolean setSafeMode(SafeModeAction action) throws IOException {
     return namesystem.setSafeMode(action);
   }
@@ -1141,38 +1108,29 @@ public class NameNode implements NamenodeProtocols, FSConstants {
     return namesystem.isInSafeMode();
   }
 
-  @Override
+  @Override // ClientProtocol
   public boolean restoreFailedStorage(String arg) 
       throws AccessControlException {
     return namesystem.restoreFailedStorage(arg);
   }
 
-  @Override
+  @Override // ClientProtocol
   public void saveNamespace() throws IOException {
     namesystem.saveNamespace();
   }
 
-  /**
-   * Refresh the list of datanodes that the namenode should allow to  
-   * connect.  Re-reads conf by creating new HdfsConfiguration object and 
-   * uses the files list in the configuration to update the list. 
-   */
+  @Override // ClientProtocol
   public void refreshNodes() throws IOException {
     namesystem.refreshNodes(new HdfsConfiguration());
   }
 
-  /**
-   * Returns the size of the current edit log.
-   */
-  @Deprecated
+  @Deprecated // NamenodeProtocol
   public long getEditLogSize() throws IOException {
     return namesystem.getEditLogSize();
   }
 
-  /**
-   * Roll the edit log.
-   */
   @Deprecated
+  @Override // NamenodeProtocol
   public CheckpointSignature rollEditLog() throws IOException {
     return namesystem.rollEditLog();
   }
@@ -1183,29 +1141,25 @@ public class NameNode implements NamenodeProtocols, FSConstants {
     return namesystem.getEditLogManifest(sinceTxId);
   }
     
+  @Override // ClientProtocol
   public void finalizeUpgrade() throws IOException {
     namesystem.finalizeUpgrade();
   }
 
+  @Override // ClientProtocol
   public UpgradeStatusReport distributedUpgradeProgress(UpgradeAction action)
       throws IOException {
     return namesystem.distributedUpgradeProgress(action);
   }
 
-  /**
-   * Dumps namenode state into specified file
-   */
+  @Override // ClientProtocol
   public void metaSave(String filename) throws IOException {
     namesystem.metaSave(filename);
   }
 
-  /**
-   * {@inheritDoc}
-   */
-  @Override
-  public CorruptFileBlocks
-    listCorruptFileBlocks(String path, String cookie) 
-    throws IOException {
+  @Override // ClientProtocol
+  public CorruptFileBlocks listCorruptFileBlocks(String path, String cookie)
+      throws IOException {
     Collection<FSNamesystem.CorruptFileBlockInfo> fbs =
       namesystem.listCorruptFileBlocks(path, cookie);
     
@@ -1219,32 +1173,31 @@ public class NameNode implements NamenodeProtocols, FSConstants {
     return new CorruptFileBlocks(files, lastCookie);
   }
   
-  /** {@inheritDoc} */
+  @Override // ClientProtocol
   public ContentSummary getContentSummary(String path) throws IOException {
     return namesystem.getContentSummary(path);
   }
 
-  /** {@inheritDoc} */
+  @Override // ClientProtocol
   public void setQuota(String path, long namespaceQuota, long diskspaceQuota) 
       throws IOException {
     namesystem.setQuota(path, namespaceQuota, diskspaceQuota);
   }
   
-  /** {@inheritDoc} */
+  @Override // ClientProtocol
   public void fsync(String src, String clientName) throws IOException {
     namesystem.fsync(src, clientName);
   }
 
-  @Override
+  @Override // ClientProtocol
   public void setTimes(String src, long mtime, long atime) 
       throws IOException {
     namesystem.setTimes(src, mtime, atime);
   }
 
-  @Override
-  public void createSymlink(String target, String link, FsPermission dirPerms, 
-                            boolean createParent) 
-      throws IOException {
+  @Override // ClientProtocol
+  public void createSymlink(String target, String link, FsPermission dirPerms,
+      boolean createParent) throws IOException {
     metrics.incrCreateSymlinkOps();
     /* We enforce the MAX_PATH_LENGTH limit even though a symlink target 
      * URI may refer to a non-HDFS file system. 
@@ -1262,7 +1215,7 @@ public class NameNode implements NamenodeProtocols, FSConstants {
       new PermissionStatus(ugi.getShortUserName(), null, dirPerms), createParent);
   }
 
-  @Override
+  @Override // ClientProtocol
   public String getLinkTarget(String path) throws IOException {
     metrics.incrGetLinkTargetOps();
     /* Resolves the first symlink in the given path, returning a
@@ -1285,11 +1238,7 @@ public class NameNode implements NamenodeProtocols, FSConstants {
   }
 
 
-  ////////////////////////////////////////////////////////////////
-  // DatanodeProtocol
-  ////////////////////////////////////////////////////////////////
-  /** 
-   */
+  @Override // DatanodeProtocol
   public DatanodeRegistration registerDatanode(DatanodeRegistration nodeReg)
       throws IOException {
     verifyVersion(nodeReg.getVersion());
@@ -1298,32 +1247,19 @@ public class NameNode implements NamenodeProtocols, FSConstants {
     return nodeReg;
   }
 
-  /**
-   * Data node notify the name node that it is alive 
-   * Return an array of block-oriented commands for the datanode to execute.
-   * This will be either a transfer or a delete operation.
-   */
+  @Override // DatanodeProtocol
   public DatanodeCommand[] sendHeartbeat(DatanodeRegistration nodeReg,
-                                       long capacity,
-                                       long dfsUsed,
-                                       long remaining,
-                                       long blockPoolUsed,
-                                       int xmitsInProgress,
-                                       int xceiverCount,
-                                       int failedVolumes) throws IOException {
+      long capacity, long dfsUsed, long remaining, long blockPoolUsed,
+      int xmitsInProgress, int xceiverCount, int failedVolumes)
+      throws IOException {
     verifyRequest(nodeReg);
     return namesystem.handleHeartbeat(nodeReg, capacity, dfsUsed, remaining,
         blockPoolUsed, xceiverCount, xmitsInProgress, failedVolumes);
   }
 
-  /**
-   * sends block report to the corresponding namenode (for the poolId)
-   * @return DataNodeCommand from the namenode
-   * @throws IOException
-   */
+  @Override // DatanodeProtocol
   public DatanodeCommand blockReport(DatanodeRegistration nodeReg,
-                                     String poolId,
-                                     long[] blocks) throws IOException {
+      String poolId, long[] blocks) throws IOException {
     verifyRequest(nodeReg);
     BlockListAsLongs blist = new BlockListAsLongs(blocks);
     if(stateChangeLog.isDebugEnabled()) {
@@ -1338,10 +1274,9 @@ public class NameNode implements NamenodeProtocols, FSConstants {
     return null;
   }
 
-  public void blockReceived(DatanodeRegistration nodeReg, 
-                            String poolId,
-                            Block blocks[],
-                            String delHints[]) throws IOException {
+  @Override // DatanodeProtocol
+  public void blockReceived(DatanodeRegistration nodeReg, String poolId,
+      Block blocks[], String delHints[]) throws IOException {
     verifyRequest(nodeReg);
     if(stateChangeLog.isDebugEnabled()) {
       stateChangeLog.debug("*BLOCK* NameNode.blockReceived: "
@@ -1352,9 +1287,7 @@ public class NameNode implements NamenodeProtocols, FSConstants {
     }
   }
 
-  /**
-   * Handle an error report from a datanode.
-   */
+  @Override // DatanodeProtocol
   public void errorReport(DatanodeRegistration nodeReg,
                           int errorCode, String msg) throws IOException { 
     String dnName = (nodeReg == null ? "unknown DataNode" : nodeReg.getName());
@@ -1375,10 +1308,12 @@ public class NameNode implements NamenodeProtocols, FSConstants {
     }
   }
     
+  @Override // DatanodeProtocol, NamenodeProtocol
   public NamespaceInfo versionRequest() throws IOException {
     return namesystem.getNamespaceInfo();
   }
 
+  @Override // DatanodeProtocol
   public UpgradeCommand processUpgradeCommand(UpgradeCommand comm) throws IOException {
     return namesystem.processDistributedUpgradeCommand(comm);
   }
@@ -1520,7 +1455,7 @@ public class NameNode implements NamenodeProtocols, FSConstants {
     return false;
   }
 
-  @Override
+  @Override // RefreshAuthorizationPolicyProtocol
   public void refreshServiceAcl() throws IOException {
     if (!serviceAuthEnabled) {
       throw new AuthorizationException("Service Level Authorization not enabled!");
@@ -1532,21 +1467,21 @@ public class NameNode implements NamenodeProtocols, FSConstants {
     }
   }
 
-  @Override
+  @Override // RefreshAuthorizationPolicyProtocol
   public void refreshUserToGroupsMappings() throws IOException {
     LOG.info("Refreshing all user-to-groups mappings. Requested by user: " + 
              UserGroupInformation.getCurrentUser().getShortUserName());
     Groups.getUserToGroupsMappingService().refresh();
   }
 
-  @Override
+  @Override // RefreshAuthorizationPolicyProtocol
   public void refreshSuperUserGroupsConfiguration() {
     LOG.info("Refreshing SuperUser proxy group mapping list ");
 
     ProxyUsers.refreshSuperUserGroupsConfiguration();
   }
   
-  @Override
+  @Override // GetUserMappingsProtocol
   public String[] getGroupsForUser(String user) throws IOException {
     if (LOG.isDebugEnabled()) {
       LOG.debug("Getting groups for user " + user);
@@ -1725,4 +1660,12 @@ public class NameNode implements NamenodeProtocols, FSConstants {
       System.exit(-1);
     }
   }
+  
+  private static String getClientMachine() {
+    String clientMachine = Server.getRemoteAddress();
+    if (clientMachine == null) {
+      clientMachine = "";
+    }
+    return clientMachine;
+  }
 }

+ 17 - 4
hdfs/src/java/org/apache/hadoop/hdfs/server/namenode/NamenodeJspHelper.java

@@ -142,12 +142,25 @@ class NamenodeJspHelper {
         + "\n</table></div>";
   }
 
-  static String getWarningText(FSNamesystem fsn) {
-    // Ideally this should be displayed in RED
+  /**
+   * Generate warning text if there are corrupt files.
+   * @return a warning if files are corrupt, otherwise return an empty string.
+   */
+  static String getCorruptFilesWarning(FSNamesystem fsn) {
     long missingBlocks = fsn.getMissingBlocksCount();
     if (missingBlocks > 0) {
-      return "<br> WARNING :" + " There are " + missingBlocks
-          + " missing blocks. Please check the log or run fsck. <br><br>";
+      StringBuilder result = new StringBuilder();
+
+      // Warning class is typically displayed in RED
+      result.append("<br/><a class=\"warning\" href=\"/corrupt_files.jsp\" title=\"List corrupt files\">\n");
+      result.append("<b>WARNING : There are " + missingBlocks
+          + " missing blocks. Please check the logs or run fsck in order to identify the missing blocks.</b>");
+      result.append("</a>");
+
+      result.append("<br/><div class=\"small\">See the Hadoop FAQ for common causes and potential solutions.");
+      result.append("<br/><br/>\n");
+
+      return result.toString();
     }
     return "";
   }

+ 2 - 1
hdfs/src/java/org/apache/hadoop/hdfs/tools/DelegationTokenFetcher.java

@@ -80,7 +80,8 @@ public class DelegationTokenFetcher {
     err.println("  --webservice <url>  Url to contact NN on");
     err.println("  --renewer <name>    Name of the delegation token renewer");
     err.println("  --cancel            Cancel the delegation token");
-    err.println("  --renew             Renew the delegation token");
+    err.println("  --renew             Renew the delegation token.  Delegation " 
+    		+ "token must have been fetched using the --renewer <name> option.");
     err.println("  --print             Print the delegation token");
     err.println();
     GenericOptionsParser.printGenericCommandUsage(err);

+ 49 - 0
hdfs/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);
+  }
+}

+ 125 - 0
hdfs/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();
+  }
+  
+}

+ 140 - 0
hdfs/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;
+}

+ 64 - 0
hdfs/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;
+}
+

+ 5 - 7
hdfs/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,8 +32,7 @@ import org.apache.hadoop.hdfs.server.datanode.BlockReceiver.PacketResponder;
 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.protocol.datatransfer.PipelineAck;
 import org.apache.hadoop.hdfs.server.protocol.DatanodeRegistration;
 import org.apache.hadoop.util.DiskChecker.DiskOutOfSpaceException;
 
@@ -83,7 +81,7 @@ privileged public aspect BlockReceiverAspects {
   }
 
   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 @@ privileged public aspect BlockReceiverAspects {
   }
   
   pointcut preventAckSending () :
-    call (void PipelineAck.write(DataOutput)) 
+    call (void PipelineAck.write(OutputStream)) 
     && within (PacketResponder);
 
   static int ackCounter = 0;
@@ -203,7 +201,7 @@ privileged public aspect BlockReceiverAspects {
   }
 
   pointcut pipelineAck(BlockReceiver.PacketResponder packetresponder) :
-    call (void PipelineAck.readFields(DataInput))
+    call (void PipelineAck.readFields(InputStream))
       && this(packetresponder);
 
   after(BlockReceiver.PacketResponder packetresponder) throws IOException

+ 6 - 5
hdfs/src/test/aop/org/apache/hadoop/hdfs/server/datanode/DataTransferProtocolAspects.aj

@@ -19,15 +19,16 @@ package org.apache.hadoop.hdfs.server.datanode;
 
 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.DataTransferProtocol.Op;
-import org.apache.hadoop.hdfs.protocol.DataTransferProtocol.Receiver;
-import org.apache.hadoop.hdfs.protocol.DataTransferProtocol.Status;
+import org.apache.hadoop.hdfs.protocol.proto.DataTransferProtos.BlockOpResponseProto;
+import org.apache.hadoop.hdfs.protocol.datatransfer.Op;
+import org.apache.hadoop.hdfs.protocol.datatransfer.Receiver;
 
 /** Aspect for DataTransferProtocol */
 public aspect DataTransferProtocolAspects {
@@ -53,9 +54,9 @@ public aspect DataTransferProtocolAspects {
   }
 
   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="

+ 1 - 1
hdfs/src/test/aop/org/apache/hadoop/hdfs/server/datanode/TestFiDataTransferProtocol.java

@@ -36,7 +36,7 @@ import org.apache.hadoop.fs.Path;
 import org.apache.hadoop.hdfs.DFSConfigKeys;
 import org.apache.hadoop.hdfs.HdfsConfiguration;
 import org.apache.hadoop.hdfs.MiniDFSCluster;
-import org.apache.hadoop.hdfs.protocol.DataTransferProtocol;
+import org.apache.hadoop.hdfs.protocol.datatransfer.DataTransferProtocol;
 import org.apache.hadoop.hdfs.protocol.DatanodeID;
 import org.apache.log4j.Level;
 import org.junit.Assert;

+ 1 - 1
hdfs/src/test/aop/org/apache/hadoop/hdfs/server/datanode/TestFiDataTransferProtocol2.java

@@ -37,7 +37,7 @@ import org.apache.hadoop.fs.Path;
 import org.apache.hadoop.hdfs.DFSClient;
 import org.apache.hadoop.hdfs.DFSConfigKeys;
 import org.apache.hadoop.hdfs.MiniDFSCluster;
-import org.apache.hadoop.hdfs.protocol.DataTransferProtocol;
+import org.apache.hadoop.hdfs.protocol.datatransfer.DataTransferProtocol;
 import org.apache.log4j.Level;
 import org.junit.Assert;
 import org.junit.Test;

+ 3 - 0
hdfs/src/test/findbugsExcludeFile.xml

@@ -2,6 +2,9 @@
      <Match>
        <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>

+ 1 - 1
hdfs/src/test/hdfs/org/apache/hadoop/cli/TestHDFSCLI.java

@@ -56,7 +56,7 @@ public class TestHDFSCLI extends CLITestHelperDFS {
                                                  .racks(racks)
                                                  .hosts(hosts)
                                                  .build();
-    
+    dfsCluster.waitClusterUp();
     namenode = conf.get(DFSConfigKeys.FS_DEFAULT_NAME_KEY, "file:///");
     
     username = System.getProperty("user.name");

+ 17 - 8
hdfs/src/test/hdfs/org/apache/hadoop/hdfs/DFSTestUtil.java

@@ -18,6 +18,8 @@
 
 package org.apache.hadoop.hdfs;
 
+import static org.junit.Assert.assertEquals;
+
 import java.io.BufferedOutputStream;
 import java.io.BufferedReader;
 import java.io.ByteArrayInputStream;
@@ -52,11 +54,12 @@ import org.apache.hadoop.fs.Path;
 import org.apache.hadoop.hdfs.DFSClient.DFSDataInputStream;
 import org.apache.hadoop.hdfs.protocol.ClientDatanodeProtocol;
 import org.apache.hadoop.hdfs.protocol.DatanodeID;
-import org.apache.hadoop.hdfs.protocol.ExtendedBlock;
-import org.apache.hadoop.hdfs.protocol.DataTransferProtocol;
 import org.apache.hadoop.hdfs.protocol.DatanodeInfo;
+import org.apache.hadoop.hdfs.protocol.ExtendedBlock;
 import org.apache.hadoop.hdfs.protocol.LocatedBlock;
 import org.apache.hadoop.hdfs.protocol.LocatedBlocks;
+import org.apache.hadoop.hdfs.protocol.datatransfer.Sender;
+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,14 +69,11 @@ import org.apache.hadoop.hdfs.server.namenode.FSNamesystem;
 import org.apache.hadoop.hdfs.server.namenode.NameNode;
 import org.apache.hadoop.hdfs.server.namenode.NameNodeAdapter;
 import org.apache.hadoop.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;
 import org.apache.hadoop.security.token.Token;
 
-import static org.junit.Assert.*;
-
 /** Utilities for HDFS tests */
 public class DFSTestUtil {
   
@@ -555,6 +555,15 @@ public class DFSTestUtil {
     IOUtils.copyBytes(is, os, s.length(), true);
   }
 
+  /* Append the given string to the given file */
+  public static void appendFile(FileSystem fs, Path p, String s) 
+      throws IOException {
+    assert fs.exists(p);
+    InputStream is = new ByteArrayInputStream(s.getBytes());
+    FSDataOutputStream os = fs.append(p);
+    IOUtils.copyBytes(is, os, s.length(), true);
+  }
+  
   // Returns url content as string.
   public static String urlGet(URL url) throws IOException {
     URLConnection conn = url.openConnection();
@@ -662,7 +671,7 @@ public class DFSTestUtil {
   }
 
   /** 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],
@@ -674,10 +683,10 @@ public class DFSTestUtil {
     final DataInputStream in = new DataInputStream(NetUtils.getInputStream(s));
 
     // send the request
-    DataTransferProtocol.Sender.opTransferBlock(out, b, dfsClient.clientName,
+    Sender.opTransferBlock(out, b, dfsClient.clientName,
         new DatanodeInfo[]{datanodes[1]}, new Token<BlockTokenIdentifier>());
     out.flush();
 
-    return DataTransferProtocol.Status.read(in);
+    return BlockOpResponseProto.parseDelimitedFrom(in);
   }
 }

+ 1 - 1
hdfs/src/test/hdfs/org/apache/hadoop/hdfs/TestClientBlockVerification.java

@@ -20,8 +20,8 @@ package org.apache.hadoop.hdfs;
 
 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;

+ 51 - 49
hdfs/src/test/hdfs/org/apache/hadoop/hdfs/TestDataTransferProtocol.java

@@ -17,13 +17,6 @@
  */
 package org.apache.hadoop.hdfs;
 
-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;
 import java.io.DataInputStream;
@@ -45,20 +38,26 @@ import org.apache.hadoop.fs.FSDataInputStream;
 import org.apache.hadoop.fs.FSDataOutputStream;
 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.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.datatransfer.BlockConstructionStage;
+import org.apache.hadoop.hdfs.protocol.datatransfer.DataTransferProtocol;
+import org.apache.hadoop.hdfs.protocol.datatransfer.Op;
+import org.apache.hadoop.hdfs.protocol.datatransfer.PacketHeader;
+import org.apache.hadoop.hdfs.protocol.datatransfer.PipelineAck;
+import org.apache.hadoop.hdfs.protocol.datatransfer.Sender;
+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;
-import org.apache.hadoop.hdfs.HdfsConfiguration;
 import org.apache.hadoop.io.IOUtils;
-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 +92,9 @@ public class TestDataTransferProtocol extends TestCase {
       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 +115,11 @@ public class TestDataTransferProtocol extends TestCase {
         }
         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,26 +165,35 @@ public class TestDataTransferProtocol extends TestCase {
     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();
     recvBuf.reset();
-    DataTransferProtocol.Sender.opWriteBlock(sendOut, block, 0,
+    Sender.opWriteBlock(sendOut, block, 0,
         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);
@@ -355,12 +367,12 @@ public class TestDataTransferProtocol extends TestCase {
     // bad ops
     sendBuf.reset();
     sendOut.writeShort((short)DataTransferProtocol.DATA_TRANSFER_VERSION);
-    sendOut.writeByte(WRITE_BLOCK.code - 1);
+    sendOut.writeByte(Op.WRITE_BLOCK.code - 1);
     sendRecvData("Wrong Op Code", true);
     
     /* Test OP_WRITE_BLOCK */
     sendBuf.reset();
-    DataTransferProtocol.Sender.opWriteBlock(sendOut, 
+    Sender.opWriteBlock(sendOut, 
         new ExtendedBlock(poolId, newBlockId), 0,
         BlockConstructionStage.PIPELINE_SETUP_CREATE, 0L, 0L, 0L, "cl", null,
         new DatanodeInfo[1], BlockTokenSecretManager.DUMMY_TOKEN);
@@ -369,12 +381,12 @@ public class TestDataTransferProtocol extends TestCase {
     // 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();
     recvBuf.reset();
-    DataTransferProtocol.Sender.opWriteBlock(sendOut,
+    Sender.opWriteBlock(sendOut,
         new ExtendedBlock(poolId, ++newBlockId), 0,
         BlockConstructionStage.PIPELINE_SETUP_CREATE, 0L, 0L, 0L, "cl", null,
         new DatanodeInfo[1], BlockTokenSecretManager.DUMMY_TOKEN);
@@ -389,16 +401,15 @@ public class TestDataTransferProtocol extends TestCase {
       -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);
 
     // test for writing a valid zero size block
     sendBuf.reset();
     recvBuf.reset();
-    DataTransferProtocol.Sender.opWriteBlock(sendOut, 
+    Sender.opWriteBlock(sendOut, 
         new ExtendedBlock(poolId, ++newBlockId), 0,
         BlockConstructionStage.PIPELINE_SETUP_CREATE, 0L, 0L, 0L, "cl", null,
         new DatanodeInfo[1], BlockTokenSecretManager.DUMMY_TOKEN);
@@ -415,9 +426,8 @@ public class TestDataTransferProtocol extends TestCase {
     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 */
@@ -429,46 +439,46 @@ public class TestDataTransferProtocol extends TestCase {
     sendBuf.reset();
     recvBuf.reset();
     blk.setBlockId(blkid-1);
-    DataTransferProtocol.Sender.opReadBlock(sendOut, blk, 0L, fileLen, "cl",
+    Sender.opReadBlock(sendOut, blk, 0L, fileLen, "cl",
           BlockTokenSecretManager.DUMMY_TOKEN);
     sendRecvData("Wrong block ID " + newBlockId + " for read", false); 
 
     // negative block start offset -1L
     sendBuf.reset();
     blk.setBlockId(blkid);
-    DataTransferProtocol.Sender.opReadBlock(sendOut, blk, -1L, fileLen, "cl",
+    Sender.opReadBlock(sendOut, blk, -1L, fileLen, "cl",
           BlockTokenSecretManager.DUMMY_TOKEN);
     sendRecvData("Negative start-offset for read for block " + 
                  firstBlock.getBlockId(), false);
 
     // bad block start offset
     sendBuf.reset();
-    DataTransferProtocol.Sender.opReadBlock(sendOut, blk, fileLen, fileLen, "cl",
+    Sender.opReadBlock(sendOut, blk, fileLen, fileLen, "cl",
           BlockTokenSecretManager.DUMMY_TOKEN);
     sendRecvData("Wrong start-offset for reading block " +
                  firstBlock.getBlockId(), false);
     
     // 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, 
+    Sender.opReadBlock(sendOut, blk, 0L, 
         -1 - random.nextInt(oneMil), "cl", BlockTokenSecretManager.DUMMY_TOKEN);
     sendRecvData("Negative length for reading block " +
                  firstBlock.getBlockId(), false);
     
     // 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, 
+    Sender.opReadBlock(sendOut, blk, 0L, 
         fileLen + 1, "cl", BlockTokenSecretManager.DUMMY_TOKEN);
     sendRecvData("Wrong length for reading block " +
                  firstBlock.getBlockId(), false);
     
     //At the end of all this, read the file to make sure that succeeds finally.
     sendBuf.reset();
-    DataTransferProtocol.Sender.opReadBlock(sendOut, blk, 0L, 
+    Sender.opReadBlock(sendOut, blk, 0L, 
         fileLen, "cl", BlockTokenSecretManager.DUMMY_TOKEN);
     readFile(fileSys, file, fileLen);
     } finally {
@@ -499,14 +509,6 @@ public class TestDataTransferProtocol extends TestCase {
     readBack.readFields(ByteBuffer.wrap(baos.toByteArray()));
     assertEquals(hdr, readBack);
 
-    // Test sanity check for good header
-    PacketHeader goodHeader = new PacketHeader(
-      4,                   // size of packet
-      0,                   // OffsetInBlock
-      100,                 // sequencenumber
-      true,                // lastPacketInBlock
-      0);                  // chunk length
-
     assertTrue(hdr.sanityCheck(99));
     assertFalse(hdr.sanityCheck(100));
   }

+ 4 - 2
hdfs/src/test/hdfs/org/apache/hadoop/hdfs/TestDecommission.java

@@ -35,6 +35,7 @@ import org.apache.hadoop.hdfs.protocol.DatanodeInfo.AdminStates;
 import org.apache.hadoop.hdfs.protocol.FSConstants.DatanodeReportType;
 import org.apache.hadoop.hdfs.server.namenode.FSNamesystem;
 import org.apache.hadoop.hdfs.server.namenode.NameNode;
+import org.apache.hadoop.hdfs.server.namenode.NameNodeAdapter;
 import static org.junit.Assert.*;
 
 import org.junit.After;
@@ -201,7 +202,8 @@ public class TestDecommission {
     nodes.add(nodename);
     writeConfigFile(excludeFile, nodes);
     cluster.getNamesystem(nnIndex).refreshNodes(conf);
-    DatanodeInfo ret = cluster.getNamesystem(nnIndex).getDatanode(info[index]);
+    DatanodeInfo ret = NameNodeAdapter.getDatanode(
+        cluster.getNameNode(nnIndex), info[index]);
     waitNodeState(ret, waitForState);
     return ret;
   }
@@ -371,7 +373,7 @@ public class TestDecommission {
       // Stop decommissioning and verify stats
       writeConfigFile(excludeFile, null);
       fsn.refreshNodes(conf);
-      DatanodeInfo ret = fsn.getDatanode(downnode);
+      DatanodeInfo ret = NameNodeAdapter.getDatanode(namenode, downnode);
       waitNodeState(ret, AdminStates.NORMAL);
       verifyStats(namenode, fsn, ret, false);
     }

+ 4 - 4
hdfs/src/test/hdfs/org/apache/hadoop/hdfs/TestReplaceDatanodeOnFailure.java

@@ -26,9 +26,10 @@ import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.fs.FSDataInputStream;
 import org.apache.hadoop.fs.FSDataOutputStream;
 import org.apache.hadoop.fs.Path;
-import org.apache.hadoop.hdfs.protocol.DataTransferProtocol;
 import org.apache.hadoop.hdfs.protocol.DatanodeID;
 import org.apache.hadoop.hdfs.protocol.DatanodeInfo;
+import org.apache.hadoop.hdfs.protocol.datatransfer.DataTransferProtocol;
+import org.apache.hadoop.hdfs.protocol.datatransfer.ReplaceDatanodeOnFailure;
 import org.apache.hadoop.io.IOUtils;
 import org.apache.log4j.Level;
 import org.junit.Assert;
@@ -53,8 +54,7 @@ public class TestReplaceDatanodeOnFailure {
   /** Test DEFAULT ReplaceDatanodeOnFailure policy. */
   @Test
   public void testDefaultPolicy() throws Exception {
-    final DataTransferProtocol.ReplaceDatanodeOnFailure p
-        = DataTransferProtocol.ReplaceDatanodeOnFailure.DEFAULT;
+    final ReplaceDatanodeOnFailure p = ReplaceDatanodeOnFailure.DEFAULT;
 
     final DatanodeInfo[] infos = new DatanodeInfo[5];
     final DatanodeInfo[][] datanodes = new DatanodeInfo[infos.length + 1][];
@@ -113,7 +113,7 @@ public class TestReplaceDatanodeOnFailure {
     final Configuration conf = new HdfsConfiguration();
     
     //always replace a datanode
-    DataTransferProtocol.ReplaceDatanodeOnFailure.ALWAYS.write(conf);
+    ReplaceDatanodeOnFailure.ALWAYS.write(conf);
 
     final String[] racks = new String[REPLICATION];
     Arrays.fill(racks, RACK0);

+ 147 - 61
hdfs/src/test/hdfs/org/apache/hadoop/hdfs/TestSafeMode.java

@@ -20,20 +20,44 @@ package org.apache.hadoop.hdfs;
 
 import java.io.IOException;
 
-import org.apache.commons.logging.Log;
-import org.apache.commons.logging.LogFactory;
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.fs.Path;
+import org.apache.hadoop.fs.FileSystem;
+import org.apache.hadoop.fs.permission.FsPermission;
 import org.apache.hadoop.hdfs.protocol.FSConstants.SafeModeAction;
 
-import junit.framework.TestCase;
+import static org.junit.Assert.*;
+import org.junit.Before;
+import org.junit.After;
+import org.junit.Test;
 
 /**
  * Tests to verify safe mode correctness.
  */
-public class TestSafeMode extends TestCase {
-  
-  static Log LOG = LogFactory.getLog(TestSafeMode.class);
+public class TestSafeMode {
+  Configuration conf; 
+  MiniDFSCluster cluster;
+  FileSystem fs;
+  DistributedFileSystem dfs;
+
+  @Before
+  public void startUp() throws IOException {
+    conf = new HdfsConfiguration();
+    cluster = new MiniDFSCluster.Builder(conf).numDataNodes(1).build();
+    cluster.waitActive();      
+    fs = cluster.getFileSystem();
+    dfs = (DistributedFileSystem)fs;
+  }
+
+  @After
+  public void tearDown() throws IOException {
+    if (fs != null) {
+      fs.close();
+    }
+    if (cluster != null) {
+      cluster.shutdown();
+    }
+  }
 
   /**
    * This test verifies that if SafeMode is manually entered, name-node does not
@@ -51,61 +75,123 @@ public class TestSafeMode extends TestCase {
    *  
    * @throws IOException
    */
-  public void testManualSafeMode() throws IOException {
-    MiniDFSCluster cluster = null;
-    DistributedFileSystem fs = null;
+  @Test
+  public void testManualSafeMode() throws IOException {      
+    fs = (DistributedFileSystem)cluster.getFileSystem();
+    Path file1 = new Path("/tmp/testManualSafeMode/file1");
+    Path file2 = new Path("/tmp/testManualSafeMode/file2");
+    
+    // create two files with one block each.
+    DFSTestUtil.createFile(fs, file1, 1000, (short)1, 0);
+    DFSTestUtil.createFile(fs, file2, 2000, (short)1, 0);
+    fs.close();
+    cluster.shutdown();
+    
+    // now bring up just the NameNode.
+    cluster = new MiniDFSCluster.Builder(conf).numDataNodes(0).format(false).build();
+    cluster.waitActive();
+    dfs = (DistributedFileSystem)cluster.getFileSystem();
+    
+    assertTrue("No datanode is started. Should be in SafeMode", 
+               dfs.setSafeMode(SafeModeAction.SAFEMODE_GET));
+    
+    // manually set safemode.
+    dfs.setSafeMode(SafeModeAction.SAFEMODE_ENTER);
+    
+    // now bring up the datanode and wait for it to be active.
+    cluster.startDataNodes(conf, 1, true, null, null);
+    cluster.waitActive();
+    
+    // wait longer than dfs.namenode.safemode.extension
     try {
-      Configuration conf = new HdfsConfiguration();
-      // disable safemode extension to make the test run faster.
-      conf.set(DFSConfigKeys.DFS_NAMENODE_SAFEMODE_EXTENSION_KEY, "1");
-      cluster = new MiniDFSCluster.Builder(conf).build();
-      cluster.waitActive();
-      
-      fs = (DistributedFileSystem)cluster.getFileSystem();
-      Path file1 = new Path("/tmp/testManualSafeMode/file1");
-      Path file2 = new Path("/tmp/testManualSafeMode/file2");
-      
-      LOG.info("Created file1 and file2.");
-      
-      // create two files with one block each.
-      DFSTestUtil.createFile(fs, file1, 1000, (short)1, 0);
-      DFSTestUtil.createFile(fs, file2, 2000, (short)1, 0);
-      fs.close();
-      cluster.shutdown();
-      
-      // now bring up just the NameNode.
-      cluster = new MiniDFSCluster.Builder(conf).numDataNodes(0).format(false).build();
-      cluster.waitActive();
-      fs = (DistributedFileSystem)cluster.getFileSystem();
-      
-      LOG.info("Restarted cluster with just the NameNode");
-      
-      assertTrue("No datanode is started. Should be in SafeMode", 
-                 fs.setSafeMode(SafeModeAction.SAFEMODE_GET));
-      
-      // manually set safemode.
-      fs.setSafeMode(SafeModeAction.SAFEMODE_ENTER);
-      
-      // now bring up the datanode and wait for it to be active.
-      cluster.startDataNodes(conf, 1, true, null, null);
-      cluster.waitActive();
-      
-      LOG.info("Datanode is started.");
-
-      // wait longer than dfs.namenode.safemode.extension
-      try {
-        Thread.sleep(2000);
-      } catch (InterruptedException ignored) {}
-      
-      assertTrue("should still be in SafeMode",
-          fs.setSafeMode(SafeModeAction.SAFEMODE_GET));
-      
-      fs.setSafeMode(SafeModeAction.SAFEMODE_LEAVE);
-      assertFalse("should not be in SafeMode",
-          fs.setSafeMode(SafeModeAction.SAFEMODE_GET));
-    } finally {
-      if(fs != null) fs.close();
-      if(cluster!= null) cluster.shutdown();
+      Thread.sleep(2000);
+    } catch (InterruptedException ignored) {}
+
+    assertTrue("should still be in SafeMode",
+        dfs.setSafeMode(SafeModeAction.SAFEMODE_GET));
+    assertFalse("should not be in SafeMode", 
+        dfs.setSafeMode(SafeModeAction.SAFEMODE_LEAVE));
+  }
+
+  public interface FSRun {
+    public abstract void run(FileSystem fs) throws IOException;
+  }
+
+  /**
+   * Assert that the given function fails to run due to a safe 
+   * mode exception.
+   */
+  public void runFsFun(String msg, FSRun f) {
+    try {
+      f.run(fs);
+      fail(msg);
+     } catch (IOException ioe) {
+       assertTrue(ioe.getMessage().contains("safe mode"));
+     }
+  }
+
+  /**
+   * Run various fs operations while the NN is in safe mode,
+   * assert that they are either allowed or fail as expected.
+   */
+  @Test
+  public void testOperationsWhileInSafeMode() throws IOException {
+    final Path file1 = new Path("/file1");
+
+    assertFalse(dfs.setSafeMode(SafeModeAction.SAFEMODE_GET));
+    DFSTestUtil.createFile(fs, file1, 1024, (short)1, 0);
+    assertTrue("Could not enter SM", 
+        dfs.setSafeMode(SafeModeAction.SAFEMODE_ENTER));
+
+    runFsFun("Set quota while in SM", new FSRun() { 
+      public void run(FileSystem fs) throws IOException {
+        ((DistributedFileSystem)fs).setQuota(file1, 1, 1); 
+      }});
+
+    runFsFun("Set perm while in SM", new FSRun() {
+      public void run(FileSystem fs) throws IOException {
+        fs.setPermission(file1, FsPermission.getDefault());
+      }});
+
+    runFsFun("Set owner while in SM", new FSRun() {
+      public void run(FileSystem fs) throws IOException {
+        fs.setOwner(file1, "user", "group");
+      }});
+
+    runFsFun("Set repl while in SM", new FSRun() {
+      public void run(FileSystem fs) throws IOException {
+        fs.setReplication(file1, (short)1);
+      }});
+
+    runFsFun("Append file while in SM", new FSRun() {
+      public void run(FileSystem fs) throws IOException {
+        DFSTestUtil.appendFile(fs, file1, "new bytes");
+      }});
+
+    runFsFun("Delete file while in SM", new FSRun() {
+      public void run(FileSystem fs) throws IOException {
+        fs.delete(file1, false);
+      }});
+
+    runFsFun("Rename file while in SM", new FSRun() {
+      public void run(FileSystem fs) throws IOException {
+        fs.rename(file1, new Path("file2"));
+      }});
+
+    try {
+      fs.setTimes(file1, 0, 0);
+    } catch (IOException ioe) {
+      fail("Set times failed while in SM");
     }
+
+    try {
+      DFSTestUtil.readFile(fs, file1);
+    } catch (IOException ioe) {
+      fail("Set times failed while in SM");
+    }
+
+    assertFalse("Could not leave SM",
+        dfs.setSafeMode(SafeModeAction.SAFEMODE_LEAVE));
   }
-}
+  
+}

+ 9 - 7
hdfs/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;
@@ -38,19 +36,21 @@ import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.fs.FileSystem;
 import org.apache.hadoop.fs.Path;
 import org.apache.hadoop.hdfs.DFSClient;
+import org.apache.hadoop.hdfs.DFSConfigKeys;
 import org.apache.hadoop.hdfs.DFSTestUtil;
 import org.apache.hadoop.hdfs.HdfsConfiguration;
 import org.apache.hadoop.hdfs.MiniDFSCluster;
-import org.apache.hadoop.hdfs.protocol.DataTransferProtocol;
 import org.apache.hadoop.hdfs.protocol.DatanodeInfo;
 import org.apache.hadoop.hdfs.protocol.ExtendedBlock;
-import org.apache.hadoop.hdfs.protocol.LocatedBlock;
 import org.apache.hadoop.hdfs.protocol.FSConstants.DatanodeReportType;
+import org.apache.hadoop.hdfs.protocol.LocatedBlock;
+import org.apache.hadoop.hdfs.protocol.datatransfer.Sender;
+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;
 import org.apache.hadoop.hdfs.util.DataTransferThrottler;
-import org.apache.hadoop.hdfs.DFSConfigKeys;
 import org.apache.hadoop.net.NetUtils;
 /**
  * This class tests if block replacement request to data nodes work correctly.
@@ -258,13 +258,15 @@ public class TestBlockReplacement extends TestCase {
     sock.setKeepAlive(true);
     // sendRequest
     DataOutputStream out = new DataOutputStream(sock.getOutputStream());
-    DataTransferProtocol.Sender.opReplaceBlock(out, block, source
+    Sender.opReplaceBlock(out, block, source
         .getStorageID(), sourceProxy, BlockTokenSecretManager.DUMMY_TOKEN);
     out.flush();
     // receiveResponse
     DataInputStream reply = new DataInputStream(sock.getInputStream());
 
-    return DataTransferProtocol.Status.read(reply) == SUCCESS;
+    BlockOpResponseProto proto =
+      BlockOpResponseProto.parseDelimitedFrom(reply);
+    return proto.getStatus() == Status.SUCCESS;
   }
 
   /**

+ 91 - 0
hdfs/src/test/hdfs/org/apache/hadoop/hdfs/server/datanode/TestDataNodeExit.java

@@ -0,0 +1,91 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.hadoop.hdfs.server.datanode;
+
+import static org.junit.Assert.assertEquals;
+import static org.junit.Assert.assertFalse;
+import static org.junit.Assert.assertTrue;
+
+import java.io.IOException;
+
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.hdfs.DFSConfigKeys;
+import org.apache.hadoop.hdfs.HdfsConfiguration;
+import org.apache.hadoop.hdfs.MiniDFSCluster;
+import org.apache.hadoop.hdfs.server.datanode.DataNode.BPOfferService;
+import org.junit.After;
+import org.junit.Before;
+import org.junit.Test;
+
+/** 
+ * Tests if DataNode process exits if all Block Pool services exit. 
+ */
+public class TestDataNodeExit {
+  private static int BASEPORT = 9923;
+  private static long WAIT_TIME_IN_MILLIS = 10;
+  Configuration conf;
+  MiniDFSCluster cluster = null;
+  
+  @Before
+  public void setUp() throws IOException {
+    conf = new HdfsConfiguration();
+    conf.setInt(DFSConfigKeys.DFS_BLOCK_SIZE_KEY, 100);
+    conf.setInt(DFSConfigKeys.DFS_BYTES_PER_CHECKSUM_KEY, 100);
+    cluster = new MiniDFSCluster.Builder(conf).numNameNodes(3)
+        .nameNodePort(BASEPORT).build();
+    for (int i = 0; i < 3; i++) {
+      cluster.waitActive(i);
+    }
+  }
+
+  @After
+  public void tearDown() throws Exception {
+    if (cluster != null)
+      cluster.shutdown();
+  }
+  
+  private void stopBPServiceThreads(int numStopThreads, DataNode dn)
+      throws Exception {
+    BPOfferService[] bpoList = dn.getAllBpOs();
+    int expected = dn.getBpOsCount() - numStopThreads;
+    int index = numStopThreads - 1;
+    while (index >= 0) {
+      bpoList[index--].stop();
+    }
+    int iterations = 3000; // Total 30 seconds MAX wait time
+    while(dn.getBpOsCount() != expected && iterations > 0) {
+      Thread.sleep(WAIT_TIME_IN_MILLIS);
+      iterations--;
+    }
+    assertEquals("Mismatch in number of BPServices running", expected,
+        dn.getBpOsCount());
+  }
+
+  /**
+   * Test BPService Thread Exit
+   */
+  @Test
+  public void testBPServiceExit() throws Exception {
+    DataNode dn = cluster.getDataNodes().get(0);
+    stopBPServiceThreads(1, dn);
+    assertTrue("DataNode should not exit", dn.isDatanodeUp());
+    stopBPServiceThreads(2, dn);
+    assertFalse("DataNode should exit", dn.isDatanodeUp());
+  }
+}

+ 52 - 3
hdfs/src/test/hdfs/org/apache/hadoop/hdfs/server/datanode/TestDatanodeJsp.java

@@ -19,24 +19,30 @@ package org.apache.hadoop.hdfs.server.datanode;
 
 import static org.junit.Assert.assertTrue;
 
+import java.io.DataOutputStream;
 import java.io.IOException;
 import java.net.InetSocketAddress;
 import java.net.URL;
 import java.net.URLEncoder;
 
+import javax.servlet.http.HttpServletRequest;
+import javax.servlet.jsp.JspWriter;
+
 import org.apache.commons.httpclient.util.URIUtil;
-import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.fs.FileSystem;
 import org.apache.hadoop.fs.Path;
 import org.apache.hadoop.hdfs.DFSTestUtil;
 import org.apache.hadoop.hdfs.HdfsConfiguration;
 import org.apache.hadoop.hdfs.MiniDFSCluster;
 import org.apache.hadoop.hdfs.server.common.JspHelper;
+import org.apache.hadoop.hdfs.server.namenode.NameNode;
 import org.junit.Test;
+import org.mockito.Mockito;
 
 public class TestDatanodeJsp {
   
   private static final String FILE_DATA = "foo bar baz biz buz";
+  private static final HdfsConfiguration CONF = new HdfsConfiguration();
   
   private static void testViewingFile(MiniDFSCluster cluster, String filePath,
       boolean doTail) throws IOException {
@@ -74,8 +80,7 @@ public class TestDatanodeJsp {
   public void testViewFileJsp() throws IOException {
     MiniDFSCluster cluster = null;
     try {
-      Configuration conf = new HdfsConfiguration();
-      cluster = new MiniDFSCluster.Builder(conf).build();
+      cluster = new MiniDFSCluster.Builder(CONF).build();
       cluster.waitActive();
       
       testViewingFile(cluster, "/test-file", false);
@@ -92,5 +97,49 @@ public class TestDatanodeJsp {
       }
     }
   }
+  
+  @Test
+  public void testGenStamp() throws Exception {
+    MiniDFSCluster cluster = new MiniDFSCluster.Builder(CONF).numDataNodes(1)
+        .build();
+    try {
+      FileSystem fs = cluster.getFileSystem();
+      Path testFile = new Path("/test/mkdirs/TestchunkSizeToView");
+      writeFile(fs, testFile);
+      JspWriter writerMock = Mockito.mock(JspWriter.class);
+      HttpServletRequest reqMock = Mockito.mock(HttpServletRequest.class);
+      setTheMockExpectationsFromReq(testFile, reqMock);
+      DatanodeJspHelper.generateFileDetails(writerMock, reqMock, CONF);
+      Mockito.verify(writerMock, Mockito.atLeastOnce()).print(
+          "<input type=\"hidden\" name=\"genstamp\" value=\"987654321\">");
+    } finally {
+      cluster.shutdown();
+    }
+  }
+
+  private void setTheMockExpectationsFromReq(Path testFile,
+      HttpServletRequest reqMock) {
+    Mockito.doReturn("987654321").when(reqMock).getParameter("genstamp");
+    Mockito.doReturn("1234").when(reqMock).getParameter("blockId");
+    Mockito.doReturn("8081").when(reqMock).getParameter("datanodePort");
+    Mockito.doReturn("8080").when(reqMock).getParameter("namenodeInfoPort");
+    Mockito.doReturn("100").when(reqMock).getParameter("chunkSizeToView");
+    Mockito.doReturn("1").when(reqMock).getParameter("startOffset");
+    Mockito.doReturn("1024").when(reqMock).getParameter("blockSize");
+    Mockito.doReturn(NameNode.getHostPortString(NameNode.getAddress(CONF)))
+        .when(reqMock).getParameter("nnaddr");
+    Mockito.doReturn(testFile.toString()).when(reqMock).getPathInfo();
+  }
+
+  static Path writeFile(FileSystem fs, Path f) throws IOException {
+    DataOutputStream out = fs.create(f);
+    try {
+      out.writeBytes("umamahesh: " + f);
+    } finally {
+      out.close();
+    }
+    assertTrue(fs.exists(f));
+    return f;
+  }
 
 }

+ 9 - 8
hdfs/src/test/hdfs/org/apache/hadoop/hdfs/server/datanode/TestDiskError.java

@@ -17,6 +17,9 @@
  */
 package org.apache.hadoop.hdfs.server.datanode;
 
+import static org.junit.Assert.assertEquals;
+import static org.junit.Assert.assertTrue;
+
 import java.io.DataOutputStream;
 import java.io.File;
 import java.net.InetSocketAddress;
@@ -26,22 +29,20 @@ import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.fs.FileSystem;
 import org.apache.hadoop.fs.Path;
 import org.apache.hadoop.fs.permission.FsPermission;
+import org.apache.hadoop.hdfs.DFSConfigKeys;
 import org.apache.hadoop.hdfs.DFSTestUtil;
+import org.apache.hadoop.hdfs.HdfsConfiguration;
 import org.apache.hadoop.hdfs.MiniDFSCluster;
 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.DataTransferProtocol.BlockConstructionStage;
-import org.apache.hadoop.hdfs.protocol.DataTransferProtocol.Sender;
+import org.apache.hadoop.hdfs.protocol.datatransfer.BlockConstructionStage;
+import org.apache.hadoop.hdfs.protocol.datatransfer.Sender;
 import org.apache.hadoop.hdfs.security.token.block.BlockTokenSecretManager;
 import org.apache.hadoop.hdfs.server.namenode.NameNodeAdapter;
-import org.apache.hadoop.hdfs.HdfsConfiguration;
-import org.apache.hadoop.hdfs.DFSConfigKeys;
-
-import org.junit.Test;
-import org.junit.Before;
 import org.junit.After;
-import static org.junit.Assert.*;
+import org.junit.Before;
+import org.junit.Test;
 
 /**
  * Test that datanodes can correctly handle errors during block read/write.

+ 5 - 3
hdfs/src/test/hdfs/org/apache/hadoop/hdfs/server/datanode/TestTransferRbw.java

@@ -29,10 +29,12 @@ import org.apache.hadoop.hdfs.DFSTestUtil;
 import org.apache.hadoop.hdfs.DistributedFileSystem;
 import org.apache.hadoop.hdfs.HdfsConfiguration;
 import org.apache.hadoop.hdfs.MiniDFSCluster;
-import org.apache.hadoop.hdfs.protocol.DataTransferProtocol;
 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.datatransfer.DataTransferProtocol;
+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 @@ public class TestTransferRbw {
         //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

+ 15 - 0
hdfs/src/test/hdfs/org/apache/hadoop/hdfs/server/namenode/NameNodeAdapter.java

@@ -21,6 +21,7 @@ import java.io.IOException;
 
 import org.apache.hadoop.ipc.Server;
 import org.apache.hadoop.hdfs.protocol.Block;
+import org.apache.hadoop.hdfs.protocol.DatanodeID;
 import org.apache.hadoop.hdfs.protocol.LocatedBlocks;
 
 /**
@@ -77,4 +78,18 @@ public class NameNodeAdapter {
   public static String getLeaseHolderForPath(NameNode namenode, String path) {
     return namenode.getNamesystem().leaseManager.getLeaseByPath(path).getHolder();
   }
+
+  /**
+   * Return the datanode descriptor for the given datanode.
+   */
+  public static DatanodeDescriptor getDatanode(NameNode namenode,
+      DatanodeID id) throws IOException {
+    FSNamesystem ns = namenode.getNamesystem();
+    ns.readLock();
+    try {
+      return ns.getDatanode(id);
+    } finally {
+      ns.readUnlock();
+    }
+  }
 }

+ 3 - 1
hdfs/src/test/hdfs/org/apache/hadoop/hdfs/server/namenode/TestBlockTokenWithDFS.java

@@ -43,6 +43,7 @@ import org.apache.hadoop.fs.FSDataOutputStream;
 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 @@ public class TestBlockTokenWithDFS extends TestCase {
             + "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 {

+ 8 - 1
hdfs/src/test/hdfs/org/apache/hadoop/hdfs/server/namenode/TestDeadDatanode.java

@@ -61,7 +61,14 @@ public class TestDeadDatanode {
     FSNamesystem namesystem = cluster.getNamesystem();
     String state = alive ? "alive" : "dead";
     while (System.currentTimeMillis() < stopTime) {
-      if (namesystem.getDatanode(nodeID).isAlive == alive) {
+      namesystem.readLock();
+      DatanodeDescriptor dd;
+      try {
+        dd = namesystem.getDatanode(nodeID);
+      } finally {
+        namesystem.readUnlock();
+      }
+      if (dd.isAlive == alive) {
         LOG.info("datanode " + nodeID + " is " + state);
         return;
       }

この差分においてかなりの量のファイルが変更されているため、一部のファイルを表示していません